From 831bef1d13bbb6675aa6ef19f35e8c0633231ce7 Mon Sep 17 00:00:00 2001 From: Ross Lawley Date: Mon, 18 Sep 2023 09:28:11 +0100 Subject: [PATCH 1/9] Refactored Command Batch Cursors. Previously had a QueryResult / QueryBatchCursor abstraction. This abstraction is no longer required as only commands are used to create cursors. Two new classes have been added: 1. SingleBatchCursor Used when commands return a single list of results but not an actual cursor. 2. CommandBatchCursor Used for commands that return a cursor and contain all the logic to manage resources and issue get more calls. The construction and resource management has been simplified by reducing the number of constructors used when creating the cursor. This will simplify future refactorings. The asynchronous cursor abstractions have been refactored to more closely follow their synchronous counterparts. Reducing the cognative costs when working on both cursor types. JAVA-5159 --- .evergreen/run-load-balancer-tests.sh | 2 +- config/codenarc/codenarc.xml | 6 - .../AsyncAggregateResponseBatchCursor.java | 3 + .../internal/async/AsyncBatchCursor.java | 8 +- .../operation/AggregateOperationImpl.java | 27 +- .../AggregateResponseBatchCursor.java | 3 + .../AsyncChangeStreamBatchCursor.java | 7 +- .../operation/AsyncCommandBatchCursor.java | 524 ++++++++++++++ .../operation/AsyncOperationHelper.java | 14 +- .../operation/AsyncQueryBatchCursor.java | 428 ------------ ...ursor.java => AsyncSingleBatchCursor.java} | 34 +- .../operation/ChangeStreamBatchCursor.java | 26 +- ...tchCursor.java => CommandBatchCursor.java} | 220 +++--- .../operation/CommandBatchCursorHelper.java | 78 +++ .../CommandCursorResult.java} | 63 +- .../internal/operation/DistinctOperation.java | 32 +- .../internal/operation/FindOperation.java | 23 +- .../operation/ListCollectionsOperation.java | 26 +- .../operation/ListDatabasesOperation.java | 26 +- .../operation/ListIndexesOperation.java | 20 +- .../operation/ListSearchIndexesOperation.java | 10 +- .../MapReduceInlineResultsAsyncCursor.java | 36 +- .../MapReduceInlineResultsCursor.java | 60 +- .../MapReduceWithInlineResultsOperation.java | 18 +- .../internal/operation/OperationHelper.java | 26 - .../internal/operation/QueryHelper.java | 35 - .../internal/operation/SingleBatchCursor.java | 95 +++ .../operation/SyncOperationHelper.java | 17 +- .../OperationFunctionalSpecification.groovy | 7 - ...chCursorAltFunctionalSpecification.groovy} | 132 ++-- ...dBatchCursorFunctionalSpecification.groovy | 496 ++++++++++++++ ...BatchCursorFunctionalSpecification.groovy} | 243 ++++--- ...stCollectionsOperationSpecification.groovy | 13 +- .../ListIndexesOperationSpecification.groovy | 13 +- .../operation/QueryOperationHelper.groovy | 35 +- ...hangeStreamBatchCursorSpecification.groovy | 11 +- ...yncCommandBatchCursorSpecification.groovy} | 226 +++--- ...syncSingleBatchCursorSpecification.groovy} | 17 +- ...hangeStreamBatchCursorSpecification.groovy | 4 +- .../CommandBatchCursorSpecification.groovy | 645 ++++++++++++++++++ .../QueryBatchCursorSpecification.groovy | 357 ---------- 41 files changed, 2538 insertions(+), 1528 deletions(-) create mode 100644 driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java delete mode 100644 driver-core/src/main/com/mongodb/internal/operation/AsyncQueryBatchCursor.java rename driver-core/src/main/com/mongodb/internal/operation/{AsyncSingleBatchQueryCursor.java => AsyncSingleBatchCursor.java} (64%) rename driver-core/src/main/com/mongodb/internal/operation/{QueryBatchCursor.java => CommandBatchCursor.java} (68%) create mode 100644 driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursorHelper.java rename driver-core/src/main/com/mongodb/internal/{connection/QueryResult.java => operation/CommandCursorResult.java} (52%) delete mode 100644 driver-core/src/main/com/mongodb/internal/operation/QueryHelper.java create mode 100644 driver-core/src/main/com/mongodb/internal/operation/SingleBatchCursor.java rename driver-core/src/test/functional/com/mongodb/internal/operation/{AsyncQueryBatchCursorFunctionalSpecification.groovy => AsyncCommandBatchCursorAltFunctionalSpecification.groovy} (68%) create mode 100644 driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalSpecification.groovy rename driver-core/src/test/functional/com/mongodb/internal/operation/{QueryBatchCursorFunctionalSpecification.groovy => CommandBatchCursorFunctionalSpecification.groovy} (60%) rename driver-core/src/test/unit/com/mongodb/internal/operation/{AsyncQueryBatchCursorSpecification.groovy => AsyncCommandBatchCursorSpecification.groovy} (68%) rename driver-core/src/test/unit/com/mongodb/internal/operation/{AsyncSingleBatchQueryCursorSpecification.groovy => AsyncSingleBatchCursorSpecification.groovy} (68%) create mode 100644 driver-core/src/test/unit/com/mongodb/internal/operation/CommandBatchCursorSpecification.groovy delete mode 100644 driver-core/src/test/unit/com/mongodb/internal/operation/QueryBatchCursorSpecification.groovy diff --git a/.evergreen/run-load-balancer-tests.sh b/.evergreen/run-load-balancer-tests.sh index 871e9c2f003..5297097971c 100755 --- a/.evergreen/run-load-balancer-tests.sh +++ b/.evergreen/run-load-balancer-tests.sh @@ -79,7 +79,7 @@ echo $second -Dorg.mongodb.test.uri=${SINGLE_MONGOS_LB_URI} \ -Dorg.mongodb.test.multi.mongos.uri=${MULTI_MONGOS_LB_URI} \ ${GRADLE_EXTRA_VARS} --stacktrace --info --continue driver-core:test \ - --tests QueryBatchCursorFunctionalSpecification + --tests CommandBatchCursorSpecification third=$? echo $third diff --git a/config/codenarc/codenarc.xml b/config/codenarc/codenarc.xml index 2d11b03296a..2bab2315e97 100644 --- a/config/codenarc/codenarc.xml +++ b/config/codenarc/codenarc.xml @@ -34,12 +34,6 @@ - - - - - - diff --git a/driver-core/src/main/com/mongodb/internal/async/AsyncAggregateResponseBatchCursor.java b/driver-core/src/main/com/mongodb/internal/async/AsyncAggregateResponseBatchCursor.java index 2e9da84550c..ccfc9f7a956 100644 --- a/driver-core/src/main/com/mongodb/internal/async/AsyncAggregateResponseBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/async/AsyncAggregateResponseBatchCursor.java @@ -16,6 +16,7 @@ package com.mongodb.internal.async; +import com.mongodb.lang.Nullable; import org.bson.BsonDocument; import org.bson.BsonTimestamp; @@ -25,8 +26,10 @@ *

This class is not part of the public API and may be removed or changed at any time

*/ public interface AsyncAggregateResponseBatchCursor extends AsyncBatchCursor { + @Nullable BsonDocument getPostBatchResumeToken(); + @Nullable BsonTimestamp getOperationTime(); boolean isFirstBatchEmpty(); diff --git a/driver-core/src/main/com/mongodb/internal/async/AsyncBatchCursor.java b/driver-core/src/main/com/mongodb/internal/async/AsyncBatchCursor.java index 14cc3faa71b..89260ac7b52 100644 --- a/driver-core/src/main/com/mongodb/internal/async/AsyncBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/async/AsyncBatchCursor.java @@ -16,6 +16,8 @@ package com.mongodb.internal.async; +import com.mongodb.internal.operation.BatchCursor; + import java.io.Closeable; import java.util.List; @@ -28,9 +30,9 @@ */ public interface AsyncBatchCursor extends Closeable { /** - * Returns the next batch of results. A tailable cursor will block until another batch exists. After the last batch, the next call - * to this method will execute the callback with a null result to indicate that there are no more batches available and the cursor - * has been closed. + * Returns the next batch of results. A tailable cursor will block until another batch exists. + * Unlike the {@link BatchCursor} this method will automatically mark the cursor as closed when there are no more expected results. + * Care should be taken to check {@link #isClosed()} between calls. * * @param callback callback to receive the next batch of results * @throws java.util.NoSuchElementException if no next batch exists diff --git a/driver-core/src/main/com/mongodb/internal/operation/AggregateOperationImpl.java b/driver-core/src/main/com/mongodb/internal/operation/AggregateOperationImpl.java index 4379845bdd1..d70c7cbac66 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/AggregateOperationImpl.java +++ b/driver-core/src/main/com/mongodb/internal/operation/AggregateOperationImpl.java @@ -18,13 +18,11 @@ import com.mongodb.MongoNamespace; import com.mongodb.client.model.Collation; -import com.mongodb.connection.ConnectionDescription; import com.mongodb.internal.async.AsyncBatchCursor; import com.mongodb.internal.async.SingleResultCallback; import com.mongodb.internal.binding.AsyncReadBinding; import com.mongodb.internal.binding.ReadBinding; import com.mongodb.internal.client.model.AggregationLevel; -import com.mongodb.internal.connection.QueryResult; import com.mongodb.internal.session.SessionContext; import com.mongodb.lang.Nullable; import org.bson.BsonArray; @@ -40,7 +38,6 @@ import java.util.List; import java.util.concurrent.TimeUnit; -import static com.mongodb.assertions.Assertions.assertNotNull; import static com.mongodb.assertions.Assertions.isTrueArgument; import static com.mongodb.assertions.Assertions.notNull; import static com.mongodb.internal.async.ErrorHandlingResultCallback.errorHandlingCallback; @@ -48,7 +45,6 @@ import static com.mongodb.internal.operation.AsyncOperationHelper.executeRetryableReadAsync; import static com.mongodb.internal.operation.CommandOperationHelper.CommandCreator; import static com.mongodb.internal.operation.OperationHelper.LOGGER; -import static com.mongodb.internal.operation.OperationHelper.cursorDocumentToQueryResult; import static com.mongodb.internal.operation.OperationReadConcernHelper.appendReadConcernToCommand; import static com.mongodb.internal.operation.SyncOperationHelper.CommandReadTransformer; import static com.mongodb.internal.operation.SyncOperationHelper.executeRetryableRead; @@ -239,25 +235,16 @@ BsonDocument getCommand(final SessionContext sessionContext, final int maxWireVe return commandDocument; } - private QueryResult createQueryResult(final BsonDocument result, final ConnectionDescription description) { - assertNotNull(result); - return cursorDocumentToQueryResult(result.getDocument(CURSOR), description.getServerAddress()); - } - - private CommandReadTransformer> transformer() { - return (result, source, connection) -> { - QueryResult queryResult = createQueryResult(result, connection.getDescription()); - return new QueryBatchCursor<>(queryResult, 0, batchSize != null ? batchSize : 0, maxAwaitTimeMS, decoder, comment, - source, connection, result); - }; + private CommandReadTransformer> transformer() { + return (result, source, connection) -> + new CommandBatchCursor<>(connection.getDescription().getServerAddress(), result, 0, batchSize != null ? batchSize : 0, + maxAwaitTimeMS, decoder, comment, source, connection); } private CommandReadTransformerAsync> asyncTransformer() { - return (result, source, connection) -> { - QueryResult queryResult = createQueryResult(result, connection.getDescription()); - return new AsyncQueryBatchCursor<>(queryResult, 0, batchSize != null ? batchSize : 0, maxAwaitTimeMS, decoder, - comment, source, connection, result); - }; + return (result, source, connection) -> + new AsyncCommandBatchCursor<>(connection.getDescription().getServerAddress(), result, 0, batchSize != null ? batchSize : 0, + maxAwaitTimeMS, decoder, comment, source, connection); } interface AggregateTarget { diff --git a/driver-core/src/main/com/mongodb/internal/operation/AggregateResponseBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/AggregateResponseBatchCursor.java index 5ec7d00bb26..e12a2249123 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/AggregateResponseBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/AggregateResponseBatchCursor.java @@ -17,6 +17,7 @@ package com.mongodb.internal.operation; import com.mongodb.annotations.NotThreadSafe; +import com.mongodb.lang.Nullable; import org.bson.BsonDocument; import org.bson.BsonTimestamp; @@ -27,8 +28,10 @@ */ @NotThreadSafe public interface AggregateResponseBatchCursor extends BatchCursor { + @Nullable BsonDocument getPostBatchResumeToken(); + @Nullable BsonTimestamp getOperationTime(); boolean isFirstBatchEmpty(); diff --git a/driver-core/src/main/com/mongodb/internal/operation/AsyncChangeStreamBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/AsyncChangeStreamBatchCursor.java index 9ccd2f17b0a..1d271107a9a 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/AsyncChangeStreamBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/AsyncChangeStreamBatchCursor.java @@ -60,8 +60,7 @@ final class AsyncChangeStreamBatchCursor implements AsyncAggregateResponseBat final int maxWireVersion) { this.changeStreamOperation = changeStreamOperation; this.wrapped = new AtomicReference<>(assertNotNull(wrapped)); - this.binding = binding; - binding.retain(); + this.binding = binding.retain(); this.resumeToken = resumeToken; this.maxWireVersion = maxWireVersion; isClosed = new AtomicBoolean(); @@ -164,8 +163,8 @@ public int getMaxWireVersion() { return maxWireVersion; } - private void cachePostBatchResumeToken(final AsyncAggregateResponseBatchCursor queryBatchCursor) { - BsonDocument resumeToken = queryBatchCursor.getPostBatchResumeToken(); + private void cachePostBatchResumeToken(final AsyncAggregateResponseBatchCursor commandBatchCursor) { + BsonDocument resumeToken = commandBatchCursor.getPostBatchResumeToken(); if (resumeToken != null) { this.resumeToken = resumeToken; } diff --git a/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java new file mode 100644 index 00000000000..bc72c363f72 --- /dev/null +++ b/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java @@ -0,0 +1,524 @@ +/* + * Copyright 2008-present MongoDB, 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.mongodb.internal.operation; + +import com.mongodb.MongoCommandException; +import com.mongodb.MongoException; +import com.mongodb.MongoNamespace; +import com.mongodb.ReadPreference; +import com.mongodb.ServerAddress; +import com.mongodb.ServerCursor; +import com.mongodb.annotations.ThreadSafe; +import com.mongodb.connection.ServerType; +import com.mongodb.internal.VisibleForTesting; +import com.mongodb.internal.async.AsyncAggregateResponseBatchCursor; +import com.mongodb.internal.async.SingleResultCallback; +import com.mongodb.internal.async.function.AsyncCallbackSupplier; +import com.mongodb.internal.binding.AsyncConnectionSource; +import com.mongodb.internal.connection.AsyncConnection; +import com.mongodb.internal.connection.Connection; +import com.mongodb.lang.Nullable; +import org.bson.BsonArray; +import org.bson.BsonDocument; +import org.bson.BsonInt64; +import org.bson.BsonString; +import org.bson.BsonTimestamp; +import org.bson.BsonValue; +import org.bson.codecs.BsonDocumentCodec; +import org.bson.codecs.Decoder; + +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Consumer; + +import static com.mongodb.assertions.Assertions.assertNotNull; +import static com.mongodb.assertions.Assertions.assertNull; +import static com.mongodb.assertions.Assertions.assertTrue; +import static com.mongodb.assertions.Assertions.fail; +import static com.mongodb.assertions.Assertions.isTrueArgument; +import static com.mongodb.assertions.Assertions.notNull; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.FIRST_BATCH; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.MESSAGE_IF_CLOSED_AS_CURSOR; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.MESSAGE_IF_CONCURRENT_OPERATION; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.NEXT_BATCH; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.NO_OP_FIELD_NAME_VALIDATOR; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.getMoreCommandDocument; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.translateCommandException; +import static com.mongodb.internal.operation.OperationHelper.LOGGER; +import static java.lang.String.format; +import static java.util.Collections.emptyList; +import static java.util.Collections.singletonList; + +class AsyncCommandBatchCursor implements AsyncAggregateResponseBatchCursor { + + private final MongoNamespace namespace; + private final int limit; + private final Decoder decoder; + private final long maxTimeMS; + @Nullable + private final BsonValue comment; + + private final AtomicInteger count = new AtomicInteger(); + private final AtomicBoolean processedInitial = new AtomicBoolean(); + private final AtomicReference> commandCursorResult = new AtomicReference<>(); + private final int maxWireVersion; + private final boolean firstBatchEmpty; + private final ResourceManager resourceManager; + private int batchSize; + + AsyncCommandBatchCursor( + final ServerAddress serverAddress, + final BsonDocument commandCursorDocument, + final int limit, final int batchSize, final long maxTimeMS, + final Decoder decoder, + @Nullable final BsonValue comment, + final AsyncConnectionSource connectionSource, + final AsyncConnection connection) { + isTrueArgument("maxTimeMS >= 0", maxTimeMS >= 0); + CommandCursorResult commandCursor = initFromCommandCursorDocument(serverAddress, FIRST_BATCH, commandCursorDocument); + this.namespace = commandCursor.getNamespace(); + this.limit = limit; + this.batchSize = batchSize; + this.maxTimeMS = maxTimeMS; + this.decoder = notNull("decoder", decoder); + this.comment = comment; + this.maxWireVersion = connection.getDescription().getMaxWireVersion(); + this.firstBatchEmpty = commandCursor.getResults().isEmpty(); + + AsyncConnection connectionToPin = null; + boolean releaseServerAndResources = false; + if (limitReached()) { + releaseServerAndResources = true; + } else if (connectionSource.getServerDescription().getType() == ServerType.LOAD_BALANCER) { + connectionToPin = connection; + } + + resourceManager = new ResourceManager(connectionSource, connectionToPin, commandCursor.getServerCursor()); + if (releaseServerAndResources) { + resourceManager.releaseServerAndClientResources(connection.retain()); + } + } + + @Override + public void next(final SingleResultCallback> callback) { + if (isClosed()) { + callback.onResult(null, new MongoException(MESSAGE_IF_CLOSED_AS_CURSOR)); + return; + } else if (!resourceManager.tryStartOperation()) { + callback.onResult(null, new MongoException(MESSAGE_IF_CONCURRENT_OPERATION)); + return; + } + + ServerCursor localServerCursor = resourceManager.serverCursor(); + boolean cursorClosed = localServerCursor == null; + List batchResults = emptyList(); + if (!processedInitial.getAndSet(true) && !firstBatchEmpty) { + batchResults = commandCursorResult.get().getResults(); + } + + if (cursorClosed || !batchResults.isEmpty()) { + resourceManager.endOperation(); + if (cursorClosed) { + close(); + } + callback.onResult(batchResults, null); + } else { + getMore(localServerCursor, callback); + } + } + + @Override + public void setBatchSize(final int batchSize) { + this.batchSize = batchSize; + } + + @Override + public int getBatchSize() { + return batchSize; + } + + @Override + public boolean isClosed() { + return !resourceManager.operable(); + } + + @Override + public void close() { + resourceManager.close(); + } + + @Override + public BsonDocument getPostBatchResumeToken() { + return commandCursorResult.get().getPostBatchResumeToken(); + } + + @Override + public BsonTimestamp getOperationTime() { + return commandCursorResult.get().getOperationTime(); + } + + @Override + public boolean isFirstBatchEmpty() { + return firstBatchEmpty; + } + + @Override + public int getMaxWireVersion() { + return maxWireVersion; + } + + @Nullable + @VisibleForTesting(otherwise = VisibleForTesting.AccessModifier.PRIVATE) + ServerCursor getServerCursor() { + return resourceManager.serverCursor(); + } + + private void getMore(final ServerCursor cursor, final SingleResultCallback> callback) { + resourceManager.executeWithConnection(callback, connection -> getMore(connection, cursor, callback)); + } + + private void getMore(final AsyncConnection connection, final ServerCursor cursor, final SingleResultCallback> callback) { + connection.commandAsync(namespace.getDatabaseName(), + getMoreCommandDocument(cursor.getId(), connection.getDescription(), namespace, + limit, batchSize, count.get(), maxTimeMS, comment), + NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), + CommandResultDocumentCodec.create(decoder, NEXT_BATCH), assertNotNull(resourceManager.connectionSource), + (commandResult, t) -> { + if (t != null) { + Throwable translatedException = + t instanceof MongoCommandException + ? translateCommandException((MongoCommandException) t, cursor) + : t; + + connection.release(); + resourceManager.endOperation(); + callback.onResult(null, translatedException); + return; + } + + CommandCursorResult commandCursor = + initFromCommandCursorDocument(connection.getDescription().getServerAddress(), NEXT_BATCH, assertNotNull(commandResult)); + resourceManager.setServerCursor(commandCursor.getServerCursor()); + + if (!resourceManager.operable()) { + resourceManager.releaseServerAndClientResources(connection); + callback.onResult(emptyList(), null); + return; + } + + if (commandCursor.getResults().isEmpty() && commandCursor.getServerCursor() != null) { + connection.release(); + getMore(commandCursor.getServerCursor(), callback); + } else { + resourceManager.endOperation(); + if (limitReached()) { + resourceManager.releaseServerAndClientResources(connection); + } else { + connection.release(); + } + callback.onResult(commandCursor.getResults(), null); + } + }); + } + + private CommandCursorResult initFromCommandCursorDocument( + final ServerAddress serverAddress, + final String fieldNameContainingBatch, + final BsonDocument commandCursorDocument) { + CommandCursorResult cursorResult = new CommandCursorResult<>(serverAddress, fieldNameContainingBatch, commandCursorDocument); + count.addAndGet(cursorResult.getResults().size()); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(format("Received batch of %d documents with cursorId %d from server %s", cursorResult.getResults().size(), + cursorResult.getCursorId(), cursorResult.getServerAddress())); + } + this.commandCursorResult.set(cursorResult); + return cursorResult; + } + + private boolean limitReached() { + return Math.abs(limit) != 0 && count.get() >= Math.abs(limit); + } + + /** + * This class maintains all resources that must be released in {@link AsyncCommandBatchCursor#close()}. + * It also implements a {@linkplain #doClose() deferred close action} such that it is totally ordered with other operations of + * {@link AsyncCommandBatchCursor} (methods {@link #tryStartOperation()}/{@link #endOperation()} must be used properly to enforce the order) + * despite the method {@link AsyncCommandBatchCursor#close()} being called concurrently with those operations. + * This total order induces the happens-before order. + *

+ * The deferred close action does not violate externally observable idempotence of {@link AsyncCommandBatchCursor#close()}, + * because {@link AsyncCommandBatchCursor#close()} is allowed to release resources "eventually". + *

+ * Only methods explicitly documented as thread-safe are thread-safe, + * others are not and rely on the total order mentioned above. + */ + @ThreadSafe + private final class ResourceManager { + private final Lock lock; + private volatile State state = State.IDLE; + @Nullable + private volatile AsyncConnectionSource connectionSource; + @Nullable + private volatile AsyncConnection pinnedConnection; + @Nullable + private volatile ServerCursor serverCursor; + private volatile boolean skipReleasingServerResourcesOnClose; + + ResourceManager(final AsyncConnectionSource connectionSource, + @Nullable final AsyncConnection connectionToPin, + @Nullable final ServerCursor serverCursor) { + lock = new ReentrantLock(); + if (serverCursor != null) { + this.connectionSource = notNull("connectionSource", connectionSource).retain(); + if (connectionToPin != null) { + this.pinnedConnection = connectionToPin.retain(); + connectionToPin.markAsPinned(Connection.PinningMode.CURSOR); + } + } + skipReleasingServerResourcesOnClose = false; + this.serverCursor = serverCursor; + } + + /** + * Thread-safe. + */ + boolean operable() { + return state.operable(); + } + + /** + * Thread-safe. + * Returns {@code true} iff started an operation. + * If {@linkplain State#operable() closed}, then returns false, otherwise completes abruptly. + * + * @throws IllegalStateException Iff another operation is in progress. + */ + private boolean tryStartOperation() throws IllegalStateException { + lock.lock(); + try { + State localState = state; + if (!localState.operable()) { + return false; + } else if (localState == State.IDLE) { + state = State.OPERATION_IN_PROGRESS; + return true; + } else if (localState == State.OPERATION_IN_PROGRESS) { + return false; + } else { + throw fail(state.toString()); + } + } finally { + lock.unlock(); + } + } + + /** + * Thread-safe. + */ + private void endOperation() { + boolean doClose = false; + lock.lock(); + try { + State localState = state; + if (localState == State.OPERATION_IN_PROGRESS) { + state = State.IDLE; + } else if (localState == State.CLOSE_PENDING) { + state = State.CLOSED; + doClose = true; + } else if (localState != State.CLOSED) { + fail(localState.toString()); + } + } finally { + lock.unlock(); + } + if (doClose) { + doClose(); + } + } + + /** + * Thread-safe. + */ + void close() { + boolean doClose = false; + lock.lock(); + try { + State localState = state; + if (localState == State.OPERATION_IN_PROGRESS) { + state = State.CLOSE_PENDING; + } else if (localState != State.CLOSED) { + state = State.CLOSED; + doClose = true; + } + } finally { + lock.unlock(); + } + if (doClose) { + doClose(); + } + } + + /** + * This method is never executed concurrently with either itself or other operations + * demarcated by {@link #tryStartOperation()}/{@link #endOperation()}. + */ + private void doClose() { + if (skipReleasingServerResourcesOnClose) { + serverCursor = null; + releaseClientResources(); + } else if (serverCursor != null) { + executeWithConnection((connection, t) -> { + // guarantee that regardless of exceptions, `serverCursor` is null and client resources are released + serverCursor = null; + releaseClientResources(); + }, resourceManager::releaseServerAndClientResources); + } + } + + void onCorruptedConnection(@Nullable final AsyncConnection corruptedConnection) { + // if `pinnedConnection` is corrupted, then we cannot kill `serverCursor` via such a connection + AsyncConnection localPinnedConnection = pinnedConnection; + if (localPinnedConnection != null) { + assertTrue(corruptedConnection == localPinnedConnection); + skipReleasingServerResourcesOnClose = true; + } + } + + void executeWithConnection( + final SingleResultCallback callback, + final Consumer function) { + assertTrue(state != State.IDLE); + if (pinnedConnection != null) { + executeWithConnection(assertNotNull(pinnedConnection).retain(), null, callback, function); + } else { + assertNotNull(connectionSource).getConnection((conn, t) -> executeWithConnection(conn, t, callback, function)); + } + } + + void executeWithConnection( + @Nullable final AsyncConnection connection, + @Nullable final Throwable t, + final SingleResultCallback callback, + final Consumer function) { + assertTrue(connection != null || t != null); + if (t != null) { + try { + onCorruptedConnection(connection); + } catch (Exception suppressed) { + t.addSuppressed(suppressed); + } + callback.onResult(null, t); + } else { + AsyncCallbackSupplier curriedFunction = c -> function.accept(connection); + curriedFunction.whenComplete(connection::release).get(callback); + } + } + + /** + * Thread-safe. + */ + @Nullable + ServerCursor serverCursor() { + return serverCursor; + } + + void setServerCursor(@Nullable final ServerCursor serverCursor) { + assertTrue(state.inProgress()); + assertNotNull(connectionSource); + this.serverCursor = serverCursor; + if (serverCursor == null) { + releaseClientResources(); + } + } + + private void releaseServerAndClientResources(final AsyncConnection connection) { + lock.lock(); + ServerCursor localServerCursor = serverCursor; + serverCursor = null; + lock.unlock(); + if (localServerCursor != null) { + killServerCursor(namespace, localServerCursor, connection); + } else { + connection.release(); + releaseClientResources(); + } + } + + private void killServerCursor(final MongoNamespace namespace, final ServerCursor serverCursor, final AsyncConnection connection) { + connection + .commandAsync(namespace.getDatabaseName(), asKillCursorsCommandDocument(namespace, serverCursor), + NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), new BsonDocumentCodec(), assertNotNull(connectionSource), + (r, t) -> { + connection.release(); + releaseClientResources(); + }); + } + + private BsonDocument asKillCursorsCommandDocument(final MongoNamespace namespace, final ServerCursor serverCursor) { + return new BsonDocument("killCursors", new BsonString(namespace.getCollectionName())) + .append("cursors", new BsonArray(singletonList(new BsonInt64(serverCursor.getId())))); + } + + private void releaseClientResources() { + assertNull(serverCursor); + lock.lock(); + AsyncConnectionSource localConnectionSource = connectionSource; + connectionSource = null; + + AsyncConnection localPinnedConnection = pinnedConnection; + pinnedConnection = null; + lock.unlock(); + + if (localConnectionSource != null) { + localConnectionSource.release(); + } + if (localPinnedConnection != null) { + localPinnedConnection.release(); + } + } + } + + private enum State { + IDLE(true, false), + OPERATION_IN_PROGRESS(true, true), + /** + * Implies {@link #OPERATION_IN_PROGRESS}. + */ + CLOSE_PENDING(false, true), + CLOSED(false, false); + + private final boolean operable; + private final boolean inProgress; + + State(final boolean operable, final boolean inProgress) { + this.operable = operable; + this.inProgress = inProgress; + } + + boolean operable() { + return operable; + } + + boolean inProgress() { + return inProgress; + } + } +} diff --git a/driver-core/src/main/com/mongodb/internal/operation/AsyncOperationHelper.java b/driver-core/src/main/com/mongodb/internal/operation/AsyncOperationHelper.java index 21b10cdff08..b670083be78 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/AsyncOperationHelper.java +++ b/driver-core/src/main/com/mongodb/internal/operation/AsyncOperationHelper.java @@ -18,9 +18,7 @@ import com.mongodb.Function; import com.mongodb.MongoException; -import com.mongodb.MongoNamespace; import com.mongodb.ReadPreference; -import com.mongodb.ServerAddress; import com.mongodb.assertions.Assertions; import com.mongodb.internal.async.AsyncBatchCursor; import com.mongodb.internal.async.SingleResultCallback; @@ -35,7 +33,6 @@ import com.mongodb.internal.binding.ReferenceCounted; import com.mongodb.internal.connection.AsyncConnection; import com.mongodb.internal.connection.OperationContext; -import com.mongodb.internal.connection.QueryResult; import com.mongodb.internal.operation.retry.AttachmentKeys; import com.mongodb.internal.validator.NoOpFieldNameValidator; import com.mongodb.lang.Nullable; @@ -56,7 +53,6 @@ import static com.mongodb.internal.operation.CommandOperationHelper.isRetryWritesEnabled; import static com.mongodb.internal.operation.CommandOperationHelper.logRetryExecute; import static com.mongodb.internal.operation.CommandOperationHelper.transformWriteException; -import static com.mongodb.internal.operation.OperationHelper.cursorDocumentToQueryResult; import static com.mongodb.internal.operation.WriteConcernHelper.throwOnWriteConcernError; final class AsyncOperationHelper { @@ -309,15 +305,15 @@ static CommandWriteTransformerAsync writeConcernErrorTransfo }; } - static AsyncBatchCursor createEmptyAsyncBatchCursor(final MongoNamespace namespace, final ServerAddress serverAddress) { - return new AsyncSingleBatchQueryCursor<>(new QueryResult<>(namespace, Collections.emptyList(), 0L, serverAddress)); + static CommandReadTransformerAsync> asyncSingleBatchCursorTransformer(final String resultsFieldName) { + return (result, source, connection) -> + new AsyncSingleBatchCursor<>(BsonDocumentWrapperHelper.toList(result, resultsFieldName), 0); } static AsyncBatchCursor cursorDocumentToAsyncBatchCursor(final BsonDocument cursorDocument, final Decoder decoder, final BsonValue comment, final AsyncConnectionSource source, final AsyncConnection connection, final int batchSize) { - return new AsyncQueryBatchCursor<>(cursorDocumentToQueryResult(cursorDocument, - source.getServerDescription().getAddress()), - 0, batchSize, 0, decoder, comment, source, connection, cursorDocument); + return new AsyncCommandBatchCursor<>(connection.getDescription().getServerAddress(), cursorDocument, + 0, batchSize, 0, decoder, comment, source, connection); } static SingleResultCallback releasingCallback(final SingleResultCallback wrapped, final AsyncConnection connection) { diff --git a/driver-core/src/main/com/mongodb/internal/operation/AsyncQueryBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/AsyncQueryBatchCursor.java deleted file mode 100644 index be079abec36..00000000000 --- a/driver-core/src/main/com/mongodb/internal/operation/AsyncQueryBatchCursor.java +++ /dev/null @@ -1,428 +0,0 @@ -/* - * Copyright 2008-present MongoDB, 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.mongodb.internal.operation; - -import com.mongodb.MongoCommandException; -import com.mongodb.MongoException; -import com.mongodb.MongoNamespace; -import com.mongodb.ReadPreference; -import com.mongodb.ServerCursor; -import com.mongodb.connection.ConnectionDescription; -import com.mongodb.connection.ServerType; -import com.mongodb.internal.async.AsyncAggregateResponseBatchCursor; -import com.mongodb.internal.async.SingleResultCallback; -import com.mongodb.internal.binding.AsyncConnectionSource; -import com.mongodb.internal.connection.AsyncConnection; -import com.mongodb.internal.connection.Connection; -import com.mongodb.internal.connection.QueryResult; -import com.mongodb.internal.diagnostics.logging.Logger; -import com.mongodb.internal.diagnostics.logging.Loggers; -import com.mongodb.internal.validator.NoOpFieldNameValidator; -import com.mongodb.lang.Nullable; -import org.bson.BsonArray; -import org.bson.BsonDocument; -import org.bson.BsonInt32; -import org.bson.BsonInt64; -import org.bson.BsonString; -import org.bson.BsonTimestamp; -import org.bson.BsonValue; -import org.bson.FieldNameValidator; -import org.bson.codecs.BsonDocumentCodec; -import org.bson.codecs.Decoder; - -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - -import static com.mongodb.assertions.Assertions.assertFalse; -import static com.mongodb.assertions.Assertions.assertNotNull; -import static com.mongodb.assertions.Assertions.isTrueArgument; -import static com.mongodb.assertions.Assertions.notNull; -import static com.mongodb.internal.async.ErrorHandlingResultCallback.errorHandlingCallback; -import static com.mongodb.internal.operation.CursorHelper.getNumberToReturn; -import static com.mongodb.internal.operation.DocumentHelper.putIfNotNull; -import static com.mongodb.internal.operation.SyncOperationHelper.getMoreCursorDocumentToQueryResult; -import static com.mongodb.internal.operation.QueryHelper.translateCommandException; -import static com.mongodb.internal.operation.ServerVersionHelper.serverIsAtLeastVersionFourDotFour; -import static java.lang.String.format; -import static java.util.Collections.singletonList; - -class AsyncQueryBatchCursor implements AsyncAggregateResponseBatchCursor { - private static final Logger LOGGER = Loggers.getLogger("operation"); - private static final FieldNameValidator NO_OP_FIELD_NAME_VALIDATOR = new NoOpFieldNameValidator(); - private static final String CURSOR = "cursor"; - private static final String POST_BATCH_RESUME_TOKEN = "postBatchResumeToken"; - private static final String OPERATION_TIME = "operationTime"; - - private final MongoNamespace namespace; - private final int limit; - private final Decoder decoder; - private final long maxTimeMS; - private volatile AsyncConnectionSource connectionSource; - private volatile AsyncConnection pinnedConnection; - private final AtomicReference cursor; - private volatile QueryResult firstBatch; - private volatile int batchSize; - private final AtomicInteger count = new AtomicInteger(); - private volatile BsonDocument postBatchResumeToken; - private final BsonTimestamp operationTime; - private final BsonValue comment; - private final boolean firstBatchEmpty; - private final int maxWireVersion; - - private final Lock lock = new ReentrantLock(); - /* protected by `lock` */ - private boolean isOperationInProgress = false; - private boolean isClosed = false; - /* protected by `lock` */ - private volatile boolean isClosePending = false; - - AsyncQueryBatchCursor(final QueryResult firstBatch, final int limit, final int batchSize, final long maxTimeMS, - final Decoder decoder, final BsonValue comment, final AsyncConnectionSource connectionSource, - final AsyncConnection connection) { - this(firstBatch, limit, batchSize, maxTimeMS, decoder, comment, connectionSource, connection, null); - } - - AsyncQueryBatchCursor(final QueryResult firstBatch, final int limit, final int batchSize, final long maxTimeMS, - final Decoder decoder, final BsonValue comment, final AsyncConnectionSource connectionSource, - @Nullable final AsyncConnection connection, @Nullable final BsonDocument result) { - isTrueArgument("maxTimeMS >= 0", maxTimeMS >= 0); - this.maxTimeMS = maxTimeMS; - this.namespace = firstBatch.getNamespace(); - this.firstBatch = firstBatch; - this.limit = limit; - this.batchSize = batchSize; - this.decoder = decoder; - this.comment = comment; - this.cursor = new AtomicReference<>(firstBatch.getCursor()); - this.count.addAndGet(firstBatch.getResults().size()); - if (result != null) { - this.operationTime = result.getTimestamp(OPERATION_TIME, null); - this.postBatchResumeToken = getPostBatchResumeTokenFromResponse(result); - } else { - this.operationTime = null; - } - - firstBatchEmpty = firstBatch.getResults().isEmpty(); - if (cursor.get() != null) { - this.connectionSource = notNull("connectionSource", connectionSource).retain(); - assertNotNull(connection); - if (limitReached()) { - killCursor(connection); - } else { - if (connectionSource.getServerDescription().getType() == ServerType.LOAD_BALANCER) { - this.pinnedConnection = connection.retain(); - this.pinnedConnection.markAsPinned(Connection.PinningMode.CURSOR); - } - } - } - this.maxWireVersion = connection == null ? 0 : connection.getDescription().getMaxWireVersion(); - logQueryResult(firstBatch); - } - - /** - * {@inheritDoc} - *

- * From the perspective of the code external to this class, this method is idempotent as required by its specification. - * However, if this method sets {@link #isClosePending}, - * then it must be called by {@code this} again to release resources. - * This behavior does not violate externally observable idempotence because this method is allowed to release resources "eventually". - */ - @Override - public void close() { - boolean doClose = false; - - lock.lock(); - try { - if (isOperationInProgress) { - isClosePending = true; - } else if (!isClosed) { - isClosed = true; - isClosePending = false; - doClose = true; - } - } finally { - lock.unlock(); - } - - if (doClose) { - killCursorOnClose(); - } - } - - @Override - public void next(final SingleResultCallback> callback) { - if (isClosed()) { - callback.onResult(null, new MongoException("next() called after the cursor was closed.")); - } else if (firstBatch != null && (!firstBatch.getResults().isEmpty())) { - // May be empty for a tailable cursor - List results = firstBatch.getResults(); - firstBatch = null; - if (getServerCursor() == null) { - close(); - } - callback.onResult(results, null); - } else { - ServerCursor localCursor = getServerCursor(); - if (localCursor == null) { - close(); - callback.onResult(null, null); - } else { - lock.lock(); - try { - if (isClosed()) { - callback.onResult(null, new MongoException("next() called after the cursor was closed.")); - return; - } - isOperationInProgress = true; - } finally { - lock.unlock(); - } - getMore(localCursor, callback); - } - } - } - - @Override - public void setBatchSize(final int batchSize) { - assertFalse(isClosed()); - this.batchSize = batchSize; - } - - @Override - public int getBatchSize() { - assertFalse(isClosed()); - return batchSize; - } - - @Override - public boolean isClosed() { - lock.lock(); - try { - return isClosed || isClosePending; - } finally { - lock.unlock(); - } - } - - @Override - public BsonDocument getPostBatchResumeToken() { - return postBatchResumeToken; - } - - @Override - public BsonTimestamp getOperationTime() { - return operationTime; - } - - @Override - public boolean isFirstBatchEmpty() { - return firstBatchEmpty; - } - - @Override - public int getMaxWireVersion() { - return maxWireVersion; - } - - private boolean limitReached() { - return Math.abs(limit) != 0 && count.get() >= Math.abs(limit); - } - - private void getMore(final ServerCursor cursor, final SingleResultCallback> callback) { - if (pinnedConnection != null) { - getMore(pinnedConnection.retain(), cursor, callback); - } else { - connectionSource.getConnection((connection, t) -> { - if (t != null) { - endOperationInProgress(); - callback.onResult(null, t); - } else { - getMore(assertNotNull(connection), cursor, callback); - } - }); - } - } - - private void getMore(final AsyncConnection connection, final ServerCursor cursor, final SingleResultCallback> callback) { - connection.commandAsync(namespace.getDatabaseName(), asGetMoreCommandDocument(cursor.getId(), connection.getDescription()), - NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), CommandResultDocumentCodec.create(decoder, "nextBatch"), - connectionSource, new CommandResultSingleResultCallback(connection, cursor, callback)); - } - - private BsonDocument asGetMoreCommandDocument(final long cursorId, final ConnectionDescription connectionDescription) { - BsonDocument document = new BsonDocument("getMore", new BsonInt64(cursorId)) - .append("collection", new BsonString(namespace.getCollectionName())); - - int batchSizeForGetMoreCommand = Math.abs(getNumberToReturn(limit, this.batchSize, count.get())); - if (batchSizeForGetMoreCommand != 0) { - document.append("batchSize", new BsonInt32(batchSizeForGetMoreCommand)); - } - if (maxTimeMS != 0) { - document.append("maxTimeMS", new BsonInt64(maxTimeMS)); - } - if (serverIsAtLeastVersionFourDotFour(connectionDescription)) { - putIfNotNull(document, "comment", comment); - } - return document; - } - - private void killCursorOnClose() { - ServerCursor localCursor = getServerCursor(); - if (localCursor != null) { - if (pinnedConnection != null) { - killCursorAsynchronouslyAndReleaseConnectionAndSource(pinnedConnection, localCursor); - } else { - connectionSource.getConnection((connection, t) -> { - if (t != null) { - connectionSource.release(); - } else { - killCursorAsynchronouslyAndReleaseConnectionAndSource(assertNotNull(connection), localCursor); - } - }); - } - } else if (pinnedConnection != null) { - pinnedConnection.release(); - } - } - - private void killCursor(final AsyncConnection connection) { - ServerCursor localCursor = cursor.getAndSet(null); - if (localCursor != null) { - killCursorAsynchronouslyAndReleaseConnectionAndSource(connection.retain(), localCursor); - } else { - connectionSource.release(); - } - } - - private void killCursorAsynchronouslyAndReleaseConnectionAndSource(final AsyncConnection connection, final ServerCursor localCursor) { - connection.commandAsync(namespace.getDatabaseName(), asKillCursorsCommandDocument(localCursor), NO_OP_FIELD_NAME_VALIDATOR, - ReadPreference.primary(), new BsonDocumentCodec(), connectionSource, (result, t) -> { - connection.release(); - connectionSource.release(); - }); - } - - private BsonDocument asKillCursorsCommandDocument(final ServerCursor localCursor) { - return new BsonDocument("killCursors", new BsonString(namespace.getCollectionName())) - .append("cursors", new BsonArray(singletonList(new BsonInt64(localCursor.getId())))); - } - - private void endOperationInProgress() { - boolean closePending; - lock.lock(); - try { - isOperationInProgress = false; - closePending = this.isClosePending; - } finally { - lock.unlock(); - } - if (closePending) { - close(); - } - } - - - private void handleGetMoreQueryResult(final AsyncConnection connection, final SingleResultCallback> callback, - final QueryResult result) { - logQueryResult(result); - cursor.set(result.getCursor()); - if (isClosePending) { - try { - connection.release(); - if (result.getCursor() == null) { - connectionSource.release(); - } - endOperationInProgress(); - } finally { - callback.onResult(null, null); - } - } else if (result.getResults().isEmpty() && result.getCursor() != null) { - getMore(connection, assertNotNull(result.getCursor()), callback); - } else { - count.addAndGet(result.getResults().size()); - if (limitReached()) { - killCursor(connection); - connection.release(); - } else { - connection.release(); - if (result.getCursor() == null) { - connectionSource.release(); - } - } - endOperationInProgress(); - - if (result.getResults().isEmpty()) { - callback.onResult(null, null); - } else { - callback.onResult(result.getResults(), null); - } - } - } - - private void logQueryResult(final QueryResult result) { - LOGGER.debug(format("Received batch of %d documents with cursorId %d from server %s", result.getResults().size(), - result.getCursorId(), result.getAddress())); - } - - private class CommandResultSingleResultCallback implements SingleResultCallback { - private final AsyncConnection connection; - private final ServerCursor cursor; - private final SingleResultCallback> callback; - - CommandResultSingleResultCallback(final AsyncConnection connection, final ServerCursor cursor, - final SingleResultCallback> callback) { - this.connection = connection; - this.cursor = cursor; - this.callback = errorHandlingCallback(callback, LOGGER); - } - - @Override - public void onResult(@Nullable final BsonDocument result, @Nullable final Throwable t) { - if (t != null) { - Throwable translatedException = t instanceof MongoCommandException - ? translateCommandException((MongoCommandException) t, cursor) - : t; - connection.release(); - endOperationInProgress(); - callback.onResult(null, translatedException); - } else { - assertNotNull(result); - QueryResult queryResult = getMoreCursorDocumentToQueryResult(result.getDocument(CURSOR), - connection.getDescription().getServerAddress()); - postBatchResumeToken = getPostBatchResumeTokenFromResponse(result); - handleGetMoreQueryResult(connection, callback, queryResult); - } - } - } - - @Nullable - ServerCursor getServerCursor() { - return cursor.get(); - } - - @Nullable - private BsonDocument getPostBatchResumeTokenFromResponse(final BsonDocument result) { - BsonDocument cursor = result.getDocument(CURSOR, null); - if (cursor != null) { - return cursor.getDocument(POST_BATCH_RESUME_TOKEN, null); - } - return null; - } -} diff --git a/driver-core/src/main/com/mongodb/internal/operation/AsyncSingleBatchQueryCursor.java b/driver-core/src/main/com/mongodb/internal/operation/AsyncSingleBatchCursor.java similarity index 64% rename from driver-core/src/main/com/mongodb/internal/operation/AsyncSingleBatchQueryCursor.java rename to driver-core/src/main/com/mongodb/internal/operation/AsyncSingleBatchCursor.java index f29cda04dae..90cabeb074f 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/AsyncSingleBatchQueryCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/AsyncSingleBatchCursor.java @@ -19,19 +19,26 @@ import com.mongodb.MongoException; import com.mongodb.internal.async.AsyncBatchCursor; import com.mongodb.internal.async.SingleResultCallback; -import com.mongodb.internal.connection.QueryResult; import java.util.List; -import static com.mongodb.assertions.Assertions.isTrue; +import static java.util.Collections.emptyList; -class AsyncSingleBatchQueryCursor implements AsyncBatchCursor { - private volatile QueryResult firstBatch; - private volatile boolean closed; +class AsyncSingleBatchCursor implements AsyncBatchCursor { - AsyncSingleBatchQueryCursor(final QueryResult firstBatch) { - this.firstBatch = firstBatch; - isTrue("Empty Cursor", firstBatch.getCursor() == null); + static AsyncSingleBatchCursor createEmptyBatchCursor(final int batchSize) { + return new AsyncSingleBatchCursor<>(emptyList(), batchSize); + } + + private final List batch; + private final int batchSize; + + private volatile boolean hasNext = true; + private volatile boolean closed = false; + + AsyncSingleBatchCursor(final List batch, final int batchSize) { + this.batch = batch; + this.batchSize = batchSize; } @Override @@ -43,13 +50,12 @@ public void close() { public void next(final SingleResultCallback> callback) { if (closed) { callback.onResult(null, new MongoException("next() called after the cursor was closed.")); - } else if (firstBatch != null && !firstBatch.getResults().isEmpty()) { - List results = firstBatch.getResults(); - firstBatch = null; - callback.onResult(results, null); + } else if (hasNext && !batch.isEmpty()) { + hasNext = false; + callback.onResult(batch, null); } else { closed = true; - callback.onResult(null, null); + callback.onResult(emptyList(), null); } } @@ -60,7 +66,7 @@ public void setBatchSize(final int batchSize) { @Override public int getBatchSize() { - return 0; + return batchSize; } @Override diff --git a/driver-core/src/main/com/mongodb/internal/operation/ChangeStreamBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/ChangeStreamBatchCursor.java index acf70090457..a5d36f86f55 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/ChangeStreamBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/ChangeStreamBatchCursor.java @@ -64,23 +64,23 @@ AggregateResponseBatchCursor getWrapped() { @Override public boolean hasNext() { - return resumeableOperation(queryBatchCursor -> { + return resumeableOperation(commandBatchCursor -> { try { - return queryBatchCursor.hasNext(); + return commandBatchCursor.hasNext(); } finally { - cachePostBatchResumeToken(queryBatchCursor); + cachePostBatchResumeToken(commandBatchCursor); } }); } @Override public List next() { - return resumeableOperation(queryBatchCursor -> { + return resumeableOperation(commandBatchCursor -> { try { - return convertAndProduceLastId(queryBatchCursor.next(), changeStreamOperation.getDecoder(), + return convertAndProduceLastId(commandBatchCursor.next(), changeStreamOperation.getDecoder(), lastId -> resumeToken = lastId); } finally { - cachePostBatchResumeToken(queryBatchCursor); + cachePostBatchResumeToken(commandBatchCursor); } }); } @@ -92,12 +92,12 @@ public int available() { @Override public List tryNext() { - return resumeableOperation(queryBatchCursor -> { + return resumeableOperation(commandBatchCursor -> { try { - return convertAndProduceLastId(queryBatchCursor.tryNext(), changeStreamOperation.getDecoder(), + return convertAndProduceLastId(commandBatchCursor.tryNext(), changeStreamOperation.getDecoder(), lastId -> resumeToken = lastId); } finally { - cachePostBatchResumeToken(queryBatchCursor); + cachePostBatchResumeToken(commandBatchCursor); } }); } @@ -155,9 +155,9 @@ public int getMaxWireVersion() { return maxWireVersion; } - private void cachePostBatchResumeToken(final AggregateResponseBatchCursor queryBatchCursor) { - if (queryBatchCursor.getPostBatchResumeToken() != null) { - resumeToken = queryBatchCursor.getPostBatchResumeToken(); + private void cachePostBatchResumeToken(final AggregateResponseBatchCursor commandBatchCursor) { + if (commandBatchCursor.getPostBatchResumeToken() != null) { + resumeToken = commandBatchCursor.getPostBatchResumeToken(); } } @@ -170,7 +170,7 @@ static List convertAndProduceLastId(@Nullable final List final Decoder decoder, final Consumer lastIdConsumer) { List results = null; - if (rawDocuments != null) { + if (rawDocuments != null && !rawDocuments.isEmpty()) { results = new ArrayList<>(); for (RawBsonDocument rawDocument : rawDocuments) { if (!rawDocument.containsKey("_id")) { diff --git a/driver-core/src/main/com/mongodb/internal/operation/QueryBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java similarity index 68% rename from driver-core/src/main/com/mongodb/internal/operation/QueryBatchCursor.java rename to driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java index 139c3e6fd27..32da9ab63f6 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/QueryBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java @@ -24,23 +24,17 @@ import com.mongodb.ServerAddress; import com.mongodb.ServerCursor; import com.mongodb.annotations.ThreadSafe; -import com.mongodb.connection.ConnectionDescription; import com.mongodb.connection.ServerType; +import com.mongodb.internal.VisibleForTesting; import com.mongodb.internal.binding.ConnectionSource; import com.mongodb.internal.connection.Connection; -import com.mongodb.internal.connection.QueryResult; -import com.mongodb.internal.diagnostics.logging.Logger; -import com.mongodb.internal.diagnostics.logging.Loggers; -import com.mongodb.internal.validator.NoOpFieldNameValidator; import com.mongodb.lang.Nullable; import org.bson.BsonArray; import org.bson.BsonDocument; -import org.bson.BsonInt32; import org.bson.BsonInt64; import org.bson.BsonString; import org.bson.BsonTimestamp; import org.bson.BsonValue; -import org.bson.FieldNameValidator; import org.bson.codecs.BsonDocumentCodec; import org.bson.codecs.Decoder; @@ -57,89 +51,69 @@ import static com.mongodb.assertions.Assertions.fail; import static com.mongodb.assertions.Assertions.isTrueArgument; import static com.mongodb.assertions.Assertions.notNull; -import static com.mongodb.internal.operation.CursorHelper.getNumberToReturn; -import static com.mongodb.internal.operation.DocumentHelper.putIfNotNull; -import static com.mongodb.internal.operation.SyncOperationHelper.getMoreCursorDocumentToQueryResult; -import static com.mongodb.internal.operation.QueryHelper.translateCommandException; -import static com.mongodb.internal.operation.ServerVersionHelper.serverIsAtLeastVersionFourDotFour; +import static com.mongodb.internal.VisibleForTesting.AccessModifier.PRIVATE; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.FIRST_BATCH; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.MESSAGE_IF_CLOSED_AS_CURSOR; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.MESSAGE_IF_CLOSED_AS_ITERATOR; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.MESSAGE_IF_CONCURRENT_OPERATION; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.NEXT_BATCH; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.NO_OP_FIELD_NAME_VALIDATOR; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.getMoreCommandDocument; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.translateCommandException; +import static com.mongodb.internal.operation.OperationHelper.LOGGER; import static java.lang.String.format; import static java.util.Collections.singletonList; -class QueryBatchCursor implements AggregateResponseBatchCursor { - private static final Logger LOGGER = Loggers.getLogger("operation"); - private static final FieldNameValidator NO_OP_FIELD_NAME_VALIDATOR = new NoOpFieldNameValidator(); - private static final String CURSOR = "cursor"; - private static final String POST_BATCH_RESUME_TOKEN = "postBatchResumeToken"; - private static final String OPERATION_TIME = "operationTime"; - private static final String MESSAGE_IF_CLOSED_AS_CURSOR = "Cursor has been closed"; - private static final String MESSAGE_IF_CLOSED_AS_ITERATOR = "Iterator has been closed"; +class CommandBatchCursor implements AggregateResponseBatchCursor { private final MongoNamespace namespace; - private final ServerAddress serverAddress; private final int limit; private final Decoder decoder; private final long maxTimeMS; - private int batchSize; + @Nullable private final BsonValue comment; + + private CommandCursorResult commandCursorResult; + private int batchSize; + @Nullable private List nextBatch; - private int count; - private BsonDocument postBatchResumeToken; - private BsonTimestamp operationTime; + private int count = 0; + private final boolean firstBatchEmpty; - private int maxWireVersion = 0; + private final int maxWireVersion; private final ResourceManager resourceManager; - QueryBatchCursor(final QueryResult firstQueryResult, final int limit, final int batchSize, final Decoder decoder) { - this(firstQueryResult, limit, batchSize, decoder, null, null); - } - - QueryBatchCursor(final QueryResult firstQueryResult, final int limit, final int batchSize, final Decoder decoder, - @Nullable final BsonValue comment, @Nullable final ConnectionSource connectionSource) { - this(firstQueryResult, limit, batchSize, 0, decoder, comment, connectionSource, null, null); - } - - QueryBatchCursor(final QueryResult firstQueryResult, final int limit, final int batchSize, final long maxTimeMS, - final Decoder decoder, @Nullable final BsonValue comment, @Nullable final ConnectionSource connectionSource, - @Nullable final Connection connection) { - this(firstQueryResult, limit, batchSize, maxTimeMS, decoder, comment, connectionSource, connection, null); - } - - QueryBatchCursor(final QueryResult firstQueryResult, final int limit, final int batchSize, final long maxTimeMS, - final Decoder decoder, @Nullable final BsonValue comment, @Nullable final ConnectionSource connectionSource, - @Nullable final Connection connection, @Nullable final BsonDocument result) { + CommandBatchCursor( + final ServerAddress serverAddress, + final BsonDocument commandCursorDocument, + final int limit, final int batchSize, final long maxTimeMS, + final Decoder decoder, + @Nullable final BsonValue comment, + final ConnectionSource connectionSource, + final Connection connection) { isTrueArgument("maxTimeMS >= 0", maxTimeMS >= 0); - this.maxTimeMS = maxTimeMS; - this.namespace = firstQueryResult.getNamespace(); - this.serverAddress = firstQueryResult.getAddress(); + initFromCommandCursorDocument(serverAddress, FIRST_BATCH, commandCursorDocument); + this.namespace = commandCursorResult.getNamespace(); this.limit = limit; - this.comment = comment; this.batchSize = batchSize; + this.maxTimeMS = maxTimeMS; this.decoder = notNull("decoder", decoder); - if (result != null) { - this.operationTime = result.getTimestamp(OPERATION_TIME, null); - this.postBatchResumeToken = getPostBatchResumeTokenFromResponse(result); - } - ServerCursor serverCursor = initFromQueryResult(firstQueryResult); - if (serverCursor != null) { - notNull("connectionSource", connectionSource); - } - firstBatchEmpty = firstQueryResult.getResults().isEmpty(); + this.comment = comment; + + firstBatchEmpty = commandCursorResult.getResults().isEmpty(); Connection connectionToPin = null; boolean releaseServerAndResources = false; - if (connection != null) { - this.maxWireVersion = connection.getDescription().getMaxWireVersion(); - if (limitReached()) { - releaseServerAndResources = true; - } else { - assertNotNull(connectionSource); - if (connectionSource.getServerDescription().getType() == ServerType.LOAD_BALANCER) { - connectionToPin = connection; - } - } + + this.maxWireVersion = connection.getDescription().getMaxWireVersion(); + if (limitReached()) { + releaseServerAndResources = true; + } else if (connectionSource.getServerDescription().getType() == ServerType.LOAD_BALANCER) { + connectionToPin = connection; } - resourceManager = new ResourceManager(connectionSource, connectionToPin, serverCursor); + + resourceManager = new ResourceManager(connectionSource, connectionToPin, commandCursorResult.getServerCursor()); if (releaseServerAndResources) { - resourceManager.releaseServerAndClientResources(assertNotNull(connection)); + resourceManager.releaseServerAndClientResources(connection); } } @@ -180,6 +154,7 @@ public int available() { return !resourceManager.operable() || nextBatch == null ? 0 : nextBatch.size(); } + @Nullable private List doNext() { if (!doHasNext()) { throw new NoSuchElementException(); @@ -190,6 +165,12 @@ private List doNext() { return retVal; } + + @VisibleForTesting(otherwise = PRIVATE) + boolean isClosed() { + return !resourceManager.operable(); + } + @Override public void setBatchSize(final int batchSize) { this.batchSize = batchSize; @@ -243,7 +224,6 @@ public ServerCursor getServerCursor() { if (!resourceManager.operable()) { throw new IllegalStateException(MESSAGE_IF_CLOSED_AS_ITERATOR); } - return resourceManager.serverCursor(); } @@ -253,17 +233,17 @@ public ServerAddress getServerAddress() { throw new IllegalStateException(MESSAGE_IF_CLOSED_AS_ITERATOR); } - return serverAddress; + return commandCursorResult.getServerAddress(); } @Override public BsonDocument getPostBatchResumeToken() { - return postBatchResumeToken; + return commandCursorResult.getPostBatchResumeToken(); } @Override public BsonTimestamp getOperationTime() { - return operationTime; + return commandCursorResult.getOperationTime(); } @Override @@ -281,78 +261,51 @@ private void getMore() { resourceManager.executeWithConnection(connection -> { ServerCursor nextServerCursor; try { - nextServerCursor = initFromCommandResult(connection.command(namespace.getDatabaseName(), - asGetMoreCommandDocument(serverCursor.getId(), connection.getDescription()), - NO_OP_FIELD_NAME_VALIDATOR, - ReadPreference.primary(), - CommandResultDocumentCodec.create(decoder, "nextBatch"), - assertNotNull(resourceManager.connectionSource))); + initFromCommandCursorDocument(connection.getDescription().getServerAddress(), NEXT_BATCH, + assertNotNull( + connection.command(namespace.getDatabaseName(), + getMoreCommandDocument(serverCursor.getId(), connection.getDescription(), namespace, + limit, batchSize, count, maxTimeMS, comment), + NO_OP_FIELD_NAME_VALIDATOR, + ReadPreference.primary(), + CommandResultDocumentCodec.create(decoder, NEXT_BATCH), + assertNotNull(resourceManager.connectionSource)))); + nextServerCursor = commandCursorResult.getServerCursor(); } catch (MongoCommandException e) { throw translateCommandException(e, serverCursor); } + resourceManager.setServerCursor(nextServerCursor); - if (limitReached()) { + if (!resourceManager.operable() || limitReached() || nextServerCursor == null) { resourceManager.releaseServerAndClientResources(connection); } }); } - private BsonDocument asGetMoreCommandDocument(final long cursorId, final ConnectionDescription connectionDescription) { - BsonDocument document = new BsonDocument("getMore", new BsonInt64(cursorId)) - .append("collection", new BsonString(namespace.getCollectionName())); - - int batchSizeForGetMoreCommand = Math.abs(getNumberToReturn(limit, this.batchSize, count)); - if (batchSizeForGetMoreCommand != 0) { - document.append("batchSize", new BsonInt32(batchSizeForGetMoreCommand)); - } - if (maxTimeMS != 0) { - document.append("maxTimeMS", new BsonInt64(maxTimeMS)); + private void initFromCommandCursorDocument(final ServerAddress serverAddress, final String fieldNameContainingBatch, + final BsonDocument commandCursorDocument) { + this.commandCursorResult = new CommandCursorResult<>(serverAddress, fieldNameContainingBatch, commandCursorDocument); + this.nextBatch = commandCursorResult.getResults().isEmpty() ? null : commandCursorResult.getResults(); + this.count += commandCursorResult.getResults().size(); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(format("Received batch of %d documents with cursorId %d from server %s", commandCursorResult.getResults().size(), + commandCursorResult.getCursorId(), commandCursorResult.getServerAddress())); } - if (serverIsAtLeastVersionFourDotFour(connectionDescription)) { - putIfNotNull(document, "comment", comment); - } - return document; - } - - @Nullable - private ServerCursor initFromQueryResult(final QueryResult queryResult) { - nextBatch = queryResult.getResults().isEmpty() ? null : queryResult.getResults(); - count += queryResult.getResults().size(); - LOGGER.debug(format("Received batch of %d documents with cursorId %d from server %s", queryResult.getResults().size(), - queryResult.getCursorId(), queryResult.getAddress())); - return queryResult.getCursor(); - } - - @Nullable - private ServerCursor initFromCommandResult(final BsonDocument getMoreCommandResultDocument) { - QueryResult queryResult = getMoreCursorDocumentToQueryResult(getMoreCommandResultDocument.getDocument(CURSOR), serverAddress); - postBatchResumeToken = getPostBatchResumeTokenFromResponse(getMoreCommandResultDocument); - operationTime = getMoreCommandResultDocument.getTimestamp(OPERATION_TIME, null); - return initFromQueryResult(queryResult); } private boolean limitReached() { return Math.abs(limit) != 0 && count >= Math.abs(limit); } - @Nullable - private BsonDocument getPostBatchResumeTokenFromResponse(final BsonDocument result) { - BsonDocument cursor = result.getDocument(CURSOR, null); - if (cursor != null) { - return cursor.getDocument(POST_BATCH_RESUME_TOKEN, null); - } - return null; - } - /** - * This class maintains all resources that must be released in {@link QueryBatchCursor#close()}. + * This class maintains all resources that must be released in {@link CommandBatchCursor#close()}. * It also implements a {@linkplain #doClose() deferred close action} such that it is totally ordered with other operations of - * {@link QueryBatchCursor} (methods {@link #tryStartOperation()}/{@link #endOperation()} must be used properly to enforce the order) - * despite the method {@link QueryBatchCursor#close()} being called concurrently with those operations. + * {@link CommandBatchCursor} (methods {@link #tryStartOperation()}/{@link #endOperation()} must be used properly to enforce the order) + * despite the method {@link CommandBatchCursor#close()} being called concurrently with those operations. * This total order induces the happens-before order. *

- * The deferred close action does not violate externally observable idempotence of {@link QueryBatchCursor#close()}, - * because {@link QueryBatchCursor#close()} is allowed to release resources "eventually". + * The deferred close action does not violate externally observable idempotence of {@link CommandBatchCursor#close()}, + * because {@link CommandBatchCursor#close()} is allowed to release resources "eventually". *

* Only methods explicitly documented as thread-safe are thread-safe, * others are not and rely on the total order mentioned above. @@ -369,12 +322,13 @@ private final class ResourceManager { private volatile ServerCursor serverCursor; private volatile boolean skipReleasingServerResourcesOnClose; - ResourceManager(@Nullable final ConnectionSource connectionSource, - @Nullable final Connection connectionToPin, @Nullable final ServerCursor serverCursor) { + ResourceManager(final ConnectionSource connectionSource, + @Nullable final Connection connectionToPin, + @Nullable final ServerCursor serverCursor) { lock = new StampedLock().asWriteLock(); state = State.IDLE; if (serverCursor != null) { - this.connectionSource = (assertNotNull(connectionSource)).retain(); + this.connectionSource = notNull("connectionSource", connectionSource).retain(); if (connectionToPin != null) { this.pinnedConnection = connectionToPin.retain(); connectionToPin.markAsPinned(Connection.PinningMode.CURSOR); @@ -395,7 +349,7 @@ boolean operable() { * Thread-safe. * Executes {@code operation} within the {@link #tryStartOperation()}/{@link #endOperation()} bounds. * - * @throws IllegalStateException If {@linkplain QueryBatchCursor#close() closed}. + * @throws IllegalStateException If {@linkplain CommandBatchCursor#close() closed}. */ @Nullable R execute(final String exceptionMessageIfClosed, final Supplier operation) throws IllegalStateException { @@ -425,7 +379,7 @@ private boolean tryStartOperation() throws IllegalStateException { state = State.OPERATION_IN_PROGRESS; return true; } else if (localState == State.OPERATION_IN_PROGRESS) { - throw new IllegalStateException("Another operation is currently in progress, concurrent operations are not supported"); + throw new IllegalStateException(MESSAGE_IF_CONCURRENT_OPERATION); } else { throw fail(state.toString()); } @@ -533,10 +487,10 @@ void executeWithConnection(final Consumer action) { private Connection connection() { assertTrue(state != State.IDLE); - if (pinnedConnection == null) { - return assertNotNull(connectionSource).getConnection(); - } else { + if (pinnedConnection != null) { return assertNotNull(pinnedConnection).retain(); + } else { + return assertNotNull(connectionSource).getConnection(); } } diff --git a/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursorHelper.java b/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursorHelper.java new file mode 100644 index 00000000000..d7e125b5eb8 --- /dev/null +++ b/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursorHelper.java @@ -0,0 +1,78 @@ +/* + * Copyright 2008-present MongoDB, 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.mongodb.internal.operation; + +import com.mongodb.MongoCommandException; +import com.mongodb.MongoCursorNotFoundException; +import com.mongodb.MongoNamespace; +import com.mongodb.MongoQueryException; +import com.mongodb.ServerCursor; +import com.mongodb.connection.ConnectionDescription; +import com.mongodb.internal.validator.NoOpFieldNameValidator; +import com.mongodb.lang.Nullable; +import org.bson.BsonDocument; +import org.bson.BsonInt32; +import org.bson.BsonInt64; +import org.bson.BsonString; +import org.bson.BsonValue; +import org.bson.FieldNameValidator; + +import static com.mongodb.internal.operation.CursorHelper.getNumberToReturn; +import static com.mongodb.internal.operation.DocumentHelper.putIfNotNull; +import static com.mongodb.internal.operation.ServerVersionHelper.serverIsAtLeastVersionFourDotFour; + +final class CommandBatchCursorHelper { + + static final String FIRST_BATCH = "firstBatch"; + static final String NEXT_BATCH = "nextBatch"; + static final FieldNameValidator NO_OP_FIELD_NAME_VALIDATOR = new NoOpFieldNameValidator(); + static final String MESSAGE_IF_CLOSED_AS_CURSOR = "Cursor has been closed"; + static final String MESSAGE_IF_CLOSED_AS_ITERATOR = "Iterator has been closed"; + + static final String MESSAGE_IF_CONCURRENT_OPERATION = "Another operation is currently in progress, concurrent operations are not " + + "supported"; + + static BsonDocument getMoreCommandDocument( + final long cursorId, final ConnectionDescription connectionDescription, final MongoNamespace namespace, final int limit, + final int batchSize, final int count, final long maxTimeMS, @Nullable final BsonValue comment) { + BsonDocument document = new BsonDocument("getMore", new BsonInt64(cursorId)) + .append("collection", new BsonString(namespace.getCollectionName())); + + int batchSizeForGetMoreCommand = Math.abs(getNumberToReturn(limit, batchSize, count)); + if (batchSizeForGetMoreCommand != 0) { + document.append("batchSize", new BsonInt32(batchSizeForGetMoreCommand)); + } + if (maxTimeMS != 0) { + document.append("maxTimeMS", new BsonInt64(maxTimeMS)); + } + if (serverIsAtLeastVersionFourDotFour(connectionDescription)) { + putIfNotNull(document, "comment", comment); + } + return document; + } + + static MongoQueryException translateCommandException(final MongoCommandException commandException, final ServerCursor cursor) { + if (commandException.getErrorCode() == 43) { + return new MongoCursorNotFoundException(cursor.getId(), commandException.getResponse(), cursor.getAddress()); + } else { + return new MongoQueryException(commandException.getResponse(), commandException.getServerAddress()); + } + } + + private CommandBatchCursorHelper() { + } +} diff --git a/driver-core/src/main/com/mongodb/internal/connection/QueryResult.java b/driver-core/src/main/com/mongodb/internal/operation/CommandCursorResult.java similarity index 52% rename from driver-core/src/main/com/mongodb/internal/connection/QueryResult.java rename to driver-core/src/main/com/mongodb/internal/operation/CommandCursorResult.java index 52970ba7b94..8fa6c1bdde3 100644 --- a/driver-core/src/main/com/mongodb/internal/connection/QueryResult.java +++ b/driver-core/src/main/com/mongodb/internal/operation/CommandCursorResult.java @@ -14,40 +14,52 @@ * limitations under the License. */ -package com.mongodb.internal.connection; +package com.mongodb.internal.operation; import com.mongodb.MongoNamespace; import com.mongodb.ServerAddress; import com.mongodb.ServerCursor; import com.mongodb.lang.Nullable; +import org.bson.BsonDocument; +import org.bson.BsonTimestamp; import java.util.List; +import static com.mongodb.assertions.Assertions.isTrue; + /** - * A batch of query results. + * The command cursor result * *

This class is not part of the public API and may be removed or changed at any time

*/ -public class QueryResult { - private final MongoNamespace namespace; +public class CommandCursorResult { + + private static final String CURSOR = "cursor"; + private static final String POST_BATCH_RESUME_TOKEN = "postBatchResumeToken"; + private static final String OPERATION_TIME = "operationTime"; + + private final BsonDocument cursorDocument; + private final ServerAddress serverAddress; private final List results; + private final MongoNamespace namespace; private final long cursorId; - private final ServerAddress serverAddress; + @Nullable + private final BsonTimestamp operationTime; + @Nullable + private final BsonDocument postBatchResumeToken; - /** - * Construct an instance. - * - * @param namespace the namespace - * @param results the query results - * @param cursorId the cursor id - * @param serverAddress the server address - */ - public QueryResult(@Nullable final MongoNamespace namespace, final List results, final long cursorId, - final ServerAddress serverAddress) { - this.namespace = namespace; - this.results = results; - this.cursorId = cursorId; + public CommandCursorResult( + final ServerAddress serverAddress, + final String fieldNameContainingBatch, + final BsonDocument commandCursorDocument) { + isTrue("Contains cursor", commandCursorDocument.isDocument(CURSOR)); this.serverAddress = serverAddress; + this.cursorDocument = commandCursorDocument.getDocument(CURSOR); + this.results = BsonDocumentWrapperHelper.toList(cursorDocument, fieldNameContainingBatch); + this.namespace = new MongoNamespace(cursorDocument.getString("ns").getValue()); + this.cursorId = cursorDocument.getNumber("id").longValue(); + this.operationTime = cursorDocument.getTimestamp(OPERATION_TIME, null); + this.postBatchResumeToken = cursorDocument.getDocument(POST_BATCH_RESUME_TOKEN, null); } /** @@ -55,7 +67,6 @@ public QueryResult(@Nullable final MongoNamespace namespace, final List resul * * @return the namespace */ - @Nullable public MongoNamespace getNamespace() { return namespace; } @@ -66,7 +77,7 @@ public MongoNamespace getNamespace() { * @return the cursor, which may be null if it's been exhausted */ @Nullable - public ServerCursor getCursor() { + public ServerCursor getServerCursor() { return cursorId == 0 ? null : new ServerCursor(cursorId, serverAddress); } @@ -84,11 +95,21 @@ public List getResults() { * * @return the server address */ - public ServerAddress getAddress() { + public ServerAddress getServerAddress() { return serverAddress; } public long getCursorId() { return cursorId; } + + @Nullable + public BsonDocument getPostBatchResumeToken() { + return postBatchResumeToken; + } + + @Nullable + public BsonTimestamp getOperationTime() { + return operationTime; + } } diff --git a/driver-core/src/main/com/mongodb/internal/operation/DistinctOperation.java b/driver-core/src/main/com/mongodb/internal/operation/DistinctOperation.java index a64c4cbfadd..488e044333e 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/DistinctOperation.java +++ b/driver-core/src/main/com/mongodb/internal/operation/DistinctOperation.java @@ -23,7 +23,6 @@ import com.mongodb.internal.async.SingleResultCallback; import com.mongodb.internal.binding.AsyncReadBinding; import com.mongodb.internal.binding.ReadBinding; -import com.mongodb.internal.connection.QueryResult; import com.mongodb.internal.session.SessionContext; import com.mongodb.lang.Nullable; import org.bson.BsonDocument; @@ -36,14 +35,12 @@ import static com.mongodb.assertions.Assertions.notNull; import static com.mongodb.internal.async.ErrorHandlingResultCallback.errorHandlingCallback; -import static com.mongodb.internal.operation.AsyncOperationHelper.CommandReadTransformerAsync; import static com.mongodb.internal.operation.AsyncOperationHelper.executeRetryableReadAsync; import static com.mongodb.internal.operation.CommandOperationHelper.CommandCreator; import static com.mongodb.internal.operation.DocumentHelper.putIfNotNull; import static com.mongodb.internal.operation.DocumentHelper.putIfNotZero; import static com.mongodb.internal.operation.OperationHelper.LOGGER; import static com.mongodb.internal.operation.OperationReadConcernHelper.appendReadConcernToCommand; -import static com.mongodb.internal.operation.SyncOperationHelper.CommandReadTransformer; import static com.mongodb.internal.operation.SyncOperationHelper.executeRetryableRead; /** @@ -119,38 +116,23 @@ public DistinctOperation comment(final BsonValue comment) { @Override public BatchCursor execute(final ReadBinding binding) { - return executeRetryableRead(binding, namespace.getDatabaseName(), getCommandCreator(binding.getSessionContext()), - createCommandDecoder(), transformer(), retryReads); + return executeRetryableRead(binding, namespace.getDatabaseName(), + getCommandCreator(binding.getSessionContext()), createCommandDecoder(), + SyncOperationHelper.singleBatchCursorTransformer(VALUES), retryReads); } @Override public void executeAsync(final AsyncReadBinding binding, final SingleResultCallback> callback) { - executeRetryableReadAsync(binding, namespace.getDatabaseName(), getCommandCreator(binding.getSessionContext()), - createCommandDecoder(), asyncTransformer(), retryReads, errorHandlingCallback(callback, LOGGER)); + executeRetryableReadAsync(binding, namespace.getDatabaseName(), + getCommandCreator(binding.getSessionContext()), createCommandDecoder(), + AsyncOperationHelper.asyncSingleBatchCursorTransformer(VALUES), retryReads, + errorHandlingCallback(callback, LOGGER)); } private Codec createCommandDecoder() { return CommandResultDocumentCodec.create(decoder, VALUES); } - private QueryResult createQueryResult(final BsonDocument result, final ConnectionDescription description) { - return new QueryResult<>(namespace, BsonDocumentWrapperHelper.toList(result, VALUES), 0L, - description.getServerAddress()); - } - - private CommandReadTransformer> transformer() { - return (result, source, connection) -> { - QueryResult queryResult = createQueryResult(result, connection.getDescription()); - return new QueryBatchCursor<>(queryResult, 0, 0, decoder, comment, source); - }; - } - - private CommandReadTransformerAsync> asyncTransformer() { - return (result, source, connection) -> { - QueryResult queryResult = createQueryResult(result, connection.getDescription()); - return new AsyncSingleBatchQueryCursor<>(queryResult); - }; - } private CommandCreator getCommandCreator(final SessionContext sessionContext) { return (serverDescription, connectionDescription) -> getCommand(sessionContext, connectionDescription); diff --git a/driver-core/src/main/com/mongodb/internal/operation/FindOperation.java b/driver-core/src/main/com/mongodb/internal/operation/FindOperation.java index dcb94211fcf..83441bdc5e2 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/FindOperation.java +++ b/driver-core/src/main/com/mongodb/internal/operation/FindOperation.java @@ -29,7 +29,6 @@ import com.mongodb.internal.binding.AsyncReadBinding; import com.mongodb.internal.binding.ReadBinding; import com.mongodb.internal.connection.NoOpSessionContext; -import com.mongodb.internal.connection.QueryResult; import com.mongodb.internal.session.SessionContext; import com.mongodb.lang.Nullable; import org.bson.BsonBoolean; @@ -57,7 +56,6 @@ import static com.mongodb.internal.operation.ExplainHelper.asExplainCommand; import static com.mongodb.internal.operation.OperationHelper.LOGGER; import static com.mongodb.internal.operation.OperationHelper.canRetryRead; -import static com.mongodb.internal.operation.OperationHelper.cursorDocumentToQueryResult; import static com.mongodb.internal.operation.OperationReadConcernHelper.appendReadConcernToCommand; import static com.mongodb.internal.operation.ServerVersionHelper.MIN_WIRE_VERSION; import static com.mongodb.internal.operation.SyncOperationHelper.CommandReadTransformer; @@ -471,13 +469,10 @@ private boolean isAwaitData() { return cursorType == CursorType.TailableAwait; } - private CommandReadTransformer> transformer() { - return (result, source, connection) -> { - QueryResult queryResult = cursorDocumentToQueryResult(result.getDocument("cursor"), - connection.getDescription().getServerAddress()); - return new QueryBatchCursor<>(queryResult, limit, batchSize, getMaxTimeForCursor(), decoder, comment, source, connection, - result); - }; + private CommandReadTransformer> transformer() { + return (result, source, connection) -> + new CommandBatchCursor<>(connection.getDescription().getServerAddress(), result, limit, batchSize, + getMaxTimeForCursor(), decoder, comment, source, connection); } private long getMaxTimeForCursor() { @@ -485,11 +480,9 @@ private long getMaxTimeForCursor() { } private CommandReadTransformerAsync> asyncTransformer() { - return (result, source, connection) -> { - QueryResult queryResult = cursorDocumentToQueryResult(result.getDocument("cursor"), - connection.getDescription().getServerAddress()); - return new AsyncQueryBatchCursor<>(queryResult, limit, batchSize, getMaxTimeForCursor(), decoder, comment, source, - connection, result); - }; + return (result, source, connection) -> + new AsyncCommandBatchCursor<>(connection.getDescription().getServerAddress(), result, limit, batchSize, + getMaxTimeForCursor(), decoder, comment, source, connection); + } } diff --git a/driver-core/src/main/com/mongodb/internal/operation/ListCollectionsOperation.java b/driver-core/src/main/com/mongodb/internal/operation/ListCollectionsOperation.java index fa2a5dcd995..28b5c9ad3af 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/ListCollectionsOperation.java +++ b/driver-core/src/main/com/mongodb/internal/operation/ListCollectionsOperation.java @@ -16,13 +16,11 @@ package com.mongodb.internal.operation; -import com.mongodb.MongoCommandException; -import com.mongodb.MongoNamespace; +import com.mongodb.MongoCommandException;; import com.mongodb.internal.async.AsyncBatchCursor; import com.mongodb.internal.async.SingleResultCallback; import com.mongodb.internal.async.function.AsyncCallbackSupplier; import com.mongodb.internal.async.function.RetryState; -import com.mongodb.internal.binding.AsyncConnectionSource; import com.mongodb.internal.binding.AsyncReadBinding; import com.mongodb.internal.binding.ReadBinding; import com.mongodb.lang.Nullable; @@ -40,7 +38,6 @@ import static com.mongodb.assertions.Assertions.notNull; import static com.mongodb.internal.async.ErrorHandlingResultCallback.errorHandlingCallback; import static com.mongodb.internal.operation.AsyncOperationHelper.CommandReadTransformerAsync; -import static com.mongodb.internal.operation.AsyncOperationHelper.createEmptyAsyncBatchCursor; import static com.mongodb.internal.operation.AsyncOperationHelper.createReadCommandAndExecuteAsync; import static com.mongodb.internal.operation.AsyncOperationHelper.cursorDocumentToAsyncBatchCursor; import static com.mongodb.internal.operation.AsyncOperationHelper.decorateReadWithRetriesAsync; @@ -52,7 +49,6 @@ import static com.mongodb.internal.operation.DocumentHelper.putIfNotNull; import static com.mongodb.internal.operation.OperationHelper.LOGGER; import static com.mongodb.internal.operation.OperationHelper.canRetryRead; -import static com.mongodb.internal.operation.OperationHelper.createEmptyBatchCursor; import static com.mongodb.internal.operation.SyncOperationHelper.CommandReadTransformer; import static com.mongodb.internal.operation.SyncOperationHelper.createReadCommandAndExecute; import static com.mongodb.internal.operation.SyncOperationHelper.cursorDocumentToBatchCursor; @@ -148,8 +144,8 @@ public BatchCursor execute(final ReadBinding binding) { return createReadCommandAndExecute(retryState, binding, source, databaseName, getCommandCreator(), createCommandDecoder(), commandTransformer(), connection); } catch (MongoCommandException e) { - return rethrowIfNotNamespaceError(e, createEmptyBatchCursor(createNamespace(), decoder, - source.getServerDescription().getAddress(), batchSize)); + return rethrowIfNotNamespaceError(e, + SingleBatchCursor.createEmptyBatchCursor(source.getServerDescription().getAddress(), batchSize)); } }) ); @@ -173,7 +169,9 @@ public void executeAsync(final AsyncReadBinding binding, final SingleResultCallb if (t != null && !isNamespaceError(t)) { releasingCallback.onResult(null, t); } else { - releasingCallback.onResult(result != null ? result : emptyAsyncCursor(source), null); + releasingCallback.onResult(result != null + ? result + : AsyncSingleBatchCursor.createEmptyBatchCursor(batchSize), null); } }); }) @@ -181,20 +179,12 @@ public void executeAsync(final AsyncReadBinding binding, final SingleResultCallb asyncRead.get(errorHandlingCallback(callback, LOGGER)); } - private AsyncBatchCursor emptyAsyncCursor(final AsyncConnectionSource source) { - return createEmptyAsyncBatchCursor(createNamespace(), source.getServerDescription().getAddress()); - } - - private MongoNamespace createNamespace() { - return new MongoNamespace(databaseName, "$cmd.listCollections"); - } - private CommandReadTransformerAsync> asyncTransformer() { - return (result, source, connection) -> cursorDocumentToAsyncBatchCursor(result.getDocument("cursor"), decoder, comment, source, connection, batchSize); + return (result, source, connection) -> cursorDocumentToAsyncBatchCursor(result, decoder, comment, source, connection, batchSize); } private CommandReadTransformer> commandTransformer() { - return (result, source, connection) -> cursorDocumentToBatchCursor(result.getDocument("cursor"), decoder, comment, source, connection, batchSize); + return (result, source, connection) -> cursorDocumentToBatchCursor(result, decoder, comment, source, connection, batchSize); } private CommandOperationHelper.CommandCreator getCommandCreator() { diff --git a/driver-core/src/main/com/mongodb/internal/operation/ListDatabasesOperation.java b/driver-core/src/main/com/mongodb/internal/operation/ListDatabasesOperation.java index bacf64601c9..f86ada76f10 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/ListDatabasesOperation.java +++ b/driver-core/src/main/com/mongodb/internal/operation/ListDatabasesOperation.java @@ -16,12 +16,10 @@ package com.mongodb.internal.operation; -import com.mongodb.connection.ConnectionDescription; import com.mongodb.internal.async.AsyncBatchCursor; import com.mongodb.internal.async.SingleResultCallback; import com.mongodb.internal.binding.AsyncReadBinding; import com.mongodb.internal.binding.ReadBinding; -import com.mongodb.internal.connection.QueryResult; import com.mongodb.lang.Nullable; import org.bson.BsonBoolean; import org.bson.BsonDocument; @@ -34,12 +32,10 @@ import static com.mongodb.assertions.Assertions.notNull; import static com.mongodb.internal.async.ErrorHandlingResultCallback.errorHandlingCallback; -import static com.mongodb.internal.operation.AsyncOperationHelper.CommandReadTransformerAsync; import static com.mongodb.internal.operation.AsyncOperationHelper.executeRetryableReadAsync; import static com.mongodb.internal.operation.CommandOperationHelper.CommandCreator; import static com.mongodb.internal.operation.DocumentHelper.putIfNotNull; import static com.mongodb.internal.operation.OperationHelper.LOGGER; -import static com.mongodb.internal.operation.SyncOperationHelper.CommandReadTransformer; import static com.mongodb.internal.operation.SyncOperationHelper.executeRetryableRead; @@ -122,28 +118,16 @@ public ListDatabasesOperation comment(@Nullable final BsonValue comment) { @Override public BatchCursor execute(final ReadBinding binding) { return executeRetryableRead(binding, "admin", getCommandCreator(), - CommandResultDocumentCodec.create(decoder, "databases"), transformer(), retryReads); + CommandResultDocumentCodec.create(decoder, "databases"), + SyncOperationHelper.singleBatchCursorTransformer("databases"), retryReads); } @Override public void executeAsync(final AsyncReadBinding binding, final SingleResultCallback> callback) { executeRetryableReadAsync(binding, "admin", getCommandCreator(), - CommandResultDocumentCodec.create(decoder, "databases"), asyncTransformer(), - retryReads, errorHandlingCallback(callback, LOGGER)); - } - - private CommandReadTransformer> transformer() { - return (result, source, connection) -> new QueryBatchCursor<>(createQueryResult(result, connection.getDescription()), 0, 0, decoder, comment, source); - } - - private CommandReadTransformerAsync> asyncTransformer() { - return (result, source, connection) -> new AsyncQueryBatchCursor<>(createQueryResult(result, connection.getDescription()), 0, 0, 0, decoder, - comment, source, connection, result); - } - - private QueryResult createQueryResult(final BsonDocument result, final ConnectionDescription description) { - return new QueryResult<>(null, BsonDocumentWrapperHelper.toList(result, "databases"), 0, - description.getServerAddress()); + CommandResultDocumentCodec.create(decoder, "databases"), + AsyncOperationHelper.asyncSingleBatchCursorTransformer("databases"), retryReads, + errorHandlingCallback(callback, LOGGER)); } private CommandCreator getCommandCreator() { diff --git a/driver-core/src/main/com/mongodb/internal/operation/ListIndexesOperation.java b/driver-core/src/main/com/mongodb/internal/operation/ListIndexesOperation.java index 62ecdc953bd..1c51862f39d 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/ListIndexesOperation.java +++ b/driver-core/src/main/com/mongodb/internal/operation/ListIndexesOperation.java @@ -22,7 +22,6 @@ import com.mongodb.internal.async.SingleResultCallback; import com.mongodb.internal.async.function.AsyncCallbackSupplier; import com.mongodb.internal.async.function.RetryState; -import com.mongodb.internal.binding.AsyncConnectionSource; import com.mongodb.internal.binding.AsyncReadBinding; import com.mongodb.internal.binding.ReadBinding; import com.mongodb.lang.Nullable; @@ -39,7 +38,6 @@ import static com.mongodb.assertions.Assertions.notNull; import static com.mongodb.internal.async.ErrorHandlingResultCallback.errorHandlingCallback; import static com.mongodb.internal.operation.AsyncOperationHelper.CommandReadTransformerAsync; -import static com.mongodb.internal.operation.AsyncOperationHelper.createEmptyAsyncBatchCursor; import static com.mongodb.internal.operation.AsyncOperationHelper.createReadCommandAndExecuteAsync; import static com.mongodb.internal.operation.AsyncOperationHelper.cursorDocumentToAsyncBatchCursor; import static com.mongodb.internal.operation.AsyncOperationHelper.decorateReadWithRetriesAsync; @@ -52,7 +50,6 @@ import static com.mongodb.internal.operation.DocumentHelper.putIfNotNull; import static com.mongodb.internal.operation.OperationHelper.LOGGER; import static com.mongodb.internal.operation.OperationHelper.canRetryRead; -import static com.mongodb.internal.operation.OperationHelper.createEmptyBatchCursor; import static com.mongodb.internal.operation.SyncOperationHelper.CommandReadTransformer; import static com.mongodb.internal.operation.SyncOperationHelper.createReadCommandAndExecute; import static com.mongodb.internal.operation.SyncOperationHelper.cursorDocumentToBatchCursor; @@ -127,8 +124,8 @@ public BatchCursor execute(final ReadBinding binding) { return createReadCommandAndExecute(retryState, binding, source, namespace.getDatabaseName(), getCommandCreator(), createCommandDecoder(), transformer(), connection); } catch (MongoCommandException e) { - return rethrowIfNotNamespaceError(e, createEmptyBatchCursor(namespace, decoder, - source.getServerDescription().getAddress(), batchSize)); + return rethrowIfNotNamespaceError(e, + SingleBatchCursor.createEmptyBatchCursor(source.getServerDescription().getAddress(), batchSize)); } }) ); @@ -152,7 +149,10 @@ public void executeAsync(final AsyncReadBinding binding, final SingleResultCallb if (t != null && !isNamespaceError(t)) { releasingCallback.onResult(null, t); } else { - releasingCallback.onResult(result != null ? result : emptyAsyncCursor(source), null); + releasingCallback.onResult(result != null + ? result + : AsyncSingleBatchCursor.createEmptyBatchCursor(batchSize), + null); } }); }) @@ -160,10 +160,6 @@ public void executeAsync(final AsyncReadBinding binding, final SingleResultCallb asyncRead.get(errorHandlingCallback(callback, LOGGER)); } - private AsyncBatchCursor emptyAsyncCursor(final AsyncConnectionSource source) { - return createEmptyAsyncBatchCursor(namespace, source.getServerDescription().getAddress()); - } - private CommandCreator getCommandCreator() { return (serverDescription, connectionDescription) -> getCommand(); } @@ -179,11 +175,11 @@ private BsonDocument getCommand() { } private CommandReadTransformer> transformer() { - return (result, source, connection) -> cursorDocumentToBatchCursor(result.getDocument("cursor"), decoder, comment, source, connection, batchSize); + return (result, source, connection) -> cursorDocumentToBatchCursor(result, decoder, comment, source, connection, batchSize); } private CommandReadTransformerAsync> asyncTransformer() { - return (result, source, connection) -> cursorDocumentToAsyncBatchCursor(result.getDocument("cursor"), decoder, comment, source, connection, batchSize); + return (result, source, connection) -> cursorDocumentToAsyncBatchCursor(result, decoder, comment, source, connection, batchSize); } private Codec createCommandDecoder() { diff --git a/driver-core/src/main/com/mongodb/internal/operation/ListSearchIndexesOperation.java b/driver-core/src/main/com/mongodb/internal/operation/ListSearchIndexesOperation.java index 4c471a16bd4..db360097f94 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/ListSearchIndexesOperation.java +++ b/driver-core/src/main/com/mongodb/internal/operation/ListSearchIndexesOperation.java @@ -34,9 +34,8 @@ import java.util.Collections; import java.util.concurrent.TimeUnit; -import static com.mongodb.internal.operation.AsyncOperationHelper.createEmptyAsyncBatchCursor; import static com.mongodb.internal.operation.CommandOperationHelper.isNamespaceError; -import static com.mongodb.internal.operation.OperationHelper.createEmptyBatchCursor; + /** * An operation that lists Alas Search indexes with the help of {@value #STAGE_LIST_SEARCH_INDEXES} pipeline stage. @@ -90,7 +89,7 @@ public BatchCursor execute(final ReadBinding binding) { if (!isNamespaceError(exception)) { throw exception; } else { - return createEmptyBatchCursor(namespace, decoder, exception.getServerAddress(), cursorBatchSize); + return SingleBatchCursor.createEmptyBatchCursor(exception.getServerAddress(), cursorBatchSize); } } } @@ -101,9 +100,8 @@ public void executeAsync(final AsyncReadBinding binding, final SingleResultCallb if (exception != null && !isNamespaceError(exception)) { callback.onResult(null, exception); } else if (exception != null) { - MongoCommandException commandException = (MongoCommandException) exception; - AsyncBatchCursor emptyAsyncBatchCursor = createEmptyAsyncBatchCursor(namespace, commandException.getServerAddress()); - callback.onResult(emptyAsyncBatchCursor, null); + int cursorBatchSize = batchSize == null ? 0 : batchSize; + callback.onResult(AsyncSingleBatchCursor.createEmptyBatchCursor(cursorBatchSize), null); } else { callback.onResult(cursor, null); } diff --git a/driver-core/src/main/com/mongodb/internal/operation/MapReduceInlineResultsAsyncCursor.java b/driver-core/src/main/com/mongodb/internal/operation/MapReduceInlineResultsAsyncCursor.java index 1da84755100..ebf331fe47b 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/MapReduceInlineResultsAsyncCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/MapReduceInlineResultsAsyncCursor.java @@ -16,18 +16,21 @@ package com.mongodb.internal.operation; -import com.mongodb.internal.connection.QueryResult; +import com.mongodb.internal.async.SingleResultCallback; + +import java.util.List; /** * Cursor representation of the results of an inline map-reduce operation. This allows users to iterate over the results that were returned * from the operation, and also provides access to the statistics returned in the results. */ -class MapReduceInlineResultsAsyncCursor extends AsyncSingleBatchQueryCursor implements MapReduceAsyncBatchCursor { +class MapReduceInlineResultsAsyncCursor implements MapReduceAsyncBatchCursor { + private final AsyncSingleBatchCursor delegate; private final MapReduceStatistics statistics; - MapReduceInlineResultsAsyncCursor(final QueryResult queryResult, final MapReduceStatistics statistics) { - super(queryResult); + MapReduceInlineResultsAsyncCursor(final AsyncSingleBatchCursor delegate, final MapReduceStatistics statistics) { + this.delegate = delegate; this.statistics = statistics; } @@ -35,4 +38,29 @@ class MapReduceInlineResultsAsyncCursor extends AsyncSingleBatchQueryCursor> callback) { + delegate.next(callback); + } + + @Override + public void setBatchSize(final int batchSize) { + delegate.setBatchSize(batchSize); + } + + @Override + public int getBatchSize() { + return delegate.getBatchSize(); + } + + @Override + public boolean isClosed() { + return delegate.isClosed(); + } + + @Override + public void close() { + delegate.close(); + } } diff --git a/driver-core/src/main/com/mongodb/internal/operation/MapReduceInlineResultsCursor.java b/driver-core/src/main/com/mongodb/internal/operation/MapReduceInlineResultsCursor.java index caa2f7fd355..564eac4a8f0 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/MapReduceInlineResultsCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/MapReduceInlineResultsCursor.java @@ -16,20 +16,21 @@ package com.mongodb.internal.operation; -import com.mongodb.internal.binding.ConnectionSource; -import com.mongodb.internal.connection.QueryResult; -import org.bson.codecs.Decoder; +import com.mongodb.ServerAddress; +import com.mongodb.ServerCursor; + +import java.util.List; /** * Cursor representation of the results of an inline map-reduce operation. This allows users to iterate over the results that were returned * from the operation, and also provides access to the statistics returned in the results. */ -class MapReduceInlineResultsCursor extends QueryBatchCursor implements MapReduceBatchCursor { +class MapReduceInlineResultsCursor implements MapReduceBatchCursor { + private final BatchCursor delegate; private final MapReduceStatistics statistics; - MapReduceInlineResultsCursor(final QueryResult queryResult, final Decoder decoder, final ConnectionSource connectionSource, - final MapReduceStatistics statistics) { - super(queryResult, 0, 0, decoder, null, connectionSource); + MapReduceInlineResultsCursor(final BatchCursor delegate, final MapReduceStatistics statistics) { + this.delegate = delegate; this.statistics = statistics; } @@ -37,4 +38,49 @@ class MapReduceInlineResultsCursor extends QueryBatchCursor implements Map public MapReduceStatistics getStatistics() { return statistics; } + + @Override + public boolean hasNext() { + return delegate.hasNext(); + } + + @Override + public List next() { + return delegate.next(); + } + + @Override + public int available() { + return delegate.available(); + } + + @Override + public void setBatchSize(final int batchSize) { + delegate.setBatchSize(batchSize); + } + + @Override + public int getBatchSize() { + return delegate.getBatchSize(); + } + + @Override + public List tryNext() { + return delegate.tryNext(); + } + + @Override + public ServerCursor getServerCursor() { + return delegate.getServerCursor(); + } + + @Override + public ServerAddress getServerAddress() { + return delegate.getServerAddress(); + } + + @Override + public void close() { + delegate.close(); + } } diff --git a/driver-core/src/main/com/mongodb/internal/operation/MapReduceWithInlineResultsOperation.java b/driver-core/src/main/com/mongodb/internal/operation/MapReduceWithInlineResultsOperation.java index 131591dd6e2..e51852db25c 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/MapReduceWithInlineResultsOperation.java +++ b/driver-core/src/main/com/mongodb/internal/operation/MapReduceWithInlineResultsOperation.java @@ -19,12 +19,10 @@ import com.mongodb.ExplainVerbosity; import com.mongodb.MongoNamespace; import com.mongodb.client.model.Collation; -import com.mongodb.connection.ConnectionDescription; import com.mongodb.internal.async.SingleResultCallback; import com.mongodb.internal.binding.AsyncReadBinding; import com.mongodb.internal.binding.ReadBinding; import com.mongodb.internal.connection.NoOpSessionContext; -import com.mongodb.internal.connection.QueryResult; import com.mongodb.internal.session.SessionContext; import com.mongodb.lang.Nullable; import org.bson.BsonDocument; @@ -215,13 +213,18 @@ private CommandReadOperation createExplainableOperation(final Expl } private CommandReadTransformer> transformer() { - return (result, source, connection) -> new MapReduceInlineResultsCursor<>(createQueryResult(result, connection.getDescription()), decoder, source, + return (result, source, connection) -> + new MapReduceInlineResultsCursor<>( + new SingleBatchCursor<>(BsonDocumentWrapperHelper.toList(result, "results"), 0, + connection.getDescription().getServerAddress()), MapReduceHelper.createStatistics(result)); } private CommandReadTransformerAsync> asyncTransformer() { - return (result, source, connection) -> new MapReduceInlineResultsAsyncCursor<>(createQueryResult(result, connection.getDescription()), - MapReduceHelper.createStatistics(result)); + return (result, source, connection) -> + new MapReduceInlineResultsAsyncCursor<>( + new AsyncSingleBatchCursor<>(BsonDocumentWrapperHelper.toList(result, "results"), 0), + MapReduceHelper.createStatistics(result)); } private CommandCreator getCommandCreator(final SessionContext sessionContext) { @@ -248,9 +251,4 @@ private BsonDocument getCommand(final SessionContext sessionContext, final int m } return commandDocument; } - - private QueryResult createQueryResult(final BsonDocument result, final ConnectionDescription description) { - return new QueryResult<>(namespace, BsonDocumentWrapperHelper.toList(result, "results"), 0, - description.getServerAddress()); - } } diff --git a/driver-core/src/main/com/mongodb/internal/operation/OperationHelper.java b/driver-core/src/main/com/mongodb/internal/operation/OperationHelper.java index 387bb2f5da6..bfa1adbd97e 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/OperationHelper.java +++ b/driver-core/src/main/com/mongodb/internal/operation/OperationHelper.java @@ -17,8 +17,6 @@ package com.mongodb.internal.operation; import com.mongodb.MongoClientException; -import com.mongodb.MongoNamespace; -import com.mongodb.ServerAddress; import com.mongodb.WriteConcern; import com.mongodb.client.model.Collation; import com.mongodb.connection.ConnectionDescription; @@ -30,18 +28,14 @@ import com.mongodb.internal.bulk.DeleteRequest; import com.mongodb.internal.bulk.UpdateRequest; import com.mongodb.internal.bulk.WriteRequest; -import com.mongodb.internal.connection.QueryResult; import com.mongodb.internal.diagnostics.logging.Logger; import com.mongodb.internal.diagnostics.logging.Loggers; import com.mongodb.internal.session.SessionContext; import com.mongodb.lang.NonNull; import com.mongodb.lang.Nullable; import org.bson.BsonDocument; -import org.bson.BsonInt64; -import org.bson.codecs.Decoder; import org.bson.conversions.Bson; -import java.util.Collections; import java.util.List; import java.util.function.Function; import java.util.function.Supplier; @@ -200,26 +194,6 @@ static boolean canRetryRead(final ServerDescription serverDescription, final Ses return true; } - static QueryBatchCursor createEmptyBatchCursor(final MongoNamespace namespace, final Decoder decoder, - final ServerAddress serverAddress, final int batchSize) { - return new QueryBatchCursor<>(new QueryResult<>(namespace, Collections.emptyList(), 0L, - serverAddress), - 0, batchSize, decoder); - } - - static QueryResult cursorDocumentToQueryResult(final BsonDocument cursorDocument, final ServerAddress serverAddress) { - return cursorDocumentToQueryResult(cursorDocument, serverAddress, "firstBatch"); - } - - static QueryResult cursorDocumentToQueryResult(final BsonDocument cursorDocument, final ServerAddress serverAddress, - final String fieldNameContainingBatch) { - long cursorId = ((BsonInt64) cursorDocument.get("id")).getValue(); - MongoNamespace queryResultNamespace = new MongoNamespace(cursorDocument.getString("ns").getValue()); - return new QueryResult<>(queryResultNamespace, BsonDocumentWrapperHelper.toList(cursorDocument, fieldNameContainingBatch), - cursorId, serverAddress); - } - - /** * This internal exception is used to *
    diff --git a/driver-core/src/main/com/mongodb/internal/operation/QueryHelper.java b/driver-core/src/main/com/mongodb/internal/operation/QueryHelper.java deleted file mode 100644 index 053e4fc8817..00000000000 --- a/driver-core/src/main/com/mongodb/internal/operation/QueryHelper.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Copyright 2008-present MongoDB, 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.mongodb.internal.operation; - -import com.mongodb.MongoCommandException; -import com.mongodb.MongoCursorNotFoundException; -import com.mongodb.MongoQueryException; -import com.mongodb.ServerCursor; - -final class QueryHelper { - static MongoQueryException translateCommandException(final MongoCommandException commandException, final ServerCursor cursor) { - if (commandException.getErrorCode() == 43) { - return new MongoCursorNotFoundException(cursor.getId(), commandException.getResponse(), cursor.getAddress()); - } else { - return new MongoQueryException(commandException.getResponse(), commandException.getServerAddress()); - } - } - - private QueryHelper() { - } -} diff --git a/driver-core/src/main/com/mongodb/internal/operation/SingleBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/SingleBatchCursor.java new file mode 100644 index 00000000000..6d734892fb3 --- /dev/null +++ b/driver-core/src/main/com/mongodb/internal/operation/SingleBatchCursor.java @@ -0,0 +1,95 @@ +/* + * Copyright 2008-present MongoDB, 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.mongodb.internal.operation; + +import com.mongodb.ServerAddress; +import com.mongodb.ServerCursor; + +import java.util.List; +import java.util.NoSuchElementException; + +import static java.util.Collections.emptyList; + +class SingleBatchCursor implements BatchCursor { + + static SingleBatchCursor createEmptyBatchCursor(final ServerAddress serverAddress, final int batchSize) { + return new SingleBatchCursor<>(emptyList(), batchSize, serverAddress); + } + + private final List batch; + private final ServerAddress serverAddress; + private final int batchSize; + private boolean hasNext; + + SingleBatchCursor(final List batch, final int batchSize, final ServerAddress serverAddress) { + this.batch = batch; + this.serverAddress = serverAddress; + this.batchSize = batchSize; + this.hasNext = !batch.isEmpty(); + } + + public List getBatch() { + return batch; + } + + @Override + public boolean hasNext() { + return hasNext; + } + + @Override + public List next() { + if (hasNext) { + hasNext = false; + return batch; + } + throw new NoSuchElementException(); + } + + @Override + public int available() { + return hasNext ? 1 : 0; + } + + @Override + public void setBatchSize(final int batchSize) { + // NOOP + } + + @Override + public int getBatchSize() { + return batchSize; + } + + @Override + public List tryNext() { + return hasNext ? next() : null; + } + + @Override + public ServerCursor getServerCursor() { + return null; + } + + @Override + public ServerAddress getServerAddress() { + return serverAddress; + } + + @Override + public void close() { + } +} diff --git a/driver-core/src/main/com/mongodb/internal/operation/SyncOperationHelper.java b/driver-core/src/main/com/mongodb/internal/operation/SyncOperationHelper.java index 67d5acf9c37..476a2676b8e 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/SyncOperationHelper.java +++ b/driver-core/src/main/com/mongodb/internal/operation/SyncOperationHelper.java @@ -18,7 +18,6 @@ import com.mongodb.MongoException; import com.mongodb.ReadPreference; -import com.mongodb.ServerAddress; import com.mongodb.internal.VisibleForTesting; import com.mongodb.internal.async.SingleResultCallback; import com.mongodb.internal.async.function.AsyncCallbackBiFunction; @@ -32,7 +31,6 @@ import com.mongodb.internal.binding.WriteBinding; import com.mongodb.internal.connection.Connection; import com.mongodb.internal.connection.OperationContext; -import com.mongodb.internal.connection.QueryResult; import com.mongodb.internal.operation.retry.AttachmentKeys; import com.mongodb.internal.validator.NoOpFieldNameValidator; import com.mongodb.lang.Nullable; @@ -56,7 +54,6 @@ import static com.mongodb.internal.operation.OperationHelper.ResourceSupplierInternalException; import static com.mongodb.internal.operation.OperationHelper.canRetryRead; import static com.mongodb.internal.operation.OperationHelper.canRetryWrite; -import static com.mongodb.internal.operation.OperationHelper.cursorDocumentToQueryResult; import static com.mongodb.internal.operation.WriteConcernHelper.throwOnWriteConcernError; final class SyncOperationHelper { @@ -303,14 +300,16 @@ static CommandWriteTransformer writeConcernErrorTransformer( }; } - static BatchCursor cursorDocumentToBatchCursor(final BsonDocument cursorDocument, final Decoder decoder, - final BsonValue comment, final ConnectionSource source, final Connection connection, final int batchSize) { - return new QueryBatchCursor<>(cursorDocumentToQueryResult(cursorDocument, source.getServerDescription().getAddress()), - 0, batchSize, 0, decoder, comment, source, connection); + static CommandReadTransformer> singleBatchCursorTransformer(final String resultsFieldName) { + return (result, source, connection) -> + new SingleBatchCursor<>(BsonDocumentWrapperHelper.toList(result, resultsFieldName), 0, + connection.getDescription().getServerAddress()); } - static QueryResult getMoreCursorDocumentToQueryResult(final BsonDocument cursorDocument, final ServerAddress serverAddress) { - return cursorDocumentToQueryResult(cursorDocument, serverAddress, "nextBatch"); + static BatchCursor cursorDocumentToBatchCursor(final BsonDocument cursorDocument, final Decoder decoder, + final BsonValue comment, final ConnectionSource source, final Connection connection, final int batchSize) { + return new CommandBatchCursor<>(connection.getDescription().getServerAddress(), cursorDocument, 0, batchSize, + 0, decoder, comment, source, connection); } private SyncOperationHelper() { diff --git a/driver-core/src/test/functional/com/mongodb/OperationFunctionalSpecification.groovy b/driver-core/src/test/functional/com/mongodb/OperationFunctionalSpecification.groovy index ddbb9f29a0d..372fdd4b82d 100644 --- a/driver-core/src/test/functional/com/mongodb/OperationFunctionalSpecification.groovy +++ b/driver-core/src/test/functional/com/mongodb/OperationFunctionalSpecification.groovy @@ -202,13 +202,6 @@ class OperationFunctionalSpecification extends Specification { } } - def consumeAsyncResults(cursor) { - def batch = next(cursor, true) - while (batch != null) { - batch = next(cursor, true) - } - } - void testOperation(Map params) { params.async = params.async != null ? params.async : false params.result = params.result != null ? params.result : null diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncQueryBatchCursorFunctionalSpecification.groovy b/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorAltFunctionalSpecification.groovy similarity index 68% rename from driver-core/src/test/functional/com/mongodb/internal/operation/AsyncQueryBatchCursorFunctionalSpecification.groovy rename to driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorAltFunctionalSpecification.groovy index 3d6f0c8b7a7..c3973a59761 100644 --- a/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncQueryBatchCursorFunctionalSpecification.groovy +++ b/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorAltFunctionalSpecification.groovy @@ -21,16 +21,15 @@ import com.mongodb.MongoException import com.mongodb.MongoTimeoutException import com.mongodb.OperationFunctionalSpecification import com.mongodb.ReadPreference +import com.mongodb.ServerAddress import com.mongodb.ServerCursor import com.mongodb.WriteConcern import com.mongodb.async.FutureResultCallback import com.mongodb.client.model.CreateCollectionOptions import com.mongodb.client.syncadapter.SyncConnection import com.mongodb.internal.binding.AsyncConnectionSource -import com.mongodb.internal.binding.AsyncReadBinding +import com.mongodb.internal.binding.AsyncReadWriteBinding import com.mongodb.internal.connection.AsyncConnection -import com.mongodb.internal.connection.Connection -import com.mongodb.internal.connection.QueryResult import com.mongodb.internal.validator.NoOpFieldNameValidator import org.bson.BsonArray import org.bson.BsonBoolean @@ -48,6 +47,7 @@ import util.spock.annotations.Slow import java.util.concurrent.CountDownLatch +import static com.mongodb.ClusterFixture.checkReferenceCountReachesTarget import static com.mongodb.ClusterFixture.getAsyncBinding import static com.mongodb.ClusterFixture.getAsyncCluster import static com.mongodb.ClusterFixture.getBinding @@ -59,7 +59,6 @@ import static com.mongodb.ClusterFixture.isSharded import static com.mongodb.ClusterFixture.serverVersionLessThan import static com.mongodb.internal.connection.ServerHelper.waitForLastRelease import static com.mongodb.internal.connection.ServerHelper.waitForRelease -import static com.mongodb.internal.operation.OperationHelper.cursorDocumentToQueryResult import static com.mongodb.internal.operation.QueryOperationHelper.makeAdditionalGetMoreCall import static java.util.Collections.singletonList import static java.util.concurrent.TimeUnit.SECONDS @@ -67,9 +66,9 @@ import static org.junit.Assert.assertEquals import static org.junit.Assert.fail @IgnoreIf({ isSharded() && serverVersionLessThan(3, 2) }) -class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecification { +class AsyncCommandBatchCursorAltFunctionalSpecification extends OperationFunctionalSpecification { AsyncConnectionSource connectionSource - AsyncQueryBatchCursor cursor + AsyncCommandBatchCursor cursor AsyncConnection connection def setup() { @@ -83,7 +82,7 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp setUpConnectionAndSource(getAsyncBinding()) } - private void setUpConnectionAndSource(final AsyncReadBinding binding) { + private void setUpConnectionAndSource(final AsyncReadWriteBinding binding) { connectionSource = getReadConnectionSource(binding) connection = getConnection(connectionSource) } @@ -102,7 +101,9 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp def 'should exhaust single batch'() { given: - cursor = new AsyncQueryBatchCursor(executeQuery(), 0, 0, 0, new DocumentCodec(), null, connectionSource, connection) + def (serverAddress, commandResult) = executeQuery() + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), + null, connectionSource, connection) expect: nextBatch().size() == 10 @@ -110,19 +111,23 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp def 'should not retain connection and source after cursor is exhausted on first batch'() { given: - cursor = new AsyncQueryBatchCursor(executeQuery(), 0, 0, 0, new DocumentCodec(), null, connectionSource, connection) + def (serverAddress, commandResult) = executeQuery() + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), + null, connectionSource, connection) when: nextBatch() then: - connection.count == 1 - connectionSource.count == 1 + getReferenceCountAfterTimeout(connection, 1) == 1 + getReferenceCountAfterTimeout(connectionSource, 1) == 1 } def 'should not retain connection and source after cursor is exhausted on getMore'() { given: - cursor = new AsyncQueryBatchCursor(executeQuery(1, 0), 1, 1, 0, new DocumentCodec(), null, connectionSource, connection) + def (serverAddress, commandResult) = executeQuery(1, 0) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 1, 1, 0, new DocumentCodec(), + null, connectionSource, connection) when: nextBatch() @@ -134,7 +139,9 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp def 'should not retain connection and source after cursor is exhausted after first batch'() { when: - cursor = new AsyncQueryBatchCursor(executeQuery(10, 10), 10, 10, 0, new DocumentCodec(), null, connectionSource, + def (serverAddress, commandResult) = executeQuery(10, 10) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 10, 10, 0, new DocumentCodec(), + null, connectionSource, connection) then: @@ -144,7 +151,9 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp def 'should exhaust single batch with limit'() { given: - cursor = new AsyncQueryBatchCursor(executeQuery(1, 0), 1, 0, 0, new DocumentCodec(), null, connectionSource, connection) + def (serverAddress, commandResult) = executeQuery(1, 0) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 1, 0, 0, new DocumentCodec(), + null, connectionSource, connection) expect: nextBatch().size() == 1 @@ -153,8 +162,9 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp def 'should exhaust multiple batches with limit'() { given: - cursor = new AsyncQueryBatchCursor(executeQuery(limit, batchSize), limit, batchSize, 0, new DocumentCodec(), null, - connectionSource, connection) + def (serverAddress, commandResult) = executeQuery(limit, batchSize) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, limit, batchSize, 0, new DocumentCodec(), + null, connectionSource, connection) when: def next = nextBatch() @@ -184,7 +194,9 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp def 'should exhaust multiple batches'() { given: - cursor = new AsyncQueryBatchCursor(executeQuery(3), 0, 2, 0, new DocumentCodec(), null, connectionSource, connection) + def (serverAddress, commandResult) = executeQuery(3) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) expect: nextBatch().size() == 3 @@ -197,7 +209,9 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp def 'should respect batch size'() { when: - cursor = new AsyncQueryBatchCursor(executeQuery(3), 0, 2, 0, new DocumentCodec(), null, connectionSource, connection) + def (serverAddress, commandResult) = executeQuery(3) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) then: cursor.batchSize == 2 @@ -212,7 +226,9 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp def 'should close when exhausted'() { given: - cursor = new AsyncQueryBatchCursor(executeQuery(), 0, 2, 0, new DocumentCodec(), null, connectionSource, connection) + def (serverAddress, commandResult) = executeQuery() + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) when: cursor.close() @@ -230,7 +246,9 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp def 'should close when not exhausted'() { given: - cursor = new AsyncQueryBatchCursor(executeQuery(3), 0, 2, 0, new DocumentCodec(), null, connectionSource, connection) + def (serverAddress, commandResult) = executeQuery(3) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) when: cursor.close() @@ -244,10 +262,13 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp given: collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(4, 0))) - def firstBatch = executeQuery(new BsonDocument('ts', new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, false) + + def (serverAddress, commandResult) = executeQuery(new BsonDocument('ts', + new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, false) when: - cursor = new AsyncQueryBatchCursor(firstBatch, 0, 2, 0, new DocumentCodec(), null, connectionSource, connection) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) def latch = new CountDownLatch(1) Thread.start { sleep(500) @@ -272,11 +293,12 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp def 'should block waiting for next batch on a tailable cursor'() { collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(5, 0))) - def firstBatch = executeQuery(new BsonDocument('ts', new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, awaitData) - + def (serverAddress, commandResult) = executeQuery(new BsonDocument('ts', + new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, awaitData) when: - cursor = new AsyncQueryBatchCursor(firstBatch, 0, 2, maxTimeMS, new DocumentCodec(), null, connectionSource, connection) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, maxTimeMS, new DocumentCodec(), + null, connectionSource, connection) def batch = nextBatch() then: @@ -315,10 +337,11 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp given: collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) collectionHelper.insertDocuments(new DocumentCodec(), Document.parse('{}')) - def firstBatch = executeQuery(new BsonDocument('_id', BsonNull.VALUE), 0, 1, true, true) + def (serverAddress, commandResult) = executeQuery(new BsonDocument('_id', BsonNull.VALUE), 0, 1, true, true) when: - cursor = new AsyncQueryBatchCursor(firstBatch, 0, 1, 500, new DocumentCodec(), null, connectionSource, connection) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 1, 500, new DocumentCodec(), + null, connectionSource, connection) Thread.start { Thread.sleep(SECONDS.toMillis(2)) cursor.close() @@ -328,14 +351,16 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp then: cursor.isClosed() batch == null - //both connection and connectionSource have reference count 1 when we pass them to the AsyncQueryBatchCursor constructor + //both connection and connectionSource have reference count 1 when we pass them to the AsyncCommandBatchCursor constructor connection.getCount() == 1 waitForRelease(connectionSource, 1) } def 'should respect limit'() { given: - cursor = new AsyncQueryBatchCursor(executeQuery(6, 3), 6, 2, 0, new DocumentCodec(), null, connectionSource, connection) + def (serverAddress, commandResult) = executeQuery(6, 3) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 6, 2, 0, new DocumentCodec(), + null, connectionSource, connection) expect: nextBatch().size() == 3 @@ -347,15 +372,14 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp @IgnoreIf({ isSharded() }) def 'should kill cursor if limit is reached on initial query'() throws InterruptedException { given: - def firstBatch = executeQuery(5) - - cursor = new AsyncQueryBatchCursor(firstBatch, 5, 0, 0, new DocumentCodec(), null, connectionSource, connection) + def (serverAddress, commandResult) = executeQuery(5) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 5, 0, 0, new DocumentCodec(), + null, connectionSource, connection) + def serverCursor = new ServerCursor(commandResult.getDocument("cursor").getInt64("id").value, serverAddress) when: - while (connection.getCount() > 1) { - Thread.sleep(5) - } - makeAdditionalGetMoreCall(getNamespace(), firstBatch.cursor, new SyncConnection(connection)) + checkReferenceCountReachesTarget(connection, 1) + makeAdditionalGetMoreCall(getNamespace(), serverCursor, new SyncConnection(connection)) then: thrown(MongoCursorNotFoundException) @@ -365,19 +389,19 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp @IgnoreIf({ isSharded() }) def 'should throw cursor not found exception'() { given: - def firstBatch = executeQuery(2) + def (serverAddress, commandResult) = executeQuery(2) when: - cursor = new AsyncQueryBatchCursor(firstBatch, 0, 2, 0, new DocumentCodec(), null, connectionSource, connection) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) def connection = new SyncConnection(getConnection(connectionSource)) - def serverCursor = cursor.cursor.get() + def serverCursor = cursor.getServerCursor() connection.command(getNamespace().databaseName, new BsonDocument('killCursors', new BsonString(namespace.getCollectionName())) .append('cursors', new BsonArray(singletonList(new BsonInt64(serverCursor.getId())))), new NoOpFieldNameValidator(), ReadPreference.primary(), - new BsonDocumentCodec() - , connectionSource) + new BsonDocumentCodec(), connectionSource) connection.release() nextBatch() @@ -399,47 +423,39 @@ class AsyncQueryBatchCursorFunctionalSpecification extends OperationFunctionalSp futureResultCallback.get() } - private QueryResult executeQuery() { + private Tuple2 executeQuery() { executeQuery(0) } - private QueryResult executeQuery(int batchSize) { + private Tuple2 executeQuery(int batchSize) { executeQuery(0, batchSize) } - private QueryResult executeQuery(int limit, int batchSize) { + private Tuple2 executeQuery(int limit, int batchSize) { executeQuery(new BsonDocument(), limit, batchSize, false, false) } - private QueryResult executeQuery(BsonDocument filter, int limit, int batchSize, boolean tailable, boolean awaitData) { + private Tuple2 executeQuery(BsonDocument filter, int limit, int batchSize, boolean tailable, + boolean awaitData) { def findCommand = new BsonDocument('find', new BsonString(getCollectionName())) .append('filter', filter) .append('tailable', BsonBoolean.valueOf(tailable)) .append('awaitData', BsonBoolean.valueOf(awaitData)) - findCommand.append('limit', new BsonInt32(Math.abs(limit))) + findCommand.append('limit', new BsonInt32(Math.abs(limit))) if (limit >= 0) { if (batchSize < 0 && Math.abs(batchSize) < limit) { findCommand.append('limit', new BsonInt32(Math.abs(batchSize))) - } else { + } else if (batchSize != 0) { findCommand.append('batchSize', new BsonInt32(Math.abs(batchSize))) - } + } } def futureResultCallback = new FutureResultCallback() connection.commandAsync(getDatabaseName(), findCommand, NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), CommandResultDocumentCodec.create(new DocumentCodec(), 'firstBatch'), connectionSource, futureResultCallback) - def response = futureResultCallback.get() - cursorDocumentToQueryResult(response.getDocument('cursor'), connection.getDescription().getServerAddress()) - } - - private void makeAdditionalGetMoreCall(ServerCursor serverCursor, Connection connection) { - connection.command(getNamespace().databaseName, - new BsonDocument('getMore', new BsonInt64(serverCursor.getId())) - .append('collection', new BsonString(namespace.getCollectionName())), - NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), new BsonDocumentCodec(), connectionSource.getSessionContext(), - connectionSource.getServerApi()) + new Tuple2(connection.getDescription().getServerAddress(), futureResultCallback.get()) } } diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalSpecification.groovy b/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalSpecification.groovy new file mode 100644 index 00000000000..87415983e0d --- /dev/null +++ b/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalSpecification.groovy @@ -0,0 +1,496 @@ +/* + * Copyright 2008-present MongoDB, 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.mongodb.internal.operation + +import com.mongodb.MongoCursorNotFoundException +import com.mongodb.MongoException +import com.mongodb.MongoQueryException +import com.mongodb.MongoTimeoutException +import com.mongodb.OperationFunctionalSpecification +import com.mongodb.ReadPreference +import com.mongodb.ServerAddress +import com.mongodb.ServerCursor +import com.mongodb.WriteConcern +import com.mongodb.async.FutureResultCallback +import com.mongodb.client.model.CreateCollectionOptions +import com.mongodb.internal.binding.AsyncConnectionSource +import com.mongodb.internal.binding.AsyncReadWriteBinding +import com.mongodb.internal.connection.AsyncConnection +import com.mongodb.internal.validator.NoOpFieldNameValidator +import org.bson.BsonArray +import org.bson.BsonBoolean +import org.bson.BsonDocument +import org.bson.BsonInt32 +import org.bson.BsonInt64 +import org.bson.BsonString +import org.bson.BsonTimestamp +import org.bson.Document +import org.bson.codecs.BsonDocumentCodec +import org.bson.codecs.DocumentCodec +import spock.lang.IgnoreIf +import util.spock.annotations.Slow + +import java.util.concurrent.CountDownLatch +import java.util.concurrent.TimeUnit + +import static com.mongodb.ClusterFixture.checkReferenceCountReachesTarget +import static com.mongodb.ClusterFixture.getAsyncBinding +import static com.mongodb.ClusterFixture.getBinding +import static com.mongodb.ClusterFixture.getConnection +import static com.mongodb.ClusterFixture.getReferenceCountAfterTimeout +import static com.mongodb.ClusterFixture.getWriteConnectionSource +import static com.mongodb.ClusterFixture.isDiscoverableReplicaSet +import static com.mongodb.ClusterFixture.isSharded +import static com.mongodb.ClusterFixture.isStandalone +import static com.mongodb.internal.operation.QueryOperationHelper.makeAdditionalGetMoreCall +import static java.util.Collections.singletonList +import static org.junit.Assert.assertEquals +import static org.junit.Assert.fail + +class AsyncCommandBatchCursorFunctionalSpecification extends OperationFunctionalSpecification { + AsyncConnectionSource connectionSource + AsyncCommandBatchCursor cursor + AsyncConnection connection + + def setup() { + def documents = [] + for (int i = 0; i < 10; i++) { + documents.add(new BsonDocument('_id', new BsonInt32(i))) + } + collectionHelper.insertDocuments(documents, + isDiscoverableReplicaSet() ? WriteConcern.MAJORITY : WriteConcern.ACKNOWLEDGED, + getBinding()) + setUpConnectionAndSource(getAsyncBinding()) + } + + private void setUpConnectionAndSource(final AsyncReadWriteBinding binding) { + connectionSource = getWriteConnectionSource(binding) + connection = getConnection(connectionSource) + } + + def cleanup() { + cursor?.close() + getReferenceCountAfterTimeout(connectionSource, 1) + getReferenceCountAfterTimeout(connection, 1) + cleanupConnectionAndSource() + } + + private void cleanupConnectionAndSource() { + connection?.release() + connectionSource?.release() + } + + def 'server cursor should not be null'() { + given: + def (serverAddress, commandResult) = executeFindCommand(2) + + when: + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), + null, connectionSource, connection) + + then: + cursor.getServerCursor() != null + } + + + def 'should get Exceptions for operations on the cursor after closing'() { + given: + def (serverAddress, commandResult) = executeFindCommand() + + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), + null, connectionSource, connection) + + when: + cursor.close() + cursor.close() + + and: + nextBatch() + + then: + thrown(MongoException) + + and: + def serverCursor = cursor.getServerCursor() + + then: + serverCursor == null + } + + def 'should throw an Exception when going off the end'() { + given: + def (serverAddress, commandResult) = executeFindCommand(1) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 2, 0, 0, new DocumentCodec(), + null, connectionSource, connection) + + when: + nextBatch() + nextBatch() + + then: + nextBatch().isEmpty() + + when: + nextBatch() + + then: + thrown(MongoException) + } + + def 'test normal exhaustion'() { + given: + def (serverAddress, commandResult) = executeFindCommand() + + when: + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), + null, connectionSource, connection) + + then: + nextBatch().size() == 10 + } + + def 'test limit exhaustion'() { + given: + def (serverAddress, commandResult) = executeFindCommand(limit, batchSize) + + when: + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, limit, batchSize, 0, new DocumentCodec(), + null, connectionSource, connection) + def batch = nextBatch() + def counter = batch.size() + while (!cursor.isClosed()) { + batch = nextBatch() + counter += batch == null ? 0 : batch.size() + } + + then: + counter == expectedTotal + + where: + limit | batchSize | expectedTotal + 5 | 2 | 5 + 5 | -2 | 2 + -5 | 2 | 5 + -5 | -2 | 5 + 2 | 5 | 2 + 2 | -5 | 2 + -2 | 5 | 2 + -2 | -5 | 2 + } + + @SuppressWarnings('EmptyCatchBlock') + def 'should block waiting for next batch on a tailable cursor'() { + given: + collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) + collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(5, 0))) + def (serverAddress, commandResult) = executeFindCommand(new BsonDocument('ts', + new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, awaitData) + + when: + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, maxTimeMS, new DocumentCodec(), + null, connectionSource, connection) + + then: + nextBatch().iterator().next().get('_id') == 1 + + when: + def latch = new CountDownLatch(1) + Thread.start { + try { + sleep(500) + collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 2).append('ts', new BsonTimestamp(6, 0))) + } catch (ignored) { + //pass + } finally { + latch.countDown() + } + } + + then: + nextBatch().iterator().next().get('_id') == 2 + + cleanup: + def cleanedUp = latch.await(10, TimeUnit.SECONDS) + if (!cleanedUp) { + throw new MongoTimeoutException('Timed out waiting for documents to be inserted') + } + + where: + awaitData | maxTimeMS + true | 0 + true | 100 + false | 0 + } + + + @SuppressWarnings('EmptyCatchBlock') + @Slow + def 'test tailable interrupt'() throws InterruptedException { + collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) + collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1)) + + def (serverAddress, commandResult) = executeFindCommand(new BsonDocument(), 0, 2, true, true) + + when: + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) + + CountDownLatch latch = new CountDownLatch(1) + def seen = 0 + def thread = Thread.start { + try { + nextBatch() + seen = 1 + nextBatch() + seen = 2 + } catch (ignored) { + // pass + } finally { + latch.countDown() + } + } + sleep(1000) + thread.interrupt() + collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 2)) + latch.await() + + then: + seen == 1 + } + + @IgnoreIf({ isSharded() }) + def 'should kill cursor if limit is reached on initial query'() throws InterruptedException { + given: + def (serverAddress, commandResult) = executeFindCommand(5) + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 5, 0, 0, new DocumentCodec(), + null, connectionSource, connection) + + when: + nextBatch() + + then: + cursor.isClosed() + cursor.getServerCursor() == null + } + + @IgnoreIf({ !isStandalone() }) + def 'should kill cursor if limit is reached on get more'() throws InterruptedException { + given: + def (serverAddress, commandResult) = executeFindCommand(3) + + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 5, 3, 0, new DocumentCodec(), + null, connectionSource, connection) + ServerCursor serverCursor = cursor.getServerCursor() + + nextBatch() + nextBatch() + + checkReferenceCountReachesTarget(connection, 1) + + when: + makeAdditionalGetMoreCall(getNamespace(), serverCursor, connection) + + then: + thrown(MongoQueryException) + } + + def 'should release connection source if limit is reached on initial query'() throws InterruptedException { + given: + def (serverAddress, commandResult) = executeFindCommand(5) + + when: + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 5, 0, 0, new DocumentCodec(), + null, connectionSource, connection) + + then: + checkReferenceCountReachesTarget(connectionSource, 1) + checkReferenceCountReachesTarget(connection, 1) + } + + def 'should release connection source if limit is reached on get more'() throws InterruptedException { + given: + def (serverAddress, commandResult) = executeFindCommand(3) + + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 5, 3, 0, new DocumentCodec(), + null, connectionSource, connection) + + when: + nextBatch() + nextBatch() + + then: + checkReferenceCountReachesTarget(connectionSource, 1) + checkReferenceCountReachesTarget(connection, 1) + } + + def 'test limit with get more'() { + given: + def (serverAddress, commandResult) = executeFindCommand(2) + + when: + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 5, 2, 0, new DocumentCodec(), + null, connectionSource, connection) + + then: + !nextBatch().isEmpty() + !nextBatch().isEmpty() + !nextBatch().isEmpty() + nextBatch().isEmpty() + } + + @Slow + def 'test limit with large documents'() { + given: + String bigString = new String('x' * 16000) + + (11..1000).each { collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', it).append('s', bigString)) } + def (serverAddress, commandResult) = executeFindCommand(300, 0) + + when: + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 300, 0, 0, new DocumentCodec(), + null, connectionSource, connection) + + def batch = nextBatch() + def counter = batch.size() + while (!cursor.isClosed()) { + batch = nextBatch() + counter += batch == null ? 0 : batch.size() + } + + then: + counter == 300 + } + + def 'should respect batch size'() { + given: + def (serverAddress, commandResult) = executeFindCommand(2) + + when: + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) + + then: + cursor.batchSize == 2 + + nextBatch().size() == 2 + nextBatch().size() == 2 + + when: + cursor.batchSize = 3 + + then: + cursor.batchSize == 3 + nextBatch().size() == 3 + nextBatch().size() == 3 + } + + def 'test normal loop with get more'() { + given: + def (serverAddress, commandResult) = executeFindCommand(2) + + when: + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) + def batch = nextBatch() + def counter = batch.size() + while (!cursor.isClosed()) { + batch = nextBatch() + counter += batch == null ? 0 : batch.size() + } + + then: + counter == 10 + } + + + @SuppressWarnings('BracesForTryCatchFinally') + @IgnoreIf({ isSharded() }) + def 'should throw cursor not found exception'() { + given: + def (serverAddress, commandResult) = executeFindCommand(2) + + when: + cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) + def serverCursor = cursor.getServerCursor() + + def callback = new FutureResultCallback<>() + connection.commandAsync(getNamespace().databaseName, + new BsonDocument('killCursors', new BsonString(namespace.getCollectionName())) + .append('cursors', new BsonArray(singletonList(new BsonInt64(serverCursor.getId())))), + new NoOpFieldNameValidator(), ReadPreference.primary(), new BsonDocumentCodec(), connectionSource, callback) + callback.get() + nextBatch() + + then: + try { + nextBatch() + } catch (MongoCursorNotFoundException e) { + assertEquals(serverCursor.getId(), e.getCursorId()) + assertEquals(serverCursor.getAddress(), e.getServerAddress()) + } catch (ignored) { + fail('Expected MongoCursorNotFoundException to be thrown but got ' + ignored.getClass()) + } + } + + List nextBatch() { + def futureResultCallback = new FutureResultCallback() + cursor.next(futureResultCallback) + futureResultCallback.get() + } + + private Tuple2 executeFindCommand() { + executeFindCommand(0) + } + + private Tuple2 executeFindCommand(int batchSize) { + executeFindCommand(new BsonDocument(), 0, batchSize, false, false, ReadPreference.primary()) + } + + private Tuple2 executeFindCommand(int limit, int batchSize) { + executeFindCommand(new BsonDocument(), limit, batchSize, false, false, ReadPreference.primary()) + } + + private Tuple2 executeFindCommand(BsonDocument filter, int limit, int batchSize, boolean tailable, + boolean awaitData) { + executeFindCommand(filter, limit, batchSize, tailable, awaitData, ReadPreference.primary()) + } + + private Tuple2 executeFindCommand(BsonDocument filter, int limit, int batchSize, boolean tailable, + boolean awaitData, ReadPreference readPreference) { + def findCommand = new BsonDocument('find', new BsonString(getCollectionName())) + .append('filter', filter) + .append('tailable', BsonBoolean.valueOf(tailable)) + .append('awaitData', BsonBoolean.valueOf(awaitData)) + + findCommand.append('limit', new BsonInt32(Math.abs(limit))) + + if (limit >= 0) { + if (batchSize < 0 && Math.abs(batchSize) < limit) { + findCommand.append('limit', new BsonInt32(Math.abs(batchSize))) + } else if (batchSize != 0) { + findCommand.append('batchSize', new BsonInt32(Math.abs(batchSize))) + } + } + + def callback = new FutureResultCallback() + connection.commandAsync(getDatabaseName(), findCommand, + NO_OP_FIELD_NAME_VALIDATOR, readPreference, + CommandResultDocumentCodec.create(new DocumentCodec(), 'firstBatch'), connectionSource, callback) + new Tuple2(connection.getDescription().getServerAddress(), callback.get()) + } +} diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/QueryBatchCursorFunctionalSpecification.groovy b/driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalSpecification.groovy similarity index 60% rename from driver-core/src/test/functional/com/mongodb/internal/operation/QueryBatchCursorFunctionalSpecification.groovy rename to driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalSpecification.groovy index 9c77bb41b89..89e96a06f22 100644 --- a/driver-core/src/test/functional/com/mongodb/internal/operation/QueryBatchCursorFunctionalSpecification.groovy +++ b/driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalSpecification.groovy @@ -17,15 +17,17 @@ package com.mongodb.internal.operation import com.mongodb.MongoCursorNotFoundException +import com.mongodb.MongoQueryException import com.mongodb.MongoTimeoutException import com.mongodb.OperationFunctionalSpecification import com.mongodb.ReadPreference +import com.mongodb.ServerAddress import com.mongodb.ServerCursor import com.mongodb.WriteConcern import com.mongodb.client.model.CreateCollectionOptions import com.mongodb.internal.binding.ConnectionSource +import com.mongodb.internal.binding.ReadWriteBinding import com.mongodb.internal.connection.Connection -import com.mongodb.internal.connection.QueryResult import com.mongodb.internal.validator.NoOpFieldNameValidator import org.bson.BsonArray import org.bson.BsonBoolean @@ -48,15 +50,15 @@ import static com.mongodb.ClusterFixture.getBinding import static com.mongodb.ClusterFixture.isDiscoverableReplicaSet import static com.mongodb.ClusterFixture.isSharded import static com.mongodb.ClusterFixture.serverVersionLessThan -import static com.mongodb.internal.operation.OperationHelper.cursorDocumentToQueryResult import static com.mongodb.internal.operation.QueryOperationHelper.makeAdditionalGetMoreCall import static java.util.Collections.singletonList import static org.junit.Assert.assertEquals import static org.junit.Assert.fail -class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecification { +class CommandBatchCursorFunctionalSpecification extends OperationFunctionalSpecification { ConnectionSource connectionSource - QueryBatchCursor cursor + Connection connection + CommandBatchCursor cursor def setup() { def documents = [] @@ -64,22 +66,33 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi documents.add(new BsonDocument('_id', new BsonInt32(i))) } collectionHelper.insertDocuments(documents, - isDiscoverableReplicaSet() ? WriteConcern.MAJORITY : WriteConcern.ACKNOWLEDGED, - getBinding()) - connectionSource = getBinding().getReadConnectionSource() + isDiscoverableReplicaSet() ? WriteConcern.MAJORITY : WriteConcern.ACKNOWLEDGED, + getBinding()) + setUpConnectionAndSource(getBinding()) + } + + private void setUpConnectionAndSource(final ReadWriteBinding binding) { + connectionSource = binding.getWriteConnectionSource() + connection = connectionSource.getConnection() } def cleanup() { cursor?.close() + cleanupConnectionAndSource() + } + + private void cleanupConnectionAndSource() { + connection?.release() connectionSource?.release() } def 'server cursor should not be null'() { given: - def firstBatch = executeQuery(2) + def (serverAddress, commandResult) = executeFindCommand(2) when: - cursor = new QueryBatchCursor(firstBatch, 0, 0, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), + null, connectionSource, connection) then: cursor.getServerCursor() != null @@ -87,19 +100,21 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi def 'test server address'() { given: - def firstBatch = executeQuery() + def (serverAddress, commandResult) = executeFindCommand() when: - cursor = new QueryBatchCursor(firstBatch, 0, 0, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), + null, connectionSource, connection) then: cursor.getServerAddress() != null } def 'should get Exceptions for operations on the cursor after closing'() { given: - def firstBatch = executeQuery() + def (serverAddress, commandResult) = executeFindCommand() - cursor = new QueryBatchCursor(firstBatch, 0, 0, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), + null, connectionSource, connection) when: cursor.close() @@ -126,9 +141,10 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi def 'should throw an Exception when going off the end'() { given: - def firstBatch = executeQuery(1) + def (serverAddress, commandResult) = executeFindCommand(1) - cursor = new QueryBatchCursor(firstBatch, 2, 0, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 2, 0, 0, new DocumentCodec(), + null, connectionSource, connection) when: cursor.next() cursor.next() @@ -140,10 +156,11 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi def 'test normal exhaustion'() { given: - def firstBatch = executeQuery() + def (serverAddress, commandResult) = executeFindCommand() when: - cursor = new QueryBatchCursor(firstBatch, 0, 0, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), + null, connectionSource, connection) then: cursor.iterator().sum { it.size() } == 10 @@ -151,18 +168,15 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi def 'test limit exhaustion'() { given: - def firstBatch = executeQuery(limit, batchSize) - def connection = connectionSource.getConnection() + def (serverAddress, commandResult) = executeFindCommand(limit, batchSize) when: - cursor = new QueryBatchCursor(firstBatch, limit, batchSize, 0, new DocumentCodec(), null, connectionSource, connection) + cursor = new CommandBatchCursor(serverAddress, commandResult, limit, batchSize, 0, new DocumentCodec(), + null, connectionSource, connection) then: cursor.iterator().sum { it.size() } == expectedTotal - cleanup: - connection?.release() - where: limit | batchSize | expectedTotal 5 | 2 | 5 @@ -177,9 +191,10 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi def 'test remove'() { given: - def firstBatch = executeQuery() + def (serverAddress, commandResult) = executeFindCommand() - cursor = new QueryBatchCursor(firstBatch, 0, 0, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), + null, connectionSource, connection) when: cursor.remove() @@ -195,10 +210,12 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi def connection = connectionSource.getConnection() collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(5, 0))) - def firstBatch = executeQuery(new BsonDocument('ts', new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, awaitData) + def (serverAddress, commandResult) = executeFindCommand(new BsonDocument('ts', + new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, awaitData) when: - cursor = new QueryBatchCursor(firstBatch, 0, 2, maxTimeMS, new DocumentCodec(), null, connectionSource, connection) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, maxTimeMS, new DocumentCodec(), + null, connectionSource, connection) then: cursor.hasNext() @@ -241,11 +258,13 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi def 'test try next with tailable'() { collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(5, 0))) - def firstBatch = executeQuery(new BsonDocument('ts', new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, true) + def (serverAddress, commandResult) = executeFindCommand(new BsonDocument('ts', + new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, true) when: - cursor = new QueryBatchCursor(firstBatch, 0, 2, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) then: cursor.tryNext().iterator().next().get('_id') == 1 @@ -264,11 +283,13 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi @Slow def 'hasNext should throw when cursor is closed in another thread'() { - Connection conn = connectionSource.getConnection() + Connection connection = connectionSource.getConnection() collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(5, 0))) - def firstBatch = executeQuery(new BsonDocument('ts', new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, true) - cursor = new QueryBatchCursor(firstBatch, 0, 2, 0, new DocumentCodec(), null, connectionSource, conn) + def (serverAddress, commandResult) = executeFindCommand(new BsonDocument('ts', + new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, true) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) cursor.next() def closeCompleted = new CountDownLatch(1) @@ -296,11 +317,13 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi def 'test maxTimeMS'() { collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(5, 0))) - def firstBatch = executeQuery(new BsonDocument('ts', new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, true) + def (serverAddress, commandResult) = executeFindCommand(new BsonDocument('ts', + new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, true) def connection = connectionSource.getConnection() - def maxTimeMS = 10 - cursor = new QueryBatchCursor(firstBatch, 0, 2, maxTimeMS, new DocumentCodec(), null, connectionSource, connection) + def maxTimeMS = 10L + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, maxTimeMS, new DocumentCodec(), + null, connectionSource, connection) cursor.tryNext() long startTime = System.currentTimeMillis() @@ -314,7 +337,7 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi cleanup: connection?.release() - } + } @SuppressWarnings('EmptyCatchBlock') @Slow @@ -322,10 +345,11 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1)) - def firstBatch = executeQuery(new BsonDocument(), 0, 2, true, true) + def (serverAddress, commandResult) = executeFindCommand(new BsonDocument(), 0, 2, true, true) when: - cursor = new QueryBatchCursor(firstBatch, 0, 2, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) CountDownLatch latch = new CountDownLatch(1) def seen = 0 @@ -353,61 +377,58 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi @IgnoreIf({ isSharded() }) def 'should kill cursor if limit is reached on initial query'() throws InterruptedException { given: - def firstBatch = executeQuery(5) - def connection = connectionSource.getConnection() - - cursor = new QueryBatchCursor(firstBatch, 5, 0, 0, new DocumentCodec(), null, connectionSource, connection) + def (serverAddress, commandResult) = executeFindCommand(5) + cursor = new CommandBatchCursor(serverAddress, commandResult, 5, 0, 0, new DocumentCodec(), + null, connectionSource, connection) when: - makeAdditionalGetMoreCall(getNamespace(), firstBatch.cursor, connection) + cursor.next() then: - thrown(MongoCursorNotFoundException) - - cleanup: - connection?.release() + !cursor.hasNext() + cursor.getServerCursor() == null } @IgnoreIf({ isSharded() }) - @Slow def 'should kill cursor if limit is reached on get more'() throws InterruptedException { given: - def firstBatch = executeQuery(3) + def (serverAddress, commandResult) = executeFindCommand(3) - cursor = new QueryBatchCursor(firstBatch, 5, 3, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 5, 3, 0, new DocumentCodec(), + null, connectionSource, connection) ServerCursor serverCursor = cursor.getServerCursor() cursor.next() cursor.next() - Thread.sleep(1000) //Note: waiting for some time for killCursor operation to be performed on a server. + checkReferenceCountReachesTarget(connection, 1) + when: - makeAdditionalGetMoreCall(getNamespace(), serverCursor, connectionSource) + makeAdditionalGetMoreCall(getNamespace(), serverCursor, connection) then: - thrown(MongoCursorNotFoundException) + thrown(MongoQueryException) } def 'should release connection source if limit is reached on initial query'() throws InterruptedException { given: - def firstBatch = executeQuery(5) - def connection = connectionSource.getConnection() + def (serverAddress, commandResult) = executeFindCommand(5) when: - cursor = new QueryBatchCursor(firstBatch, 5, 0, 0, new DocumentCodec(), null, connectionSource, connection) + cursor = new CommandBatchCursor(serverAddress, commandResult, 5, 0, 0, new DocumentCodec(), + null, connectionSource, connection) then: checkReferenceCountReachesTarget(connectionSource, 1) - - cleanup: - connection?.release() + checkReferenceCountReachesTarget(connection, 1) } def 'should release connection source if limit is reached on get more'() throws InterruptedException { given: - def firstBatch = executeQuery(3) + def (serverAddress, commandResult) = executeFindCommand(3) - cursor = new QueryBatchCursor(firstBatch, 5, 3, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 5, 3, 0, new DocumentCodec(), + null, connectionSource, connection) when: cursor.next() @@ -415,14 +436,16 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi then: checkReferenceCountReachesTarget(connectionSource, 1) + checkReferenceCountReachesTarget(connection, 1) } def 'test limit with get more'() { given: - def firstBatch = executeQuery(2) + def (serverAddress, commandResult) = executeFindCommand(2) when: - cursor = new QueryBatchCursor(firstBatch, 5, 2, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 5, 2, 0, new DocumentCodec(), + null, connectionSource, connection) then: cursor.next() != null @@ -434,14 +457,14 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi @Slow def 'test limit with large documents'() { given: - char[] array = 'x' * 16000 - String bigString = new String(array) + String bigString = new String('x' * 16000) (11..1000).each { collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', it).append('s', bigString)) } - def firstBatch = executeQuery(300, 0) + def (serverAddress, commandResult) = executeFindCommand(300, 0) when: - cursor = new QueryBatchCursor(firstBatch, 300, 0, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 300, 0, 0, new DocumentCodec(), + null, connectionSource, connection) then: cursor.iterator().sum { it.size() } == 300 @@ -449,10 +472,11 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi def 'should respect batch size'() { given: - def firstBatch = executeQuery(2) + def (serverAddress, commandResult) = executeFindCommand(2) when: - cursor = new QueryBatchCursor(firstBatch, 0, 2, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) then: cursor.batchSize == 2 @@ -486,10 +510,11 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi def 'test normal loop with get more'() { given: - def firstBatch = executeQuery(2) + def (serverAddress, commandResult) = executeFindCommand(2) when: - cursor = new QueryBatchCursor(firstBatch, 0, 2, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) def results = cursor.iterator().collectMany { it*.get('_id') } then: @@ -499,10 +524,11 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi def 'test next without has next with get more'() { given: - def firstBatch = executeQuery(2) + def (serverAddress, commandResult) = executeFindCommand(2) when: - cursor = new QueryBatchCursor(firstBatch, 0, 2, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) then: (0..4).each { cursor.next() } @@ -520,10 +546,11 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi @IgnoreIf({ isSharded() }) def 'should throw cursor not found exception'() { given: - def firstBatch = executeQuery(2) + def (serverAddress, commandResult) = executeFindCommand(2) when: - cursor = new QueryBatchCursor(firstBatch, 0, 2, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) def serverCursor = cursor.getServerCursor() def connection = connectionSource.getConnection() connection.command(getNamespace().databaseName, @@ -546,10 +573,11 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi def 'should report available documents'() { given: - def firstBatch = executeQuery(3) + def (serverAddress, commandResult) = executeFindCommand(3) when: - cursor = new QueryBatchCursor(firstBatch, 0, 2, new DocumentCodec(), null, connectionSource) + cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), + null, connectionSource, connection) then: cursor.available() == 3 @@ -591,52 +619,43 @@ class QueryBatchCursorFunctionalSpecification extends OperationFunctionalSpecifi cursor.available() == 0 } - private QueryResult executeQuery() { - executeQuery(0) + private Tuple2 executeFindCommand() { + executeFindCommand(0) } - private QueryResult executeQuery(int batchSize) { - executeQuery(new BsonDocument(), 0, batchSize, false, false, ReadPreference.primary()) + private Tuple2 executeFindCommand(int batchSize) { + executeFindCommand(new BsonDocument(), 0, batchSize, false, false, ReadPreference.primary()) } - private QueryResult executeQuery(int batchSize, ReadPreference readPreference) { - executeQuery(new BsonDocument(), 0, batchSize, false, false, readPreference) + private Tuple2 executeFindCommand(int limit, int batchSize) { + executeFindCommand(new BsonDocument(), limit, batchSize, false, false, ReadPreference.primary()) } - private QueryResult executeQuery(int limit, int batchSize) { - executeQuery(new BsonDocument(), limit, batchSize, false, false, ReadPreference.primary()) + private Tuple2 executeFindCommand(BsonDocument filter, int limit, int batchSize, boolean tailable, + boolean awaitData) { + executeFindCommand(filter, limit, batchSize, tailable, awaitData, ReadPreference.primary()) } + private Tuple2 executeFindCommand(BsonDocument filter, int limit, int batchSize, boolean tailable, + boolean awaitData, ReadPreference readPreference) { + def findCommand = new BsonDocument('find', new BsonString(getCollectionName())) + .append('filter', filter) + .append('tailable', BsonBoolean.valueOf(tailable)) + .append('awaitData', BsonBoolean.valueOf(awaitData)) - private QueryResult executeQuery(BsonDocument filter, int limit, int batchSize, boolean tailable, boolean awaitData) { - executeQuery(filter, limit, batchSize, tailable, awaitData, ReadPreference.primary()) - } + findCommand.append('limit', new BsonInt32(Math.abs(limit))) - private QueryResult executeQuery(BsonDocument filter, int limit, int batchSize, boolean tailable, boolean awaitData, - ReadPreference readPreference) { - def connection = connectionSource.getConnection() - try { - def findCommand = new BsonDocument('find', new BsonString(getCollectionName())) - .append('filter', filter) - .append('tailable', BsonBoolean.valueOf(tailable)) - .append('awaitData', BsonBoolean.valueOf(awaitData)) - - findCommand.append('limit', new BsonInt32(Math.abs(limit))) - - if (limit >= 0) { - if (batchSize < 0 && Math.abs(batchSize) < limit) { - findCommand.append('limit', new BsonInt32(Math.abs(batchSize))) - } else { - findCommand.append('batchSize', new BsonInt32(Math.abs(batchSize))) - } + if (limit >= 0) { + if (batchSize < 0 && Math.abs(batchSize) < limit) { + findCommand.append('limit', new BsonInt32(Math.abs(batchSize))) + } else { + findCommand.append('batchSize', new BsonInt32(Math.abs(batchSize))) } - - def response = connection.command(getDatabaseName(), findCommand, - NO_OP_FIELD_NAME_VALIDATOR, readPreference, - CommandResultDocumentCodec.create(new DocumentCodec(), 'firstBatch'), connectionSource) - cursorDocumentToQueryResult(response.getDocument('cursor'), connection.getDescription().getServerAddress()) - } finally { - connection.release() } + + def response = connection.command(getDatabaseName(), findCommand, + NO_OP_FIELD_NAME_VALIDATOR, readPreference, + CommandResultDocumentCodec.create(new DocumentCodec(), 'firstBatch'), connectionSource) + new Tuple2(connection.getDescription().getServerAddress(), response) } } diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/ListCollectionsOperationSpecification.groovy b/driver-core/src/test/functional/com/mongodb/internal/operation/ListCollectionsOperationSpecification.groovy index 9e2d8937818..18b1724fdff 100644 --- a/driver-core/src/test/functional/com/mongodb/internal/operation/ListCollectionsOperationSpecification.groovy +++ b/driver-core/src/test/functional/com/mongodb/internal/operation/ListCollectionsOperationSpecification.groovy @@ -16,13 +16,10 @@ package com.mongodb.internal.operation - import com.mongodb.MongoExecutionTimeoutException import com.mongodb.MongoNamespace import com.mongodb.OperationFunctionalSpecification import com.mongodb.ReadPreference -import com.mongodb.ServerAddress -import com.mongodb.ServerCursor import com.mongodb.async.FutureResultCallback import com.mongodb.client.model.CreateCollectionOptions import com.mongodb.connection.ConnectionDescription @@ -34,7 +31,6 @@ import com.mongodb.internal.binding.ConnectionSource import com.mongodb.internal.binding.ReadBinding import com.mongodb.internal.connection.AsyncConnection import com.mongodb.internal.connection.Connection -import com.mongodb.internal.connection.QueryResult import org.bson.BsonBoolean import org.bson.BsonDocument import org.bson.BsonDouble @@ -84,7 +80,7 @@ class ListCollectionsOperationSpecification extends OperationFunctionalSpecifica cursor.next(callback) then: - callback.get() == null + callback.get() == [] cleanup: collectionHelper.dropDatabase(madeUpDatabase) @@ -380,7 +376,7 @@ class ListCollectionsOperationSpecification extends OperationFunctionalSpecifica cursor.getBatchSize() == 2 cleanup: - consumeAsyncResults(cursor) + cursor?.close() } @IgnoreIf({ isSharded() }) @@ -479,11 +475,6 @@ class ListCollectionsOperationSpecification extends OperationFunctionalSpecifica threeSixConnectionDescription : Stub(ConnectionDescription) { getMaxWireVersion() >> 3 }, - queryResult: Stub(QueryResult) { - getNamespace() >> new MongoNamespace('db', 'coll') - getResults() >> [] - getCursor() >> new ServerCursor(1, Stub(ServerAddress)) - }, commandResult: new BsonDocument('ok', new BsonDouble(1.0)) .append('cursor', new BsonDocument('id', new BsonInt64(1)).append('ns', new BsonString('db.coll')) .append('firstBatch', new BsonArrayWrapper([]))) diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/ListIndexesOperationSpecification.groovy b/driver-core/src/test/functional/com/mongodb/internal/operation/ListIndexesOperationSpecification.groovy index 4ca91524e9f..20f842c3fff 100644 --- a/driver-core/src/test/functional/com/mongodb/internal/operation/ListIndexesOperationSpecification.groovy +++ b/driver-core/src/test/functional/com/mongodb/internal/operation/ListIndexesOperationSpecification.groovy @@ -16,13 +16,10 @@ package com.mongodb.internal.operation - import com.mongodb.MongoExecutionTimeoutException import com.mongodb.MongoNamespace import com.mongodb.OperationFunctionalSpecification import com.mongodb.ReadPreference -import com.mongodb.ServerAddress -import com.mongodb.ServerCursor import com.mongodb.async.FutureResultCallback import com.mongodb.connection.ConnectionDescription import com.mongodb.internal.async.AsyncBatchCursor @@ -34,7 +31,6 @@ import com.mongodb.internal.binding.ReadBinding import com.mongodb.internal.bulk.IndexRequest import com.mongodb.internal.connection.AsyncConnection import com.mongodb.internal.connection.Connection -import com.mongodb.internal.connection.QueryResult import org.bson.BsonDocument import org.bson.BsonDouble import org.bson.BsonInt32 @@ -76,7 +72,7 @@ class ListIndexesOperationSpecification extends OperationFunctionalSpecification cursor.next(callback) then: - callback.get() == null + callback.get() == [] } @@ -210,7 +206,7 @@ class ListIndexesOperationSpecification extends OperationFunctionalSpecification cursor.getBatchSize() == 2 cleanup: - consumeAsyncResults(cursor) + cursor?.close() } @IgnoreIf({ isSharded() }) @@ -310,11 +306,6 @@ class ListIndexesOperationSpecification extends OperationFunctionalSpecification threeSixConnectionDescription : Stub(ConnectionDescription) { getMaxWireVersion() >> 3 }, - queryResult: Stub(QueryResult) { - getNamespace() >> new MongoNamespace('db', 'coll') - getResults() >> [] - getCursor() >> new ServerCursor(1, Stub(ServerAddress)) - }, commandResult: new BsonDocument('ok', new BsonDouble(1.0)) .append('cursor', new BsonDocument('id', new BsonInt64(1)).append('ns', new BsonString('db.coll')) .append('firstBatch', new BsonArrayWrapper([]))) diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/QueryOperationHelper.groovy b/driver-core/src/test/functional/com/mongodb/internal/operation/QueryOperationHelper.groovy index c11d113f80c..8c609306f75 100644 --- a/driver-core/src/test/functional/com/mongodb/internal/operation/QueryOperationHelper.groovy +++ b/driver-core/src/test/functional/com/mongodb/internal/operation/QueryOperationHelper.groovy @@ -22,9 +22,10 @@ import com.mongodb.MongoNamespace import com.mongodb.MongoQueryException import com.mongodb.ReadPreference import com.mongodb.ServerCursor +import com.mongodb.async.FutureResultCallback import com.mongodb.internal.IgnorableRequestContext -import com.mongodb.internal.binding.ConnectionSource import com.mongodb.internal.binding.StaticBindingContext +import com.mongodb.internal.connection.AsyncConnection import com.mongodb.internal.connection.Connection import com.mongodb.internal.connection.NoOpSessionContext import com.mongodb.internal.connection.OperationContext @@ -53,18 +54,8 @@ class QueryOperationHelper { } } - static void makeAdditionalGetMoreCall(MongoNamespace namespace, ServerCursor serverCursor, - ConnectionSource connectionSource) { - def connection = connectionSource.getConnection() - try { - makeAdditionalGetMoreCall(namespace, serverCursor, connection) - } finally { - connection.release() - } - } - static void makeAdditionalGetMoreCall(MongoNamespace namespace, ServerCursor serverCursor, Connection connection) { - try { + makeAdditionalGetMoreCallHandleError(serverCursor) { connection.command(namespace.databaseName, new BsonDocument('getMore', new BsonInt64(serverCursor.getId())) .append('collection', new BsonString(namespace.getCollectionName())), @@ -72,6 +63,26 @@ class QueryOperationHelper { new BsonDocumentCodec(), new StaticBindingContext(new NoOpSessionContext(), getServerApi(), IgnorableRequestContext.INSTANCE, new OperationContext())) + } + } + + static void makeAdditionalGetMoreCall(MongoNamespace namespace, ServerCursor serverCursor, AsyncConnection connection) { + def callback = new FutureResultCallback<>() + connection.commandAsync(namespace.databaseName, + new BsonDocument('getMore', new BsonInt64(serverCursor.getId())) + .append('collection', new BsonString(namespace.getCollectionName())), + new NoOpFieldNameValidator(), ReadPreference.primary(), + new BsonDocumentCodec(), + new StaticBindingContext(new NoOpSessionContext(), getServerApi(), IgnorableRequestContext.INSTANCE, + new OperationContext()), + callback + ) + makeAdditionalGetMoreCallHandleError(serverCursor) { callback.get() } + } + + static void makeAdditionalGetMoreCallHandleError(ServerCursor serverCursor, Runnable runnable) { + try { + runnable.run() } catch (MongoCommandException e) { if (e.getErrorCode() == 43) { throw new MongoCursorNotFoundException(serverCursor.getId(), e.getResponse(), serverCursor.getAddress()) diff --git a/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncChangeStreamBatchCursorSpecification.groovy b/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncChangeStreamBatchCursorSpecification.groovy index 1142ce5f91c..f58f58f3cb7 100644 --- a/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncChangeStreamBatchCursorSpecification.groovy +++ b/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncChangeStreamBatchCursorSpecification.groovy @@ -27,11 +27,12 @@ import static java.util.concurrent.TimeUnit.SECONDS class AsyncChangeStreamBatchCursorSpecification extends Specification { - def 'should call the underlying AsyncQueryBatchCursor'() { + def 'should call the underlying AsyncCommandBatchCursor'() { given: def changeStreamOpertation = Stub(ChangeStreamOperation) def binding = Mock(AsyncReadBinding) - def wrapped = Mock(AsyncQueryBatchCursor) + 1 * binding.retain() >> binding + def wrapped = Mock(AsyncCommandBatchCursor) def callback = Stub(SingleResultCallback) def cursor = new AsyncChangeStreamBatchCursor(changeStreamOpertation, wrapped, binding, null, ServerVersionHelper.FOUR_DOT_FOUR_WIRE_VERSION) @@ -66,7 +67,8 @@ class AsyncChangeStreamBatchCursorSpecification extends Specification { def 'should not close the cursor in next if the cursor was closed before next completed'() { def changeStreamOpertation = Stub(ChangeStreamOperation) def binding = Mock(AsyncReadBinding) - def wrapped = Mock(AsyncQueryBatchCursor) + 1 * binding.retain() >> binding + def wrapped = Mock(AsyncCommandBatchCursor) def callback = Stub(SingleResultCallback) def cursor = new AsyncChangeStreamBatchCursor(changeStreamOpertation, wrapped, binding, null, ServerVersionHelper.FOUR_DOT_FOUR_WIRE_VERSION) @@ -91,7 +93,8 @@ class AsyncChangeStreamBatchCursorSpecification extends Specification { def 'should throw a MongoException when next/tryNext is called after the cursor is closed'() { def changeStreamOpertation = Stub(ChangeStreamOperation) def binding = Mock(AsyncReadBinding) - def wrapped = Mock(AsyncQueryBatchCursor) + 1 * binding.retain() >> binding + def wrapped = Mock(AsyncCommandBatchCursor) def cursor = new AsyncChangeStreamBatchCursor(changeStreamOpertation, wrapped, binding, null, ServerVersionHelper.FOUR_DOT_FOUR_WIRE_VERSION) diff --git a/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncQueryBatchCursorSpecification.groovy b/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncCommandBatchCursorSpecification.groovy similarity index 68% rename from driver-core/src/test/unit/com/mongodb/internal/operation/AsyncQueryBatchCursorSpecification.groovy rename to driver-core/src/test/unit/com/mongodb/internal/operation/AsyncCommandBatchCursorSpecification.groovy index 5efcbc736ab..d12ae8744ff 100644 --- a/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncQueryBatchCursorSpecification.groovy +++ b/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncCommandBatchCursorSpecification.groovy @@ -30,7 +30,6 @@ import com.mongodb.connection.ServerVersion import com.mongodb.internal.async.SingleResultCallback import com.mongodb.internal.binding.AsyncConnectionSource import com.mongodb.internal.connection.AsyncConnection -import com.mongodb.internal.connection.QueryResult import org.bson.BsonArray import org.bson.BsonDocument import org.bson.BsonInt32 @@ -42,17 +41,18 @@ import spock.lang.Specification import static OperationUnitSpecification.getMaxWireVersionForServerVersion import static com.mongodb.ReadPreference.primary +import static com.mongodb.internal.operation.CommandBatchCursorHelper.MESSAGE_IF_CLOSED_AS_CURSOR -class AsyncQueryBatchCursorSpecification extends Specification { +class AsyncCommandBatchCursorSpecification extends Specification { def 'should generate expected command with batchSize and maxTimeMS'() { given: def connection = referenceCountedAsyncConnection() def connectionSource = getAsyncConnectionSource(connection) - def firstBatch = new QueryResult(NAMESPACE, [], 42, SERVER_ADDRESS) - def cursor = new AsyncQueryBatchCursor(firstBatch, 0, batchSize, maxTimeMS, CODEC, null, connectionSource, - connection) + def firstBatch = createCommandResult([]) + def cursor = new AsyncCommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, batchSize, maxTimeMS, CODEC, + null, connectionSource, connection) def expectedCommand = new BsonDocument('getMore': new BsonInt64(CURSOR_ID)) .append('collection', new BsonString(NAMESPACE.getCollectionName())) if (batchSize != 0) { @@ -62,7 +62,7 @@ class AsyncQueryBatchCursorSpecification extends Specification { expectedCommand.append('maxTimeMS', new BsonInt64(expectedMaxTimeFieldValue)) } - def reply = documentResponse([], 0) + def reply = getMoreResponse([], 0) when: def batch = nextBatch(cursor) @@ -71,7 +71,7 @@ class AsyncQueryBatchCursorSpecification extends Specification { 1 * connection.commandAsync(NAMESPACE.getDatabaseName(), expectedCommand, *_) >> { it.last().onResult(reply, null) } - batch == null + batch.isEmpty() then: !cursor.isClosed() @@ -95,14 +95,15 @@ class AsyncQueryBatchCursorSpecification extends Specification { def serverVersion = new ServerVersion([3, 6, 0]) def connection = referenceCountedAsyncConnection(serverVersion) def connectionSource = getAsyncConnectionSource(connection) + def cursor = new AsyncCommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connection) when: - def cursor = new AsyncQueryBatchCursor(firstBatch, 0, 0, 0, CODEC, null, connectionSource, connection) cursor.close() then: - if (firstBatch.getCursor() != null) { - 1 * connection.commandAsync(NAMESPACE.databaseName, createKillCursorsDocument(firstBatch.cursor), _, primary(), *_) >> { + if (cursor.getServerCursor() != null) { + 1 * connection.commandAsync(NAMESPACE.databaseName, createKillCursorsDocument(cursor.getServerCursor()), _, primary(), *_) >> { it.last().onResult(null, null) } } @@ -112,15 +113,18 @@ class AsyncQueryBatchCursorSpecification extends Specification { connectionSource.getCount() == 0 where: - firstBatch << [queryResult(), queryResult(FIRST_BATCH, 0)] + firstBatch << [createCommandResult(), createCommandResult(FIRST_BATCH, 0)] } def 'should return the expected results from next'() { given: - def connectionSource = getAsyncConnectionSource(referenceCountedAsyncConnection()) + def connection = referenceCountedAsyncConnection() + def connectionSource = getAsyncConnectionSource(connection) when: - def cursor = new AsyncQueryBatchCursor(queryResult(FIRST_BATCH, 0), 0, 0, 0, CODEC, null, connectionSource, null) + def firstBatch = createCommandResult(FIRST_BATCH, 0) + def cursor = new AsyncCommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connection) then: nextBatch(cursor) == FIRST_BATCH @@ -136,23 +140,23 @@ class AsyncQueryBatchCursorSpecification extends Specification { then: def exception = thrown(MongoException) - exception.getMessage() == 'next() called after the cursor was closed.' + exception.getMessage() == MESSAGE_IF_CLOSED_AS_CURSOR } def 'should respect the limit'() { given: def serverVersion = new ServerVersion([3, 6, 0]) - def connectionA = referenceCountedAsyncConnection(serverVersion) - def connectionB = referenceCountedAsyncConnection(serverVersion) + def connectionA = referenceCountedAsyncConnection(serverVersion, 'connectionA') + def connectionB = referenceCountedAsyncConnection(serverVersion, 'connectionB') def connectionSource = getAsyncConnectionSource(connectionA, connectionB) def firstBatch = [new Document('_id', 1), new Document('_id', 2), new Document('_id', 3)] - def secondBatch = [new Document('_id', 4), new Document('_id', 5)] - def thirdBatch = [new Document('_id', 6)] + def secondBatch = [new Document('_id', 4), new Document('_id', 5), new Document('_id', 6)] + def thirdBatch = [new Document('_id', 7)] when: - def cursor = new AsyncQueryBatchCursor(queryResult(firstBatch), 6, 2, 0, CODEC, null, connectionSource, - connectionA) + def cursor = new AsyncCommandBatchCursor(SERVER_ADDRESS, createCommandResult(firstBatch), 7, 3, 0, CODEC, + null, connectionSource, connectionA) def batch = nextBatch(cursor) then: @@ -162,7 +166,7 @@ class AsyncQueryBatchCursorSpecification extends Specification { batch = nextBatch(cursor) then: - 1 * connectionA.commandAsync(*_) >> { it.last().onResult(documentResponse(secondBatch), null) } + 1 * connectionA.commandAsync(*_) >> { it.last().onResult(getMoreResponse(secondBatch), null) } then: batch == secondBatch @@ -176,7 +180,7 @@ class AsyncQueryBatchCursorSpecification extends Specification { 1 * connectionB.commandAsync(*_) >> { connectionB.getCount() == 1 connectionSource.getCount() == 1 - it.last().onResult(documentResponse(thirdBatch, 0), null) + it.last().onResult(getMoreResponse(thirdBatch, 0), null) } then: @@ -188,7 +192,7 @@ class AsyncQueryBatchCursorSpecification extends Specification { batch = nextBatch(cursor) then: - batch == null + batch.isEmpty() connectionSource.getCount() == 0 } @@ -198,16 +202,15 @@ class AsyncQueryBatchCursorSpecification extends Specification { def serverVersion = new ServerVersion([3, 6, 0]) def connection = referenceCountedAsyncConnection(serverVersion) def connectionSource = getAsyncConnectionSource(connection) - def queryResult = queryResult() + def firstBatch = createCommandResult() when: - def cursor = new AsyncQueryBatchCursor(queryResult, 1, 0, 0, CODEC, null, connectionSource, connection) + def cursor = new AsyncCommandBatchCursor(SERVER_ADDRESS, firstBatch, 1, 0, 0, CODEC, + null, connectionSource, connection) then: - 1 * connection.commandAsync(NAMESPACE.databaseName, createKillCursorsDocument(queryResult.cursor), _, primary(), - *_) >> { - it.last().onResult(null, null) - } + 1 * connection.commandAsync(NAMESPACE.databaseName, + createKillCursorsDocument(new ServerCursor(42, SERVER_ADDRESS)), _, primary(), *_) >> { it.last().onResult(null, null) } when: cursor.close() @@ -222,53 +225,59 @@ class AsyncQueryBatchCursorSpecification extends Specification { def 'should handle getMore when there are empty results but there is a cursor'() { given: - def connection = referenceCountedAsyncConnection(serverVersion) - def connectionSource = getAsyncConnectionSource(connection) + def connectionA = referenceCountedAsyncConnection(serverVersion, 'connectionA') + def connectionB = referenceCountedAsyncConnection(serverVersion, 'connectionB') + def connectionSource = getAsyncConnectionSource(connectionA, connectionB) when: - def cursor = new AsyncQueryBatchCursor(queryResult([], 42), 3, 0, 0, CODEC, null, connectionSource, connection) + def firstBatch = createCommandResult([], CURSOR_ID) + def cursor = new AsyncCommandBatchCursor(SERVER_ADDRESS, firstBatch, 3, 0, 0, CODEC, + null, connectionSource, connectionA) def batch = nextBatch(cursor) then: - 1 * connection.commandAsync(*_) >> { - connection.getCount() == 1 + 1 * connectionA.commandAsync(*_) >> { + connectionA.getCount() == 1 connectionSource.getCount() == 1 - it.last().onResult(response, null) + it.last().onResult(response, null) } - 1 * connection.commandAsync(*_) >> { - connection.getCount() == 1 + 1 * connectionB.commandAsync(*_) >> { + connectionB.getCount() == 1 connectionSource.getCount() == 1 - it.last().onResult(response2, null) + it.last().onResult(response2, null) } then: batch == SECOND_BATCH then: - connection.getCount() == 0 + connectionA.getCount() == 0 + connectionB.getCount() == 0 connectionSource.getCount() == 0 when: cursor.close() then: - 0 * connection._ + 0 * connectionA._ + 0 * connectionB._ connectionSource.getCount() == 0 where: - serverVersion | response | response2 - new ServerVersion([3, 6, 0]) | documentResponse([]) | documentResponse(SECOND_BATCH, 0) + serverVersion | response | response2 + new ServerVersion([3, 6, 0]) | getMoreResponse([]) | getMoreResponse(SECOND_BATCH, 0) } def 'should kill the cursor in the getMore if limit is reached'() { given: def connection = referenceCountedAsyncConnection(serverVersion) def connectionSource = getAsyncConnectionSource(connection) - def initialResult = queryResult() + def firstBatch = createCommandResult() when: - def cursor = new AsyncQueryBatchCursor(initialResult, 3, 0, 0, CODEC, null, connectionSource, connection) + def cursor = new AsyncCommandBatchCursor(SERVER_ADDRESS, firstBatch, 3, 0, 0, CODEC, + null, connectionSource, connection) def batch = nextBatch(cursor) then: @@ -281,7 +290,7 @@ class AsyncQueryBatchCursorSpecification extends Specification { 1 * connection.commandAsync(*_) >> { it.last().onResult(response, null) } - 1 * connection.commandAsync(NAMESPACE.databaseName, createKillCursorsDocument(initialResult.cursor), _, primary(), _, + 1 * connection.commandAsync(NAMESPACE.databaseName, createKillCursorsDocument(cursor.getServerCursor()), _, primary(), _, connectionSource, *_) >> { it.last().onResult(null, null) } @@ -299,19 +308,20 @@ class AsyncQueryBatchCursorSpecification extends Specification { where: serverVersion | response - new ServerVersion([3, 2, 0]) | documentResponse(SECOND_BATCH) + new ServerVersion([3, 2, 0]) | getMoreResponse(SECOND_BATCH) } def 'should close cursor after getMore finishes if cursor was closed while getMore was in progress and getMore returns a response'() { given: def serverVersion = new ServerVersion([3, 6, 0]) - def connectionA = referenceCountedAsyncConnection(serverVersion) - def connectionB = referenceCountedAsyncConnection(serverVersion) + def connectionA = referenceCountedAsyncConnection(serverVersion, 'connectionA') + def connectionB = referenceCountedAsyncConnection(serverVersion, 'connectionB') def connectionSource = getAsyncConnectionSource(serverType, connectionA, connectionB) - def initialResult = queryResult() + def firstBatch = createCommandResult() when: - def cursor = new AsyncQueryBatchCursor(initialResult, 0, 0, 0, CODEC, null, connectionSource, connectionA) + def cursor = new AsyncCommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connectionA) def batch = nextBatch(cursor) then: @@ -324,10 +334,10 @@ class AsyncQueryBatchCursorSpecification extends Specification { numberOfInvocations * connectionA.commandAsync(*_) >> { // Simulate the user calling close while the getMore is in flight cursor.close() - ((SingleResultCallback) it.last()).onResult(response, null) + ((SingleResultCallback) it.last()).onResult(response, null) } >> { - // `killCursors` command - ((SingleResultCallback) it.last()).onResult(response2, null) + // `killCursors` command + ((SingleResultCallback) it.last()).onResult(null, null) } then: @@ -335,26 +345,29 @@ class AsyncQueryBatchCursorSpecification extends Specification { then: connectionA.getCount() == 0 + connectionB.getCount() == 0 + connectionSource.getCount() == 0 cursor.isClosed() where: - response | response2 | getMoreResponseHasCursor | serverType | numberOfInvocations - documentResponse([]) | documentResponse([], 0) | true | ServerType.LOAD_BALANCER | 2 - documentResponse([], 0) | null | false | ServerType.LOAD_BALANCER | 1 - documentResponse([]) | documentResponse([], 0) | true | ServerType.STANDALONE | 1 - documentResponse([], 0) | null | false | ServerType.STANDALONE | 1 + response | serverType | numberOfInvocations + getMoreResponse([]) | ServerType.LOAD_BALANCER | 2 + getMoreResponse([], 0) | ServerType.LOAD_BALANCER | 1 + getMoreResponse([]) | ServerType.STANDALONE | 2 + getMoreResponse([], 0) | ServerType.STANDALONE | 1 } def 'should close cursor after getMore finishes if cursor was closed while getMore was in progress and getMore throws exception'() { given: - def serverVersion = new ServerVersion([3, 2, 0]) - def connectionA = referenceCountedAsyncConnection(serverVersion) - def connectionB = referenceCountedAsyncConnection(serverVersion) + def serverVersion = new ServerVersion([4, 4, 0]) + def connectionA = referenceCountedAsyncConnection(serverVersion, 'connectionA') + def connectionB = referenceCountedAsyncConnection(serverVersion, 'connectionB') def connectionSource = getAsyncConnectionSource(serverType, connectionA, connectionB) - def initialResult = queryResult() + def firstBatch = createCommandResult() when: - def cursor = new AsyncQueryBatchCursor(initialResult, 0, 0, 0, CODEC, null, connectionSource, connectionA) + def cursor = new AsyncCommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connectionA) def batch = nextBatch(cursor) then: @@ -367,10 +380,10 @@ class AsyncQueryBatchCursorSpecification extends Specification { numberOfInvocations * connectionA.commandAsync(*_) >> { // Simulate the user calling close while the getMore is throwing a MongoException cursor.close() - ((SingleResultCallback) it.last()).onResult(null, MONGO_EXCEPTION) - } >> { - // `killCursors` command - ((SingleResultCallback) it.last()).onResult(null, null) + ((SingleResultCallback) it.last()).onResult(null, MONGO_EXCEPTION) + } >> { + // `killCursors` command + ((SingleResultCallback) it.last()).onResult(null, null) } then: @@ -390,15 +403,12 @@ class AsyncQueryBatchCursorSpecification extends Specification { given: def connection = referenceCountedAsyncConnection() def connectionSource = getAsyncConnectionSourceWithResult(ServerType.STANDALONE) { [null, MONGO_EXCEPTION] } - def cursor = new AsyncQueryBatchCursor(queryResult(), 0, 0, 0, CODEC, null, connectionSource, connection) + def firstBatch = createCommandResult() + def cursor = new AsyncCommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connection) when: cursor.close() - nextBatch(cursor) - - then: - def exception = thrown(MongoException) - exception.getMessage() == 'next() called after the cursor was closed.' then: cursor.isClosed() @@ -412,7 +422,9 @@ class AsyncQueryBatchCursorSpecification extends Specification { def connectionSource = getAsyncConnectionSourceWithResult(ServerType.STANDALONE) { [null, MONGO_EXCEPTION] } when: - def cursor = new AsyncQueryBatchCursor(queryResult(), 0, 0, 0, CODEC, null, connectionSource, connection) + def firstBatch = createCommandResult() + def cursor = new AsyncCommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connection) then: nextBatch(cursor) @@ -424,39 +436,35 @@ class AsyncQueryBatchCursorSpecification extends Specification { thrown(MongoException) then: + connection.getCount() == 0 connectionSource.getCount() == 1 - - when: - cursor.close() - - then: - connectionSource.getCount() == 0 } def 'should handle errors when calling getMore'() { given: def serverVersion = new ServerVersion([3, 6, 0]) - def connectionA = referenceCountedAsyncConnection(serverVersion) - def connectionB = referenceCountedAsyncConnection(serverVersion) + def connectionA = referenceCountedAsyncConnection(serverVersion, 'connectionA') + def connectionB = referenceCountedAsyncConnection(serverVersion, 'connectionB') def connectionSource = getAsyncConnectionSource(connectionA, connectionB) when: - def cursor = new AsyncQueryBatchCursor(queryResult([]), 0, 0, 0, CODEC, null, connectionSource, - connectionA) + def firstBatch = createCommandResult() + def cursor = new AsyncCommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connectionA) then: connectionSource.getCount() == 1 when: nextBatch(cursor) + nextBatch(cursor) then: 1 * connectionA.commandAsync(*_) >> { connectionA.getCount() == 1 connectionSource.getCount() == 1 - it.last().onResult(null, exception) + it.last().onResult(null, exception) } - then: thrown(MongoException) @@ -468,13 +476,22 @@ class AsyncQueryBatchCursorSpecification extends Specification { cursor.close() then: - connectionSource.getCount() == 1 + 1 * connectionB.commandAsync(*_) >> { + connectionB.getCount() == 1 + connectionSource.getCount() == 1 + it.last().onResult(null, null) + } + + then: + connectionA.getCount() == 0 + connectionB.getCount() == 0 + connectionSource.getCount() == 0 where: exception << [COMMAND_EXCEPTION, MONGO_EXCEPTION] } - List nextBatch(AsyncQueryBatchCursor cursor) { + List nextBatch(AsyncCommandBatchCursor cursor) { def futureResultCallback = new FutureResultCallback() cursor.next(futureResultCallback) futureResultCallback.get() @@ -490,25 +507,32 @@ class AsyncQueryBatchCursorSpecification extends Specification { private static final COMMAND_EXCEPTION = new MongoCommandException(BsonDocument.parse('{"ok": false, "errmsg": "error"}'), SERVER_ADDRESS) - private static BsonDocument documentResponse(results, cursorId = 42) { - new BsonDocument('ok', new BsonInt32(1)).append('cursor', - new BsonDocument('id', new BsonInt64(cursorId)).append('ns', - new BsonString(NAMESPACE.getFullName())) - .append('nextBatch', new BsonArrayWrapper(results))) + private static BsonDocument getMoreResponse(results, cursorId = CURSOR_ID) { + createCommandResult(results, cursorId, "nextBatch") } - private static QueryResult queryResult(results = FIRST_BATCH, cursorId = 42) { - new QueryResult(NAMESPACE, results, cursorId, SERVER_ADDRESS) + private static BsonDocument createCommandResult(List results = FIRST_BATCH, Long cursorId = CURSOR_ID, + String fieldNameContainingBatch = "firstBatch") { + new BsonDocument("ok", new BsonInt32(1)) + .append("cursor", + new BsonDocument("ns", new BsonString(NAMESPACE.fullName)) + .append("id", new BsonInt64(cursorId)) + .append(fieldNameContainingBatch, new BsonArrayWrapper(results))) } - def referenceCountedAsyncConnection() { + private static BsonDocument createKillCursorsDocument(ServerCursor serverCursor) { + new BsonDocument('killCursors', new BsonString(NAMESPACE.getCollectionName())) + .append('cursors', new BsonArray(Collections.singletonList(new BsonInt64(serverCursor.id)))) + } + + AsyncConnection referenceCountedAsyncConnection() { referenceCountedAsyncConnection(new ServerVersion([3, 2, 0])) } - def referenceCountedAsyncConnection(ServerVersion serverVersion) { + AsyncConnection referenceCountedAsyncConnection(ServerVersion serverVersion, String name = 'connection') { def released = false def counter = 0 - def mock = Mock(AsyncConnection) { + def mock = Mock(AsyncConnection, name: name) { _ * getDescription() >> Stub(ConnectionDescription) { getMaxWireVersion() >> getMaxWireVersionForServerVersion(serverVersion.getVersionList()) } @@ -581,10 +605,4 @@ class AsyncQueryBatchCursorSpecification extends Specification { mock.getCount() >> { counter } mock } - - BsonDocument createKillCursorsDocument(ServerCursor serverCursor) { - new BsonDocument('killCursors', new BsonString(NAMESPACE.getCollectionName())) - .append('cursors', new BsonArray(Collections.singletonList(new BsonInt64(serverCursor.id)))) - } - } diff --git a/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchQueryCursorSpecification.groovy b/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorSpecification.groovy similarity index 68% rename from driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchQueryCursorSpecification.groovy rename to driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorSpecification.groovy index 22f9035404f..ebc007b987c 100644 --- a/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchQueryCursorSpecification.groovy +++ b/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorSpecification.groovy @@ -17,27 +17,24 @@ package com.mongodb.internal.operation import com.mongodb.MongoException -import com.mongodb.MongoNamespace -import com.mongodb.ServerAddress import com.mongodb.async.FutureResultCallback -import com.mongodb.internal.connection.QueryResult import org.bson.Document import spock.lang.Specification -class AsyncSingleBatchQueryCursorSpecification extends Specification { +class AsyncSingleBatchCursorSpecification extends Specification { def 'should work as expected'() { given: - def cursor = new AsyncSingleBatchQueryCursor(firstBatch) + def cursor = new AsyncSingleBatchCursor(firstBatch, 0) when: def batch = nextBatch(cursor) then: - batch == firstBatch.getResults() + batch == firstBatch then: - nextBatch(cursor) == null + nextBatch(cursor) == [] when: nextBatch(cursor) @@ -48,7 +45,7 @@ class AsyncSingleBatchQueryCursorSpecification extends Specification { def 'should not support setting batchsize'() { given: - def cursor = new AsyncSingleBatchQueryCursor(firstBatch) + def cursor = new AsyncSingleBatchCursor(firstBatch, 0) when: cursor.setBatchSize(1) @@ -58,11 +55,11 @@ class AsyncSingleBatchQueryCursorSpecification extends Specification { } - List nextBatch(AsyncSingleBatchQueryCursor cursor) { + List nextBatch(AsyncSingleBatchCursor cursor) { def futureResultCallback = new FutureResultCallback() cursor.next(futureResultCallback) futureResultCallback.get() } - def firstBatch = new QueryResult(new MongoNamespace('db', 'coll'), [new Document('a', 1)], 0, new ServerAddress()) + def firstBatch = [new Document('a', 1)] } diff --git a/driver-core/src/test/unit/com/mongodb/internal/operation/ChangeStreamBatchCursorSpecification.groovy b/driver-core/src/test/unit/com/mongodb/internal/operation/ChangeStreamBatchCursorSpecification.groovy index e654c2ef5ca..3d5219c15f5 100644 --- a/driver-core/src/test/unit/com/mongodb/internal/operation/ChangeStreamBatchCursorSpecification.groovy +++ b/driver-core/src/test/unit/com/mongodb/internal/operation/ChangeStreamBatchCursorSpecification.groovy @@ -23,11 +23,11 @@ import spock.lang.Specification class ChangeStreamBatchCursorSpecification extends Specification { - def 'should call the underlying QueryBatchCursor'() { + def 'should call the underlying CommandBatchCursor'() { given: def changeStreamOperation = Stub(ChangeStreamOperation) def binding = Stub(ReadBinding) - def wrapped = Mock(QueryBatchCursor) + def wrapped = Mock(CommandBatchCursor) def resumeToken = new BsonDocument('_id': new BsonInt32(1)) def cursor = new ChangeStreamBatchCursor(changeStreamOperation, wrapped, binding, resumeToken, ServerVersionHelper.FOUR_DOT_FOUR_WIRE_VERSION) diff --git a/driver-core/src/test/unit/com/mongodb/internal/operation/CommandBatchCursorSpecification.groovy b/driver-core/src/test/unit/com/mongodb/internal/operation/CommandBatchCursorSpecification.groovy new file mode 100644 index 00000000000..a5538500b75 --- /dev/null +++ b/driver-core/src/test/unit/com/mongodb/internal/operation/CommandBatchCursorSpecification.groovy @@ -0,0 +1,645 @@ +/* + * Copyright 2008-present MongoDB, 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.mongodb.internal.operation + +import com.mongodb.MongoCommandException +import com.mongodb.MongoException +import com.mongodb.MongoNamespace +import com.mongodb.MongoSocketException +import com.mongodb.MongoSocketOpenException +import com.mongodb.ServerAddress +import com.mongodb.ServerCursor +import com.mongodb.connection.ConnectionDescription +import com.mongodb.connection.ServerConnectionState +import com.mongodb.connection.ServerDescription +import com.mongodb.connection.ServerType +import com.mongodb.connection.ServerVersion +import com.mongodb.internal.binding.ConnectionSource +import com.mongodb.internal.connection.Connection +import org.bson.BsonArray +import org.bson.BsonDocument +import org.bson.BsonInt32 +import org.bson.BsonInt64 +import org.bson.BsonString +import org.bson.Document +import org.bson.codecs.DocumentCodec +import spock.lang.Specification + +import static com.mongodb.ReadPreference.primary +import static com.mongodb.internal.operation.OperationUnitSpecification.getMaxWireVersionForServerVersion + +class CommandBatchCursorSpecification extends Specification { + + def 'should generate expected command with batchSize and maxTimeMS'() { + given: + def connection = referenceCountedConnection() + def connectionSource = getConnectionSource(connection) + + def firstBatch = createCommandResult([]) + def cursor = new CommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, batchSize, maxTimeMS, CODEC, + null, connectionSource, connection) + def expectedCommand = new BsonDocument('getMore': new BsonInt64(CURSOR_ID)) + .append('collection', new BsonString(NAMESPACE.getCollectionName())) + if (batchSize != 0) { + expectedCommand.append('batchSize', new BsonInt32(batchSize)) + } + if (expectedMaxTimeFieldValue != null) { + expectedCommand.append('maxTimeMS', new BsonInt64(expectedMaxTimeFieldValue)) + } + + def reply = getMoreResponse([], 0) + + when: + cursor.hasNext() + + then: + 1 * connection.command(NAMESPACE.getDatabaseName(), expectedCommand, *_) >> reply + + then: + !cursor.isClosed() + + then: + cursor.close() + + then: + connection.getCount() == 0 + connectionSource.getCount() == 0 + + where: + batchSize | maxTimeMS | expectedMaxTimeFieldValue + 0 | 0 | null + 2 | 0 | null + 0 | 100 | 100 + } + + def 'should close the cursor'() { + given: + def serverVersion = new ServerVersion([3, 6, 0]) + def connection = referenceCountedConnection(serverVersion) + def connectionSource = getConnectionSource(connection) + def cursor = new CommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connection) + + when: + cursor.close() + + then: + if (cursor.getServerCursor() != null) { + 1 * connection.command(NAMESPACE.databaseName, createKillCursorsDocument(cursor.getServerCursor()), _, primary(), *_) + } + + then: + connection.getCount() == 0 + connectionSource.getCount() == 0 + + where: + firstBatch << [createCommandResult(FIRST_BATCH, 42), createCommandResult(FIRST_BATCH, 0)] + } + + def 'should return the expected results from next'() { + given: + def connection = referenceCountedConnection() + def connectionSource = getConnectionSource(connection) + + when: + def firstBatch = createCommandResult(FIRST_BATCH, 0) + def cursor = new CommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connection) + + then: + cursor.next() == FIRST_BATCH + + then: + connectionSource.getCount() == 0 + + then: + // Unlike the AsyncCommandBatchCursor - the cursor isn't automatically closed + !cursor.isClosed() + } + + def 'should respect the limit'() { + given: + def serverVersion = new ServerVersion([3, 6, 0]) + def connectionA = referenceCountedConnection(serverVersion, 'connectionA') + def connectionB = referenceCountedConnection(serverVersion, 'connectionB') + def connectionSource = getConnectionSource(connectionA, connectionB) + + def firstBatch = [new Document('_id', 1), new Document('_id', 2), new Document('_id', 3)] + def secondBatch = [new Document('_id', 4), new Document('_id', 5), new Document('_id', 6)] + def thirdBatch = [new Document('_id', 7)] + + when: + def cursor = new CommandBatchCursor(SERVER_ADDRESS, createCommandResult(firstBatch), 7, 2, 0, CODEC, + null, connectionSource, connectionA) + def batch = cursor.next() + + then: + batch == firstBatch + + when: + batch = cursor.next() + + then: + 1 * connectionA.command(*_) >> getMoreResponse(secondBatch) + + then: + batch == secondBatch + connectionA.getCount() == 0 + connectionSource.getCount() == 1 + + when: + batch = cursor.next() + + then: + 1 * connectionB.command(*_) >> { + connectionB.getCount() == 1 + connectionSource.getCount() == 1 + getMoreResponse(thirdBatch, 0) + } + + then: + batch == thirdBatch + connectionB.getCount() == 0 + connectionSource.getCount() == 0 + !cursor.hasNext() + } + + + def 'should close the cursor immediately if the limit has been reached'() { + given: + def serverVersion = new ServerVersion([3, 6, 0]) + def connection = referenceCountedConnection(serverVersion) + def connectionSource = getConnectionSource(connection) + def firstBatch = createCommandResult() + + when: + def cursor = new CommandBatchCursor(SERVER_ADDRESS, firstBatch, 1, 0, 0, CODEC, + null, connectionSource, connection) + + then: + 1 * connection.command(NAMESPACE.databaseName, + createKillCursorsDocument(new ServerCursor(42, SERVER_ADDRESS)), _, primary(), *_) >> null + + when: + cursor.close() + + then: + 0 * connection.command(_, _, _, _, _) + + then: + connection.getCount() == 0 + connectionSource.getCount() == 0 + } + + def 'should handle getMore when there are empty results but there is a cursor'() { + given: + def connectionA = referenceCountedConnection(serverVersion, 'connectionA') + def connectionB = referenceCountedConnection(serverVersion, 'connectionB') + def connectionSource = getConnectionSource(connectionA, connectionB) + + when: + def firstBatch = createCommandResult([], CURSOR_ID) + def cursor = new CommandBatchCursor(SERVER_ADDRESS, firstBatch, 3, 0, 0, CODEC, + null, connectionSource, connectionA) + def batch = cursor.next() + + then: + 1 * connectionA.command(*_) >> { + connectionA.getCount() == 1 + connectionSource.getCount() == 1 + response + } + + 1 * connectionB.command(*_) >> { + connectionB.getCount() == 1 + connectionSource.getCount() == 1 + response2 + } + + then: + batch == SECOND_BATCH + + then: + connectionA.getCount() == 0 + connectionB.getCount() == 0 + connectionSource.getCount() == 0 + + when: + cursor.close() + + then: + 0 * connectionA._ + 0 * connectionB._ + connectionSource.getCount() == 0 + + where: + serverVersion | response | response2 + new ServerVersion([3, 6, 0]) | getMoreResponse([]) | getMoreResponse(SECOND_BATCH, 0) + } + + def 'should kill the cursor in the getMore if limit is reached'() { + given: + def connection = referenceCountedConnection(serverVersion) + def connectionSource = getConnectionSource(connection) + def firstBatch = createCommandResult() + + when: + def cursor = new CommandBatchCursor(SERVER_ADDRESS, firstBatch, 3, 0, 0, CODEC, + null, connectionSource, connection) + def batch = cursor.next() + + then: + batch == FIRST_BATCH + + when: + cursor.next() + + then: + 1 * connection.command(*_) >> response + 1 * connection.command(NAMESPACE.databaseName, createKillCursorsDocument(cursor.getServerCursor()), _, primary(), _, + connectionSource, *_) >> null + + then: + connection.getCount() == 0 + connectionSource.getCount() == 0 + + when: + cursor.close() + + then: + 0 * connection.command(*_) + connectionSource.getCount() == 0 + + where: + serverVersion | response + new ServerVersion([3, 2, 0]) | getMoreResponse(SECOND_BATCH) + } + + def 'should close cursor after getMore finishes if cursor was closed while getMore was in progress and getMore returns a response'() { + given: + def serverVersion = new ServerVersion([3, 6, 0]) + def connectionA = referenceCountedConnection(serverVersion, 'connectionA') + def connectionB = referenceCountedConnection(serverVersion, 'connectionB') + def connectionSource = getConnectionSource(serverType, connectionA, connectionB) + def firstBatch = createCommandResult() + + when: + CommandBatchCursor cursor = new CommandBatchCursor<>(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connectionA) + List batch = cursor.next() + + then: + batch == FIRST_BATCH + + when: + cursor.next() + + then: + // simulate the user calling `close` while `getMore` is in flight + // in LB mode the same connection is used to execute both `getMore` and `killCursors` + numberOfInvocations * connectionA.command(*_) >> { + // `getMore` command + cursor.close() + response + } >> { + // `killCursors` command + null + } + + then: + IllegalStateException e = thrown() + e.getMessage() == 'Cursor has been closed' + + then: + connectionA.getCount() == 0 + connectionB.getCount() == 0 + connectionSource.getCount() == 0 + cursor.isClosed() + + where: + response | serverType | numberOfInvocations + getMoreResponse([]) | ServerType.LOAD_BALANCER | 2 + getMoreResponse([], 0) | ServerType.LOAD_BALANCER | 1 + getMoreResponse([]) | ServerType.STANDALONE | 2 + getMoreResponse([], 0) | ServerType.STANDALONE | 1 + } + + def 'should close cursor after getMore finishes if cursor was closed while getMore was in progress and getMore throws exception'() { + given: + def serverVersion = new ServerVersion([4, 4, 0]) + def connectionA = referenceCountedConnection(serverVersion, 'connectionA') + def connectionB = referenceCountedConnection(serverVersion, 'connectionB') + def connectionSource = getConnectionSource(serverType, connectionA, connectionB) + def firstBatch = createCommandResult() + + when: + def cursor = new CommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connectionA) + def batch = cursor.next() + + then: + batch == FIRST_BATCH + + when: + cursor.next() + + then: + numberOfInvocations * connectionA.command(*_) >> { + // Simulate the user calling close while the getMore is throwing a MongoException + cursor.close() + throw MONGO_EXCEPTION + } >> { + // `killCursors` command + null + } + + then: + thrown(MongoException) + + then: + connectionA.getCount() == 0 + cursor.isClosed() + + where: + serverType | numberOfInvocations + ServerType.LOAD_BALANCER | 2 + ServerType.STANDALONE | 1 + } + + def 'should handle errors when calling close'() { + given: + def connection = referenceCountedConnection() + def connectionSource = getConnectionSourceWithResult(ServerType.STANDALONE) { throw MONGO_EXCEPTION } + def firstBatch = createCommandResult() + def cursor = new CommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connection) + + when: + cursor.close() + + then: + cursor.isClosed() + connectionSource.getCount() == 0 + } + + + def 'should handle errors when getting a connection for getMore'() { + given: + def connection = referenceCountedConnection() + def connectionSource = getConnectionSourceWithResult(ServerType.STANDALONE) { throw MONGO_EXCEPTION } + + when: + def firstBatch = createCommandResult() + def cursor = new CommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connection) + + then: + cursor.next() + + when: + cursor.hasNext() + + then: + thrown(MongoException) + + then: + connection.getCount() == 0 + connectionSource.getCount() == 1 + } + + def 'should handle errors when calling getMore'() { + given: + def serverVersion = new ServerVersion([3, 6, 0]) + def connectionA = referenceCountedConnection(serverVersion, 'connectionA') + def connectionB = referenceCountedConnection(serverVersion, 'connectionB') + def connectionSource = getConnectionSource(connectionA, connectionB) + + when: + def firstBatch = createCommandResult() + def cursor = new CommandBatchCursor(SERVER_ADDRESS, firstBatch, 0, 0, 0, CODEC, + null, connectionSource, connectionA) + + then: + connectionSource.getCount() == 1 + + when: + cursor.next() + cursor.next() + + then: + 1 * connectionA.command(*_) >> { + connectionA.getCount() == 1 + connectionSource.getCount() == 1 + throw exception + } + + then: + thrown(MongoException) + + then: + connectionA.getCount() == 0 + connectionSource.getCount() == 1 + + when: + cursor.close() + + then: + 1 * connectionB.command(*_) >> { + connectionB.getCount() == 1 + connectionSource.getCount() == 1 + null + } + + then: + connectionA.getCount() == 0 + connectionB.getCount() == 0 + connectionSource.getCount() == 0 + + where: + exception << [COMMAND_EXCEPTION, MONGO_EXCEPTION] + } + + def 'should handle exceptions when closing'() { + given: + def connection = Mock(Connection) { + _ * getDescription() >> Stub(ConnectionDescription) { + getMaxWireVersion() >> 4 + } + _ * command(*_) >> { throw new MongoSocketException('No MongoD', SERVER_ADDRESS) } + } + def connectionSource = Stub(ConnectionSource) { + getServerApi() >> null + getConnection() >> { connection } + } + connectionSource.retain() >> connectionSource + + def initialResults = createCommandResult([]) + def cursor = new CommandBatchCursor(SERVER_ADDRESS, initialResults, 0, 2, 100, new DocumentCodec(), + null, connectionSource, connection) + + when: + cursor.close() + + then: + notThrown(MongoSocketException) + + when: + cursor.close() + + then: + notThrown(Exception) + } + + def 'should handle exceptions when killing cursor and a connection can not be obtained'() { + given: + def connection = Mock(Connection) { + _ * getDescription() >> Stub(ConnectionDescription) { + getMaxWireVersion() >> 4 + } + } + def connectionSource = Stub(ConnectionSource) { + getConnection() >> { throw new MongoSocketOpenException("can't open socket", SERVER_ADDRESS, new IOException()) } + getServerApi() >> null + } + connectionSource.retain() >> connectionSource + + def initialResults = createCommandResult([]) + def cursor = new CommandBatchCursor(SERVER_ADDRESS, initialResults, 0, 2, 100, new DocumentCodec(), + null, connectionSource, connection) + + when: + cursor.close() + + then: + notThrown(MongoSocketException) + + when: + cursor.close() + + then: + notThrown(Exception) + } + + private static final MongoNamespace NAMESPACE = new MongoNamespace('db', 'coll') + private static final ServerAddress SERVER_ADDRESS = new ServerAddress() + private static final CURSOR_ID = 42 + private static final FIRST_BATCH = [new Document('_id', 1), new Document('_id', 2)] + private static final SECOND_BATCH = [new Document('_id', 3), new Document('_id', 4)] + private static final CODEC = new DocumentCodec() + private static final MONGO_EXCEPTION = new MongoException('error') + private static final COMMAND_EXCEPTION = new MongoCommandException(BsonDocument.parse('{"ok": false, "errmsg": "error"}'), + SERVER_ADDRESS) + + private static BsonDocument getMoreResponse(results, cursorId = CURSOR_ID) { + createCommandResult(results, cursorId, "nextBatch") + } + + private static BsonDocument createCommandResult(List results = FIRST_BATCH, Long cursorId = CURSOR_ID, + String fieldNameContainingBatch = "firstBatch") { + new BsonDocument("ok", new BsonInt32(1)) + .append("cursor", + new BsonDocument("ns", new BsonString(NAMESPACE.fullName)) + .append("id", new BsonInt64(cursorId)) + .append(fieldNameContainingBatch, new BsonArrayWrapper(results))) + } + + private static BsonDocument createKillCursorsDocument(ServerCursor serverCursor) { + new BsonDocument('killCursors', new BsonString(NAMESPACE.getCollectionName())) + .append('cursors', new BsonArray(Collections.singletonList(new BsonInt64(serverCursor.id)))) + } + + Connection referenceCountedConnection() { + referenceCountedConnection(new ServerVersion([3, 2, 0])) + } + + Connection referenceCountedConnection(ServerVersion serverVersion, String name = 'connection') { + def released = false + def counter = 0 + def mock = Mock(Connection, name: name) { + _ * getDescription() >> Stub(ConnectionDescription) { + getMaxWireVersion() >> getMaxWireVersionForServerVersion(serverVersion.getVersionList()) + } + } + mock.retain() >> { + if (released) { + throw new IllegalStateException('Tried to retain Connection when already released') + } else { + counter += 1 + } + mock + } + mock.release() >> { + counter -= 1 + if (counter == 0) { + released = true + } else if (counter < 0) { + throw new IllegalStateException('Tried to release Connection below 0') + } + counter + } + mock.getCount() >> { counter } + mock + } + + ConnectionSource getConnectionSource(Connection... connections) { + getConnectionSource(ServerType.STANDALONE, connections) + } + + ConnectionSource getConnectionSource(ServerType serverType, Connection... connections) { + def index = -1 + getConnectionSourceWithResult(serverType) { index += 1; connections.toList().get(index).retain() } + } + + def getConnectionSourceWithResult(ServerType serverType, Closure connectionCallbackResults) { + def released = false + int counter = 0 + def mock = Mock(ConnectionSource) + mock.getServerDescription() >> { + ServerDescription.builder() + .address(new ServerAddress()) + .type(serverType) + .state(ServerConnectionState.CONNECTED) + .build() + } + mock.getConnection() >> { + if (counter == 0) { + throw new IllegalStateException('Tried to use released ConnectionSource') + } + connectionCallbackResults() + } + mock.retain() >> { + if (released) { + throw new IllegalStateException('Tried to retain ConnectionSource when already released') + } else { + counter += 1 + } + mock + } + mock.release() >> { + counter -= 1 + if (counter == 0) { + released = true + } else if (counter < 0) { + throw new IllegalStateException('Tried to release ConnectionSource below 0') + } + counter + } + mock.getCount() >> { counter } + mock + } + +} diff --git a/driver-core/src/test/unit/com/mongodb/internal/operation/QueryBatchCursorSpecification.groovy b/driver-core/src/test/unit/com/mongodb/internal/operation/QueryBatchCursorSpecification.groovy deleted file mode 100644 index db6831138e1..00000000000 --- a/driver-core/src/test/unit/com/mongodb/internal/operation/QueryBatchCursorSpecification.groovy +++ /dev/null @@ -1,357 +0,0 @@ -/* - * Copyright 2008-present MongoDB, 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.mongodb.internal.operation - -import com.mongodb.MongoException -import com.mongodb.MongoNamespace -import com.mongodb.MongoSocketException -import com.mongodb.MongoSocketOpenException -import com.mongodb.ServerAddress -import com.mongodb.connection.ConnectionDescription -import com.mongodb.connection.ServerConnectionState -import com.mongodb.connection.ServerDescription -import com.mongodb.connection.ServerType -import com.mongodb.connection.ServerVersion -import com.mongodb.internal.binding.ConnectionSource -import com.mongodb.internal.connection.Connection -import com.mongodb.internal.connection.QueryResult -import org.bson.BsonDocument -import org.bson.BsonInt32 -import org.bson.BsonInt64 -import org.bson.BsonString -import org.bson.Document -import org.bson.codecs.BsonDocumentCodec -import org.bson.codecs.DocumentCodec -import spock.lang.Specification - -import static com.mongodb.internal.operation.OperationUnitSpecification.getMaxWireVersionForServerVersion - -class QueryBatchCursorSpecification extends Specification { - private static final MongoNamespace NAMESPACE = new MongoNamespace('db', 'coll') - private static final ServerAddress SERVER_ADDRESS = new ServerAddress() - - def 'should generate expected command with batchSize and maxTimeMS'() { - given: - def connection = Mock(Connection) { - _ * getDescription() >> Stub(ConnectionDescription) { - getMaxWireVersion() >> 4 - } - } - def connectionSource = Stub(ConnectionSource) { - getConnection() >> { connection } - getServerApi() >> null - } - connectionSource.retain() >> connectionSource - - def cursorId = 42 - - def firstBatch = new QueryResult(NAMESPACE, [], cursorId, SERVER_ADDRESS) - def cursor = new QueryBatchCursor(firstBatch, 0, batchSize, maxTimeMS, new BsonDocumentCodec(), null, connectionSource, - connection) - def expectedCommand = new BsonDocument('getMore': new BsonInt64(cursorId)) - .append('collection', new BsonString(NAMESPACE.getCollectionName())) - if (batchSize != 0) { - expectedCommand.append('batchSize', new BsonInt32(batchSize)) - } - if (expectedMaxTimeFieldValue != null) { - expectedCommand.append('maxTimeMS', new BsonInt64(expectedMaxTimeFieldValue)) - } - - def reply = new BsonDocument('ok', new BsonInt32(1)) - .append('cursor', - new BsonDocument('id', new BsonInt64(0)) - .append('ns', new BsonString(NAMESPACE.getFullName())) - .append('nextBatch', new BsonArrayWrapper([]))) - - when: - cursor.hasNext() - - then: - 1 * connection.command(NAMESPACE.getDatabaseName(), expectedCommand, _, _, _, connectionSource) >> { - reply - } - 1 * connection.release() - - where: - batchSize | maxTimeMS | expectedMaxTimeFieldValue - 0 | 0 | null - 2 | 0 | null - 0 | 100 | 100 - } - - def 'should handle exceptions when closing'() { - given: - def connection = Mock(Connection) { - _ * getDescription() >> Stub(ConnectionDescription) { - getMaxWireVersion() >> 4 - } - _ * command(*_) >> { throw new MongoSocketException('No MongoD', SERVER_ADDRESS) } - } - def connectionSource = Stub(ConnectionSource) { - getServerApi() >> null - getConnection() >> { connection } - } - connectionSource.retain() >> connectionSource - - def firstBatch = new QueryResult(NAMESPACE, [], 42, SERVER_ADDRESS) - def cursor = new QueryBatchCursor(firstBatch, 0, 2, 100, new DocumentCodec(), null, connectionSource, connection) - - when: - cursor.close() - - then: - notThrown(MongoSocketException) - - when: - cursor.close() - - then: - notThrown(Exception) - } - - def 'should handle exceptions when killing cursor and a connection can not be obtained'() { - given: - def connection = Mock(Connection) { - _ * getDescription() >> Stub(ConnectionDescription) { - getMaxWireVersion() >> 4 - } - } - def connectionSource = Stub(ConnectionSource) { - getConnection() >> { throw new MongoSocketOpenException("can't open socket", SERVER_ADDRESS, new IOException()) } - getServerApi() >> null - } - connectionSource.retain() >> connectionSource - - def firstBatch = new QueryResult(NAMESPACE, [], 42, SERVER_ADDRESS) - def cursor = new QueryBatchCursor(firstBatch, 0, 2, 100, new DocumentCodec(), null, connectionSource, connection) - - when: - cursor.close() - - then: - notThrown(MongoSocketException) - - when: - cursor.close() - - then: - notThrown(Exception) - } - - def 'should close cursor after getMore finishes if cursor was closed while getMore was in progress and getMore returns a response'() { - given: - Connection conn = mockConnection(serverVersion) - ConnectionSource connSource - if (serverType == ServerType.LOAD_BALANCER) { - connSource = mockConnectionSource(SERVER_ADDRESS, serverType) - } else { - connSource = mockConnectionSource(SERVER_ADDRESS, serverType, conn, mockConnection(serverVersion)) - } - List firstBatch = [new Document()] - QueryResult initialResult = new QueryResult<>(NAMESPACE, firstBatch, 1, SERVER_ADDRESS) - Object getMoreResponse = useCommand - ? emptyGetMoreCommandResponse(NAMESPACE, getMoreResponseHasCursor ? 42 : 0) - : emptyGetMoreQueryResponse(NAMESPACE, SERVER_ADDRESS, getMoreResponseHasCursor ? 42 : 0) - - when: - QueryBatchCursor cursor = new QueryBatchCursor<>(initialResult, 0, 0, 0, new DocumentCodec(), null, connSource, conn) - List batch = cursor.next() - - then: - batch == firstBatch - - when: - cursor.next() - - then: - // simulate the user calling `close` while `getMore` is in flight - if (useCommand) { - // in LB mode the same connection is used to execute both `getMore` and `killCursors` - int numberOfInvocations = serverType == ServerType.LOAD_BALANCER - ? getMoreResponseHasCursor ? 2 : 1 - : 1 - numberOfInvocations * conn.command(*_) >> { - // `getMore` command - cursor.close() - getMoreResponse - } >> { - // `killCursors` command - null - } - } else { - 1 * conn.getMore(*_) >> { - cursor.close() - getMoreResponse - } - } - - then: - IllegalStateException e = thrown() - e.getMessage() == 'Cursor has been closed' - - then: - conn.getCount() == 1 - connSource.getCount() == 1 - - where: - serverVersion | useCommand | getMoreResponseHasCursor | serverType - new ServerVersion([5, 0, 0]) | true | true | ServerType.LOAD_BALANCER - new ServerVersion([5, 0, 0]) | true | false | ServerType.LOAD_BALANCER - new ServerVersion([3, 2, 0]) | true | true | ServerType.STANDALONE - new ServerVersion([3, 2, 0]) | true | false | ServerType.STANDALONE - } - - def 'should close cursor after getMore finishes if cursor was closed while getMore was in progress and getMore throws exception'() { - given: - Connection conn = mockConnection(serverVersion) - ConnectionSource connSource - if (serverType == ServerType.LOAD_BALANCER) { - connSource = mockConnectionSource(SERVER_ADDRESS, serverType) - } else { - connSource = mockConnectionSource(SERVER_ADDRESS, serverType, conn, mockConnection(serverVersion)) - } - List firstBatch = [new Document()] - QueryResult initialResult = new QueryResult<>(NAMESPACE, firstBatch, 1, SERVER_ADDRESS) - String exceptionMessage = 'test' - - when: - QueryBatchCursor cursor = new QueryBatchCursor<>(initialResult, 0, 0, 0, new DocumentCodec(), null, connSource, conn) - List batch = cursor.next() - - then: - batch == firstBatch - - when: - cursor.next() - - then: - // simulate the user calling `close` while `getMore` is in flight - if (useCommand) { - // in LB mode the same connection is used to execute both `getMore` and `killCursors` - int numberOfInvocations = serverType == ServerType.LOAD_BALANCER ? 2 : 1 - numberOfInvocations * conn.command(*_) >> { - // `getMore` command - cursor.close() - throw new MongoException(exceptionMessage) - } >> { - // `killCursors` command - null - } - } else { - 1 * conn.getMore(*_) >> { - cursor.close() - throw new MongoException(exceptionMessage) - } - } - - then: - MongoException e = thrown() - e.getMessage() == exceptionMessage - - then: - conn.getCount() == 1 - connSource.getCount() == 1 - - where: - serverVersion | useCommand | serverType - new ServerVersion([5, 0, 0]) | true | ServerType.LOAD_BALANCER - new ServerVersion([3, 2, 0]) | true | ServerType.STANDALONE - } - - /** - * Creates a {@link Connection} with {@link Connection#getCount()} returning 1. - */ - private Connection mockConnection(ServerVersion serverVersion) { - int refCounter = 1 - Connection mockConn = Mock(Connection) { - getDescription() >> Stub(ConnectionDescription) { - getMaxWireVersion() >> getMaxWireVersionForServerVersion(serverVersion.getVersionList()) - } - } - mockConn.retain() >> { - if (refCounter == 0) { - throw new IllegalStateException('Tried to retain Connection when already released') - } else { - refCounter += 1 - } - mockConn - } - mockConn.release() >> { - refCounter -= 1 - if (refCounter < 0) { - throw new IllegalStateException('Tried to release Connection below 0') - } - refCounter - } - mockConn.getCount() >> { refCounter } - mockConn - } - - private ConnectionSource mockConnectionSource(ServerAddress serverAddress, ServerType serverType, Connection... connections) { - int connIdx = 0 - int refCounter = 1 - ConnectionSource mockConnectionSource = Mock(ConnectionSource) - mockConnectionSource.getServerDescription() >> { - ServerDescription.builder() - .address(serverAddress) - .type(serverType) - .state(ServerConnectionState.CONNECTED) - .build() - } - mockConnectionSource.retain() >> { - if (refCounter == 0) { - throw new IllegalStateException('Tried to retain ConnectionSource when already released') - } else { - refCounter += 1 - } - mockConnectionSource - } - mockConnectionSource.release() >> { - refCounter -= 1 - if (refCounter < 0) { - throw new IllegalStateException('Tried to release ConnectionSource below 0') - } - refCounter - } - mockConnectionSource.getCount() >> { refCounter } - mockConnectionSource.getConnection() >> { - if (refCounter == 0) { - throw new IllegalStateException('Tried to use released ConnectionSource') - } - Connection conn - if (connIdx < connections.length) { - conn = connections[connIdx] - } else { - throw new IllegalStateException('Requested more than maxConnections=' + maxConnections) - } - connIdx++ - conn.retain() - } - mockConnectionSource - } - - private static BsonDocument emptyGetMoreCommandResponse(MongoNamespace namespace, long cursorId) { - new BsonDocument('ok', new BsonInt32(1)) - .append('cursor', new BsonDocument('id', new BsonInt64(cursorId)) - .append('ns', new BsonString(namespace.getFullName())) - .append('nextBatch', new BsonArrayWrapper([]))) - } - - private static QueryResult emptyGetMoreQueryResponse(MongoNamespace namespace, ServerAddress serverAddress, long cursorId) { - new QueryResult(namespace, [], cursorId, serverAddress) - } -} From aed0c334abf3737c3b5f9195e7cd55a8125ad02c Mon Sep 17 00:00:00 2001 From: Ross Lawley Date: Thu, 28 Sep 2023 11:24:59 +0100 Subject: [PATCH 2/9] Use a volatile variable for CommandCursorResult instead of an atomic ref --- .../internal/operation/AsyncCommandBatchCursor.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java index bc72c363f72..dac408a8b64 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java @@ -44,7 +44,6 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; @@ -78,10 +77,10 @@ class AsyncCommandBatchCursor implements AsyncAggregateResponseBatchCursor private final AtomicInteger count = new AtomicInteger(); private final AtomicBoolean processedInitial = new AtomicBoolean(); - private final AtomicReference> commandCursorResult = new AtomicReference<>(); private final int maxWireVersion; private final boolean firstBatchEmpty; private final ResourceManager resourceManager; + private volatile CommandCursorResult commandCursorResult; private int batchSize; AsyncCommandBatchCursor( @@ -131,7 +130,7 @@ public void next(final SingleResultCallback> callback) { boolean cursorClosed = localServerCursor == null; List batchResults = emptyList(); if (!processedInitial.getAndSet(true) && !firstBatchEmpty) { - batchResults = commandCursorResult.get().getResults(); + batchResults = commandCursorResult.getResults(); } if (cursorClosed || !batchResults.isEmpty()) { @@ -167,12 +166,12 @@ public void close() { @Override public BsonDocument getPostBatchResumeToken() { - return commandCursorResult.get().getPostBatchResumeToken(); + return commandCursorResult.getPostBatchResumeToken(); } @Override public BsonTimestamp getOperationTime() { - return commandCursorResult.get().getOperationTime(); + return commandCursorResult.getOperationTime(); } @Override @@ -249,7 +248,7 @@ private CommandCursorResult initFromCommandCursorDocument( LOGGER.debug(format("Received batch of %d documents with cursorId %d from server %s", cursorResult.getResults().size(), cursorResult.getCursorId(), cursorResult.getServerAddress())); } - this.commandCursorResult.set(cursorResult); + this.commandCursorResult = cursorResult; return cursorResult; } From e19f9460e1ae1605423c0d146d6792f02630a009 Mon Sep 17 00:00:00 2001 From: Ross Lawley Date: Fri, 29 Sep 2023 10:03:40 +0100 Subject: [PATCH 3/9] Shared a CursorResourceManager between async and sync code JAVA-5159 --- .../operation/AsyncCommandBatchCursor.java | 292 +++-------------- .../operation/CommandBatchCursor.java | 300 +++--------------- .../operation/CursorResourceManager.java | 264 +++++++++++++++ ...dBatchCursorFunctionalSpecification.groovy | 18 +- ...dBatchCursorFunctionalSpecification.groovy | 12 +- 5 files changed, 378 insertions(+), 508 deletions(-) create mode 100644 driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java diff --git a/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java index dac408a8b64..0b6695ccd43 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java @@ -19,6 +19,7 @@ import com.mongodb.MongoCommandException; import com.mongodb.MongoException; import com.mongodb.MongoNamespace; +import com.mongodb.MongoSocketException; import com.mongodb.ReadPreference; import com.mongodb.ServerAddress; import com.mongodb.ServerCursor; @@ -32,10 +33,7 @@ import com.mongodb.internal.connection.AsyncConnection; import com.mongodb.internal.connection.Connection; import com.mongodb.lang.Nullable; -import org.bson.BsonArray; import org.bson.BsonDocument; -import org.bson.BsonInt64; -import org.bson.BsonString; import org.bson.BsonTimestamp; import org.bson.BsonValue; import org.bson.codecs.BsonDocumentCodec; @@ -44,14 +42,10 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import static com.mongodb.assertions.Assertions.assertNotNull; -import static com.mongodb.assertions.Assertions.assertNull; import static com.mongodb.assertions.Assertions.assertTrue; -import static com.mongodb.assertions.Assertions.fail; import static com.mongodb.assertions.Assertions.isTrueArgument; import static com.mongodb.assertions.Assertions.notNull; import static com.mongodb.internal.operation.CommandBatchCursorHelper.FIRST_BATCH; @@ -64,7 +58,6 @@ import static com.mongodb.internal.operation.OperationHelper.LOGGER; import static java.lang.String.format; import static java.util.Collections.emptyList; -import static java.util.Collections.singletonList; class AsyncCommandBatchCursor implements AsyncAggregateResponseBatchCursor { @@ -110,9 +103,9 @@ class AsyncCommandBatchCursor implements AsyncAggregateResponseBatchCursor connectionToPin = connection; } - resourceManager = new ResourceManager(connectionSource, connectionToPin, commandCursor.getServerCursor()); + resourceManager = new ResourceManager(namespace, connectionSource, connectionToPin, commandCursor.getServerCursor()); if (releaseServerAndResources) { - resourceManager.releaseServerAndClientResources(connection.retain()); + resourceManager.releaseServerAndClientResources(connection); } } @@ -191,7 +184,7 @@ ServerCursor getServerCursor() { } private void getMore(final ServerCursor cursor, final SingleResultCallback> callback) { - resourceManager.executeWithConnection(callback, connection -> getMore(connection, cursor, callback)); + resourceManager.executeWithConnection(connection -> getMore(connection, cursor, callback), callback); } private void getMore(final AsyncConnection connection, final ServerCursor cursor, final SingleResultCallback> callback) { @@ -219,6 +212,7 @@ private void getMore(final AsyncConnection connection, final ServerCursor cursor if (!resourceManager.operable()) { resourceManager.releaseServerAndClientResources(connection); + connection.release(); callback.onResult(emptyList(), null); return; } @@ -230,9 +224,8 @@ private void getMore(final AsyncConnection connection, final ServerCursor cursor resourceManager.endOperation(); if (limitReached()) { resourceManager.releaseServerAndClientResources(connection); - } else { - connection.release(); } + connection.release(); callback.onResult(commandCursor.getResults(), null); } }); @@ -256,268 +249,85 @@ private boolean limitReached() { return Math.abs(limit) != 0 && count.get() >= Math.abs(limit); } - /** - * This class maintains all resources that must be released in {@link AsyncCommandBatchCursor#close()}. - * It also implements a {@linkplain #doClose() deferred close action} such that it is totally ordered with other operations of - * {@link AsyncCommandBatchCursor} (methods {@link #tryStartOperation()}/{@link #endOperation()} must be used properly to enforce the order) - * despite the method {@link AsyncCommandBatchCursor#close()} being called concurrently with those operations. - * This total order induces the happens-before order. - *

    - * The deferred close action does not violate externally observable idempotence of {@link AsyncCommandBatchCursor#close()}, - * because {@link AsyncCommandBatchCursor#close()} is allowed to release resources "eventually". - *

    - * Only methods explicitly documented as thread-safe are thread-safe, - * others are not and rely on the total order mentioned above. - */ @ThreadSafe - private final class ResourceManager { - private final Lock lock; - private volatile State state = State.IDLE; - @Nullable - private volatile AsyncConnectionSource connectionSource; - @Nullable - private volatile AsyncConnection pinnedConnection; - @Nullable - private volatile ServerCursor serverCursor; - private volatile boolean skipReleasingServerResourcesOnClose; - - ResourceManager(final AsyncConnectionSource connectionSource, - @Nullable final AsyncConnection connectionToPin, - @Nullable final ServerCursor serverCursor) { - lock = new ReentrantLock(); - if (serverCursor != null) { - this.connectionSource = notNull("connectionSource", connectionSource).retain(); - if (connectionToPin != null) { - this.pinnedConnection = connectionToPin.retain(); - connectionToPin.markAsPinned(Connection.PinningMode.CURSOR); - } - } - skipReleasingServerResourcesOnClose = false; - this.serverCursor = serverCursor; - } - - /** - * Thread-safe. - */ - boolean operable() { - return state.operable(); - } - - /** - * Thread-safe. - * Returns {@code true} iff started an operation. - * If {@linkplain State#operable() closed}, then returns false, otherwise completes abruptly. - * - * @throws IllegalStateException Iff another operation is in progress. - */ - private boolean tryStartOperation() throws IllegalStateException { - lock.lock(); - try { - State localState = state; - if (!localState.operable()) { - return false; - } else if (localState == State.IDLE) { - state = State.OPERATION_IN_PROGRESS; - return true; - } else if (localState == State.OPERATION_IN_PROGRESS) { - return false; - } else { - throw fail(state.toString()); - } - } finally { - lock.unlock(); - } - } - - /** - * Thread-safe. - */ - private void endOperation() { - boolean doClose = false; - lock.lock(); - try { - State localState = state; - if (localState == State.OPERATION_IN_PROGRESS) { - state = State.IDLE; - } else if (localState == State.CLOSE_PENDING) { - state = State.CLOSED; - doClose = true; - } else if (localState != State.CLOSED) { - fail(localState.toString()); - } - } finally { - lock.unlock(); - } - if (doClose) { - doClose(); - } - } + private static final class ResourceManager extends CursorResourceManager { - /** - * Thread-safe. - */ - void close() { - boolean doClose = false; - lock.lock(); - try { - State localState = state; - if (localState == State.OPERATION_IN_PROGRESS) { - state = State.CLOSE_PENDING; - } else if (localState != State.CLOSED) { - state = State.CLOSED; - doClose = true; - } - } finally { - lock.unlock(); - } - if (doClose) { - doClose(); - } + ResourceManager( + final MongoNamespace namespace, + final AsyncConnectionSource connectionSource, + @Nullable final AsyncConnection connectionToPin, + @Nullable final ServerCursor serverCursor) { + super(namespace, connectionSource, connectionToPin, serverCursor); } - /** - * This method is never executed concurrently with either itself or other operations - * demarcated by {@link #tryStartOperation()}/{@link #endOperation()}. - */ - private void doClose() { - if (skipReleasingServerResourcesOnClose) { - serverCursor = null; - releaseClientResources(); - } else if (serverCursor != null) { - executeWithConnection((connection, t) -> { - // guarantee that regardless of exceptions, `serverCursor` is null and client resources are released - serverCursor = null; - releaseClientResources(); - }, resourceManager::releaseServerAndClientResources); - } + @Override + void markAsPinned(final AsyncConnection connectionToPin, final Connection.PinningMode pinningMode) { + connectionToPin.markAsPinned(pinningMode); } - void onCorruptedConnection(@Nullable final AsyncConnection corruptedConnection) { - // if `pinnedConnection` is corrupted, then we cannot kill `serverCursor` via such a connection - AsyncConnection localPinnedConnection = pinnedConnection; - if (localPinnedConnection != null) { - assertTrue(corruptedConnection == localPinnedConnection); - skipReleasingServerResourcesOnClose = true; - } + @Override + void executeWithConnection(final Consumer action) { + throw new UnsupportedOperationException(); } - void executeWithConnection( - final SingleResultCallback callback, - final Consumer function) { + @Override + void executeWithConnection(final Consumer action, final SingleResultCallback callback) { assertTrue(state != State.IDLE); if (pinnedConnection != null) { - executeWithConnection(assertNotNull(pinnedConnection).retain(), null, callback, function); + executeWithConnection(assertNotNull(pinnedConnection).retain(), null, action, callback); } else { - assertNotNull(connectionSource).getConnection((conn, t) -> executeWithConnection(conn, t, callback, function)); + assertNotNull(connectionSource).getConnection((conn, t) -> executeWithConnection(conn, t, action, callback)); } } void executeWithConnection( @Nullable final AsyncConnection connection, @Nullable final Throwable t, - final SingleResultCallback callback, - final Consumer function) { + final Consumer function, + final SingleResultCallback callback) { assertTrue(connection != null || t != null); if (t != null) { - try { - onCorruptedConnection(connection); - } catch (Exception suppressed) { - t.addSuppressed(suppressed); - } callback.onResult(null, t); } else { AsyncCallbackSupplier curriedFunction = c -> function.accept(connection); - curriedFunction.whenComplete(connection::release).get(callback); + curriedFunction.whenComplete(() -> { + System.out.println("COMPLETED!!!!"); + connection.release(); + }).get((result, error) -> { + if (error instanceof MongoSocketException) { + onCorruptedConnection(connection); + } + callback.onResult(result, error); + }); } } - /** - * Thread-safe. - */ - @Nullable - ServerCursor serverCursor() { - return serverCursor; - } - - void setServerCursor(@Nullable final ServerCursor serverCursor) { - assertTrue(state.inProgress()); - assertNotNull(connectionSource); - this.serverCursor = serverCursor; - if (serverCursor == null) { - releaseClientResources(); + @Override + void doClose() { + if (skipReleasingServerResourcesOnClose) { + serverCursor = null; } - } - private void releaseServerAndClientResources(final AsyncConnection connection) { - lock.lock(); - ServerCursor localServerCursor = serverCursor; - serverCursor = null; - lock.unlock(); - if (localServerCursor != null) { - killServerCursor(namespace, localServerCursor, connection); + if (serverCursor != null) { + executeWithConnection(conn -> { + releaseServerResources(conn); + conn.release(); + }, (r, t) -> { + // guarantee that regardless of exceptions, `serverCursor` is null and client resources are released + serverCursor = null; + releaseClientResources(); + }); } else { - connection.release(); releaseClientResources(); } } - private void killServerCursor(final MongoNamespace namespace, final ServerCursor serverCursor, final AsyncConnection connection) { + @Override + void killServerCursor(final MongoNamespace namespace, final ServerCursor serverCursor, final AsyncConnection connection) { connection - .commandAsync(namespace.getDatabaseName(), asKillCursorsCommandDocument(namespace, serverCursor), + .commandAsync(namespace.getDatabaseName(), getKillCursorsCommand(namespace, serverCursor), NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), new BsonDocumentCodec(), assertNotNull(connectionSource), - (r, t) -> { - connection.release(); - releaseClientResources(); - }); - } - - private BsonDocument asKillCursorsCommandDocument(final MongoNamespace namespace, final ServerCursor serverCursor) { - return new BsonDocument("killCursors", new BsonString(namespace.getCollectionName())) - .append("cursors", new BsonArray(singletonList(new BsonInt64(serverCursor.getId())))); - } - - private void releaseClientResources() { - assertNull(serverCursor); - lock.lock(); - AsyncConnectionSource localConnectionSource = connectionSource; - connectionSource = null; - - AsyncConnection localPinnedConnection = pinnedConnection; - pinnedConnection = null; - lock.unlock(); - - if (localConnectionSource != null) { - localConnectionSource.release(); - } - if (localPinnedConnection != null) { - localPinnedConnection.release(); - } - } - } - - private enum State { - IDLE(true, false), - OPERATION_IN_PROGRESS(true, true), - /** - * Implies {@link #OPERATION_IN_PROGRESS}. - */ - CLOSE_PENDING(false, true), - CLOSED(false, false); - - private final boolean operable; - private final boolean inProgress; - - State(final boolean operable, final boolean inProgress) { - this.operable = operable; - this.inProgress = inProgress; - } - - boolean operable() { - return operable; - } - - boolean inProgress() { - return inProgress; + (r, t) -> releaseClientResources()); } } } diff --git a/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java index 32da9ab63f6..b77b13adb6e 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java @@ -26,13 +26,11 @@ import com.mongodb.annotations.ThreadSafe; import com.mongodb.connection.ServerType; import com.mongodb.internal.VisibleForTesting; +import com.mongodb.internal.async.SingleResultCallback; import com.mongodb.internal.binding.ConnectionSource; import com.mongodb.internal.connection.Connection; import com.mongodb.lang.Nullable; -import org.bson.BsonArray; import org.bson.BsonDocument; -import org.bson.BsonInt64; -import org.bson.BsonString; import org.bson.BsonTimestamp; import org.bson.BsonValue; import org.bson.codecs.BsonDocumentCodec; @@ -40,29 +38,22 @@ import java.util.List; import java.util.NoSuchElementException; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.StampedLock; import java.util.function.Consumer; import java.util.function.Supplier; import static com.mongodb.assertions.Assertions.assertNotNull; -import static com.mongodb.assertions.Assertions.assertNull; import static com.mongodb.assertions.Assertions.assertTrue; -import static com.mongodb.assertions.Assertions.fail; -import static com.mongodb.assertions.Assertions.isTrueArgument; import static com.mongodb.assertions.Assertions.notNull; import static com.mongodb.internal.VisibleForTesting.AccessModifier.PRIVATE; import static com.mongodb.internal.operation.CommandBatchCursorHelper.FIRST_BATCH; import static com.mongodb.internal.operation.CommandBatchCursorHelper.MESSAGE_IF_CLOSED_AS_CURSOR; import static com.mongodb.internal.operation.CommandBatchCursorHelper.MESSAGE_IF_CLOSED_AS_ITERATOR; -import static com.mongodb.internal.operation.CommandBatchCursorHelper.MESSAGE_IF_CONCURRENT_OPERATION; import static com.mongodb.internal.operation.CommandBatchCursorHelper.NEXT_BATCH; import static com.mongodb.internal.operation.CommandBatchCursorHelper.NO_OP_FIELD_NAME_VALIDATOR; import static com.mongodb.internal.operation.CommandBatchCursorHelper.getMoreCommandDocument; import static com.mongodb.internal.operation.CommandBatchCursorHelper.translateCommandException; import static com.mongodb.internal.operation.OperationHelper.LOGGER; import static java.lang.String.format; -import static java.util.Collections.singletonList; class CommandBatchCursor implements AggregateResponseBatchCursor { @@ -73,7 +64,7 @@ class CommandBatchCursor implements AggregateResponseBatchCursor { @Nullable private final BsonValue comment; - private CommandCursorResult commandCursorResult; + private CommandCursorResult commandCursor; private int batchSize; @Nullable private List nextBatch; @@ -91,27 +82,25 @@ class CommandBatchCursor implements AggregateResponseBatchCursor { @Nullable final BsonValue comment, final ConnectionSource connectionSource, final Connection connection) { - isTrueArgument("maxTimeMS >= 0", maxTimeMS >= 0); - initFromCommandCursorDocument(serverAddress, FIRST_BATCH, commandCursorDocument); - this.namespace = commandCursorResult.getNamespace(); + this.commandCursor = initFromCommandCursorDocument(serverAddress, FIRST_BATCH, commandCursorDocument); + this.namespace = commandCursor.getNamespace(); this.limit = limit; this.batchSize = batchSize; this.maxTimeMS = maxTimeMS; this.decoder = notNull("decoder", decoder); this.comment = comment; + this.maxWireVersion = connection.getDescription().getMaxWireVersion(); + this.firstBatchEmpty = commandCursor.getResults().isEmpty(); - firstBatchEmpty = commandCursorResult.getResults().isEmpty(); Connection connectionToPin = null; boolean releaseServerAndResources = false; - - this.maxWireVersion = connection.getDescription().getMaxWireVersion(); if (limitReached()) { releaseServerAndResources = true; } else if (connectionSource.getServerDescription().getType() == ServerType.LOAD_BALANCER) { connectionToPin = connection; } - resourceManager = new ResourceManager(connectionSource, connectionToPin, commandCursorResult.getServerCursor()); + resourceManager = new ResourceManager(namespace, connectionSource, connectionToPin, commandCursor.getServerCursor()); if (releaseServerAndResources) { resourceManager.releaseServerAndClientResources(connection); } @@ -233,17 +222,17 @@ public ServerAddress getServerAddress() { throw new IllegalStateException(MESSAGE_IF_CLOSED_AS_ITERATOR); } - return commandCursorResult.getServerAddress(); + return commandCursor.getServerAddress(); } @Override public BsonDocument getPostBatchResumeToken() { - return commandCursorResult.getPostBatchResumeToken(); + return commandCursor.getPostBatchResumeToken(); } @Override public BsonTimestamp getOperationTime() { - return commandCursorResult.getOperationTime(); + return commandCursor.getOperationTime(); } @Override @@ -261,7 +250,7 @@ private void getMore() { resourceManager.executeWithConnection(connection -> { ServerCursor nextServerCursor; try { - initFromCommandCursorDocument(connection.getDescription().getServerAddress(), NEXT_BATCH, + commandCursor = initFromCommandCursorDocument(connection.getDescription().getServerAddress(), NEXT_BATCH, assertNotNull( connection.command(namespace.getDatabaseName(), getMoreCommandDocument(serverCursor.getId(), connection.getDescription(), namespace, @@ -270,7 +259,7 @@ private void getMore() { ReadPreference.primary(), CommandResultDocumentCodec.create(decoder, NEXT_BATCH), assertNotNull(resourceManager.connectionSource)))); - nextServerCursor = commandCursorResult.getServerCursor(); + nextServerCursor = commandCursor.getServerCursor(); } catch (MongoCommandException e) { throw translateCommandException(e, serverCursor); } @@ -282,67 +271,32 @@ private void getMore() { }); } - private void initFromCommandCursorDocument(final ServerAddress serverAddress, final String fieldNameContainingBatch, + private CommandCursorResult initFromCommandCursorDocument(final ServerAddress serverAddress, final String fieldNameContainingBatch, final BsonDocument commandCursorDocument) { - this.commandCursorResult = new CommandCursorResult<>(serverAddress, fieldNameContainingBatch, commandCursorDocument); + CommandCursorResult commandCursorResult = new CommandCursorResult<>(serverAddress, fieldNameContainingBatch, + commandCursorDocument); this.nextBatch = commandCursorResult.getResults().isEmpty() ? null : commandCursorResult.getResults(); this.count += commandCursorResult.getResults().size(); if (LOGGER.isDebugEnabled()) { LOGGER.debug(format("Received batch of %d documents with cursorId %d from server %s", commandCursorResult.getResults().size(), commandCursorResult.getCursorId(), commandCursorResult.getServerAddress())); } + return commandCursorResult; } private boolean limitReached() { return Math.abs(limit) != 0 && count >= Math.abs(limit); } - /** - * This class maintains all resources that must be released in {@link CommandBatchCursor#close()}. - * It also implements a {@linkplain #doClose() deferred close action} such that it is totally ordered with other operations of - * {@link CommandBatchCursor} (methods {@link #tryStartOperation()}/{@link #endOperation()} must be used properly to enforce the order) - * despite the method {@link CommandBatchCursor#close()} being called concurrently with those operations. - * This total order induces the happens-before order. - *

    - * The deferred close action does not violate externally observable idempotence of {@link CommandBatchCursor#close()}, - * because {@link CommandBatchCursor#close()} is allowed to release resources "eventually". - *

    - * Only methods explicitly documented as thread-safe are thread-safe, - * others are not and rely on the total order mentioned above. - */ @ThreadSafe - private final class ResourceManager { - private final Lock lock; - private volatile State state; - @Nullable - private volatile ConnectionSource connectionSource; - @Nullable - private volatile Connection pinnedConnection; - @Nullable - private volatile ServerCursor serverCursor; - private volatile boolean skipReleasingServerResourcesOnClose; + private static final class ResourceManager extends CursorResourceManager { - ResourceManager(final ConnectionSource connectionSource, + public ResourceManager( + final MongoNamespace namespace, + final ConnectionSource connectionSource, @Nullable final Connection connectionToPin, @Nullable final ServerCursor serverCursor) { - lock = new StampedLock().asWriteLock(); - state = State.IDLE; - if (serverCursor != null) { - this.connectionSource = notNull("connectionSource", connectionSource).retain(); - if (connectionToPin != null) { - this.pinnedConnection = connectionToPin.retain(); - connectionToPin.markAsPinned(Connection.PinningMode.CURSOR); - } - } - skipReleasingServerResourcesOnClose = false; - this.serverCursor = serverCursor; - } - - /** - * Thread-safe. - */ - boolean operable() { - return state.operable(); + super(namespace, connectionSource, connectionToPin, serverCursor); } /** @@ -363,112 +317,21 @@ R execute(final String exceptionMessageIfClosed, final Supplier operation } } - /** - * Thread-safe. - * Returns {@code true} iff started an operation. - * If {@linkplain #operable() closed}, then returns false, otherwise completes abruptly. - * @throws IllegalStateException Iff another operation is in progress. - */ - private boolean tryStartOperation() throws IllegalStateException { - lock.lock(); - try { - State localState = state; - if (!localState.operable()) { - return false; - } else if (localState == State.IDLE) { - state = State.OPERATION_IN_PROGRESS; - return true; - } else if (localState == State.OPERATION_IN_PROGRESS) { - throw new IllegalStateException(MESSAGE_IF_CONCURRENT_OPERATION); - } else { - throw fail(state.toString()); - } - } finally { - lock.unlock(); - } - } - - /** - * Thread-safe. - */ - private void endOperation() { - boolean doClose = false; - lock.lock(); - try { - State localState = state; - if (localState == State.OPERATION_IN_PROGRESS) { - state = State.IDLE; - } else if (localState == State.CLOSE_PENDING) { - state = State.CLOSED; - doClose = true; - } else { - fail(localState.toString()); - } - } finally { - lock.unlock(); - } - if (doClose) { - doClose(); - } - } - - /** - * Thread-safe. - */ - void close() { - boolean doClose = false; - lock.lock(); - try { - State localState = state; - if (localState == State.OPERATION_IN_PROGRESS) { - state = State.CLOSE_PENDING; - } else if (localState != State.CLOSED) { - state = State.CLOSED; - doClose = true; - } - } finally { - lock.unlock(); - } - if (doClose) { - doClose(); - } - } - - /** - * This method is never executed concurrently with either itself or other operations - * demarcated by {@link #tryStartOperation()}/{@link #endOperation()}. - */ - private void doClose() { - try { - if (skipReleasingServerResourcesOnClose) { - serverCursor = null; - } else if (serverCursor != null) { - Connection connection = connection(); - try { - releaseServerResources(connection); - } finally { - connection.release(); - } - } - } catch (MongoException e) { - // ignore exceptions when releasing server resources - } finally { - // guarantee that regardless of exceptions, `serverCursor` is null and client resources are released - serverCursor = null; - releaseClientResources(); + private Connection connection() { + assertTrue(state != State.IDLE); + if (pinnedConnection != null) { + return assertNotNull(pinnedConnection).retain(); + } else { + return assertNotNull(connectionSource).getConnection(); } } - void onCorruptedConnection(final Connection corruptedConnection) { - assertTrue(state.inProgress()); - // if `pinnedConnection` is corrupted, then we cannot kill `serverCursor` via such a connection - Connection localPinnedConnection = pinnedConnection; - if (localPinnedConnection != null) { - assertTrue(corruptedConnection == localPinnedConnection); - skipReleasingServerResourcesOnClose = true; - } + @Override + void markAsPinned(final Connection connectionToPin, final Connection.PinningMode pinningMode) { + connectionToPin.markAsPinned(pinningMode); } + @Override void executeWithConnection(final Consumer action) { Connection connection = connection(); try { @@ -485,102 +348,35 @@ void executeWithConnection(final Consumer action) { } } - private Connection connection() { - assertTrue(state != State.IDLE); - if (pinnedConnection != null) { - return assertNotNull(pinnedConnection).retain(); - } else { - return assertNotNull(connectionSource).getConnection(); - } - } - - /** - * Thread-safe. - */ - @Nullable - ServerCursor serverCursor() { - return serverCursor; - } - - void setServerCursor(@Nullable final ServerCursor serverCursor) { - assertTrue(state.inProgress()); - assertNotNull(this.serverCursor); - // without `connectionSource` we will not be able to kill `serverCursor` later - assertNotNull(connectionSource); - this.serverCursor = serverCursor; - if (serverCursor == null) { - releaseClientResources(); - } + @Override + void executeWithConnection(final Consumer action, final SingleResultCallback callback) { + throw new UnsupportedOperationException(); } - - void releaseServerAndClientResources(final Connection connection) { - try { - releaseServerResources(assertNotNull(connection)); - } finally { - releaseClientResources(); + @Override + void doClose() { + if (skipReleasingServerResourcesOnClose) { + serverCursor = null; } - } - private void releaseServerResources(final Connection connection) { try { - ServerCursor localServerCursor = serverCursor; - if (localServerCursor != null) { - killServerCursor(namespace, localServerCursor, assertNotNull(connection)); + if (serverCursor != null) { + executeWithConnection(this::releaseServerResources); } + } catch (MongoException e) { + // ignore exceptions when releasing server resources } finally { + // guarantee that regardless of exceptions, `serverCursor` is null and client resources are released serverCursor = null; + releaseClientResources(); } } - private void killServerCursor(final MongoNamespace namespace, final ServerCursor serverCursor, final Connection connection) { - connection.command(namespace.getDatabaseName(), asKillCursorsCommandDocument(namespace, serverCursor), + @Override + void killServerCursor(final MongoNamespace namespace, final ServerCursor serverCursor, final Connection connection) { + connection.command(namespace.getDatabaseName(), getKillCursorsCommand(namespace, serverCursor), NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), new BsonDocumentCodec(), assertNotNull(connectionSource)); - } - - private BsonDocument asKillCursorsCommandDocument(final MongoNamespace namespace, final ServerCursor serverCursor) { - return new BsonDocument("killCursors", new BsonString(namespace.getCollectionName())) - .append("cursors", new BsonArray(singletonList(new BsonInt64(serverCursor.getId())))); - } - - private void releaseClientResources() { - assertNull(serverCursor); - ConnectionSource localConnectionSource = connectionSource; - if (localConnectionSource != null) { - localConnectionSource.release(); - connectionSource = null; - } - Connection localPinnedConnection = pinnedConnection; - if (localPinnedConnection != null) { - localPinnedConnection.release(); - pinnedConnection = null; - } - } - } - - private enum State { - IDLE(true, false), - OPERATION_IN_PROGRESS(true, true), - /** - * Implies {@link #OPERATION_IN_PROGRESS}. - */ - CLOSE_PENDING(false, true), - CLOSED(false, false); - - private final boolean operable; - private final boolean inProgress; - - State(final boolean operable, final boolean inProgress) { - this.operable = operable; - this.inProgress = inProgress; - } - - boolean operable() { - return operable; - } - - boolean inProgress() { - return inProgress; + releaseClientResources(); } } } diff --git a/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java b/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java new file mode 100644 index 00000000000..c4a53a2b7d1 --- /dev/null +++ b/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java @@ -0,0 +1,264 @@ +package com.mongodb.internal.operation; + +import com.mongodb.MongoNamespace; +import com.mongodb.ServerCursor; +import com.mongodb.annotations.ThreadSafe; +import com.mongodb.internal.async.SingleResultCallback; +import com.mongodb.internal.binding.ReferenceCounted; +import com.mongodb.internal.connection.Connection; +import com.mongodb.lang.Nullable; +import org.bson.BsonArray; +import org.bson.BsonDocument; +import org.bson.BsonInt64; +import org.bson.BsonString; + +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.StampedLock; +import java.util.function.Consumer; + +import static com.mongodb.assertions.Assertions.assertNotNull; +import static com.mongodb.assertions.Assertions.assertNull; +import static com.mongodb.assertions.Assertions.assertTrue; +import static com.mongodb.assertions.Assertions.fail; +import static com.mongodb.assertions.Assertions.notNull; +import static com.mongodb.internal.operation.CommandBatchCursorHelper.MESSAGE_IF_CONCURRENT_OPERATION; +import static java.util.Collections.singletonList; + +/** + * This is the resource manager for {@link CommandBatchCursor} or {@link AsyncCommandBatchCursor} implementations. + *

    + * This class maintains all resources that must be released in {@link CommandBatchCursor#close()} / + * {@link AsyncCommandBatchCursor#close()}. The abstract {@linkplain #doClose() deferred close action} is such that it is totally + * ordered with other operations of {@link CommandBatchCursor} / {@link AsyncCommandBatchCursor} (methods {@link #tryStartOperation()}/ + * {@link #endOperation()} must be used properly to enforce the order) despite the method {@link CommandBatchCursor#close()} / + * {@link AsyncCommandBatchCursor#close()} being called concurrently with those operations. + *

    + * This total order induces the happens-before order. + *

    + * The deferred close action does not violate externally observable idempotence of {@link CommandBatchCursor#close()} / + * {@link AsyncCommandBatchCursor#close()}, because the close method is allowed to release resources "eventually". + *

    + * Only methods explicitly documented as thread-safe are thread-safe, + * others are not and rely on the total order mentioned above. + */ +@ThreadSafe +abstract class CursorResourceManager { + + private final MongoNamespace namespace; + private final Lock lock; + volatile State state; + @Nullable + volatile CS connectionSource; + @Nullable + volatile C pinnedConnection; + @Nullable + volatile ServerCursor serverCursor; + volatile boolean skipReleasingServerResourcesOnClose; + + CursorResourceManager( + final MongoNamespace namespace, + final CS connectionSource, + @Nullable final C connectionToPin, + @Nullable final ServerCursor serverCursor) { + this.namespace = notNull("mongoNamespace", namespace); + this.lock = new StampedLock().asWriteLock(); + this.state = State.IDLE; + if (serverCursor != null) { + connectionSource.retain(); + this.connectionSource = notNull("connectionSource", connectionSource); + if (connectionToPin != null) { + connectionToPin.retain(); + markAsPinned(connectionToPin, Connection.PinningMode.CURSOR); + this.pinnedConnection = connectionToPin; + } + } + this.skipReleasingServerResourcesOnClose = false; + this.serverCursor = serverCursor; + } + + abstract void markAsPinned(C connectionToPin, Connection.PinningMode pinningMode); + + abstract void executeWithConnection(final Consumer action); + + abstract void executeWithConnection(final Consumer action, final SingleResultCallback callback); + + /** + * This method is never executed concurrently with either itself or other operations + * demarcated by {@link #tryStartOperation()}/{@link #endOperation()}. + */ + abstract void doClose(); + + abstract void killServerCursor(final MongoNamespace namespace, final ServerCursor serverCursor, final C connection); + + /** + * Thread-safe. + */ + boolean operable() { + return state.operable(); + } + + /** + * Thread-safe. + * Returns {@code true} iff started an operation. + * If {@linkplain #operable() closed}, then returns false, otherwise completes abruptly. + * + * @throws IllegalStateException Iff another operation is in progress. + */ + boolean tryStartOperation() throws IllegalStateException { + lock.lock(); + try { + State localState = state; + if (!localState.operable()) { + return false; + } else if (localState == State.IDLE) { + state = State.OPERATION_IN_PROGRESS; + return true; + } else if (localState == State.OPERATION_IN_PROGRESS) { + throw new IllegalStateException(MESSAGE_IF_CONCURRENT_OPERATION); + } else { + throw fail(state.toString()); + } + } finally { + lock.unlock(); + } + } + + /** + * Thread-safe. + */ + void endOperation() { + boolean doClose = false; + lock.lock(); + try { + State localState = state; + if (localState == State.OPERATION_IN_PROGRESS) { + state = State.IDLE; + } else if (localState == State.CLOSE_PENDING) { + state = State.CLOSED; + doClose = true; + } else { + fail(localState.toString()); + } + } finally { + lock.unlock(); + } + if (doClose) { + doClose(); + } + } + + /** + * Thread-safe. + */ + void close() { + boolean doClose = false; + lock.lock(); + try { + State localState = state; + if (localState == State.OPERATION_IN_PROGRESS) { + state = State.CLOSE_PENDING; + } else if (localState != State.CLOSED) { + state = State.CLOSED; + doClose = true; + } + } finally { + lock.unlock(); + } + if (doClose) { + doClose(); + } + } + + void onCorruptedConnection(@Nullable final C corruptedConnection) { + // if `pinnedConnection` is corrupted, then we cannot kill `serverCursor` via such a connection + C localPinnedConnection = pinnedConnection; + if (localPinnedConnection != null) { + assertTrue(corruptedConnection == localPinnedConnection); + skipReleasingServerResourcesOnClose = true; + } + } + + /** + * Thread-safe. + */ + @Nullable + ServerCursor serverCursor() { + return serverCursor; + } + + void setServerCursor(@Nullable final ServerCursor serverCursor) { + assertTrue(state.inProgress()); + assertNotNull(this.serverCursor); + // without `connectionSource` we will not be able to kill `serverCursor` later + assertNotNull(connectionSource); + this.serverCursor = serverCursor; + if (serverCursor == null) { + releaseClientResources(); + } + } + + void releaseServerAndClientResources(final C connection) { + lock.lock(); + ServerCursor localServerCursor = serverCursor; + serverCursor = null; + lock.unlock(); + if (localServerCursor != null) { + killServerCursor(namespace, localServerCursor, connection); + } else { + releaseClientResources(); + } + } + + void releaseServerResources(final C connection) { + ServerCursor localServerCursor = serverCursor; + serverCursor = null; + if (localServerCursor != null) { + killServerCursor(namespace, localServerCursor, connection); + } + } + + BsonDocument getKillCursorsCommand(final MongoNamespace namespace, final ServerCursor serverCursor) { + return new BsonDocument("killCursors", new BsonString(namespace.getCollectionName())) + .append("cursors", new BsonArray(singletonList(new BsonInt64(serverCursor.getId())))); + } + + void releaseClientResources() { + assertNull(serverCursor); + CS localConnectionSource = connectionSource; + if (localConnectionSource != null) { + localConnectionSource.release(); + connectionSource = null; + } + C localPinnedConnection = pinnedConnection; + if (localPinnedConnection != null) { + localPinnedConnection.release(); + pinnedConnection = null; + } + } + + enum State { + IDLE(true, false), + OPERATION_IN_PROGRESS(true, true), + /** + * Implies {@link #OPERATION_IN_PROGRESS}. + */ + CLOSE_PENDING(false, true), + CLOSED(false, false); + + private final boolean operable; + private final boolean inProgress; + + State(final boolean operable, final boolean inProgress) { + this.operable = operable; + this.inProgress = inProgress; + } + + boolean operable() { + return operable; + } + + boolean inProgress() { + return inProgress; + } + } +} diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalSpecification.groovy b/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalSpecification.groovy index 87415983e0d..3e5f9b705f8 100644 --- a/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalSpecification.groovy +++ b/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalSpecification.groovy @@ -84,12 +84,6 @@ class AsyncCommandBatchCursorFunctionalSpecification extends OperationFunctional def cleanup() { cursor?.close() - getReferenceCountAfterTimeout(connectionSource, 1) - getReferenceCountAfterTimeout(connection, 1) - cleanupConnectionAndSource() - } - - private void cleanupConnectionAndSource() { connection?.release() connectionSource?.release() } @@ -180,6 +174,10 @@ class AsyncCommandBatchCursorFunctionalSpecification extends OperationFunctional then: counter == expectedTotal + then: + checkReferenceCountReachesTarget(connectionSource, 1) + checkReferenceCountReachesTarget(connection, 1) + where: limit | batchSize | expectedTotal 5 | 2 | 5 @@ -274,17 +272,15 @@ class AsyncCommandBatchCursorFunctionalSpecification extends OperationFunctional @IgnoreIf({ isSharded() }) def 'should kill cursor if limit is reached on initial query'() throws InterruptedException { - given: + when: def (serverAddress, commandResult) = executeFindCommand(5) cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 5, 0, 0, new DocumentCodec(), null, connectionSource, connection) - when: - nextBatch() - then: - cursor.isClosed() cursor.getServerCursor() == null + checkReferenceCountReachesTarget(connectionSource, 1) + checkReferenceCountReachesTarget(connection, 1) } @IgnoreIf({ !isStandalone() }) diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalSpecification.groovy b/driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalSpecification.groovy index 89e96a06f22..88319325fed 100644 --- a/driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalSpecification.groovy +++ b/driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalSpecification.groovy @@ -47,6 +47,7 @@ import java.util.concurrent.TimeUnit import static com.mongodb.ClusterFixture.checkReferenceCountReachesTarget import static com.mongodb.ClusterFixture.getBinding +import static com.mongodb.ClusterFixture.getReferenceCountAfterTimeout import static com.mongodb.ClusterFixture.isDiscoverableReplicaSet import static com.mongodb.ClusterFixture.isSharded import static com.mongodb.ClusterFixture.serverVersionLessThan @@ -78,10 +79,8 @@ class CommandBatchCursorFunctionalSpecification extends OperationFunctionalSpeci def cleanup() { cursor?.close() - cleanupConnectionAndSource() - } - - private void cleanupConnectionAndSource() { + getReferenceCountAfterTimeout(connectionSource, 1) + getReferenceCountAfterTimeout(connection, 1) connection?.release() connectionSource?.release() } @@ -177,6 +176,10 @@ class CommandBatchCursorFunctionalSpecification extends OperationFunctionalSpeci then: cursor.iterator().sum { it.size() } == expectedTotal + then: + checkReferenceCountReachesTarget(connectionSource, 1) + checkReferenceCountReachesTarget(connection, 1) + where: limit | batchSize | expectedTotal 5 | 2 | 5 @@ -419,6 +422,7 @@ class CommandBatchCursorFunctionalSpecification extends OperationFunctionalSpeci null, connectionSource, connection) then: + cursor.getServerCursor() == null checkReferenceCountReachesTarget(connectionSource, 1) checkReferenceCountReachesTarget(connection, 1) } From 66245ee0bd9920556c8c453567b62f40711eeead Mon Sep 17 00:00:00 2001 From: Ross Lawley Date: Fri, 29 Sep 2023 14:42:59 +0100 Subject: [PATCH 4/9] Add Apache license to new file --- .../operation/CursorResourceManager.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java b/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java index c4a53a2b7d1..bd6da661447 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java +++ b/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-present MongoDB, 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.mongodb.internal.operation; import com.mongodb.MongoNamespace; From 5ff0c8fca86a649f9c1e7ce47786d4b524c14c1e Mon Sep 17 00:00:00 2001 From: Ross Lawley Date: Fri, 29 Sep 2023 16:10:22 +0100 Subject: [PATCH 5/9] JAVAify the test --- .../operation/AsyncCommandBatchCursor.java | 33 +- .../operation/CommandBatchCursor.java | 27 +- .../operation/CursorResourceManager.java | 40 +- .../mongodb/client/model/OperationTest.java | 35 +- ...tchCursorAltFunctionalSpecification.groovy | 461 ------------ ...dBatchCursorFunctionalSpecification.groovy | 492 ------------- ...AsyncCommandBatchCursorFunctionalTest.java | 440 ++++++++++++ ...dBatchCursorFunctionalSpecification.groovy | 665 ------------------ .../CommandBatchCursorFunctionalTest.java | 554 +++++++++++++++ ...syncCommandBatchCursorSpecification.groovy | 8 +- .../client/internal/BatchCursorFluxTest.java | 1 - 11 files changed, 1089 insertions(+), 1667 deletions(-) delete mode 100644 driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorAltFunctionalSpecification.groovy delete mode 100644 driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalSpecification.groovy create mode 100644 driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalTest.java delete mode 100644 driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalSpecification.groovy create mode 100644 driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalTest.java diff --git a/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java index 0b6695ccd43..b73d88be6c7 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java @@ -119,7 +119,7 @@ public void next(final SingleResultCallback> callback) { return; } - ServerCursor localServerCursor = resourceManager.serverCursor(); + ServerCursor localServerCursor = resourceManager.getServerCursor(); boolean cursorClosed = localServerCursor == null; List batchResults = emptyList(); if (!processedInitial.getAndSet(true) && !firstBatchEmpty) { @@ -180,7 +180,7 @@ public int getMaxWireVersion() { @Nullable @VisibleForTesting(otherwise = VisibleForTesting.AccessModifier.PRIVATE) ServerCursor getServerCursor() { - return resourceManager.serverCursor(); + return resourceManager.getServerCursor(); } private void getMore(final ServerCursor cursor, final SingleResultCallback> callback) { @@ -192,7 +192,7 @@ private void getMore(final AsyncConnection connection, final ServerCursor cursor getMoreCommandDocument(cursor.getId(), connection.getDescription(), namespace, limit, batchSize, count.get(), maxTimeMS, comment), NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), - CommandResultDocumentCodec.create(decoder, NEXT_BATCH), assertNotNull(resourceManager.connectionSource), + CommandResultDocumentCodec.create(decoder, NEXT_BATCH), assertNotNull(resourceManager.getConnectionSource()), (commandResult, t) -> { if (t != null) { Throwable translatedException = @@ -224,6 +224,7 @@ private void getMore(final AsyncConnection connection, final ServerCursor cursor resourceManager.endOperation(); if (limitReached()) { resourceManager.releaseServerAndClientResources(connection); + close(); } connection.release(); callback.onResult(commandCursor.getResults(), null); @@ -272,11 +273,12 @@ void executeWithConnection(final Consumer action) { @Override void executeWithConnection(final Consumer action, final SingleResultCallback callback) { - assertTrue(state != State.IDLE); + assertTrue(getState() != State.IDLE); + AsyncConnection pinnedConnection = getPinnedConnection(); if (pinnedConnection != null) { executeWithConnection(assertNotNull(pinnedConnection).retain(), null, action, callback); } else { - assertNotNull(connectionSource).getConnection((conn, t) -> executeWithConnection(conn, t, action, callback)); + assertNotNull(getConnectionSource()).getConnection((conn, t) -> executeWithConnection(conn, t, action, callback)); } } @@ -290,10 +292,7 @@ void executeWithConnection( callback.onResult(null, t); } else { AsyncCallbackSupplier curriedFunction = c -> function.accept(connection); - curriedFunction.whenComplete(() -> { - System.out.println("COMPLETED!!!!"); - connection.release(); - }).get((result, error) -> { + curriedFunction.whenComplete(connection::release).get((result, error) -> { if (error instanceof MongoSocketException) { onCorruptedConnection(connection); } @@ -304,17 +303,17 @@ void executeWithConnection( @Override void doClose() { - if (skipReleasingServerResourcesOnClose) { - serverCursor = null; + if (isSkipReleasingServerResourcesOnClose()) { + unsetServerCursor(); } - if (serverCursor != null) { + if (getServerCursor() != null) { executeWithConnection(conn -> { releaseServerResources(conn); conn.release(); }, (r, t) -> { // guarantee that regardless of exceptions, `serverCursor` is null and client resources are released - serverCursor = null; + unsetServerCursor(); releaseClientResources(); }); } else { @@ -325,9 +324,13 @@ void doClose() { @Override void killServerCursor(final MongoNamespace namespace, final ServerCursor serverCursor, final AsyncConnection connection) { connection + .retain() .commandAsync(namespace.getDatabaseName(), getKillCursorsCommand(namespace, serverCursor), - NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), new BsonDocumentCodec(), assertNotNull(connectionSource), - (r, t) -> releaseClientResources()); + NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), new BsonDocumentCodec(), + assertNotNull(getConnectionSource()), (r, t) -> { + connection.release(); + releaseClientResources(); + }); } } } diff --git a/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java index b77b13adb6e..bff59e96f41 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java @@ -120,7 +120,7 @@ private boolean doHasNext() { return false; } - while (resourceManager.serverCursor() != null) { + while (resourceManager.getServerCursor() != null) { getMore(); if (!resourceManager.operable()) { throw new IllegalStateException(MESSAGE_IF_CLOSED_AS_CURSOR); @@ -200,7 +200,7 @@ private boolean tryHasNext() { return false; } - if (resourceManager.serverCursor() != null) { + if (resourceManager.getServerCursor() != null) { getMore(); } @@ -213,7 +213,7 @@ public ServerCursor getServerCursor() { if (!resourceManager.operable()) { throw new IllegalStateException(MESSAGE_IF_CLOSED_AS_ITERATOR); } - return resourceManager.serverCursor(); + return resourceManager.getServerCursor(); } @Override @@ -246,7 +246,7 @@ public int getMaxWireVersion() { } private void getMore() { - ServerCursor serverCursor = assertNotNull(resourceManager.serverCursor()); + ServerCursor serverCursor = assertNotNull(resourceManager.getServerCursor()); resourceManager.executeWithConnection(connection -> { ServerCursor nextServerCursor; try { @@ -258,7 +258,7 @@ private void getMore() { NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), CommandResultDocumentCodec.create(decoder, NEXT_BATCH), - assertNotNull(resourceManager.connectionSource)))); + assertNotNull(resourceManager.getConnectionSource())))); nextServerCursor = commandCursor.getServerCursor(); } catch (MongoCommandException e) { throw translateCommandException(e, serverCursor); @@ -291,7 +291,7 @@ private boolean limitReached() { @ThreadSafe private static final class ResourceManager extends CursorResourceManager { - public ResourceManager( + ResourceManager( final MongoNamespace namespace, final ConnectionSource connectionSource, @Nullable final Connection connectionToPin, @@ -318,11 +318,12 @@ R execute(final String exceptionMessageIfClosed, final Supplier operation } private Connection connection() { - assertTrue(state != State.IDLE); + assertTrue(getState() != State.IDLE); + Connection pinnedConnection = getPinnedConnection(); if (pinnedConnection != null) { return assertNotNull(pinnedConnection).retain(); } else { - return assertNotNull(connectionSource).getConnection(); + return assertNotNull(getConnectionSource()).getConnection(); } } @@ -355,19 +356,19 @@ void executeWithConnection(final Consumer action, final SingleRe @Override void doClose() { - if (skipReleasingServerResourcesOnClose) { - serverCursor = null; + if (isSkipReleasingServerResourcesOnClose()) { + unsetServerCursor(); } try { - if (serverCursor != null) { + if (getServerCursor() != null) { executeWithConnection(this::releaseServerResources); } } catch (MongoException e) { // ignore exceptions when releasing server resources } finally { // guarantee that regardless of exceptions, `serverCursor` is null and client resources are released - serverCursor = null; + unsetServerCursor(); releaseClientResources(); } } @@ -375,7 +376,7 @@ void doClose() { @Override void killServerCursor(final MongoNamespace namespace, final ServerCursor serverCursor, final Connection connection) { connection.command(namespace.getDatabaseName(), getKillCursorsCommand(namespace, serverCursor), - NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), new BsonDocumentCodec(), assertNotNull(connectionSource)); + NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), new BsonDocumentCodec(), assertNotNull(getConnectionSource())); releaseClientResources(); } } diff --git a/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java b/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java index bd6da661447..533b6800c28 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java +++ b/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java @@ -62,14 +62,14 @@ abstract class CursorResourceManager action); + abstract void executeWithConnection(Consumer action); - abstract void executeWithConnection(final Consumer action, final SingleResultCallback callback); + abstract void executeWithConnection(Consumer action, SingleResultCallback callback); /** * This method is never executed concurrently with either itself or other operations @@ -104,7 +122,7 @@ abstract class CursorResourceManager getCollectionHelper(final MongoNamespace return new CollectionHelper<>(new BsonDocumentCodec(), namespace); } - private String getDatabaseName() { + protected String getDatabaseName() { return ClusterFixture.getDefaultDatabaseName(); } - private String getCollectionName() { + protected String getCollectionName() { return "test"; } - MongoNamespace getNamespace() { + protected MongoNamespace getNamespace() { return new MongoNamespace(getDatabaseName(), getCollectionName()); } @@ -97,7 +104,6 @@ public static BsonDocument toBsonDocument(final BsonDocument bsonDocument) { return getDefaultCodecRegistry().get(BsonDocument.class).decode(bsonDocument.asBsonReader(), DecoderContext.builder().build()); } - protected List assertPipeline(final String stageAsString, final Bson stage) { List pipeline = Collections.singletonList(stage); return assertPipeline(stageAsString, pipeline); @@ -159,4 +165,25 @@ protected List aggregateWithWindowFields(@Nullable final Object partitio .map(doc -> doc.get("result")) .collect(toList()); } + + protected void ifNotNull(@Nullable final T maybeNull, final Consumer consumer) { + if (maybeNull != null) { + consumer.accept(maybeNull); + } + } + + protected void sleep(final long ms) { + try { + Thread.sleep(ms); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + + protected T block(final Consumer> consumer) { + FutureResultCallback cb = new FutureResultCallback<>(); + consumer.accept(cb); + return cb.get(TIMEOUT, TimeUnit.SECONDS); + } } diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorAltFunctionalSpecification.groovy b/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorAltFunctionalSpecification.groovy deleted file mode 100644 index c3973a59761..00000000000 --- a/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorAltFunctionalSpecification.groovy +++ /dev/null @@ -1,461 +0,0 @@ -/* - * Copyright 2008-present MongoDB, 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.mongodb.internal.operation - -import com.mongodb.MongoCursorNotFoundException -import com.mongodb.MongoException -import com.mongodb.MongoTimeoutException -import com.mongodb.OperationFunctionalSpecification -import com.mongodb.ReadPreference -import com.mongodb.ServerAddress -import com.mongodb.ServerCursor -import com.mongodb.WriteConcern -import com.mongodb.async.FutureResultCallback -import com.mongodb.client.model.CreateCollectionOptions -import com.mongodb.client.syncadapter.SyncConnection -import com.mongodb.internal.binding.AsyncConnectionSource -import com.mongodb.internal.binding.AsyncReadWriteBinding -import com.mongodb.internal.connection.AsyncConnection -import com.mongodb.internal.validator.NoOpFieldNameValidator -import org.bson.BsonArray -import org.bson.BsonBoolean -import org.bson.BsonDocument -import org.bson.BsonInt32 -import org.bson.BsonInt64 -import org.bson.BsonNull -import org.bson.BsonString -import org.bson.BsonTimestamp -import org.bson.Document -import org.bson.codecs.BsonDocumentCodec -import org.bson.codecs.DocumentCodec -import spock.lang.IgnoreIf -import util.spock.annotations.Slow - -import java.util.concurrent.CountDownLatch - -import static com.mongodb.ClusterFixture.checkReferenceCountReachesTarget -import static com.mongodb.ClusterFixture.getAsyncBinding -import static com.mongodb.ClusterFixture.getAsyncCluster -import static com.mongodb.ClusterFixture.getBinding -import static com.mongodb.ClusterFixture.getConnection -import static com.mongodb.ClusterFixture.getReadConnectionSource -import static com.mongodb.ClusterFixture.getReferenceCountAfterTimeout -import static com.mongodb.ClusterFixture.isDiscoverableReplicaSet -import static com.mongodb.ClusterFixture.isSharded -import static com.mongodb.ClusterFixture.serverVersionLessThan -import static com.mongodb.internal.connection.ServerHelper.waitForLastRelease -import static com.mongodb.internal.connection.ServerHelper.waitForRelease -import static com.mongodb.internal.operation.QueryOperationHelper.makeAdditionalGetMoreCall -import static java.util.Collections.singletonList -import static java.util.concurrent.TimeUnit.SECONDS -import static org.junit.Assert.assertEquals -import static org.junit.Assert.fail - -@IgnoreIf({ isSharded() && serverVersionLessThan(3, 2) }) -class AsyncCommandBatchCursorAltFunctionalSpecification extends OperationFunctionalSpecification { - AsyncConnectionSource connectionSource - AsyncCommandBatchCursor cursor - AsyncConnection connection - - def setup() { - def documents = [] - for (int i = 0; i < 10; i++) { - documents.add(new BsonDocument('_id', new BsonInt32(i))) - } - collectionHelper.insertDocuments(documents, - isDiscoverableReplicaSet() ? WriteConcern.MAJORITY : WriteConcern.ACKNOWLEDGED, - getBinding()) - setUpConnectionAndSource(getAsyncBinding()) - } - - private void setUpConnectionAndSource(final AsyncReadWriteBinding binding) { - connectionSource = getReadConnectionSource(binding) - connection = getConnection(connectionSource) - } - - def cleanup() { - cursor?.close() - cleanupConnectionAndSource() - } - - private void cleanupConnectionAndSource() { - connection?.release() - connectionSource?.release() - waitForLastRelease(connectionSource.getServerDescription().getAddress(), getAsyncCluster()) - waitForRelease(connectionSource, 0) - } - - def 'should exhaust single batch'() { - given: - def (serverAddress, commandResult) = executeQuery() - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - expect: - nextBatch().size() == 10 - } - - def 'should not retain connection and source after cursor is exhausted on first batch'() { - given: - def (serverAddress, commandResult) = executeQuery() - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - when: - nextBatch() - - then: - getReferenceCountAfterTimeout(connection, 1) == 1 - getReferenceCountAfterTimeout(connectionSource, 1) == 1 - } - - def 'should not retain connection and source after cursor is exhausted on getMore'() { - given: - def (serverAddress, commandResult) = executeQuery(1, 0) - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 1, 1, 0, new DocumentCodec(), - null, connectionSource, connection) - - when: - nextBatch() - - then: - getReferenceCountAfterTimeout(connection, 1) == 1 - getReferenceCountAfterTimeout(connectionSource, 1) == 1 - } - - def 'should not retain connection and source after cursor is exhausted after first batch'() { - when: - def (serverAddress, commandResult) = executeQuery(10, 10) - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 10, 10, 0, new DocumentCodec(), - null, connectionSource, - connection) - - then: - getReferenceCountAfterTimeout(connection, 1) == 1 - getReferenceCountAfterTimeout(connectionSource, 1) == 1 - } - - def 'should exhaust single batch with limit'() { - given: - def (serverAddress, commandResult) = executeQuery(1, 0) - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 1, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - expect: - nextBatch().size() == 1 - cursor.isClosed() || !nextBatch() && cursor.isClosed() - } - - def 'should exhaust multiple batches with limit'() { - given: - def (serverAddress, commandResult) = executeQuery(limit, batchSize) - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, limit, batchSize, 0, new DocumentCodec(), - null, connectionSource, connection) - - when: - def next = nextBatch() - def total = 0 - while (next) { - total += next.size() - if (cursor.isClosed()) { - break - } - next = nextBatch() - } - - then: - total == expectedTotal - - where: - limit | batchSize | expectedTotal - 5 | 2 | 5 - 5 | -2 | 2 - -5 | 2 | 5 - -5 | -2 | 5 - 2 | 5 | 2 - 2 | -5 | 2 - -2 | 5 | 2 - -2 | -5 | 2 - } - - def 'should exhaust multiple batches'() { - given: - def (serverAddress, commandResult) = executeQuery(3) - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - expect: - nextBatch().size() == 3 - nextBatch().size() == 2 - nextBatch().size() == 2 - nextBatch().size() == 2 - nextBatch().size() == 1 - !nextBatch() - } - - def 'should respect batch size'() { - when: - def (serverAddress, commandResult) = executeQuery(3) - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.batchSize == 2 - - when: - nextBatch() - cursor.batchSize = 4 - - then: - nextBatch().size() == 4 - } - - def 'should close when exhausted'() { - given: - def (serverAddress, commandResult) = executeQuery() - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - when: - cursor.close() - waitForRelease(connectionSource, 1) - - then: - connectionSource.count == 1 - - when: - nextBatch() - - then: - thrown(MongoException) - } - - def 'should close when not exhausted'() { - given: - def (serverAddress, commandResult) = executeQuery(3) - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - when: - cursor.close() - - then: - waitForRelease(connectionSource, 1) - } - - @Slow - def 'should block waiting for first batch on a tailable cursor'() { - given: - collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(4, 0))) - - def (serverAddress, commandResult) = executeQuery(new BsonDocument('ts', - new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, false) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - def latch = new CountDownLatch(1) - Thread.start { - sleep(500) - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 2).append('ts', new BsonTimestamp(5, 0))) - latch.countDown() - } - - def batch = nextBatch() - - then: - batch.size() == 1 - batch[0].get('_id') == 2 - - cleanup: - def cleanedUp = latch.await(10, SECONDS) // Workaround for codenarc bug - if (!cleanedUp) { - throw new MongoTimeoutException('Timed out waiting for documents to be inserted') - } - } - - @Slow - def 'should block waiting for next batch on a tailable cursor'() { - collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(5, 0))) - def (serverAddress, commandResult) = executeQuery(new BsonDocument('ts', - new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, awaitData) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, maxTimeMS, new DocumentCodec(), - null, connectionSource, connection) - def batch = nextBatch() - - then: - batch.size() == 1 - batch[0].get('_id') == 1 - - when: - def latch = new CountDownLatch(1) - Thread.start { - sleep(500) - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 2).append('ts', new BsonTimestamp(6, 0))) - latch.countDown() - } - - batch = nextBatch() - - then: - batch.size() == 1 - batch[0].get('_id') == 2 - - cleanup: - def cleanedUp = latch.await(10, SECONDS) - if (!cleanedUp) { - throw new MongoTimeoutException('Timed out waiting for documents to be inserted') - } - - where: - awaitData | maxTimeMS - true | 0 - true | 100 - false | 0 - } - - @Slow - def 'should unblock if closed while waiting for more data from tailable cursor'() { - given: - collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) - collectionHelper.insertDocuments(new DocumentCodec(), Document.parse('{}')) - def (serverAddress, commandResult) = executeQuery(new BsonDocument('_id', BsonNull.VALUE), 0, 1, true, true) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 1, 500, new DocumentCodec(), - null, connectionSource, connection) - Thread.start { - Thread.sleep(SECONDS.toMillis(2)) - cursor.close() - } - def batch = nextBatch() - - then: - cursor.isClosed() - batch == null - //both connection and connectionSource have reference count 1 when we pass them to the AsyncCommandBatchCursor constructor - connection.getCount() == 1 - waitForRelease(connectionSource, 1) - } - - def 'should respect limit'() { - given: - def (serverAddress, commandResult) = executeQuery(6, 3) - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 6, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - expect: - nextBatch().size() == 3 - nextBatch().size() == 2 - nextBatch().size() == 1 - !nextBatch() - } - - @IgnoreIf({ isSharded() }) - def 'should kill cursor if limit is reached on initial query'() throws InterruptedException { - given: - def (serverAddress, commandResult) = executeQuery(5) - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 5, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - def serverCursor = new ServerCursor(commandResult.getDocument("cursor").getInt64("id").value, serverAddress) - - when: - checkReferenceCountReachesTarget(connection, 1) - makeAdditionalGetMoreCall(getNamespace(), serverCursor, new SyncConnection(connection)) - - then: - thrown(MongoCursorNotFoundException) - } - - @SuppressWarnings('BracesForTryCatchFinally') - @IgnoreIf({ isSharded() }) - def 'should throw cursor not found exception'() { - given: - def (serverAddress, commandResult) = executeQuery(2) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - def connection = new SyncConnection(getConnection(connectionSource)) - def serverCursor = cursor.getServerCursor() - connection.command(getNamespace().databaseName, - new BsonDocument('killCursors', new BsonString(namespace.getCollectionName())) - .append('cursors', new BsonArray(singletonList(new BsonInt64(serverCursor.getId())))), - new NoOpFieldNameValidator(), ReadPreference.primary(), - new BsonDocumentCodec(), connectionSource) - connection.release() - nextBatch() - - then: - try { - nextBatch() - fail('expected MongoCursorNotFoundException but no exception was thrown') - } catch (MongoCursorNotFoundException e) { - assertEquals(serverCursor.getId(), e.getCursorId()) - assertEquals(serverCursor.getAddress(), e.getServerAddress()) - } catch (ignored) { - fail('Expected MongoCursorNotFoundException to be thrown but got ' + ignored.getClass()) - } - } - - List nextBatch() { - def futureResultCallback = new FutureResultCallback() - cursor.next(futureResultCallback) - futureResultCallback.get() - } - - private Tuple2 executeQuery() { - executeQuery(0) - } - - private Tuple2 executeQuery(int batchSize) { - executeQuery(0, batchSize) - } - - private Tuple2 executeQuery(int limit, int batchSize) { - executeQuery(new BsonDocument(), limit, batchSize, false, false) - } - - private Tuple2 executeQuery(BsonDocument filter, int limit, int batchSize, boolean tailable, - boolean awaitData) { - def findCommand = new BsonDocument('find', new BsonString(getCollectionName())) - .append('filter', filter) - .append('tailable', BsonBoolean.valueOf(tailable)) - .append('awaitData', BsonBoolean.valueOf(awaitData)) - - findCommand.append('limit', new BsonInt32(Math.abs(limit))) - - if (limit >= 0) { - if (batchSize < 0 && Math.abs(batchSize) < limit) { - findCommand.append('limit', new BsonInt32(Math.abs(batchSize))) - } else if (batchSize != 0) { - findCommand.append('batchSize', new BsonInt32(Math.abs(batchSize))) - } - } - - def futureResultCallback = new FutureResultCallback() - connection.commandAsync(getDatabaseName(), findCommand, NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), - CommandResultDocumentCodec.create(new DocumentCodec(), 'firstBatch'), connectionSource, - futureResultCallback) - new Tuple2(connection.getDescription().getServerAddress(), futureResultCallback.get()) - } -} diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalSpecification.groovy b/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalSpecification.groovy deleted file mode 100644 index 3e5f9b705f8..00000000000 --- a/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalSpecification.groovy +++ /dev/null @@ -1,492 +0,0 @@ -/* - * Copyright 2008-present MongoDB, 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.mongodb.internal.operation - -import com.mongodb.MongoCursorNotFoundException -import com.mongodb.MongoException -import com.mongodb.MongoQueryException -import com.mongodb.MongoTimeoutException -import com.mongodb.OperationFunctionalSpecification -import com.mongodb.ReadPreference -import com.mongodb.ServerAddress -import com.mongodb.ServerCursor -import com.mongodb.WriteConcern -import com.mongodb.async.FutureResultCallback -import com.mongodb.client.model.CreateCollectionOptions -import com.mongodb.internal.binding.AsyncConnectionSource -import com.mongodb.internal.binding.AsyncReadWriteBinding -import com.mongodb.internal.connection.AsyncConnection -import com.mongodb.internal.validator.NoOpFieldNameValidator -import org.bson.BsonArray -import org.bson.BsonBoolean -import org.bson.BsonDocument -import org.bson.BsonInt32 -import org.bson.BsonInt64 -import org.bson.BsonString -import org.bson.BsonTimestamp -import org.bson.Document -import org.bson.codecs.BsonDocumentCodec -import org.bson.codecs.DocumentCodec -import spock.lang.IgnoreIf -import util.spock.annotations.Slow - -import java.util.concurrent.CountDownLatch -import java.util.concurrent.TimeUnit - -import static com.mongodb.ClusterFixture.checkReferenceCountReachesTarget -import static com.mongodb.ClusterFixture.getAsyncBinding -import static com.mongodb.ClusterFixture.getBinding -import static com.mongodb.ClusterFixture.getConnection -import static com.mongodb.ClusterFixture.getReferenceCountAfterTimeout -import static com.mongodb.ClusterFixture.getWriteConnectionSource -import static com.mongodb.ClusterFixture.isDiscoverableReplicaSet -import static com.mongodb.ClusterFixture.isSharded -import static com.mongodb.ClusterFixture.isStandalone -import static com.mongodb.internal.operation.QueryOperationHelper.makeAdditionalGetMoreCall -import static java.util.Collections.singletonList -import static org.junit.Assert.assertEquals -import static org.junit.Assert.fail - -class AsyncCommandBatchCursorFunctionalSpecification extends OperationFunctionalSpecification { - AsyncConnectionSource connectionSource - AsyncCommandBatchCursor cursor - AsyncConnection connection - - def setup() { - def documents = [] - for (int i = 0; i < 10; i++) { - documents.add(new BsonDocument('_id', new BsonInt32(i))) - } - collectionHelper.insertDocuments(documents, - isDiscoverableReplicaSet() ? WriteConcern.MAJORITY : WriteConcern.ACKNOWLEDGED, - getBinding()) - setUpConnectionAndSource(getAsyncBinding()) - } - - private void setUpConnectionAndSource(final AsyncReadWriteBinding binding) { - connectionSource = getWriteConnectionSource(binding) - connection = getConnection(connectionSource) - } - - def cleanup() { - cursor?.close() - connection?.release() - connectionSource?.release() - } - - def 'server cursor should not be null'() { - given: - def (serverAddress, commandResult) = executeFindCommand(2) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.getServerCursor() != null - } - - - def 'should get Exceptions for operations on the cursor after closing'() { - given: - def (serverAddress, commandResult) = executeFindCommand() - - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - when: - cursor.close() - cursor.close() - - and: - nextBatch() - - then: - thrown(MongoException) - - and: - def serverCursor = cursor.getServerCursor() - - then: - serverCursor == null - } - - def 'should throw an Exception when going off the end'() { - given: - def (serverAddress, commandResult) = executeFindCommand(1) - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 2, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - when: - nextBatch() - nextBatch() - - then: - nextBatch().isEmpty() - - when: - nextBatch() - - then: - thrown(MongoException) - } - - def 'test normal exhaustion'() { - given: - def (serverAddress, commandResult) = executeFindCommand() - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - nextBatch().size() == 10 - } - - def 'test limit exhaustion'() { - given: - def (serverAddress, commandResult) = executeFindCommand(limit, batchSize) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, limit, batchSize, 0, new DocumentCodec(), - null, connectionSource, connection) - def batch = nextBatch() - def counter = batch.size() - while (!cursor.isClosed()) { - batch = nextBatch() - counter += batch == null ? 0 : batch.size() - } - - then: - counter == expectedTotal - - then: - checkReferenceCountReachesTarget(connectionSource, 1) - checkReferenceCountReachesTarget(connection, 1) - - where: - limit | batchSize | expectedTotal - 5 | 2 | 5 - 5 | -2 | 2 - -5 | 2 | 5 - -5 | -2 | 5 - 2 | 5 | 2 - 2 | -5 | 2 - -2 | 5 | 2 - -2 | -5 | 2 - } - - @SuppressWarnings('EmptyCatchBlock') - def 'should block waiting for next batch on a tailable cursor'() { - given: - collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(5, 0))) - def (serverAddress, commandResult) = executeFindCommand(new BsonDocument('ts', - new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, awaitData) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, maxTimeMS, new DocumentCodec(), - null, connectionSource, connection) - - then: - nextBatch().iterator().next().get('_id') == 1 - - when: - def latch = new CountDownLatch(1) - Thread.start { - try { - sleep(500) - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 2).append('ts', new BsonTimestamp(6, 0))) - } catch (ignored) { - //pass - } finally { - latch.countDown() - } - } - - then: - nextBatch().iterator().next().get('_id') == 2 - - cleanup: - def cleanedUp = latch.await(10, TimeUnit.SECONDS) - if (!cleanedUp) { - throw new MongoTimeoutException('Timed out waiting for documents to be inserted') - } - - where: - awaitData | maxTimeMS - true | 0 - true | 100 - false | 0 - } - - - @SuppressWarnings('EmptyCatchBlock') - @Slow - def 'test tailable interrupt'() throws InterruptedException { - collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1)) - - def (serverAddress, commandResult) = executeFindCommand(new BsonDocument(), 0, 2, true, true) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - CountDownLatch latch = new CountDownLatch(1) - def seen = 0 - def thread = Thread.start { - try { - nextBatch() - seen = 1 - nextBatch() - seen = 2 - } catch (ignored) { - // pass - } finally { - latch.countDown() - } - } - sleep(1000) - thread.interrupt() - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 2)) - latch.await() - - then: - seen == 1 - } - - @IgnoreIf({ isSharded() }) - def 'should kill cursor if limit is reached on initial query'() throws InterruptedException { - when: - def (serverAddress, commandResult) = executeFindCommand(5) - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 5, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.getServerCursor() == null - checkReferenceCountReachesTarget(connectionSource, 1) - checkReferenceCountReachesTarget(connection, 1) - } - - @IgnoreIf({ !isStandalone() }) - def 'should kill cursor if limit is reached on get more'() throws InterruptedException { - given: - def (serverAddress, commandResult) = executeFindCommand(3) - - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 5, 3, 0, new DocumentCodec(), - null, connectionSource, connection) - ServerCursor serverCursor = cursor.getServerCursor() - - nextBatch() - nextBatch() - - checkReferenceCountReachesTarget(connection, 1) - - when: - makeAdditionalGetMoreCall(getNamespace(), serverCursor, connection) - - then: - thrown(MongoQueryException) - } - - def 'should release connection source if limit is reached on initial query'() throws InterruptedException { - given: - def (serverAddress, commandResult) = executeFindCommand(5) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 5, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - checkReferenceCountReachesTarget(connectionSource, 1) - checkReferenceCountReachesTarget(connection, 1) - } - - def 'should release connection source if limit is reached on get more'() throws InterruptedException { - given: - def (serverAddress, commandResult) = executeFindCommand(3) - - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 5, 3, 0, new DocumentCodec(), - null, connectionSource, connection) - - when: - nextBatch() - nextBatch() - - then: - checkReferenceCountReachesTarget(connectionSource, 1) - checkReferenceCountReachesTarget(connection, 1) - } - - def 'test limit with get more'() { - given: - def (serverAddress, commandResult) = executeFindCommand(2) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 5, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - !nextBatch().isEmpty() - !nextBatch().isEmpty() - !nextBatch().isEmpty() - nextBatch().isEmpty() - } - - @Slow - def 'test limit with large documents'() { - given: - String bigString = new String('x' * 16000) - - (11..1000).each { collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', it).append('s', bigString)) } - def (serverAddress, commandResult) = executeFindCommand(300, 0) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 300, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - def batch = nextBatch() - def counter = batch.size() - while (!cursor.isClosed()) { - batch = nextBatch() - counter += batch == null ? 0 : batch.size() - } - - then: - counter == 300 - } - - def 'should respect batch size'() { - given: - def (serverAddress, commandResult) = executeFindCommand(2) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.batchSize == 2 - - nextBatch().size() == 2 - nextBatch().size() == 2 - - when: - cursor.batchSize = 3 - - then: - cursor.batchSize == 3 - nextBatch().size() == 3 - nextBatch().size() == 3 - } - - def 'test normal loop with get more'() { - given: - def (serverAddress, commandResult) = executeFindCommand(2) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - def batch = nextBatch() - def counter = batch.size() - while (!cursor.isClosed()) { - batch = nextBatch() - counter += batch == null ? 0 : batch.size() - } - - then: - counter == 10 - } - - - @SuppressWarnings('BracesForTryCatchFinally') - @IgnoreIf({ isSharded() }) - def 'should throw cursor not found exception'() { - given: - def (serverAddress, commandResult) = executeFindCommand(2) - - when: - cursor = new AsyncCommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - def serverCursor = cursor.getServerCursor() - - def callback = new FutureResultCallback<>() - connection.commandAsync(getNamespace().databaseName, - new BsonDocument('killCursors', new BsonString(namespace.getCollectionName())) - .append('cursors', new BsonArray(singletonList(new BsonInt64(serverCursor.getId())))), - new NoOpFieldNameValidator(), ReadPreference.primary(), new BsonDocumentCodec(), connectionSource, callback) - callback.get() - nextBatch() - - then: - try { - nextBatch() - } catch (MongoCursorNotFoundException e) { - assertEquals(serverCursor.getId(), e.getCursorId()) - assertEquals(serverCursor.getAddress(), e.getServerAddress()) - } catch (ignored) { - fail('Expected MongoCursorNotFoundException to be thrown but got ' + ignored.getClass()) - } - } - - List nextBatch() { - def futureResultCallback = new FutureResultCallback() - cursor.next(futureResultCallback) - futureResultCallback.get() - } - - private Tuple2 executeFindCommand() { - executeFindCommand(0) - } - - private Tuple2 executeFindCommand(int batchSize) { - executeFindCommand(new BsonDocument(), 0, batchSize, false, false, ReadPreference.primary()) - } - - private Tuple2 executeFindCommand(int limit, int batchSize) { - executeFindCommand(new BsonDocument(), limit, batchSize, false, false, ReadPreference.primary()) - } - - private Tuple2 executeFindCommand(BsonDocument filter, int limit, int batchSize, boolean tailable, - boolean awaitData) { - executeFindCommand(filter, limit, batchSize, tailable, awaitData, ReadPreference.primary()) - } - - private Tuple2 executeFindCommand(BsonDocument filter, int limit, int batchSize, boolean tailable, - boolean awaitData, ReadPreference readPreference) { - def findCommand = new BsonDocument('find', new BsonString(getCollectionName())) - .append('filter', filter) - .append('tailable', BsonBoolean.valueOf(tailable)) - .append('awaitData', BsonBoolean.valueOf(awaitData)) - - findCommand.append('limit', new BsonInt32(Math.abs(limit))) - - if (limit >= 0) { - if (batchSize < 0 && Math.abs(batchSize) < limit) { - findCommand.append('limit', new BsonInt32(Math.abs(batchSize))) - } else if (batchSize != 0) { - findCommand.append('batchSize', new BsonInt32(Math.abs(batchSize))) - } - } - - def callback = new FutureResultCallback() - connection.commandAsync(getDatabaseName(), findCommand, - NO_OP_FIELD_NAME_VALIDATOR, readPreference, - CommandResultDocumentCodec.create(new DocumentCodec(), 'firstBatch'), connectionSource, callback) - new Tuple2(connection.getDescription().getServerAddress(), callback.get()) - } -} diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalTest.java b/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalTest.java new file mode 100644 index 00000000000..edfa8e9bbbc --- /dev/null +++ b/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalTest.java @@ -0,0 +1,440 @@ +/* + * Copyright 2008-present MongoDB, 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.mongodb.internal.operation; + + +import com.mongodb.MongoCursorNotFoundException; +import com.mongodb.MongoException; +import com.mongodb.MongoQueryException; +import com.mongodb.ReadPreference; +import com.mongodb.ServerAddress; +import com.mongodb.ServerCursor; +import com.mongodb.client.model.CreateCollectionOptions; +import com.mongodb.client.model.OperationTest; +import com.mongodb.internal.binding.AsyncConnectionSource; +import com.mongodb.internal.connection.AsyncConnection; +import org.bson.BsonArray; +import org.bson.BsonBoolean; +import org.bson.BsonDocument; +import org.bson.BsonInt32; +import org.bson.BsonInt64; +import org.bson.BsonString; +import org.bson.BsonTimestamp; +import org.bson.Document; +import org.bson.codecs.BsonDocumentCodec; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static com.mongodb.ClusterFixture.checkReferenceCountReachesTarget; +import static com.mongodb.ClusterFixture.getAsyncBinding; +import static com.mongodb.ClusterFixture.getConnection; +import static com.mongodb.ClusterFixture.getReferenceCountAfterTimeout; +import static com.mongodb.ClusterFixture.getWriteConnectionSource; +import static com.mongodb.ClusterFixture.isSharded; +import static com.mongodb.internal.operation.QueryOperationHelper.makeAdditionalGetMoreCall; +import static java.util.Collections.singletonList; +import static java.util.stream.Stream.generate; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assumptions.assumeFalse; +import static org.junit.jupiter.params.provider.Arguments.arguments; + +public class AsyncCommandBatchCursorFunctionalTest extends OperationTest { + + private AsyncConnectionSource connectionSource; + private AsyncConnection connection; + private AsyncCommandBatchCursor cursor; + + @BeforeEach + void setup() throws Throwable { + List documents = IntStream.rangeClosed(1, 10) + .mapToObj(i -> new BsonDocument("i", new BsonInt32(i))) + .collect(Collectors.toList()); + getCollectionHelper().insertDocuments(documents); + + connectionSource = getWriteConnectionSource(getAsyncBinding()); + connection = getConnection(connectionSource); + } + + @AfterEach + void cleanup() { + ifNotNull(cursor, AsyncCommandBatchCursor::close); + ifNotNull(connectionSource, cs -> { + getReferenceCountAfterTimeout(cs, 1); + cs.release(); + }); + ifNotNull(connection, c -> { + getReferenceCountAfterTimeout(c, 1); + c.release(); + }); + } + + @Test + @DisplayName("server cursor should not be null") + void theServerCursorShouldNotBeNull() { + BsonDocument commandResult = executeFindCommand(2); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 0, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertNotNull(cursor.getServerCursor()); + } + + + @Test + @DisplayName("should get Exceptions for operations on the cursor after closing") + void shouldGetExceptionsForOperationsOnTheCursorAfterClosing() { + BsonDocument commandResult = executeFindCommand(); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 0, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + cursor.close(); + + assertDoesNotThrow(() -> cursor.close()); + assertThrows(MongoException.class, this::cursorNext); + assertNull(cursor.getServerCursor()); + } + + @Test + @DisplayName("should throw an Exception when going off the end") + void shouldThrowAnExceptionWhenGoingOffTheEnd() { + BsonDocument commandResult = executeFindCommand(1); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 2, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + cursorNext(); + cursorNext(); + + assertThrows(MongoException.class, this::cursorNext); + } + + + @Test + @DisplayName("test normal exhaustion") + void testNormalExhaustion() { + BsonDocument commandResult = executeFindCommand(); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 0, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertEquals(10, cursorFlatten().size()); + } + + @ParameterizedTest(name = "{index} => limit={0}, batchSize={1}, expectedTotal={2}") + @MethodSource + @DisplayName("test limit exhaustion") + void testLimitExhaustion(final int limit, final int batchSize, final int expectedTotal) { + BsonDocument commandResult = executeFindCommand(limit, batchSize); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, limit, batchSize, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + + assertEquals(expectedTotal, cursorFlatten().size()); + + checkReferenceCountReachesTarget(connectionSource, 1); + checkReferenceCountReachesTarget(connection, 1); + } + + @ParameterizedTest(name = "{index} => awaitData={0}, maxTimeMS={1}") + @MethodSource + @DisplayName("should block waiting for next batch on a tailable cursor") + void shouldBlockWaitingForNextBatchOnATailableCursor(final boolean awaitData, final int maxTimeMS) { + + getCollectionHelper().create(getCollectionName(), new CreateCollectionOptions().capped(true).sizeInBytes(1000)); + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", 1).append("ts", new BsonTimestamp(5, 0))); + + BsonDocument commandResult = executeFindCommand(new BsonDocument("ts", + new BsonDocument("$gte", new BsonTimestamp(5, 0))), 0, 2, true, awaitData); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 0, 2, maxTimeMS, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertFalse(cursor.isClosed()); + assertEquals(1, cursorNext().get(0).get("_id")); + + new Thread(() -> { + sleep(100); + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", 2).append("ts", new BsonTimestamp(6, 0))); + }).start(); + + assertFalse(cursor.isClosed()); + assertEquals(2, cursorNext().get(0).get("_id")); + } + + @Test + @DisplayName("test tailable interrupt") + void testTailableInterrupt() throws InterruptedException { + getCollectionHelper().create(getCollectionName(), new CreateCollectionOptions().capped(true).sizeInBytes(1000)); + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", 1).append("ts", new BsonTimestamp(5, 0))); + + BsonDocument commandResult = executeFindCommand(new BsonDocument("ts", + new BsonDocument("$gte", new BsonTimestamp(5, 0))), 0, 2, true, true); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 0, 2, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + CountDownLatch latch = new CountDownLatch(1); + AtomicInteger seen = new AtomicInteger(); + Thread thread = new Thread(() -> { + try { + cursorNext(); + seen.incrementAndGet(); + cursorNext(); + seen.incrementAndGet(); + } catch (Exception e) { + // pass + } finally { + latch.countDown(); + } + }); + + thread.start(); + sleep(1000); + thread.interrupt(); + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", 2)); + latch.await(); + + assertTrue(latch.await(5, TimeUnit.SECONDS)); + assertEquals(1, seen.intValue()); + } + + @Test + @DisplayName("should kill cursor if limit is reached on initial query") + void shouldKillCursorIfLimitIsReachedOnInitialQuery() { + assumeFalse(isSharded()); + BsonDocument commandResult = executeFindCommand(5); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 5, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertNotNull(cursorNext()); + assertTrue(cursor.isClosed()); + assertNull(cursor.getServerCursor()); + } + + @Test + @DisplayName("should kill cursor if limit is reached on getMore") + void shouldKillCursorIfLimitIsReachedOnGetMore() { + assumeFalse(isSharded()); + BsonDocument commandResult = executeFindCommand(); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 5, 3, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + ServerCursor serverCursor = cursor.getServerCursor(); + assertNotNull(serverCursor); + assertNotNull(cursorNext()); + assertNotNull(cursorNext()); + + assertDoesNotThrow(() -> checkReferenceCountReachesTarget(connection, 1)); + assertThrows(MongoQueryException.class, () -> + makeAdditionalGetMoreCall(getNamespace(), serverCursor, connection) + ); + } + + @Test + @DisplayName("should release connection source if limit is reached on initial query") + void shouldReleaseConnectionSourceIfLimitIsReachedOnInitialQuery() { + assumeFalse(isSharded()); + BsonDocument commandResult = executeFindCommand(5); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 5, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertNull(cursor.getServerCursor()); + assertDoesNotThrow(() -> checkReferenceCountReachesTarget(connectionSource, 1)); + assertDoesNotThrow(() -> checkReferenceCountReachesTarget(connection, 1)); + } + + @Test + @DisplayName("should release connection source if limit is reached on getMore") + void shouldReleaseConnectionSourceIfLimitIsReachedOnGetMore() { + assumeFalse(isSharded()); + BsonDocument commandResult = executeFindCommand(3); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 5, 3, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertNotNull(cursorNext()); + assertNotNull(cursorNext()); + assertDoesNotThrow(() -> checkReferenceCountReachesTarget(connectionSource, 1)); + assertDoesNotThrow(() -> checkReferenceCountReachesTarget(connection, 1)); + } + + @Test + @DisplayName("test limit with get more") + void testLimitWithGetMore() { + BsonDocument commandResult = executeFindCommand(2); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 5, 2, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertNotNull(cursorNext()); + assertNotNull(cursorNext()); + assertNotNull(cursorNext()); + + assertDoesNotThrow(() -> checkReferenceCountReachesTarget(connectionSource, 1)); + assertTrue(cursor.isClosed()); + } + + @Test + @DisplayName("test limit with large documents") + void testLimitWithLargeDocuments() { + String bigString = generate(() -> "x") + .limit(16000) + .collect(Collectors.joining()); + + IntStream.range(11, 1000).forEach(i -> + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", i).append("s", bigString)) + ); + + BsonDocument commandResult = executeFindCommand(300, 0); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 300, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertEquals(300, cursorFlatten().size()); + } + + @Test + @DisplayName("should respect batch size") + void shouldRespectBatchSize() { + BsonDocument commandResult = executeFindCommand(2); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 0, 2, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertEquals(2, cursor.getBatchSize()); + assertEquals(2, cursorNext().size()); + assertEquals(2, cursorNext().size()); + + cursor.setBatchSize(3); + assertEquals(3, cursor.getBatchSize()); + assertEquals(3, cursorNext().size()); + assertEquals(3, cursorNext().size()); + } + + @Test + @DisplayName("should throw cursor not found exception") + void shouldThrowCursorNotFoundException() throws Throwable { + BsonDocument commandResult = executeFindCommand(2); + cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 0, 2, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + ServerCursor serverCursor = cursor.getServerCursor(); + assertNotNull(serverCursor); + AsyncConnection localConnection = getConnection(connectionSource); + this.block(cb -> localConnection.commandAsync(getNamespace().getDatabaseName(), + new BsonDocument("killCursors", new BsonString(getNamespace().getCollectionName())) + .append("cursors", new BsonArray(singletonList(new BsonInt64(serverCursor.getId())))), + NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), new BsonDocumentCodec(), connectionSource, cb)); + localConnection.release(); + + cursorNext(); + + MongoCursorNotFoundException exception = assertThrows(MongoCursorNotFoundException.class, this::cursorNext); + assertEquals(serverCursor.getId(), exception.getCursorId()); + assertEquals(serverCursor.getAddress(), exception.getServerAddress()); + } + + + private static Stream shouldBlockWaitingForNextBatchOnATailableCursor() { + return Stream.of( + arguments(true, 0), + arguments(true, 100), + arguments(false, 0)); + } + + private static Stream testLimitExhaustion() { + return Stream.of( + arguments(5, 2, 5), + arguments(5, -2, 2), + arguments(-5, -2, 5), + arguments(-5, 2, 5), + arguments(2, 5, 2), + arguments(2, -5, 2), + arguments(-2, 5, 2), + arguments(-2, -5, 2) + ); + } + + private ServerAddress getServerAddress() { + return connection.getDescription().getServerAddress(); + } + + private BsonDocument executeFindCommand() { + return executeFindCommand(0); + } + + private BsonDocument executeFindCommand(final int batchSize) { + return executeFindCommand(new BsonDocument(), 0, batchSize, false, false); + } + + private BsonDocument executeFindCommand(final int limit, final int batchSize) { + return executeFindCommand(new BsonDocument(), limit, batchSize, false, false); + } + + private BsonDocument executeFindCommand(final BsonDocument filter, final int limit, final int batchSize, final boolean tailable, + final boolean awaitData) { + return executeFindCommand(filter, limit, batchSize, tailable, awaitData, ReadPreference.primary()); + } + + private BsonDocument executeFindCommand(final BsonDocument filter, final int limit, final int batchSize, + final boolean tailable, final boolean awaitData, final ReadPreference readPreference) { + BsonDocument findCommand = new BsonDocument("find", new BsonString(getCollectionName())) + .append("filter", filter) + .append("tailable", BsonBoolean.valueOf(tailable)) + .append("awaitData", BsonBoolean.valueOf(awaitData)); + + findCommand.append("limit", new BsonInt32(Math.abs(limit))); + if (limit >= 0) { + if (batchSize < 0 && Math.abs(batchSize) < limit) { + findCommand.append("limit", new BsonInt32(Math.abs(batchSize))); + } else { + findCommand.append("batchSize", new BsonInt32(Math.abs(batchSize))); + } + } + + BsonDocument results = block(cb -> connection.commandAsync(getDatabaseName(), findCommand, + NO_OP_FIELD_NAME_VALIDATOR, readPreference, + CommandResultDocumentCodec.create(DOCUMENT_DECODER, "firstBatch"), + connectionSource, cb)); + + assertNotNull(results); + return results; + } + + private List cursorNext() { + return block(cb -> cursor.next(cb)); + } + + private List cursorFlatten() { + List results = new ArrayList<>(); + while (!cursor.isClosed()) { + results.addAll(cursorNext()); + } + return results; + } +} diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalSpecification.groovy b/driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalSpecification.groovy deleted file mode 100644 index 88319325fed..00000000000 --- a/driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalSpecification.groovy +++ /dev/null @@ -1,665 +0,0 @@ -/* - * Copyright 2008-present MongoDB, 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.mongodb.internal.operation - -import com.mongodb.MongoCursorNotFoundException -import com.mongodb.MongoQueryException -import com.mongodb.MongoTimeoutException -import com.mongodb.OperationFunctionalSpecification -import com.mongodb.ReadPreference -import com.mongodb.ServerAddress -import com.mongodb.ServerCursor -import com.mongodb.WriteConcern -import com.mongodb.client.model.CreateCollectionOptions -import com.mongodb.internal.binding.ConnectionSource -import com.mongodb.internal.binding.ReadWriteBinding -import com.mongodb.internal.connection.Connection -import com.mongodb.internal.validator.NoOpFieldNameValidator -import org.bson.BsonArray -import org.bson.BsonBoolean -import org.bson.BsonDocument -import org.bson.BsonInt32 -import org.bson.BsonInt64 -import org.bson.BsonString -import org.bson.BsonTimestamp -import org.bson.Document -import org.bson.codecs.BsonDocumentCodec -import org.bson.codecs.DocumentCodec -import spock.lang.IgnoreIf -import util.spock.annotations.Slow - -import java.util.concurrent.CountDownLatch -import java.util.concurrent.TimeUnit - -import static com.mongodb.ClusterFixture.checkReferenceCountReachesTarget -import static com.mongodb.ClusterFixture.getBinding -import static com.mongodb.ClusterFixture.getReferenceCountAfterTimeout -import static com.mongodb.ClusterFixture.isDiscoverableReplicaSet -import static com.mongodb.ClusterFixture.isSharded -import static com.mongodb.ClusterFixture.serverVersionLessThan -import static com.mongodb.internal.operation.QueryOperationHelper.makeAdditionalGetMoreCall -import static java.util.Collections.singletonList -import static org.junit.Assert.assertEquals -import static org.junit.Assert.fail - -class CommandBatchCursorFunctionalSpecification extends OperationFunctionalSpecification { - ConnectionSource connectionSource - Connection connection - CommandBatchCursor cursor - - def setup() { - def documents = [] - for (int i = 0; i < 10; i++) { - documents.add(new BsonDocument('_id', new BsonInt32(i))) - } - collectionHelper.insertDocuments(documents, - isDiscoverableReplicaSet() ? WriteConcern.MAJORITY : WriteConcern.ACKNOWLEDGED, - getBinding()) - setUpConnectionAndSource(getBinding()) - } - - private void setUpConnectionAndSource(final ReadWriteBinding binding) { - connectionSource = binding.getWriteConnectionSource() - connection = connectionSource.getConnection() - } - - def cleanup() { - cursor?.close() - getReferenceCountAfterTimeout(connectionSource, 1) - getReferenceCountAfterTimeout(connection, 1) - connection?.release() - connectionSource?.release() - } - - def 'server cursor should not be null'() { - given: - def (serverAddress, commandResult) = executeFindCommand(2) - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.getServerCursor() != null - } - - def 'test server address'() { - given: - def (serverAddress, commandResult) = executeFindCommand() - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - then: - cursor.getServerAddress() != null - } - - def 'should get Exceptions for operations on the cursor after closing'() { - given: - def (serverAddress, commandResult) = executeFindCommand() - - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - when: - cursor.close() - cursor.close() - - and: - cursor.next() - - then: - thrown(IllegalStateException) - - when: - cursor.hasNext() - - then: - thrown(IllegalStateException) - - when: - cursor.getServerCursor() - - then: - thrown(IllegalStateException) - } - - def 'should throw an Exception when going off the end'() { - given: - def (serverAddress, commandResult) = executeFindCommand(1) - - cursor = new CommandBatchCursor(serverAddress, commandResult, 2, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - when: - cursor.next() - cursor.next() - cursor.next() - - then: - thrown(NoSuchElementException) - } - - def 'test normal exhaustion'() { - given: - def (serverAddress, commandResult) = executeFindCommand() - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.iterator().sum { it.size() } == 10 - } - - def 'test limit exhaustion'() { - given: - def (serverAddress, commandResult) = executeFindCommand(limit, batchSize) - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, limit, batchSize, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.iterator().sum { it.size() } == expectedTotal - - then: - checkReferenceCountReachesTarget(connectionSource, 1) - checkReferenceCountReachesTarget(connection, 1) - - where: - limit | batchSize | expectedTotal - 5 | 2 | 5 - 5 | -2 | 2 - -5 | 2 | 5 - -5 | -2 | 5 - 2 | 5 | 2 - 2 | -5 | 2 - -2 | 5 | 2 - -2 | -5 | 2 - } - - def 'test remove'() { - given: - def (serverAddress, commandResult) = executeFindCommand() - - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - when: - cursor.remove() - - then: - thrown(UnsupportedOperationException) - } - - @SuppressWarnings('EmptyCatchBlock') - @Slow - def 'should block waiting for next batch on a tailable cursor'() { - given: - def connection = connectionSource.getConnection() - collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(5, 0))) - def (serverAddress, commandResult) = executeFindCommand(new BsonDocument('ts', - new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, awaitData) - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, maxTimeMS, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.hasNext() - cursor.next().iterator().next().get('_id') == 1 - - when: - def latch = new CountDownLatch(1) - Thread.start { - try { - sleep(500) - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 2).append('ts', new BsonTimestamp(6, 0))) - } catch (ignored) { - //pass - } finally { - latch.countDown() - } - } - - // Note: this test is racy. - // The sleep above does not guarantee that we're testing what we're trying to, which is the loop in the hasNext() method. - then: - cursor.hasNext() - cursor.next().iterator().next().get('_id') == 2 - - cleanup: - def cleanedUp = latch.await(10, TimeUnit.SECONDS) - if (!cleanedUp) { - throw new MongoTimeoutException('Timed out waiting for documents to be inserted') - } - connection?.release() - - where: - awaitData | maxTimeMS - true | 0 - true | 100 - false | 0 - } - - @Slow - def 'test try next with tailable'() { - collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(5, 0))) - def (serverAddress, commandResult) = executeFindCommand(new BsonDocument('ts', - new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, true) - - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.tryNext().iterator().next().get('_id') == 1 - - then: - !cursor.tryNext() - - when: - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 2).append('ts', new BsonTimestamp(6, 0))) - def nextBatch = cursor.tryNext() - - then: - nextBatch - nextBatch.iterator().next().get('_id') == 2 - } - - @Slow - def 'hasNext should throw when cursor is closed in another thread'() { - Connection connection = connectionSource.getConnection() - collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(5, 0))) - def (serverAddress, commandResult) = executeFindCommand(new BsonDocument('ts', - new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, true) - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - cursor.next() - def closeCompleted = new CountDownLatch(1) - - // wait a second then close the cursor - new Thread({ - sleep(1000) - cursor.close() - closeCompleted.countDown() - } as Runnable).start() - - when: - cursor.hasNext() - - then: - thrown(Exception) - closeCompleted.await(5, TimeUnit.SECONDS) - conn.getCount() == 1 - - cleanup: - conn.release() - } - - @IgnoreIf({ serverVersionLessThan(3, 2) || isSharded() }) - @Slow - def 'test maxTimeMS'() { - collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1).append('ts', new BsonTimestamp(5, 0))) - def (serverAddress, commandResult) = executeFindCommand(new BsonDocument('ts', - new BsonDocument('$gte', new BsonTimestamp(5, 0))), 0, 2, true, true) - - def connection = connectionSource.getConnection() - def maxTimeMS = 10L - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, maxTimeMS, new DocumentCodec(), - null, connectionSource, connection) - cursor.tryNext() - long startTime = System.currentTimeMillis() - - when: - def result = cursor.tryNext() - - then: - result == null - // RACY TEST: no guarantee assertion will fire within the given timeframe - System.currentTimeMillis() - startTime < (maxTimeMS + 200) - - cleanup: - connection?.release() - } - - @SuppressWarnings('EmptyCatchBlock') - @Slow - def 'test tailable interrupt'() throws InterruptedException { - collectionHelper.create(collectionName, new CreateCollectionOptions().capped(true).sizeInBytes(1000)) - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 1)) - - def (serverAddress, commandResult) = executeFindCommand(new BsonDocument(), 0, 2, true, true) - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - CountDownLatch latch = new CountDownLatch(1) - def seen = 0 - def thread = Thread.start { - try { - cursor.next() - seen = 1 - cursor.next() - seen = 2 - } catch (ignored) { - // pass - } finally { - latch.countDown() - } - } - sleep(1000) - thread.interrupt() - collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', 2)) - latch.await() - - then: - seen == 1 - } - - @IgnoreIf({ isSharded() }) - def 'should kill cursor if limit is reached on initial query'() throws InterruptedException { - given: - def (serverAddress, commandResult) = executeFindCommand(5) - cursor = new CommandBatchCursor(serverAddress, commandResult, 5, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - when: - cursor.next() - - then: - !cursor.hasNext() - cursor.getServerCursor() == null - } - - @IgnoreIf({ isSharded() }) - def 'should kill cursor if limit is reached on get more'() throws InterruptedException { - given: - def (serverAddress, commandResult) = executeFindCommand(3) - - cursor = new CommandBatchCursor(serverAddress, commandResult, 5, 3, 0, new DocumentCodec(), - null, connectionSource, connection) - ServerCursor serverCursor = cursor.getServerCursor() - - cursor.next() - cursor.next() - - checkReferenceCountReachesTarget(connection, 1) - - when: - makeAdditionalGetMoreCall(getNamespace(), serverCursor, connection) - - then: - thrown(MongoQueryException) - } - - def 'should release connection source if limit is reached on initial query'() throws InterruptedException { - given: - def (serverAddress, commandResult) = executeFindCommand(5) - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 5, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.getServerCursor() == null - checkReferenceCountReachesTarget(connectionSource, 1) - checkReferenceCountReachesTarget(connection, 1) - } - - def 'should release connection source if limit is reached on get more'() throws InterruptedException { - given: - def (serverAddress, commandResult) = executeFindCommand(3) - - cursor = new CommandBatchCursor(serverAddress, commandResult, 5, 3, 0, new DocumentCodec(), - null, connectionSource, connection) - - when: - cursor.next() - cursor.next() - - then: - checkReferenceCountReachesTarget(connectionSource, 1) - checkReferenceCountReachesTarget(connection, 1) - } - - def 'test limit with get more'() { - given: - def (serverAddress, commandResult) = executeFindCommand(2) - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 5, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.next() != null - cursor.next() != null - cursor.next() != null - !cursor.hasNext() - } - - @Slow - def 'test limit with large documents'() { - given: - String bigString = new String('x' * 16000) - - (11..1000).each { collectionHelper.insertDocuments(new DocumentCodec(), new Document('_id', it).append('s', bigString)) } - def (serverAddress, commandResult) = executeFindCommand(300, 0) - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 300, 0, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.iterator().sum { it.size() } == 300 - } - - def 'should respect batch size'() { - given: - def (serverAddress, commandResult) = executeFindCommand(2) - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.batchSize == 2 - - when: - def nextBatch = cursor.next() - - then: - nextBatch.size() == 2 - - when: - nextBatch = cursor.next() - - then: - nextBatch.size() == 2 - - when: - cursor.batchSize = 3 - nextBatch = cursor.next() - - then: - cursor.batchSize == 3 - nextBatch.size() == 3 - - when: - nextBatch = cursor.next() - - then: - nextBatch.size() == 3 - } - - def 'test normal loop with get more'() { - given: - def (serverAddress, commandResult) = executeFindCommand(2) - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - def results = cursor.iterator().collectMany { it*.get('_id') } - - then: - results == (0..9).toList() - !cursor.hasNext() - } - - def 'test next without has next with get more'() { - given: - def (serverAddress, commandResult) = executeFindCommand(2) - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - (0..4).each { cursor.next() } - !cursor.hasNext() - !cursor.hasNext() - - when: - cursor.next() - - then: - thrown(NoSuchElementException) - } - - @SuppressWarnings('BracesForTryCatchFinally') - @IgnoreIf({ isSharded() }) - def 'should throw cursor not found exception'() { - given: - def (serverAddress, commandResult) = executeFindCommand(2) - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - def serverCursor = cursor.getServerCursor() - def connection = connectionSource.getConnection() - connection.command(getNamespace().databaseName, - new BsonDocument('killCursors', new BsonString(namespace.getCollectionName())) - .append('cursors', new BsonArray(singletonList(new BsonInt64(serverCursor.getId())))), - new NoOpFieldNameValidator(), ReadPreference.primary(), new BsonDocumentCodec(), connectionSource) - connection.release() - cursor.next() - - then: - try { - cursor.next() - } catch (MongoCursorNotFoundException e) { - assertEquals(serverCursor.getId(), e.getCursorId()) - assertEquals(serverCursor.getAddress(), e.getServerAddress()) - } catch (ignored) { - fail('Expected MongoCursorNotFoundException to be thrown but got ' + ignored.getClass()) - } - } - - def 'should report available documents'() { - given: - def (serverAddress, commandResult) = executeFindCommand(3) - - when: - cursor = new CommandBatchCursor(serverAddress, commandResult, 0, 2, 0, new DocumentCodec(), - null, connectionSource, connection) - - then: - cursor.available() == 3 - - when: - cursor.hasNext() - - then: - cursor.available() == 3 - - when: - cursor.next() - - then: - cursor.available() == 0 - - when: - cursor.hasNext() - - then: - cursor.available() == 2 - - when: - cursor.next() - - then: - cursor.available() == 0 - - when: - cursor.hasNext() - - then: - cursor.available() == 2 - - when: - cursor.close() - - then: - cursor.available() == 0 - } - - private Tuple2 executeFindCommand() { - executeFindCommand(0) - } - - private Tuple2 executeFindCommand(int batchSize) { - executeFindCommand(new BsonDocument(), 0, batchSize, false, false, ReadPreference.primary()) - } - - private Tuple2 executeFindCommand(int limit, int batchSize) { - executeFindCommand(new BsonDocument(), limit, batchSize, false, false, ReadPreference.primary()) - } - - private Tuple2 executeFindCommand(BsonDocument filter, int limit, int batchSize, boolean tailable, - boolean awaitData) { - executeFindCommand(filter, limit, batchSize, tailable, awaitData, ReadPreference.primary()) - } - - private Tuple2 executeFindCommand(BsonDocument filter, int limit, int batchSize, boolean tailable, - boolean awaitData, ReadPreference readPreference) { - def findCommand = new BsonDocument('find', new BsonString(getCollectionName())) - .append('filter', filter) - .append('tailable', BsonBoolean.valueOf(tailable)) - .append('awaitData', BsonBoolean.valueOf(awaitData)) - - findCommand.append('limit', new BsonInt32(Math.abs(limit))) - - if (limit >= 0) { - if (batchSize < 0 && Math.abs(batchSize) < limit) { - findCommand.append('limit', new BsonInt32(Math.abs(batchSize))) - } else { - findCommand.append('batchSize', new BsonInt32(Math.abs(batchSize))) - } - } - - def response = connection.command(getDatabaseName(), findCommand, - NO_OP_FIELD_NAME_VALIDATOR, readPreference, - CommandResultDocumentCodec.create(new DocumentCodec(), 'firstBatch'), connectionSource) - new Tuple2(connection.getDescription().getServerAddress(), response) - } -} diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalTest.java b/driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalTest.java new file mode 100644 index 00000000000..10d0997f5d1 --- /dev/null +++ b/driver-core/src/test/functional/com/mongodb/internal/operation/CommandBatchCursorFunctionalTest.java @@ -0,0 +1,554 @@ +/* + * Copyright 2008-present MongoDB, 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.mongodb.internal.operation; + + +import com.mongodb.MongoCursorNotFoundException; +import com.mongodb.MongoQueryException; +import com.mongodb.ReadPreference; +import com.mongodb.ServerAddress; +import com.mongodb.ServerCursor; +import com.mongodb.client.model.CreateCollectionOptions; +import com.mongodb.client.model.OperationTest; +import com.mongodb.internal.binding.ConnectionSource; +import com.mongodb.internal.connection.Connection; +import org.bson.BsonArray; +import org.bson.BsonBoolean; +import org.bson.BsonDocument; +import org.bson.BsonInt32; +import org.bson.BsonInt64; +import org.bson.BsonString; +import org.bson.BsonTimestamp; +import org.bson.Document; +import org.bson.codecs.BsonDocumentCodec; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static com.mongodb.ClusterFixture.checkReferenceCountReachesTarget; +import static com.mongodb.ClusterFixture.getBinding; +import static com.mongodb.ClusterFixture.getReferenceCountAfterTimeout; +import static com.mongodb.ClusterFixture.isSharded; +import static com.mongodb.internal.operation.QueryOperationHelper.makeAdditionalGetMoreCall; +import static java.util.Collections.singletonList; +import static java.util.stream.Stream.generate; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assumptions.assumeFalse; +import static org.junit.jupiter.params.provider.Arguments.arguments; + +public class CommandBatchCursorFunctionalTest extends OperationTest { + + private ConnectionSource connectionSource; + private Connection connection; + private CommandBatchCursor cursor; + + @BeforeEach + void setup() { + List documents = IntStream.rangeClosed(1, 10) + .mapToObj(i -> new BsonDocument("i", new BsonInt32(i))) + .collect(Collectors.toList()); + getCollectionHelper().insertDocuments(documents); + + connectionSource = getBinding().getWriteConnectionSource(); + connection = connectionSource.getConnection(); + } + + @AfterEach + void cleanup() { + ifNotNull(cursor, CommandBatchCursor::close); + ifNotNull(connectionSource, cs -> { + getReferenceCountAfterTimeout(cs, 1); + cs.release(); + }); + ifNotNull(connection, c -> { + getReferenceCountAfterTimeout(c, 1); + c.release(); + }); + } + + @Test + @DisplayName("server cursor should not be null") + void theServerCursorShouldNotBeNull() { + BsonDocument commandResult = executeFindCommand(2); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 0, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertNotNull(cursor.getServerCursor()); + } + + @Test + @DisplayName("test server address should not be null") + void theServerAddressShouldNotNull() { + BsonDocument commandResult = executeFindCommand(); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 0, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertNotNull(cursor.getServerAddress()); + } + + @Test + @DisplayName("should get Exceptions for operations on the cursor after closing") + void shouldGetExceptionsForOperationsOnTheCursorAfterClosing() { + BsonDocument commandResult = executeFindCommand(); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 0, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + cursor.close(); + + assertDoesNotThrow(() -> cursor.close()); + assertThrows(IllegalStateException.class, () -> cursor.hasNext()); + assertThrows(IllegalStateException.class, () -> cursor.next()); + assertThrows(IllegalStateException.class, () -> cursor.getServerCursor()); + } + + @Test + @DisplayName("should throw an Exception when going off the end") + void shouldThrowAnExceptionWhenGoingOffTheEnd() { + BsonDocument commandResult = executeFindCommand(1); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 2, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + cursor.next(); + cursor.next(); + assertThrows(NoSuchElementException.class, () -> cursor.next()); + } + + @Test + @DisplayName("test cursor remove") + void testCursorRemove() { + BsonDocument commandResult = executeFindCommand(); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 0, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertThrows(UnsupportedOperationException.class, () -> cursor.remove()); + } + + @Test + @DisplayName("test normal exhaustion") + void testNormalExhaustion() { + BsonDocument commandResult = executeFindCommand(); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 0, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertEquals(10, cursorFlatten().size()); + } + + @ParameterizedTest(name = "{index} => limit={0}, batchSize={1}, expectedTotal={2}") + @MethodSource + @DisplayName("test limit exhaustion") + void testLimitExhaustion(final int limit, final int batchSize, final int expectedTotal) { + BsonDocument commandResult = executeFindCommand(limit, batchSize); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, limit, batchSize, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertEquals(expectedTotal, cursorFlatten().size()); + + checkReferenceCountReachesTarget(connectionSource, 1); + checkReferenceCountReachesTarget(connection, 1); + } + + @ParameterizedTest(name = "{index} => awaitData={0}, maxTimeMS={1}") + @MethodSource + @DisplayName("should block waiting for next batch on a tailable cursor") + void shouldBlockWaitingForNextBatchOnATailableCursor(final boolean awaitData, final int maxTimeMS) { + + getCollectionHelper().create(getCollectionName(), new CreateCollectionOptions().capped(true).sizeInBytes(1000)); + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", 1).append("ts", new BsonTimestamp(5, 0))); + + BsonDocument commandResult = executeFindCommand(new BsonDocument("ts", + new BsonDocument("$gte", new BsonTimestamp(5, 0))), 0, 2, true, awaitData); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 0, 2, maxTimeMS, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertTrue(cursor.hasNext()); + assertEquals(1, cursor.next().get(0).get("_id")); + + new Thread(() -> { + sleep(100); + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", 2).append("ts", new BsonTimestamp(6, 0))); + }).start(); + + assertTrue(cursor.hasNext()); + assertEquals(2, cursor.next().get(0).get("_id")); + } + + @Test + @DisplayName("test tryNext with tailable") + void testTryNextWithTailable() { + getCollectionHelper().create(getCollectionName(), new CreateCollectionOptions().capped(true).sizeInBytes(1000)); + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", 1).append("ts", new BsonTimestamp(5, 0))); + + BsonDocument commandResult = executeFindCommand(new BsonDocument("ts", + new BsonDocument("$gte", new BsonTimestamp(5, 0))), 0, 2, true, true); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 0, 2, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + + List nextBatch = cursor.tryNext(); + assertNotNull(nextBatch); + assertEquals(1, nextBatch.get(0).get("_id")); + + nextBatch = cursor.tryNext(); + assertNull(nextBatch); + + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", 2).append("ts", new BsonTimestamp(6, 0))); + + nextBatch = cursor.tryNext(); + assertNotNull(nextBatch); + assertEquals(2, nextBatch.get(0).get("_id")); + } + + @Test + @DisplayName("hasNext should throw when cursor is closed in another thread") + void hasNextShouldThrowWhenCursorIsClosedInAnotherThread() throws InterruptedException { + + getCollectionHelper().create(getCollectionName(), new CreateCollectionOptions().capped(true).sizeInBytes(1000)); + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", 1).append("ts", new BsonTimestamp(5, 0))); + + BsonDocument commandResult = executeFindCommand(new BsonDocument("ts", + new BsonDocument("$gte", new BsonTimestamp(5, 0))), 0, 2, true, true); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 0, 2, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertTrue(cursor.hasNext()); + assertEquals(1, cursor.next().get(0).get("_id")); + + CountDownLatch latch = new CountDownLatch(1); + new Thread(() -> { + sleep(100); + cursor.close(); + latch.countDown(); + }).start(); + + assertTrue(latch.await(5, TimeUnit.SECONDS)); + assertThrows(IllegalStateException.class, () -> cursor.hasNext()); + } + + @Test + @DisplayName("test maxTimeMS") + void testMaxTimeMS() { + assumeFalse(isSharded()); + getCollectionHelper().create(getCollectionName(), new CreateCollectionOptions().capped(true).sizeInBytes(1000)); + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", 1).append("ts", new BsonTimestamp(5, 0))); + + long maxTimeMS = 500; + BsonDocument commandResult = executeFindCommand(new BsonDocument("ts", + new BsonDocument("$gte", new BsonTimestamp(5, 0))), 0, 2, true, true); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 0, 2, maxTimeMS, DOCUMENT_DECODER, + null, connectionSource, connection); + + + List nextBatch = cursor.tryNext(); + assertNotNull(nextBatch); + + long startTime = System.currentTimeMillis(); + nextBatch = cursor.tryNext(); + long endTime = System.currentTimeMillis(); + + assertNull(nextBatch); + + // RACY TEST: no guarantee assertion will fire within the given timeframe + assertTrue(endTime - startTime < (maxTimeMS + 200)); + } + + @Test + @DisplayName("test tailable interrupt") + void testTailableInterrupt() throws InterruptedException { + getCollectionHelper().create(getCollectionName(), new CreateCollectionOptions().capped(true).sizeInBytes(1000)); + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", 1).append("ts", new BsonTimestamp(5, 0))); + + BsonDocument commandResult = executeFindCommand(new BsonDocument("ts", + new BsonDocument("$gte", new BsonTimestamp(5, 0))), 0, 2, true, true); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 0, 2, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + CountDownLatch latch = new CountDownLatch(1); + AtomicInteger seen = new AtomicInteger(); + Thread thread = new Thread(() -> { + try { + cursor.next(); + seen.incrementAndGet(); + cursor.next(); + seen.incrementAndGet(); + } catch (Exception e) { + // pass + } finally { + latch.countDown(); + } + }); + + thread.start(); + sleep(1000); + thread.interrupt(); + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", 2)); + latch.await(); + + assertTrue(latch.await(5, TimeUnit.SECONDS)); + assertEquals(1, seen.intValue()); + } + + @Test + @DisplayName("should kill cursor if limit is reached on initial query") + void shouldKillCursorIfLimitIsReachedOnInitialQuery() { + assumeFalse(isSharded()); + BsonDocument commandResult = executeFindCommand(5); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 5, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertNotNull(cursor.next()); + assertFalse(cursor.hasNext()); + assertNull(cursor.getServerCursor()); + } + + @Test + @DisplayName("should kill cursor if limit is reached on getMore") + void shouldKillCursorIfLimitIsReachedOnGetMore() { + assumeFalse(isSharded()); + BsonDocument commandResult = executeFindCommand(); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 5, 3, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + ServerCursor serverCursor = cursor.getServerCursor(); + assertNotNull(serverCursor); + assertNotNull(cursor.next()); + assertNotNull(cursor.next()); + + assertDoesNotThrow(() -> checkReferenceCountReachesTarget(connection, 1)); + assertThrows(MongoQueryException.class, () -> + makeAdditionalGetMoreCall(getNamespace(), serverCursor, connection) + ); + } + + @Test + @DisplayName("should release connection source if limit is reached on initial query") + void shouldReleaseConnectionSourceIfLimitIsReachedOnInitialQuery() { + assumeFalse(isSharded()); + BsonDocument commandResult = executeFindCommand(5); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 5, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertNull(cursor.getServerCursor()); + assertDoesNotThrow(() -> checkReferenceCountReachesTarget(connectionSource, 1)); + assertDoesNotThrow(() -> checkReferenceCountReachesTarget(connection, 1)); + } + + @Test + @DisplayName("should release connection source if limit is reached on getMore") + void shouldReleaseConnectionSourceIfLimitIsReachedOnGetMore() { + assumeFalse(isSharded()); + BsonDocument commandResult = executeFindCommand(3); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 5, 3, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertNotNull(cursor.next()); + assertNotNull(cursor.next()); + assertDoesNotThrow(() -> checkReferenceCountReachesTarget(connectionSource, 1)); + assertDoesNotThrow(() -> checkReferenceCountReachesTarget(connection, 1)); + } + + @Test + @DisplayName("test limit with get more") + void testLimitWithGetMore() { + BsonDocument commandResult = executeFindCommand(2); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 5, 2, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertNotNull(cursor.next()); + assertNotNull(cursor.next()); + assertNotNull(cursor.next()); + assertFalse(cursor.hasNext()); + } + + @Test + @DisplayName("test limit with large documents") + void testLimitWithLargeDocuments() { + String bigString = generate(() -> "x") + .limit(16000) + .collect(Collectors.joining()); + + IntStream.range(11, 1000).forEach(i -> + getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", i).append("s", bigString)) + ); + + BsonDocument commandResult = executeFindCommand(300, 0); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 300, 0, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertEquals(300, cursorFlatten().size()); + } + + @Test + @DisplayName("should respect batch size") + void shouldRespectBatchSize() { + BsonDocument commandResult = executeFindCommand(2); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 0, 2, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertEquals(2, cursor.getBatchSize()); + assertEquals(2, cursor.next().size()); + assertEquals(2, cursor.next().size()); + + cursor.setBatchSize(3); + assertEquals(3, cursor.getBatchSize()); + assertEquals(3, cursor.next().size()); + assertEquals(3, cursor.next().size()); + } + + @Test + @DisplayName("should throw cursor not found exception") + void shouldThrowCursorNotFoundException() { + BsonDocument commandResult = executeFindCommand(2); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 0, 2, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + ServerCursor serverCursor = cursor.getServerCursor(); + assertNotNull(serverCursor); + Connection localConnection = connectionSource.getConnection(); + localConnection.command(getNamespace().getDatabaseName(), + new BsonDocument("killCursors", new BsonString(getNamespace().getCollectionName())) + .append("cursors", new BsonArray(singletonList(new BsonInt64(serverCursor.getId())))), + NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), new BsonDocumentCodec(), connectionSource); + localConnection.release(); + + cursor.next(); + + MongoCursorNotFoundException exception = assertThrows(MongoCursorNotFoundException.class, () -> cursor.next()); + assertEquals(serverCursor.getId(), exception.getCursorId()); + assertEquals(serverCursor.getAddress(), exception.getServerAddress()); + } + + @Test + @DisplayName("should report available documents") + void shouldReportAvailableDocuments() { + BsonDocument commandResult = executeFindCommand(3); + cursor = new CommandBatchCursor<>(getServerAddress(), commandResult, 0, 2, 0, DOCUMENT_DECODER, + null, connectionSource, connection); + + assertEquals(3, cursor.available()); + + cursor.next(); + assertEquals(0, cursor.available()); + + assertTrue(cursor.hasNext()); + assertEquals(2, cursor.available()); + + cursor.next(); + assertEquals(0, cursor.available()); + + assertTrue(cursor.hasNext()); + assertEquals(2, cursor.available()); + + cursor.close(); + assertEquals(0, cursor.available()); + } + + + private static Stream shouldBlockWaitingForNextBatchOnATailableCursor() { + return Stream.of( + arguments(true, 0), + arguments(true, 100), + arguments(false, 0)); + } + + private static Stream testLimitExhaustion() { + return Stream.of( + arguments(5, 2, 5), + arguments(5, -2, 2), + arguments(-5, -2, 5), + arguments(-5, 2, 5), + arguments(2, 5, 2), + arguments(2, -5, 2), + arguments(-2, 5, 2), + arguments(-2, -5, 2) + ); + } + + private ServerAddress getServerAddress() { + return connection.getDescription().getServerAddress(); + } + + private BsonDocument executeFindCommand() { + return executeFindCommand(0); + } + + private BsonDocument executeFindCommand(final int batchSize) { + return executeFindCommand(new BsonDocument(), 0, batchSize, false, false); + } + + private BsonDocument executeFindCommand(final int limit, final int batchSize) { + return executeFindCommand(new BsonDocument(), limit, batchSize, false, false); + } + + private BsonDocument executeFindCommand(final BsonDocument filter, final int limit, final int batchSize, final boolean tailable, + final boolean awaitData) { + return executeFindCommand(filter, limit, batchSize, tailable, awaitData, ReadPreference.primary()); + } + + private BsonDocument executeFindCommand(final BsonDocument filter, final int limit, final int batchSize, + final boolean tailable, final boolean awaitData, final ReadPreference readPreference) { + BsonDocument findCommand = new BsonDocument("find", new BsonString(getCollectionName())) + .append("filter", filter) + .append("tailable", BsonBoolean.valueOf(tailable)) + .append("awaitData", BsonBoolean.valueOf(awaitData)); + + findCommand.append("limit", new BsonInt32(Math.abs(limit))); + if (limit >= 0) { + if (batchSize < 0 && Math.abs(batchSize) < limit) { + findCommand.append("limit", new BsonInt32(Math.abs(batchSize))); + } else { + findCommand.append("batchSize", new BsonInt32(Math.abs(batchSize))); + } + } + + BsonDocument results = connection.command(getDatabaseName(), findCommand, + NO_OP_FIELD_NAME_VALIDATOR, readPreference, + CommandResultDocumentCodec.create(DOCUMENT_DECODER, "firstBatch"), + connectionSource); + + assertNotNull(results); + return results; + } + + private List cursorFlatten() { + List results = new ArrayList<>(); + while (cursor.hasNext()) { + results.addAll(cursor.next()); + } + return results; + } + +} diff --git a/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncCommandBatchCursorSpecification.groovy b/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncCommandBatchCursorSpecification.groovy index d12ae8744ff..a1878a20bb5 100644 --- a/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncCommandBatchCursorSpecification.groovy +++ b/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncCommandBatchCursorSpecification.groovy @@ -187,13 +187,7 @@ class AsyncCommandBatchCursorSpecification extends Specification { batch == thirdBatch connectionB.getCount() == 0 connectionSource.getCount() == 0 - - when: - batch = nextBatch(cursor) - - then: - batch.isEmpty() - connectionSource.getCount() == 0 + cursor.isClosed() } diff --git a/driver-reactive-streams/src/test/functional/com/mongodb/reactivestreams/client/internal/BatchCursorFluxTest.java b/driver-reactive-streams/src/test/functional/com/mongodb/reactivestreams/client/internal/BatchCursorFluxTest.java index 91f44bfddb7..ac5730d0d00 100644 --- a/driver-reactive-streams/src/test/functional/com/mongodb/reactivestreams/client/internal/BatchCursorFluxTest.java +++ b/driver-reactive-streams/src/test/functional/com/mongodb/reactivestreams/client/internal/BatchCursorFluxTest.java @@ -318,7 +318,6 @@ public void testBatchCursorDoesNotDropAnError() { Flux.fromIterable(asList(1, 2)) .flatMap(x -> Flux.from(collection.find())) .take(1) - ) .collectList() .block(TIMEOUT_DURATION); From ade133cf0f8495bca6849d1810492406dde62b67 Mon Sep 17 00:00:00 2001 From: Ross Lawley Date: Tue, 3 Oct 2023 12:58:54 +0100 Subject: [PATCH 6/9] Fix race condition ConnectionSource and Connection also need to be retained pre kill cursors and released in the kill cursors callback --- .../operation/AsyncCommandBatchCursor.java | 25 ++++--- .../operation/CommandBatchCursor.java | 4 +- .../operation/CursorResourceManager.java | 75 ++++++++++++------- .../internal/connection/ServerHelper.java | 15 +++- ...AsyncCommandBatchCursorFunctionalTest.java | 6 +- 5 files changed, 82 insertions(+), 43 deletions(-) diff --git a/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java index b73d88be6c7..d10a1f9705e 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/AsyncCommandBatchCursor.java @@ -296,6 +296,7 @@ void executeWithConnection( if (error instanceof MongoSocketException) { onCorruptedConnection(connection); } + connection.release(); callback.onResult(result, error); }); } @@ -308,9 +309,9 @@ void doClose() { } if (getServerCursor() != null) { - executeWithConnection(conn -> { - releaseServerResources(conn); - conn.release(); + executeWithConnection(c -> { + releaseServerResources(c); + c.release(); }, (r, t) -> { // guarantee that regardless of exceptions, `serverCursor` is null and client resources are released unsetServerCursor(); @@ -323,14 +324,16 @@ void doClose() { @Override void killServerCursor(final MongoNamespace namespace, final ServerCursor serverCursor, final AsyncConnection connection) { - connection - .retain() - .commandAsync(namespace.getDatabaseName(), getKillCursorsCommand(namespace, serverCursor), - NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), new BsonDocumentCodec(), - assertNotNull(getConnectionSource()), (r, t) -> { - connection.release(); - releaseClientResources(); - }); + connection.retain(); + AsyncConnectionSource connectionSource = assertNotNull(getConnectionSource()).retain(); + connection.commandAsync(namespace.getDatabaseName(), getKillCursorsCommand(namespace, serverCursor), + NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), new BsonDocumentCodec(), + connectionSource, (r, t) -> { + connectionSource.release(); + connection.release(); + releaseClientResources(); + }); } } + } diff --git a/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java index bff59e96f41..d48cc9906c7 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/CommandBatchCursor.java @@ -374,8 +374,8 @@ void doClose() { } @Override - void killServerCursor(final MongoNamespace namespace, final ServerCursor serverCursor, final Connection connection) { - connection.command(namespace.getDatabaseName(), getKillCursorsCommand(namespace, serverCursor), + void killServerCursor(final MongoNamespace namespace, final ServerCursor localServerCursor, final Connection localConnection) { + localConnection.command(namespace.getDatabaseName(), getKillCursorsCommand(namespace, localServerCursor), NO_OP_FIELD_NAME_VALIDATOR, ReadPreference.primary(), new BsonDocumentCodec(), assertNotNull(getConnectionSource())); releaseClientResources(); } diff --git a/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java b/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java index 533b6800c28..dcf0a232658 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java +++ b/driver-core/src/main/com/mongodb/internal/operation/CursorResourceManager.java @@ -31,6 +31,7 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.StampedLock; import java.util.function.Consumer; +import java.util.function.Supplier; import static com.mongodb.assertions.Assertions.assertNotNull; import static com.mongodb.assertions.Assertions.assertNull; @@ -139,8 +140,7 @@ boolean operable() { * @throws IllegalStateException Iff another operation is in progress. */ boolean tryStartOperation() throws IllegalStateException { - lock.lock(); - try { + return withLock(() -> { State localState = state; if (!localState.operable()) { return false; @@ -152,30 +152,26 @@ boolean tryStartOperation() throws IllegalStateException { } else { throw fail(state.toString()); } - } finally { - lock.unlock(); - } + }); } /** * Thread-safe. */ void endOperation() { - boolean doClose = false; - lock.lock(); - try { + boolean doClose = withLock(() -> { State localState = state; if (localState == State.OPERATION_IN_PROGRESS) { state = State.IDLE; } else if (localState == State.CLOSE_PENDING) { state = State.CLOSED; - doClose = true; - } else { + return true; + } else if (localState != State.CLOSED) { fail(localState.toString()); } - } finally { - lock.unlock(); - } + return false; + }); + if (doClose) { doClose(); } @@ -185,19 +181,17 @@ void endOperation() { * Thread-safe. */ void close() { - boolean doClose = false; - lock.lock(); - try { + boolean doClose = withLock(() -> { State localState = state; if (localState == State.OPERATION_IN_PROGRESS) { state = State.CLOSE_PENDING; } else if (localState != State.CLOSED) { state = State.CLOSED; - doClose = true; + return true; } - } finally { - lock.unlock(); - } + return false; + }); + if (doClose) { doClose(); } @@ -221,7 +215,9 @@ ServerCursor getServerCursor() { } void unsetServerCursor() { - this.serverCursor = null; + withLock(() -> { + this.serverCursor = null; + }); } void setServerCursor(@Nullable final ServerCursor serverCursor) { @@ -236,10 +232,11 @@ void setServerCursor(@Nullable final ServerCursor serverCursor) { } void releaseServerAndClientResources(final C connection) { - lock.lock(); - ServerCursor localServerCursor = serverCursor; - serverCursor = null; - lock.unlock(); + ServerCursor localServerCursor = withLockNullable(() -> { + ServerCursor local = serverCursor; + serverCursor = null; + return local; + }); if (localServerCursor != null) { killServerCursor(namespace, localServerCursor, connection); } else { @@ -274,6 +271,34 @@ void releaseClientResources() { } } + private void withLock(final Runnable runnable) { + try { + lock.lock(); + runnable.run(); + } finally { + lock.unlock(); + } + } + + @Nullable + private T withLockNullable(final Supplier supplier) { + try { + lock.lock(); + return supplier.get(); + } finally { + lock.unlock(); + } + } + + private T withLock(final Supplier supplier) { + try { + lock.lock(); + return supplier.get(); + } finally { + lock.unlock(); + } + } + enum State { IDLE(true, false), OPERATION_IN_PROGRESS(true, true), diff --git a/driver-core/src/test/functional/com/mongodb/internal/connection/ServerHelper.java b/driver-core/src/test/functional/com/mongodb/internal/connection/ServerHelper.java index ecbf4befb73..ef0f99966a0 100644 --- a/driver-core/src/test/functional/com/mongodb/internal/connection/ServerHelper.java +++ b/driver-core/src/test/functional/com/mongodb/internal/connection/ServerHelper.java @@ -64,9 +64,20 @@ public static void waitForLastRelease(final ServerAddress address, final Cluster private static void checkPool(final ServerAddress address, final Cluster cluster) { ConcurrentPool pool = connectionPool( cluster.selectServer(new ServerAddressSelector(address), new OperationContext()).getServer()); - if (pool.getInUseCount() > 0) { - throw new IllegalStateException("Connection pool in use count is " + pool.getInUseCount()); + + int counter = 0; + while (counter < 5) { + if (pool.getInUseCount() == 0) { + return; + } + try { + Thread.sleep(500); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + counter++; } + throw new IllegalStateException("Connection pool in use count is " + pool.getInUseCount()); } private static ConcurrentPool connectionPool(final Server server) { diff --git a/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalTest.java b/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalTest.java index edfa8e9bbbc..09c3a51be25 100644 --- a/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalTest.java +++ b/driver-core/src/test/functional/com/mongodb/internal/operation/AsyncCommandBatchCursorFunctionalTest.java @@ -116,13 +116,14 @@ void theServerCursorShouldNotBeNull() { @Test @DisplayName("should get Exceptions for operations on the cursor after closing") void shouldGetExceptionsForOperationsOnTheCursorAfterClosing() { - BsonDocument commandResult = executeFindCommand(); + BsonDocument commandResult = executeFindCommand(5); cursor = new AsyncCommandBatchCursor<>(getServerAddress(), commandResult, 0, 0, 0, DOCUMENT_DECODER, null, connectionSource, connection); cursor.close(); - assertDoesNotThrow(() -> cursor.close()); + + checkReferenceCountReachesTarget(connectionSource, 1); assertThrows(MongoException.class, this::cursorNext); assertNull(cursor.getServerCursor()); } @@ -170,7 +171,6 @@ void testLimitExhaustion(final int limit, final int batchSize, final int expecte @MethodSource @DisplayName("should block waiting for next batch on a tailable cursor") void shouldBlockWaitingForNextBatchOnATailableCursor(final boolean awaitData, final int maxTimeMS) { - getCollectionHelper().create(getCollectionName(), new CreateCollectionOptions().capped(true).sizeInBytes(1000)); getCollectionHelper().insertDocuments(DOCUMENT_DECODER, new Document("_id", 1).append("ts", new BsonTimestamp(5, 0))); From e1331af59fd925f5899ae47203e7998a99fbff0b Mon Sep 17 00:00:00 2001 From: Ross Lawley Date: Mon, 9 Oct 2023 11:34:21 +0100 Subject: [PATCH 7/9] Added SingleBatchCursor unit tests --- .../internal/operation/SingleBatchCursor.java | 4 - ...AsyncSingleBatchCursorSpecification.groovy | 65 ------------- .../operation/AsyncSingleBatchCursorTest.java | 84 +++++++++++++++++ .../operation/SingleBatchCursorTest.java | 92 +++++++++++++++++++ 4 files changed, 176 insertions(+), 69 deletions(-) delete mode 100644 driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorSpecification.groovy create mode 100644 driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorTest.java create mode 100644 driver-core/src/test/unit/com/mongodb/internal/operation/SingleBatchCursorTest.java diff --git a/driver-core/src/main/com/mongodb/internal/operation/SingleBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/SingleBatchCursor.java index 6d734892fb3..74ee4b79301 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/SingleBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/SingleBatchCursor.java @@ -41,10 +41,6 @@ static SingleBatchCursor createEmptyBatchCursor(final ServerAddress serve this.hasNext = !batch.isEmpty(); } - public List getBatch() { - return batch; - } - @Override public boolean hasNext() { return hasNext; diff --git a/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorSpecification.groovy b/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorSpecification.groovy deleted file mode 100644 index ebc007b987c..00000000000 --- a/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorSpecification.groovy +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Copyright 2008-present MongoDB, 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.mongodb.internal.operation - -import com.mongodb.MongoException -import com.mongodb.async.FutureResultCallback -import org.bson.Document -import spock.lang.Specification - -class AsyncSingleBatchCursorSpecification extends Specification { - - def 'should work as expected'() { - given: - def cursor = new AsyncSingleBatchCursor(firstBatch, 0) - - when: - def batch = nextBatch(cursor) - - then: - batch == firstBatch - - then: - nextBatch(cursor) == [] - - when: - nextBatch(cursor) - - then: - thrown(MongoException) - } - - def 'should not support setting batchsize'() { - given: - def cursor = new AsyncSingleBatchCursor(firstBatch, 0) - - when: - cursor.setBatchSize(1) - - then: - cursor.getBatchSize() == 0 - } - - - List nextBatch(AsyncSingleBatchCursor cursor) { - def futureResultCallback = new FutureResultCallback() - cursor.next(futureResultCallback) - futureResultCallback.get() - } - - def firstBatch = [new Document('a', 1)] -} diff --git a/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorTest.java b/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorTest.java new file mode 100644 index 00000000000..2a27aaf428b --- /dev/null +++ b/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorTest.java @@ -0,0 +1,84 @@ +/* + * Copyright 2008-present MongoDB, 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.mongodb.internal.operation; + +import com.mongodb.MongoException; +import com.mongodb.async.FutureResultCallback; +import org.bson.Document; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static com.mongodb.ClusterFixture.TIMEOUT; +import static com.mongodb.internal.operation.AsyncSingleBatchCursor.createEmptyBatchCursor; +import static java.util.Arrays.asList; +import static java.util.Collections.emptyList; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertIterableEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + + +class AsyncSingleBatchCursorTest { + + private static final List SINGLE_BATCH = asList(new Document("a", 1), new Document("b", 2)); + + @Test + @DisplayName("should work as expected") + void shouldWorkAsExpected() { + try (AsyncSingleBatchCursor cursor = new AsyncSingleBatchCursor<>(SINGLE_BATCH, 0)) { + + assertIterableEquals(SINGLE_BATCH, nextBatch(cursor)); + assertIterableEquals(emptyList(), nextBatch(cursor)); + assertTrue(cursor.isClosed()); + + assertThrows(MongoException.class, () -> nextBatch(cursor)); + } + } + + @Test + @DisplayName("should work as expected emptyCursor") + void shouldWorkAsExpectedEmptyCursor() { + try (AsyncSingleBatchCursor cursor = createEmptyBatchCursor(0)) { + assertIterableEquals(emptyList(), nextBatch(cursor)); + assertTrue(cursor.isClosed()); + + assertThrows(MongoException.class, () -> nextBatch(cursor)); + } + } + + @Test + @DisplayName("should not support setting batch size") + void shouldNotSupportSettingBatchSize() { + try (AsyncSingleBatchCursor cursor = new AsyncSingleBatchCursor<>(SINGLE_BATCH, 0)) { + + assertEquals(0, cursor.getBatchSize()); + + cursor.setBatchSize(1); + assertEquals(0, cursor.getBatchSize()); + } + } + + List nextBatch(final AsyncSingleBatchCursor cursor) { + FutureResultCallback> futureResultCallback = new FutureResultCallback<>(); + cursor.next(futureResultCallback); + return futureResultCallback.get(TIMEOUT, TimeUnit.MILLISECONDS); + } + +} diff --git a/driver-core/src/test/unit/com/mongodb/internal/operation/SingleBatchCursorTest.java b/driver-core/src/test/unit/com/mongodb/internal/operation/SingleBatchCursorTest.java new file mode 100644 index 00000000000..f15812687b4 --- /dev/null +++ b/driver-core/src/test/unit/com/mongodb/internal/operation/SingleBatchCursorTest.java @@ -0,0 +1,92 @@ +/* + * Copyright 2008-present MongoDB, 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.mongodb.internal.operation; + +import com.mongodb.ServerAddress; +import org.bson.Document; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.NoSuchElementException; + +import static com.mongodb.assertions.Assertions.assertFalse; +import static com.mongodb.assertions.Assertions.assertNull; +import static com.mongodb.internal.connection.tlschannel.util.Util.assertTrue; +import static java.util.Arrays.asList; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertIterableEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + + +class SingleBatchCursorTest { + + private static final List SINGLE_BATCH = asList(new Document("a", 1), new Document("b", 2)); + private static final ServerAddress SERVER_ADDRESS = new ServerAddress(); + + @Test + @DisplayName("should work as expected") + void shouldWorkAsExpected() { + + try (SingleBatchCursor cursor = new SingleBatchCursor<>(SINGLE_BATCH, 0, SERVER_ADDRESS)) { + assertEquals(SERVER_ADDRESS, cursor.getServerAddress()); + assertEquals(1, cursor.available()); + assertNull(cursor.getServerCursor()); + + assertTrue(cursor.hasNext()); + assertIterableEquals(SINGLE_BATCH, cursor.next()); + assertEquals(0, cursor.available()); + + assertFalse(cursor.hasNext()); + assertThrows(NoSuchElementException.class, cursor::next); + } + } + + @Test + @DisplayName("should work as expected emptyCursor") + void shouldWorkAsExpectedEmptyCursor() { + try (SingleBatchCursor cursor = SingleBatchCursor.createEmptyBatchCursor(SERVER_ADDRESS, 0)) { + assertEquals(SERVER_ADDRESS, cursor.getServerAddress()); + assertEquals(0, cursor.available()); + assertNull(cursor.getServerCursor()); + + assertFalse(cursor.hasNext()); + assertThrows(NoSuchElementException.class, cursor::next); + } + } + + @Test + @DisplayName("should work as expected with try methods") + void shouldWorkAsExpectedWithTryMethods() { + try (SingleBatchCursor cursor = new SingleBatchCursor<>(SINGLE_BATCH, 0, SERVER_ADDRESS)) { + assertIterableEquals(SINGLE_BATCH, cursor.tryNext()); + assertNull(cursor.tryNext()); + } + } + + @Test + @DisplayName("should not support setting batch size") + void shouldNotSupportSettingBatchSize() { + try (SingleBatchCursor cursor = new SingleBatchCursor<>(SINGLE_BATCH, 0, SERVER_ADDRESS)) { + assertEquals(0, cursor.getBatchSize()); + + cursor.setBatchSize(1); + assertEquals(0, cursor.getBatchSize()); + } + } + +} From dd440d1e061fa9e53c02db05855b9509b7ebc1ac Mon Sep 17 00:00:00 2001 From: Ross Lawley Date: Mon, 9 Oct 2023 11:39:26 +0100 Subject: [PATCH 8/9] Use static method imports --- .../internal/operation/AsyncSingleBatchCursor.java | 2 +- .../internal/operation/ListCollectionsOperation.java | 10 +++++++--- .../internal/operation/ListIndexesOperation.java | 6 ++++-- .../internal/operation/ListSearchIndexesOperation.java | 6 ++++-- .../mongodb/internal/operation/SingleBatchCursor.java | 2 +- .../internal/operation/AsyncSingleBatchCursorTest.java | 4 ++-- .../internal/operation/SingleBatchCursorTest.java | 3 ++- 7 files changed, 21 insertions(+), 12 deletions(-) diff --git a/driver-core/src/main/com/mongodb/internal/operation/AsyncSingleBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/AsyncSingleBatchCursor.java index 90cabeb074f..57b20ff1711 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/AsyncSingleBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/AsyncSingleBatchCursor.java @@ -26,7 +26,7 @@ class AsyncSingleBatchCursor implements AsyncBatchCursor { - static AsyncSingleBatchCursor createEmptyBatchCursor(final int batchSize) { + static AsyncSingleBatchCursor createEmptyAsyncSingleBatchCursor(final int batchSize) { return new AsyncSingleBatchCursor<>(emptyList(), batchSize); } diff --git a/driver-core/src/main/com/mongodb/internal/operation/ListCollectionsOperation.java b/driver-core/src/main/com/mongodb/internal/operation/ListCollectionsOperation.java index 28b5c9ad3af..f88e95d8b2f 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/ListCollectionsOperation.java +++ b/driver-core/src/main/com/mongodb/internal/operation/ListCollectionsOperation.java @@ -16,7 +16,7 @@ package com.mongodb.internal.operation; -import com.mongodb.MongoCommandException;; +import com.mongodb.MongoCommandException; import com.mongodb.internal.async.AsyncBatchCursor; import com.mongodb.internal.async.SingleResultCallback; import com.mongodb.internal.async.function.AsyncCallbackSupplier; @@ -42,6 +42,7 @@ import static com.mongodb.internal.operation.AsyncOperationHelper.cursorDocumentToAsyncBatchCursor; import static com.mongodb.internal.operation.AsyncOperationHelper.decorateReadWithRetriesAsync; import static com.mongodb.internal.operation.AsyncOperationHelper.withAsyncSourceAndConnection; +import static com.mongodb.internal.operation.AsyncSingleBatchCursor.createEmptyAsyncSingleBatchCursor; import static com.mongodb.internal.operation.CommandOperationHelper.initialRetryState; import static com.mongodb.internal.operation.CommandOperationHelper.isNamespaceError; import static com.mongodb.internal.operation.CommandOperationHelper.rethrowIfNotNamespaceError; @@ -49,12 +50,15 @@ import static com.mongodb.internal.operation.DocumentHelper.putIfNotNull; import static com.mongodb.internal.operation.OperationHelper.LOGGER; import static com.mongodb.internal.operation.OperationHelper.canRetryRead; +import static com.mongodb.internal.operation.SingleBatchCursor.createEmptySingleBatchCursor; import static com.mongodb.internal.operation.SyncOperationHelper.CommandReadTransformer; import static com.mongodb.internal.operation.SyncOperationHelper.createReadCommandAndExecute; import static com.mongodb.internal.operation.SyncOperationHelper.cursorDocumentToBatchCursor; import static com.mongodb.internal.operation.SyncOperationHelper.decorateReadWithRetries; import static com.mongodb.internal.operation.SyncOperationHelper.withSourceAndConnection; +; + /** * An operation that provides a cursor allowing iteration through the metadata of all the collections in a database. This operation * ensures that the value of the {@code name} field of each returned document is the simple name of the collection rather than the full @@ -145,7 +149,7 @@ public BatchCursor execute(final ReadBinding binding) { createCommandDecoder(), commandTransformer(), connection); } catch (MongoCommandException e) { return rethrowIfNotNamespaceError(e, - SingleBatchCursor.createEmptyBatchCursor(source.getServerDescription().getAddress(), batchSize)); + createEmptySingleBatchCursor(source.getServerDescription().getAddress(), batchSize)); } }) ); @@ -171,7 +175,7 @@ public void executeAsync(final AsyncReadBinding binding, final SingleResultCallb } else { releasingCallback.onResult(result != null ? result - : AsyncSingleBatchCursor.createEmptyBatchCursor(batchSize), null); + : createEmptyAsyncSingleBatchCursor(batchSize), null); } }); }) diff --git a/driver-core/src/main/com/mongodb/internal/operation/ListIndexesOperation.java b/driver-core/src/main/com/mongodb/internal/operation/ListIndexesOperation.java index 1c51862f39d..9ac0ea92458 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/ListIndexesOperation.java +++ b/driver-core/src/main/com/mongodb/internal/operation/ListIndexesOperation.java @@ -42,6 +42,7 @@ import static com.mongodb.internal.operation.AsyncOperationHelper.cursorDocumentToAsyncBatchCursor; import static com.mongodb.internal.operation.AsyncOperationHelper.decorateReadWithRetriesAsync; import static com.mongodb.internal.operation.AsyncOperationHelper.withAsyncSourceAndConnection; +import static com.mongodb.internal.operation.AsyncSingleBatchCursor.createEmptyAsyncSingleBatchCursor; import static com.mongodb.internal.operation.CommandOperationHelper.CommandCreator; import static com.mongodb.internal.operation.CommandOperationHelper.initialRetryState; import static com.mongodb.internal.operation.CommandOperationHelper.isNamespaceError; @@ -50,6 +51,7 @@ import static com.mongodb.internal.operation.DocumentHelper.putIfNotNull; import static com.mongodb.internal.operation.OperationHelper.LOGGER; import static com.mongodb.internal.operation.OperationHelper.canRetryRead; +import static com.mongodb.internal.operation.SingleBatchCursor.createEmptySingleBatchCursor; import static com.mongodb.internal.operation.SyncOperationHelper.CommandReadTransformer; import static com.mongodb.internal.operation.SyncOperationHelper.createReadCommandAndExecute; import static com.mongodb.internal.operation.SyncOperationHelper.cursorDocumentToBatchCursor; @@ -125,7 +127,7 @@ public BatchCursor execute(final ReadBinding binding) { createCommandDecoder(), transformer(), connection); } catch (MongoCommandException e) { return rethrowIfNotNamespaceError(e, - SingleBatchCursor.createEmptyBatchCursor(source.getServerDescription().getAddress(), batchSize)); + createEmptySingleBatchCursor(source.getServerDescription().getAddress(), batchSize)); } }) ); @@ -151,7 +153,7 @@ public void executeAsync(final AsyncReadBinding binding, final SingleResultCallb } else { releasingCallback.onResult(result != null ? result - : AsyncSingleBatchCursor.createEmptyBatchCursor(batchSize), + : createEmptyAsyncSingleBatchCursor(batchSize), null); } }); diff --git a/driver-core/src/main/com/mongodb/internal/operation/ListSearchIndexesOperation.java b/driver-core/src/main/com/mongodb/internal/operation/ListSearchIndexesOperation.java index db360097f94..4d9069b9009 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/ListSearchIndexesOperation.java +++ b/driver-core/src/main/com/mongodb/internal/operation/ListSearchIndexesOperation.java @@ -34,7 +34,9 @@ import java.util.Collections; import java.util.concurrent.TimeUnit; +import static com.mongodb.internal.operation.AsyncSingleBatchCursor.createEmptyAsyncSingleBatchCursor; import static com.mongodb.internal.operation.CommandOperationHelper.isNamespaceError; +import static com.mongodb.internal.operation.SingleBatchCursor.createEmptySingleBatchCursor; /** @@ -89,7 +91,7 @@ public BatchCursor execute(final ReadBinding binding) { if (!isNamespaceError(exception)) { throw exception; } else { - return SingleBatchCursor.createEmptyBatchCursor(exception.getServerAddress(), cursorBatchSize); + return createEmptySingleBatchCursor(exception.getServerAddress(), cursorBatchSize); } } } @@ -101,7 +103,7 @@ public void executeAsync(final AsyncReadBinding binding, final SingleResultCallb callback.onResult(null, exception); } else if (exception != null) { int cursorBatchSize = batchSize == null ? 0 : batchSize; - callback.onResult(AsyncSingleBatchCursor.createEmptyBatchCursor(cursorBatchSize), null); + callback.onResult(createEmptyAsyncSingleBatchCursor(cursorBatchSize), null); } else { callback.onResult(cursor, null); } diff --git a/driver-core/src/main/com/mongodb/internal/operation/SingleBatchCursor.java b/driver-core/src/main/com/mongodb/internal/operation/SingleBatchCursor.java index 74ee4b79301..8a673ee93d9 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/SingleBatchCursor.java +++ b/driver-core/src/main/com/mongodb/internal/operation/SingleBatchCursor.java @@ -25,7 +25,7 @@ class SingleBatchCursor implements BatchCursor { - static SingleBatchCursor createEmptyBatchCursor(final ServerAddress serverAddress, final int batchSize) { + static SingleBatchCursor createEmptySingleBatchCursor(final ServerAddress serverAddress, final int batchSize) { return new SingleBatchCursor<>(emptyList(), batchSize, serverAddress); } diff --git a/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorTest.java b/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorTest.java index 2a27aaf428b..561a4cf9f31 100644 --- a/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorTest.java +++ b/driver-core/src/test/unit/com/mongodb/internal/operation/AsyncSingleBatchCursorTest.java @@ -26,7 +26,7 @@ import java.util.concurrent.TimeUnit; import static com.mongodb.ClusterFixture.TIMEOUT; -import static com.mongodb.internal.operation.AsyncSingleBatchCursor.createEmptyBatchCursor; +import static com.mongodb.internal.operation.AsyncSingleBatchCursor.createEmptyAsyncSingleBatchCursor; import static java.util.Arrays.asList; import static java.util.Collections.emptyList; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -55,7 +55,7 @@ void shouldWorkAsExpected() { @Test @DisplayName("should work as expected emptyCursor") void shouldWorkAsExpectedEmptyCursor() { - try (AsyncSingleBatchCursor cursor = createEmptyBatchCursor(0)) { + try (AsyncSingleBatchCursor cursor = createEmptyAsyncSingleBatchCursor(0)) { assertIterableEquals(emptyList(), nextBatch(cursor)); assertTrue(cursor.isClosed()); diff --git a/driver-core/src/test/unit/com/mongodb/internal/operation/SingleBatchCursorTest.java b/driver-core/src/test/unit/com/mongodb/internal/operation/SingleBatchCursorTest.java index f15812687b4..a71f067f5d6 100644 --- a/driver-core/src/test/unit/com/mongodb/internal/operation/SingleBatchCursorTest.java +++ b/driver-core/src/test/unit/com/mongodb/internal/operation/SingleBatchCursorTest.java @@ -27,6 +27,7 @@ import static com.mongodb.assertions.Assertions.assertFalse; import static com.mongodb.assertions.Assertions.assertNull; import static com.mongodb.internal.connection.tlschannel.util.Util.assertTrue; +import static com.mongodb.internal.operation.SingleBatchCursor.createEmptySingleBatchCursor; import static java.util.Arrays.asList; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertIterableEquals; @@ -59,7 +60,7 @@ void shouldWorkAsExpected() { @Test @DisplayName("should work as expected emptyCursor") void shouldWorkAsExpectedEmptyCursor() { - try (SingleBatchCursor cursor = SingleBatchCursor.createEmptyBatchCursor(SERVER_ADDRESS, 0)) { + try (SingleBatchCursor cursor = createEmptySingleBatchCursor(SERVER_ADDRESS, 0)) { assertEquals(SERVER_ADDRESS, cursor.getServerAddress()); assertEquals(0, cursor.available()); assertNull(cursor.getServerCursor()); From fd734ed255f5fc77a890d3fa6d7149a41fb00277 Mon Sep 17 00:00:00 2001 From: Ross Lawley Date: Mon, 9 Oct 2023 13:25:37 +0100 Subject: [PATCH 9/9] Checkstyle fix --- .../mongodb/internal/operation/ListCollectionsOperation.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/driver-core/src/main/com/mongodb/internal/operation/ListCollectionsOperation.java b/driver-core/src/main/com/mongodb/internal/operation/ListCollectionsOperation.java index f88e95d8b2f..ea122fe0250 100644 --- a/driver-core/src/main/com/mongodb/internal/operation/ListCollectionsOperation.java +++ b/driver-core/src/main/com/mongodb/internal/operation/ListCollectionsOperation.java @@ -57,8 +57,6 @@ import static com.mongodb.internal.operation.SyncOperationHelper.decorateReadWithRetries; import static com.mongodb.internal.operation.SyncOperationHelper.withSourceAndConnection; -; - /** * An operation that provides a cursor allowing iteration through the metadata of all the collections in a database. This operation * ensures that the value of the {@code name} field of each returned document is the simple name of the collection rather than the full