From 9689c2b33e6c794504d38082258ae0b3bf88fc6e Mon Sep 17 00:00:00 2001 From: Olav Loite Date: Thu, 15 Oct 2020 10:18:20 +0200 Subject: [PATCH 1/4] feat: remove session pool preparing --- .../spanner/AsyncTransactionManagerImpl.java | 10 +- .../cloud/spanner/DatabaseClientImpl.java | 105 +- .../spanner/MetricRegistryConstants.java | 5 + .../com/google/cloud/spanner/SessionImpl.java | 18 +- .../com/google/cloud/spanner/SessionPool.java | 560 +-------- .../cloud/spanner/SessionPoolOptions.java | 23 +- .../com/google/cloud/spanner/SpannerImpl.java | 3 +- .../google/cloud/spanner/SpannerOptions.java | 36 - .../cloud/spanner/TransactionManagerImpl.java | 9 +- .../cloud/spanner/TransactionRunnerImpl.java | 7 +- .../google/cloud/spanner/AsyncRunnerTest.java | 31 +- .../spanner/AsyncTransactionManagerTest.java | 53 +- .../spanner/BatchCreateSessionsTest.java | 70 -- .../cloud/spanner/DatabaseClientImplTest.java | 145 +-- .../spanner/ITSessionPoolIntegrationTest.java | 22 +- .../cloud/spanner/InlineBeginBenchmark.java | 6 +- .../spanner/InlineBeginTransactionTest.java | 6 +- .../IntegrationTestWithClosedSessionsEnv.java | 22 +- ...adWriteTransactionWithInlineBeginTest.java | 4 - .../RetryOnInvalidatedSessionTest.java | 76 +- .../cloud/spanner/SessionPoolLeakTest.java | 27 +- .../spanner/SessionPoolMaintainerTest.java | 40 +- .../cloud/spanner/SessionPoolStressTest.java | 21 +- .../google/cloud/spanner/SessionPoolTest.java | 1025 ++++------------- .../com/google/cloud/spanner/SpanTest.java | 30 +- .../cloud/spanner/SpannerGaxRetryTest.java | 4 +- .../TransactionManagerAbortedTest.java | 16 +- .../spanner/TransactionManagerImplTest.java | 3 +- .../spanner/TransactionRunnerImplTest.java | 18 +- .../google/cloud/spanner/it/ITDMLTest.java | 21 +- .../it/ITTransactionManagerAsyncTest.java | 22 +- .../spanner/it/ITTransactionManagerTest.java | 23 +- .../cloud/spanner/it/ITTransactionTest.java | 23 +- 33 files changed, 491 insertions(+), 1993 deletions(-) diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/AsyncTransactionManagerImpl.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/AsyncTransactionManagerImpl.java index f6085adf38..9230df5838 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/AsyncTransactionManagerImpl.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/AsyncTransactionManagerImpl.java @@ -39,16 +39,14 @@ final class AsyncTransactionManagerImpl private final SessionImpl session; private Span span; - private final boolean inlineBegin; private TransactionRunnerImpl.TransactionContextImpl txn; private TransactionState txnState; private final SettableApiFuture commitTimestamp = SettableApiFuture.create(); - AsyncTransactionManagerImpl(SessionImpl session, Span span, boolean inlineBegin) { + AsyncTransactionManagerImpl(SessionImpl session, Span span) { this.session = session; this.span = span; - this.inlineBegin = inlineBegin; } @Override @@ -69,15 +67,15 @@ public TransactionContextFutureImpl beginAsync() { return begin; } - private ApiFuture internalBeginAsync(boolean setActive) { + private ApiFuture internalBeginAsync(boolean firstAttempt) { txnState = TransactionState.STARTED; txn = session.newTransaction(); - if (setActive) { + if (firstAttempt) { session.setActive(this); } final SettableApiFuture res = SettableApiFuture.create(); final ApiFuture fut; - if (inlineBegin) { + if (firstAttempt) { fut = ApiFutures.immediateFuture(null); } else { fut = txn.ensureTxnAsync(); diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/DatabaseClientImpl.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/DatabaseClientImpl.java index 3883134582..bc0e341642 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/DatabaseClientImpl.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/DatabaseClientImpl.java @@ -29,40 +29,26 @@ class DatabaseClientImpl implements DatabaseClient { private static final String READ_WRITE_TRANSACTION = "CloudSpanner.ReadWriteTransaction"; - private static final String READ_WRITE_TRANSACTION_WITH_INLINE_BEGIN = - "CloudSpanner.ReadWriteTransactionWithInlineBegin"; private static final String READ_ONLY_TRANSACTION = "CloudSpanner.ReadOnlyTransaction"; private static final String PARTITION_DML_TRANSACTION = "CloudSpanner.PartitionDMLTransaction"; private static final Tracer tracer = Tracing.getTracer(); - private enum SessionMode { - READ, - READ_WRITE - } - @VisibleForTesting final String clientId; @VisibleForTesting final SessionPool pool; - private final boolean inlineBeginReadWriteTransactions; @VisibleForTesting - DatabaseClientImpl(SessionPool pool, boolean inlineBeginReadWriteTransactions) { - this("", pool, inlineBeginReadWriteTransactions); + DatabaseClientImpl(SessionPool pool) { + this("", pool); } - DatabaseClientImpl(String clientId, SessionPool pool, boolean inlineBeginReadWriteTransactions) { + DatabaseClientImpl(String clientId, SessionPool pool) { this.clientId = clientId; this.pool = pool; - this.inlineBeginReadWriteTransactions = inlineBeginReadWriteTransactions; } @VisibleForTesting PooledSessionFuture getReadSession() { - return pool.getReadSession(); - } - - @VisibleForTesting - PooledSessionFuture getReadWriteSession() { - return pool.getReadWriteSession(); + return pool.get(); } @Override @@ -70,7 +56,6 @@ public Timestamp write(final Iterable mutations) throws SpannerExcepti Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { return runWithSessionRetry( - SessionMode.READ_WRITE, new Function() { @Override public Timestamp apply(Session session) { @@ -90,7 +75,6 @@ public Timestamp writeAtLeastOnce(final Iterable mutations) throws Spa Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { return runWithSessionRetry( - SessionMode.READ_WRITE, new Function() { @Override public Timestamp apply(Session session) { @@ -173,55 +157,21 @@ public ReadOnlyTransaction readOnlyTransaction(TimestampBound bound) { @Override public TransactionRunner readWriteTransaction() { - return inlineBeginReadWriteTransactions - ? inlinedReadWriteTransaction() - : preparedReadWriteTransaction(); - } - - private TransactionRunner preparedReadWriteTransaction() { Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { - return getReadWriteSession().readWriteTransaction(); - } catch (RuntimeException e) { - TraceUtil.setWithFailure(span, e); - throw e; - } finally { - span.end(TraceUtil.END_SPAN_OPTIONS); - } - } - - private TransactionRunner inlinedReadWriteTransaction() { - Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION_WITH_INLINE_BEGIN).startSpan(); - try (Scope s = tracer.withSpan(span)) { - // An inlined read/write transaction does not need a write-prepared session. return getReadSession().readWriteTransaction(); } catch (RuntimeException e) { TraceUtil.endSpanWithFailure(span, e); throw e; + } finally { + span.end(TraceUtil.END_SPAN_OPTIONS); } } @Override public TransactionManager transactionManager() { - return inlineBeginReadWriteTransactions - ? inlinedTransactionManager() - : preparedTransactionManager(); - } - - private TransactionManager preparedTransactionManager() { Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { - return getReadWriteSession().transactionManager(); - } catch (RuntimeException e) { - TraceUtil.endSpanWithFailure(span, e); - throw e; - } - } - - private TransactionManager inlinedTransactionManager() { - Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION_WITH_INLINE_BEGIN).startSpan(); - try (Scope s = tracer.withSpan(span)) { - // An inlined read/write transaction does not need a write-prepared session. return getReadSession().transactionManager(); } catch (RuntimeException e) { TraceUtil.endSpanWithFailure(span, e); @@ -231,21 +181,7 @@ private TransactionManager inlinedTransactionManager() { @Override public AsyncRunner runAsync() { - return inlineBeginReadWriteTransactions ? inlinedRunAsync() : preparedRunAsync(); - } - - private AsyncRunner preparedRunAsync() { Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION).startSpan(); - try (Scope s = tracer.withSpan(span)) { - return getReadWriteSession().runAsync(); - } catch (RuntimeException e) { - TraceUtil.endSpanWithFailure(span, e); - throw e; - } - } - - private AsyncRunner inlinedRunAsync() { - Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION_WITH_INLINE_BEGIN).startSpan(); try (Scope s = tracer.withSpan(span)) { return getReadSession().runAsync(); } catch (RuntimeException e) { @@ -256,23 +192,7 @@ private AsyncRunner inlinedRunAsync() { @Override public AsyncTransactionManager transactionManagerAsync() { - return inlineBeginReadWriteTransactions - ? inlinedTransactionManagerAsync() - : preparedTransactionManagerAsync(); - } - - private AsyncTransactionManager preparedTransactionManagerAsync() { Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION).startSpan(); - try (Scope s = tracer.withSpan(span)) { - return getReadWriteSession().transactionManagerAsync(); - } catch (RuntimeException e) { - TraceUtil.endSpanWithFailure(span, e); - throw e; - } - } - - private AsyncTransactionManager inlinedTransactionManagerAsync() { - Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION_WITH_INLINE_BEGIN).startSpan(); try (Scope s = tracer.withSpan(span)) { return getReadSession().transactionManagerAsync(); } catch (RuntimeException e) { @@ -285,10 +205,7 @@ private AsyncTransactionManager inlinedTransactionManagerAsync() { public long executePartitionedUpdate(final Statement stmt) { Span span = tracer.spanBuilder(PARTITION_DML_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { - // A partitioned update transaction does not need a prepared write session, as the transaction - // object will start a new transaction with specific options anyway. return runWithSessionRetry( - SessionMode.READ, new Function() { @Override public Long apply(Session session) { @@ -301,17 +218,13 @@ public Long apply(Session session) { } } - private T runWithSessionRetry(SessionMode mode, Function callable) { - PooledSessionFuture session = - mode == SessionMode.READ_WRITE ? getReadWriteSession() : getReadSession(); + private T runWithSessionRetry(Function callable) { + PooledSessionFuture session = getReadSession(); while (true) { try { return callable.apply(session); } catch (SessionNotFoundException e) { - session = - mode == SessionMode.READ_WRITE - ? pool.replaceReadWriteSession(e, session) - : pool.replaceReadSession(e, session); + session = pool.replaceSession(e, session); } } } diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/MetricRegistryConstants.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/MetricRegistryConstants.java index 8da8ee1506..392fda729e 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/MetricRegistryConstants.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/MetricRegistryConstants.java @@ -36,9 +36,14 @@ class MetricRegistryConstants { private static final LabelValue UNSET_LABEL = LabelValue.create(null); static final LabelValue NUM_IN_USE_SESSIONS = LabelValue.create("num_in_use_sessions"); + + @Deprecated static final LabelValue NUM_SESSIONS_BEING_PREPARED = LabelValue.create("num_sessions_being_prepared"); + static final LabelValue NUM_READ_SESSIONS = LabelValue.create("num_read_sessions"); + + @Deprecated static final LabelValue NUM_WRITE_SESSIONS = LabelValue.create("num_write_prepared_sessions"); static final ImmutableList SPANNER_LABEL_KEYS = diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionImpl.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionImpl.java index d707cf0ef6..6a91d85fef 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionImpl.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionImpl.java @@ -228,34 +228,24 @@ public ReadOnlyTransaction readOnlyTransaction(TimestampBound bound) { @Override public TransactionRunner readWriteTransaction() { return setActive( - new TransactionRunnerImpl( - this, - spanner.getRpc(), - spanner.getDefaultPrefetchChunks(), - spanner.getOptions().isInlineBeginForReadWriteTransaction())); + new TransactionRunnerImpl(this, spanner.getRpc(), spanner.getDefaultPrefetchChunks())); } @Override public AsyncRunner runAsync() { return new AsyncRunnerImpl( setActive( - new TransactionRunnerImpl( - this, - spanner.getRpc(), - spanner.getDefaultPrefetchChunks(), - spanner.getOptions().isInlineBeginForReadWriteTransaction()))); + new TransactionRunnerImpl(this, spanner.getRpc(), spanner.getDefaultPrefetchChunks()))); } @Override public TransactionManager transactionManager() { - return new TransactionManagerImpl( - this, currentSpan, spanner.getOptions().isInlineBeginForReadWriteTransaction()); + return new TransactionManagerImpl(this, currentSpan); } @Override public AsyncTransactionManagerImpl transactionManagerAsync() { - return new AsyncTransactionManagerImpl( - this, currentSpan, spanner.getOptions().isInlineBeginForReadWriteTransaction()); + return new AsyncTransactionManagerImpl(this, currentSpan); } @Override diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPool.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPool.java index 2512024117..5707bae0fb 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPool.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPool.java @@ -64,7 +64,6 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.SettableFuture; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.protobuf.Empty; import io.opencensus.common.Scope; import io.opencensus.common.ToLongFunction; @@ -81,7 +80,6 @@ import io.opencensus.trace.Tracer; import io.opencensus.trace.Tracing; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; @@ -92,10 +90,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; @@ -109,10 +105,8 @@ import org.threeten.bp.Instant; /** - * Maintains a pool of sessions some of which might be prepared for write by invoking - * BeginTransaction rpc. It maintains two queues of sessions(read and write prepared) and two queues - * of waiters who are waiting for a session to become available. This class itself is thread safe - * and is meant to be used concurrently across multiple threads. + * Maintains a pool of sessions. This class itself is thread safe and is meant to be used + * concurrently across multiple threads. */ final class SessionPool { @@ -319,7 +313,7 @@ private void replaceSessionIfPossible(SessionNotFoundException notFound) { if (isSingleUse || !sessionUsedForQuery) { // This class is only used by read-only transactions, so we know that we only need a // read-only session. - session = sessionPool.replaceReadSession(notFound, session); + session = sessionPool.replaceSession(notFound, session); readContextDelegate = readContextDelegateSupplier.apply(session); } else { throw notFound; @@ -735,7 +729,7 @@ public TransactionContext begin() { try { return internalBegin(); } catch (SessionNotFoundException e) { - session = sessionPool.replaceReadWriteSession(e, session); + session = sessionPool.replaceSession(e, session); delegate = session.get().delegate.transactionManager(); } } @@ -748,7 +742,7 @@ private TransactionContext internalBegin() { } private SpannerException handleSessionNotFound(SessionNotFoundException notFound) { - session = sessionPool.replaceReadWriteSession(notFound, session); + session = sessionPool.replaceSession(notFound, session); delegate = session.get().delegate.transactionManager(); restartedAfterSessionNotFound = true; return SpannerExceptionFactory.newSpannerException( @@ -789,7 +783,7 @@ public TransactionContext resetForRetry() { return new SessionPoolTransactionContext(delegate.resetForRetry()); } } catch (SessionNotFoundException e) { - session = sessionPool.replaceReadWriteSession(e, session); + session = sessionPool.replaceSession(e, session); delegate = session.get().delegate.transactionManager(); restartedAfterSessionNotFound = true; } @@ -828,7 +822,7 @@ public TransactionState getState() { /** * {@link TransactionRunner} that automatically handles {@link SessionNotFoundException}s by - * replacing the underlying read/write session and then restarts the transaction. + * replacing the underlying session and then restarts the transaction. */ private static final class SessionPoolTransactionRunner implements TransactionRunner { private final SessionPool sessionPool; @@ -857,7 +851,7 @@ public T run(TransactionCallable callable) { result = getRunner().run(callable); break; } catch (SessionNotFoundException e) { - session = sessionPool.replaceReadWriteSession(e, session); + session = sessionPool.replaceSession(e, session); runner = session.get().delegate.readWriteTransaction(); } } @@ -915,8 +909,7 @@ public void run() { se = SpannerExceptionFactory.newSpannerException(t); } finally { if (se != null && se instanceof SessionNotFoundException) { - session = - sessionPool.replaceReadWriteSession((SessionNotFoundException) se, session); + session = sessionPool.replaceSession((SessionNotFoundException) se, session); } else { break; } @@ -965,109 +958,6 @@ private enum SessionState { CLOSING, } - /** - * Forwarding future that will return a {@link PooledSession}. If {@link #inProcessPrepare} has - * been set to true, the returned session will be prepared with a read/write session using the - * thread of the caller to {@link #get()}. This ensures that the executor that is responsible for - * background preparing of read/write transactions is not overwhelmed by requests in case of a - * large burst of write requests. Instead of filling up the queue of the background executor, the - * caller threads will be used for the BeginTransaction call. - */ - private final class ForwardingListenablePooledSessionFuture - extends SimpleForwardingListenableFuture { - private final boolean inProcessPrepare; - private final Span span; - private volatile boolean initialized = false; - private final Object prepareLock = new Object(); - private volatile PooledSession result; - private volatile SpannerException error; - - private ForwardingListenablePooledSessionFuture( - ListenableFuture delegate, boolean inProcessPrepare, Span span) { - super(delegate); - this.inProcessPrepare = inProcessPrepare; - this.span = span; - } - - @Override - public PooledSession get() throws InterruptedException, ExecutionException { - try { - return initialize(super.get()); - } catch (ExecutionException e) { - throw SpannerExceptionFactory.newSpannerException(e.getCause()); - } catch (InterruptedException e) { - throw SpannerExceptionFactory.propagateInterrupt(e); - } - } - - @Override - public PooledSession get(long timeout, TimeUnit unit) - throws InterruptedException, ExecutionException, TimeoutException { - try { - return initialize(super.get(timeout, unit)); - } catch (ExecutionException e) { - throw SpannerExceptionFactory.newSpannerException(e.getCause()); - } catch (InterruptedException e) { - throw SpannerExceptionFactory.propagateInterrupt(e); - } catch (TimeoutException e) { - throw SpannerExceptionFactory.propagateTimeout(e); - } - } - - private PooledSession initialize(PooledSession sess) { - if (!initialized) { - synchronized (prepareLock) { - if (!initialized) { - try { - result = prepare(sess); - } catch (Throwable t) { - error = SpannerExceptionFactory.newSpannerException(t); - } finally { - initialized = true; - } - } - } - } - if (error != null) { - throw error; - } - return result; - } - - private PooledSession prepare(PooledSession sess) { - if (inProcessPrepare && !sess.delegate.hasReadyTransaction()) { - while (true) { - try { - sess.prepareReadWriteTransaction(); - synchronized (lock) { - stopAutomaticPrepare = false; - } - break; - } catch (Throwable t) { - if (isClosed()) { - span.addAnnotation("Pool has been closed"); - throw new IllegalStateException("Pool has been closed"); - } - SpannerException e = newSpannerException(t); - WaiterFuture waiter = new WaiterFuture(); - synchronized (lock) { - handlePrepareSessionFailure(e, sess, false); - if (!isSessionNotFound(e)) { - throw e; - } - readWaiters.add(waiter); - } - sess = waiter.get(); - if (sess.delegate.hasReadyTransaction()) { - break; - } - } - } - } - return sess; - } - } - private PooledSessionFuture createPooledSessionFuture( ListenableFuture future, Span span) { return new PooledSessionFuture(future, span); @@ -1634,18 +1524,15 @@ private void removeIdleSessions(Instant currTime) { synchronized (lock) { // Determine the minimum last use time for a session to be deemed to still be alive. Remove // all sessions that have a lastUseTime before that time, unless it would cause us to go - // below MinSessions. Prefer to remove read sessions above write-prepared sessions. + // below MinSessions. Instant minLastUseTime = currTime.minus(options.getRemoveInactiveSessionAfter()); - for (Iterator iterator : - Arrays.asList( - readSessions.descendingIterator(), writePreparedSessions.descendingIterator())) { - while (iterator.hasNext()) { - PooledSession session = iterator.next(); - if (session.lastUseTime.isBefore(minLastUseTime)) { - if (session.state != SessionState.CLOSING) { - removeFromPool(session); - iterator.remove(); - } + Iterator iterator = sessions.descendingIterator(); + while (iterator.hasNext()) { + PooledSession session = iterator.next(); + if (session.lastUseTime.isBefore(minLastUseTime)) { + if (session.state != SessionState.CLOSING) { + removeFromPool(session); + iterator.remove(); } } } @@ -1675,12 +1562,7 @@ private void keepAliveSessions(Instant currTime) { while (numSessionsToKeepAlive > 0) { PooledSession sessionToKeepAlive = null; synchronized (lock) { - sessionToKeepAlive = findSessionToKeepAlive(readSessions, keepAliveThreshold, 0); - if (sessionToKeepAlive == null) { - sessionToKeepAlive = - findSessionToKeepAlive( - writePreparedSessions, keepAliveThreshold, readSessions.size()); - } + sessionToKeepAlive = findSessionToKeepAlive(sessions, keepAliveThreshold, 0); } if (sessionToKeepAlive == null) { break; @@ -1716,9 +1598,7 @@ private static enum Position { private final SessionClient sessionClient; private final ScheduledExecutorService executor; private final ExecutorFactory executorFactory; - private final ScheduledExecutorService prepareExecutor; - private final int prepareThreadPoolSize; final PoolMaintainer poolMaintainer; private final Clock clock; private final Object lock = new Object(); @@ -1740,19 +1620,10 @@ private static enum Position { private boolean stopAutomaticPrepare; @GuardedBy("lock") - private final LinkedList readSessions = new LinkedList<>(); - - @GuardedBy("lock") - private final LinkedList writePreparedSessions = new LinkedList<>(); - - @GuardedBy("lock") - private final Queue readWaiters = new LinkedList<>(); + private final LinkedList sessions = new LinkedList<>(); @GuardedBy("lock") - private final Queue readWriteWaiters = new LinkedList<>(); - - @GuardedBy("lock") - private int numSessionsBeingPrepared = 0; + private final Queue waiters = new LinkedList<>(); @GuardedBy("lock") private int numSessionsBeingCreated = 0; @@ -1769,12 +1640,6 @@ private static enum Position { @GuardedBy("lock") private long numSessionsReleased = 0; - @GuardedBy("lock") - private long numSessionsInProcessPrepared = 0; - - @GuardedBy("lock") - private long numSessionsAsyncPrepared = 0; - @GuardedBy("lock") private long numIdleSessionsRemoved = 0; @@ -1859,18 +1724,6 @@ private SessionPool( this.options = options; this.executorFactory = executorFactory; this.executor = executor; - if (executor instanceof ThreadPoolExecutor) { - prepareThreadPoolSize = Math.max(((ThreadPoolExecutor) executor).getCorePoolSize(), 1); - } else { - prepareThreadPoolSize = 8; - } - this.prepareExecutor = - Executors.newScheduledThreadPool( - prepareThreadPoolSize, - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("session-pool-prepare-%d") - .build()); this.sessionClient = sessionClient; this.clock = clock; this.poolMaintainer = new PoolMaintainer(); @@ -1884,19 +1737,6 @@ int getNumberOfSessionsInUse() { } } - long getNumberOfSessionsInProcessPrepared() { - synchronized (lock) { - return numSessionsInProcessPrepared; - } - } - - @VisibleForTesting - long getNumberOfSessionsAsyncPrepared() { - synchronized (lock) { - return numSessionsAsyncPrepared; - } - } - void removeFromPool(PooledSession session) { synchronized (lock) { if (isClosed()) { @@ -1918,24 +1758,10 @@ long numIdleSessionsRemoved() { } } - @VisibleForTesting - int getNumberOfAvailableWritePreparedSessions() { - synchronized (lock) { - return writePreparedSessions.size(); - } - } - @VisibleForTesting int getNumberOfSessionsInPool() { synchronized (lock) { - return readSessions.size() + writePreparedSessions.size() + numSessionsBeingPrepared; - } - } - - @VisibleForTesting - int getNumberOfWriteSessionsInPool() { - synchronized (lock) { - return writePreparedSessions.size() + numSessionsBeingPrepared; + return sessions.size(); } } @@ -1946,13 +1772,6 @@ int getNumberOfSessionsBeingCreated() { } } - @VisibleForTesting - int getNumberOfSessionsBeingPrepared() { - synchronized (lock) { - return numSessionsBeingPrepared; - } - } - @VisibleForTesting long getNumWaiterTimeouts() { return numWaiterTimeouts.get(); @@ -1989,11 +1808,6 @@ private boolean isDatabaseOrInstanceNotFound(SpannerException e) { return e instanceof DatabaseNotFoundException || e instanceof InstanceNotFoundException; } - private boolean shouldStopPrepareSessions(SpannerException e) { - return isDatabaseOrInstanceNotFound(e) - || SHOULD_STOP_PREPARE_SESSIONS_ERROR_CODES.contains(e.getErrorCode()); - } - private void invalidateSession(PooledSession session) { synchronized (lock) { if (isClosed()) { @@ -2031,7 +1845,7 @@ boolean isValid() { } /** - * Returns a session to be used for read requests to spanner. It will block if a session is not + * Returns a session to be used for requests to spanner. It will block if a session is not * currently available. In case the pool is exhausted and {@link * SessionPoolOptions#isFailIfPoolExhausted()} has been set, it will throw an exception. Returned * session must be closed by calling {@link Session#close()}. @@ -2040,13 +1854,12 @@ boolean isValid() { * *
    *
  1. If a read session is available, return that. - *
  2. Otherwise if a writePreparedSession is available, return that. *
  3. Otherwise if a session can be created, fire a creation request. *
  4. Wait for a session to become available. Note that this can be unblocked either by a * session being returned to the pool or a new session being created. *
*/ - PooledSessionFuture getReadSession() throws SpannerException { + PooledSessionFuture get() throws SpannerException { Span span = Tracing.getTracer().getCurrentSpan(); span.addAnnotation("Acquiring session"); WaiterFuture waiter = null; @@ -2065,151 +1878,39 @@ PooledSessionFuture getReadSession() throws SpannerException { resourceNotFoundException.getMessage()), resourceNotFoundException); } - sess = readSessions.poll(); + sess = sessions.poll(); if (sess == null) { - sess = writePreparedSessions.poll(); - if (sess == null) { - span.addAnnotation("No session available"); - maybeCreateSession(); - waiter = new WaiterFuture(); - readWaiters.add(waiter); - } else { - span.addAnnotation("Acquired read write session"); - } + span.addAnnotation("No session available"); + maybeCreateSession(); + waiter = new WaiterFuture(); + waiters.add(waiter); } else { - span.addAnnotation("Acquired read only session"); + span.addAnnotation("Acquired rsession"); } - return checkoutSession(span, sess, waiter, false, false); - } - } - - /** - * Returns a session which has been prepared for writes by invoking BeginTransaction rpc. It will - * block if such a session is not currently available.In case the pool is exhausted and {@link - * SessionPoolOptions#isFailIfPoolExhausted()} has been set, it will throw an exception. Returned - * session must closed by invoking {@link Session#close()}. - * - *

Implementation strategy: - * - *

    - *
  1. If a writePreparedSession is available, return that. - *
  2. Otherwise if we have an extra session being prepared for write, wait for that. - *
  3. Otherwise, if there is a read session available, start preparing that for write and wait. - *
  4. Otherwise start creating a new session and wait. - *
  5. Wait for write prepared session to become available. This can be unblocked either by the - * session create/prepare request we fired in above request or by a session being released - * to the pool which is then write prepared. - *
- */ - PooledSessionFuture getReadWriteSession() { - Span span = Tracing.getTracer().getCurrentSpan(); - span.addAnnotation("Acquiring read write session"); - PooledSession sess = null; - WaiterFuture waiter = null; - boolean inProcessPrepare = stopAutomaticPrepare; - synchronized (lock) { - if (closureFuture != null) { - span.addAnnotation("Pool has been closed"); - throw new IllegalStateException("Pool has been closed", closedException); - } - if (resourceNotFoundException != null) { - span.addAnnotation("Database has been deleted"); - throw SpannerExceptionFactory.newSpannerException( - ErrorCode.NOT_FOUND, - String.format( - "The session pool has been invalidated because a previous RPC returned 'Database not found': %s", - resourceNotFoundException.getMessage()), - resourceNotFoundException); - } - sess = writePreparedSessions.poll(); - if (sess == null) { - if (!inProcessPrepare && numSessionsBeingPrepared <= prepareThreadPoolSize) { - if (numSessionsBeingPrepared <= readWriteWaiters.size()) { - PooledSession readSession = readSessions.poll(); - if (readSession != null) { - span.addAnnotation( - "Acquired read only session. Preparing for read write transaction"); - prepareSession(readSession); - } else { - span.addAnnotation("No session available"); - maybeCreateSession(); - } - } - } else { - inProcessPrepare = true; - numSessionsInProcessPrepared++; - PooledSession readSession = readSessions.poll(); - if (readSession != null) { - // Create a read/write transaction in-process if there is already a queue for prepared - // sessions. This is more efficient than doing it asynchronously, as it scales with - // the number of user threads. The thread pool for asynchronously preparing sessions - // is fixed. - span.addAnnotation( - "Acquired read only session. Preparing in-process for read write transaction"); - sess = readSession; - } else { - span.addAnnotation("No session available"); - maybeCreateSession(); - } - } - if (sess == null) { - waiter = new WaiterFuture(); - if (inProcessPrepare) { - // inProcessPrepare=true means that we have already determined that the queue for - // preparing read/write sessions is larger than the number of threads in the prepare - // thread pool, and that it's more efficient to do the prepare in-process. We will - // therefore create a waiter for a read-only session, even though a read/write session - // has been requested. - readWaiters.add(waiter); - } else { - readWriteWaiters.add(waiter); - } - } - } else { - span.addAnnotation("Acquired read write session"); - } - return checkoutSession(span, sess, waiter, true, inProcessPrepare); + return checkoutSession(span, sess, waiter); } } private PooledSessionFuture checkoutSession( - final Span span, - final PooledSession readySession, - WaiterFuture waiter, - boolean write, - final boolean inProcessPrepare) { + final Span span, final PooledSession readySession, WaiterFuture waiter) { ListenableFuture sessionFuture; if (waiter != null) { logger.log( Level.FINE, "No session available in the pool. Blocking for one to become available/created"); - span.addAnnotation( - String.format( - "Waiting for %s session to be available", write ? "read write" : "read only")); + span.addAnnotation(String.format("Waiting for a session to come available")); sessionFuture = waiter; } else { SettableFuture fut = SettableFuture.create(); fut.set(readySession); sessionFuture = fut; } - ForwardingListenablePooledSessionFuture forwardingFuture = - new ForwardingListenablePooledSessionFuture(sessionFuture, inProcessPrepare, span); - PooledSessionFuture res = createPooledSessionFuture(forwardingFuture, span); + PooledSessionFuture res = createPooledSessionFuture(sessionFuture, span); res.markCheckedOut(); return res; } - PooledSessionFuture replaceReadSession(SessionNotFoundException e, PooledSessionFuture session) { - return replaceSession(e, session, false); - } - - PooledSessionFuture replaceReadWriteSession( - SessionNotFoundException e, PooledSessionFuture session) { - return replaceSession(e, session, true); - } - - private PooledSessionFuture replaceSession( - SessionNotFoundException e, PooledSessionFuture session, boolean write) { + PooledSessionFuture replaceSession(SessionNotFoundException e, PooledSessionFuture session) { if (!options.isFailIfSessionNotFound() && session.get().allowReplacing) { synchronized (lock) { numSessionsInUse--; @@ -2218,7 +1919,7 @@ private PooledSessionFuture replaceSession( } session.leakedException = null; invalidateSession(session.get()); - return write ? getReadWriteSession() : getReadSession(); + return get(); } else { throw e; } @@ -2258,47 +1959,29 @@ private void maybeCreateSession() { } } } - /** - * Releases a session back to the pool. This might cause one of the waiters to be unblocked. - * - *

Implementation note: - * - *

    - *
  1. If there are no pending waiters, either add to the read sessions queue or start preparing - * for write depending on what fraction of sessions are already prepared for writes. - *
  2. Otherwise either unblock a waiting reader or start preparing for a write. Exact strategy - * on which option we chose, in case there are both waiting readers and writers, is - * implemented in {@link #shouldUnblockReader} - *
- */ + /** Releases a session back to the pool. This might cause one of the waiters to be unblocked. */ private void releaseSession(PooledSession session, Position position) { Preconditions.checkNotNull(session); synchronized (lock) { if (closureFuture != null) { return; } - if (readWaiters.size() == 0 && numSessionsBeingPrepared >= readWriteWaiters.size()) { + if (waiters.size() == 0) { // No pending waiters - if (shouldPrepareSession()) { - prepareSession(session); - } else { - switch (position) { - case RANDOM: - if (!readSessions.isEmpty()) { - int pos = random.nextInt(readSessions.size() + 1); - readSessions.add(pos, session); - break; - } - // fallthrough - case FIRST: - default: - readSessions.addFirst(session); - } + switch (position) { + case RANDOM: + if (!sessions.isEmpty()) { + int pos = random.nextInt(sessions.size() + 1); + sessions.add(pos, session); + break; + } + // fallthrough + case FIRST: + default: + sessions.addFirst(session); } - } else if (shouldUnblockReader()) { - readWaiters.poll().put(session); } else { - prepareSession(session); + waiters.poll().put(session); } } } @@ -2306,10 +1989,8 @@ private void releaseSession(PooledSession session, Position position) { private void handleCreateSessionsFailure(SpannerException e, int count) { synchronized (lock) { for (int i = 0; i < count; i++) { - if (readWaiters.size() > 0) { - readWaiters.poll().put(e); - } else if (readWriteWaiters.size() > 0) { - readWriteWaiters.poll().put(e); + if (waiters.size() > 0) { + waiters.poll().put(e); } else { break; } @@ -2320,42 +2001,6 @@ private void handleCreateSessionsFailure(SpannerException e, int count) { } } - private void handlePrepareSessionFailure( - SpannerException e, PooledSession session, boolean informFirstWaiter) { - synchronized (lock) { - if (isSessionNotFound(e)) { - invalidateSession(session); - } else if (shouldStopPrepareSessions(e)) { - // Database has been deleted or the user has no permission to write to this database, or - // there is some other semi-permanent error. We should stop trying to prepare any - // transactions. Also propagate the error to all waiters if the database or instance has - // been deleted, as any further waiting is pointless. - stopAutomaticPrepare = true; - while (readWriteWaiters.size() > 0) { - readWriteWaiters.poll().put(e); - } - while (readWaiters.size() > 0) { - readWaiters.poll().put(e); - } - if (isDatabaseOrInstanceNotFound(e)) { - // Remove the session from the pool. - if (isClosed()) { - decrementPendingClosures(1); - } - allSessions.remove(session); - setResourceNotFoundException((ResourceNotFoundException) e); - } else { - releaseSession(session, Position.FIRST); - } - } else if (informFirstWaiter && readWriteWaiters.size() > 0) { - releaseSession(session, Position.FIRST); - readWriteWaiters.poll().put(e); - } else { - releaseSession(session, Position.FIRST); - } - } - } - void setResourceNotFoundException(ResourceNotFoundException e) { this.resourceNotFoundException = MoreObjects.firstNonNull(this.resourceNotFoundException, e); } @@ -2368,9 +2013,9 @@ private void decrementPendingClosures(int count) { } /** - * Close all the sessions. Once this method is invoked {@link #getReadSession()} and {@link - * #getReadWriteSession()} will start throwing {@code IllegalStateException}. The returned future - * blocks till all the sessions created in this pool have been closed. + * Close all the sessions. Once this method is invoked {@link #get()} will start throwing {@code + * IllegalStateException}. The returned future blocks till all the sessions created in this pool + * have been closed. */ ListenableFuture closeAsync(ClosedException closedException) { ListenableFuture retFuture = null; @@ -2380,40 +2025,18 @@ ListenableFuture closeAsync(ClosedException closedException) { } this.closedException = closedException; // Fail all pending waiters. - WaiterFuture waiter = readWaiters.poll(); - while (waiter != null) { - waiter.put(newSpannerException(ErrorCode.INTERNAL, "Client has been closed")); - waiter = readWaiters.poll(); - } - waiter = readWriteWaiters.poll(); + WaiterFuture waiter = waiters.poll(); while (waiter != null) { waiter.put(newSpannerException(ErrorCode.INTERNAL, "Client has been closed")); - waiter = readWriteWaiters.poll(); + waiter = waiters.poll(); } closureFuture = SettableFuture.create(); retFuture = closureFuture; pendingClosure = - totalSessions() - + numSessionsBeingCreated - + 2 /* For pool maintenance thread + prepareExecutor */; + totalSessions() + numSessionsBeingCreated + 1 /* For pool maintenance thread */; poolMaintainer.close(); - readSessions.clear(); - writePreparedSessions.clear(); - prepareExecutor.shutdown(); - executor.submit( - new Runnable() { - @Override - public void run() { - try { - prepareExecutor.awaitTermination(5L, TimeUnit.SECONDS); - } catch (Throwable t) { - } - synchronized (lock) { - decrementPendingClosures(1); - } - } - }); + sessions.clear(); for (PooledSessionFuture session : checkedOutSessions) { if (session.leakedException != null) { if (options.isFailOnSessionLeak()) { @@ -2440,29 +2063,9 @@ public void run() { return retFuture; } - private boolean shouldUnblockReader() { - // This might not be the best strategy since a continuous burst of read requests can starve - // a write request. Maybe maintain a timestamp in the queue and unblock according to that - // or just flip a weighted coin. - synchronized (lock) { - int numWriteWaiters = readWriteWaiters.size() - numSessionsBeingPrepared; - return readWaiters.size() > numWriteWaiters; - } - } - - private boolean shouldPrepareSession() { - synchronized (lock) { - if (stopAutomaticPrepare) { - return false; - } - int preparedSessions = writePreparedSessions.size() + numSessionsBeingPrepared; - return preparedSessions < Math.floor(options.getWriteSessionsFraction() * totalSessions()); - } - } - private int numWaiters() { synchronized (lock) { - return readWaiters.size() + readWriteWaiters.size(); + return waiters.size(); } } @@ -2497,43 +2100,6 @@ public void run() { return res; } - private void prepareSession(final PooledSession sess) { - synchronized (lock) { - numSessionsBeingPrepared++; - } - prepareExecutor.submit( - new Runnable() { - @Override - public void run() { - try { - logger.log(Level.FINE, "Preparing session"); - sess.prepareReadWriteTransaction(); - logger.log(Level.FINE, "Session prepared"); - synchronized (lock) { - numSessionsAsyncPrepared++; - numSessionsBeingPrepared--; - if (!isClosed()) { - if (readWriteWaiters.size() > 0) { - readWriteWaiters.poll().put(sess); - } else if (readWaiters.size() > 0) { - readWaiters.poll().put(sess); - } else { - writePreparedSessions.add(sess); - } - } - } - } catch (Throwable t) { - synchronized (lock) { - numSessionsBeingPrepared--; - if (!isClosed()) { - handlePrepareSessionFailure(newSpannerException(t), sess, true); - } - } - } - } - }); - } - /** * Returns the minimum of the wanted number of sessions that the caller wants to create and the * actual max number that may be created at this moment. @@ -2742,7 +2308,8 @@ public long applyAsLong(SessionPool sessionPool) { new ToLongFunction() { @Override public long applyAsLong(SessionPool sessionPool) { - return sessionPool.numSessionsBeingPrepared; + // TODO: Remove metric. + return 0L; } }); @@ -2766,7 +2333,7 @@ public long applyAsLong(SessionPool sessionPool) { new ToLongFunction() { @Override public long applyAsLong(SessionPool sessionPool) { - return sessionPool.readSessions.size(); + return sessionPool.sessions.size(); } }); @@ -2778,7 +2345,8 @@ public long applyAsLong(SessionPool sessionPool) { new ToLongFunction() { @Override public long applyAsLong(SessionPool sessionPool) { - return sessionPool.writePreparedSessions.size(); + // TODO: Remove metric. + return 0L; } }); } diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPoolOptions.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPoolOptions.java index 57dbd4debd..591a696d36 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPoolOptions.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPoolOptions.java @@ -31,7 +31,7 @@ public class SessionPoolOptions { private final int maxSessions; private final int incStep; private final int maxIdleSessions; - private final float writeSessionsFraction; + @Deprecated private final float writeSessionsFraction; private final ActionOnExhaustion actionOnExhaustion; private final long loopFrequency; private final int keepAliveIntervalMinutes; @@ -74,6 +74,12 @@ public int getMaxIdleSessions() { return maxIdleSessions; } + /** + * @deprecated This value is no longer used. The session pool does not prepare any sessions for + * read/write transactions. Instead, a transaction will be started by including a + * BeginTransaction option with the first statement of a transaction. + */ + @Deprecated public float getWriteSessionsFraction() { return writeSessionsFraction; } @@ -139,7 +145,7 @@ public static class Builder { private int maxSessions = DEFAULT_MAX_SESSIONS; private int incStep = DEFAULT_INC_STEP; private int maxIdleSessions; - private float writeSessionsFraction = 0.2f; + @Deprecated private float writeSessionsFraction = 0.2f; private ActionOnExhaustion actionOnExhaustion = DEFAULT_ACTION; private long initialWaitForSessionTimeoutMillis = 30_000L; private ActionOnSessionNotFound actionOnSessionNotFound = ActionOnSessionNotFound.RETRY; @@ -260,12 +266,10 @@ Builder setFailOnSessionLeak() { } /** - * Fraction of sessions to be kept prepared for write transactions. This is an optimisation to - * avoid the cost of sending a BeginTransaction() rpc. If all such sessions are in use and a - * write request comes, we will make the BeginTransaction() rpc inline. It must be between 0 and - * 1(inclusive). - * - *

Default value is 0.2. + * @deprecated This configuration value is no longer in use. The session pool does not prepare + * any sessions for read/write transactions. Instead, a transaction will automatically be + * started by the first statement that is executed by a transaction by including a + * BeginTransaction option with that statement. */ public Builder setWriteSessionsFraction(float writeSessionsFraction) { this.writeSessionsFraction = writeSessionsFraction; @@ -288,9 +292,6 @@ private void validate() { } Preconditions.checkArgument( keepAliveIntervalMinutes < 60, "Keep alive interval should be less than" + "60 minutes"); - Preconditions.checkArgument( - writeSessionsFraction >= 0 && writeSessionsFraction <= 1, - "Fraction of write sessions must be between 0 and 1 (inclusive)"); } } } diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SpannerImpl.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SpannerImpl.java index 7df8105c8f..2d034eda88 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SpannerImpl.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SpannerImpl.java @@ -233,8 +233,7 @@ public DatabaseClient getDatabaseClient(DatabaseId db) { @VisibleForTesting DatabaseClientImpl createDatabaseClient(String clientId, SessionPool pool) { - return new DatabaseClientImpl( - clientId, pool, getOptions().isInlineBeginForReadWriteTransaction()); + return new DatabaseClientImpl(clientId, pool); } @Override diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SpannerOptions.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SpannerOptions.java index ba702dc093..6bd3d0f90c 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SpannerOptions.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SpannerOptions.java @@ -97,7 +97,6 @@ public class SpannerOptions extends ServiceOptions { private final int prefetchChunks; private final int numChannels; private final ImmutableMap sessionLabels; - private final boolean inlineBeginForReadWriteTransaction; private final SpannerStubSettings spannerStubSettings; private final InstanceAdminStubSettings instanceAdminStubSettings; private final DatabaseAdminStubSettings databaseAdminStubSettings; @@ -547,7 +546,6 @@ private SpannerOptions(Builder builder) { : SessionPoolOptions.newBuilder().build(); prefetchChunks = builder.prefetchChunks; sessionLabels = builder.sessionLabels; - inlineBeginForReadWriteTransaction = builder.inlineBeginForReadWriteTransaction; try { spannerStubSettings = builder.spannerStubSettingsBuilder.build(); instanceAdminStubSettings = builder.instanceAdminStubSettingsBuilder.build(); @@ -626,7 +624,6 @@ public static class Builder private int prefetchChunks = DEFAULT_PREFETCH_CHUNKS; private SessionPoolOptions sessionPoolOptions; private ImmutableMap sessionLabels; - private boolean inlineBeginForReadWriteTransaction; private SpannerStubSettings.Builder spannerStubSettingsBuilder = SpannerStubSettings.newBuilder(); private InstanceAdminStubSettings.Builder instanceAdminStubSettingsBuilder = @@ -676,7 +673,6 @@ private Builder() { this.sessionPoolOptions = options.sessionPoolOptions; this.prefetchChunks = options.prefetchChunks; this.sessionLabels = options.sessionLabels; - this.inlineBeginForReadWriteTransaction = options.inlineBeginForReadWriteTransaction; this.spannerStubSettingsBuilder = options.spannerStubSettings.toBuilder(); this.instanceAdminStubSettingsBuilder = options.instanceAdminStubSettings.toBuilder(); this.databaseAdminStubSettingsBuilder = options.databaseAdminStubSettings.toBuilder(); @@ -769,34 +765,6 @@ public Builder setSessionLabels(Map sessionLabels) { return this; } - /** - * Sets whether {@link DatabaseClient}s should inline the BeginTransaction option with the first - * query or update statement that is executed by a read/write transaction instead of using a - * write-prepared session from the session pool. Enabling this option can improve execution - * times for read/write transactions in the following scenarios: - * - *

- * - *

    - *
  • Applications with a very high rate of read/write transactions where the session pool is - * not able to prepare new read/write transactions at the same rate as the application is - * requesting read/write transactions. - *
  • Applications with a very low rate of read/write transactions where sessions with a - * prepared read/write transaction are kept in the session pool for a long time without - * being used. - *
- * - * If you enable this option, you should also re-evaluate the value for {@link - * SessionPoolOptions.Builder#setWriteSessionsFraction(float)}. When this option is enabled, - * write-prepared sessions are only used for calls to {@link DatabaseClient#write(Iterable)}. If - * your application does not use this method, you should set the write fraction for the session - * pool to zero. - */ - public Builder setInlineBeginForReadWriteTransaction(boolean inlineBegin) { - this.inlineBeginForReadWriteTransaction = inlineBegin; - return this; - } - /** * {@link SpannerOptions.Builder} does not support global retry settings, as it creates three * different gRPC clients: {@link Spanner}, {@link DatabaseAdminClient} and {@link @@ -1093,10 +1061,6 @@ public Map getSessionLabels() { return sessionLabels; } - public boolean isInlineBeginForReadWriteTransaction() { - return inlineBeginForReadWriteTransaction; - } - public SpannerStubSettings getSpannerStubSettings() { return spannerStubSettings; } diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/TransactionManagerImpl.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/TransactionManagerImpl.java index 3aa7a008ab..b18e2f25d9 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/TransactionManagerImpl.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/TransactionManagerImpl.java @@ -30,15 +30,13 @@ final class TransactionManagerImpl implements TransactionManager, SessionTransac private final SessionImpl session; private Span span; - private final boolean inlineBegin; private TransactionRunnerImpl.TransactionContextImpl txn; private TransactionState txnState; - TransactionManagerImpl(SessionImpl session, Span span, boolean inlineBegin) { + TransactionManagerImpl(SessionImpl session, Span span) { this.session = session; this.span = span; - this.inlineBegin = inlineBegin; } Span getSpan() { @@ -56,9 +54,6 @@ public TransactionContext begin() { try (Scope s = tracer.withSpan(span)) { txn = session.newTransaction(); session.setActive(this); - if (!inlineBegin) { - txn.ensureTxn(); - } txnState = TransactionState.STARTED; return txn; } @@ -105,7 +100,7 @@ public TransactionContext resetForRetry() { "resetForRetry can only be called if the previous attempt" + " aborted"); } try (Scope s = tracer.withSpan(span)) { - boolean useInlinedBegin = inlineBegin && txn.transactionId != null; + boolean useInlinedBegin = txn.transactionId != null; txn = session.newTransaction(); if (!useInlinedBegin) { txn.ensureTxn(); diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/TransactionRunnerImpl.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/TransactionRunnerImpl.java index f885492a03..bcfabe5329 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/TransactionRunnerImpl.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/TransactionRunnerImpl.java @@ -718,7 +718,6 @@ public ListenableAsyncResultSet executeQueryAsync( private boolean blockNestedTxn = true; private final SessionImpl session; private Span span; - private final boolean inlineBegin; private TransactionContextImpl txn; private volatile boolean isValid = true; @@ -728,10 +727,8 @@ public TransactionRunner allowNestedTransaction() { return this; } - TransactionRunnerImpl( - SessionImpl session, SpannerRpc rpc, int defaultPrefetchChunks, boolean inlineBegin) { + TransactionRunnerImpl(SessionImpl session, SpannerRpc rpc, int defaultPrefetchChunks) { this.session = session; - this.inlineBegin = inlineBegin; this.txn = session.newTransaction(); } @@ -765,7 +762,7 @@ private T runInternal(final TransactionCallable txCallable) { new Callable() { @Override public T call() { - boolean useInlinedBegin = inlineBegin; + boolean useInlinedBegin = true; if (attempt.get() > 0) { if (useInlinedBegin) { // Do not inline the BeginTransaction during a retry if the initial attempt did not diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/AsyncRunnerTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/AsyncRunnerTest.java index 3869dbdfcf..2af185ae14 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/AsyncRunnerTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/AsyncRunnerTest.java @@ -140,7 +140,7 @@ public void asyncRunnerUpdateAborted() throws Exception { @Override public ApiFuture doWorkAsync(TransactionContext txn) { if (attempt.incrementAndGet() == 1) { - mockSpanner.abortTransaction(txn); + mockSpanner.abortNextStatement(); } else { // Set the result of the update statement back to 1 row. mockSpanner.putStatementResult( @@ -199,7 +199,7 @@ public void asyncRunnerUpdateAbortedWithoutGettingResult() throws Exception { @Override public ApiFuture doWorkAsync(TransactionContext txn) { if (attempt.incrementAndGet() == 1) { - mockSpanner.abortTransaction(txn); + mockSpanner.abortNextStatement(); } // This update statement will be aborted, but the error will not propagated to the // transaction runner and cause the transaction to retry. Instead, the commit call @@ -217,9 +217,9 @@ public ApiFuture doWorkAsync(TransactionContext txn) { assertThat(mockSpanner.getRequestTypes()) .containsExactly( BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, ExecuteSqlRequest.class, - CommitRequest.class, + // The retry will use an explicit BeginTransaction RPC because the first statement of + // the transaction did not return a transaction id during the initial attempt. BeginTransactionRequest.class, ExecuteSqlRequest.class, CommitRequest.class); @@ -272,10 +272,7 @@ public ApiFuture doWorkAsync(TransactionContext txn) { res.get(); assertThat(mockSpanner.getRequestTypes()) .containsExactly( - BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, - ExecuteSqlRequest.class, - CommitRequest.class); + BatchCreateSessionsRequest.class, ExecuteSqlRequest.class, CommitRequest.class); } @Test @@ -418,9 +415,14 @@ public void asyncRunnerBatchUpdateAbortedWithoutGettingResult() throws Exception @Override public ApiFuture doWorkAsync(TransactionContext txn) { if (attempt.incrementAndGet() == 1) { - mockSpanner.abortTransaction(txn); + mockSpanner.abortNextTransaction(); } - // This update statement will be aborted, but the error will not propagated to the + // This statement will succeed and return a transaction id. The transaction will be + // marked as aborted on the mock server. + txn.executeUpdate(UPDATE_STATEMENT); + + // This batch update statement will be aborted, but the error will not propagated to + // the // transaction runner and cause the transaction to retry. Instead, the commit call // will do that. txn.batchUpdateAsync(ImmutableList.of(UPDATE_STATEMENT, UPDATE_STATEMENT)); @@ -436,10 +438,10 @@ public ApiFuture doWorkAsync(TransactionContext txn) { assertThat(mockSpanner.getRequestTypes()) .containsExactly( BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, + ExecuteSqlRequest.class, ExecuteBatchDmlRequest.class, CommitRequest.class, - BeginTransactionRequest.class, + ExecuteSqlRequest.class, ExecuteBatchDmlRequest.class, CommitRequest.class); } @@ -491,10 +493,7 @@ public ApiFuture doWorkAsync(TransactionContext txn) { res.get(); assertThat(mockSpanner.getRequestTypes()) .containsExactly( - BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, - ExecuteBatchDmlRequest.class, - CommitRequest.class); + BatchCreateSessionsRequest.class, ExecuteBatchDmlRequest.class, CommitRequest.class); } @Test diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/AsyncTransactionManagerTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/AsyncTransactionManagerTest.java index c7b95f33f6..09523bc7b2 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/AsyncTransactionManagerTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/AsyncTransactionManagerTest.java @@ -301,7 +301,7 @@ public ApiFuture apply(TransactionContext txn, Long input) public void asyncTransactionManagerFireAndForgetInvalidUpdate() throws Exception { final SettableApiFuture updateCount = SettableApiFuture.create(); - try (AsyncTransactionManager mgr = client().transactionManagerAsync()) { + try (AsyncTransactionManager mgr = clientWithEmptySessionPool().transactionManagerAsync()) { TransactionContextFuture txn = mgr.beginAsync(); while (true) { try { @@ -312,6 +312,8 @@ public void asyncTransactionManagerFireAndForgetInvalidUpdate() throws Exception public ApiFuture apply(TransactionContext txn, Void input) throws Exception { // This fire-and-forget update statement should not fail the transaction. + // The exception will however cause the transaction to be retried, as the + // statement will not return a transaction id. txn.executeUpdateAsync(INVALID_UPDATE_STATEMENT); ApiFutures.addCallback( txn.executeUpdateAsync(UPDATE_STATEMENT), @@ -332,14 +334,26 @@ public void onSuccess(Long result) { }, executor) .commitAsync(); - assertThat(updateCount.get()).isEqualTo(UPDATE_COUNT); assertThat(ts.get()).isNotNull(); + assertThat(updateCount.get()).isEqualTo(UPDATE_COUNT); break; } catch (AbortedException e) { txn = mgr.resetForRetryAsync(); } } } + assertThat(mockSpanner.getRequestTypes()) + .containsExactly( + BatchCreateSessionsRequest.class, + // The first update that fails. This will cause a transaction retry. + ExecuteSqlRequest.class, + // The retry will use an explicit BeginTransaction call. + BeginTransactionRequest.class, + // The first update will again fail, but now there is a transaction id, so the + // transaction can continue. + ExecuteSqlRequest.class, + ExecuteSqlRequest.class, + CommitRequest.class); } @Test @@ -439,7 +453,7 @@ public ApiFuture apply(TransactionContext txn, Void input) throws Exception { if (attempt.incrementAndGet() == 1) { // Abort the first attempt. - mockSpanner.abortTransaction(txn); + mockSpanner.abortNextStatement(); } else { // Set the result of the update statement back to 1 row. mockSpanner.putStatementResult( @@ -479,7 +493,7 @@ public void asyncTransactionManagerUpdateAbortedWithoutGettingResult() throws Ex public ApiFuture apply(TransactionContext txn, Void input) throws Exception { if (attempt.incrementAndGet() == 1) { - mockSpanner.abortTransaction(txn); + mockSpanner.abortNextStatement(); } // This update statement will be aborted, but the error will not // propagated to the transaction runner and cause the transaction to @@ -501,8 +515,8 @@ public ApiFuture apply(TransactionContext txn, Void input) assertThat(mockSpanner.getRequestTypes()) .containsAtLeast( BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, ExecuteSqlRequest.class, + // The retry will use a BeginTransaction RPC. BeginTransactionRequest.class, ExecuteSqlRequest.class, CommitRequest.class); @@ -566,10 +580,7 @@ public ApiFuture apply(TransactionContext txn, Void input) .get(); assertThat(mockSpanner.getRequestTypes()) .containsExactly( - BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, - ExecuteSqlRequest.class, - CommitRequest.class); + BatchCreateSessionsRequest.class, ExecuteSqlRequest.class, CommitRequest.class); break; } catch (AbortedException e) { txn = mgr.resetForRetryAsync(); @@ -685,7 +696,6 @@ public ApiFuture apply(TransactionContext txn, Void input) assertThat(mockSpanner.getRequestTypes()) .containsExactly( BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, ExecuteBatchDmlRequest.class, ExecuteBatchDmlRequest.class, CommitRequest.class); @@ -727,7 +737,6 @@ public ApiFuture apply(TransactionContext txn, Void input) assertThat(mockSpanner.getRequestTypes()) .containsExactly( BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, ExecuteBatchDmlRequest.class, BeginTransactionRequest.class, ExecuteBatchDmlRequest.class, @@ -747,7 +756,7 @@ public void asyncTransactionManagerBatchUpdateAbortedBeforeFirstStatement() thro public ApiFuture apply(TransactionContext txn, Void input) throws Exception { if (attempt.incrementAndGet() == 1) { - mockSpanner.abortTransaction(txn); + mockSpanner.abortNextStatement(); } return txn.batchUpdateAsync( ImmutableList.of(UPDATE_STATEMENT, UPDATE_STATEMENT)); @@ -768,7 +777,6 @@ public ApiFuture apply(TransactionContext txn, Void input) assertThat(mockSpanner.getRequestTypes()) .containsExactly( BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, ExecuteBatchDmlRequest.class, BeginTransactionRequest.class, ExecuteBatchDmlRequest.class, @@ -830,7 +838,6 @@ public ApiFuture apply(TransactionContext txn, long[] input) assertThat(mockSpanner.getRequestTypes()) .containsExactly( BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, ExecuteBatchDmlRequest.class, CommitRequest.class, BeginTransactionRequest.class, @@ -851,7 +858,7 @@ public void asyncTransactionManagerBatchUpdateAbortedWithoutGettingResult() thro public ApiFuture apply(TransactionContext txn, Void input) throws Exception { if (attempt.incrementAndGet() == 1) { - mockSpanner.abortTransaction(txn); + mockSpanner.abortNextStatement(); } // This update statement will be aborted, but the error will not propagated to // the transaction manager and cause the transaction to retry. Instead, the @@ -875,12 +882,11 @@ public ApiFuture apply(TransactionContext txn, Void input) assertThat(attempt.get()).isEqualTo(2); Iterable> requests = mockSpanner.getRequestTypes(); int size = Iterables.size(requests); - assertThat(size).isIn(Range.closed(6, 7)); - if (size == 6) { + assertThat(size).isIn(Range.closed(5, 6)); + if (size == 5) { assertThat(requests) .containsExactly( BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, ExecuteBatchDmlRequest.class, BeginTransactionRequest.class, ExecuteBatchDmlRequest.class, @@ -889,7 +895,6 @@ public ApiFuture apply(TransactionContext txn, Void input) assertThat(requests) .containsExactly( BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, ExecuteBatchDmlRequest.class, CommitRequest.class, BeginTransactionRequest.class, @@ -929,10 +934,7 @@ public void asyncTransactionManagerWithBatchUpdateCommitFails() throws Exception } assertThat(mockSpanner.getRequestTypes()) .containsExactly( - BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, - ExecuteBatchDmlRequest.class, - CommitRequest.class); + BatchCreateSessionsRequest.class, ExecuteBatchDmlRequest.class, CommitRequest.class); } @Test @@ -961,10 +963,7 @@ public ApiFuture apply(TransactionContext txn, Void input) } assertThat(mockSpanner.getRequestTypes()) .containsExactly( - BatchCreateSessionsRequest.class, - BeginTransactionRequest.class, - ExecuteBatchDmlRequest.class, - CommitRequest.class); + BatchCreateSessionsRequest.class, ExecuteBatchDmlRequest.class, CommitRequest.class); } @Test diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/BatchCreateSessionsTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/BatchCreateSessionsTest.java index abac3bd134..7dac8c8bfe 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/BatchCreateSessionsTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/BatchCreateSessionsTest.java @@ -19,13 +19,11 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.fail; import com.google.api.gax.grpc.testing.LocalChannelProvider; import com.google.cloud.NoCredentials; import com.google.cloud.spanner.MockSpannerServiceImpl.SimulatedExecutionTime; import com.google.cloud.spanner.MockSpannerServiceImpl.StatementResult; -import com.google.cloud.spanner.TransactionRunner.TransactionCallable; import com.google.common.base.Stopwatch; import com.google.protobuf.ListValue; import com.google.spanner.v1.ResultSetMetadata; @@ -235,72 +233,4 @@ public void testSpannerReturnsResourceExhausted() throws InterruptedException { // Verify that all sessions have been deleted. assertThat(client.pool.totalSessions(), is(equalTo(0))); } - - @Test - public void testPrepareSessionFailPropagatesToUser() { - // Do not create any sessions by default. - // This also means that when a read/write session is requested, the session pool - // will start preparing a read session at that time. Any errors that might occur - // during the BeginTransaction call will be propagated to the user. - int minSessions = 0; - int maxSessions = 1000; - DatabaseClientImpl client = null; - mockSpanner.setBeginTransactionExecutionTime( - SimulatedExecutionTime.ofStickyException( - Status.ABORTED.withDescription("BeginTransaction failed").asRuntimeException())); - try (Spanner spanner = createSpanner(minSessions, maxSessions)) { - client = - (DatabaseClientImpl) - spanner.getDatabaseClient(DatabaseId.of("[PROJECT]", "[INSTANCE]", "[DATABASE]")); - TransactionRunner runner = client.readWriteTransaction(); - runner.run( - new TransactionCallable() { - @Override - public Void run(TransactionContext transaction) { - return null; - } - }); - fail("missing expected exception"); - } catch (SpannerException e) { - assertThat(e.getErrorCode(), is(equalTo(ErrorCode.ABORTED))); - assertThat(e.getMessage().endsWith("BeginTransaction failed"), is(true)); - } - } - - @Test - public void testPrepareSessionFailDoesNotPropagateToUser() throws InterruptedException { - // Create 5 sessions and 20% write prepared sessions. - // That should prepare exactly 1 session for r/w. - int minSessions = 5; - int maxSessions = 1000; - DatabaseClientImpl client = null; - // The first prepare should fail. - // The prepare will then be retried and should succeed. - mockSpanner.setBeginTransactionExecutionTime( - SimulatedExecutionTime.ofException( - Status.ABORTED.withDescription("BeginTransaction failed").asRuntimeException())); - try (Spanner spanner = createSpanner(minSessions, maxSessions)) { - client = - (DatabaseClientImpl) - spanner.getDatabaseClient(DatabaseId.of("[PROJECT]", "[INSTANCE]", "[DATABASE]")); - // Wait until the session pool has initialized and a session has been prepared. - Stopwatch watch = Stopwatch.createStarted(); - while ((client.pool.totalSessions() < minSessions - || client.pool.getNumberOfAvailableWritePreparedSessions() != 1) - && watch.elapsed(TimeUnit.SECONDS) < 10) { - Thread.sleep(10L); - } - - // There should be 1 prepared session and a r/w transaction should succeed. - assertThat(client.pool.getNumberOfAvailableWritePreparedSessions(), is(equalTo(1))); - TransactionRunner runner = client.readWriteTransaction(); - runner.run( - new TransactionCallable() { - @Override - public Void run(TransactionContext transaction) { - return null; - } - }); - } - } } diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/DatabaseClientImplTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/DatabaseClientImplTest.java index 8775fb1b18..2747dc314f 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/DatabaseClientImplTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/DatabaseClientImplTest.java @@ -809,65 +809,6 @@ public void testPartitionedDmlRetriesOnUnavailable() { } } - @Test - public void testDatabaseOrInstanceDoesNotExistOnPrepareSession() throws Exception { - StatusRuntimeException[] exceptions = - new StatusRuntimeException[] { - SpannerExceptionFactoryTest.newStatusResourceNotFoundException( - "Database", SpannerExceptionFactory.DATABASE_RESOURCE_TYPE, DATABASE_NAME), - SpannerExceptionFactoryTest.newStatusResourceNotFoundException( - "Instance", SpannerExceptionFactory.INSTANCE_RESOURCE_TYPE, INSTANCE_NAME) - }; - for (StatusRuntimeException exception : exceptions) { - try (Spanner spanner = - SpannerOptions.newBuilder() - .setProjectId(TEST_PROJECT) - .setChannelProvider(channelProvider) - .setCredentials(NoCredentials.getInstance()) - .build() - .getService()) { - mockSpanner.setBeginTransactionExecutionTime( - SimulatedExecutionTime.ofStickyException(exception)); - DatabaseClientImpl dbClient = - (DatabaseClientImpl) - spanner.getDatabaseClient( - DatabaseId.of(TEST_PROJECT, TEST_INSTANCE, TEST_DATABASE)); - // Wait until all sessions have been created. - Stopwatch watch = Stopwatch.createStarted(); - while (watch.elapsed(TimeUnit.SECONDS) < 5 - && dbClient.pool.getNumberOfSessionsBeingCreated() > 0) { - Thread.sleep(1L); - } - // Ensure that no sessions could be prepared and that the session pool gives up trying to - // prepare sessions. - watch = watch.reset().start(); - while (watch.elapsed(TimeUnit.SECONDS) < 5 - && dbClient.pool.getNumberOfSessionsBeingPrepared() > 0) { - Thread.sleep(1L); - } - assertThat(dbClient.pool.getNumberOfSessionsBeingPrepared()).isEqualTo(0); - assertThat(dbClient.pool.getNumberOfAvailableWritePreparedSessions()).isEqualTo(0); - int currentNumRequest = mockSpanner.getRequests().size(); - try { - dbClient - .readWriteTransaction() - .run( - new TransactionCallable() { - @Override - public Void run(TransactionContext transaction) { - return null; - } - }); - fail("missing expected exception"); - } catch (DatabaseNotFoundException | InstanceNotFoundException e) { - } - assertThat(mockSpanner.getRequests()).hasSize(currentNumRequest); - mockSpanner.reset(); - mockSpanner.removeAllExecutionTimes(); - } - } - } - @Test public void testDatabaseOrInstanceDoesNotExistOnInitialization() throws Exception { StatusRuntimeException[] exceptions = @@ -1001,89 +942,6 @@ public void testDatabaseOrInstanceDoesNotExistOnReplenish() throws Exception { } } - @Test - public void testPermissionDeniedOnPrepareSession() throws Exception { - testExceptionOnPrepareSession( - Status.PERMISSION_DENIED - .withDescription( - "Caller is missing IAM permission spanner.databases.beginOrRollbackReadWriteTransaction on resource") - .asRuntimeException()); - } - - @Test - public void testFailedPreconditionOnPrepareSession() throws Exception { - testExceptionOnPrepareSession( - Status.FAILED_PRECONDITION - .withDescription("FAILED_PRECONDITION: Database is in read-only mode") - .asRuntimeException()); - } - - private void testExceptionOnPrepareSession(StatusRuntimeException exception) - throws InterruptedException { - mockSpanner.setBeginTransactionExecutionTime( - SimulatedExecutionTime.ofStickyException(exception)); - DatabaseClientImpl dbClient = - (DatabaseClientImpl) - spanner.getDatabaseClient(DatabaseId.of(TEST_PROJECT, TEST_INSTANCE, TEST_DATABASE)); - // Wait until all sessions have been created. - Stopwatch watch = Stopwatch.createStarted(); - while (watch.elapsed(TimeUnit.SECONDS) < 5 - && dbClient.pool.getNumberOfSessionsBeingCreated() > 0) { - Thread.sleep(1L); - } - // Ensure that no sessions could be prepared and that the session pool gives up trying to - // prepare sessions. - watch = watch.reset().start(); - while (watch.elapsed(TimeUnit.SECONDS) < 5 - && dbClient.pool.getNumberOfSessionsBeingPrepared() > 0) { - Thread.sleep(1L); - } - assertThat(dbClient.pool.getNumberOfSessionsBeingPrepared()).isEqualTo(0); - assertThat(dbClient.pool.getNumberOfAvailableWritePreparedSessions()).isEqualTo(0); - try { - dbClient - .readWriteTransaction() - .run( - new TransactionCallable() { - @Override - public Void run(TransactionContext transaction) { - return null; - } - }); - fail(String.format("missing expected %s exception", exception.getStatus().getCode().name())); - } catch (SpannerException e) { - assertThat(e.getErrorCode()).isEqualTo(ErrorCode.fromGrpcStatus(exception.getStatus())); - } - // Remove the semi-permanent error condition. Getting a read/write transaction should now - // succeed, and the automatic preparing of sessions should be restarted. - mockSpanner.setBeginTransactionExecutionTime(SimulatedExecutionTime.none()); - dbClient - .readWriteTransaction() - .run( - new TransactionCallable() { - @Override - public Void run(TransactionContext transaction) { - return null; - } - }); - for (int i = 0; i < spanner.getOptions().getSessionPoolOptions().getMinSessions(); i++) { - dbClient.pool.getReadSession().close(); - } - int expectedPreparedSessions = - (int) - Math.ceil( - dbClient.pool.getNumberOfSessionsInPool() - * spanner.getOptions().getSessionPoolOptions().getWriteSessionsFraction()); - watch = watch.reset().start(); - while (watch.elapsed(TimeUnit.SECONDS) < 5 - && dbClient.pool.getNumberOfAvailableWritePreparedSessions() < expectedPreparedSessions) { - Thread.sleep(1L); - } - assertThat(dbClient.pool.getNumberOfSessionsBeingPrepared()).isEqualTo(0); - assertThat(dbClient.pool.getNumberOfAvailableWritePreparedSessions()) - .isEqualTo(expectedPreparedSessions); - } - /** * Test showing that when a database is deleted while it is in use by a database client and then * re-created with the same name, will continue to return {@link DatabaseNotFoundException}s until @@ -1113,8 +971,7 @@ public void testDatabaseOrInstanceIsDeletedAndThenRecreated() throws Exception { // Wait until all sessions have been created and prepared. Stopwatch watch = Stopwatch.createStarted(); while (watch.elapsed(TimeUnit.SECONDS) < 5 - && (dbClient.pool.getNumberOfSessionsBeingCreated() > 0 - || dbClient.pool.getNumberOfSessionsBeingPrepared() > 0)) { + && (dbClient.pool.getNumberOfSessionsBeingCreated() > 0)) { Thread.sleep(1L); } // Simulate that the database or instance has been deleted. diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/ITSessionPoolIntegrationTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/ITSessionPoolIntegrationTest.java index 66256489e8..73f2111019 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/ITSessionPoolIntegrationTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/ITSessionPoolIntegrationTest.java @@ -98,12 +98,12 @@ public ScheduledExecutorService get() { @Test public void sessionCreation() { - try (PooledSessionFuture session = pool.getReadSession()) { + try (PooledSessionFuture session = pool.get()) { assertThat(session.get()).isNotNull(); } - try (PooledSessionFuture session = pool.getReadSession(); - PooledSessionFuture session2 = pool.getReadSession()) { + try (PooledSessionFuture session = pool.get(); + PooledSessionFuture session2 = pool.get()) { assertThat(session.get()).isNotNull(); assertThat(session2.get()).isNotNull(); } @@ -111,14 +111,14 @@ public void sessionCreation() { @Test public void poolExhaustion() throws Exception { - Session session1 = pool.getReadSession().get(); - Session session2 = pool.getReadSession().get(); + Session session1 = pool.get().get(); + Session session2 = pool.get().get(); final CountDownLatch latch = new CountDownLatch(1); new Thread( new Runnable() { @Override public void run() { - try (Session session3 = pool.getReadSession().get()) { + try (Session session3 = pool.get().get()) { latch.countDown(); } } @@ -132,8 +132,8 @@ public void run() { @Test public void multipleWaiters() throws Exception { - Session session1 = pool.getReadSession().get(); - Session session2 = pool.getReadSession().get(); + Session session1 = pool.get().get(); + Session session2 = pool.get().get(); int numSessions = 5; final CountDownLatch latch = new CountDownLatch(numSessions); for (int i = 0; i < numSessions; i++) { @@ -141,7 +141,7 @@ public void multipleWaiters() throws Exception { new Runnable() { @Override public void run() { - try (Session session = pool.getReadSession().get()) { + try (Session session = pool.get().get()) { latch.countDown(); } } @@ -161,13 +161,13 @@ public void closeQuicklyDoesNotBlockIndefinitely() throws Exception { @Test public void closeAfterInitialCreateDoesNotBlockIndefinitely() throws Exception { - pool.getReadSession().close(); + pool.get().close(); pool.closeAsync(new SpannerImpl.ClosedException()).get(); } @Test public void closeWhenSessionsActiveFinishes() throws Exception { - pool.getReadSession().get(); + pool.get().get(); // This will log a warning that a session has been leaked, as the session that we retrieved in // the previous statement was never returned to the pool. pool.closeAsync(new SpannerImpl.ClosedException()).get(); diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/InlineBeginBenchmark.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/InlineBeginBenchmark.java index 3e08f0f633..ecd8f4410d 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/InlineBeginBenchmark.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/InlineBeginBenchmark.java @@ -77,10 +77,10 @@ public static class BenchmarkState { private Spanner spanner; private DatabaseClientImpl client; - @Param({"false", "true"}) + @Param({"true"}) boolean inlineBegin; - @Param({"0.0", "0.2"}) + @Param({"0.2"}) float writeFraction; @Setup(Level.Invocation) @@ -122,7 +122,6 @@ SpannerOptions createBenchmarkServerOptions(TransportChannelProvider channelProv .setCredentials(NoCredentials.getInstance()) .setSessionPoolOption( SessionPoolOptions.newBuilder().setWriteSessionsFraction(writeFraction).build()) - .setInlineBeginForReadWriteTransaction(inlineBegin) .build(); } @@ -130,7 +129,6 @@ SpannerOptions createRealServerOptions() throws IOException { return SpannerOptions.newBuilder() .setSessionPoolOption( SessionPoolOptions.newBuilder().setWriteSessionsFraction(writeFraction).build()) - .setInlineBeginForReadWriteTransaction(inlineBegin) .build(); } diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/InlineBeginTransactionTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/InlineBeginTransactionTest.java index f40fed8615..74b047ecf9 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/InlineBeginTransactionTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/InlineBeginTransactionTest.java @@ -174,9 +174,6 @@ public void setUp() throws IOException { .setProjectId("[PROJECT]") .setChannelProvider(channelProvider) .setCredentials(NoCredentials.getInstance()) - .setInlineBeginForReadWriteTransaction(true) - .setSessionPoolOption( - SessionPoolOptions.newBuilder().setWriteSessionsFraction(0.0f).build()) .build() .getService(); } @@ -1061,7 +1058,8 @@ public ApiFuture apply(TransactionContext txn, Long input) } } } - assertThat(countRequests(BeginTransactionRequest.class)).isEqualTo(0); + // The retry will use a BeginTransaction RPC. + assertThat(countRequests(BeginTransactionRequest.class)).isEqualTo(1); assertThat(countTransactionsStarted()).isEqualTo(2); } diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/IntegrationTestWithClosedSessionsEnv.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/IntegrationTestWithClosedSessionsEnv.java index 4c50667a39..8070f542db 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/IntegrationTestWithClosedSessionsEnv.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/IntegrationTestWithClosedSessionsEnv.java @@ -47,8 +47,7 @@ private static class SpannerWithClosedSessionsImpl extends SpannerImpl { @Override DatabaseClientImpl createDatabaseClient(String clientId, SessionPool pool) { - return new DatabaseClientWithClosedSessionImpl( - clientId, pool, getOptions().isInlineBeginForReadWriteTransaction()); + return new DatabaseClientWithClosedSessionImpl(clientId, pool); } } @@ -60,9 +59,8 @@ public static class DatabaseClientWithClosedSessionImpl extends DatabaseClientIm private boolean invalidateNextSession = false; private boolean allowReplacing = true; - DatabaseClientWithClosedSessionImpl( - String clientId, SessionPool pool, boolean inlineBeginReadWriteTransactions) { - super(clientId, pool, inlineBeginReadWriteTransactions); + DatabaseClientWithClosedSessionImpl(String clientId, SessionPool pool) { + super(clientId, pool); } /** Invalidate the next session that is checked out from the pool. */ @@ -89,20 +87,6 @@ PooledSessionFuture getReadSession() { return session; } - @Override - PooledSessionFuture getReadWriteSession() { - PooledSessionFuture session = super.getReadWriteSession(); - if (invalidateNextSession) { - session.get().delegate.close(); - session.get().setAllowReplacing(false); - awaitDeleted(session.get().delegate); - session.get().setAllowReplacing(allowReplacing); - invalidateNextSession = false; - } - session.get().setAllowReplacing(allowReplacing); - return session; - } - /** * Deleting a session server side takes some time. This method checks and waits until the * session really has been deleted. diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/ReadWriteTransactionWithInlineBeginTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/ReadWriteTransactionWithInlineBeginTest.java index c6ff7cdbf2..4690a30aa7 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/ReadWriteTransactionWithInlineBeginTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/ReadWriteTransactionWithInlineBeginTest.java @@ -125,13 +125,9 @@ public static void stopServer() throws InterruptedException { public void setUp() throws IOException { mockSpanner.reset(); mockSpanner.removeAllExecutionTimes(); - // Create a Spanner instance with no read/write prepared sessions in the pool. spanner = SpannerOptions.newBuilder() .setProjectId("[PROJECT]") - .setInlineBeginForReadWriteTransaction(true) - .setSessionPoolOption( - SessionPoolOptions.newBuilder().setWriteSessionsFraction(0.0f).build()) .setChannelProvider(channelProvider) .setCredentials(NoCredentials.getInstance()) .build() diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/RetryOnInvalidatedSessionTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/RetryOnInvalidatedSessionTest.java index fcf1c6e35b..5e732c1eab 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/RetryOnInvalidatedSessionTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/RetryOnInvalidatedSessionTest.java @@ -217,17 +217,6 @@ public void tearDown() { spanner.close(); } - private static void initReadWriteSessionPool() throws InterruptedException { - // Wait for at least one read/write session to be ready. - Stopwatch watch = Stopwatch.createStarted(); - while (((DatabaseClientImpl) client).pool.getNumberOfAvailableWritePreparedSessions() == 0) { - if (watch.elapsed(TimeUnit.SECONDS) > 5L) { - fail("No read/write sessions prepared"); - } - Thread.sleep(5L); - } - } - private static void invalidateSessionPool() throws InterruptedException { invalidateSessionPool(client, spanner.getOptions().getSessionPoolOptions().getMinSessions()); } @@ -576,16 +565,10 @@ public void readOnlyTransactionReadRowUsingIndexNonRecoverable() throws Interrup } } - /** - * Test with one read-only session in the pool that is invalidated. The session pool will try to - * prepare this session for read/write, which will fail with a {@link SessionNotFoundException}. - * That again will trigger the creation of a new session. This will always succeed. - */ @Test public void readWriteTransactionReadOnlySessionInPool() throws InterruptedException { // Create a session pool with only read sessions. - SessionPoolOptions.Builder builder = - SessionPoolOptions.newBuilder().setWriteSessionsFraction(0.0f); + SessionPoolOptions.Builder builder = SessionPoolOptions.newBuilder(); if (failOnInvalidatedSession) { builder.setFailIfSessionNotFound(); } @@ -600,27 +583,31 @@ public void readWriteTransactionReadOnlySessionInPool() throws InterruptedExcept DatabaseClient client = spanner.getDatabaseClient(DatabaseId.of("[PROJECT]", "[INSTANCE]", "[DATABASE]")); invalidateSessionPool(client, spanner.getOptions().getSessionPoolOptions().getMinSessions()); - TransactionRunner runner = client.readWriteTransaction(); - int count = - runner.run( - new TransactionCallable() { - @Override - public Integer run(TransactionContext transaction) { - int count = 0; - try (ResultSet rs = transaction.executeQuery(SELECT1AND2)) { - while (rs.next()) { - count++; + try { + TransactionRunner runner = client.readWriteTransaction(); + int count = + runner.run( + new TransactionCallable() { + @Override + public Integer run(TransactionContext transaction) { + int count = 0; + try (ResultSet rs = transaction.executeQuery(SELECT1AND2)) { + while (rs.next()) { + count++; + } } + return count; } - return count; - } - }); - assertThat(count).isEqualTo(2); + }); + assertThat(count).isEqualTo(2); + assertThat(failOnInvalidatedSession).isFalse(); + } catch (SessionNotFoundException e) { + assertThat(failOnInvalidatedSession).isTrue(); + } } @Test public void readWriteTransactionSelect() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try { TransactionRunner runner = client.readWriteTransaction(); @@ -647,7 +634,6 @@ public Integer run(TransactionContext transaction) { @Test public void readWriteTransactionRead() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try { TransactionRunner runner = client.readWriteTransaction(); @@ -674,7 +660,6 @@ public Integer run(TransactionContext transaction) { @Test public void readWriteTransactionReadUsingIndex() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try { TransactionRunner runner = client.readWriteTransaction(); @@ -703,7 +688,6 @@ public Integer run(TransactionContext transaction) { @Test public void readWriteTransactionReadRow() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try { TransactionRunner runner = client.readWriteTransaction(); @@ -724,7 +708,6 @@ public Struct run(TransactionContext transaction) { @Test public void readWriteTransactionReadRowUsingIndex() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try { TransactionRunner runner = client.readWriteTransaction(); @@ -746,7 +729,6 @@ public Struct run(TransactionContext transaction) { @Test public void readWriteTransactionUpdate() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try { TransactionRunner runner = client.readWriteTransaction(); @@ -767,7 +749,6 @@ public Long run(TransactionContext transaction) { @Test public void readWriteTransactionBatchUpdate() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try { TransactionRunner runner = client.readWriteTransaction(); @@ -789,7 +770,6 @@ public long[] run(TransactionContext transaction) { @Test public void readWriteTransactionBuffer() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try { TransactionRunner runner = client.readWriteTransaction(); @@ -1022,14 +1002,16 @@ public void transactionManagerReadOnlySessionInPool() throws InterruptedExceptio transaction = manager.resetForRetry(); } } + assertThat(count).isEqualTo(2); + assertThat(failOnInvalidatedSession).isFalse(); + } catch (SessionNotFoundException e) { + assertThat(failOnInvalidatedSession).isTrue(); } - assertThat(count).isEqualTo(2); } @SuppressWarnings("resource") @Test public void transactionManagerSelect() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try (TransactionManager manager = client.transactionManager()) { int count = 0; @@ -1058,7 +1040,6 @@ public void transactionManagerSelect() throws InterruptedException { @SuppressWarnings("resource") @Test public void transactionManagerRead() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try (TransactionManager manager = client.transactionManager()) { int count = 0; @@ -1087,7 +1068,6 @@ public void transactionManagerRead() throws InterruptedException { @SuppressWarnings("resource") @Test public void transactionManagerReadUsingIndex() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try (TransactionManager manager = client.transactionManager()) { int count = 0; @@ -1117,7 +1097,6 @@ public void transactionManagerReadUsingIndex() throws InterruptedException { @SuppressWarnings("resource") @Test public void transactionManagerReadRow() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try (TransactionManager manager = client.transactionManager()) { Struct row; @@ -1142,7 +1121,6 @@ public void transactionManagerReadRow() throws InterruptedException { @SuppressWarnings("resource") @Test public void transactionManagerReadRowUsingIndex() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try (TransactionManager manager = client.transactionManager()) { Struct row; @@ -1167,7 +1145,6 @@ public void transactionManagerReadRowUsingIndex() throws InterruptedException { @SuppressWarnings("resource") @Test public void transactionManagerUpdate() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try (TransactionManager manager = client.transactionManager()) { long count; @@ -1192,7 +1169,6 @@ public void transactionManagerUpdate() throws InterruptedException { @SuppressWarnings("resource") @Test public void transactionManagerBatchUpdate() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try (TransactionManager manager = client.transactionManager()) { long[] count; @@ -1218,7 +1194,6 @@ public void transactionManagerBatchUpdate() throws InterruptedException { @SuppressWarnings("resource") @Test public void transactionManagerBuffer() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try (TransactionManager manager = client.transactionManager()) { TransactionContext transaction = manager.begin(); @@ -1417,7 +1392,6 @@ public void transactionManagerReadRowUsingIndexInvalidatedDuringTransaction() @Test public void partitionedDml() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try { assertThat(client.executePartitionedUpdate(UPDATE_STATEMENT)).isEqualTo(UPDATE_COUNT); @@ -1429,7 +1403,6 @@ public void partitionedDml() throws InterruptedException { @Test public void write() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try { Timestamp timestamp = client.write(Arrays.asList(Mutation.delete("FOO", KeySet.all()))); @@ -1442,7 +1415,6 @@ public void write() throws InterruptedException { @Test public void writeAtLeastOnce() throws InterruptedException { - initReadWriteSessionPool(); invalidateSessionPool(); try { Timestamp timestamp = diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolLeakTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolLeakTest.java index 2dc31bb28a..f559a04b94 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolLeakTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolLeakTest.java @@ -80,15 +80,10 @@ public void setUp() { .setProjectId("[PROJECT]") .setChannelProvider(channelProvider) .setCredentials(NoCredentials.getInstance()); - // Make sure the session pool is empty by default, does not contain any write-prepared sessions, + // Make sure the session pool is empty by default, does not contain any sessions, // contains at most 2 sessions, and creates sessions in steps of 1. builder.setSessionPoolOption( - SessionPoolOptions.newBuilder() - .setMinSessions(0) - .setMaxSessions(2) - .setIncStep(1) - .setWriteSessionsFraction(0.0f) - .build()); + SessionPoolOptions.newBuilder().setMinSessions(0).setMaxSessions(2).setIncStep(1).build()); spanner = builder.build().getService(); client = spanner.getDatabaseClient(DatabaseId.of("[PROJECT]", "[INSTANCE]", "[DATABASE]")); pool = ((DatabaseClientImpl) client).pool; @@ -162,15 +157,15 @@ public void run() { @Test public void testTransactionManagerExceptionOnBegin() { - transactionManagerTest( - new Runnable() { - @Override - public void run() { - mockSpanner.setBeginTransactionExecutionTime( - SimulatedExecutionTime.ofException(FAILED_PRECONDITION)); - } - }, - 1); + assertThat(pool.getNumberOfSessionsInPool(), is(equalTo(0))); + mockSpanner.setBeginTransactionExecutionTime( + SimulatedExecutionTime.ofException(FAILED_PRECONDITION)); + try (TransactionManager txManager = client.transactionManager()) { + // This should not cause an error, as the actual BeginTransaction will be included with the + // first statement of the transaction. + txManager.begin(); + } + assertThat(pool.getNumberOfSessionsInPool(), is(equalTo(1))); } private void transactionManagerTest(Runnable setup, int expectedNumberOfSessionsAfterExecution) { diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolMaintainerTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolMaintainerTest.java index 0e72b2b9bc..b5d35a4d89 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolMaintainerTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolMaintainerTest.java @@ -150,8 +150,8 @@ public void testKeepAlive() throws Exception { // Checkout two sessions and do a maintenance loop. Still no sessions should be getting any // pings. - Session session1 = pool.getReadSession(); - Session session2 = pool.getReadSession(); + Session session1 = pool.get(); + Session session2 = pool.get(); runMaintainanceLoop(clock, pool, 1); assertThat(pingedSessions).isEmpty(); @@ -173,9 +173,9 @@ public void testKeepAlive() throws Exception { // Now check out three sessions so the pool will create an additional session. The pool will // only keep 2 sessions alive, as that is the setting for MinSessions. - Session session3 = pool.getReadSession(); - Session session4 = pool.getReadSession(); - Session session5 = pool.getReadSession(); + Session session3 = pool.get(); + Session session4 = pool.get(); + Session session5 = pool.get(); // Note that session2 was now the first session in the pool as it was the last to receive a // ping. assertThat(session3.getName()).isEqualTo(session2.getName()); @@ -192,7 +192,7 @@ public void testKeepAlive() throws Exception { // should cause only one session to get a ping. clock.currentTimeMillis += TimeUnit.MINUTES.toMillis(options.getKeepAliveIntervalMinutes()) + 1; // We are now checking out session2 because - Session session6 = pool.getReadSession(); + Session session6 = pool.get(); // The session that was first in the pool now is equal to the initial first session as each full // round of pings will swap the order of the first MinSessions sessions in the pool. assertThat(session6.getName()).isEqualTo(session1.getName()); @@ -208,9 +208,9 @@ public void testKeepAlive() throws Exception { // Now check out 3 sessions again and make sure the 'extra' session is checked in last. That // will make it eligible for pings. - Session session7 = pool.getReadSession(); - Session session8 = pool.getReadSession(); - Session session9 = pool.getReadSession(); + Session session7 = pool.get(); + Session session8 = pool.get(); + Session session9 = pool.get(); assertThat(session7.getName()).isEqualTo(session1.getName()); assertThat(session8.getName()).isEqualTo(session2.getName()); @@ -244,8 +244,8 @@ public void testIdleSessions() throws Exception { assertThat(idledSessions).isEmpty(); // Checkout two sessions and do a maintenance loop. Still no sessions should be removed. - Session session1 = pool.getReadSession(); - Session session2 = pool.getReadSession(); + Session session1 = pool.get(); + Session session2 = pool.get(); runMaintainanceLoop(clock, pool, 1); assertThat(idledSessions).isEmpty(); @@ -262,9 +262,9 @@ public void testIdleSessions() throws Exception { // Now check out three sessions so the pool will create an additional session. The pool will // only keep 2 sessions alive, as that is the setting for MinSessions. - Session session3 = pool.getReadSession().get(); - Session session4 = pool.getReadSession().get(); - Session session5 = pool.getReadSession().get(); + Session session3 = pool.get().get(); + Session session4 = pool.get().get(); + Session session5 = pool.get().get(); // Note that session2 was now the first session in the pool as it was the last to receive a // ping. assertThat(session3.getName()).isEqualTo(session2.getName()); @@ -279,9 +279,9 @@ public void testIdleSessions() throws Exception { assertThat(pool.totalSessions()).isEqualTo(2); // Check out three sessions again and keep one session checked out. - Session session6 = pool.getReadSession().get(); - Session session7 = pool.getReadSession().get(); - Session session8 = pool.getReadSession().get(); + Session session6 = pool.get().get(); + Session session7 = pool.get().get(); + Session session8 = pool.get().get(); session8.close(); session7.close(); // Now advance the clock to idle sessions. This should remove session8 from the pool. @@ -293,9 +293,9 @@ public void testIdleSessions() throws Exception { // Check out three sessions and keep them all checked out. No sessions should be removed from // the pool. - Session session9 = pool.getReadSession().get(); - Session session10 = pool.getReadSession().get(); - Session session11 = pool.getReadSession().get(); + Session session9 = pool.get().get(); + Session session10 = pool.get().get(); + Session session11 = pool.get().get(); runMaintainanceLoop(clock, pool, loopsToIdleSessions); assertThat(idledSessions).containsExactly(session5, session8); assertThat(pool.totalSessions()).isEqualTo(3); diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolStressTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolStressTest.java index b806f5fad6..3916df8652 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolStressTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolStressTest.java @@ -205,15 +205,6 @@ private void expireSession(Session session) { } } - private void assertWritePrepared(Session session) { - String name = session.getName(); - synchronized (lock) { - if (!sessions.containsKey(name) || !sessions.get(name)) { - setFailed(); - } - } - } - private void resetTransaction(SessionImpl session) { String name = session.getName(); synchronized (lock) { @@ -242,7 +233,6 @@ public void stressTest() throws Exception { final int numOperationsPerThread = 1000; final CountDownLatch releaseThreads = new CountDownLatch(1); final CountDownLatch threadsDone = new CountDownLatch(concurrentThreads); - final int writeOperationFraction = 5; setupSpanner(db); int minSessions = 2; int maxSessions = concurrentThreads / 2; @@ -280,15 +270,8 @@ public void run() { Uninterruptibles.awaitUninterruptibly(releaseThreads); for (int j = 0; j < numOperationsPerThread; j++) { try { - PooledSessionFuture session = null; - if (random.nextInt(10) < writeOperationFraction) { - session = pool.getReadWriteSession(); - PooledSession sess = session.get(); - assertWritePrepared(sess); - } else { - session = pool.getReadSession(); - session.get(); - } + PooledSessionFuture session = pool.get(); + session.get(); Uninterruptibles.sleepUninterruptibly( random.nextInt(5), TimeUnit.MILLISECONDS); resetTransaction(session.get().delegate); diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolTest.java index 8c7e920f78..2087fc1eda 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolTest.java @@ -54,7 +54,6 @@ import com.google.cloud.spanner.TransactionRunnerImpl.TransactionContextImpl; import com.google.cloud.spanner.spi.v1.SpannerRpc; import com.google.cloud.spanner.spi.v1.SpannerRpc.ResultStreamConsumer; -import com.google.common.base.Stopwatch; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.Uninterruptibles; import com.google.protobuf.ByteString; @@ -180,7 +179,7 @@ public void testClosedPoolIncludesClosedException() { assertThat(pool.isValid()).isTrue(); closePoolWithStacktrace(); try { - pool.getReadSession(); + pool.get(); fail("missing expected exception"); } catch (IllegalStateException e) { assertThat(e.getCause()).isInstanceOf(ClosedException.class); @@ -198,7 +197,7 @@ private void closePoolWithStacktrace() { public void sessionCreation() { setupMockSessionCreation(); pool = createPool(); - try (Session session = pool.getReadSession()) { + try (Session session = pool.get()) { assertThat(session).isNotNull(); } } @@ -207,25 +206,18 @@ public void sessionCreation() { public void poolLifo() { setupMockSessionCreation(); pool = createPool(); - Session session1 = pool.getReadSession().get(); - Session session2 = pool.getReadSession().get(); + Session session1 = pool.get().get(); + Session session2 = pool.get().get(); assertThat(session1).isNotEqualTo(session2); session2.close(); session1.close(); - Session session3 = pool.getReadSession().get(); - Session session4 = pool.getReadSession().get(); + Session session3 = pool.get().get(); + Session session4 = pool.get().get(); assertThat(session3).isEqualTo(session1); assertThat(session4).isEqualTo(session2); session3.close(); session4.close(); - - Session session5 = pool.getReadWriteSession().get(); - Session session6 = pool.getReadWriteSession().get(); - assertThat(session5).isEqualTo(session4); - assertThat(session6).isEqualTo(session3); - session6.close(); - session5.close(); } @Test @@ -260,9 +252,9 @@ public void run() { .when(sessionClient) .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); pool = createPool(); - Session session1 = pool.getReadSession(); + Session session1 = pool.get(); // Leaked sessions - PooledSessionFuture leakedSession = pool.getReadSession(); + PooledSessionFuture leakedSession = pool.get(); // Clear the leaked exception to suppress logging of expected exceptions. leakedSession.clearLeakedException(); session1.close(); @@ -338,7 +330,7 @@ public Void call() throws Exception { .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); pool = createPool(); - PooledSessionFuture leakedSession = pool.getReadSession(); + PooledSessionFuture leakedSession = pool.get(); // Suppress expected leakedSession warning. leakedSession.clearLeakedException(); AtomicBoolean failed = new AtomicBoolean(false); @@ -396,12 +388,12 @@ public Void call() throws Exception { .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); pool = createPool(); - PooledSessionFuture leakedSession = pool.getReadSession(); + PooledSessionFuture leakedSession = pool.get(); // Suppress expected leakedSession warning. leakedSession.clearLeakedException(); AtomicBoolean failed = new AtomicBoolean(false); CountDownLatch latch = new CountDownLatch(1); - getReadWriteSessionAsync(latch, failed); + getSessionAsync(latch, failed); insideCreation.await(); pool.closeAsync(new SpannerImpl.ClosedException()); releaseCreation.countDown(); @@ -446,51 +438,6 @@ public Void call() throws Exception { assertThat(f.isDone()).isTrue(); } - @Test - public void poolClosesEvenIfPreparationFails() throws Exception { - final SessionImpl session = mockSession(); - doAnswer( - new Answer() { - @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Runnable() { - @Override - public void run() { - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - consumer.onSessionReady(session); - } - }); - return null; - } - }) - .when(sessionClient) - .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); - final CountDownLatch insidePrepare = new CountDownLatch(1); - final CountDownLatch releasePrepare = new CountDownLatch(1); - doAnswer( - new Answer() { - @Override - public Session answer(InvocationOnMock invocation) throws Throwable { - insidePrepare.countDown(); - releasePrepare.await(); - throw SpannerExceptionFactory.newSpannerException(new RuntimeException()); - } - }) - .when(session) - .prepareReadWriteTransaction(); - pool = createPool(); - AtomicBoolean failed = new AtomicBoolean(false); - CountDownLatch latch = new CountDownLatch(1); - getReadWriteSessionAsync(latch, failed); - insidePrepare.await(); - ListenableFuture f = pool.closeAsync(new SpannerImpl.ClosedException()); - releasePrepare.countDown(); - f.get(); - assertThat(f.isDone()).isTrue(); - } - @Test public void poolClosureFailsNewRequests() { final SessionImpl session = mockSession(); @@ -513,13 +460,13 @@ public void run() { .when(sessionClient) .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); pool = createPool(); - PooledSessionFuture leakedSession = pool.getReadSession(); + PooledSessionFuture leakedSession = pool.get(); leakedSession.get(); // Suppress expected leakedSession warning. leakedSession.clearLeakedException(); pool.closeAsync(new SpannerImpl.ClosedException()); try { - pool.getReadSession(); + pool.get(); fail("Expected exception"); } catch (IllegalStateException ex) { assertNotNull(ex.getMessage()); @@ -566,283 +513,13 @@ public Void call() { .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); pool = createPool(); try { - pool.getReadSession().get(); - fail("Expected exception"); - } catch (SpannerException ex) { - assertThat(ex.getErrorCode()).isEqualTo(ErrorCode.INTERNAL); - } - } - - @Test - public void creationExceptionPropagatesToReadWriteSession() { - doAnswer( - new Answer() { - @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Callable() { - @Override - public Void call() { - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - consumer.onSessionCreateFailure( - SpannerExceptionFactory.newSpannerException(ErrorCode.INTERNAL, ""), 1); - return null; - } - }); - return null; - } - }) - .when(sessionClient) - .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); - pool = createPool(); - try { - pool.getReadWriteSession().get(); + pool.get().get(); fail("Expected exception"); } catch (SpannerException ex) { assertThat(ex.getErrorCode()).isEqualTo(ErrorCode.INTERNAL); } } - @Test - public void prepareExceptionPropagatesToReadWriteSession() { - final SessionImpl session = mockSession(); - doAnswer( - new Answer() { - @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Runnable() { - @Override - public void run() { - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - consumer.onSessionReady(session); - } - }); - return null; - } - }) - .when(sessionClient) - .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); - doThrow(SpannerExceptionFactory.newSpannerException(ErrorCode.INTERNAL, "")) - .when(session) - .prepareReadWriteTransaction(); - pool = createPool(); - try { - pool.getReadWriteSession().get(); - fail("Expected exception"); - } catch (SpannerException ex) { - assertThat(ex.getErrorCode()).isEqualTo(ErrorCode.INTERNAL); - } - } - - @Test - public void getReadWriteSession() { - final SessionImpl mockSession = mockSession(); - doAnswer( - new Answer() { - @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Runnable() { - @Override - public void run() { - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - consumer.onSessionReady(mockSession); - } - }); - return null; - } - }) - .when(sessionClient) - .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); - pool = createPool(); - try (PooledSessionFuture session = pool.getReadWriteSession()) { - assertThat(session).isNotNull(); - session.get(); - verify(mockSession).prepareReadWriteTransaction(); - } - } - - @Test - public void getMultipleReadWriteSessions() throws Exception { - SessionImpl mockSession1 = mockSession(); - SessionImpl mockSession2 = mockSession(); - final LinkedList sessions = - new LinkedList<>(Arrays.asList(mockSession1, mockSession2)); - doAnswer( - new Answer() { - @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Runnable() { - @Override - public void run() { - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - consumer.onSessionReady(sessions.pop()); - } - }); - return null; - } - }) - .when(sessionClient) - .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); - pool = createPool(); - PooledSessionFuture session1 = pool.getReadWriteSession(); - PooledSessionFuture session2 = pool.getReadWriteSession(); - session1.get(); - session2.get(); - verify(mockSession1).prepareReadWriteTransaction(); - verify(mockSession2).prepareReadWriteTransaction(); - session1.close(); - session2.close(); - } - - @Test - public void getMultipleConcurrentReadWriteSessions() { - AtomicBoolean failed = new AtomicBoolean(false); - final SessionImpl session = mockSession(); - doAnswer( - new Answer() { - @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Runnable() { - @Override - public void run() { - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - consumer.onSessionReady(session); - } - }); - return null; - } - }) - .when(sessionClient) - .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); - - pool = createPool(); - int numSessions = 5; - final CountDownLatch latch = new CountDownLatch(numSessions); - for (int i = 0; i < numSessions; i++) { - getReadWriteSessionAsync(latch, failed); - } - Uninterruptibles.awaitUninterruptibly(latch); - } - - @Test - public void sessionIsPrePrepared() { - final SessionImpl mockSession1 = mockSession(); - final SessionImpl mockSession2 = mockSession(); - final CountDownLatch prepareLatch = new CountDownLatch(1); - doAnswer( - new Answer() { - - @Override - public Void answer(InvocationOnMock arg0) { - prepareLatch.countDown(); - return null; - } - }) - .when(mockSession1) - .prepareReadWriteTransaction(); - doAnswer( - new Answer() { - - @Override - public Void answer(InvocationOnMock arg0) { - prepareLatch.countDown(); - return null; - } - }) - .when(mockSession2) - .prepareReadWriteTransaction(); - doAnswer( - new Answer() { - @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Runnable() { - @Override - public void run() { - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - consumer.onSessionReady(mockSession1); - consumer.onSessionReady(mockSession2); - } - }); - return null; - } - }) - .when(sessionClient) - .asyncBatchCreateSessions(Mockito.eq(2), Mockito.anyBoolean(), any(SessionConsumer.class)); - - options = - SessionPoolOptions.newBuilder() - .setMinSessions(2) - .setMaxSessions(2) - .setWriteSessionsFraction(0.5f) - .build(); - pool = createPool(); - // One of the sessions would be pre prepared. - Uninterruptibles.awaitUninterruptibly(prepareLatch); - PooledSession readSession = pool.getReadSession().get(); - PooledSession writeSession = pool.getReadWriteSession().get(); - verify(writeSession.delegate, times(1)).prepareReadWriteTransaction(); - verify(readSession.delegate, never()).prepareReadWriteTransaction(); - readSession.close(); - writeSession.close(); - } - - @Test - public void getReadSessionFallsBackToWritePreparedSession() throws Exception { - final SessionImpl mockSession1 = mockSession(); - final CountDownLatch prepareLatch = new CountDownLatch(2); - doAnswer( - new Answer() { - @Override - public Void answer(InvocationOnMock arg0) { - prepareLatch.countDown(); - return null; - } - }) - .when(mockSession1) - .prepareReadWriteTransaction(); - doAnswer( - new Answer() { - @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Runnable() { - @Override - public void run() { - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - consumer.onSessionReady(mockSession1); - } - }); - return null; - } - }) - .when(sessionClient) - .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); - options = - SessionPoolOptions.newBuilder() - .setMinSessions(minSessions) - .setMaxSessions(1) - .setWriteSessionsFraction(1.0f) - .build(); - pool = createPool(); - pool.getReadWriteSession().close(); - prepareLatch.await(); - // This session should also be write prepared. - PooledSession readSession = pool.getReadSession().get(); - verify(readSession.delegate, times(2)).prepareReadWriteTransaction(); - } - @Test public void failOnPoolExhaustion() { options = @@ -870,50 +547,19 @@ public void run() { .when(sessionClient) .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); pool = createPool(); - Session session1 = pool.getReadSession(); + Session session1 = pool.get(); try { - pool.getReadSession(); + pool.get(); fail("Expected exception"); } catch (SpannerException ex) { assertThat(ex.getErrorCode()).isEqualTo(ErrorCode.RESOURCE_EXHAUSTED); } session1.close(); - session1 = pool.getReadSession(); + session1 = pool.get(); assertThat(session1).isNotNull(); session1.close(); } - @Test - public void poolWorksWhenSessionNotFound() { - SessionImpl mockSession1 = mockSession(); - SessionImpl mockSession2 = mockSession(); - final LinkedList sessions = - new LinkedList<>(Arrays.asList(mockSession1, mockSession2)); - doThrow(SpannerExceptionFactoryTest.newSessionNotFoundException(sessionName)) - .when(mockSession1) - .prepareReadWriteTransaction(); - doAnswer( - new Answer() { - @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Runnable() { - @Override - public void run() { - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - consumer.onSessionReady(sessions.pop()); - } - }); - return null; - } - }) - .when(sessionClient) - .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); - pool = createPool(); - assertThat(pool.getReadWriteSession().get().delegate).isEqualTo(mockSession2); - } - @Test public void idleSessionCleanup() throws Exception { options = @@ -953,12 +599,12 @@ public void run() { clock.currentTimeMillis = System.currentTimeMillis(); pool = createPool(clock); // Make sure pool has been initialized - pool.getReadSession().close(); + pool.get().close(); runMaintainanceLoop(clock, pool, pool.poolMaintainer.numClosureCycles); assertThat(pool.numIdleSessionsRemoved()).isEqualTo(0L); - PooledSessionFuture readSession1 = pool.getReadSession(); - PooledSessionFuture readSession2 = pool.getReadSession(); - PooledSessionFuture readSession3 = pool.getReadSession(); + PooledSessionFuture readSession1 = pool.get(); + PooledSessionFuture readSession2 = pool.get(); + PooledSessionFuture readSession3 = pool.get(); // Wait until the sessions have actually been gotten in order to make sure they are in use in // parallel. readSession1.get(); @@ -973,9 +619,9 @@ public void run() { assertThat(pool.numIdleSessionsRemoved()).isEqualTo(0L); // Counters have now been reset // Use all 3 sessions sequentially - pool.getReadSession().close(); - pool.getReadSession().close(); - pool.getReadSession().close(); + pool.get().close(); + pool.get().close(); + pool.get().close(); // Advance the time by running the maintainer. This should cause // one session to be kept alive and two sessions to be removed. long cycles = @@ -1017,8 +663,8 @@ public void run() { FakeClock clock = new FakeClock(); clock.currentTimeMillis = System.currentTimeMillis(); pool = createPool(clock); - PooledSessionFuture session1 = pool.getReadSession(); - PooledSessionFuture session2 = pool.getReadSession(); + PooledSessionFuture session1 = pool.get(); + PooledSessionFuture session2 = pool.get(); session1.get(); session2.get(); session1.close(); @@ -1029,7 +675,7 @@ public void run() { verify(session, times(2)).singleUse(any(TimestampBound.class)); clock.currentTimeMillis += clock.currentTimeMillis + (options.getKeepAliveIntervalMinutes() + 5) * 60 * 1000; - session1 = pool.getReadSession(); + session1 = pool.get(); session1.writeAtLeastOnce(new ArrayList()); session1.close(); runMaintainanceLoop(clock, pool, pool.poolMaintainer.numKeepAliveCycles); @@ -1040,156 +686,53 @@ public void run() { } @Test - public void testMaintainerKeepsWriteProportion() throws Exception { + public void blockAndTimeoutOnPoolExhaustion() throws Exception { + // Create a session pool with max 1 session and a low timeout for waiting for a session. options = SessionPoolOptions.newBuilder() - .setMinSessions(10) - .setMaxSessions(20) - .setWriteSessionsFraction(0.5f) + .setMinSessions(minSessions) + .setMaxSessions(1) + .setInitialWaitForSessionTimeoutMillis(20L) .build(); - final SessionImpl session = mockSession(); - mockKeepAlive(session); - // This is cheating as we are returning the same session each but it makes the verification - // easier. - doAnswer( - new Answer() { + setupMockSessionCreation(); + pool = createPool(); + // Take the only session that can be in the pool. + PooledSessionFuture checkedOutSession = pool.get(); + checkedOutSession.get(); + ExecutorService executor = Executors.newFixedThreadPool(1); + final CountDownLatch latch = new CountDownLatch(1); + // Then try asynchronously to take another session. This attempt should time out. + Future fut = + executor.submit( + new Callable() { @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Runnable() { - @Override - public void run() { - int sessionCount = invocation.getArgumentAt(0, Integer.class); - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - for (int i = 0; i < sessionCount; i++) { - consumer.onSessionReady(session); - } - } - }); + public Void call() { + latch.countDown(); + PooledSessionFuture session = pool.get(); + session.close(); return null; } - }) - .when(sessionClient) - .asyncBatchCreateSessions(anyInt(), Mockito.anyBoolean(), any(SessionConsumer.class)); - FakeClock clock = new FakeClock(); - clock.currentTimeMillis = System.currentTimeMillis(); - pool = createPool(clock); - // Wait until all sessions have been created and prepared. - waitForExpectedSessionPool(options.getMinSessions(), options.getWriteSessionsFraction()); - assertThat(pool.getNumberOfSessionsInPool()).isEqualTo(options.getMinSessions()); - assertThat(pool.getNumberOfAvailableWritePreparedSessions()) - .isEqualTo((int) Math.ceil(options.getMinSessions() * options.getWriteSessionsFraction())); - - // Run maintainer numKeepAliveCycles. No pings should be executed during these. - runMaintainanceLoop(clock, pool, pool.poolMaintainer.numKeepAliveCycles); - verify(session, never()).singleUse(any(TimestampBound.class)); - // Run maintainer numKeepAliveCycles again. All sessions should now be pinged. - runMaintainanceLoop(clock, pool, pool.poolMaintainer.numKeepAliveCycles); - verify(session, times(options.getMinSessions())).singleUse(any(TimestampBound.class)); - // Verify that all sessions are still in the pool, and that the write fraction is maintained. - assertThat(pool.getNumberOfSessionsInPool()).isEqualTo(options.getMinSessions()); - assertThat(pool.getNumberOfWriteSessionsInPool()) - .isEqualTo( - (int) Math.ceil(pool.getNumberOfSessionsInPool() * options.getWriteSessionsFraction())); - - // Check out MaxSessions sessions to add additional sessions to the pool. - List sessions = new ArrayList<>(options.getMaxSessions()); - for (int i = 0; i < options.getMaxSessions(); i++) { - sessions.add(pool.getReadSession()); - } - for (Session s : sessions) { - s.close(); - } - // There should be MaxSessions in the pool and the writeFraction should be respected. - waitForExpectedSessionPool(options.getMaxSessions(), options.getWriteSessionsFraction()); - assertThat(pool.getNumberOfSessionsInPool()).isEqualTo(options.getMaxSessions()); - assertThat(pool.getNumberOfAvailableWritePreparedSessions()) - .isEqualTo((int) Math.ceil(options.getMaxSessions() * options.getWriteSessionsFraction())); - - // Advance the clock to allow the sessions to time out or be kept alive. - clock.currentTimeMillis += - clock.currentTimeMillis + (options.getKeepAliveIntervalMinutes() + 5) * 60 * 1000; - runMaintainanceLoop(clock, pool, pool.poolMaintainer.numKeepAliveCycles); - // The session pool only keeps MinSessions alive. - verify(session, times(options.getMinSessions())).singleUse(any(TimestampBound.class)); - // Verify that MinSessions and WriteFraction are respected. - waitForExpectedSessionPool(options.getMinSessions(), options.getWriteSessionsFraction()); - assertThat(pool.getNumberOfSessionsInPool()).isEqualTo(options.getMinSessions()); - assertThat(pool.getNumberOfAvailableWritePreparedSessions()) - .isEqualTo((int) Math.ceil(options.getMinSessions() * options.getWriteSessionsFraction())); - - pool.closeAsync(new SpannerImpl.ClosedException()).get(5L, TimeUnit.SECONDS); - } - - private void waitForExpectedSessionPool(int expectedSessions, float writeFraction) - throws InterruptedException { - Stopwatch watch = Stopwatch.createStarted(); - while ((pool.getNumberOfSessionsInPool() < expectedSessions - || pool.getNumberOfAvailableWritePreparedSessions() - < Math.ceil(expectedSessions * writeFraction)) - && watch.elapsed(TimeUnit.SECONDS) < 5) { - Thread.sleep(1L); + }); + // Wait until the background thread is actually waiting for a session. + latch.await(); + // Wait until the request has timed out. + int waitCount = 0; + while (pool.getNumWaiterTimeouts() == 0L && waitCount < 1000) { + Thread.sleep(5L); + waitCount++; } - } + // Return the checked out session to the pool so the async request will get a session and + // finish. + checkedOutSession.close(); + // Verify that the async request also succeeds. + fut.get(10L, TimeUnit.SECONDS); + executor.shutdown(); - @Test - public void blockAndTimeoutOnPoolExhaustion() throws Exception { - // Try to take a read or a read/write session. These requests should block. - for (Boolean write : new Boolean[] {true, false}) { - // Create a session pool with max 1 session and a low timeout for waiting for a session. - options = - SessionPoolOptions.newBuilder() - .setMinSessions(minSessions) - .setMaxSessions(1) - .setInitialWaitForSessionTimeoutMillis(20L) - .build(); - setupMockSessionCreation(); - pool = createPool(); - // Take the only session that can be in the pool. - PooledSessionFuture checkedOutSession = pool.getReadSession(); - checkedOutSession.get(); - final Boolean finWrite = write; - ExecutorService executor = Executors.newFixedThreadPool(1); - final CountDownLatch latch = new CountDownLatch(1); - // Then try asynchronously to take another session. This attempt should time out. - Future fut = - executor.submit( - new Callable() { - @Override - public Void call() { - PooledSessionFuture session; - latch.countDown(); - if (finWrite) { - session = pool.getReadWriteSession(); - } else { - session = pool.getReadSession(); - } - session.close(); - return null; - } - }); - // Wait until the background thread is actually waiting for a session. - latch.await(); - // Wait until the request has timed out. - int waitCount = 0; - while (pool.getNumWaiterTimeouts() == 0L && waitCount < 1000) { - Thread.sleep(5L); - waitCount++; - } - // Return the checked out session to the pool so the async request will get a session and - // finish. - checkedOutSession.close(); - // Verify that the async request also succeeds. - fut.get(10L, TimeUnit.SECONDS); - executor.shutdown(); - - // Verify that the session was returned to the pool and that we can get it again. - Session session = pool.getReadSession(); - assertThat(session).isNotNull(); - session.close(); - assertThat(pool.getNumWaiterTimeouts()).isAtLeast(1L); - } + // Verify that the session was returned to the pool and that we can get it again. + Session session = pool.get(); + assertThat(session).isNotNull(); + session.close(); + assertThat(pool.getNumWaiterTimeouts()).isAtLeast(1L); } @Test @@ -1247,7 +790,7 @@ public void run() { FakeClock clock = new FakeClock(); clock.currentTimeMillis = System.currentTimeMillis(); pool = createPool(clock); - ReadContext context = pool.getReadSession().singleUse(); + ReadContext context = pool.get().singleUse(); ResultSet resultSet = context.executeQuery(statement); assertThat(resultSet.next()).isTrue(); } @@ -1303,7 +846,7 @@ public void run() { FakeClock clock = new FakeClock(); clock.currentTimeMillis = System.currentTimeMillis(); pool = createPool(clock); - ReadOnlyTransaction transaction = pool.getReadSession().readOnlyTransaction(); + ReadOnlyTransaction transaction = pool.get().readOnlyTransaction(); ResultSet resultSet = transaction.executeQuery(statement); assertThat(resultSet.next()).isTrue(); } @@ -1327,254 +870,171 @@ public void testSessionNotFoundReadWriteTransaction() { for (ReadWriteTransactionTestStatementType statementType : ReadWriteTransactionTestStatementType.values()) { final ReadWriteTransactionTestStatementType executeStatementType = statementType; - for (boolean prepared : new boolean[] {true, false}) { - final boolean hasPreparedTransaction = prepared; - SpannerRpc.StreamingCall closedStreamingCall = mock(SpannerRpc.StreamingCall.class); - doThrow(sessionNotFound).when(closedStreamingCall).request(Mockito.anyInt()); - SpannerRpc rpc = mock(SpannerRpc.class); - when(rpc.asyncDeleteSession(Mockito.anyString(), Mockito.anyMap())) - .thenReturn(ApiFutures.immediateFuture(Empty.getDefaultInstance())); - when(rpc.executeQuery( - any(ExecuteSqlRequest.class), any(ResultStreamConsumer.class), any(Map.class))) - .thenReturn(closedStreamingCall); - when(rpc.executeQuery(any(ExecuteSqlRequest.class), any(Map.class))) - .thenThrow(sessionNotFound); - when(rpc.executeBatchDml(any(ExecuteBatchDmlRequest.class), any(Map.class))) - .thenThrow(sessionNotFound); - when(rpc.commitAsync(any(CommitRequest.class), any(Map.class))) - .thenReturn(ApiFutures.immediateFailedFuture(sessionNotFound)); - when(rpc.rollbackAsync(any(RollbackRequest.class), any(Map.class))) - .thenReturn(ApiFutures.immediateFailedFuture(sessionNotFound)); - final SessionImpl closedSession = mock(SessionImpl.class); - when(closedSession.getName()) - .thenReturn("projects/dummy/instances/dummy/database/dummy/sessions/session-closed"); - ByteString preparedTransactionId = - hasPreparedTransaction ? ByteString.copyFromUtf8("test-txn") : null; - final TransactionContextImpl closedTransactionContext = - TransactionContextImpl.newBuilder() - .setSession(closedSession) - .setTransactionId(preparedTransactionId) - .setRpc(rpc) - .build(); - when(closedSession.asyncClose()) - .thenReturn(ApiFutures.immediateFuture(Empty.getDefaultInstance())); - when(closedSession.newTransaction()).thenReturn(closedTransactionContext); - when(closedSession.beginTransactionAsync()).thenThrow(sessionNotFound); - TransactionRunnerImpl closedTransactionRunner = - new TransactionRunnerImpl(closedSession, rpc, 10, false); - closedTransactionRunner.setSpan(mock(Span.class)); - when(closedSession.readWriteTransaction()).thenReturn(closedTransactionRunner); - - final SessionImpl openSession = mock(SessionImpl.class); - when(openSession.asyncClose()) - .thenReturn(ApiFutures.immediateFuture(Empty.getDefaultInstance())); - when(openSession.getName()) - .thenReturn("projects/dummy/instances/dummy/database/dummy/sessions/session-open"); - final TransactionContextImpl openTransactionContext = mock(TransactionContextImpl.class); - when(openSession.newTransaction()).thenReturn(openTransactionContext); - when(openSession.beginTransactionAsync()) - .thenReturn(ApiFutures.immediateFuture(ByteString.copyFromUtf8("open-txn"))); - TransactionRunnerImpl openTransactionRunner = - new TransactionRunnerImpl(openSession, mock(SpannerRpc.class), 10, false); - openTransactionRunner.setSpan(mock(Span.class)); - when(openSession.readWriteTransaction()).thenReturn(openTransactionRunner); - - ResultSet openResultSet = mock(ResultSet.class); - when(openResultSet.next()).thenReturn(true, false); - ResultSet planResultSet = mock(ResultSet.class); - when(planResultSet.getStats()).thenReturn(ResultSetStats.getDefaultInstance()); - when(openTransactionContext.executeQuery(queryStatement)).thenReturn(openResultSet); - when(openTransactionContext.analyzeQuery(queryStatement, QueryAnalyzeMode.PLAN)) - .thenReturn(planResultSet); - when(openTransactionContext.executeUpdate(updateStatement)).thenReturn(1L); - when(openTransactionContext.batchUpdate(Arrays.asList(updateStatement, updateStatement))) - .thenReturn(new long[] {1L, 1L}); - SpannerImpl spanner = mock(SpannerImpl.class); - SessionClient sessionClient = mock(SessionClient.class); - when(spanner.getSessionClient(db)).thenReturn(sessionClient); - - doAnswer( - new Answer() { - @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Runnable() { - @Override - public void run() { - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - consumer.onSessionReady(closedSession); - } - }); - return null; - } - }) - .doAnswer( - new Answer() { - @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Runnable() { - @Override - public void run() { - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - consumer.onSessionReady(openSession); - } - }); - return null; + SpannerRpc.StreamingCall closedStreamingCall = mock(SpannerRpc.StreamingCall.class); + doThrow(sessionNotFound).when(closedStreamingCall).request(Mockito.anyInt()); + SpannerRpc rpc = mock(SpannerRpc.class); + when(rpc.asyncDeleteSession(Mockito.anyString(), Mockito.anyMap())) + .thenReturn(ApiFutures.immediateFuture(Empty.getDefaultInstance())); + when(rpc.executeQuery( + any(ExecuteSqlRequest.class), any(ResultStreamConsumer.class), any(Map.class))) + .thenReturn(closedStreamingCall); + when(rpc.executeQuery(any(ExecuteSqlRequest.class), any(Map.class))) + .thenThrow(sessionNotFound); + when(rpc.executeBatchDml(any(ExecuteBatchDmlRequest.class), any(Map.class))) + .thenThrow(sessionNotFound); + when(rpc.commitAsync(any(CommitRequest.class), any(Map.class))) + .thenReturn(ApiFutures.immediateFailedFuture(sessionNotFound)); + when(rpc.rollbackAsync(any(RollbackRequest.class), any(Map.class))) + .thenReturn(ApiFutures.immediateFailedFuture(sessionNotFound)); + final SessionImpl closedSession = mock(SessionImpl.class); + when(closedSession.getName()) + .thenReturn("projects/dummy/instances/dummy/database/dummy/sessions/session-closed"); + final TransactionContextImpl closedTransactionContext = + TransactionContextImpl.newBuilder().setSession(closedSession).setRpc(rpc).build(); + when(closedSession.asyncClose()) + .thenReturn(ApiFutures.immediateFuture(Empty.getDefaultInstance())); + when(closedSession.newTransaction()).thenReturn(closedTransactionContext); + when(closedSession.beginTransactionAsync()).thenThrow(sessionNotFound); + TransactionRunnerImpl closedTransactionRunner = + new TransactionRunnerImpl(closedSession, rpc, 10); + closedTransactionRunner.setSpan(mock(Span.class)); + when(closedSession.readWriteTransaction()).thenReturn(closedTransactionRunner); + + final SessionImpl openSession = mock(SessionImpl.class); + when(openSession.asyncClose()) + .thenReturn(ApiFutures.immediateFuture(Empty.getDefaultInstance())); + when(openSession.getName()) + .thenReturn("projects/dummy/instances/dummy/database/dummy/sessions/session-open"); + final TransactionContextImpl openTransactionContext = mock(TransactionContextImpl.class); + when(openSession.newTransaction()).thenReturn(openTransactionContext); + when(openSession.beginTransactionAsync()) + .thenReturn(ApiFutures.immediateFuture(ByteString.copyFromUtf8("open-txn"))); + TransactionRunnerImpl openTransactionRunner = + new TransactionRunnerImpl(openSession, mock(SpannerRpc.class), 10); + openTransactionRunner.setSpan(mock(Span.class)); + when(openSession.readWriteTransaction()).thenReturn(openTransactionRunner); + + ResultSet openResultSet = mock(ResultSet.class); + when(openResultSet.next()).thenReturn(true, false); + ResultSet planResultSet = mock(ResultSet.class); + when(planResultSet.getStats()).thenReturn(ResultSetStats.getDefaultInstance()); + when(openTransactionContext.executeQuery(queryStatement)).thenReturn(openResultSet); + when(openTransactionContext.analyzeQuery(queryStatement, QueryAnalyzeMode.PLAN)) + .thenReturn(planResultSet); + when(openTransactionContext.executeUpdate(updateStatement)).thenReturn(1L); + when(openTransactionContext.batchUpdate(Arrays.asList(updateStatement, updateStatement))) + .thenReturn(new long[] {1L, 1L}); + SpannerImpl spanner = mock(SpannerImpl.class); + SessionClient sessionClient = mock(SessionClient.class); + when(spanner.getSessionClient(db)).thenReturn(sessionClient); + + doAnswer( + new Answer() { + @Override + public Void answer(final InvocationOnMock invocation) { + executor.submit( + new Runnable() { + @Override + public void run() { + SessionConsumerImpl consumer = + invocation.getArgumentAt(2, SessionConsumerImpl.class); + consumer.onSessionReady(closedSession); + } + }); + return null; + } + }) + .doAnswer( + new Answer() { + @Override + public Void answer(final InvocationOnMock invocation) { + executor.submit( + new Runnable() { + @Override + public void run() { + SessionConsumerImpl consumer = + invocation.getArgumentAt(2, SessionConsumerImpl.class); + consumer.onSessionReady(openSession); + } + }); + return null; + } + }) + .when(sessionClient) + .asyncBatchCreateSessions( + Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); + SessionPoolOptions options = + SessionPoolOptions.newBuilder() + .setMinSessions(0) // The pool should not auto-create any sessions + .setMaxSessions(2) + .setIncStep(1) + .setBlockIfPoolExhausted() + .build(); + SpannerOptions spannerOptions = mock(SpannerOptions.class); + when(spannerOptions.getSessionPoolOptions()).thenReturn(options); + when(spannerOptions.getNumChannels()).thenReturn(4); + when(spanner.getOptions()).thenReturn(spannerOptions); + SessionPool pool = + SessionPool.createPool(options, new TestExecutorFactory(), spanner.getSessionClient(db)); + try (PooledSessionFuture readWriteSession = pool.get()) { + TransactionRunner runner = readWriteSession.readWriteTransaction(); + try { + runner.run( + new TransactionCallable() { + private int callNumber = 0; + + @Override + public Integer run(TransactionContext transaction) { + callNumber++; + if (callNumber == 1) { + assertThat(transaction).isEqualTo(closedTransactionContext); + } else { + assertThat(transaction).isEqualTo(openTransactionContext); } - }) - .when(sessionClient) - .asyncBatchCreateSessions( - Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); - SessionPoolOptions options = - SessionPoolOptions.newBuilder() - .setMinSessions(0) // The pool should not auto-create any sessions - .setMaxSessions(2) - .setIncStep(1) - .setBlockIfPoolExhausted() - .build(); - SpannerOptions spannerOptions = mock(SpannerOptions.class); - when(spannerOptions.getSessionPoolOptions()).thenReturn(options); - when(spannerOptions.getNumChannels()).thenReturn(4); - when(spanner.getOptions()).thenReturn(spannerOptions); - SessionPool pool = - SessionPool.createPool( - options, new TestExecutorFactory(), spanner.getSessionClient(db)); - try (PooledSessionFuture readWriteSession = pool.getReadWriteSession()) { - TransactionRunner runner = readWriteSession.readWriteTransaction(); - try { - runner.run( - new TransactionCallable() { - private int callNumber = 0; - - @Override - public Integer run(TransactionContext transaction) { - callNumber++; - if (hasPreparedTransaction) { - // If the session had a prepared read/write transaction, that transaction will - // be given to the runner in the first place and the SessionNotFoundException - // will occur on the first query / update statement. - if (callNumber == 1) { - assertThat(transaction).isEqualTo(closedTransactionContext); - } else { - assertThat(transaction).isEqualTo(openTransactionContext); - } - } else { - // If the session did not have a prepared read/write transaction, the library - // tried to create a new transaction before handing it to the transaction - // runner. The creation of the new transaction failed with a - // SessionNotFoundException, and the session was re-created before the run - // method was called. - assertThat(transaction).isEqualTo(openTransactionContext); - } - switch (executeStatementType) { - case QUERY: - ResultSet resultSet = transaction.executeQuery(queryStatement); - assertThat(resultSet.next()).isTrue(); - break; - case ANALYZE: - ResultSet planResultSet = - transaction.analyzeQuery(queryStatement, QueryAnalyzeMode.PLAN); - assertThat(planResultSet.next()).isFalse(); - assertThat(planResultSet.getStats()).isNotNull(); - break; - case UPDATE: - long updateCount = transaction.executeUpdate(updateStatement); - assertThat(updateCount).isEqualTo(1L); - break; - case BATCH_UPDATE: - long[] updateCounts = - transaction.batchUpdate( - Arrays.asList(updateStatement, updateStatement)); - assertThat(updateCounts).isEqualTo(new long[] {1L, 1L}); - break; - case WRITE: - transaction.buffer(Mutation.delete("FOO", Key.of(1L))); - break; - case EXCEPTION: - throw new RuntimeException("rollback at call " + callNumber); - default: - fail("Unknown statement type: " + executeStatementType); - } - return callNumber; + switch (executeStatementType) { + case QUERY: + ResultSet resultSet = transaction.executeQuery(queryStatement); + assertThat(resultSet.next()).isTrue(); + break; + case ANALYZE: + ResultSet planResultSet = + transaction.analyzeQuery(queryStatement, QueryAnalyzeMode.PLAN); + assertThat(planResultSet.next()).isFalse(); + assertThat(planResultSet.getStats()).isNotNull(); + break; + case UPDATE: + long updateCount = transaction.executeUpdate(updateStatement); + assertThat(updateCount).isEqualTo(1L); + break; + case BATCH_UPDATE: + long[] updateCounts = + transaction.batchUpdate(Arrays.asList(updateStatement, updateStatement)); + assertThat(updateCounts).isEqualTo(new long[] {1L, 1L}); + break; + case WRITE: + transaction.buffer(Mutation.delete("FOO", Key.of(1L))); + break; + case EXCEPTION: + throw new RuntimeException("rollback at call " + callNumber); + default: + fail("Unknown statement type: " + executeStatementType); } - }); - } catch (Exception e) { - // The rollback will also cause a SessionNotFoundException, but this is caught, logged - // and further ignored by the library, meaning that the session will not be re-created - // for retry. Hence rollback at call 1. - assertThat(executeStatementType) - .isEqualTo(ReadWriteTransactionTestStatementType.EXCEPTION); - assertThat(e.getMessage()).contains("rollback at call 1"); - - // assertThat( - // executeStatementType == - // ReadWriteTransactionTestStatementType.EXCEPTION - // && e.getMessage().contains("rollback at call 1")) - // .isTrue(); - } + return callNumber; + } + }); + } catch (Exception e) { + // The rollback will also cause a SessionNotFoundException, but this is caught, logged + // and further ignored by the library, meaning that the session will not be re-created + // for retry. Hence rollback at call 1. + assertThat(executeStatementType) + .isEqualTo(ReadWriteTransactionTestStatementType.EXCEPTION); + assertThat(e.getMessage()).contains("rollback at call 1"); } - pool.closeAsync(new SpannerImpl.ClosedException()); } + pool.closeAsync(new SpannerImpl.ClosedException()); } } - @Test - public void testSessionNotFoundOnPrepareTransaction() { - final SpannerException sessionNotFound = - SpannerExceptionFactoryTest.newSessionNotFoundException(sessionName); - final SessionImpl closedSession = mock(SessionImpl.class); - when(closedSession.getName()) - .thenReturn("projects/dummy/instances/dummy/database/dummy/sessions/session-closed"); - when(closedSession.beginTransaction()).thenThrow(sessionNotFound); - doThrow(sessionNotFound).when(closedSession).prepareReadWriteTransaction(); - - final SessionImpl openSession = mock(SessionImpl.class); - when(openSession.getName()) - .thenReturn("projects/dummy/instances/dummy/database/dummy/sessions/session-open"); - doAnswer( - new Answer() { - @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Runnable() { - @Override - public void run() { - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - consumer.onSessionReady(closedSession); - } - }); - return null; - } - }) - .doAnswer( - new Answer() { - @Override - public Void answer(final InvocationOnMock invocation) { - executor.submit( - new Runnable() { - @Override - public void run() { - SessionConsumerImpl consumer = - invocation.getArgumentAt(2, SessionConsumerImpl.class); - consumer.onSessionReady(openSession); - } - }); - return null; - } - }) - .when(sessionClient) - .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); - FakeClock clock = new FakeClock(); - clock.currentTimeMillis = System.currentTimeMillis(); - pool = createPool(clock); - PooledSession session = pool.getReadWriteSession().get(); - assertThat(session.delegate).isEqualTo(openSession); - } - @Test public void testSessionNotFoundWrite() { SpannerException sessionNotFound = @@ -1623,7 +1083,7 @@ public void run() { FakeClock clock = new FakeClock(); clock.currentTimeMillis = System.currentTimeMillis(); pool = createPool(clock); - DatabaseClientImpl impl = new DatabaseClientImpl(pool, false); + DatabaseClientImpl impl = new DatabaseClientImpl(pool); assertThat(impl.write(mutations)).isNotNull(); } @@ -1674,7 +1134,7 @@ public void run() { FakeClock clock = new FakeClock(); clock.currentTimeMillis = System.currentTimeMillis(); pool = createPool(clock); - DatabaseClientImpl impl = new DatabaseClientImpl(pool, false); + DatabaseClientImpl impl = new DatabaseClientImpl(pool); assertThat(impl.writeAtLeastOnce(mutations)).isNotNull(); } @@ -1725,7 +1185,7 @@ public void run() { FakeClock clock = new FakeClock(); clock.currentTimeMillis = System.currentTimeMillis(); pool = createPool(clock); - DatabaseClientImpl impl = new DatabaseClientImpl(pool, false); + DatabaseClientImpl impl = new DatabaseClientImpl(pool); assertThat(impl.executePartitionedUpdate(statement)).isEqualTo(1L); } @@ -1751,8 +1211,8 @@ public void testSessionMetrics() throws Exception { setupMockSessionCreation(); pool = createPool(clock, metricRegistry, labelValues); - PooledSessionFuture session1 = pool.getReadSession(); - PooledSessionFuture session2 = pool.getReadSession(); + PooledSessionFuture session1 = pool.get(); + PooledSessionFuture session2 = pool.get(); session1.get(); session2.get(); @@ -1830,7 +1290,7 @@ public void testSessionMetrics() throws Exception { @Override public Void call() { latch.countDown(); - Session session = pool.getReadSession(); + Session session = pool.get(); session.close(); return null; } @@ -1888,7 +1348,7 @@ private void getSessionAsync(final CountDownLatch latch, final AtomicBoolean fai new Runnable() { @Override public void run() { - try (PooledSessionFuture future = pool.getReadSession()) { + try (PooledSessionFuture future = pool.get()) { PooledSession session = future.get(); failed.compareAndSet(false, session == null); Uninterruptibles.sleepUninterruptibly(10, TimeUnit.MILLISECONDS); @@ -1901,23 +1361,4 @@ public void run() { }) .start(); } - - private void getReadWriteSessionAsync(final CountDownLatch latch, final AtomicBoolean failed) { - new Thread( - new Runnable() { - @Override - public void run() { - try (PooledSessionFuture future = pool.getReadWriteSession()) { - PooledSession session = future.get(); - failed.compareAndSet(false, session == null); - Uninterruptibles.sleepUninterruptibly(2, TimeUnit.MILLISECONDS); - } catch (SpannerException e) { - failed.compareAndSet(false, true); - } finally { - latch.countDown(); - } - } - }) - .start(); - } } diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SpanTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SpanTest.java index 7dcc9b65e1..75552c52e1 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SpanTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SpanTest.java @@ -96,22 +96,6 @@ public class SpanTest { private static final SimulatedExecutionTime ONE_SECOND = SimulatedExecutionTime.ofMinimumAndRandomTime(1000, 0); - private static final Statement SELECT1AND2 = - Statement.of("SELECT 1 AS COL1 UNION ALL SELECT 2 AS COL1"); - private static final ResultSetMetadata SELECT1AND2_METADATA = - ResultSetMetadata.newBuilder() - .setRowType( - StructType.newBuilder() - .addFields( - Field.newBuilder() - .setName("COL1") - .setType( - com.google.spanner.v1.Type.newBuilder() - .setCode(TypeCode.INT64) - .build()) - .build()) - .build()) - .build(); private static final StatusRuntimeException FAILED_PRECONDITION = io.grpc.Status.FAILED_PRECONDITION .withDescription("Non-retryable test exception.") @@ -162,11 +146,7 @@ public void setUp() throws Exception { .setProjectId(TEST_PROJECT) .setChannelProvider(channelProvider) .setCredentials(NoCredentials.getInstance()) - .setSessionPoolOption( - SessionPoolOptions.newBuilder() - .setMinSessions(0) - .setWriteSessionsFraction(0.0f) - .build()); + .setSessionPoolOption(SessionPoolOptions.newBuilder().setMinSessions(0).build()); spanner = builder.build().getService(); @@ -227,7 +207,7 @@ public void tearDown() { @Test public void singleUseNonRetryableErrorOnNext() { - try (ResultSet rs = client.singleUse().executeQuery(SELECT1AND2)) { + try (ResultSet rs = client.singleUse().executeQuery(SELECT1)) { mockSpanner.addException(FAILED_PRECONDITION); while (rs.next()) { // Just consume the result set. @@ -241,7 +221,7 @@ public void singleUseNonRetryableErrorOnNext() { @Test public void singleUseExecuteStreamingSqlTimeout() { - try (ResultSet rs = clientWithTimeout.singleUse().executeQuery(SELECT1AND2)) { + try (ResultSet rs = clientWithTimeout.singleUse().executeQuery(SELECT1)) { mockSpanner.setExecuteStreamingSqlExecutionTime(ONE_SECOND); while (rs.next()) { // Just consume the result set. @@ -302,7 +282,6 @@ public Void run(TransactionContext transaction) { assertThat(spans).containsEntry("CloudSpannerOperation.BatchCreateSessions", true); assertThat(spans).containsEntry("SessionPool.WaitForSession", true); assertThat(spans).containsEntry("CloudSpannerOperation.BatchCreateSessionsRequest", true); - assertThat(spans).containsEntry("CloudSpannerOperation.BeginTransaction", true); assertThat(spans).containsEntry("CloudSpannerOperation.Commit", true); } @@ -324,11 +303,10 @@ public Void run(TransactionContext transaction) { } Map spans = failOnOverkillTraceComponent.getSpans(); - assertThat(spans.size()).isEqualTo(5); + assertThat(spans.size()).isEqualTo(4); assertThat(spans).containsEntry("CloudSpanner.ReadWriteTransaction", true); assertThat(spans).containsEntry("CloudSpannerOperation.BatchCreateSessions", true); assertThat(spans).containsEntry("SessionPool.WaitForSession", true); assertThat(spans).containsEntry("CloudSpannerOperation.BatchCreateSessionsRequest", true); - assertThat(spans).containsEntry("CloudSpannerOperation.BeginTransaction", true); } } diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SpannerGaxRetryTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SpannerGaxRetryTest.java index b98702f87c..cda4cf5f8f 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SpannerGaxRetryTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SpannerGaxRetryTest.java @@ -360,7 +360,7 @@ public void readWriteTransactionStatementAborted() { @Override public Long run(TransactionContext transaction) { if (attempts.getAndIncrement() == 0) { - mockSpanner.abortTransaction(transaction); + mockSpanner.abortNextStatement(); } return transaction.executeUpdate(UPDATE_STATEMENT); } @@ -418,7 +418,7 @@ public Long run(TransactionContext transaction) { @SuppressWarnings("resource") @Test public void transactionManagerTimeout() { - mockSpanner.setBeginTransactionExecutionTime(ONE_SECOND); + mockSpanner.setExecuteSqlExecutionTime(ONE_SECOND); try (TransactionManager txManager = clientWithTimeout.transactionManager()) { TransactionContext tx = txManager.begin(); while (true) { diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/TransactionManagerAbortedTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/TransactionManagerAbortedTest.java index dec674bd6c..0291e67868 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/TransactionManagerAbortedTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/TransactionManagerAbortedTest.java @@ -194,7 +194,7 @@ public void testTransactionManagerAbortOnCommit() throws InterruptedException { attempts++; try { if (attempts == 1) { - mockSpanner.abortAllTransactions(); + mockSpanner.abortNextTransaction(); } manager.commit(); break; @@ -219,7 +219,7 @@ public void testTransactionManagerAbortOnUpdate() throws InterruptedException { attempts++; try { if (attempts == 1) { - mockSpanner.abortAllTransactions(); + mockSpanner.abortNextTransaction(); } long updateCount = txn.executeUpdate(UPDATE_STATEMENT); assertThat(updateCount, is(equalTo(UPDATE_COUNT))); @@ -246,7 +246,7 @@ public void testTransactionManagerAbortOnBatchUpdate() throws InterruptedExcepti attempts++; try { if (attempts == 1) { - mockSpanner.abortAllTransactions(); + mockSpanner.abortNextTransaction(); } long[] updateCounts = txn.batchUpdate(Arrays.asList(UPDATE_STATEMENT, UPDATE_STATEMENT)); assertThat(updateCounts, is(equalTo(new long[] {UPDATE_COUNT, UPDATE_COUNT}))); @@ -301,7 +301,7 @@ public void testTransactionManagerAbortOnSelect() throws InterruptedException { attempts++; try { if (attempts == 1) { - mockSpanner.abortAllTransactions(); + mockSpanner.abortNextTransaction(); } try (ResultSet rs = txn.executeQuery(SELECT1AND2)) { int rows = 0; @@ -333,7 +333,7 @@ public void testTransactionManagerAbortOnRead() throws InterruptedException { attempts++; try { if (attempts == 1) { - mockSpanner.abortAllTransactions(); + mockSpanner.abortNextTransaction(); } try (ResultSet rs = txn.read("FOO", KeySet.all(), Arrays.asList("BAR"))) { int rows = 0; @@ -365,7 +365,7 @@ public void testTransactionManagerAbortOnReadUsingIndex() throws InterruptedExce attempts++; try { if (attempts == 1) { - mockSpanner.abortAllTransactions(); + mockSpanner.abortNextTransaction(); } try (ResultSet rs = txn.readUsingIndex("FOO", "INDEX", KeySet.all(), Arrays.asList("BAR"))) { @@ -398,7 +398,7 @@ public void testTransactionManagerAbortOnReadRow() throws InterruptedException { attempts++; try { if (attempts == 1) { - mockSpanner.abortAllTransactions(); + mockSpanner.abortNextTransaction(); } Struct row = txn.readRow("FOO", Key.of(), Arrays.asList("BAR")); assertThat(row.getLong(0), is(equalTo(1L))); @@ -425,7 +425,7 @@ public void testTransactionManagerAbortOnReadRowUsingIndex() throws InterruptedE attempts++; try { if (attempts == 1) { - mockSpanner.abortAllTransactions(); + mockSpanner.abortNextTransaction(); } Struct row = txn.readRowUsingIndex("FOO", "INDEX", Key.of(), Arrays.asList("BAR")); assertThat(row.getLong(0), is(equalTo(1L))); diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/TransactionManagerImplTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/TransactionManagerImplTest.java index 340d24c55e..149002531a 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/TransactionManagerImplTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/TransactionManagerImplTest.java @@ -82,7 +82,7 @@ public void release(ScheduledExecutorService exec) { @Before public void setUp() { initMocks(this); - manager = new TransactionManagerImpl(session, mock(Span.class), false); + manager = new TransactionManagerImpl(session, mock(Span.class)); } @Test @@ -301,7 +301,6 @@ public void inlineBegin() { SessionPoolOptions sessionPoolOptions = SessionPoolOptions.newBuilder().setMinSessions(0).setIncStep(1).build(); when(options.getSessionPoolOptions()).thenReturn(sessionPoolOptions); - when(options.isInlineBeginForReadWriteTransaction()).thenReturn(true); when(options.getSessionLabels()).thenReturn(Collections.emptyMap()); when(options.getDefaultQueryOptions(Mockito.any(DatabaseId.class))) .thenReturn(QueryOptions.getDefaultInstance()); diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/TransactionRunnerImplTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/TransactionRunnerImplTest.java index 8b66ecb959..71a34950bb 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/TransactionRunnerImplTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/TransactionRunnerImplTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.fail; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -122,7 +123,7 @@ public ResultSet answer(InvocationOnMock invocation) throws Throwable { return builder.build(); } }); - transactionRunner = new TransactionRunnerImpl(session, rpc, 1, false); + transactionRunner = new TransactionRunnerImpl(session, rpc, 1); when(rpc.commitAsync(Mockito.any(CommitRequest.class), Mockito.anyMap())) .thenReturn( ApiFutures.immediateFuture( @@ -216,7 +217,7 @@ public Void run(TransactionContext transaction) { } }); assertThat(numCalls.get()).isEqualTo(1); - verify(txn).ensureTxn(); + verify(txn, never()).ensureTxn(); verify(txn).commit(); } @@ -224,7 +225,7 @@ public Void run(TransactionContext transaction) { public void runAbort() { when(txn.isAborted()).thenReturn(true); runTransaction(abortedWithRetryInfo()); - verify(txn, times(2)).ensureTxn(); + verify(txn).ensureTxn(); } @Test @@ -242,7 +243,8 @@ public Void run(TransactionContext transaction) { } }); assertThat(numCalls.get()).isEqualTo(2); - verify(txn, times(2)).ensureTxn(); + // ensureTxn() is only called during retry. + verify(txn).ensureTxn(); } @Test @@ -266,7 +268,7 @@ public Void run(TransactionContext transaction) { assertThat(e.getErrorCode()).isEqualTo(ErrorCode.UNKNOWN); } assertThat(numCalls.get()).isEqualTo(1); - verify(txn, times(1)).ensureTxn(); + verify(txn, never()).ensureTxn(); verify(txn, times(1)).commit(); } @@ -320,9 +322,7 @@ public void prepareReadWriteTransaction() { } }; session.setCurrentSpan(mock(Span.class)); - // Create a transaction runner that will inline the BeginTransaction call with the first - // statement. - TransactionRunnerImpl runner = new TransactionRunnerImpl(session, rpc, 10, true); + TransactionRunnerImpl runner = new TransactionRunnerImpl(session, rpc, 10); runner.setSpan(mock(Span.class)); assertThat(usedInlinedBegin).isFalse(); runner.run( @@ -354,7 +354,7 @@ private long[] batchDmlException(int status) { .thenReturn( ApiFutures.immediateFuture(ByteString.copyFromUtf8(UUID.randomUUID().toString()))); when(session.getName()).thenReturn(SessionId.of("p", "i", "d", "test").getName()); - TransactionRunnerImpl runner = new TransactionRunnerImpl(session, rpc, 10, false); + TransactionRunnerImpl runner = new TransactionRunnerImpl(session, rpc, 10); runner.setSpan(mock(Span.class)); ExecuteBatchDmlResponse response1 = ExecuteBatchDmlResponse.newBuilder() diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITDMLTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITDMLTest.java index b8a4ae81ae..04a6cbd539 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITDMLTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITDMLTest.java @@ -39,7 +39,6 @@ import com.google.cloud.spanner.TransactionRunner; import com.google.cloud.spanner.TransactionRunner.TransactionCallable; import java.util.Arrays; -import java.util.Collection; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; @@ -47,13 +46,11 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameter; -import org.junit.runners.Parameterized.Parameters; +import org.junit.runners.JUnit4; /** Integration tests for DML. */ @Category(ParallelIntegrationTest.class) -@RunWith(Parameterized.class) +@RunWith(JUnit4.class) public final class ITDMLTest { @ClassRule public static IntegrationTestEnv env = new IntegrationTestEnv(); private static Database db; @@ -71,12 +68,6 @@ public final class ITDMLTest { private static boolean throwAbortOnce = false; - @Parameters(name = "InlineBeginTx = {0}") - public static Collection data() { - return Arrays.asList(new Object[][] {{false}, {true}}); - } - - @Parameter public boolean inlineBeginTx; private Spanner spanner; private DatabaseClient client; @@ -93,13 +84,7 @@ public static void setUpDatabase() { @Before public void setupClient() { - spanner = - env.getTestHelper() - .getOptions() - .toBuilder() - .setInlineBeginForReadWriteTransaction(inlineBeginTx) - .build() - .getService(); + spanner = env.getTestHelper().getClient(); client = spanner.getDatabaseClient(db.getId()); client.writeAtLeastOnce(Arrays.asList(Mutation.delete("T", KeySet.all()))); id++; diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionManagerAsyncTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionManagerAsyncTest.java index 684803ae54..7b4f340f0b 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionManagerAsyncTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionManagerAsyncTest.java @@ -64,19 +64,13 @@ public class ITTransactionManagerAsyncTest { @Parameter(0) public Executor executor; - @Parameter(1) - public boolean inlineBegin; - - @Parameters(name = "executor = {0}, inlineBegin = {1}") + @Parameters(name = "executor = {0}") public static Collection data() { return Arrays.asList( new Object[][] { - {MoreExecutors.directExecutor(), false}, - {MoreExecutors.directExecutor(), true}, - {Executors.newSingleThreadExecutor(), false}, - {Executors.newSingleThreadExecutor(), true}, - {Executors.newFixedThreadPool(4), false}, - {Executors.newFixedThreadPool(4), true} + {MoreExecutors.directExecutor()}, + {Executors.newSingleThreadExecutor()}, + {Executors.newFixedThreadPool(4)}, }); } @@ -99,13 +93,7 @@ public static void setUpDatabase() { @Before public void clearTable() { - spanner = - env.getTestHelper() - .getOptions() - .toBuilder() - .setInlineBeginForReadWriteTransaction(inlineBegin) - .build() - .getService(); + spanner = env.getTestHelper().getClient(); client = spanner.getDatabaseClient(db.getId()); client.write(ImmutableList.of(Mutation.delete("T", KeySet.all()))); } diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionManagerTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionManagerTest.java index 0898547ad1..28d785d9df 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionManagerTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionManagerTest.java @@ -38,7 +38,6 @@ import com.google.cloud.spanner.TransactionManager.TransactionState; import com.google.common.collect.ImmutableList; import java.util.Arrays; -import java.util.Collection; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; @@ -46,22 +45,12 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameter; -import org.junit.runners.Parameterized.Parameters; +import org.junit.runners.JUnit4; @Category(ParallelIntegrationTest.class) -@RunWith(Parameterized.class) +@RunWith(JUnit4.class) public class ITTransactionManagerTest { - @Parameter(0) - public boolean inlineBegin; - - @Parameters(name = "inlineBegin = {0}") - public static Collection data() { - return Arrays.asList(new Object[][] {{false}, {true}}); - } - @ClassRule public static IntegrationTestEnv env = new IntegrationTestEnv(); private static Database db; private Spanner spanner; @@ -81,13 +70,7 @@ public static void setUpDatabase() { @Before public void setupClient() { - spanner = - env.getTestHelper() - .getOptions() - .toBuilder() - .setInlineBeginForReadWriteTransaction(inlineBegin) - .build() - .getService(); + spanner = env.getTestHelper().getClient(); client = spanner.getDatabaseClient(db.getId()); client.write(ImmutableList.of(Mutation.delete("T", KeySet.all()))); } diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionTest.java index 1d7a3e0c4f..fba7694cd8 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionTest.java @@ -52,7 +52,6 @@ import com.google.common.util.concurrent.Uninterruptibles; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.List; import java.util.Vector; import java.util.concurrent.CountDownLatch; @@ -64,23 +63,13 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameter; -import org.junit.runners.Parameterized.Parameters; +import org.junit.runners.JUnit4; /** Integration tests for read-write transactions. */ @Category(ParallelIntegrationTest.class) -@RunWith(Parameterized.class) +@RunWith(JUnit4.class) public class ITTransactionTest { - @Parameter(0) - public boolean inlineBegin; - - @Parameters(name = "inlineBegin = {0}") - public static Collection data() { - return Arrays.asList(new Object[][] {{false}, {true}}); - } - @ClassRule public static IntegrationTestEnv env = new IntegrationTestEnv(); private static Database db; private Spanner spanner; @@ -101,13 +90,7 @@ public static void setUpDatabase() { @Before public void setupClient() { - spanner = - env.getTestHelper() - .getOptions() - .toBuilder() - .setInlineBeginForReadWriteTransaction(inlineBegin) - .build() - .getService(); + spanner = env.getTestHelper().getClient(); client = spanner.getDatabaseClient(db.getId()); } From 9f71bd3e3f30675a25cae85afc3b9a4b8c1f831d Mon Sep 17 00:00:00 2001 From: Olav Loite Date: Thu, 15 Oct 2020 10:49:26 +0200 Subject: [PATCH 2/4] fix: fix integration tests --- .../google/cloud/spanner/it/ITDMLTest.java | 16 +++------------ .../spanner/it/ITTransactionManagerTest.java | 15 +++----------- .../cloud/spanner/it/ITTransactionTest.java | 20 ++----------------- 3 files changed, 8 insertions(+), 43 deletions(-) diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITDMLTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITDMLTest.java index 04a6cbd539..915efa604e 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITDMLTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITDMLTest.java @@ -30,7 +30,6 @@ import com.google.cloud.spanner.Mutation; import com.google.cloud.spanner.ParallelIntegrationTest; import com.google.cloud.spanner.ResultSet; -import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerException; import com.google.cloud.spanner.SpannerExceptionFactory; import com.google.cloud.spanner.Statement; @@ -39,7 +38,6 @@ import com.google.cloud.spanner.TransactionRunner; import com.google.cloud.spanner.TransactionRunner.TransactionCallable; import java.util.Arrays; -import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -54,6 +52,7 @@ public final class ITDMLTest { @ClassRule public static IntegrationTestEnv env = new IntegrationTestEnv(); private static Database db; + private static DatabaseClient client; /** Sequence for assigning unique keys to test cases. */ private static int seq; @@ -68,9 +67,6 @@ public final class ITDMLTest { private static boolean throwAbortOnce = false; - private Spanner spanner; - private DatabaseClient client; - @BeforeClass public static void setUpDatabase() { db = @@ -80,21 +76,15 @@ public static void setUpDatabase() { + " K STRING(MAX) NOT NULL," + " V INT64," + ") PRIMARY KEY (K)"); + client = env.getTestHelper().getDatabaseClient(db); } @Before - public void setupClient() { - spanner = env.getTestHelper().getClient(); - client = spanner.getDatabaseClient(db.getId()); + public void increaseTestIdAndDeleteTestData() { client.writeAtLeastOnce(Arrays.asList(Mutation.delete("T", KeySet.all()))); id++; } - @After - public void teardownClient() { - spanner.close(); - } - private static String uniqueKey() { return "k" + seq++; } diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionManagerTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionManagerTest.java index 28d785d9df..4d65af67ed 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionManagerTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionManagerTest.java @@ -30,7 +30,6 @@ import com.google.cloud.spanner.KeySet; import com.google.cloud.spanner.Mutation; import com.google.cloud.spanner.ParallelIntegrationTest; -import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerException; import com.google.cloud.spanner.Struct; import com.google.cloud.spanner.TransactionContext; @@ -38,7 +37,6 @@ import com.google.cloud.spanner.TransactionManager.TransactionState; import com.google.common.collect.ImmutableList; import java.util.Arrays; -import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -53,8 +51,7 @@ public class ITTransactionManagerTest { @ClassRule public static IntegrationTestEnv env = new IntegrationTestEnv(); private static Database db; - private Spanner spanner; - private DatabaseClient client; + private static DatabaseClient client; @BeforeClass public static void setUpDatabase() { @@ -66,20 +63,14 @@ public static void setUpDatabase() { + " K STRING(MAX) NOT NULL," + " BoolValue BOOL," + ") PRIMARY KEY (K)"); + client = env.getTestHelper().getDatabaseClient(db); } @Before - public void setupClient() { - spanner = env.getTestHelper().getClient(); - client = spanner.getDatabaseClient(db.getId()); + public void deleteTestData() { client.write(ImmutableList.of(Mutation.delete("T", KeySet.all()))); } - @After - public void closeClient() { - spanner.close(); - } - @SuppressWarnings("resource") @Test public void simpleInsert() throws InterruptedException { diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionTest.java index fba7694cd8..1bfae5f7d6 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITTransactionTest.java @@ -38,7 +38,6 @@ import com.google.cloud.spanner.ReadContext; import com.google.cloud.spanner.ReadOnlyTransaction; import com.google.cloud.spanner.ResultSet; -import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerException; import com.google.cloud.spanner.Statement; import com.google.cloud.spanner.Struct; @@ -56,8 +55,6 @@ import java.util.Vector; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import org.junit.After; -import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; @@ -69,11 +66,9 @@ @Category(ParallelIntegrationTest.class) @RunWith(JUnit4.class) public class ITTransactionTest { - @ClassRule public static IntegrationTestEnv env = new IntegrationTestEnv(); private static Database db; - private Spanner spanner; - private DatabaseClient client; + private static DatabaseClient client; /** Sequence for assigning unique keys to test cases. */ private static int seq; @@ -86,18 +81,7 @@ public static void setUpDatabase() { + " K STRING(MAX) NOT NULL," + " V INT64," + ") PRIMARY KEY (K)"); - } - - @Before - public void setupClient() { - spanner = env.getTestHelper().getClient(); - client = spanner.getDatabaseClient(db.getId()); - } - - @After - public void closeClient() { - client.writeAtLeastOnce(ImmutableList.of(Mutation.delete("T", KeySet.all()))); - spanner.close(); + client = env.getTestHelper().getDatabaseClient(db); } private static String uniqueKey() { From 26a73b00e1f21bd352327cd5e57ad2350cd349bc Mon Sep 17 00:00:00 2001 From: Olav Loite Date: Thu, 15 Oct 2020 15:10:02 +0200 Subject: [PATCH 3/4] test: fix malformed retry loop in test case --- .../cloud/spanner/it/ITClosedSessionTest.java | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITClosedSessionTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITClosedSessionTest.java index 1e00015cdf..22dc4c5c45 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITClosedSessionTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/it/ITClosedSessionTest.java @@ -245,21 +245,21 @@ public void testTransactionManager() throws InterruptedException { for (int run = 0; run < 2; run++) { try (TransactionManager manager = client.transactionManager()) { TransactionContext txn = manager.begin(); - while (true) { - for (int i = 0; i < 2; i++) { - try (ResultSet rs = txn.executeQuery(Statement.of("SELECT 1"))) { - assertThat(rs.next()).isTrue(); - assertThat(rs.getLong(0)).isEqualTo(1L); - assertThat(rs.next()).isFalse(); + try { + while (true) { + for (int i = 0; i < 2; i++) { + try (ResultSet rs = txn.executeQuery(Statement.of("SELECT 1"))) { + assertThat(rs.next()).isTrue(); + assertThat(rs.getLong(0)).isEqualTo(1L); + assertThat(rs.next()).isFalse(); + } } - } - try { manager.commit(); break; - } catch (AbortedException e) { - Thread.sleep(e.getRetryDelayInMillis() / 1000); - txn = manager.resetForRetry(); } + } catch (AbortedException e) { + Thread.sleep(e.getRetryDelayInMillis() / 1000); + txn = manager.resetForRetry(); } } } From c547f865168dd1f313e871295c3a3d4332240555 Mon Sep 17 00:00:00 2001 From: Olav Loite Date: Tue, 20 Oct 2020 11:55:05 +0200 Subject: [PATCH 4/4] fix: review comments --- .../cloud/spanner/DatabaseClientImpl.java | 26 +++---- .../spanner/MetricRegistryConstants.java | 8 +++ .../com/google/cloud/spanner/SessionPool.java | 16 ++--- .../cloud/spanner/SessionPoolOptions.java | 14 +++- .../cloud/spanner/TransactionRunnerImpl.java | 8 +-- .../spanner/ITSessionPoolIntegrationTest.java | 22 +++--- .../IntegrationTestWithClosedSessionsEnv.java | 4 +- .../spanner/SessionPoolMaintainerTest.java | 40 +++++------ .../cloud/spanner/SessionPoolStressTest.java | 2 +- .../google/cloud/spanner/SessionPoolTest.java | 72 +++++++++---------- 10 files changed, 115 insertions(+), 97 deletions(-) diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/DatabaseClientImpl.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/DatabaseClientImpl.java index bc0e341642..fa2fa9d971 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/DatabaseClientImpl.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/DatabaseClientImpl.java @@ -47,8 +47,8 @@ class DatabaseClientImpl implements DatabaseClient { } @VisibleForTesting - PooledSessionFuture getReadSession() { - return pool.get(); + PooledSessionFuture getSession() { + return pool.getSession(); } @Override @@ -93,7 +93,7 @@ public Timestamp apply(Session session) { public ReadContext singleUse() { Span span = tracer.spanBuilder(READ_ONLY_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { - return getReadSession().singleUse(); + return getSession().singleUse(); } catch (RuntimeException e) { TraceUtil.endSpanWithFailure(span, e); throw e; @@ -104,7 +104,7 @@ public ReadContext singleUse() { public ReadContext singleUse(TimestampBound bound) { Span span = tracer.spanBuilder(READ_ONLY_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { - return getReadSession().singleUse(bound); + return getSession().singleUse(bound); } catch (RuntimeException e) { TraceUtil.endSpanWithFailure(span, e); throw e; @@ -115,7 +115,7 @@ public ReadContext singleUse(TimestampBound bound) { public ReadOnlyTransaction singleUseReadOnlyTransaction() { Span span = tracer.spanBuilder(READ_ONLY_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { - return getReadSession().singleUseReadOnlyTransaction(); + return getSession().singleUseReadOnlyTransaction(); } catch (RuntimeException e) { TraceUtil.endSpanWithFailure(span, e); throw e; @@ -126,7 +126,7 @@ public ReadOnlyTransaction singleUseReadOnlyTransaction() { public ReadOnlyTransaction singleUseReadOnlyTransaction(TimestampBound bound) { Span span = tracer.spanBuilder(READ_ONLY_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { - return getReadSession().singleUseReadOnlyTransaction(bound); + return getSession().singleUseReadOnlyTransaction(bound); } catch (RuntimeException e) { TraceUtil.endSpanWithFailure(span, e); throw e; @@ -137,7 +137,7 @@ public ReadOnlyTransaction singleUseReadOnlyTransaction(TimestampBound bound) { public ReadOnlyTransaction readOnlyTransaction() { Span span = tracer.spanBuilder(READ_ONLY_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { - return getReadSession().readOnlyTransaction(); + return getSession().readOnlyTransaction(); } catch (RuntimeException e) { TraceUtil.endSpanWithFailure(span, e); throw e; @@ -148,7 +148,7 @@ public ReadOnlyTransaction readOnlyTransaction() { public ReadOnlyTransaction readOnlyTransaction(TimestampBound bound) { Span span = tracer.spanBuilder(READ_ONLY_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { - return getReadSession().readOnlyTransaction(bound); + return getSession().readOnlyTransaction(bound); } catch (RuntimeException e) { TraceUtil.endSpanWithFailure(span, e); throw e; @@ -159,7 +159,7 @@ public ReadOnlyTransaction readOnlyTransaction(TimestampBound bound) { public TransactionRunner readWriteTransaction() { Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { - return getReadSession().readWriteTransaction(); + return getSession().readWriteTransaction(); } catch (RuntimeException e) { TraceUtil.endSpanWithFailure(span, e); throw e; @@ -172,7 +172,7 @@ public TransactionRunner readWriteTransaction() { public TransactionManager transactionManager() { Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { - return getReadSession().transactionManager(); + return getSession().transactionManager(); } catch (RuntimeException e) { TraceUtil.endSpanWithFailure(span, e); throw e; @@ -183,7 +183,7 @@ public TransactionManager transactionManager() { public AsyncRunner runAsync() { Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { - return getReadSession().runAsync(); + return getSession().runAsync(); } catch (RuntimeException e) { TraceUtil.endSpanWithFailure(span, e); throw e; @@ -194,7 +194,7 @@ public AsyncRunner runAsync() { public AsyncTransactionManager transactionManagerAsync() { Span span = tracer.spanBuilder(READ_WRITE_TRANSACTION).startSpan(); try (Scope s = tracer.withSpan(span)) { - return getReadSession().transactionManagerAsync(); + return getSession().transactionManagerAsync(); } catch (RuntimeException e) { TraceUtil.endSpanWithFailure(span, e); throw e; @@ -219,7 +219,7 @@ public Long apply(Session session) { } private T runWithSessionRetry(Function callable) { - PooledSessionFuture session = getReadSession(); + PooledSessionFuture session = getSession(); while (true) { try { return callable.apply(session); diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/MetricRegistryConstants.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/MetricRegistryConstants.java index 392fda729e..3512a75732 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/MetricRegistryConstants.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/MetricRegistryConstants.java @@ -37,12 +37,20 @@ class MetricRegistryConstants { static final LabelValue NUM_IN_USE_SESSIONS = LabelValue.create("num_in_use_sessions"); + /** + * The session pool no longer prepares a fraction of the sessions with a read/write transaction. + * This metric will therefore always be zero and may be removed in the future. + */ @Deprecated static final LabelValue NUM_SESSIONS_BEING_PREPARED = LabelValue.create("num_sessions_being_prepared"); static final LabelValue NUM_READ_SESSIONS = LabelValue.create("num_read_sessions"); + /** + * The session pool no longer prepares a fraction of the sessions with a read/write transaction. + * This metric will therefore always be zero and may be removed in the future. + */ @Deprecated static final LabelValue NUM_WRITE_SESSIONS = LabelValue.create("num_write_prepared_sessions"); diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPool.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPool.java index 5707bae0fb..5e8788e166 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPool.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPool.java @@ -1845,8 +1845,8 @@ boolean isValid() { } /** - * Returns a session to be used for requests to spanner. It will block if a session is not - * currently available. In case the pool is exhausted and {@link + * Returns a session to be used for requests to spanner. This method is always non-blocking and + * returns a {@link PooledSessionFuture}. In case the pool is exhausted and {@link * SessionPoolOptions#isFailIfPoolExhausted()} has been set, it will throw an exception. Returned * session must be closed by calling {@link Session#close()}. * @@ -1859,7 +1859,7 @@ boolean isValid() { * session being returned to the pool or a new session being created. * */ - PooledSessionFuture get() throws SpannerException { + PooledSessionFuture getSession() throws SpannerException { Span span = Tracing.getTracer().getCurrentSpan(); span.addAnnotation("Acquiring session"); WaiterFuture waiter = null; @@ -1885,7 +1885,7 @@ PooledSessionFuture get() throws SpannerException { waiter = new WaiterFuture(); waiters.add(waiter); } else { - span.addAnnotation("Acquired rsession"); + span.addAnnotation("Acquired session"); } return checkoutSession(span, sess, waiter); } @@ -1919,7 +1919,7 @@ PooledSessionFuture replaceSession(SessionNotFoundException e, PooledSessionFutu } session.leakedException = null; invalidateSession(session.get()); - return get(); + return getSession(); } else { throw e; } @@ -2013,9 +2013,9 @@ private void decrementPendingClosures(int count) { } /** - * Close all the sessions. Once this method is invoked {@link #get()} will start throwing {@code - * IllegalStateException}. The returned future blocks till all the sessions created in this pool - * have been closed. + * Close all the sessions. Once this method is invoked {@link #getSession()} will start throwing + * {@code IllegalStateException}. The returned future blocks till all the sessions created in this + * pool have been closed. */ ListenableFuture closeAsync(ClosedException closedException) { ListenableFuture retFuture = null; diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPoolOptions.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPoolOptions.java index 591a696d36..2c68fd317e 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPoolOptions.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/SessionPoolOptions.java @@ -31,7 +31,12 @@ public class SessionPoolOptions { private final int maxSessions; private final int incStep; private final int maxIdleSessions; + /** + * The session pool no longer prepares a fraction of the sessions with a read/write transaction. + * This setting therefore does not have any meaning anymore, and may be removed in the future. + */ @Deprecated private final float writeSessionsFraction; + private final ActionOnExhaustion actionOnExhaustion; private final long loopFrequency; private final int keepAliveIntervalMinutes; @@ -77,7 +82,8 @@ public int getMaxIdleSessions() { /** * @deprecated This value is no longer used. The session pool does not prepare any sessions for * read/write transactions. Instead, a transaction will be started by including a - * BeginTransaction option with the first statement of a transaction. + * BeginTransaction option with the first statement of a transaction. This method may be + * removed in a future release. */ @Deprecated public float getWriteSessionsFraction() { @@ -145,7 +151,12 @@ public static class Builder { private int maxSessions = DEFAULT_MAX_SESSIONS; private int incStep = DEFAULT_INC_STEP; private int maxIdleSessions; + /** + * The session pool no longer prepares a fraction of the sessions with a read/write transaction. + * This setting therefore does not have any meaning anymore, and may be removed in the future. + */ @Deprecated private float writeSessionsFraction = 0.2f; + private ActionOnExhaustion actionOnExhaustion = DEFAULT_ACTION; private long initialWaitForSessionTimeoutMillis = 30_000L; private ActionOnSessionNotFound actionOnSessionNotFound = ActionOnSessionNotFound.RETRY; @@ -270,6 +281,7 @@ Builder setFailOnSessionLeak() { * any sessions for read/write transactions. Instead, a transaction will automatically be * started by the first statement that is executed by a transaction by including a * BeginTransaction option with that statement. + *

This method may be removed in a future release. */ public Builder setWriteSessionsFraction(float writeSessionsFraction) { this.writeSessionsFraction = writeSessionsFraction; diff --git a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/TransactionRunnerImpl.java b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/TransactionRunnerImpl.java index bcfabe5329..b825ea7a03 100644 --- a/google-cloud-spanner/src/main/java/com/google/cloud/spanner/TransactionRunnerImpl.java +++ b/google-cloud-spanner/src/main/java/com/google/cloud/spanner/TransactionRunnerImpl.java @@ -764,11 +764,9 @@ private T runInternal(final TransactionCallable txCallable) { public T call() { boolean useInlinedBegin = true; if (attempt.get() > 0) { - if (useInlinedBegin) { - // Do not inline the BeginTransaction during a retry if the initial attempt did not - // actually start a transaction. - useInlinedBegin = txn.transactionId != null; - } + // Do not inline the BeginTransaction during a retry if the initial attempt did not + // actually start a transaction. + useInlinedBegin = txn.transactionId != null; txn = session.newTransaction(); } checkState( diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/ITSessionPoolIntegrationTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/ITSessionPoolIntegrationTest.java index 73f2111019..548f88172f 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/ITSessionPoolIntegrationTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/ITSessionPoolIntegrationTest.java @@ -98,12 +98,12 @@ public ScheduledExecutorService get() { @Test public void sessionCreation() { - try (PooledSessionFuture session = pool.get()) { + try (PooledSessionFuture session = pool.getSession()) { assertThat(session.get()).isNotNull(); } - try (PooledSessionFuture session = pool.get(); - PooledSessionFuture session2 = pool.get()) { + try (PooledSessionFuture session = pool.getSession(); + PooledSessionFuture session2 = pool.getSession()) { assertThat(session.get()).isNotNull(); assertThat(session2.get()).isNotNull(); } @@ -111,14 +111,14 @@ public void sessionCreation() { @Test public void poolExhaustion() throws Exception { - Session session1 = pool.get().get(); - Session session2 = pool.get().get(); + Session session1 = pool.getSession().get(); + Session session2 = pool.getSession().get(); final CountDownLatch latch = new CountDownLatch(1); new Thread( new Runnable() { @Override public void run() { - try (Session session3 = pool.get().get()) { + try (Session session3 = pool.getSession().get()) { latch.countDown(); } } @@ -132,8 +132,8 @@ public void run() { @Test public void multipleWaiters() throws Exception { - Session session1 = pool.get().get(); - Session session2 = pool.get().get(); + Session session1 = pool.getSession().get(); + Session session2 = pool.getSession().get(); int numSessions = 5; final CountDownLatch latch = new CountDownLatch(numSessions); for (int i = 0; i < numSessions; i++) { @@ -141,7 +141,7 @@ public void multipleWaiters() throws Exception { new Runnable() { @Override public void run() { - try (Session session = pool.get().get()) { + try (Session session = pool.getSession().get()) { latch.countDown(); } } @@ -161,13 +161,13 @@ public void closeQuicklyDoesNotBlockIndefinitely() throws Exception { @Test public void closeAfterInitialCreateDoesNotBlockIndefinitely() throws Exception { - pool.get().close(); + pool.getSession().close(); pool.closeAsync(new SpannerImpl.ClosedException()).get(); } @Test public void closeWhenSessionsActiveFinishes() throws Exception { - pool.get().get(); + pool.getSession().get(); // This will log a warning that a session has been leaked, as the session that we retrieved in // the previous statement was never returned to the pool. pool.closeAsync(new SpannerImpl.ClosedException()).get(); diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/IntegrationTestWithClosedSessionsEnv.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/IntegrationTestWithClosedSessionsEnv.java index 8070f542db..84c7185e1f 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/IntegrationTestWithClosedSessionsEnv.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/IntegrationTestWithClosedSessionsEnv.java @@ -74,8 +74,8 @@ public void setAllowSessionReplacing(boolean allow) { } @Override - PooledSessionFuture getReadSession() { - PooledSessionFuture session = super.getReadSession(); + PooledSessionFuture getSession() { + PooledSessionFuture session = super.getSession(); if (invalidateNextSession) { session.get().delegate.close(); session.get().setAllowReplacing(false); diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolMaintainerTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolMaintainerTest.java index b5d35a4d89..0c965a5573 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolMaintainerTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolMaintainerTest.java @@ -150,8 +150,8 @@ public void testKeepAlive() throws Exception { // Checkout two sessions and do a maintenance loop. Still no sessions should be getting any // pings. - Session session1 = pool.get(); - Session session2 = pool.get(); + Session session1 = pool.getSession(); + Session session2 = pool.getSession(); runMaintainanceLoop(clock, pool, 1); assertThat(pingedSessions).isEmpty(); @@ -173,9 +173,9 @@ public void testKeepAlive() throws Exception { // Now check out three sessions so the pool will create an additional session. The pool will // only keep 2 sessions alive, as that is the setting for MinSessions. - Session session3 = pool.get(); - Session session4 = pool.get(); - Session session5 = pool.get(); + Session session3 = pool.getSession(); + Session session4 = pool.getSession(); + Session session5 = pool.getSession(); // Note that session2 was now the first session in the pool as it was the last to receive a // ping. assertThat(session3.getName()).isEqualTo(session2.getName()); @@ -192,7 +192,7 @@ public void testKeepAlive() throws Exception { // should cause only one session to get a ping. clock.currentTimeMillis += TimeUnit.MINUTES.toMillis(options.getKeepAliveIntervalMinutes()) + 1; // We are now checking out session2 because - Session session6 = pool.get(); + Session session6 = pool.getSession(); // The session that was first in the pool now is equal to the initial first session as each full // round of pings will swap the order of the first MinSessions sessions in the pool. assertThat(session6.getName()).isEqualTo(session1.getName()); @@ -208,9 +208,9 @@ public void testKeepAlive() throws Exception { // Now check out 3 sessions again and make sure the 'extra' session is checked in last. That // will make it eligible for pings. - Session session7 = pool.get(); - Session session8 = pool.get(); - Session session9 = pool.get(); + Session session7 = pool.getSession(); + Session session8 = pool.getSession(); + Session session9 = pool.getSession(); assertThat(session7.getName()).isEqualTo(session1.getName()); assertThat(session8.getName()).isEqualTo(session2.getName()); @@ -244,8 +244,8 @@ public void testIdleSessions() throws Exception { assertThat(idledSessions).isEmpty(); // Checkout two sessions and do a maintenance loop. Still no sessions should be removed. - Session session1 = pool.get(); - Session session2 = pool.get(); + Session session1 = pool.getSession(); + Session session2 = pool.getSession(); runMaintainanceLoop(clock, pool, 1); assertThat(idledSessions).isEmpty(); @@ -262,9 +262,9 @@ public void testIdleSessions() throws Exception { // Now check out three sessions so the pool will create an additional session. The pool will // only keep 2 sessions alive, as that is the setting for MinSessions. - Session session3 = pool.get().get(); - Session session4 = pool.get().get(); - Session session5 = pool.get().get(); + Session session3 = pool.getSession().get(); + Session session4 = pool.getSession().get(); + Session session5 = pool.getSession().get(); // Note that session2 was now the first session in the pool as it was the last to receive a // ping. assertThat(session3.getName()).isEqualTo(session2.getName()); @@ -279,9 +279,9 @@ public void testIdleSessions() throws Exception { assertThat(pool.totalSessions()).isEqualTo(2); // Check out three sessions again and keep one session checked out. - Session session6 = pool.get().get(); - Session session7 = pool.get().get(); - Session session8 = pool.get().get(); + Session session6 = pool.getSession().get(); + Session session7 = pool.getSession().get(); + Session session8 = pool.getSession().get(); session8.close(); session7.close(); // Now advance the clock to idle sessions. This should remove session8 from the pool. @@ -293,9 +293,9 @@ public void testIdleSessions() throws Exception { // Check out three sessions and keep them all checked out. No sessions should be removed from // the pool. - Session session9 = pool.get().get(); - Session session10 = pool.get().get(); - Session session11 = pool.get().get(); + Session session9 = pool.getSession().get(); + Session session10 = pool.getSession().get(); + Session session11 = pool.getSession().get(); runMaintainanceLoop(clock, pool, loopsToIdleSessions); assertThat(idledSessions).containsExactly(session5, session8); assertThat(pool.totalSessions()).isEqualTo(3); diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolStressTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolStressTest.java index 3916df8652..a3b2a3c542 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolStressTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolStressTest.java @@ -270,7 +270,7 @@ public void run() { Uninterruptibles.awaitUninterruptibly(releaseThreads); for (int j = 0; j < numOperationsPerThread; j++) { try { - PooledSessionFuture session = pool.get(); + PooledSessionFuture session = pool.getSession(); session.get(); Uninterruptibles.sleepUninterruptibly( random.nextInt(5), TimeUnit.MILLISECONDS); diff --git a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolTest.java b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolTest.java index 2087fc1eda..0620bfb0e9 100644 --- a/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolTest.java +++ b/google-cloud-spanner/src/test/java/com/google/cloud/spanner/SessionPoolTest.java @@ -179,7 +179,7 @@ public void testClosedPoolIncludesClosedException() { assertThat(pool.isValid()).isTrue(); closePoolWithStacktrace(); try { - pool.get(); + pool.getSession(); fail("missing expected exception"); } catch (IllegalStateException e) { assertThat(e.getCause()).isInstanceOf(ClosedException.class); @@ -197,7 +197,7 @@ private void closePoolWithStacktrace() { public void sessionCreation() { setupMockSessionCreation(); pool = createPool(); - try (Session session = pool.get()) { + try (Session session = pool.getSession()) { assertThat(session).isNotNull(); } } @@ -206,14 +206,14 @@ public void sessionCreation() { public void poolLifo() { setupMockSessionCreation(); pool = createPool(); - Session session1 = pool.get().get(); - Session session2 = pool.get().get(); + Session session1 = pool.getSession().get(); + Session session2 = pool.getSession().get(); assertThat(session1).isNotEqualTo(session2); session2.close(); session1.close(); - Session session3 = pool.get().get(); - Session session4 = pool.get().get(); + Session session3 = pool.getSession().get(); + Session session4 = pool.getSession().get(); assertThat(session3).isEqualTo(session1); assertThat(session4).isEqualTo(session2); session3.close(); @@ -252,9 +252,9 @@ public void run() { .when(sessionClient) .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); pool = createPool(); - Session session1 = pool.get(); + Session session1 = pool.getSession(); // Leaked sessions - PooledSessionFuture leakedSession = pool.get(); + PooledSessionFuture leakedSession = pool.getSession(); // Clear the leaked exception to suppress logging of expected exceptions. leakedSession.clearLeakedException(); session1.close(); @@ -330,7 +330,7 @@ public Void call() throws Exception { .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); pool = createPool(); - PooledSessionFuture leakedSession = pool.get(); + PooledSessionFuture leakedSession = pool.getSession(); // Suppress expected leakedSession warning. leakedSession.clearLeakedException(); AtomicBoolean failed = new AtomicBoolean(false); @@ -388,7 +388,7 @@ public Void call() throws Exception { .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); pool = createPool(); - PooledSessionFuture leakedSession = pool.get(); + PooledSessionFuture leakedSession = pool.getSession(); // Suppress expected leakedSession warning. leakedSession.clearLeakedException(); AtomicBoolean failed = new AtomicBoolean(false); @@ -460,13 +460,13 @@ public void run() { .when(sessionClient) .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); pool = createPool(); - PooledSessionFuture leakedSession = pool.get(); + PooledSessionFuture leakedSession = pool.getSession(); leakedSession.get(); // Suppress expected leakedSession warning. leakedSession.clearLeakedException(); pool.closeAsync(new SpannerImpl.ClosedException()); try { - pool.get(); + pool.getSession(); fail("Expected exception"); } catch (IllegalStateException ex) { assertNotNull(ex.getMessage()); @@ -513,7 +513,7 @@ public Void call() { .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); pool = createPool(); try { - pool.get().get(); + pool.getSession().get(); fail("Expected exception"); } catch (SpannerException ex) { assertThat(ex.getErrorCode()).isEqualTo(ErrorCode.INTERNAL); @@ -547,15 +547,15 @@ public void run() { .when(sessionClient) .asyncBatchCreateSessions(Mockito.eq(1), Mockito.anyBoolean(), any(SessionConsumer.class)); pool = createPool(); - Session session1 = pool.get(); + Session session1 = pool.getSession(); try { - pool.get(); + pool.getSession(); fail("Expected exception"); } catch (SpannerException ex) { assertThat(ex.getErrorCode()).isEqualTo(ErrorCode.RESOURCE_EXHAUSTED); } session1.close(); - session1 = pool.get(); + session1 = pool.getSession(); assertThat(session1).isNotNull(); session1.close(); } @@ -599,12 +599,12 @@ public void run() { clock.currentTimeMillis = System.currentTimeMillis(); pool = createPool(clock); // Make sure pool has been initialized - pool.get().close(); + pool.getSession().close(); runMaintainanceLoop(clock, pool, pool.poolMaintainer.numClosureCycles); assertThat(pool.numIdleSessionsRemoved()).isEqualTo(0L); - PooledSessionFuture readSession1 = pool.get(); - PooledSessionFuture readSession2 = pool.get(); - PooledSessionFuture readSession3 = pool.get(); + PooledSessionFuture readSession1 = pool.getSession(); + PooledSessionFuture readSession2 = pool.getSession(); + PooledSessionFuture readSession3 = pool.getSession(); // Wait until the sessions have actually been gotten in order to make sure they are in use in // parallel. readSession1.get(); @@ -619,9 +619,9 @@ public void run() { assertThat(pool.numIdleSessionsRemoved()).isEqualTo(0L); // Counters have now been reset // Use all 3 sessions sequentially - pool.get().close(); - pool.get().close(); - pool.get().close(); + pool.getSession().close(); + pool.getSession().close(); + pool.getSession().close(); // Advance the time by running the maintainer. This should cause // one session to be kept alive and two sessions to be removed. long cycles = @@ -663,8 +663,8 @@ public void run() { FakeClock clock = new FakeClock(); clock.currentTimeMillis = System.currentTimeMillis(); pool = createPool(clock); - PooledSessionFuture session1 = pool.get(); - PooledSessionFuture session2 = pool.get(); + PooledSessionFuture session1 = pool.getSession(); + PooledSessionFuture session2 = pool.getSession(); session1.get(); session2.get(); session1.close(); @@ -675,7 +675,7 @@ public void run() { verify(session, times(2)).singleUse(any(TimestampBound.class)); clock.currentTimeMillis += clock.currentTimeMillis + (options.getKeepAliveIntervalMinutes() + 5) * 60 * 1000; - session1 = pool.get(); + session1 = pool.getSession(); session1.writeAtLeastOnce(new ArrayList()); session1.close(); runMaintainanceLoop(clock, pool, pool.poolMaintainer.numKeepAliveCycles); @@ -697,7 +697,7 @@ public void blockAndTimeoutOnPoolExhaustion() throws Exception { setupMockSessionCreation(); pool = createPool(); // Take the only session that can be in the pool. - PooledSessionFuture checkedOutSession = pool.get(); + PooledSessionFuture checkedOutSession = pool.getSession(); checkedOutSession.get(); ExecutorService executor = Executors.newFixedThreadPool(1); final CountDownLatch latch = new CountDownLatch(1); @@ -708,7 +708,7 @@ public void blockAndTimeoutOnPoolExhaustion() throws Exception { @Override public Void call() { latch.countDown(); - PooledSessionFuture session = pool.get(); + PooledSessionFuture session = pool.getSession(); session.close(); return null; } @@ -729,7 +729,7 @@ public Void call() { executor.shutdown(); // Verify that the session was returned to the pool and that we can get it again. - Session session = pool.get(); + Session session = pool.getSession(); assertThat(session).isNotNull(); session.close(); assertThat(pool.getNumWaiterTimeouts()).isAtLeast(1L); @@ -790,7 +790,7 @@ public void run() { FakeClock clock = new FakeClock(); clock.currentTimeMillis = System.currentTimeMillis(); pool = createPool(clock); - ReadContext context = pool.get().singleUse(); + ReadContext context = pool.getSession().singleUse(); ResultSet resultSet = context.executeQuery(statement); assertThat(resultSet.next()).isTrue(); } @@ -846,7 +846,7 @@ public void run() { FakeClock clock = new FakeClock(); clock.currentTimeMillis = System.currentTimeMillis(); pool = createPool(clock); - ReadOnlyTransaction transaction = pool.get().readOnlyTransaction(); + ReadOnlyTransaction transaction = pool.getSession().readOnlyTransaction(); ResultSet resultSet = transaction.executeQuery(statement); assertThat(resultSet.next()).isTrue(); } @@ -976,7 +976,7 @@ public void run() { when(spanner.getOptions()).thenReturn(spannerOptions); SessionPool pool = SessionPool.createPool(options, new TestExecutorFactory(), spanner.getSessionClient(db)); - try (PooledSessionFuture readWriteSession = pool.get()) { + try (PooledSessionFuture readWriteSession = pool.getSession()) { TransactionRunner runner = readWriteSession.readWriteTransaction(); try { runner.run( @@ -1211,8 +1211,8 @@ public void testSessionMetrics() throws Exception { setupMockSessionCreation(); pool = createPool(clock, metricRegistry, labelValues); - PooledSessionFuture session1 = pool.get(); - PooledSessionFuture session2 = pool.get(); + PooledSessionFuture session1 = pool.getSession(); + PooledSessionFuture session2 = pool.getSession(); session1.get(); session2.get(); @@ -1290,7 +1290,7 @@ public void testSessionMetrics() throws Exception { @Override public Void call() { latch.countDown(); - Session session = pool.get(); + Session session = pool.getSession(); session.close(); return null; } @@ -1348,7 +1348,7 @@ private void getSessionAsync(final CountDownLatch latch, final AtomicBoolean fai new Runnable() { @Override public void run() { - try (PooledSessionFuture future = pool.get()) { + try (PooledSessionFuture future = pool.getSession()) { PooledSession session = future.get(); failed.compareAndSet(false, session == null); Uninterruptibles.sleepUninterruptibly(10, TimeUnit.MILLISECONDS);