From 24dd201afd60392e157bb20273e733a04293e0b9 Mon Sep 17 00:00:00 2001
From: Akshay Sundarraj
Date: Thu, 14 Aug 2025 13:42:26 +0530
Subject: [PATCH 01/11] Async Java APIs
This is the initial commit for support of Async Java APIs. This commit address
some of the major code changes needed for Async, which are listed below. This
commit does not contain code changes for new Async NoSQLHandle but provides
scaffolding for the same.
- Java version is updated to 11 from 8
- Client#execute, which is the core of the request processing, is now returns
CompletableFuture. This method is changed to non-blocking with retries
and error handling
- HttpClient#runRequest returns CompletableFuture. HttpClient
now abstract out the Channel acquisition and submitting the request along with
timeout. Consumers of the HttpClient need not worry about channel acquisition
and release. The new signature of runRequest method is
`CompletableFuture runRequest(HttpRequest request, int timeoutMs)`
- AuthorizationProvider API is updated to introduce async versions of
getAuthorizationString and setRequiredHeaders
- NoSQLHandleImpl sync methods are updated to wait on CompletableFuture
- StoreAccessTokenProvider and SignatureProvider are refactored into non-blocking classes
Testing:
All existing unit tests are passing
---
driver/pom.xml | 8 +-
.../nosql/driver/AuthorizationProvider.java | 38 +
.../nosql/driver/DefaultRetryHandler.java | 5 +
.../oracle/nosql/driver/RetryHandler.java | 2 +
.../java/oracle/nosql/driver/http/Client.java | 1248 +++++++++--------
.../nosql/driver/http/NoSQLHandleImpl.java | 94 +-
.../driver/httpclient/ConnectionPool.java | 134 +-
.../nosql/driver/httpclient/HttpClient.java | 295 ++--
.../driver/httpclient/HttpClientHandler.java | 58 +-
.../driver/iam/FederationRequestHelper.java | 16 +-
.../driver/iam/InstanceMetadataHelper.java | 11 +-
.../iam/OkeWorkloadIdentityProvider.java | 14 +-
.../nosql/driver/iam/SignatureProvider.java | 296 ++--
.../driver/kv/StoreAccessTokenProvider.java | 178 ++-
.../java/oracle/nosql/driver/ops/Request.java | 2 +-
.../nosql/driver/query/ReceiveIter.java | 7 +-
.../nosql/driver/util/ConcurrentUtil.java | 32 +
.../nosql/driver/util/HttpRequestUtil.java | 157 +--
.../oracle/nosql/driver/ProxyTestBase.java | 8 +
.../driver/httpclient/ConnectionPoolTest.java | 5 +-
examples/pom.xml | 8 +-
examples/src/main/java/Common.java | 6 +
22 files changed, 1455 insertions(+), 1167 deletions(-)
create mode 100644 driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java
diff --git a/driver/pom.xml b/driver/pom.xml
index 3ac3da44..bba6b1b6 100644
--- a/driver/pom.xml
+++ b/driver/pom.xml
@@ -39,8 +39,8 @@
UTF-8
- 1.8
- 1.8
+ 11
+ 11${maven.build.timestamp}d-MMMM-yyyyCopyright (c) 2011, 2025 Oracle and/or its affiliates. All rights reserved.
@@ -274,8 +274,8 @@
3.11.0true
- 1.8
- 1.8
+ ${maven.compiler.source}
+ ${maven.compiler.target}truetrue-Xlint:all
diff --git a/driver/src/main/java/oracle/nosql/driver/AuthorizationProvider.java b/driver/src/main/java/oracle/nosql/driver/AuthorizationProvider.java
index a6c491af..2cc7dfbd 100644
--- a/driver/src/main/java/oracle/nosql/driver/AuthorizationProvider.java
+++ b/driver/src/main/java/oracle/nosql/driver/AuthorizationProvider.java
@@ -12,6 +12,8 @@
import io.netty.handler.codec.http.HttpHeaders;
import oracle.nosql.driver.ops.Request;
+import java.util.concurrent.CompletableFuture;
+
/**
* A callback interface used by the driver to obtain an authorization string
* for a request. {@link NoSQLHandle} calls this interface when and
@@ -34,6 +36,21 @@ public interface AuthorizationProvider {
*/
public String getAuthorizationString(Request request);
+ /**
+ * Returns an authorization string for specified request. This is sent to
+ * the server in the request for authorization. Authorization information
+ * can be request-dependent.
+ *
+ * @param request the request being processed
+ *
+ * @return a CompletableFuture of a string indicating that the application
+ * is authorized to perform the request
+ */
+ public default CompletableFuture
+ getAuthorizationStringAsync(Request request) {
+ return CompletableFuture.completedFuture(null);
+ }
+
/**
* Release resources provider is using.
*/
@@ -75,6 +92,27 @@ public default void setRequiredHeaders(String authString,
}
}
+ /**
+ * Set HTTP headers required by the provider asynchronously.
+ *
+ * @param authString the authorization string for the request
+ *
+ * @param request the request being processed
+ *
+ * @param headers the HTTP headers
+ *
+ * @param content the request content bytes
+ */
+ default CompletableFuture setRequiredHeadersAsync(String authString,
+ Request request,
+ HttpHeaders headers,
+ byte[] content) {
+ if (authString != null) {
+ headers.set(AUTHORIZATION, authString);
+ }
+ return CompletableFuture.completedFuture(null);
+ }
+
/**
* Invalidate any cached authorization strings.
*/
diff --git a/driver/src/main/java/oracle/nosql/driver/DefaultRetryHandler.java b/driver/src/main/java/oracle/nosql/driver/DefaultRetryHandler.java
index b7e4be88..ed106a46 100644
--- a/driver/src/main/java/oracle/nosql/driver/DefaultRetryHandler.java
+++ b/driver/src/main/java/oracle/nosql/driver/DefaultRetryHandler.java
@@ -85,6 +85,11 @@ public void delay(Request request,
request.addRetryDelayMs(delayMs);
}
+ @Override
+ public int delayTime(Request request, int numRetries, RetryableException re) {
+ return Math.max(0, computeBackoffDelay(request, fixedDelayMs));
+ }
+
/**
* Compute an incremental backoff delay in milliseconds.
* This method also checks the request's timeout and ensures the
diff --git a/driver/src/main/java/oracle/nosql/driver/RetryHandler.java b/driver/src/main/java/oracle/nosql/driver/RetryHandler.java
index 3d625c27..c3b841b0 100644
--- a/driver/src/main/java/oracle/nosql/driver/RetryHandler.java
+++ b/driver/src/main/java/oracle/nosql/driver/RetryHandler.java
@@ -69,4 +69,6 @@ public interface RetryHandler {
* @param re the exception that was thrown
*/
void delay(Request request, int numRetries, RetryableException re);
+
+ int delayTime(Request request, int numRetries, RetryableException re);
}
diff --git a/driver/src/main/java/oracle/nosql/driver/http/Client.java b/driver/src/main/java/oracle/nosql/driver/http/Client.java
index 014d84a2..4c3a9c5f 100644
--- a/driver/src/main/java/oracle/nosql/driver/http/Client.java
+++ b/driver/src/main/java/oracle/nosql/driver/http/Client.java
@@ -44,21 +44,30 @@
import java.util.Iterator;
import java.util.Map;
import java.util.Properties;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Function;
+import java.util.function.Supplier;
import java.util.logging.Level;
import java.util.logging.Logger;
+import io.netty.buffer.Unpooled;
import oracle.nosql.driver.AuthorizationProvider;
-import oracle.nosql.driver.DefaultRetryHandler;
import oracle.nosql.driver.InvalidAuthorizationException;
import oracle.nosql.driver.NoSQLException;
import oracle.nosql.driver.NoSQLHandleConfig;
@@ -66,7 +75,6 @@
import oracle.nosql.driver.RateLimiter;
import oracle.nosql.driver.ReadThrottlingException;
import oracle.nosql.driver.RequestSizeLimitException;
-import oracle.nosql.driver.RequestTimeoutException;
import oracle.nosql.driver.RetryHandler;
import oracle.nosql.driver.RetryableException;
import oracle.nosql.driver.SecurityInfoNotReadyException;
@@ -76,7 +84,6 @@
import oracle.nosql.driver.UnsupportedQueryVersionException;
import oracle.nosql.driver.WriteThrottlingException;
import oracle.nosql.driver.httpclient.HttpClient;
-import oracle.nosql.driver.httpclient.ResponseHandler;
import oracle.nosql.driver.kv.AuthenticationException;
import oracle.nosql.driver.kv.StoreAccessTokenProvider;
import oracle.nosql.driver.ops.AddReplicaRequest;
@@ -105,15 +112,16 @@
import oracle.nosql.driver.query.QueryDriver;
import oracle.nosql.driver.query.TopologyInfo;
import oracle.nosql.driver.util.ByteInputStream;
+import oracle.nosql.driver.util.ConcurrentUtil;
import oracle.nosql.driver.util.HttpConstants;
import oracle.nosql.driver.util.NettyByteInputStream;
import oracle.nosql.driver.util.NettyByteOutputStream;
import oracle.nosql.driver.util.RateLimiterMap;
import oracle.nosql.driver.util.SerializationUtil;
+import oracle.nosql.driver.util.SimpleRateLimiter;
import oracle.nosql.driver.values.MapValue;
import io.netty.buffer.ByteBuf;
-import io.netty.channel.Channel;
import io.netty.handler.codec.http.DefaultFullHttpRequest;
import io.netty.handler.codec.http.FullHttpRequest;
import io.netty.handler.codec.http.HttpHeaderNames;
@@ -152,7 +160,7 @@ public class Client {
/**
* Tracks the unique client scoped request id.
*/
- private final AtomicInteger maxRequestId = new AtomicInteger(1);
+ private final AtomicLong maxRequestId = new AtomicLong(1);
private final HttpClient httpClient;
@@ -199,7 +207,7 @@ public class Client {
/**
* config for statistics
*/
- private StatsControlImpl statsControl;
+ private final StatsControlImpl statsControl;
/**
* list of Request instances to refresh when auth changes. This will only
@@ -218,13 +226,18 @@ public class Client {
private final String SESSION_COOKIE_FIELD = "session=";
/* for keeping track of SDKs usage */
- private String userAgent;
+ private final String userAgent;
private volatile TopologyInfo topology;
/* for internal testing */
private final String prepareFilename;
+ /* thread-pool for scheduling tasks */
+ private final ScheduledExecutorService taskExecutor;
+
+ private final ReentrantLock lock = new ReentrantLock();
+
public Client(Logger logger,
NoSQLHandleConfig httpConfig) {
@@ -312,6 +325,21 @@ public Client(Logger logger,
/* for internal testing */
prepareFilename = System.getProperty("test.preparefilename");
+
+ taskExecutor = new ScheduledThreadPoolExecutor(8 /* core threads */,
+ new ThreadFactory() {
+ private final AtomicInteger threadNumber = new AtomicInteger(1);
+ @Override
+ public Thread newThread(Runnable r) {
+ final Thread t = Executors.defaultThreadFactory().newThread(r);
+ t.setName(String.format("task-executor-%s", threadNumber.getAndIncrement()));
+ t.setDaemon(true);
+ t.setUncaughtExceptionHandler((thread, error) -> {
+ // TODO handle this
+ });
+ return t;
+ }
+ });
}
/**
@@ -332,6 +360,9 @@ public void shutdown() {
if (threadPool != null) {
threadPool.shutdown();
}
+ if (taskExecutor != null) {
+ taskExecutor.shutdown();
+ }
}
public int getAcquiredChannelCount() {
@@ -350,12 +381,63 @@ public int getFreeChannelCount() {
* Get the next client-scoped request id. It needs to be combined with the
* client id to obtain a globally unique scope.
*/
- private int nextRequestId() {
+ private long nextRequestId() {
return maxRequestId.addAndGet(1);
}
/**
- * Execute the KV request and return the response. This is the top-level
+ * RequestContext class to encapsulate request-specific data.
+ * This helps in passing context through asynchronous CompletableFuture chains.
+ * It now includes requestId and a Supplier to generate new IDs for retries.
+ */
+ private static class RequestContext {
+ private final Request kvRequest;
+ private final String requestClass;
+ private volatile String requestId;
+ private final long startNanos;
+ private final int timeoutMs;
+ private final Supplier nextIdSupplier;
+ private volatile Throwable exception;
+ private final AtomicInteger rateDelayedMs = new AtomicInteger(0);
+ private volatile RateLimiter readLimiter;
+ private volatile RateLimiter writeLimiter;
+ private volatile boolean checkReadUnits;
+ private volatile boolean checkWriteUnits;
+ private volatile int reqSize;
+ private volatile int resSize;
+ private volatile short serialVersionUsed;
+ private volatile short queryVersionUsed;
+ private volatile long latencyNanos;
+ public volatile long networkLatency;
+
+ RequestContext(Request kvRequest, long startNanos, int timeoutMs,
+ Supplier nextIdSupplier, RateLimiter readLimiter,
+ RateLimiter writeLimiter, boolean checkReadUnits,
+ boolean checkWriteUnits) {
+ this.kvRequest = kvRequest;
+ this.startNanos = startNanos;
+ this.timeoutMs = timeoutMs;
+ this.nextIdSupplier = nextIdSupplier;
+ this.readLimiter = readLimiter;
+ this.writeLimiter = writeLimiter;
+ this.checkReadUnits = checkReadUnits;
+ this.checkWriteUnits = checkWriteUnits;
+
+ this.requestId = Long.toString(nextIdSupplier.get());
+ this.requestClass = kvRequest.getClass().getSimpleName();
+ }
+
+// RequestContext nextRetryContext() {
+// // Generate a new ID for the next retry attempt using the provided supplier
+// return new RequestContext(this.kvRequest,
+// this.retryCount + 1, this.nextIdSupplier, readLimiter,
+// writeLimiter, checkReadUnits, checkWriteUnits);
+// }
+ }
+
+ /**
+ * Execute the KV request and return the future response. This is the
+ * top-level
* method for request execution.
*
* This method handles exceptions to distinguish between what can be retried
@@ -372,12 +454,11 @@ private int nextRequestId() {
*
* @param kvRequest the KV request to be executed by the server
*
- * @return the Result of the request
+ * @return the future representing the result of the request
*/
- public Result execute(Request kvRequest) {
+ public CompletableFuture execute(Request kvRequest) {
requireNonNull(kvRequest, "NoSQLHandle: request must be non-null");
-
/*
* Before execution, call Request object to assign default values
* from config object if they are not overridden. This allows code
@@ -391,7 +472,11 @@ public Result execute(Request kvRequest) {
* fails for a given Request instance it will throw
* IllegalArgumentException.
*/
- kvRequest.validate();
+ try {
+ kvRequest.validate();
+ } catch (Throwable t) {
+ return CompletableFuture.failedFuture(t);
+ }
/* clear any retry stats that may exist on this request object */
kvRequest.setRetryStats(null);
@@ -401,7 +486,7 @@ public Result execute(Request kvRequest) {
* non-internal QueryRequest */
if (kvRequest.isQueryRequest()) {
- QueryRequest qreq = (QueryRequest)kvRequest;
+ QueryRequest qreq = (QueryRequest) kvRequest;
/* Set the topo seq num in the request, if it has not been set
* already */
@@ -419,7 +504,8 @@ public Result execute(Request kvRequest) {
*/
if (qreq.hasDriver()) {
trace("QueryRequest has QueryDriver", 2);
- return new QueryResult(qreq, false);
+ return CompletableFuture.completedFuture(
+ new QueryResult(qreq, false));
}
/*
@@ -434,7 +520,8 @@ public Result execute(Request kvRequest) {
trace("QueryRequest has no QueryDriver, but is prepared", 2);
QueryDriver driver = new QueryDriver(qreq);
driver.setClient(this);
- return new QueryResult(qreq, false);
+ return CompletableFuture.completedFuture(
+ new QueryResult(qreq, false));
}
/*
@@ -452,10 +539,6 @@ public Result execute(Request kvRequest) {
qreq.incBatchCounter();
}
- int timeoutMs = kvRequest.getTimeoutInternal();
-
- Throwable exception = null;
-
/*
* If the request doesn't set an explicit compartment, use
* the config default if provided.
@@ -465,7 +548,6 @@ public Result execute(Request kvRequest) {
config.getDefaultCompartment());
}
- int rateDelayedMs = 0;
boolean checkReadUnits = false;
boolean checkWriteUnits = false;
@@ -481,15 +563,15 @@ public Result execute(Request kvRequest) {
/* if not, see if we have limiters in our map for the given table */
if (rateLimiterMap != null &&
- readLimiter == null && writeLimiter == null) {
+ readLimiter == null && writeLimiter == null) {
String tableName = kvRequest.getTableName();
- if (tableName != null && tableName.length() > 0) {
+ if (tableName != null && !tableName.isEmpty()) {
readLimiter = rateLimiterMap.getReadLimiter(tableName);
writeLimiter = rateLimiterMap.getWriteLimiter(tableName);
if (readLimiter == null && writeLimiter == null) {
if (kvRequest.doesReads() || kvRequest.doesWrites()) {
backgroundUpdateLimiters(tableName,
- kvRequest.getCompartment());
+ kvRequest.getCompartment());
}
} else {
checkReadUnits = kvRequest.doesReads();
@@ -499,526 +581,504 @@ public Result execute(Request kvRequest) {
}
}
}
+ kvRequest.setStartNanos(System.nanoTime());
- final long startNanos = System.nanoTime();
- kvRequest.setStartNanos(startNanos);
- final String requestClass = kvRequest.getClass().getSimpleName();
+ RequestContext ctx = new RequestContext(kvRequest,
+ kvRequest.getStartNanos(), kvRequest.getTimeoutInternal(),
+ maxRequestId::getAndIncrement, readLimiter, writeLimiter,
+ checkReadUnits, checkWriteUnits);
- /*
- * boolean that indicates whether content must be signed. Cross
- * region operations must include content when signing. See comment
- * on the method
- */
- final boolean signContent = requireContentSigned(kvRequest);
- String requestId = "";
- int thisIterationTimeoutMs = 0;
-
- do {
- thisIterationTimeoutMs =
- getIterationTimeoutMs(timeoutMs, startNanos);
- /*
- * Check rate limiters before executing the request.
- * Wait for read and/or write limiters to be below their limits
- * before continuing. Be aware of the timeout given.
- */
- if (readLimiter != null && checkReadUnits == true) {
- try {
- /*
- * this may sleep for a while, up to thisIterationTimeoutMs
- * and may throw TimeoutException
- */
- rateDelayedMs += readLimiter.consumeUnitsWithTimeout(
- 0, thisIterationTimeoutMs, false);
- } catch (Exception e) {
- exception = e;
- break;
- }
- }
- if (writeLimiter != null && checkWriteUnits == true) {
- try {
- /*
- * this may sleep for a while, up to thisIterationTimeoutMs
- * and may throw TimeoutException
- */
- rateDelayedMs += writeLimiter.consumeUnitsWithTimeout(
- 0, thisIterationTimeoutMs, false);
- } catch (Exception e) {
- exception = e;
- break;
- }
- }
-
- /* update iteration timeout in case limiters slept for some time */
- thisIterationTimeoutMs =
- getIterationTimeoutMs(timeoutMs, startNanos);
+ return executeWithRetry(ctx);
+ }
- /* ensure limiting didn't throw us over the timeout */
- if (thisIterationTimeoutMs <= 0) {
- break;
- }
- final String authString =
- authProvider.getAuthorizationString(kvRequest);
- authProvider.validateAuthString(authString);
+ /**
+ * Core method which creates the request and send to the server.
+ * If the request fails, it performs retry.
+ */
+ private CompletableFuture executeWithRetry(RequestContext ctx) {
+ /*
+ * Check rate limiters before executing the request.
+ * Wait for read and/or write limiters to be below their limits
+ * before continuing. Be aware of the timeout given.
+ */
+ Request kvRequest = ctx.kvRequest;
- if (kvRequest.getNumRetries() > 0) {
- logRetries(kvRequest.getNumRetries(), exception);
- }
+ if (kvRequest.getNumRetries() > 0) {
+ logRetries(kvRequest.getNumRetries(), ctx.exception);
+ }
- if (serialVersion < 3 && kvRequest instanceof DurableRequest) {
- if (((DurableRequest)kvRequest).getDurability() != null) {
- oneTimeMessage("The requested feature is not supported " +
- "by the connected server: Durability");
- }
+ if (serialVersion < 3 && kvRequest instanceof DurableRequest) {
+ if (((DurableRequest)kvRequest).getDurability() != null) {
+ oneTimeMessage("The requested feature is not supported " +
+ "by the connected server: Durability");
}
-
- if (serialVersion < 3 && kvRequest instanceof TableRequest) {
- TableLimits limits = ((TableRequest)kvRequest).getTableLimits();
- if (limits != null &&
+ }
+ if (serialVersion < 3 && kvRequest instanceof TableRequest) {
+ TableLimits limits = ((TableRequest)kvRequest).getTableLimits();
+ if (limits != null &&
limits.getMode() == CapacityMode.ON_DEMAND) {
- oneTimeMessage("The requested feature is not supported " +
- "by the connected server: on demand " +
- "capacity table");
- }
+ oneTimeMessage("The requested feature is not supported " +
+ "by the connected server: on demand " +
+ "capacity table");
}
+ }
- ResponseHandler responseHandler = null;
- short serialVersionUsed = serialVersion;
- short queryVersionUsed = queryVersion;
- ByteBuf buffer = null;
- try {
- /*
- * NOTE: the ResponseHandler will release the Channel
- * in its close() method, which is always called in the
- * finally clause. This handles both successful and retried
- * operations in the loop.
- */
- Channel channel = httpClient.getChannel(thisIterationTimeoutMs);
- /* update iteration timeout in case channel took some time */
- thisIterationTimeoutMs =
- getIterationTimeoutMs(timeoutMs, startNanos);
- /* ensure limiting didn't throw us over the timeout */
- if (thisIterationTimeoutMs <= 0) {
- break;
- }
-
- requestId = Long.toString(nextRequestId());
- responseHandler =
- new ResponseHandler(httpClient, logger, channel,
- requestId, kvRequest.shouldRetry());
- buffer = channel.alloc().directBuffer();
- buffer.retain();
-
- /*
- * we expressly check size limit below based on onprem versus
- * cloud. Set the request to not check size limit inside
- * writeContent().
- */
- kvRequest.setCheckRequestSize(false);
-
- /* Set the topo seq num in the request, if it has not been set
- * already */
- if (!(kvRequest instanceof QueryRequest) ||
- kvRequest.isQueryRequest()) {
- kvRequest.setTopoSeqNum(getTopoSeqNum());
- }
-
- /*
- * Temporarily change the timeout in the request object so
- * the serialized timeout sent to the server is correct for
- * this iteration. After serializing the request, set the
- * timeout back to the overall request timeout so that other
- * processing (retry delays, etc) work correctly.
- */
- kvRequest.setTimeoutInternal(thisIterationTimeoutMs);
- serialVersionUsed = writeContent(buffer, kvRequest,
- queryVersionUsed);
- kvRequest.setTimeoutInternal(timeoutMs);
- /*
- * If on-premises the authProvider will always be a
- * StoreAccessTokenProvider. If so, check against
- * configurable limit. Otherwise check against internal
- * hardcoded cloud limit.
- */
- if (authProvider instanceof StoreAccessTokenProvider) {
- if (buffer.readableBytes() >
- httpClient.getMaxContentLength()) {
- throw new RequestSizeLimitException("The request " +
- "size of " + buffer.readableBytes() +
- " exceeded the limit of " +
- httpClient.getMaxContentLength());
+ /* Check this request doesn't cause throttle */
+ int preRateLimitDelayMs = 0;
+ if (ctx.readLimiter != null && ctx.checkReadUnits) {
+ preRateLimitDelayMs +=
+ ((SimpleRateLimiter) ctx.readLimiter).consumeExternally(0);
+ }
+ if (ctx.writeLimiter != null && ctx.checkWriteUnits) {
+ preRateLimitDelayMs +=
+ ((SimpleRateLimiter) ctx.writeLimiter).consumeExternally(0);
+ }
+
+ final int timeoutMs = ctx.timeoutMs;
+ final long startNanos = ctx.startNanos;
+ final int thisIterationTimeoutMs = getIterationTimeoutMs(timeoutMs, startNanos);
+
+ /* If this result in timeout, complete with exception */
+ if (thisIterationTimeoutMs <= preRateLimitDelayMs) {
+ final TimeoutException ex = new TimeoutException(
+ "timed out waiting "
+ + thisIterationTimeoutMs
+ + "ms due to rate limiting");
+ return createDelayFuture(thisIterationTimeoutMs).thenCompose(d ->
+ CompletableFuture.failedFuture(ex));
+ }
+
+ /* Stage-1 RateLimit */
+ return createDelayFuture(preRateLimitDelayMs)
+ .whenComplete((delay, err) -> ctx.rateDelayedMs.addAndGet(delay))
+ /* Stage-2 Get auth token */
+ //TODO is this thenComposeAsync?
+ .thenCompose(delay -> authProvider.getAuthorizationStringAsync(kvRequest)) // CompletableFuture
+ .thenApply(authString -> {
+ /* Check whether timed out while acquiring the auth token */
+ if (timeoutRequest(kvRequest.getStartNanos(),
+ kvRequest.getTimeoutInternal(), null)) {
+ TimeoutException ex = new TimeoutException(
+ "timed out during auth token acquisition");
+ throw new CompletionException(ex);
+ }
+ authProvider.validateAuthString(authString);
+ return authString;
+ })
+ .thenCompose(authString -> {
+ /* stage-3 create HTTP request */
+ final ByteBuf buffer = Unpooled.buffer();
+ buffer.retain();
+ return createRequest(ctx, authString, buffer)
+ .whenComplete((res, err) -> {
+ if (err != null) {
+ buffer.release();
}
- } else {
- kvRequest.setCheckRequestSize(true);
- BinaryProtocol.checkRequestSizeLimit(
- kvRequest, buffer.readableBytes());
- }
-
- final FullHttpRequest request =
- new DefaultFullHttpRequest(
- HTTP_1_1, POST, kvRequestURI,
- buffer,
- headersFactory().withValidation(false),
- trailersFactory().withValidation(false));
- HttpHeaders headers = request.headers();
- addCommonHeaders(headers);
- int contentLength = buffer.readableBytes();
- headers.add(HttpHeaderNames.HOST, host)
- .add(REQUEST_ID_HEADER, requestId)
- .setInt(CONTENT_LENGTH, contentLength);
- if (sessionCookie != null) {
- headers.add(COOKIE, sessionCookie);
- }
-
- String serdeVersion = getSerdeVersion(kvRequest);
- if (serdeVersion != null) {
- headers.add("x-nosql-serde-version", serdeVersion);
- }
-
- /*
- * If the request doesn't set an explicit compartment, use
- * the config default if provided.
- */
- if (kvRequest.getCompartment() == null) {
- kvRequest.setCompartmentInternal(
- config.getDefaultCompartment());
- }
-
- /*
- * Get request body bytes if the request needed to be signed
- * with content
- */
- byte[] content = signContent ? getBodyBytes(buffer) : null;
- authProvider.setRequiredHeaders(authString, kvRequest, headers,
- content);
-
- String namespace = kvRequest.getNamespace();
- if (namespace == null) {
- namespace = config.getDefaultNamespace();
- }
- if (namespace != null) {
- headers.add(REQUEST_NAMESPACE_HEADER, namespace);
- }
-
+ })
+ .thenCompose(request -> {
+ /* Stage-4 Send the request to server */
+ if (isLoggable(logger, Level.FINE) &&
+ !kvRequest.getIsRefresh()) {
+ logTrace(logger, "Request: " + ctx.requestClass +
+ ", requestId=" + ctx.requestId);
+ }
+ ctx.latencyNanos = System.nanoTime();
+ return httpClient.runRequest(request,
+ getIterationTimeoutMs(timeoutMs, startNanos)) //CompletableFuture
+ .whenComplete((res, err) -> {
+ ctx.networkLatency =
+ (System.nanoTime() - ctx.latencyNanos) / 1_000_000;
+ // Release our retained request ByteBuf
+ buffer.release();
+ });
+ });
+ }) // Now we have CompletableFuture
+ .thenApply(fhr -> {
+ /* Stage-5 Process the response */
if (isLoggable(logger, Level.FINE) &&
- !kvRequest.getIsRefresh()) {
- logTrace(logger, "Request: " + requestClass +
- ", requestId=" + requestId);
+ !kvRequest.getIsRefresh()) {
+ logTrace(logger, "Response: " + ctx.requestClass +
+ ", status=" + fhr.status() +
+ ", requestId=" + ctx.requestId );
}
- long latencyNanos = System.nanoTime();
- httpClient.runRequest(request, responseHandler, channel);
-
- boolean isTimeout =
- responseHandler.await(thisIterationTimeoutMs);
- if (isTimeout) {
- throw new TimeoutException("Request timed out after " +
- timeoutMs + " milliseconds: requestId=" + requestId);
- }
-
- if (isLoggable(logger, Level.FINE) &&
- !kvRequest.getIsRefresh()) {
- logTrace(logger, "Response: " + requestClass +
- ", status=" +
- responseHandler.getStatus() +
- ", requestId=" + requestId );
+ try {
+ Result result = processResponse(fhr.status(), fhr.headers(),
+ fhr.content(), ctx);
+ ctx.rateDelayedMs.addAndGet(getRateDelayedFromHeader(fhr.headers()));
+ ctx.resSize = fhr.content().readerIndex();
+ return result;
+ } finally {
+ fhr.release(); //release response
}
-
- ByteBuf wireContent = responseHandler.getContent();
- Result res = processResponse(responseHandler.getStatus(),
- responseHandler.getHeaders(),
- wireContent,
- kvRequest,
- serialVersionUsed,
- queryVersionUsed);
- rateDelayedMs += getRateDelayedFromHeader(
- responseHandler.getHeaders());
- int resSize = wireContent.readerIndex();
- long networkLatency =
- (System.nanoTime() - latencyNanos) / 1_000_000;
-
- setTopology(res.getTopology());
-
- if (serialVersionUsed < 3) {
+ }) // Now we have CompletableFuture
+ .thenCompose(result -> {
+ /* Stage-6 Get the Result and bookkeeping */
+ setTopology(result.getTopology());
+ if (ctx.serialVersionUsed < 3) {
/* so we can emit a one-time message if the app */
/* tries to access modificationTime */
- if (res instanceof GetResult) {
- ((GetResult)res).setClient(this);
- } else if (res instanceof WriteResult) {
- ((WriteResult)res).setClient(this);
+ if (result instanceof GetResult) {
+ ((GetResult)result).setClient(this);
+ } else if (result instanceof WriteResult) {
+ ((WriteResult)result).setClient(this);
}
}
-
- if (res instanceof QueryResult && kvRequest.isQueryRequest()) {
+ if (result instanceof QueryResult && kvRequest.isQueryRequest()) {
QueryRequest qreq = (QueryRequest)kvRequest;
- qreq.addQueryTraces(((QueryResult)res).getQueryTraces());
+ qreq.addQueryTraces(((QueryResult)result).getQueryTraces());
}
-
- if (res instanceof TableResult && rateLimiterMap != null) {
+ if (result instanceof TableResult && rateLimiterMap != null) {
/* update rate limiter settings for table */
- TableLimits tl = ((TableResult)res).getTableLimits();
- updateRateLimiters(((TableResult)res).getTableName(), tl);
+ TableLimits tl = ((TableResult)result).getTableLimits();
+ updateRateLimiters(((TableResult)result).getTableName(), tl);
}
-
/*
* We may not have rate limiters yet because queries may
* not have a tablename until after the first request.
* So try to get rate limiters if we don't have them yet and
* this is a QueryRequest.
*/
- if (rateLimiterMap != null && readLimiter == null) {
- readLimiter = getQueryRateLimiter(kvRequest, true);
+ if (rateLimiterMap != null && ctx.readLimiter == null) {
+ ctx.readLimiter = getQueryRateLimiter(kvRequest, true);
}
- if (rateLimiterMap != null && writeLimiter == null) {
- writeLimiter = getQueryRateLimiter(kvRequest, false);
+ if (rateLimiterMap != null && ctx.writeLimiter == null) {
+ ctx.writeLimiter = getQueryRateLimiter(kvRequest, false);
}
- /* consume rate limiter units based on actual usage */
- rateDelayedMs += consumeLimiterUnits(readLimiter,
- res.getReadUnitsInternal(),
- thisIterationTimeoutMs);
- rateDelayedMs += consumeLimiterUnits(writeLimiter,
- res.getWriteUnitsInternal(),
- thisIterationTimeoutMs);
- res.setRateLimitDelayedMs(rateDelayedMs);
+ int postRateLimitDelayMs = consumeLimiterUnits(ctx.readLimiter,
+ result.getReadUnitsInternal());
+ postRateLimitDelayMs += consumeLimiterUnits(ctx.writeLimiter,
+ result.getWriteUnitsInternal());
+
+ /* Post-Response RateLimiting */
+ return createDelayFuture(postRateLimitDelayMs)
+ .thenApply(rateDelay -> {
+ ctx.rateDelayedMs.addAndGet(rateDelay);
+ result.setRateLimitDelayedMs(ctx.rateDelayedMs.get());
+
+ /* copy retry stats to Result on successful operation */
+ result.setRetryStats(kvRequest.getRetryStats());
+ kvRequest.setRateLimitDelayedMs(ctx.rateDelayedMs.get());
+
+ statsControl.observe(kvRequest,
+ Math.toIntExact(ctx.networkLatency),
+ ctx.reqSize, ctx.resSize);
+ checkAuthRefreshList(kvRequest);
+ return result;
+ });
+ })
+ .handle((res,err) -> {
+ /* Handle error and retry */
+ if (err != null) {
+ ctx.exception = err;
+ return handleError(ctx);
+ } else {
+ return CompletableFuture.completedFuture(res);
+ }
+ })
+ .thenCompose(Function.identity());
+ }
- /* copy retry stats to Result on successful operation */
- res.setRetryStats(kvRequest.getRetryStats());
- kvRequest.setRateLimitDelayedMs(rateDelayedMs);
+ private CompletableFuture createRequest(RequestContext ctx,
+ String authString,
+ ByteBuf buffer) {
- statsControl.observe(kvRequest, Math.toIntExact(networkLatency),
- contentLength, resSize);
+ try {
+ final Request kvRequest = ctx.kvRequest;
+ /*
+ * we expressly check size limit below based on onprem versus
+ * cloud. Set the request to not check size limit inside
+ * writeContent().
+ */
+ kvRequest.setCheckRequestSize(false);
- checkAuthRefreshList(kvRequest);
+ /* Set the topo seq num in the request, if it has not been set
+ * already */
+ if (!(kvRequest instanceof QueryRequest) ||
+ kvRequest.isQueryRequest()) {
+ kvRequest.setTopoSeqNum(getTopoSeqNum());
+ }
- return res;
+ /*
+ * Temporarily change the timeout in the request object so
+ * the serialized timeout sent to the server is correct for
+ * this iteration. After serializing the request, set the
+ * timeout back to the overall request timeout so that other
+ * processing (retry delays, etc) work correctly.
+ */
+ kvRequest.setTimeoutInternal(
+ getIterationTimeoutMs(ctx.timeoutMs, ctx.startNanos));
+ writeContent(buffer, ctx);
+ kvRequest.setTimeoutInternal(ctx.timeoutMs);
- } catch (AuthenticationException rae) {
- if (authProvider instanceof StoreAccessTokenProvider) {
- final StoreAccessTokenProvider satp =
- (StoreAccessTokenProvider) authProvider;
- satp.bootstrapLogin(kvRequest);
- kvRequest.addRetryException(rae.getClass());
- kvRequest.incrementRetries();
- exception = rae;
- continue;
+ /*
+ * If on-premises the authProvider will always be a
+ * StoreAccessTokenProvider. If so, check against
+ * configurable limit. Otherwise check against internal
+ * hardcoded cloud limit.
+ */
+ if (authProvider instanceof StoreAccessTokenProvider) {
+ if (buffer.readableBytes() >
+ httpClient.getMaxContentLength()) {
+ throw new RequestSizeLimitException("The request " +
+ "size of " + buffer.readableBytes() +
+ " exceeded the limit of " +
+ httpClient.getMaxContentLength());
+ }
+ } else {
+ kvRequest.setCheckRequestSize(true);
+ BinaryProtocol.checkRequestSizeLimit(
+ kvRequest, buffer.readableBytes());
+ }
+ final FullHttpRequest request =
+ new DefaultFullHttpRequest(
+ HTTP_1_1, POST, kvRequestURI,
+ buffer,
+ headersFactory().withValidation(false),
+ trailersFactory().withValidation(false));
+ HttpHeaders headers = request.headers();
+ addCommonHeaders(headers);
+ int contentLength = buffer.readableBytes();
+ ctx.reqSize = contentLength;
+ headers.add(HttpHeaderNames.HOST, host)
+ .add(REQUEST_ID_HEADER, ctx.requestId)
+ .setInt(CONTENT_LENGTH, contentLength);
+ if (sessionCookie != null) {
+ headers.add(COOKIE, sessionCookie);
+ }
+ String serdeVersion = getSerdeVersion(kvRequest);
+ if (serdeVersion != null) {
+ headers.add("x-nosql-serde-version", serdeVersion);
+ }
+
+ /*
+ * boolean that indicates whether content must be signed. Cross
+ * region operations must include content when signing. See comment
+ * on the method
+ */
+ final boolean signContent = requireContentSigned(kvRequest);
+
+ /*
+ * Get request body bytes if the request needed to be signed
+ * with content
+ */
+ byte[] content = signContent ? getBodyBytes(buffer) : null;
+ return authProvider.setRequiredHeadersAsync(authString, kvRequest, headers, content)
+ .thenApply(n -> {
+ String namespace = kvRequest.getNamespace();
+ if (namespace == null) {
+ namespace = config.getDefaultNamespace();
+ }
+ if (namespace != null) {
+ headers.add(REQUEST_NAMESPACE_HEADER, namespace);
}
- kvRequest.setRateLimitDelayedMs(rateDelayedMs);
- statsControl.observeError(kvRequest);
+ return request;
+ });
+ } catch (Exception e) {
+ return CompletableFuture.failedFuture(e);
+ }
+ }
+ private CompletableFuture handleError(RequestContext ctx) {
+ Request kvRequest = ctx.kvRequest;
+ Throwable err = ctx.exception;
+ Throwable actualCause = (err instanceof CompletionException) ?
+ err.getCause() : err;
+ boolean retryable = false;
+ int delayMs = 0;
+
+ try {
+ throw actualCause;
+ } catch (AuthenticationException rae) {
+ if (authProvider instanceof StoreAccessTokenProvider) {
+ final StoreAccessTokenProvider satp =
+ (StoreAccessTokenProvider) authProvider;
+ satp.bootstrapLogin(kvRequest);
+ retryable = true;
+ } else {
logInfo(logger, "Unexpected authentication exception: " +
rae);
throw new NoSQLException("Unexpected exception: " +
rae.getMessage(), rae);
- } catch (InvalidAuthorizationException iae) {
- /*
- * Allow a single retry for invalid/expired auth
- * This includes "clock skew" errors
- * This does not include permissions-related errors
- */
- if (kvRequest.getNumRetries() > 0) {
- /* same as NoSQLException below */
- kvRequest.setRateLimitDelayedMs(rateDelayedMs);
- statsControl.observeError(kvRequest);
- logFine(logger, "Client execute NoSQLException: " +
- iae.getMessage());
- throw iae;
- }
- /* flush auth cache and do one retry */
+ }
+ } catch (InvalidAuthorizationException iae) {
+ /*
+ * Allow a single retry for invalid/expired auth
+ * This includes "clock skew" errors
+ * This does not include permissions-related errors
+ */
+ if (kvRequest.getNumRetries() == 0) {
authProvider.flushCache();
- kvRequest.addRetryException(iae.getClass());
- kvRequest.incrementRetries();
- exception = iae;
logFine(logger,
- "Client retrying on InvalidAuthorizationException: " +
- iae.getMessage());
- continue;
- } catch (SecurityInfoNotReadyException sinre) {
- kvRequest.addRetryException(sinre.getClass());
- exception = sinre;
- int delayMs = SEC_ERROR_DELAY_MS;
- if (kvRequest.getNumRetries() > 10) {
- delayMs =
- DefaultRetryHandler.computeBackoffDelay(kvRequest, 0);
- if (delayMs <= 0) {
- break;
- }
- }
- try {
- Thread.sleep(delayMs);
- } catch (InterruptedException ie) {}
- kvRequest.incrementRetries();
- kvRequest.addRetryDelayMs(delayMs);
- continue;
- } catch (RetryableException re) {
-
- if (re instanceof WriteThrottlingException &&
- writeLimiter != null) {
- /* ensure we check write limits next loop */
- checkWriteUnits = true;
- /* set limiter to its limit, if not over already */
- if (writeLimiter.getCurrentRate() < 100.0) {
- writeLimiter.setCurrentRate(100.0);
- }
- /* call retry handler to manage sleep/delay */
+ "Client retrying on InvalidAuthorizationException: " +
+ iae.getMessage());
+ retryable = true;
+ } else {
+ throw iae;
+ }
+
+ } catch (SecurityInfoNotReadyException sinre) {
+ kvRequest.addRetryException(sinre.getClass());
+ if (kvRequest.getNumRetries() < 20) {
+ delayMs = SEC_ERROR_DELAY_MS;
+ retryable = true;
+ }
+ } catch (RetryableException re) {
+ if (re instanceof WriteThrottlingException &&
+ ctx.writeLimiter != null) {
+ /* ensure we check write limits next loop */
+ ctx.checkWriteUnits = true;
+ /* set limiter to its limit, if not over already */
+ if (ctx.writeLimiter.getCurrentRate() < 100.0) {
+ ctx.writeLimiter.setCurrentRate(100.0);
}
- if (re instanceof ReadThrottlingException &&
- readLimiter != null) {
- /* ensure we check read limits next loop */
- checkReadUnits = true;
- /* set limiter to its limit, if not over already */
- if (readLimiter.getCurrentRate() < 100.0) {
- readLimiter.setCurrentRate(100.0);
- }
- /* call retry handler to manage sleep/delay */
+ /* call retry handler to manage sleep/delay */
+ }
+ if (re instanceof ReadThrottlingException &&
+ ctx.readLimiter != null) {
+ /* ensure we check read limits next loop */
+ ctx.checkReadUnits = true;
+ /* set limiter to its limit, if not over already */
+ if (ctx.readLimiter.getCurrentRate() < 100.0) {
+ ctx.readLimiter.setCurrentRate(100.0);
}
+ /* call retry handler to manage sleep/delay */
+ }
- logFine(logger, "Retryable exception: " +
- re.getMessage());
- /*
- * Handle automatic retries. If this does not throw an error,
- * then the delay (if any) will have been performed and the
- * request should be retried.
- *
- * If there have been too many retries this method will
- * throw the original exception.
- */
+ logFine(logger, "Retryable exception: " +
+ re.getMessage());
+ /*
+ * Handle automatic retries. If this does not throw an error,
+ * then the delay (if any) will have been performed and the
+ * request should be retried.
+ *
+ * If there have been too many retries this method will
+ * throw the original exception.
+ */
- kvRequest.addRetryException(re.getClass());
- handleRetry(re, kvRequest);
- kvRequest.incrementRetries();
- exception = re;
- continue;
- } catch (UnsupportedQueryVersionException uqve) {
- /* decrement query version and try again */
- if (decrementQueryVersion(queryVersionUsed) == true) {
- logFine(logger, "Got unsupported query version error " +
- "from server: decrementing query version to " +
- queryVersion + " and trying again.");
- continue;
- }
+ kvRequest.addRetryException(re.getClass());
+ delayMs = handleRetry(re, kvRequest);
+ retryable = true;
+ } catch (UnsupportedQueryVersionException uqve) {
+ /* decrement query version and try again */
+ if (decrementQueryVersion(ctx.queryVersionUsed) == true) {
+ logFine(logger, "Got unsupported query version error " +
+ "from server: decrementing query version to " +
+ queryVersion + " and trying again.");
+ retryable = true;
+ } else {
throw uqve;
- } catch (UnsupportedProtocolException upe) {
- /* decrement protocol version and try again */
- if (decrementSerialVersion(serialVersionUsed) == true) {
- /* Don't set this exception: it's misleading */
- /* exception = upe; */
- logFine(logger, "Got unsupported protocol error " +
- "from server: decrementing serial version to " +
- serialVersion + " and trying again.");
- continue;
- }
+ }
+ } catch (UnsupportedProtocolException upe) {
+ /* decrement protocol version and try again */
+ if (decrementSerialVersion(ctx.serialVersionUsed) == true) {
+ /* Don't set this exception: it's misleading */
+ /* exception = upe; */
+ logFine(logger, "Got unsupported protocol error " +
+ "from server: decrementing serial version to " +
+ serialVersion + " and trying again.");
+ retryable = true;
+ } else {
throw upe;
- } catch (NoSQLException nse) {
- kvRequest.setRateLimitDelayedMs(rateDelayedMs);
- statsControl.observeError(kvRequest);
- logFine(logger, "Client execute NoSQLException: " +
- nse.getMessage());
- throw nse; /* pass through */
- } catch (RuntimeException e) {
- kvRequest.setRateLimitDelayedMs(rateDelayedMs);
- statsControl.observeError(kvRequest);
- if (!kvRequest.getIsRefresh()) {
- /* don't log expected failures from refresh */
- logFine(logger, "Client execute runtime exception: " +
- e.getMessage());
- }
- throw e;
- } catch (IOException ioe) {
- String name = ioe.getClass().getName();
- logFine(logger, "Client execution IOException, name: " +
- name + ", message: " + ioe.getMessage());
- /*
- * An exception in the channel, e.g. the server may have
- * disconnected. Retry.
- */
- kvRequest.addRetryException(ioe.getClass());
- kvRequest.incrementRetries();
- exception = ioe;
-
- try {
- Thread.sleep(10);
- } catch (InterruptedException ie) {}
-
- continue;
- } catch (InterruptedException ie) {
- kvRequest.setRateLimitDelayedMs(rateDelayedMs);
- statsControl.observeError(kvRequest);
- logInfo(logger, "Client interrupted exception: " +
- ie.getMessage());
- /* this exception shouldn't retry -- direct throw */
- throw new NoSQLException("Request interrupted: " +
- ie.getMessage());
- } catch (ExecutionException ee) {
- /*
- * This can happen if a channel is bad in HttpClient.getChannel.
- * This happens if the channel is shut down by the server side
- * or the server (proxy) is restarted, etc. Treat it like
- * IOException above, but retry without waiting
- */
- String name = ee.getCause().getClass().getName();
- logFine(logger, "Client ExecutionException, name: " +
- name + ", message: " + ee.getMessage() + ", retrying");
+ }
+ } catch (NoSQLException nse) {
+ logFine(logger, "Client execute NoSQLException: " +
+ nse.getMessage());
+ throw nse;
+ } catch (RuntimeException e) {
+ if (!kvRequest.getIsRefresh()) {
+ /* don't log expected failures from refresh */
+ logFine(logger, "Client execute runtime exception: " +
+ e.getMessage());
+ }
+ } catch (IOException ioe) {
+ String name = ioe.getClass().getName();
+ logFine(logger, "Client execution IOException, name: " +
+ name + ", message: " + ioe.getMessage());
+ if (kvRequest.getNumRetries() == 0) {
+ retryable = true;
+ delayMs = 10;
+ }
+ } catch (InterruptedException ie) {
+ logInfo(logger, "Client interrupted exception: " +
+ ie.getMessage());
+ } catch (ExecutionException ee) {
+ /*
+ * This can happen if a channel is bad in HttpClient.getChannel.
+ * This happens if the channel is shut down by the server side
+ * or the server (proxy) is restarted, etc. Treat it like
+ * IOException above, but retry without waiting
+ */
+ String name = ee.getCause().getClass().getName();
+ logFine(logger, "Client ExecutionException, name: " +
+ name + ", message: " + ee.getMessage() + ", retrying");
- kvRequest.addRetryException(ee.getCause().getClass());
- kvRequest.incrementRetries();
- exception = ee.getCause();
- continue;
- } catch (TimeoutException te) {
- exception = te;
- logInfo(logger, "Timeout exception: " + te);
- break; /* fall through to exception below */
- } catch (Throwable t) {
- /*
- * this is likely an exception from Netty, perhaps a bad
- * connection. Retry.
- */
- /* Maybe make this logFine */
- String name = t.getClass().getName();
- logInfo(logger, "Client execute Throwable, name: " +
- name + "message: " + t.getMessage());
+ if (kvRequest.getNumRetries() == 0) {
+ retryable = true;
+ }
+ } catch (TimeoutException te) {
+ //TODO handle this
+ } catch (Throwable t) {
+ /*
+ * this is likely an exception from Netty, perhaps a bad
+ * connection. Retry.
+ */
+ /* Maybe make this logFine */
+ String name = t.getClass().getName();
+ logInfo(logger, "Client execute Throwable, name: " +
+ name + "message: " + t.getMessage());
+ }
- kvRequest.addRetryException(t.getClass());
+ if (retryable) {
+ if (!(actualCause instanceof UnsupportedProtocolException)
+ && !(actualCause instanceof UnsupportedQueryVersionException)) {
+ kvRequest.addRetryException(err.getClass());
kvRequest.incrementRetries();
- exception = t;
- continue;
- } finally {
- /*
- * Because the buffer.retain() is called after initialized, so
- * the reference count of buffer should be always > 0 here, just
- * call buffer.release(refCnt) to release it.
- */
- if (buffer != null) {
- buffer.release(buffer.refCnt());
- }
- if (responseHandler != null) {
- responseHandler.close();
- }
}
- } while (! timeoutRequest(startNanos, timeoutMs, exception));
- kvRequest.setRateLimitDelayedMs(rateDelayedMs);
+
+ CompletableFuture retryFuture = new CompletableFuture<>();
+ int finalDelayMs = delayMs;
+ taskExecutor.schedule(() -> {
+ /* Increment request-id for retry */
+ ctx.requestId = String.valueOf(ctx.nextIdSupplier.get());
+ executeWithRetry(ctx)
+ .whenComplete((res,e) -> {
+ kvRequest.addRetryDelayMs(finalDelayMs);
+ if (e != null) {
+ retryFuture.completeExceptionally(e);
+ } else {
+ retryFuture.complete(res);
+ }
+ });
+ }, delayMs, TimeUnit.MILLISECONDS);
+ return retryFuture;
+ }
+ /* No retry, complete with exception */
+ kvRequest.setRateLimitDelayedMs(ctx.rateDelayedMs.get());
statsControl.observeError(kvRequest);
- /*
- * If the request timed out in a single iteration, and the
- * timeout was fairly long, and there was no delay due to
- * rate limiting, reset the session cookie so the next request
- * may use a different server.
- */
- if (timeoutMs == thisIterationTimeoutMs &&
- timeoutMs >= 2000 &&
- rateDelayedMs == 0) {
- setSessionCookieValue(null);
- }
- throw new RequestTimeoutException(timeoutMs,
- requestClass + " timed out:" +
- (requestId.isEmpty() ? "" : " requestId=" + requestId) +
- " nextRequestId=" + nextRequestId() +
- " iterationTimeout=" + thisIterationTimeoutMs + "ms " +
- (kvRequest.getRetryStats() != null ?
- kvRequest.getRetryStats() : ""), exception);
+ return CompletableFuture.failedFuture(actualCause);
}
+ /**
+ * Helper method to create a CompletableFuture that completes after a delay.
+ * This is used for non-blocking asynchronous delays for rate limiting.
+ *
+ * @param delayMs The delay in milliseconds.
+ * @return A CompletableFuture that completes after the specified delay.
+ */
+ private CompletableFuture createDelayFuture(int delayMs) {
+ CompletableFuture delayFuture = new CompletableFuture<>();
+ if (delayMs > 0) {
+ taskExecutor.schedule(() -> delayFuture.complete(delayMs), delayMs,
+ TimeUnit.MILLISECONDS);
+ } else {
+ delayFuture.complete(delayMs); // Complete immediately if no delay
+ }
+ return delayFuture;
+ }
/**
* Calculate the timeout for the next iteration.
* This is basically the given timeout minus the time
@@ -1055,7 +1115,7 @@ private RateLimiter getQueryRateLimiter(Request request, boolean read) {
* the tablename from the request and apply rate limiting.
*/
String tableName = ((QueryRequest)request).getTableName();
- if (tableName == null || tableName == "") {
+ if (tableName == null || tableName.isEmpty()) {
return null;
}
@@ -1079,7 +1139,7 @@ private RateLimiter getQueryRateLimiter(Request request, boolean read) {
* @return the number of milliseconds delayed due to rate limiting
*/
private int consumeLimiterUnits(RateLimiter rl,
- long units, int timeoutMs) {
+ long units) {
if (rl == null || units <= 0) {
return 0;
@@ -1098,13 +1158,7 @@ private int consumeLimiterUnits(RateLimiter rl,
* better to avoid spikes in throughput and oscillation that
* can result from it.
*/
-
- try {
- return rl.consumeUnitsWithTimeout(units, timeoutMs, false);
- } catch (TimeoutException e) {
- /* Don't throw - operation succeeded. Just return timeoutMs. */
- return timeoutMs;
- }
+ return ((SimpleRateLimiter) rl).consumeExternally(units);
}
@@ -1141,11 +1195,10 @@ public boolean updateRateLimiters(String tableName, TableLimits limits) {
*/
/* allow tests to override this hardcoded setting */
- int durationSeconds = Integer.getInteger("test.rldurationsecs", 30)
- .intValue();
+ int durationSeconds = Integer.getInteger("test.rldurationsecs", 30);
- double RUs = (double)limits.getReadUnits();
- double WUs = (double)limits.getWriteUnits();
+ double RUs = limits.getReadUnits();
+ double WUs = limits.getWriteUnits();
/* if there's a specified rate limiter percentage, use that */
double rlPercent = config.getDefaultRateLimitingPercentage();
@@ -1188,26 +1241,26 @@ boolean timeoutRequest(long startNanos,
*
* @throws IOException
*/
- private short writeContent(ByteBuf content, Request kvRequest,
- short queryVersion)
+ private void writeContent(ByteBuf content, RequestContext ctx)
throws IOException {
-
+ final Request kvRequest = ctx.kvRequest;
final NettyByteOutputStream bos = new NettyByteOutputStream(content);
- final short versionUsed = serialVersion;
+ ctx.serialVersionUsed = serialVersion;
+ ctx.queryVersionUsed = queryVersion;
+
SerializerFactory factory = chooseFactory(kvRequest);
- factory.writeSerialVersion(versionUsed, bos);
+ factory.writeSerialVersion(ctx.serialVersionUsed, bos);
if (kvRequest instanceof QueryRequest ||
kvRequest instanceof PrepareRequest) {
kvRequest.createSerializer(factory).serialize(kvRequest,
- versionUsed,
- queryVersion,
+ ctx.serialVersionUsed,
+ ctx.queryVersionUsed,
bos);
} else {
kvRequest.createSerializer(factory).serialize(kvRequest,
- versionUsed,
+ ctx.serialVersionUsed,
bos);
}
- return versionUsed;
}
/**
@@ -1221,9 +1274,7 @@ private short writeContent(ByteBuf content, Request kvRequest,
final Result processResponse(HttpResponseStatus status,
HttpHeaders headers,
ByteBuf content,
- Request kvRequest,
- short serialVersionUsed,
- short queryVersionUsed) {
+ RequestContext ctx) {
if (!HttpResponseStatus.OK.equals(status)) {
processNotOKResponse(status, content);
@@ -1237,8 +1288,8 @@ final Result processResponse(HttpResponseStatus status,
Result res = null;
try (ByteInputStream bis = new NettyByteInputStream(content)) {
- res = processOKResponse(bis, kvRequest, serialVersionUsed,
- queryVersionUsed);
+ res = processOKResponse(bis, ctx.kvRequest, ctx.serialVersionUsed,
+ ctx.queryVersionUsed);
}
String sv = headers.get(SERVER_SERIAL_VERSION);
if (sv != null) {
@@ -1367,8 +1418,10 @@ private void setSessionCookie(HttpHeaders headers) {
}
}
- private synchronized void setSessionCookieValue(String pVal) {
- sessionCookie = pVal;
+ private void setSessionCookieValue(String pVal) {
+ ConcurrentUtil.synchronizedCall(this.lock, () -> {
+ sessionCookie = pVal;
+ });
}
/**
@@ -1418,20 +1471,22 @@ private void setTableNeedsRefresh(String tableName, boolean needsRefresh) {
* Query table limits and create rate limiters for a table in a
* short-lived background thread.
*/
- private synchronized void backgroundUpdateLimiters(String tableName,
- String compartmentId) {
- if (tableNeedsRefresh(tableName) == false) {
- return;
- }
- setTableNeedsRefresh(tableName, false);
+ private void backgroundUpdateLimiters(String tableName,
+ String compartmentId) {
+ ConcurrentUtil.synchronizedCall(this.lock, () -> {
+ if (!tableNeedsRefresh(tableName)) {
+ return;
+ }
+ setTableNeedsRefresh(tableName, false);
- try {
- threadPool.execute(() -> {
- updateTableLimiters(tableName, compartmentId);
- });
- } catch (RejectedExecutionException e) {
- setTableNeedsRefresh(tableName, true);
- }
+ try {
+ threadPool.execute(() -> {
+ updateTableLimiters(tableName, compartmentId);
+ });
+ } catch (RejectedExecutionException e) {
+ setTableNeedsRefresh(tableName, true);
+ }
+ });
}
/*
@@ -1447,7 +1502,7 @@ private void updateTableLimiters(String tableName, String compartmentId) {
try {
logFine(logger, "Starting GetTableRequest for table '" +
tableName + "'");
- res = (TableResult) this.execute(gtr);
+ res = (TableResult) this.execute(gtr).get();
} catch (Exception e) {
logFine(logger, "GetTableRequest for table '" +
tableName + "' returned exception: " + e.getMessage());
@@ -1475,7 +1530,7 @@ private void updateTableLimiters(String tableName, String compartmentId) {
}
- private void handleRetry(RetryableException re,
+ private int handleRetry(RetryableException re,
Request kvRequest) {
int numRetries = kvRequest.getNumRetries();
String msg = "Retry for request " +
@@ -1487,7 +1542,7 @@ private void handleRetry(RetryableException re,
logFine(logger, "Too many retries");
throw re;
}
- handler.delay(kvRequest, numRetries, re);
+ return handler.delayTime(kvRequest, numRetries, re);
}
private void logRetries(int numRetries, Throwable exception) {
@@ -1605,19 +1660,22 @@ StatsControl getStatsControl() {
* @return true: version was decremented
* false: already at lowest version number.
*/
- private synchronized boolean decrementSerialVersion(short versionUsed) {
- if (serialVersion != versionUsed) {
- return true;
- }
- if (serialVersion == V4) {
- serialVersion = V3;
- return true;
- }
- if (serialVersion == V3) {
- serialVersion = V2;
- return true;
- }
- return false;
+ private boolean decrementSerialVersion(short versionUsed) {
+ return ConcurrentUtil.synchronizedCall(this.lock,
+ () -> {
+ if (serialVersion != versionUsed) {
+ return true;
+ }
+ if (serialVersion == V4) {
+ serialVersion = V3;
+ return true;
+ }
+ if (serialVersion == V3) {
+ serialVersion = V2;
+ return true;
+ }
+ return false;
+ });
}
/**
@@ -1627,18 +1685,20 @@ private synchronized boolean decrementSerialVersion(short versionUsed) {
* @return true: version was decremented
* false: already at lowest version number.
*/
- private synchronized boolean decrementQueryVersion(short versionUsed) {
-
- if (queryVersion != versionUsed) {
- return true;
- }
+ private boolean decrementQueryVersion(short versionUsed) {
+ return ConcurrentUtil.synchronizedCall(this.lock,
+ () -> {
+ if (queryVersion != versionUsed) {
+ return true;
+ }
- if (queryVersion == QueryDriver.QUERY_V3) {
- return false;
- }
+ if (queryVersion == QueryDriver.QUERY_V3) {
+ return false;
+ }
- --queryVersion;
- return true;
+ --queryVersion;
+ return true;
+ });
}
/**
@@ -1772,38 +1832,44 @@ private boolean stringsEqualOrNull(String s1, String s2) {
* Add get, put, delete to cover all auth types
* This is synchronized to avoid 2 requests adding the same table
*/
- private synchronized void addRequestToRefreshList(Request request) {
- logFine(logger, "Adding table to request list: " +
- request.getCompartment() + ":" + request.getTableName());
- PutRequest pr =
- new PutRequest().setTableName(request.getTableName());
- pr.setCompartmentInternal(request.getCompartment());
- pr.setValue(badValue);
- pr.setIsRefresh(true);
- authRefreshRequests.add(pr);
- GetRequest gr =
- new GetRequest().setTableName(request.getTableName());
- gr.setCompartmentInternal(request.getCompartment());
- gr.setKey(badValue);
- gr.setIsRefresh(true);
- authRefreshRequests.add(gr);
- DeleteRequest dr =
- new DeleteRequest().setTableName(request.getTableName());
- dr.setCompartmentInternal(request.getCompartment());
- dr.setKey(badValue);
- dr.setIsRefresh(true);
- authRefreshRequests.add(dr);
+ private void addRequestToRefreshList(Request request) {
+ ConcurrentUtil.synchronizedCall(this.lock,
+ () -> {
+ logFine(logger, "Adding table to request list: " +
+ request.getCompartment() + ":" + request.getTableName());
+ PutRequest pr =
+ new PutRequest().setTableName(request.getTableName());
+ pr.setCompartmentInternal(request.getCompartment());
+ pr.setValue(badValue);
+ pr.setIsRefresh(true);
+ authRefreshRequests.add(pr);
+ GetRequest gr =
+ new GetRequest().setTableName(request.getTableName());
+ gr.setCompartmentInternal(request.getCompartment());
+ gr.setKey(badValue);
+ gr.setIsRefresh(true);
+ authRefreshRequests.add(gr);
+ DeleteRequest dr =
+ new DeleteRequest().setTableName(request.getTableName());
+ dr.setCompartmentInternal(request.getCompartment());
+ dr.setKey(badValue);
+ dr.setIsRefresh(true);
+ authRefreshRequests.add(dr);
+ });
}
/**
* @hidden
* for internal use
*/
- public synchronized void oneTimeMessage(String msg) {
- if (oneTimeMessages.add(msg) == false) {
- return;
- }
- logWarning(logger, msg);
+ public void oneTimeMessage(String msg) {
+ ConcurrentUtil.synchronizedCall(this.lock,
+ () -> {
+ if (oneTimeMessages.add(msg) == false) {
+ return;
+ }
+ logWarning(logger, msg);
+ });
}
private SerializerFactory chooseFactory(Request rq) {
@@ -1907,20 +1973,24 @@ public TopologyInfo getTopology() {
return topology;
}
- private synchronized int getTopoSeqNum() {
- return (topology == null ? -1 : topology.getSeqNum());
+ private int getTopoSeqNum() {
+ return ConcurrentUtil.synchronizedCall(this.lock,
+ () -> (topology == null ? -1 : topology.getSeqNum()));
}
- private synchronized void setTopology(TopologyInfo topo) {
+ private void setTopology(TopologyInfo topo) {
- if (topo == null) {
- return;
- }
+ ConcurrentUtil.synchronizedCall(this.lock,
+ () -> {
+ if (topo == null) {
+ return;
+ }
- if (topology == null || topology.getSeqNum() < topo.getSeqNum()) {
- topology = topo;
- trace("New topology: " + topo, 1);
- }
+ if (topology == null || topology.getSeqNum() < topo.getSeqNum()) {
+ topology = topo;
+ trace("New topology: " + topo, 1);
+ }
+ });
}
/*
diff --git a/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
index b29b039f..07230b57 100644
--- a/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
+++ b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
@@ -8,11 +8,14 @@
package oracle.nosql.driver.http;
import java.util.ArrayList;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
import java.util.logging.Logger;
import javax.net.ssl.SSLException;
import oracle.nosql.driver.AuthorizationProvider;
+import oracle.nosql.driver.NoSQLException;
import oracle.nosql.driver.NoSQLHandle;
import oracle.nosql.driver.NoSQLHandleConfig;
import oracle.nosql.driver.StatsControl;
@@ -41,6 +44,8 @@
import oracle.nosql.driver.ops.QueryResult;
import oracle.nosql.driver.ops.ReplicaStatsRequest;
import oracle.nosql.driver.ops.ReplicaStatsResult;
+import oracle.nosql.driver.ops.Request;
+import oracle.nosql.driver.ops.Result;
import oracle.nosql.driver.ops.SystemRequest;
import oracle.nosql.driver.ops.SystemResult;
import oracle.nosql.driver.ops.SystemStatusRequest;
@@ -176,40 +181,59 @@ private void configAuthProvider(Logger logger, NoSQLHandleConfig config) {
@Override
public DeleteResult delete(DeleteRequest request) {
- checkClient();
- return (DeleteResult) client.execute(request);
+ return executeSync(request);
}
@Override
public GetResult get(GetRequest request) {
- checkClient();
- return (GetResult) client.execute(request);
+ return executeSync(request);
}
@Override
public PutResult put(PutRequest request) {
- checkClient();
- return (PutResult) client.execute(request);
+ return executeSync(request);
}
@Override
public WriteMultipleResult writeMultiple(WriteMultipleRequest request) {
- checkClient();
- return (WriteMultipleResult) client.execute(request);
+ return executeSync(request);
}
@Override
public MultiDeleteResult multiDelete(MultiDeleteRequest request) {
- checkClient();
- return (MultiDeleteResult) client.execute(request);
+ return executeSync(request);
}
@Override
public QueryResult query(QueryRequest request) {
- checkClient();
- return (QueryResult) client.execute(request);
+ try {
+ return queryAsync(request).get();
+ } catch (InterruptedException ie) {
+ throw new NoSQLException("Request interrupted: " + ie.getMessage(), ie);
+ } catch (ExecutionException e) {
+ if (e.getCause() instanceof RuntimeException) {
+ throw ((RuntimeException) e.getCause());
+ }
+ throw new NoSQLException("ExecutionException: " + e.getMessage(),
+ e.getCause());
+ }
}
+ public CompletableFuture queryAsync(QueryRequest request) {
+
+ return client.execute(request)
+ .thenCompose(result -> {
+ if (!request.isSimpleQuery()) {
+ // TODO supplyAsync runs in fork-join pool.
+ // Change to dedicated pool
+ return CompletableFuture.supplyAsync(() -> result);
+ }
+ return CompletableFuture.completedFuture(result);
+ })
+ .thenApply(result -> ((QueryResult) result));
+ }
+
+
@Override
public QueryIterableResult queryIterable(QueryRequest request) {
checkClient();
@@ -218,14 +242,13 @@ public QueryIterableResult queryIterable(QueryRequest request) {
@Override
public PrepareResult prepare(PrepareRequest request) {
- checkClient();
- return (PrepareResult) client.execute(request);
+ return executeSync(request);
}
@Override
public TableResult tableRequest(TableRequest request) {
checkClient();
- TableResult res = (TableResult) client.execute(request);
+ TableResult res = executeSync(request);
/* update rate limiters, if table has limits */
client.updateRateLimiters(res.getTableName(), res.getTableLimits());
return res;
@@ -234,7 +257,7 @@ public TableResult tableRequest(TableRequest request) {
@Override
public TableResult getTable(GetTableRequest request) {
checkClient();
- TableResult res = (TableResult) client.execute(request);
+ TableResult res = executeSync(request);
/* update rate limiters, if table has limits */
client.updateRateLimiters(res.getTableName(), res.getTableLimits());
return res;
@@ -243,49 +266,42 @@ public TableResult getTable(GetTableRequest request) {
@Override
public SystemResult systemRequest(SystemRequest request) {
checkClient();
- return (SystemResult) client.execute(request);
+ return executeSync(request);
}
@Override
public SystemResult systemStatus(SystemStatusRequest request) {
- checkClient();
- return (SystemResult) client.execute(request);
+ return executeSync(request);
}
@Override
public TableUsageResult getTableUsage(TableUsageRequest request) {
- checkClient();
- return (TableUsageResult) client.execute(request);
+ return executeSync(request);
}
@Override
public ListTablesResult listTables(ListTablesRequest request) {
- checkClient();
- return (ListTablesResult) client.execute(request);
+ return executeSync(request);
}
@Override
public GetIndexesResult getIndexes(GetIndexesRequest request) {
- checkClient();
- return (GetIndexesResult) client.execute(request);
+ return executeSync(request);
}
@Override
public TableResult addReplica(AddReplicaRequest request) {
- checkClient();
- return (TableResult) client.execute(request);
+ return executeSync(request);
}
@Override
public TableResult dropReplica(DropReplicaRequest request) {
- checkClient();
- return (TableResult) client.execute(request);
+ return executeSync(request);
}
@Override
public ReplicaStatsResult getReplicaStats(ReplicaStatsRequest request) {
- checkClient();
- return (ReplicaStatsResult) client.execute(request);
+ return executeSync(request);
}
@Override
@@ -470,4 +486,20 @@ public void refresh(long refreshMs) {
client.doRefresh(refreshMs);
}
}
+
+ @SuppressWarnings("unchecked")
+ private T executeSync(Request request) {
+ checkClient();
+ try {
+ return (T) client.execute(request).get();
+ } catch (InterruptedException ie) {
+ throw new NoSQLException("Request interrupted: " + ie.getMessage(), ie);
+ } catch (ExecutionException e) {
+ if (e.getCause() instanceof RuntimeException) {
+ throw ((RuntimeException) e.getCause());
+ }
+ throw new NoSQLException("ExecutionException: " + e.getMessage(),
+ e.getCause());
+ }
+ }
}
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java b/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
index e4f7f800..b0696c41 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
@@ -12,9 +12,12 @@
import java.io.IOException;
import java.util.Map;
+import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.logging.Logger;
import io.netty.bootstrap.Bootstrap;
@@ -26,6 +29,7 @@
import io.netty.channel.pool.ChannelPoolHandler;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.Promise;
+import io.netty.util.concurrent.ScheduledFuture;
/**
* A class to manage and pool Netty Channels (connections). This is used
@@ -106,7 +110,7 @@ class ConnectionPool {
*/
@FunctionalInterface
interface KeepAlive {
- boolean keepAlive(Channel ch);
+ CompletableFuture keepAlive(Channel ch);
}
/**
@@ -208,45 +212,28 @@ final Future acquire() {
*/
final Future acquire(final Promise promise) {
try {
- while (true) {
- /* this *removes* the channel from the queue */
- final Channel channel = queue.pollFirst();
- if (channel == null) {
- /* need a new Channel */
- Bootstrap bs = bootstrap.clone();
- ChannelFuture fut = bs.connect();
- if (fut.isDone()) {
- notifyOnConnect(fut, promise);
- } else {
- fut.addListener(new ChannelFutureListener() {
- @Override
- public void operationComplete(
- ChannelFuture future) throws Exception {
- notifyOnConnect(future, promise);
- }
- });
- }
- return promise;
+ /* this *removes* the channel from the queue */
+ final Channel channel = queue.pollFirst();
+ if (channel == null) {
+ /* need a new Channel */
+ Bootstrap bs = bootstrap.clone();
+ ChannelFuture fut = bs.connect();
+ if (fut.isDone()) {
+ notifyOnConnect(fut,promise);
+ } else {
+ fut.addListener((ChannelFutureListener) future ->
+ notifyOnConnect(future,promise));
}
+ } else {
/*
* This logic must happen in the event loop
*/
EventLoop loop = channel.eventLoop();
if (loop.inEventLoop()) {
- if (checkChannel(channel, promise)) {
- /* bad channel, try again */
- continue;
- }
+ checkChannel(channel, promise);
} else {
- /*
- * Note: run() may be executed some time after this method
- * returns a promise. So the caller may have to wait a
- * few milliseconds for the promise to be completed
- * (successfully or not).
- */
loop.execute(() -> checkChannel(channel, promise));
}
- break;
}
} catch (Throwable t) {
promise.tryFailure(t);
@@ -329,26 +316,38 @@ private void notifyOnConnect(ChannelFuture future,
}
}
- private boolean checkChannel(final Channel channel,
- final Promise promise) {
-
- /*
- * If channel isn't healthy close it. It's been removed from
- * the queue
- */
- if (!channel.isActive()) {
- logFine(logger,
- "Inactive channel found, closing: " + channel);
- removeChannel(channel);
- promise.tryFailure(new IOException("inactive channel"));
- return true;
- }
+ private void checkChannel(final Channel channel,
+ final Promise promise) {
try {
- updateStats(channel, true);
- handler.channelAcquired(channel);
- } catch (Exception e) {} /* ignore */
- promise.setSuccess(channel);
- return false;
+ /*
+ * If channel isn't healthy close it. It's been removed from
+ * the queue
+ */
+ if (!channel.isActive()) {
+ logFine(logger,
+ "Inactive channel found, closing: " + channel);
+ removeChannel(channel);
+ /* retry channel acquire */
+ acquire(promise);
+ } else {
+ try {
+ updateStats(channel, true);
+ handler.channelAcquired(channel);
+ } catch (Exception e) {} /* ignore */
+ if (!promise.trySuccess(channel)) {
+ release(channel);
+ }
+ }
+ } catch (Throwable cause) {
+ if (channel != null) {
+ try {
+ channel.close();
+ } catch (Throwable t) {
+ promise.tryFailure(t);
+ }
+ }
+ promise.tryFailure(cause);
+ }
}
/**
@@ -455,7 +454,7 @@ int doKeepAlive(int keepAlivePeriod) {
* Don't remove a channel from the queue until it's clear that
* it will be used
*/
- int numSent = 0;
+ AtomicInteger numSent = new AtomicInteger();
for (Channel ch : queue) {
if (!ch.isActive()) {
continue;
@@ -473,17 +472,22 @@ int doKeepAlive(int keepAlivePeriod) {
}
logFine(logger,
"Sending keepalive on channel " + ch + ", stats: " + cs);
- boolean didKeepalive = keepAlive.keepAlive(ch);
- if (!didKeepalive) {
- logFine(logger,
- "Keepalive failed on channel " + ch +
- ", removing from pool");
- removeChannel(ch);
- continue;
- }
- cs.acquired(); /* update lastAcquired time */
- numSent++;
- queue.addFirst(ch);
+ keepAlive.keepAlive(ch).handle((didKeepalive, err) -> {
+ if (err != null) {
+ // TODO log err
+ logFine(logger,
+ "Keepalive failed on channel "
+ + ch
+ + ", removing from pool");
+ removeChannel(ch);
+ } else {
+ cs.acquired(); /* update lastAcquired time */
+ numSent.getAndIncrement();
+ //TODO is this operation safe inside for-each?
+ queue.addFirst(ch);
+ }
+ return null;
+ });
}
/*
* channels that are not used but have been used within the
@@ -494,9 +498,9 @@ int doKeepAlive(int keepAlivePeriod) {
break;
}
}
- validatePool("doKeepAlive");
-
- return numSent;
+ //TODO how to validate this without blocking?
+ //validatePool("doKeepAlive");
+ return numSent.get();
}
private void validatePool(final String caller) {
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
index 492e62b7..b7e684b1 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
@@ -17,7 +17,7 @@
import static oracle.nosql.driver.util.LogUtil.logWarning;
import java.io.IOException;
-import java.util.concurrent.ExecutionException;
+import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.logging.Logger;
@@ -29,14 +29,17 @@
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.http.DefaultFullHttpRequest;
+import io.netty.handler.codec.http.FullHttpResponse;
import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.ssl.SslContext;
+import io.netty.handler.timeout.ReadTimeoutHandler;
import io.netty.util.AttributeKey;
import io.netty.util.concurrent.Future;
/*
* If this code is ever made generic, the proxy information obtained
* from this config needs to be abstracted to a generic class.
*/
+import io.netty.util.concurrent.FutureListener;
import oracle.nosql.driver.NoSQLHandleConfig;
/**
@@ -84,8 +87,11 @@ public class HttpClient {
static final int DEFAULT_HANDSHAKE_TIMEOUT_MS = 3000;
static final int DEFAULT_MIN_POOL_SIZE = 2; // min pool size
- static final AttributeKey STATE_KEY =
- AttributeKey.valueOf("rqstate");
+ /* AttributeKey to attach a CompletableFuture to the Channel,
+ * allowing the HttpResponseHandler to signal completion.
+ */
+ public static final AttributeKey> STATE_KEY =
+ AttributeKey.valueOf("rqstate");
//private final FixedChannelPool pool;
private final ConnectionPool pool;
@@ -268,7 +274,7 @@ private HttpClient(String host,
/* this is the main request client */
pool.setKeepAlive(new ConnectionPool.KeepAlive() {
@Override
- public boolean keepAlive(Channel ch) {
+ public CompletableFuture keepAlive(Channel ch) {
return doKeepAlive(ch);
}
});
@@ -388,71 +394,73 @@ public void shutdown() {
syncUninterruptibly();
}
- public Channel getChannel(int timeoutMs)
- throws InterruptedException, ExecutionException, TimeoutException {
-
- long startMs = System.currentTimeMillis();
- long now = startMs;
- int retries = 0;
-
- while (true) {
- long msDiff = now - startMs;
+// public Channel getChannel(int timeoutMs)
+// throws InterruptedException, ExecutionException, TimeoutException {
+//
+// long startMs = System.currentTimeMillis();
+// long now = startMs;
+// int retries = 0;
+//
+// while (true) {
+// long msDiff = now - startMs;
+//
+// /* retry loop with at most (retryInterval) ms timeouts */
+// long thisTimeoutMs = (timeoutMs - msDiff);
+// if (thisTimeoutMs <= 0) {
+// String msg = "Timed out trying to acquire channel";
+// logInfo(logger, "HttpClient " + name + " " + msg);
+// throw new TimeoutException(msg);
+// }
+// if (thisTimeoutMs > acquireRetryIntervalMs) {
+// thisTimeoutMs = acquireRetryIntervalMs;
+// }
+// Future fut = pool.acquire();
+// Channel retChan = null;
+// try {
+// retChan = fut.get(thisTimeoutMs, TimeUnit.MILLISECONDS);
+// } catch (TimeoutException e) {
+// if (retries == 0) {
+// logFine(logger, "Timed out after " +
+// (System.currentTimeMillis() - startMs) +
+// "ms trying to acquire channel: retrying");
+// }
+// /* fall through */
+// }
+//
+// /*
+// * Ensure that the channel is in good shape. retChan is null
+// * on a timeout exception from above; that path will retry.
+// */
+// if (retChan != null) {
+// if (fut.isSuccess() && retChan.isActive()) {
+// /*
+// * Clear out any previous state. The channel should not
+// * have any state associated with it, but this code is here
+// * just in case it does.
+// */
+// if (retChan.attr(STATE_KEY).get() != null) {
+// if (isFineEnabled(logger)) {
+// logFine(logger,
+// "HttpClient acquired a channel with " +
+// "a still-active state: clearing.");
+// }
+// retChan.attr(STATE_KEY).set(null);
+// }
+// return retChan;
+// }
+// logFine(logger,
+// "HttpClient " + name + ", acquired an inactive " +
+// "channel, releasing it and retrying, reason: " +
+// fut.cause());
+// releaseChannel(retChan);
+// }
+// /* reset "now" and increment retries */
+// now = System.currentTimeMillis();
+// retries++;
+// }
+// }
- /* retry loop with at most (retryInterval) ms timeouts */
- long thisTimeoutMs = (timeoutMs - msDiff);
- if (thisTimeoutMs <= 0) {
- String msg = "Timed out trying to acquire channel";
- logInfo(logger, "HttpClient " + name + " " + msg);
- throw new TimeoutException(msg);
- }
- if (thisTimeoutMs > acquireRetryIntervalMs) {
- thisTimeoutMs = acquireRetryIntervalMs;
- }
- Future fut = pool.acquire();
- Channel retChan = null;
- try {
- retChan = fut.get(thisTimeoutMs, TimeUnit.MILLISECONDS);
- } catch (TimeoutException e) {
- if (retries == 0) {
- logFine(logger, "Timed out after " +
- (System.currentTimeMillis() - startMs) +
- "ms trying to acquire channel: retrying");
- }
- /* fall through */
- }
- /*
- * Ensure that the channel is in good shape. retChan is null
- * on a timeout exception from above; that path will retry.
- */
- if (retChan != null) {
- if (fut.isSuccess() && retChan.isActive()) {
- /*
- * Clear out any previous state. The channel should not
- * have any state associated with it, but this code is here
- * just in case it does.
- */
- if (retChan.attr(STATE_KEY).get() != null) {
- if (isFineEnabled(logger)) {
- logFine(logger,
- "HttpClient acquired a channel with " +
- "a still-active state: clearing.");
- }
- retChan.attr(STATE_KEY).set(null);
- }
- return retChan;
- }
- logFine(logger,
- "HttpClient " + name + ", acquired an inactive " +
- "channel, releasing it and retrying, reason: " +
- fut.cause());
- releaseChannel(retChan);
- }
- /* reset "now" and increment retries */
- now = System.currentTimeMillis();
- retries++;
- }
- }
public void releaseChannel(Channel channel) {
/* Clear any response handler state from channel before releasing it */
@@ -479,89 +487,116 @@ public void removeChannel(Channel channel) {
* use for the (asynchronous) response.
*
* @param request the request
- * @param handler the response handler
- * @param channel the Channel to use for the request/response
*
* @throws IOException if there is a network problem (bad channel). Such
* exceptions can be retried.
*/
- public void runRequest(HttpRequest request,
- ResponseHandler handler,
- Channel channel)
-
- throws IOException {
+ public CompletableFuture runRequest(HttpRequest request,
+ int timeoutMs) {
+
+ CompletableFuture responseFuture = new CompletableFuture<>();
+ long deadlineNs = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(timeoutMs);
+ pool.acquire().addListener((FutureListener) channelFuture -> {
+ if (channelFuture.isSuccess()) {
+ Channel channel = channelFuture.getNow();
+
+ if (isTimeout(deadlineNs)) {
+ String msg = "Timed out trying to acquire channel";
+ logInfo(logger, "HttpClient " + name + " " + msg);
+ responseFuture.completeExceptionally(new TimeoutException(msg));
+ releaseChannel(channel);
+ return;
+ }
- /*
- * If the channel goes bad throw IOE to allow the caller to retry
- */
- if (!channel.isActive()) {
- String msg = "HttpClient " + name + ", runRequest, channel " +
- channel + " is not active: ";
- logWarning(logger, msg);
- throw new IOException(msg);
- }
+ long remainingTimeoutNs = deadlineNs - System.nanoTime();
+ long remainingTimeoutMs = Math.max(1, TimeUnit.NANOSECONDS.toMillis(remainingTimeoutNs));
+
+ // Execute the request on the acquired channel
+ CompletableFuture requestExecutionFuture =
+ runRequest(request, channel, remainingTimeoutMs);
+
+ // When the request execution future completes (either successfully or exceptionally),
+ // complete the public responseFuture and ensure the channel is released back to the pool.
+ requestExecutionFuture.whenComplete((response, throwable) -> {
+ pool.release(channel); // Always release the channel
+ if (throwable != null) {
+ responseFuture.completeExceptionally(throwable);
+ } else {
+ responseFuture.complete(response);
+ }
+ });
+ } else {
+ /* channel acquisition failed */
+ //TODO send proper exception
+ responseFuture.completeExceptionally(channelFuture.cause());
+ }
+ });
+ return responseFuture;
+ }
- RequestState state = new RequestState(handler);
- channel.attr(STATE_KEY).set(state);
+ public CompletableFuture runRequest(HttpRequest request,
+ Channel channel,
+ long timeoutMs) {
+ CompletableFuture responseFuture = new CompletableFuture<>();
+ // Attach the CompletableFuture to the channel's attributes
+ channel.attr(STATE_KEY).set(responseFuture);
+
+ // Add timeout handler to the pipeline
+ channel.pipeline().addFirst(
+ new ReadTimeoutHandler(timeoutMs, TimeUnit.MILLISECONDS));
+
+ // Write the request to the channel and flush it
+ channel.writeAndFlush(request)
+ .addListener((ChannelFutureListener) writeFuture -> {
+ if (!writeFuture.isSuccess()) {
+ // If write fails, complete the future exceptionally
+ channel.attr(STATE_KEY).set(null);
+ responseFuture.completeExceptionally(writeFuture.cause());
+ }
+ });
+ return responseFuture;
+ }
- /*
- * Send the request. If the operation fails set the exception
- * on the ResponseHandler where it will be thrown synchronously to
- * users of that object. operationComplete will likely be called in
- * another thread.
- */
- channel.writeAndFlush(request).
- addListener((ChannelFutureListener) future -> {
- if (!future.isSuccess()) {
- /* handleException logs this exception */
- handler.handleException("HttpClient: send failed",
- future.cause());
- }
- });
+ private static boolean isTimeout(long deadlineNs) {
+ return System.nanoTime() >= deadlineNs;
}
/**
* Use HTTP HEAD method to refresh the channel
*/
- boolean doKeepAlive(Channel ch) {
+ CompletableFuture doKeepAlive(Channel ch) {
+ CompletableFuture future = new CompletableFuture<>();
+ // TODO Handle timeout
final int keepAliveTimeout = 3000; /* ms */
- ResponseHandler responseHandler =
- new ResponseHandler(this, logger, ch);
- try {
- final HttpRequest request =
+ final HttpRequest request =
new DefaultFullHttpRequest(HTTP_1_1, HEAD, "/");
-
- /*
- * All requests need a HOST header or the LBaaS (nginx) or
- * other server may reject them and close the connection
- */
- request.headers().add(HOST, host);
- runRequest(request, responseHandler, ch);
- boolean isTimeout = responseHandler.await(keepAliveTimeout);
- if (isTimeout) {
- logFine(logger,
- "Timeout on keepalive HEAD request on channel " + ch);
- return false;
- }
+ /*
+ * All requests need a HOST header or the LBaaS (nginx) or
+ * other server may reject them and close the connection
+ */
+ request.headers().add(HOST, host);
+ runRequest(request, ch, keepAliveTimeout)
+ .thenApply(fullHttpResponse -> {
/*
* LBaaS will return a non-200 status but that is expected as the
* path "/" does not map to the service. This is ok because all that
* matters is that the connection remain alive.
*/
- String conn = responseHandler.getHeaders().get(CONNECTION);
- if (conn == null || !"keep-alive".equalsIgnoreCase(conn)) {
+ String conn = fullHttpResponse.headers().get(CONNECTION);
+ if (!"keep-alive".equalsIgnoreCase(conn)) {
logFine(logger,
- "Keepalive HEAD request did not return keep-alive " +
- "in connection header, is: " + conn);
+ "Keepalive HEAD request did not return keep-alive "
+ + "in connection header, is: " + conn);
}
-
- return true;
- } catch (Throwable t) {
- logFine(logger, "Exception sending HTTP HEAD: " + t);
- } finally {
- responseHandler.releaseResponse();
- }
- /* something went wrong, caller is responsible for disposition */
- return false;
+ return fullHttpResponse;
+ }).whenComplete((res, err) -> {
+ res.release();
+ if (err != null) {
+ future.complete(false);
+ } else {
+ future.complete(true);
+ }
+ });
+ return future;
}
}
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClientHandler.java b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClientHandler.java
index 649d851a..ba8626da 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClientHandler.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClientHandler.java
@@ -7,18 +7,23 @@
package oracle.nosql.driver.httpclient;
+import static oracle.nosql.driver.httpclient.HttpClient.STATE_KEY;
import static oracle.nosql.driver.util.HttpConstants.REQUEST_ID_HEADER;
import static oracle.nosql.driver.util.LogUtil.isFineEnabled;
import static oracle.nosql.driver.util.LogUtil.logFine;
import static oracle.nosql.driver.util.LogUtil.logWarning;
import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeoutException;
import java.util.logging.Logger;
import io.netty.channel.ChannelHandler.Sharable;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.timeout.ReadTimeoutException;
+import io.netty.handler.timeout.ReadTimeoutHandler;
/**
*
@@ -34,8 +39,8 @@ public class HttpClientHandler extends ChannelInboundHandlerAdapter {
@Override
public void channelRead(ChannelHandlerContext ctx, Object msg) {
- final RequestState state =
- ctx.channel().attr(HttpClient.STATE_KEY).get();
+ final CompletableFuture responseFuture =
+ ctx.channel().attr(STATE_KEY).getAndSet(null);
/*
* TODO/think about:
@@ -44,10 +49,14 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) {
* o redirects
*/
+ if (ctx.pipeline().get(ReadTimeoutHandler.class) != null) {
+ ctx.pipeline().remove(ReadTimeoutHandler.class);
+ }
+
if (msg instanceof FullHttpResponse) {
FullHttpResponse fhr = (FullHttpResponse) msg;
- if (state == null) {
+ if (responseFuture == null) {
/*
* This message came in after the client was done processing
* a request in a different thread.
@@ -60,46 +69,41 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) {
requestId = "(none)";
}
logFine(logger, "Discarding message with no response " +
- "handler. requestId=" + requestId);
+ "handler. requestId=" + requestId);
}
fhr.release();
- return;
+ } else {
+ responseFuture.complete(fhr);
}
-
- state.setResponse(fhr);
-
- /*
- * Notify the response handler
- */
- state.getHandler().receive(state);
-
- return;
+ } else {
+ logWarning(logger,
+ "HttpClientHandler, response not FullHttpResponse: " +
+ msg.getClass());
}
- logWarning(logger,
- "HttpClientHandler, response not FullHttpResponse: " +
- msg.getClass());
}
@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
- final RequestState state =
- ctx.channel().attr(HttpClient.STATE_KEY).get();
- if (state != null) {
+ final CompletableFuture responseFuture =
+ ctx.channel().attr(STATE_KEY).getAndSet(null);
+ if (responseFuture != null) {
/* handleException logs */
- state.getHandler().handleException("HttpClientHandler read failed",
- cause);
+ logFine(logger, "HttpClientHandler read failed, cause: " + cause);
+ responseFuture.completeExceptionally(cause);
}
ctx.close();
}
@Override
- public void channelInactive(ChannelHandlerContext ctx) throws Exception {
- final RequestState state =
- ctx.channel().attr(HttpClient.STATE_KEY).get();
+ public void channelInactive(ChannelHandlerContext ctx) {
+ final CompletableFuture responseFuture =
+ ctx.channel().attr(STATE_KEY).getAndSet(null);
/* handleException logs */
- if (state != null) {
+ if (responseFuture != null && !responseFuture.isDone()) {
String msg = "Channel is inactive: " + ctx.channel();
- state.getHandler().handleException(msg, new IOException(msg));
+ Throwable cause = new IOException(msg);
+ logFine(logger, msg + ", cause: " + cause);
+ responseFuture.completeExceptionally(cause);
}
/* should the context be closed? */
ctx.close();
diff --git a/driver/src/main/java/oracle/nosql/driver/iam/FederationRequestHelper.java b/driver/src/main/java/oracle/nosql/driver/iam/FederationRequestHelper.java
index 51ab9219..cff84e2a 100644
--- a/driver/src/main/java/oracle/nosql/driver/iam/FederationRequestHelper.java
+++ b/driver/src/main/java/oracle/nosql/driver/iam/FederationRequestHelper.java
@@ -18,6 +18,7 @@
import java.nio.charset.StandardCharsets;
import java.util.Date;
import java.util.Set;
+import java.util.concurrent.ExecutionException;
import java.util.logging.Logger;
import oracle.nosql.driver.httpclient.HttpClient;
@@ -53,10 +54,17 @@ static String getSecurityToken(HttpClient client,
byte[] payloadByte = new byte[buf.remaining()];
buf.get(payloadByte);
- HttpResponse response = HttpRequestUtil.doPostRequest(
- client, endpoint.toString(),
- headers(tenantId, endpoint, payloadByte, pair, logger),
- payloadByte, timeoutMs, logger);
+ HttpResponse response = null;
+ try {
+ response = HttpRequestUtil.doPostRequest(
+ client, endpoint.toString(),
+ headers(tenantId, endpoint, payloadByte, pair, logger),
+ payloadByte, timeoutMs, logger).get();
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ } catch (ExecutionException e) {
+ throw new RuntimeException(e);
+ }
int responseCode = response.getStatusCode();
if (responseCode > 299) {
diff --git a/driver/src/main/java/oracle/nosql/driver/iam/InstanceMetadataHelper.java b/driver/src/main/java/oracle/nosql/driver/iam/InstanceMetadataHelper.java
index f3e79f58..8bf90d4d 100644
--- a/driver/src/main/java/oracle/nosql/driver/iam/InstanceMetadataHelper.java
+++ b/driver/src/main/java/oracle/nosql/driver/iam/InstanceMetadataHelper.java
@@ -13,6 +13,7 @@
import static oracle.nosql.driver.util.HttpConstants.CONTENT_TYPE;
import java.io.IOException;
+import java.util.concurrent.ExecutionException;
import java.util.logging.Logger;
import oracle.nosql.driver.httpclient.HttpClient;
@@ -67,7 +68,7 @@ static InstanceMetadata fetchMetadata(int timeout, Logger logger) {
logger);
HttpRequestUtil.HttpResponse response = HttpRequestUtil.doGetRequest
- (client, instanceMDURL, headers(), timeout, logger);
+ (client, instanceMDURL, headers(), timeout, logger).get();
int status = response.getStatusCode();
if (status == 404) {
@@ -76,14 +77,14 @@ static InstanceMetadata fetchMetadata(int timeout, Logger logger) {
baseMetadataURL = FALLBACK_METADATA_SERVICE_URL;
instanceMDURL = getInstanceMetadaURL(baseMetadataURL);
response = HttpRequestUtil.doGetRequest
- (client, instanceMDURL, headers(), timeout, logger);
+ (client, instanceMDURL, headers(), timeout, logger).get();
if (response.getStatusCode() != 200) {
throw new IllegalStateException(
String.format("Unable to get federation URL from" +
"instance metadata " + METADATA_SERVICE_BASE_URL +
" or fallback to " + FALLBACK_METADATA_SERVICE_URL +
", status code: %d, output: %s",
- response.getOutput()));
+ response.getStatusCode(), response.getOutput()));
}
} else if (status != 200) {
throw new IllegalStateException(
@@ -98,6 +99,10 @@ static InstanceMetadata fetchMetadata(int timeout, Logger logger) {
String insRegion = findRegion(response.getOutput());
logTrace(logger, "Instance region " + insRegion);
return new InstanceMetadata(insRegion, baseMetadataURL);
+ } catch (ExecutionException e) {
+ throw new RuntimeException(e);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
} finally {
if (client != null) {
client.shutdown();
diff --git a/driver/src/main/java/oracle/nosql/driver/iam/OkeWorkloadIdentityProvider.java b/driver/src/main/java/oracle/nosql/driver/iam/OkeWorkloadIdentityProvider.java
index 39b7a55a..6ad8ddb3 100644
--- a/driver/src/main/java/oracle/nosql/driver/iam/OkeWorkloadIdentityProvider.java
+++ b/driver/src/main/java/oracle/nosql/driver/iam/OkeWorkloadIdentityProvider.java
@@ -26,6 +26,7 @@
import java.security.interfaces.RSAPublicKey;
import java.util.Base64;
import java.util.Map;
+import java.util.concurrent.ExecutionException;
import java.util.logging.Logger;
import javax.net.ssl.SSLException;
@@ -233,9 +234,16 @@ private String getSecurityToken() {
private String getSecurityTokenFromProxymux(String requestId,
String saToken,
byte[] payloadByte) {
- HttpRequestUtil.HttpResponse response = HttpRequestUtil.doPostRequest(
- okeTokenClient, tokenURL.toString(), headers(saToken, requestId),
- payloadByte, timeoutMs, logger);
+ HttpRequestUtil.HttpResponse response = null;
+ try {
+ response = HttpRequestUtil.doPostRequest(
+ okeTokenClient, tokenURL.toString(), headers(saToken, requestId),
+ payloadByte, timeoutMs, logger).get();
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ } catch (ExecutionException e) {
+ throw new RuntimeException(e);
+ }
final int responseCode = response.getStatusCode();
String responseOutput = response.getOutput();
diff --git a/driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java b/driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java
index bc2f76ec..10c9afe9 100644
--- a/driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java
+++ b/driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java
@@ -26,6 +26,9 @@
import java.util.Date;
import java.util.Timer;
import java.util.TimerTask;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.locks.ReentrantLock;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -38,6 +41,7 @@
import oracle.nosql.driver.ops.Request;
import io.netty.handler.codec.http.HttpHeaders;
+import oracle.nosql.driver.util.ConcurrentUtil;
/**
* Cloud service only.
@@ -159,6 +163,7 @@ public class SignatureProvider
private String serviceHost;
private Region region;
private Logger logger;
+ private final ReentrantLock lock = new ReentrantLock();
/**
* A callback interface called when the signature is refreshed. This
@@ -858,16 +863,29 @@ public SignatureProvider(AuthenticationProfileProvider profileProvider,
@Override
public String getAuthorizationString(Request request) {
- if (serviceHost == null) {
- throw new IllegalArgumentException(
- "Unable to find service host, use setServiceHost " +
- "to load from NoSQLHandleConfig");
+ try {
+ return getAuthorizationStringAsync(request).get();
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ } catch (ExecutionException e) {
+ throw new RuntimeException(e);
}
- SignatureDetails sigDetails = getSignatureDetails(request);
- if (sigDetails != null) {
- return sigDetails.getSignatureHeader();
+ }
+
+ @Override
+ public CompletableFuture getAuthorizationStringAsync(Request request) {
+ if (serviceHost == null) {
+ CompletableFuture.failedFuture(new IllegalArgumentException(
+ "Unable to find service host, use setServiceHost " +
+ "to load from NoSQLHandleConfig"));
}
- return null;
+
+ return getSignatureDetails(request).thenApply(signatureDetails -> {
+ if (signatureDetails == null) {
+ return null;
+ }
+ return signatureDetails.getSignatureHeader();
+ });
}
@Override
@@ -875,45 +893,65 @@ public void setRequiredHeaders(String authString,
Request request,
HttpHeaders headers,
byte[] content) {
-
- SignatureDetails sigDetails = (content != null) ?
- getSignatureWithContent(request, headers, content):
- getSignatureDetails(request);
- if (sigDetails == null) {
- return;
+ try {
+ setRequiredHeadersAsync(authString, request, headers, content).get();
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ } catch (ExecutionException e) {
+ throw new RuntimeException(e);
}
- headers.add(AUTHORIZATION, sigDetails.getSignatureHeader());
- headers.add(DATE, sigDetails.getDate());
+ }
- final String token = getDelegationToken(request);
- if (token != null) {
- headers.add(OBO_TOKEN_HEADER, token);
- }
- String compartment = request.getCompartment();
- if (compartment == null) {
- /*
- * If request doesn't has compartment id, set the tenant id as the
- * default compartment, which is the root compartment in IAM if
- * using user principal. If using an instance principal this
- * value is null.
- */
- compartment = getTenantOCID();
- }
+ @Override
+ public CompletableFuture setRequiredHeadersAsync(String authString,
+ Request request,
+ HttpHeaders headers,
+ byte[] content) {
- if (compartment != null) {
- headers.add(REQUEST_COMPARTMENT_ID, compartment);
+ CompletableFuture sigDetailsFuture;
+ if (content != null) {
+ sigDetailsFuture = getSignatureWithContent(request, headers, content);
} else {
- throw new IllegalArgumentException(
- "Compartment is null. When authenticating using an " +
- "Instance Principal the compartment for the operation " +
- "must be specified.");
+ sigDetailsFuture = getSignatureDetails(request);
}
+
+ return sigDetailsFuture.thenAccept(sigDetails -> {
+ if (sigDetails != null) {
+ headers.add(AUTHORIZATION, sigDetails.getSignatureHeader());
+ headers.add(DATE, sigDetails.getDate());
+ final String token = getDelegationToken(request);
+ if (token != null) {
+ headers.add(OBO_TOKEN_HEADER, token);
+ }
+ String compartment = request.getCompartment();
+ if (compartment == null) {
+ /*
+ * If request doesn't has compartment id, set the tenant id as the
+ * default compartment, which is the root compartment in IAM if
+ * using user principal. If using an instance principal this
+ * value is null.
+ */
+ compartment = getTenantOCID();
+ }
+ if (compartment != null) {
+ headers.add(REQUEST_COMPARTMENT_ID, compartment);
+ } else {
+ throw new IllegalArgumentException(
+ "Compartment is null. When authenticating using an " +
+ "Instance Principal the compartment for the operation " +
+ "must be specified.");
+ }
+ }
+ });
}
@Override
- public synchronized void flushCache() {
- currentSigDetails = null;
- refreshSigDetails = null;
+ public void flushCache() {
+ ConcurrentUtil.synchronizedCall(lock,
+ () -> {
+ currentSigDetails = null;
+ refreshSigDetails = null;
+ });
}
/**
@@ -1040,115 +1078,128 @@ private void logMessage(Level level, String msg) {
}
}
- private SignatureDetails getSignatureDetails(Request request) {
+ private CompletableFuture getSignatureDetails(Request request) {
SignatureDetails sigDetails =
(request.getIsRefresh() ? refreshSigDetails : currentSigDetails);
if (sigDetails != null) {
- return sigDetails;
+ return CompletableFuture.completedFuture(sigDetails);
}
if (request.getIsRefresh()) {
/* try current details before failing */
sigDetails = currentSigDetails;
if (sigDetails != null) {
- return sigDetails;
+ return CompletableFuture.completedFuture(sigDetails);
}
}
return getSignatureDetailsForCache(false);
}
- private SignatureDetails getSignatureWithContent(Request request,
- HttpHeaders headers,
- byte[] content) {
- return getSignatureDetailsInternal(false, request, headers, content);
+ private CompletableFuture getSignatureWithContent(Request request,
+ HttpHeaders headers,
+ byte[] content) {
+ return getSignatureDetailsAsync(false, request, headers, content);
+ }
+
+ private CompletableFuture getSignatureDetailsForCache(boolean isRefresh) {
+ return getSignatureDetailsAsync(isRefresh,
+ null /* request */,
+ null /* headers */,
+ null /* content */);
}
- synchronized SignatureDetails
- getSignatureDetailsForCache(boolean isRefresh) {
- return getSignatureDetailsInternal(isRefresh,
- null /* request */,
- null /* headers */,
- null /* content */);
+ private CompletableFuture
+ getSignatureDetailsAsync(boolean isRefresh,
+ Request request,
+ HttpHeaders headers,
+ byte[] content) {
+ return CompletableFuture.supplyAsync(() ->
+ getSignatureDetailsInternal(isRefresh, request, headers, content));
}
/* visible for testing */
- synchronized SignatureDetails
+ SignatureDetails
getSignatureDetailsInternal(boolean isRefresh,
Request request,
HttpHeaders headers,
byte[] content) {
- /*
- * add one minute to the current time, so that any caching is
- * effective over a more valid time period.
- */
- long nowPlus = System.currentTimeMillis() + 60_000L;
- String date = createFormatter().format(new Date(nowPlus));
- String keyId = provider.getKeyId();
-
- /*
- * Security token based providers may refresh the security token
- * and associated private key in above getKeyId() method, reload
- * private key to PrivateKeyProvider to avoid a mismatch, which
- * will create an invalid signature, cause authentication error.
- */
- if (provider instanceof SecurityTokenBasedProvider) {
- privateKeyProvider.reload(provider.getPrivateKey(),
- provider.getPassphraseCharacters());
- }
- String signature;
+ lock.lock();
try {
- signature = sign(signingContent(date, request, headers, content),
- privateKeyProvider.getKey());
- } catch (Exception e) {
- logMessage(Level.SEVERE, "Error signing request " + e.getMessage());
- return null;
- }
+ /*
+ * add one minute to the current time, so that any caching is
+ * effective over a more valid time period.
+ */
+ long nowPlus = System.currentTimeMillis() + 60_000L;
+ String date = createFormatter().format(new Date(nowPlus));
+ String keyId = provider.getKeyId();
- String token = getDelegationToken(request);
- String signingHeader;
- if (content != null) {
- signingHeader = (token == null)
- ? SIGNING_HEADERS_WITH_CONTENT :
- SIGNING_HEADERS_WITH_CONTENT_OBO;
- } else {
- signingHeader = (token == null)
- ? SIGNING_HEADERS : SIGNING_HEADERS_WITH_OBO;
- }
+ /*
+ * Security token based providers may refresh the security token
+ * and associated private key in above getKeyId() method, reload
+ * private key to PrivateKeyProvider to avoid a mismatch, which
+ * will create an invalid signature, cause authentication error.
+ */
+ if (provider instanceof SecurityTokenBasedProvider) {
+ privateKeyProvider.reload(provider.getPrivateKey(),
+ provider.getPassphraseCharacters());
+ }
+ String signature;
+ try {
+ signature = sign(signingContent(date, request, headers, content),
+ privateKeyProvider.getKey());
+ } catch (Exception e) {
+ logMessage(Level.SEVERE, "Error signing request " + e.getMessage());
+ return null;
+ }
- String sigHeader = String.format(SIGNATURE_HEADER_FORMAT,
- signingHeader,
- keyId,
- RSA,
- signature,
- SINGATURE_VERSION);
- SignatureDetails sigDetails = new SignatureDetails(sigHeader, date);
+ String token = getDelegationToken(request);
+ String signingHeader;
+ if (content != null) {
+ signingHeader = (token == null)
+ ? SIGNING_HEADERS_WITH_CONTENT :
+ SIGNING_HEADERS_WITH_CONTENT_OBO;
+ } else {
+ signingHeader = (token == null)
+ ? SIGNING_HEADERS : SIGNING_HEADERS_WITH_OBO;
+ }
- /*
- * Don't cache the signature generated with content, which
- * needs to be associated with its request
- */
- if (content != null) {
- return sigDetails;
- }
+ String sigHeader = String.format(SIGNATURE_HEADER_FORMAT,
+ signingHeader,
+ keyId,
+ RSA,
+ signature,
+ SINGATURE_VERSION);
+ SignatureDetails sigDetails = new SignatureDetails(sigHeader, date);
- if (!isRefresh) {
- /*
- * if this is not a refresh, use the normal key and schedule a
- * refresh
- */
- currentSigDetails = sigDetails;
- scheduleRefresh();
- } else {
/*
- * If this is a refresh put the object in a temporary key.
- * The caller (the refresh task) will:
- * 1. perform callbacks if needed and when done,
- * 2. move the object to the normal key and schedule a refresh
+ * Don't cache the signature generated with content, which
+ * needs to be associated with its request
*/
- refreshSigDetails = sigDetails;
+ if (content != null) {
+ return sigDetails;
+ }
+
+ if (!isRefresh) {
+ /*
+ * if this is not a refresh, use the normal key and schedule a
+ * refresh
+ */
+ currentSigDetails = sigDetails;
+ scheduleRefresh();
+ } else {
+ /*
+ * If this is a refresh put the object in a temporary key.
+ * The caller (the refresh task) will:
+ * 1. perform callbacks if needed and when done,
+ * 2. move the object to the normal key and schedule a refresh
+ */
+ refreshSigDetails = sigDetails;
+ }
+ return sigDetails;
+ } finally {
+ lock.unlock();
}
- return sigDetails;
}
/*
@@ -1164,11 +1215,14 @@ private String getDelegationToken(Request req) {
req.getOboToken() : delegationToken;
}
- private synchronized void setRefreshKey() {
- if (refreshSigDetails != null) {
- currentSigDetails = refreshSigDetails;
- refreshSigDetails = null;
- }
+ private void setRefreshKey() {
+ ConcurrentUtil.synchronizedCall(lock,
+ () -> {
+ if (refreshSigDetails != null) {
+ currentSigDetails = refreshSigDetails;
+ refreshSigDetails = null;
+ }
+ });
}
private String signingContent(String date,
diff --git a/driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java b/driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java
index e3f6d0f3..67b3e644 100644
--- a/driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java
+++ b/driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java
@@ -15,7 +15,10 @@
import java.util.Base64;
import java.util.Timer;
import java.util.TimerTask;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
import java.util.logging.Logger;
import oracle.nosql.driver.AuthorizationProvider;
@@ -24,6 +27,7 @@
import oracle.nosql.driver.NoSQLHandleConfig;
import oracle.nosql.driver.httpclient.HttpClient;
import oracle.nosql.driver.ops.Request;
+import oracle.nosql.driver.util.ConcurrentUtil;
import oracle.nosql.driver.util.HttpRequestUtil;
import oracle.nosql.driver.util.HttpRequestUtil.HttpResponse;
import oracle.nosql.driver.values.JsonUtils;
@@ -98,7 +102,7 @@ public class StoreAccessTokenProvider implements AuthorizationProvider {
/*
* Login token expiration time.
*/
- private long expirationTime;
+ private volatile long expirationTime;
/*
* A timer task used to periodically renew the login token.
@@ -153,7 +157,7 @@ public class StoreAccessTokenProvider implements AuthorizationProvider {
/*
* Whether this provider is closed
*/
- private boolean isClosed = false;
+ private volatile boolean isClosed = false;
/*
* SslContext used by http client
@@ -170,6 +174,8 @@ public class StoreAccessTokenProvider implements AuthorizationProvider {
*/
public static boolean disableSSLHook;
+ private final ReentrantLock lock = new ReentrantLock();
+ private volatile CompletableFuture tokenRefreshInProgress = null;
/**
* This method is used for access to a store without security enabled.
@@ -223,14 +229,28 @@ public StoreAccessTokenProvider(String userName,
*
* Bootstrap login using the provided credentials
*/
- public synchronized void bootstrapLogin(Request request) {
-
- /* re-check the authString in case of a race */
- if (!isSecure || isClosed || authString.get() != null) {
- return;
- }
+ public CompletableFuture bootstrapLogin(Request request) {
+ lock.lock();
try {
+ if (!isSecure || isClosed ) {
+ return CompletableFuture.completedFuture(null);
+ }
+
+ /* re-check the authString in case of a race */
+ if (authString.get() != null) {
+ return CompletableFuture.completedFuture(authString.get());
+ }
+
+ if (tokenRefreshInProgress != null && !tokenRefreshInProgress.isDone()) {
+ return tokenRefreshInProgress;
+ }
+
+ if (timer != null) {
+ timer.cancel();
+ timer = null;
+ }
+
/*
* Convert the user:password pair in base 64 format with
* Basic prefix
@@ -247,36 +267,37 @@ public synchronized void bootstrapLogin(Request request) {
/*
* Send request to server for login token
*/
- HttpResponse response = sendRequest(BASIC_PREFIX + encoded,
- LOGIN_SERVICE, timeoutMs);
-
- /*
- * login fail
- */
- if (response.getStatusCode() != HttpResponseStatus.OK.code()) {
- throw new InvalidAuthorizationException(
- "Fail to login to service: " + response.getOutput());
- }
-
- if (isClosed) {
- return;
- }
-
- /*
- * Generate the authentication string using login token
- */
- authString.set(BEARER_PREFIX +
- parseJsonResult(response.getOutput()));
-
- /*
- * Schedule login token renew thread
- */
- scheduleRefresh();
-
- } catch (InvalidAuthorizationException iae) {
- throw iae;
- } catch (Exception e) {
- throw new NoSQLException("Bootstrap login fail", e);
+ CompletableFuture refreshFuture =
+ sendRequest(BASIC_PREFIX + encoded, LOGIN_SERVICE, timeoutMs)
+ .thenApply(response -> {
+ /* login fail */
+ if (response.getStatusCode() != HttpResponseStatus.OK.code()) {
+ throw new InvalidAuthorizationException(
+ "Fail to login to service: "
+ + response.getOutput());
+ }
+ if (isClosed) {
+ return null;
+ }
+ /*
+ * Generate the authentication string using login token
+ */
+ authString.set(BEARER_PREFIX + parseJsonResult(response.getOutput()));
+ /*
+ * Schedule login token renew thread
+ */
+ scheduleRefresh();
+ return authString.get();
+ }).exceptionally(err -> {
+ if (!(err instanceof InvalidAuthorizationException)) {
+ throw new NoSQLException("Bootstrap login fail", err);
+ }
+ return null;
+ });
+ tokenRefreshInProgress = refreshFuture;
+ return refreshFuture;
+ } finally {
+ lock.unlock();
}
}
@@ -285,16 +306,19 @@ public synchronized void bootstrapLogin(Request request) {
*/
@Override
public String getAuthorizationString(Request request) {
-
- if (!isSecure) {
- return null;
+ try {
+ return getAuthorizationStringAsync(request).get();
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ } catch (ExecutionException e) {
+ throw new RuntimeException(e);
}
+ }
- /*
- * Already close
- */
- if (isClosed) {
- return null;
+ @Override
+ public CompletableFuture getAuthorizationStringAsync(Request request) {
+ if (!isSecure || isClosed) {
+ return CompletableFuture.completedFuture(null);
}
/*
@@ -302,9 +326,9 @@ public String getAuthorizationString(Request request) {
* the login token and generate the auth string.
*/
if (authString.get() == null) {
- bootstrapLogin(request);
+ return bootstrapLogin(request);
}
- return authString.get();
+ return CompletableFuture.completedFuture(authString.get());
}
/**
@@ -324,21 +348,22 @@ public void validateAuthString(String input) {
* token.
*/
@Override
- public synchronized void close() {
+ public void close() {
- /*
- * Don't do anything for non-secure case
- */
- if (!isSecure || isClosed) {
- return;
- }
+ lock.lock();
/*
* Send request for logout
*/
try {
+ /*
+ * Don't do anything for non-secure case
+ */
+ if (!isSecure || isClosed) {
+ return;
+ }
final HttpResponse response =
- sendRequest(authString.get(), LOGOUT_SERVICE, 0);
+ sendRequest(authString.get(), LOGOUT_SERVICE, 0).get();
if (response.getStatusCode() != HttpResponseStatus.OK.code()) {
if (logger != null) {
logger.info("Failed to logout user " + userName +
@@ -350,19 +375,32 @@ public synchronized void close() {
logger.info("Failed to logout user " + userName +
": " + e);
}
+ /*
+ * Clean up
+ */
+ isClosed = true;
+ authString = null;
+ expirationTime = 0;
+ Arrays.fill(password, ' ');
+ if (timer != null) {
+ timer.cancel();
+ timer = null;
+ }
+ } finally {
+ lock.unlock();
}
+ }
- /*
- * Clean up
- */
- isClosed = true;
- authString = null;
- expirationTime = 0;
- Arrays.fill(password, ' ');
- if (timer != null) {
- timer.cancel();
- timer = null;
- }
+ @Override
+ public void flushCache() {
+ ConcurrentUtil.synchronizedCall(lock,
+ () -> {
+ authString.set(null);
+ if (timer != null) {
+ timer.cancel();
+ timer = null;
+ }
+ });
}
/**
@@ -504,9 +542,9 @@ private String parseJsonResult(String jsonResult) {
* Send HTTPS request to login/renew/logout service location with proper
* authentication information.
*/
- private HttpResponse sendRequest(String authHeader,
+ private CompletableFuture sendRequest(String authHeader,
String serviceName,
- int timeoutMs) throws Exception {
+ int timeoutMs) {
HttpClient client = null;
try {
final HttpHeaders headers = new DefaultHttpHeaders();
@@ -573,7 +611,7 @@ public void run() {
final String oldAuth = authString.get();
HttpResponse response = sendRequest(oldAuth,
RENEW_SERVICE,
- 0);
+ 0).get();
final String token = parseJsonResult(response.getOutput());
if (response.getStatusCode() != HttpResponseStatus.OK.code()) {
throw new InvalidAuthorizationException(token);
diff --git a/driver/src/main/java/oracle/nosql/driver/ops/Request.java b/driver/src/main/java/oracle/nosql/driver/ops/Request.java
index 565036c8..e4908dd0 100644
--- a/driver/src/main/java/oracle/nosql/driver/ops/Request.java
+++ b/driver/src/main/java/oracle/nosql/driver/ops/Request.java
@@ -61,7 +61,7 @@ public abstract class Request {
/**
* @hidden
*/
- private long startNanos;
+ private volatile long startNanos;
/**
* @hidden
diff --git a/driver/src/main/java/oracle/nosql/driver/query/ReceiveIter.java b/driver/src/main/java/oracle/nosql/driver/query/ReceiveIter.java
index 615e0ac5..bb0fece0 100644
--- a/driver/src/main/java/oracle/nosql/driver/query/ReceiveIter.java
+++ b/driver/src/main/java/oracle/nosql/driver/query/ReceiveIter.java
@@ -12,6 +12,7 @@
import java.util.HashSet;
import java.util.List;
import java.util.TreeSet;
+import java.util.concurrent.ExecutionException;
import oracle.nosql.driver.NoSQLException;
import oracle.nosql.driver.RetryableException;
@@ -592,9 +593,13 @@ private QueryResult execute(RuntimeControlBlock rcb,
NoSQLException e = null;
QueryResult result = null;
try {
- result = (QueryResult)rcb.getClient().execute(reqCopy);
+ result = (QueryResult)rcb.getClient().execute(reqCopy).get();
} catch (NoSQLException qe) {
e = qe;
+ } catch (ExecutionException ex) {
+ throw new RuntimeException(ex);
+ } catch (InterruptedException ex) {
+ throw new RuntimeException(ex);
}
/*
* Copy values back to original request, even when the execute()
diff --git a/driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java b/driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java
new file mode 100644
index 00000000..38a6a9a2
--- /dev/null
+++ b/driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java
@@ -0,0 +1,32 @@
+package oracle.nosql.driver.util;
+
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Supplier;
+
+public class ConcurrentUtil {
+ /**
+ * A convenient function to hold the lock and run.
+ */
+ public static T synchronizedCall(ReentrantLock lock,
+ Supplier s) {
+ lock.lock();
+ try {
+ return s.get();
+ } finally {
+ lock.unlock();
+ }
+ }
+
+ /**
+ * A convenient function to hold the lock and run.
+ */
+ public static void synchronizedCall(ReentrantLock lock,
+ Runnable r) {
+ lock.lock();
+ try {
+ r.run();
+ } finally {
+ lock.unlock();
+ }
+ }
+}
diff --git a/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java b/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
index d0ea97bb..897bf98f 100644
--- a/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
+++ b/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
@@ -23,11 +23,14 @@
import java.net.URI;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
+import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
import java.util.logging.Logger;
import javax.net.ssl.SSLException;
+import io.netty.buffer.Unpooled;
import oracle.nosql.driver.RequestTimeoutException;
import oracle.nosql.driver.httpclient.HttpClient;
import oracle.nosql.driver.httpclient.ResponseHandler;
@@ -74,7 +77,7 @@ public class HttpRequestUtil {
*
* @return HTTP response, a object encapsulate status code and response
*/
- public static HttpResponse doGetRequest(HttpClient httpClient,
+ public static CompletableFuture doGetRequest(HttpClient httpClient,
String uri,
HttpHeaders headers,
int timeoutMs,
@@ -112,7 +115,7 @@ public static HttpResponse doGetRequest(HttpClient httpClient,
*
* @return HTTP response, a object encapsulate status code and response
*/
- public static HttpResponse doPostRequest(HttpClient httpClient,
+ public static CompletableFuture doPostRequest(HttpClient httpClient,
String uri,
HttpHeaders headers,
byte[] payload,
@@ -151,7 +154,7 @@ public static HttpResponse doPostRequest(HttpClient httpClient,
*
* @return HTTP response, a object encapsulate status code and response
*/
- public static HttpResponse doPutRequest(HttpClient httpClient,
+ public static CompletableFuture doPutRequest(HttpClient httpClient,
String uri,
HttpHeaders headers,
byte[] payload,
@@ -188,7 +191,7 @@ public static HttpResponse doPutRequest(HttpClient httpClient,
*
* @return HTTP response, a object encapsulate status code and response
*/
- public static HttpResponse doDeleteRequest(HttpClient httpClient,
+ public static CompletableFuture doDeleteRequest(HttpClient httpClient,
String uri,
HttpHeaders headers,
int timeoutMs,
@@ -198,7 +201,7 @@ public static HttpResponse doDeleteRequest(HttpClient httpClient,
timeoutMs, logger);
}
- private static HttpResponse doRequest(HttpClient httpClient,
+ private static CompletableFuture doRequest(HttpClient httpClient,
String uri,
HttpHeaders headers,
HttpMethod method,
@@ -209,116 +212,47 @@ private static HttpResponse doRequest(HttpClient httpClient,
final long startTime = System.currentTimeMillis();
int numRetries = 0;
Throwable exception = null;
- HttpResponse res = null;
- do {
- if (numRetries > 0) {
- logInfo(logger, "Client, doing retry: " + numRetries +
- (exception != null ? ", exception: " + exception : ""));
- }
- Channel channel = null;
- ResponseHandler responseHandler = null;
- try {
- channel = httpClient.getChannel(timeoutMs);
- responseHandler =
- new ResponseHandler(httpClient, logger, channel);
-
- FullHttpRequest request;
- if (payload == null) {
- request = buildRequest(uri, method, headers);
- } else {
- request = buildRequest(
- uri, headers, method, payload, channel);
- }
- addRequiredHeaders(request);
- logFine(logger, request.headers().toString());
- httpClient.runRequest(request, responseHandler, channel);
- if (responseHandler.await(timeoutMs)) {
- throw new TimeoutException("Request timed out after " +
- timeoutMs + " milliseconds");
- }
-
- final HttpResponseStatus status = responseHandler.getStatus();
- if (status == null) {
- throw new IllegalStateException("Invalid null response");
- }
- res = processResponse(status.code(),
- responseHandler.getContent());
+ FullHttpRequest request;
+ if (payload == null) {
+ request = buildRequest(uri, method, headers);
+ } else {
+ request = buildRequest(uri, headers, method, payload);
+ }
+ addRequiredHeaders(request);
+ logFine(logger, request.headers().toString());
- /*
- * Retry upon status code larger than 500, in general,
- * this indicates server internal error.
- */
- if (res.getStatusCode() >= 500) {
- logFine(logger,
- "Remote server temporarily unavailable," +
- " status code " + res.getStatusCode() +
- " , response " + res.getOutput());
- delay();
- ++numRetries;
- continue;
- }
- return res;
- } catch (RuntimeException e) {
- logFine(logger, "Client execute runtime exception: " +
- e.getMessage());
- throw e;
- } catch (IOException ioe) {
- String name = ioe.getClass().getName();
- logFine(logger, "Client execute IOException, name: " +
- name + ", message: " + ioe.getMessage());
- /*
- * An exception in the channel, e.g. the server may have
- * disconnected. Retry.
- */
- exception = ioe;
- ++numRetries;
- if (ioe instanceof SSLException) {
- /* disconnect the channel to force a new one */
- if (channel != null) {
- logFine(logger,
- "Client disconnecting channel due to: " + ioe);
- channel.disconnect();
- }
- } else {
- delay();
- }
- continue;
- } catch (InterruptedException ie) {
- throw new RuntimeException(
- "Client interrupted exception: ", ie);
- } catch (ExecutionException ee) {
- throw new RuntimeException(
- "Unable to execute request: ", ee);
- } catch (TimeoutException te) {
- throw new RuntimeException("Timeout exception: host=" +
- httpClient.getHost() + " port=" +
- httpClient.getPort() + " uri=" +
- uri, te);
- } catch (Throwable t) {
- /*
- * this is likely an exception from Netty, perhaps a bad
- * connection. Retry.
- */
- String name = t.getClass().getName();
- logFine(logger, "Client execute Throwable, name: " +
- name + "message: " + t.getMessage());
- exception = t;
- delay();
- ++numRetries;
- continue;
+ return httpClient.runRequest(request, timeoutMs)
+ .thenApply(fhr -> {
+ if (fhr.status() == null) {
+ throw new IllegalStateException("Invalid null response");
+ }
+ try {
+ return processResponse(fhr.status().code(), fhr.content());
} finally {
- if (responseHandler != null) {
- responseHandler.close();
- }
+ fhr.release();
}
- } while ((System.currentTimeMillis()- startTime) < timeoutMs);
+ })
+ .handle((res, err) -> {
+ if (err != null) {
+ return handleError(res, err);
+ } else if(res.getStatusCode() >= 500) {
+ logFine(logger,
+ "Remote server temporarily unavailable," +
+ " status code " + res.getStatusCode() +
+ " , response " + res.getOutput());
+ return handleError(res, err);
+ }
+ return CompletableFuture.completedFuture(res);
+ })
+ .thenCompose(Function.identity());
+ }
- throw new RequestTimeoutException(timeoutMs,
- "Request timed out after " + numRetries +
- (numRetries == 1 ? " retry." : " retries."),
- exception);
+ static CompletableFuture handleError(HttpResponse response,
+ Throwable err) {
+ //TODO handle error
+ return null;
}
private static FullHttpRequest buildRequest(String requestURI,
@@ -333,9 +267,8 @@ private static FullHttpRequest buildRequest(String requestURI,
private static FullHttpRequest buildRequest(String requestURI,
HttpHeaders headers,
HttpMethod method,
- byte[] payload,
- Channel channel) {
- final ByteBuf buffer = channel.alloc().directBuffer();
+ byte[] payload) {
+ final ByteBuf buffer = Unpooled.wrappedBuffer(payload);
buffer.writeBytes(payload);
final FullHttpRequest request =
diff --git a/driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java b/driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java
index dda592ef..87b47650 100644
--- a/driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java
+++ b/driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java
@@ -30,6 +30,7 @@
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
+import java.util.concurrent.CompletableFuture;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -518,6 +519,13 @@ public String getAuthorizationString(Request request) {
@Override
public void close() {
}
+
+ @Override
+ public CompletableFuture
+ getAuthorizationStringAsync(Request request) {
+ return CompletableFuture.completedFuture(
+ "Bearer cloudsim");
+ }
});
}
}
diff --git a/driver/src/test/java/oracle/nosql/driver/httpclient/ConnectionPoolTest.java b/driver/src/test/java/oracle/nosql/driver/httpclient/ConnectionPoolTest.java
index 48295fb4..e12b11a5 100644
--- a/driver/src/test/java/oracle/nosql/driver/httpclient/ConnectionPoolTest.java
+++ b/driver/src/test/java/oracle/nosql/driver/httpclient/ConnectionPoolTest.java
@@ -9,6 +9,7 @@
import static org.junit.Assert.assertEquals;
+import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Future;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -77,7 +78,7 @@ public void poolTest() throws Exception {
pool.setKeepAlive(new ConnectionPool.KeepAlive() {
@Override
- public boolean keepAlive(Channel ch) {
+ public CompletableFuture keepAlive(Channel ch) {
return client.doKeepAlive(ch);
}
});
@@ -176,7 +177,7 @@ public void testCloudTimeout() throws Exception {
pool.setKeepAlive(new ConnectionPool.KeepAlive() {
@Override
- public boolean keepAlive(Channel ch) {
+ public CompletableFuture keepAlive(Channel ch) {
return client.doKeepAlive(ch);
}
});
diff --git a/examples/pom.xml b/examples/pom.xml
index b6c9acd8..74b43c31 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -12,8 +12,8 @@
UTF-8
- 1.8
- 1.8
+ 11
+ 11truetrue
@@ -37,8 +37,8 @@
3.11.0true
- 1.8
- 1.8
+ ${maven.compiler.source}
+ ${maven.compiler.target}truetrue
diff --git a/examples/src/main/java/Common.java b/examples/src/main/java/Common.java
index dcb800ac..ae301572 100644
--- a/examples/src/main/java/Common.java
+++ b/examples/src/main/java/Common.java
@@ -8,6 +8,7 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
+import java.util.concurrent.CompletableFuture;
import oracle.nosql.driver.AuthorizationProvider;
import oracle.nosql.driver.NoSQLHandle;
@@ -358,6 +359,11 @@ public String getAuthorizationString(Request request) {
return id;
}
+ @Override
+ public CompletableFuture getAuthorizationStringAsync(Request request) {
+ return CompletableFuture.completedFuture(id);
+ }
+
@Override
public void close() {}
}
From f6c77b0783a943a1e50b9424c734c33d51c656a8 Mon Sep 17 00:00:00 2001
From: Akshay Sundarraj
Date: Thu, 14 Aug 2025 23:11:17 +0530
Subject: [PATCH 02/11] Refactor Client#executeWithRetry into multiple small
methods.
executeWithRetry method is very big and challenging to comprehend. Split this
into multiple methods to handle individual stages like, check for rate limit,
get auth token from auth provider, serialize the request, send request to
server, process response, and so on.
---
.../java/oracle/nosql/driver/http/Client.java | 295 ++++++++++--------
1 file changed, 165 insertions(+), 130 deletions(-)
diff --git a/driver/src/main/java/oracle/nosql/driver/http/Client.java b/driver/src/main/java/oracle/nosql/driver/http/Client.java
index 4c3a9c5f..a2ec6838 100644
--- a/driver/src/main/java/oracle/nosql/driver/http/Client.java
+++ b/driver/src/main/java/oracle/nosql/driver/http/Client.java
@@ -67,6 +67,8 @@
import java.util.logging.Logger;
import io.netty.buffer.Unpooled;
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.HttpRequest;
import oracle.nosql.driver.AuthorizationProvider;
import oracle.nosql.driver.InvalidAuthorizationException;
import oracle.nosql.driver.NoSQLException;
@@ -625,6 +627,27 @@ private CompletableFuture executeWithRetry(RequestContext ctx) {
}
}
+ return handlePreRateLimit(ctx)
+ //TODO is this thenComposeAsync?
+ .thenCompose( (Integer delay) -> getAuthString(ctx, authProvider))
+ .thenCompose((String authString) -> createRequest(ctx, authString))
+ .thenCompose((FullHttpRequest request) -> submitRequest(ctx,request))
+ .thenApply((FullHttpResponse response) -> handleResponse(ctx, response))
+ .thenApply((Result result) -> handleResult(ctx, result))
+ .thenCompose((Result result) -> handlePostRateLimit(ctx, result))
+ .handle((Result result, Throwable err) -> {
+ /* Handle error and retry */
+ if (err != null) {
+ ctx.exception = err;
+ return handleError(ctx);
+ } else {
+ return CompletableFuture.completedFuture(result);
+ }
+ })
+ .thenCompose(Function.identity());
+ }
+
+ private CompletableFuture handlePreRateLimit(RequestContext ctx) {
/* Check this request doesn't cause throttle */
int preRateLimitDelayMs = 0;
@@ -648,15 +671,22 @@ private CompletableFuture executeWithRetry(RequestContext ctx) {
+ thisIterationTimeoutMs
+ "ms due to rate limiting");
return createDelayFuture(thisIterationTimeoutMs).thenCompose(d ->
- CompletableFuture.failedFuture(ex));
+ CompletableFuture.failedFuture(ex));
}
- /* Stage-1 RateLimit */
+ /* sleep for delay ms */
return createDelayFuture(preRateLimitDelayMs)
- .whenComplete((delay, err) -> ctx.rateDelayedMs.addAndGet(delay))
- /* Stage-2 Get auth token */
- //TODO is this thenComposeAsync?
- .thenCompose(delay -> authProvider.getAuthorizationStringAsync(kvRequest)) // CompletableFuture
+ .whenComplete((delay, err) -> ctx.rateDelayedMs.addAndGet(delay));
+ }
+
+ /**
+ * Get auth token from auth provider.
+ * This may contact the server to get the token.
+ */
+ private CompletableFuture getAuthString(RequestContext ctx,
+ AuthorizationProvider authProvider) {
+ final Request kvRequest = ctx.kvRequest;
+ return authProvider.getAuthorizationStringAsync(kvRequest)
.thenApply(authString -> {
/* Check whether timed out while acquiring the auth token */
if (timeoutRequest(kvRequest.getStartNanos(),
@@ -665,128 +695,22 @@ private CompletableFuture executeWithRetry(RequestContext ctx) {
"timed out during auth token acquisition");
throw new CompletionException(ex);
}
+ /* validate the token is valid or not */
authProvider.validateAuthString(authString);
return authString;
- })
- .thenCompose(authString -> {
- /* stage-3 create HTTP request */
- final ByteBuf buffer = Unpooled.buffer();
- buffer.retain();
- return createRequest(ctx, authString, buffer)
- .whenComplete((res, err) -> {
- if (err != null) {
- buffer.release();
- }
- })
- .thenCompose(request -> {
- /* Stage-4 Send the request to server */
- if (isLoggable(logger, Level.FINE) &&
- !kvRequest.getIsRefresh()) {
- logTrace(logger, "Request: " + ctx.requestClass +
- ", requestId=" + ctx.requestId);
- }
- ctx.latencyNanos = System.nanoTime();
- return httpClient.runRequest(request,
- getIterationTimeoutMs(timeoutMs, startNanos)) //CompletableFuture
- .whenComplete((res, err) -> {
- ctx.networkLatency =
- (System.nanoTime() - ctx.latencyNanos) / 1_000_000;
- // Release our retained request ByteBuf
- buffer.release();
- });
- });
- }) // Now we have CompletableFuture
- .thenApply(fhr -> {
- /* Stage-5 Process the response */
- if (isLoggable(logger, Level.FINE) &&
- !kvRequest.getIsRefresh()) {
- logTrace(logger, "Response: " + ctx.requestClass +
- ", status=" + fhr.status() +
- ", requestId=" + ctx.requestId );
- }
- try {
- Result result = processResponse(fhr.status(), fhr.headers(),
- fhr.content(), ctx);
- ctx.rateDelayedMs.addAndGet(getRateDelayedFromHeader(fhr.headers()));
- ctx.resSize = fhr.content().readerIndex();
- return result;
- } finally {
- fhr.release(); //release response
- }
- }) // Now we have CompletableFuture
- .thenCompose(result -> {
- /* Stage-6 Get the Result and bookkeeping */
- setTopology(result.getTopology());
- if (ctx.serialVersionUsed < 3) {
- /* so we can emit a one-time message if the app */
- /* tries to access modificationTime */
- if (result instanceof GetResult) {
- ((GetResult)result).setClient(this);
- } else if (result instanceof WriteResult) {
- ((WriteResult)result).setClient(this);
- }
- }
- if (result instanceof QueryResult && kvRequest.isQueryRequest()) {
- QueryRequest qreq = (QueryRequest)kvRequest;
- qreq.addQueryTraces(((QueryResult)result).getQueryTraces());
- }
- if (result instanceof TableResult && rateLimiterMap != null) {
- /* update rate limiter settings for table */
- TableLimits tl = ((TableResult)result).getTableLimits();
- updateRateLimiters(((TableResult)result).getTableName(), tl);
- }
- /*
- * We may not have rate limiters yet because queries may
- * not have a tablename until after the first request.
- * So try to get rate limiters if we don't have them yet and
- * this is a QueryRequest.
- */
- if (rateLimiterMap != null && ctx.readLimiter == null) {
- ctx.readLimiter = getQueryRateLimiter(kvRequest, true);
- }
- if (rateLimiterMap != null && ctx.writeLimiter == null) {
- ctx.writeLimiter = getQueryRateLimiter(kvRequest, false);
- }
-
- int postRateLimitDelayMs = consumeLimiterUnits(ctx.readLimiter,
- result.getReadUnitsInternal());
- postRateLimitDelayMs += consumeLimiterUnits(ctx.writeLimiter,
- result.getWriteUnitsInternal());
-
- /* Post-Response RateLimiting */
- return createDelayFuture(postRateLimitDelayMs)
- .thenApply(rateDelay -> {
- ctx.rateDelayedMs.addAndGet(rateDelay);
- result.setRateLimitDelayedMs(ctx.rateDelayedMs.get());
-
- /* copy retry stats to Result on successful operation */
- result.setRetryStats(kvRequest.getRetryStats());
- kvRequest.setRateLimitDelayedMs(ctx.rateDelayedMs.get());
-
- statsControl.observe(kvRequest,
- Math.toIntExact(ctx.networkLatency),
- ctx.reqSize, ctx.resSize);
- checkAuthRefreshList(kvRequest);
- return result;
- });
- })
- .handle((res,err) -> {
- /* Handle error and retry */
- if (err != null) {
- ctx.exception = err;
- return handleError(ctx);
- } else {
- return CompletableFuture.completedFuture(res);
- }
- })
- .thenCompose(Function.identity());
+ });
}
-
+ /**
+ * Create Netty HTTP request.
+ * This will serialize the request body and fill the HTTP headers and
+ * body.
+ * This may contact the server to sign the request body.
+ */
private CompletableFuture createRequest(RequestContext ctx,
- String authString,
- ByteBuf buffer) {
-
+ String authString) {
+ ByteBuf buffer = null;
try {
+ buffer = Unpooled.buffer();
final Request kvRequest = ctx.kvRequest;
/*
* we expressly check size limit below based on onprem versus
@@ -834,18 +758,18 @@ private CompletableFuture createRequest(RequestContext ctx,
kvRequest, buffer.readableBytes());
}
final FullHttpRequest request =
- new DefaultFullHttpRequest(
- HTTP_1_1, POST, kvRequestURI,
- buffer,
- headersFactory().withValidation(false),
- trailersFactory().withValidation(false));
+ new DefaultFullHttpRequest(
+ HTTP_1_1, POST, kvRequestURI,
+ buffer,
+ headersFactory().withValidation(false),
+ trailersFactory().withValidation(false));
HttpHeaders headers = request.headers();
addCommonHeaders(headers);
int contentLength = buffer.readableBytes();
ctx.reqSize = contentLength;
headers.add(HttpHeaderNames.HOST, host)
- .add(REQUEST_ID_HEADER, ctx.requestId)
- .setInt(CONTENT_LENGTH, contentLength);
+ .add(REQUEST_ID_HEADER, ctx.requestId)
+ .setInt(CONTENT_LENGTH, contentLength);
if (sessionCookie != null) {
headers.add(COOKIE, sessionCookie);
}
@@ -878,9 +802,120 @@ private CompletableFuture createRequest(RequestContext ctx,
return request;
});
} catch (Exception e) {
+ /* Release the buffer on error */
+ if (buffer != null) {
+ buffer.release();
+ }
return CompletableFuture.failedFuture(e);
}
}
+
+ /**
+ * Send the HTTP request to server and get the response back.
+ */
+ private CompletableFuture submitRequest(RequestContext ctx,
+ HttpRequest request) {
+ final Request kvRequest = ctx.kvRequest;
+ if (isLoggable(logger, Level.FINE) && !kvRequest.getIsRefresh()) {
+ logTrace(logger, "Request: " + ctx.requestClass +
+ ", requestId=" + ctx.requestId);
+ }
+ ctx.latencyNanos = System.nanoTime();
+ int timeoutMs = getIterationTimeoutMs(ctx.timeoutMs, ctx.startNanos);
+
+ return httpClient.runRequest(request, timeoutMs)
+ .whenComplete((res, err) -> {
+ ctx.networkLatency = (System.nanoTime() - ctx.latencyNanos) / 1_000_000;
+ });
+ }
+
+ /**
+ * Deserialize HTTP response into NoSQL Result.
+ */
+ private Result handleResponse(RequestContext ctx, FullHttpResponse fhr) {
+ final Request kvRequest = ctx.kvRequest;
+ if (isLoggable(logger, Level.FINE) && !kvRequest.getIsRefresh()) {
+ logTrace(logger, "Response: " + ctx.requestClass +
+ ", status=" + fhr.status() +
+ ", requestId=" + ctx.requestId );
+ }
+ try {
+ Result result = processResponse(fhr.status(), fhr.headers(), fhr.content(), ctx);
+ ctx.rateDelayedMs.addAndGet(getRateDelayedFromHeader(fhr.headers()));
+ ctx.resSize = fhr.content().readerIndex();
+ return result;
+ } finally {
+ fhr.release(); //release response
+ }
+ }
+
+ /**
+ * Update stats from the result.
+ */
+ private Result handleResult(RequestContext ctx, Result result) {
+ final Request kvRequest = ctx.kvRequest;
+ setTopology(result.getTopology());
+ if (ctx.serialVersionUsed < 3) {
+ /* so we can emit a one-time message if the app */
+ /* tries to access modificationTime */
+ if (result instanceof GetResult) {
+ ((GetResult)result).setClient(this);
+ } else if (result instanceof WriteResult) {
+ ((WriteResult)result).setClient(this);
+ }
+ }
+ if (result instanceof QueryResult && kvRequest.isQueryRequest()) {
+ QueryRequest qreq = (QueryRequest)kvRequest;
+ qreq.addQueryTraces(((QueryResult)result).getQueryTraces());
+ }
+ if (result instanceof TableResult && rateLimiterMap != null) {
+ /* update rate limiter settings for table */
+ TableLimits tl = ((TableResult)result).getTableLimits();
+ updateRateLimiters(((TableResult)result).getTableName(), tl);
+ }
+ /*
+ * We may not have rate limiters yet because queries may
+ * not have a tablename until after the first request.
+ * So try to get rate limiters if we don't have them yet and
+ * this is a QueryRequest.
+ */
+ if (rateLimiterMap != null && ctx.readLimiter == null) {
+ ctx.readLimiter = getQueryRateLimiter(kvRequest, true);
+ }
+ if (rateLimiterMap != null && ctx.writeLimiter == null) {
+ ctx.writeLimiter = getQueryRateLimiter(kvRequest, false);
+ }
+ return result;
+ }
+
+ /**
+ * Handle rate limit from the Result.
+ * This will consume actual units used by the request and sleep.
+ */
+ private CompletableFuture handlePostRateLimit(RequestContext ctx, Result result) {
+ final Request kvRequest = ctx.kvRequest;
+ int postRateLimitDelayMs = consumeLimiterUnits(ctx.readLimiter,
+ result.getReadUnitsInternal());
+ postRateLimitDelayMs += consumeLimiterUnits(ctx.writeLimiter,
+ result.getWriteUnitsInternal());
+
+ return createDelayFuture(postRateLimitDelayMs)
+ .thenApply(rateDelay -> {
+ ctx.rateDelayedMs.addAndGet(rateDelay);
+ result.setRateLimitDelayedMs(ctx.rateDelayedMs.get());
+
+ /* copy retry stats to Result on successful operation */
+ result.setRetryStats(kvRequest.getRetryStats());
+ kvRequest.setRateLimitDelayedMs(ctx.rateDelayedMs.get());
+
+ statsControl.observe(kvRequest,
+ Math.toIntExact(ctx.networkLatency),
+ ctx.reqSize, ctx.resSize);
+ checkAuthRefreshList(kvRequest);
+ return result;
+ });
+ }
+
private CompletableFuture handleError(RequestContext ctx) {
Request kvRequest = ctx.kvRequest;
Throwable err = ctx.exception;
From 8e4b6ee26b073f168dbbd228d8a6328076d0df0d Mon Sep 17 00:00:00 2001
From: Akshay Sundarraj
Date: Fri, 29 Aug 2025 19:12:30 +0530
Subject: [PATCH 03/11] Async refactor changes-II
- Incorporate changes suggested by George
o. Limit line width to 80 characters
o. Keep core part of SignatureProvider i.e.
SignatureProvider#getSignatureDetailsInternal same as existing blocking code
and wrap it with CompletableFuture.supplyAsync
o. Keep core part of StoreAccessTokenProvider i.e. bootstraplogin same as existing
blocking code and wrap it with CompletableFuture.supplyAsync
o. Implement flushCache method for StoreAccessTokenProvider. In the current
implementation, flushCache is not implemented for the StoreAccessTokenProvider,
which can result in bootstrap login to never contact the server after the
initial acquisition of the auth token
o. Revert HttpRequestUtil to a blocking version as this code always executes in blocking mode
- Error handling enhancement for client execute
o. In the current implementation error handling is a very big piece of code with
lots of catch blocks with a lot of repeated code.
o. Introduced a strategy pattern where a HashMap,
ErrorHandler> is used to maintain a list of possible exceptions encountered by
the system and error handler for each exception. In this way, we can keep the
core error handling and retry code to be simple and delegate actual error to
handlers.
o. Please see below for more details
Files:
M driver/src/main/java/oracle/nosql/driver/DefaultRetryHandler.java
M examples/src/main/java/Common.java
M driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
M driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
- Limit line width to 80
M driver/src/main/java/oracle/nosql/driver/http/Client.java
- Use core number of threads for task executor
- Introduced a ErrorHandler interface to handle individual errors
- Added a condition to check overall timeout before retrying a request
- Split handleError() method to multiple error handlers
- Added findErrorHandler() method which finds the specific handler for given error
- Replaced synchronized blocks with ReentrantLock
M driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
- Replaced call to CompletableFuture.get() with utility method ConcurrentUtil.awaitFuture
M driver/src/main/java/oracle/nosql/driver/httpclient/HttpClientHandler.java
- When timeout occurs throw java.util.concurrent.TimeoutException instead of Netty ReadTimeoutException
M driver/src/main/java/oracle/nosql/driver/iam/FederationRequestHelper.java
M driver/src/main/java/oracle/nosql/driver/iam/InstanceMetadataHelper.java
M driver/src/main/java/oracle/nosql/driver/iam/OkeWorkloadIdentityProvider.java
- Revert async changes
M driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java
- Added async wrapper over core getSignatureDetailsInternal method
- Replaced synchronized blocks with ReentrantLock
M driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java
- Added async wrapper over core bootstrap login method
- Override flushCache and implement
- Replaced synchronized blocks with ReentrantLock
M driver/src/main/java/oracle/nosql/driver/query/ReceiveIter.java
- Replaced call to CompletableFuture.get() with utility method ConcurrentUtil.awaitFuture
M driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java
- Added a utility method awaitFuture() to wait on future
- Added a utility method unwrapCompletionException()
M driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
- Reverted async changes
M driver/src/main/java/oracle/nosql/driver/util/LogUtil.java
- Added a utility method getStackTrace() to get stack trace from Throwable
---
.../nosql/driver/DefaultRetryHandler.java | 4 +-
.../java/oracle/nosql/driver/http/Client.java | 852 +++++++++++-------
.../nosql/driver/http/NoSQLHandleImpl.java | 33 +-
.../driver/httpclient/ConnectionPool.java | 6 +-
.../nosql/driver/httpclient/HttpClient.java | 108 +--
.../driver/httpclient/HttpClientHandler.java | 22 +-
.../driver/iam/FederationRequestHelper.java | 16 +-
.../driver/iam/InstanceMetadataHelper.java | 11 +-
.../iam/OkeWorkloadIdentityProvider.java | 14 +-
.../nosql/driver/iam/SignatureProvider.java | 249 +++--
.../driver/kv/StoreAccessTokenProvider.java | 196 ++--
.../nosql/driver/query/ReceiveIter.java | 11 +-
.../nosql/driver/util/ConcurrentUtil.java | 39 +
.../nosql/driver/util/HttpRequestUtil.java | 146 ++-
.../oracle/nosql/driver/util/LogUtil.java | 17 +
examples/src/main/java/Common.java | 3 +-
16 files changed, 945 insertions(+), 782 deletions(-)
diff --git a/driver/src/main/java/oracle/nosql/driver/DefaultRetryHandler.java b/driver/src/main/java/oracle/nosql/driver/DefaultRetryHandler.java
index ed106a46..684cc681 100644
--- a/driver/src/main/java/oracle/nosql/driver/DefaultRetryHandler.java
+++ b/driver/src/main/java/oracle/nosql/driver/DefaultRetryHandler.java
@@ -86,7 +86,9 @@ public void delay(Request request,
}
@Override
- public int delayTime(Request request, int numRetries, RetryableException re) {
+ public int delayTime(Request request,
+ int numRetries,
+ RetryableException re) {
return Math.max(0, computeBackoffDelay(request, fixedDelayMs));
}
diff --git a/driver/src/main/java/oracle/nosql/driver/http/Client.java b/driver/src/main/java/oracle/nosql/driver/http/Client.java
index a2ec6838..93903d10 100644
--- a/driver/src/main/java/oracle/nosql/driver/http/Client.java
+++ b/driver/src/main/java/oracle/nosql/driver/http/Client.java
@@ -40,6 +40,7 @@
import java.io.FileOutputStream;
import java.io.IOException;
import java.net.URL;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
@@ -70,6 +71,7 @@
import io.netty.handler.codec.http.FullHttpResponse;
import io.netty.handler.codec.http.HttpRequest;
import oracle.nosql.driver.AuthorizationProvider;
+import oracle.nosql.driver.DefaultRetryHandler;
import oracle.nosql.driver.InvalidAuthorizationException;
import oracle.nosql.driver.NoSQLException;
import oracle.nosql.driver.NoSQLHandleConfig;
@@ -77,6 +79,7 @@
import oracle.nosql.driver.RateLimiter;
import oracle.nosql.driver.ReadThrottlingException;
import oracle.nosql.driver.RequestSizeLimitException;
+import oracle.nosql.driver.RequestTimeoutException;
import oracle.nosql.driver.RetryHandler;
import oracle.nosql.driver.RetryableException;
import oracle.nosql.driver.SecurityInfoNotReadyException;
@@ -116,6 +119,7 @@
import oracle.nosql.driver.util.ByteInputStream;
import oracle.nosql.driver.util.ConcurrentUtil;
import oracle.nosql.driver.util.HttpConstants;
+import oracle.nosql.driver.util.LogUtil;
import oracle.nosql.driver.util.NettyByteInputStream;
import oracle.nosql.driver.util.NettyByteOutputStream;
import oracle.nosql.driver.util.RateLimiterMap;
@@ -134,9 +138,10 @@
/**
* The HTTP driver client.
*/
-public class Client {
+public final class Client {
public static int traceLevel = 0;
+ private static final int cores = Runtime.getRuntime().availableProcessors();
private final NoSQLHandleConfig config;
@@ -238,8 +243,73 @@ public class Client {
/* thread-pool for scheduling tasks */
private final ScheduledExecutorService taskExecutor;
+ /* Lock to access data structures */
private final ReentrantLock lock = new ReentrantLock();
+ /*
+ * Centralized error handling for request execution.
+ * This class maps specific {@link Throwable} types to error-handling
+ * strategies (retry, fail, or protocol downgrade).
+ * It uses a HashMap of {@link ErrorHandler} functions
+ * to keep {@link #handleError(RequestContext, Throwable)} short and
+ * maintainable.
+ */
+ private final Map, ErrorHandler>
+ errorHandlers = new HashMap<>();
+
+ /*
+ * Functional interface for all error handlers.
+ * Each handler inspects the exception and decides whether
+ * to retry the request or fail with an exception.
+ */
+ @FunctionalInterface
+ private interface ErrorHandler {
+ CompletableFuture handle(RequestContext ctx, Throwable error);
+ }
+
+ /**
+ * RequestContext class to encapsulate request-specific data.
+ * This helps in passing context through asynchronous chains.
+ * It now includes requestId and a Supplier to generate new IDs for retries.
+ */
+ private static class RequestContext {
+ private final Request kvRequest;
+ private final String requestClass;
+ private volatile String requestId;
+ private final long startNanos;
+ private final int timeoutMs;
+ private final Supplier nextIdSupplier;
+ private volatile Throwable exception;
+ private final AtomicInteger rateDelayedMs = new AtomicInteger(0);
+ private volatile RateLimiter readLimiter;
+ private volatile RateLimiter writeLimiter;
+ private volatile boolean checkReadUnits;
+ private volatile boolean checkWriteUnits;
+ private volatile int reqSize;
+ private volatile int resSize;
+ private volatile short serialVersionUsed;
+ private volatile short queryVersionUsed;
+ private volatile long latencyNanos;
+ public volatile long networkLatency;
+
+ RequestContext(Request kvRequest, long startNanos, int timeoutMs,
+ Supplier nextIdSupplier, RateLimiter readLimiter,
+ RateLimiter writeLimiter, boolean checkReadUnits,
+ boolean checkWriteUnits) {
+ this.kvRequest = kvRequest;
+ this.startNanos = startNanos;
+ this.timeoutMs = timeoutMs;
+ this.nextIdSupplier = nextIdSupplier;
+ this.readLimiter = readLimiter;
+ this.writeLimiter = writeLimiter;
+ this.checkReadUnits = checkReadUnits;
+ this.checkWriteUnits = checkWriteUnits;
+
+ this.requestId = Long.toString(nextIdSupplier.get());
+ this.requestClass = kvRequest.getClass().getSimpleName();
+ }
+ }
+
public Client(Logger logger,
NoSQLHandleConfig httpConfig) {
@@ -328,20 +398,33 @@ public Client(Logger logger,
/* for internal testing */
prepareFilename = System.getProperty("test.preparefilename");
- taskExecutor = new ScheduledThreadPoolExecutor(8 /* core threads */,
+ taskExecutor = new ScheduledThreadPoolExecutor(cores /* core threads */,
new ThreadFactory() {
private final AtomicInteger threadNumber = new AtomicInteger(1);
@Override
public Thread newThread(Runnable r) {
final Thread t = Executors.defaultThreadFactory().newThread(r);
- t.setName(String.format("task-executor-%s", threadNumber.getAndIncrement()));
+ t.setName(String.format("nosql-task-executor-%s",
+ threadNumber.getAndIncrement()));
t.setDaemon(true);
t.setUncaughtExceptionHandler((thread, error) -> {
- // TODO handle this
+ if (ConcurrentUtil.unwrapCompletionException(error)
+ instanceof RejectedExecutionException) {
+ /*
+ * Ignore uncaught error for rejected exception
+ * since that is expected to happen during
+ * executor shut down.
+ */
+ return;
+ }
+ logger.warning(() -> String.format(
+ "Uncaught exception from %s: %s",
+ error, LogUtil.getStackTrace(error)));
});
return t;
}
});
+ initErrorHandlers();
}
/**
@@ -387,60 +470,9 @@ private long nextRequestId() {
return maxRequestId.addAndGet(1);
}
- /**
- * RequestContext class to encapsulate request-specific data.
- * This helps in passing context through asynchronous CompletableFuture chains.
- * It now includes requestId and a Supplier to generate new IDs for retries.
- */
- private static class RequestContext {
- private final Request kvRequest;
- private final String requestClass;
- private volatile String requestId;
- private final long startNanos;
- private final int timeoutMs;
- private final Supplier nextIdSupplier;
- private volatile Throwable exception;
- private final AtomicInteger rateDelayedMs = new AtomicInteger(0);
- private volatile RateLimiter readLimiter;
- private volatile RateLimiter writeLimiter;
- private volatile boolean checkReadUnits;
- private volatile boolean checkWriteUnits;
- private volatile int reqSize;
- private volatile int resSize;
- private volatile short serialVersionUsed;
- private volatile short queryVersionUsed;
- private volatile long latencyNanos;
- public volatile long networkLatency;
-
- RequestContext(Request kvRequest, long startNanos, int timeoutMs,
- Supplier nextIdSupplier, RateLimiter readLimiter,
- RateLimiter writeLimiter, boolean checkReadUnits,
- boolean checkWriteUnits) {
- this.kvRequest = kvRequest;
- this.startNanos = startNanos;
- this.timeoutMs = timeoutMs;
- this.nextIdSupplier = nextIdSupplier;
- this.readLimiter = readLimiter;
- this.writeLimiter = writeLimiter;
- this.checkReadUnits = checkReadUnits;
- this.checkWriteUnits = checkWriteUnits;
-
- this.requestId = Long.toString(nextIdSupplier.get());
- this.requestClass = kvRequest.getClass().getSimpleName();
- }
-
-// RequestContext nextRetryContext() {
-// // Generate a new ID for the next retry attempt using the provided supplier
-// return new RequestContext(this.kvRequest,
-// this.retryCount + 1, this.nextIdSupplier, readLimiter,
-// writeLimiter, checkReadUnits, checkWriteUnits);
-// }
- }
-
/**
* Execute the KV request and return the future response. This is the
- * top-level
- * method for request execution.
+ * top-level method for request execution.
*
* This method handles exceptions to distinguish between what can be retried
* and what cannot, making sure that root cause exceptions are
@@ -461,6 +493,7 @@ private static class RequestContext {
public CompletableFuture execute(Request kvRequest) {
requireNonNull(kvRequest, "NoSQLHandle: request must be non-null");
+
/*
* Before execution, call Request object to assign default values
* from config object if they are not overridden. This allows code
@@ -488,7 +521,7 @@ public CompletableFuture execute(Request kvRequest) {
* non-internal QueryRequest */
if (kvRequest.isQueryRequest()) {
- QueryRequest qreq = (QueryRequest) kvRequest;
+ QueryRequest qreq = (QueryRequest)kvRequest;
/* Set the topo seq num in the request, if it has not been set
* already */
@@ -565,15 +598,15 @@ public CompletableFuture execute(Request kvRequest) {
/* if not, see if we have limiters in our map for the given table */
if (rateLimiterMap != null &&
- readLimiter == null && writeLimiter == null) {
+ readLimiter == null && writeLimiter == null) {
String tableName = kvRequest.getTableName();
- if (tableName != null && !tableName.isEmpty()) {
+ if (tableName != null && tableName.length() > 0) {
readLimiter = rateLimiterMap.getReadLimiter(tableName);
writeLimiter = rateLimiterMap.getWriteLimiter(tableName);
if (readLimiter == null && writeLimiter == null) {
if (kvRequest.doesReads() || kvRequest.doesWrites()) {
backgroundUpdateLimiters(tableName,
- kvRequest.getCompartment());
+ kvRequest.getCompartment());
}
} else {
checkReadUnits = kvRequest.doesReads();
@@ -583,30 +616,42 @@ public CompletableFuture execute(Request kvRequest) {
}
}
}
- kvRequest.setStartNanos(System.nanoTime());
+ kvRequest.setStartNanos(System.nanoTime());
RequestContext ctx = new RequestContext(kvRequest,
kvRequest.getStartNanos(), kvRequest.getTimeoutInternal(),
- maxRequestId::getAndIncrement, readLimiter, writeLimiter,
+ this::nextRequestId, readLimiter, writeLimiter,
checkReadUnits, checkWriteUnits);
-
return executeWithRetry(ctx);
}
- /**
+ /*
* Core method which creates the request and send to the server.
* If the request fails, it performs retry.
*/
private CompletableFuture executeWithRetry(RequestContext ctx) {
- /*
- * Check rate limiters before executing the request.
- * Wait for read and/or write limiters to be below their limits
- * before continuing. Be aware of the timeout given.
- */
- Request kvRequest = ctx.kvRequest;
+ final Request kvRequest = ctx.kvRequest;
+ final int timeoutMs = ctx.timeoutMs;
+ final long startNanos = ctx.startNanos;
+ final int thisIterationTimeoutMs =
+ getIterationTimeoutMs(timeoutMs, startNanos);
+
+ /* Check for over all request timeout first */
+ if (thisIterationTimeoutMs <= 0) {
+ RequestTimeoutException rte = new RequestTimeoutException(timeoutMs,
+ ctx.requestClass + " timed out:" +
+ (ctx.requestId.isEmpty() ? "" : " requestId=" + ctx.requestId) +
+ " nextRequestId=" + nextRequestId() +
+ " iterationTimeout=" + thisIterationTimeoutMs + "ms " +
+ (kvRequest.getRetryStats() != null ?
+ kvRequest.getRetryStats() : ""), ctx.exception);
+ return CompletableFuture.failedFuture(rte);
+ }
+
+ /* Log retry */
if (kvRequest.getNumRetries() > 0) {
logRetries(kvRequest.getNumRetries(), ctx.exception);
}
@@ -614,21 +659,20 @@ private CompletableFuture executeWithRetry(RequestContext ctx) {
if (serialVersion < 3 && kvRequest instanceof DurableRequest) {
if (((DurableRequest)kvRequest).getDurability() != null) {
oneTimeMessage("The requested feature is not supported " +
- "by the connected server: Durability");
+ "by the connected server: Durability");
}
}
if (serialVersion < 3 && kvRequest instanceof TableRequest) {
TableLimits limits = ((TableRequest)kvRequest).getTableLimits();
if (limits != null &&
- limits.getMode() == CapacityMode.ON_DEMAND) {
+ limits.getMode() == CapacityMode.ON_DEMAND) {
oneTimeMessage("The requested feature is not supported " +
- "by the connected server: on demand " +
- "capacity table");
+ "by the connected server: on demand " +
+ "capacity table");
}
}
return handlePreRateLimit(ctx)
- //TODO is this thenComposeAsync?
.thenCompose( (Integer delay) -> getAuthString(ctx, authProvider))
.thenCompose((String authString) -> createRequest(ctx, authString))
.thenCompose((FullHttpRequest request) -> submitRequest(ctx,request))
@@ -638,8 +682,7 @@ private CompletableFuture executeWithRetry(RequestContext ctx) {
.handle((Result result, Throwable err) -> {
/* Handle error and retry */
if (err != null) {
- ctx.exception = err;
- return handleError(ctx);
+ return handleError(ctx, err);
} else {
return CompletableFuture.completedFuture(result);
}
@@ -648,32 +691,33 @@ private CompletableFuture executeWithRetry(RequestContext ctx) {
}
private CompletableFuture handlePreRateLimit(RequestContext ctx) {
-
- /* Check this request doesn't cause throttle */
+ /*
+ * Check rate limiters before executing the request.
+ * Wait for read and/or write limiters to be below their limits
+ * before continuing. Be aware of the timeout given.
+ */
int preRateLimitDelayMs = 0;
if (ctx.readLimiter != null && ctx.checkReadUnits) {
- preRateLimitDelayMs +=
- ((SimpleRateLimiter) ctx.readLimiter).consumeExternally(0);
+ preRateLimitDelayMs += ((SimpleRateLimiter) ctx.readLimiter)
+ .consumeExternally(0);
}
if (ctx.writeLimiter != null && ctx.checkWriteUnits) {
- preRateLimitDelayMs +=
- ((SimpleRateLimiter) ctx.writeLimiter).consumeExternally(0);
+ preRateLimitDelayMs += ((SimpleRateLimiter) ctx.writeLimiter)
+ .consumeExternally(0);
}
- final int timeoutMs = ctx.timeoutMs;
- final long startNanos = ctx.startNanos;
- final int thisIterationTimeoutMs = getIterationTimeoutMs(timeoutMs, startNanos);
+ int thisIterationTimeoutMs =
+ getIterationTimeoutMs(ctx.timeoutMs, ctx.startNanos);
- /* If this result in timeout, complete with exception */
+ /* If rate limit result in timeout, complete with exception. */
if (thisIterationTimeoutMs <= preRateLimitDelayMs) {
final TimeoutException ex = new TimeoutException(
"timed out waiting "
+ thisIterationTimeoutMs
+ "ms due to rate limiting");
- return createDelayFuture(thisIterationTimeoutMs).thenCompose(d ->
- CompletableFuture.failedFuture(ex));
+ return createDelayFuture(thisIterationTimeoutMs)
+ .thenCompose(d -> CompletableFuture.failedFuture(ex));
}
-
/* sleep for delay ms */
return createDelayFuture(preRateLimitDelayMs)
.whenComplete((delay, err) -> ctx.rateDelayedMs.addAndGet(delay));
@@ -684,7 +728,7 @@ private CompletableFuture handlePreRateLimit(RequestContext ctx) {
* This may contact the server to get the token.
*/
private CompletableFuture getAuthString(RequestContext ctx,
- AuthorizationProvider authProvider) {
+ AuthorizationProvider authProvider) {
final Request kvRequest = ctx.kvRequest;
return authProvider.getAuthorizationStringAsync(kvRequest)
.thenApply(authString -> {
@@ -707,7 +751,7 @@ private CompletableFuture getAuthString(RequestContext ctx,
* This may contact the server to sign the request body.
*/
private CompletableFuture createRequest(RequestContext ctx,
- String authString) {
+ String authString) {
ByteBuf buffer = null;
try {
buffer = Unpooled.buffer();
@@ -734,7 +778,7 @@ private CompletableFuture createRequest(RequestContext ctx,
* processing (retry delays, etc) work correctly.
*/
kvRequest.setTimeoutInternal(
- getIterationTimeoutMs(ctx.timeoutMs, ctx.startNanos));
+ getIterationTimeoutMs(ctx.timeoutMs, ctx.startNanos));
writeContent(buffer, ctx);
kvRequest.setTimeoutInternal(ctx.timeoutMs);
@@ -790,7 +834,8 @@ private CompletableFuture createRequest(RequestContext ctx,
* with content
*/
byte[] content = signContent ? getBodyBytes(buffer) : null;
- return authProvider.setRequiredHeadersAsync(authString, kvRequest, headers, content)
+ return authProvider.
+ setRequiredHeadersAsync(authString, kvRequest, headers, content)
.thenApply(n -> {
String namespace = kvRequest.getNamespace();
if (namespace == null) {
@@ -801,7 +846,7 @@ private CompletableFuture createRequest(RequestContext ctx,
}
return request;
});
- } catch (Exception e) {
+ } catch (Throwable e) {
/* Release the buffer on error */
if (buffer != null) {
buffer.release();
@@ -814,7 +859,7 @@ private CompletableFuture createRequest(RequestContext ctx,
* Send the HTTP request to server and get the response back.
*/
private CompletableFuture submitRequest(RequestContext ctx,
- HttpRequest request) {
+ HttpRequest request) {
final Request kvRequest = ctx.kvRequest;
if (isLoggable(logger, Level.FINE) && !kvRequest.getIsRefresh()) {
logTrace(logger, "Request: " + ctx.requestClass +
@@ -825,7 +870,8 @@ private CompletableFuture submitRequest(RequestContext ctx,
return httpClient.runRequest(request, timeoutMs)
.whenComplete((res, err) -> {
- ctx.networkLatency = (System.nanoTime() - ctx.latencyNanos) / 1_000_000;
+ ctx.networkLatency =
+ (System.nanoTime() - ctx.latencyNanos) / 1_000_000;
});
}
@@ -840,8 +886,10 @@ private Result handleResponse(RequestContext ctx, FullHttpResponse fhr) {
", requestId=" + ctx.requestId );
}
try {
- Result result = processResponse(fhr.status(), fhr.headers(), fhr.content(), ctx);
- ctx.rateDelayedMs.addAndGet(getRateDelayedFromHeader(fhr.headers()));
+ Result result = processResponse(
+ fhr.status(), fhr.headers(), fhr.content(), ctx);
+ ctx.rateDelayedMs.addAndGet(
+ getRateDelayedFromHeader(fhr.headers()));
ctx.resSize = fhr.content().readerIndex();
return result;
} finally {
@@ -892,7 +940,8 @@ private Result handleResult(RequestContext ctx, Result result) {
* Handle rate limit from the Result.
* This will consume actual units used by the request and sleep.
*/
- private CompletableFuture handlePostRateLimit(RequestContext ctx, Result result) {
+ private CompletableFuture handlePostRateLimit(RequestContext ctx,
+ Result result) {
final Request kvRequest = ctx.kvRequest;
int postRateLimitDelayMs = consumeLimiterUnits(ctx.readLimiter,
result.getReadUnitsInternal());
@@ -916,185 +965,281 @@ private CompletableFuture handlePostRateLimit(RequestContext ctx, Result
});
}
- private CompletableFuture handleError(RequestContext ctx) {
- Request kvRequest = ctx.kvRequest;
- Throwable err = ctx.exception;
- Throwable actualCause = (err instanceof CompletionException) ?
+ /*
+ * Main error handling entry point.
+ */
+ private CompletableFuture handleError(RequestContext ctx,
+ Throwable err) {
+ final Throwable actualCause =
+ (err instanceof CompletionException && err.getCause() != null) ?
err.getCause() : err;
- boolean retryable = false;
- int delayMs = 0;
- try {
- throw actualCause;
- } catch (AuthenticationException rae) {
- if (authProvider instanceof StoreAccessTokenProvider) {
- final StoreAccessTokenProvider satp =
- (StoreAccessTokenProvider) authProvider;
- satp.bootstrapLogin(kvRequest);
- retryable = true;
- } else {
- logInfo(logger, "Unexpected authentication exception: " +
- rae);
- throw new NoSQLException("Unexpected exception: " +
- rae.getMessage(), rae);
- }
- } catch (InvalidAuthorizationException iae) {
- /*
- * Allow a single retry for invalid/expired auth
- * This includes "clock skew" errors
- * This does not include permissions-related errors
- */
- if (kvRequest.getNumRetries() == 0) {
- authProvider.flushCache();
- logFine(logger,
- "Client retrying on InvalidAuthorizationException: " +
- iae.getMessage());
- retryable = true;
- } else {
- throw iae;
- }
+ /* set exception on context */
+ ctx.exception = actualCause;
- } catch (SecurityInfoNotReadyException sinre) {
- kvRequest.addRetryException(sinre.getClass());
- if (kvRequest.getNumRetries() < 20) {
- delayMs = SEC_ERROR_DELAY_MS;
- retryable = true;
- }
- } catch (RetryableException re) {
- if (re instanceof WriteThrottlingException &&
- ctx.writeLimiter != null) {
- /* ensure we check write limits next loop */
- ctx.checkWriteUnits = true;
- /* set limiter to its limit, if not over already */
- if (ctx.writeLimiter.getCurrentRate() < 100.0) {
- ctx.writeLimiter.setCurrentRate(100.0);
- }
- /* call retry handler to manage sleep/delay */
- }
- if (re instanceof ReadThrottlingException &&
- ctx.readLimiter != null) {
- /* ensure we check read limits next loop */
- ctx.checkReadUnits = true;
- /* set limiter to its limit, if not over already */
- if (ctx.readLimiter.getCurrentRate() < 100.0) {
- ctx.readLimiter.setCurrentRate(100.0);
- }
- /* call retry handler to manage sleep/delay */
- }
+ /* Get the appropriate error handler and delegate */
+ ErrorHandler handler = findErrorHandler(actualCause.getClass());
+ if (handler != null) {
+ return handler.handle(ctx, actualCause);
+ }
- logFine(logger, "Retryable exception: " +
- re.getMessage());
- /*
- * Handle automatic retries. If this does not throw an error,
- * then the delay (if any) will have been performed and the
- * request should be retried.
- *
- * If there have been too many retries this method will
- * throw the original exception.
- */
+ /* Default throwable: retry with small delay */
+ final String name = actualCause.getClass().getName();
+ logInfo(logger, "Client execute Throwable, name: " +
+ name + "message: " + actualCause.getMessage());
+ return retryRequest(ctx, 10, actualCause);
+ }
- kvRequest.addRetryException(re.getClass());
- delayMs = handleRetry(re, kvRequest);
- retryable = true;
- } catch (UnsupportedQueryVersionException uqve) {
- /* decrement query version and try again */
- if (decrementQueryVersion(ctx.queryVersionUsed) == true) {
- logFine(logger, "Got unsupported query version error " +
- "from server: decrementing query version to " +
- queryVersion + " and trying again.");
- retryable = true;
- } else {
- throw uqve;
- }
- } catch (UnsupportedProtocolException upe) {
- /* decrement protocol version and try again */
- if (decrementSerialVersion(ctx.serialVersionUsed) == true) {
- /* Don't set this exception: it's misleading */
- /* exception = upe; */
- logFine(logger, "Got unsupported protocol error " +
- "from server: decrementing serial version to " +
- serialVersion + " and trying again.");
- retryable = true;
- } else {
- throw upe;
- }
- } catch (NoSQLException nse) {
- logFine(logger, "Client execute NoSQLException: " +
- nse.getMessage());
- throw nse;
- } catch (RuntimeException e) {
- if (!kvRequest.getIsRefresh()) {
- /* don't log expected failures from refresh */
- logFine(logger, "Client execute runtime exception: " +
- e.getMessage());
- }
- } catch (IOException ioe) {
- String name = ioe.getClass().getName();
- logFine(logger, "Client execution IOException, name: " +
- name + ", message: " + ioe.getMessage());
- if (kvRequest.getNumRetries() == 0) {
- retryable = true;
- delayMs = 10;
+ /*
+ * Initializes the error handlers map with specific exception types
+ * and their corresponding handling strategies.
+ * This method sets up a mapping between various exception classes
+ * and the methods responsible for handling them,facilitating appropriate
+ * error management and retry logic during request execution.
+ */
+ private void initErrorHandlers() {
+ errorHandlers.put(AuthenticationException.class,
+ this::handleAuthException);
+ errorHandlers.put(InvalidAuthorizationException.class,
+ this::handleInvalidAuthError);
+ errorHandlers.put(SecurityInfoNotReadyException.class,
+ this::handleSecurityNotReadyError);
+ errorHandlers.put(RetryableException.class,
+ this::handleRetryableError);
+ errorHandlers.put(UnsupportedQueryVersionException.class,
+ this::handleQueryVerError);
+ errorHandlers.put(UnsupportedProtocolException.class,
+ this::handleProtocolVerError);
+ errorHandlers.put(RequestTimeoutException.class, this::failRequest);
+ errorHandlers.put(NoSQLException.class, this::failRequest);
+ errorHandlers.put(RuntimeException.class, this::failRequest);
+ errorHandlers.put(IOException.class, this::handleIOError);
+ errorHandlers.put(InterruptedException.class,
+ this::handleInterruptedError);
+ errorHandlers.put(ExecutionException.class, this::handleExecutionError);
+ errorHandlers.put(TimeoutException.class, this::handleTimeoutError);
+ /* Add any new error handlers here */
+ }
+
+ /*
+ * Marks the request as failed and returns failed {@link CompletableFuture}.
+ */
+ private CompletableFuture failRequest(RequestContext ctx,
+ Throwable ex) {
+ final String name = ex.getClass().getName();
+ final String message = String.format("Client execute %s: %s",
+ name, ex.getMessage());
+ logFine(logger, message);
+ ctx.kvRequest.setRateLimitDelayedMs(ctx.rateDelayedMs.get());
+ statsControl.observeError(ctx.kvRequest);
+ return CompletableFuture.failedFuture(ex);
+ }
+
+ /*
+ * Schedules a retry for the request with the given delay.
+ * Updates retry counters and statistics.
+ */
+ private CompletableFuture retryRequest(RequestContext ctx,
+ int delayMs, Throwable ex) {
+ Request kvRequest = ctx.kvRequest;
+ /* query and protocol exceptions are not errors, do not add them to
+ * retry stats.
+ */
+ if (!(ex instanceof UnsupportedProtocolException
+ || ex instanceof UnsupportedQueryVersionException)) {
+ kvRequest.addRetryException(ex.getClass());
+ kvRequest.incrementRetries();
+ kvRequest.addRetryDelayMs(delayMs);
+ }
+ return scheduleRetry(ctx, delayMs);
+ }
+
+ /**
+ * Looks up an error handler for the given class by traversing the
+ * class hierarchy until a registered handler is found.
+ *
+ * @param clazz Exception class to resolve
+ * @return A matching {@link ErrorHandler} or {@code null} if none found
+ */
+ private ErrorHandler findErrorHandler(Class> clazz) {
+ while (clazz != null) {
+ if (errorHandlers.containsKey(clazz)) {
+ return errorHandlers.get(clazz);
}
- } catch (InterruptedException ie) {
- logInfo(logger, "Client interrupted exception: " +
- ie.getMessage());
- } catch (ExecutionException ee) {
- /*
- * This can happen if a channel is bad in HttpClient.getChannel.
- * This happens if the channel is shut down by the server side
- * or the server (proxy) is restarted, etc. Treat it like
- * IOException above, but retry without waiting
- */
- String name = ee.getCause().getClass().getName();
- logFine(logger, "Client ExecutionException, name: " +
- name + ", message: " + ee.getMessage() + ", retrying");
+ clazz = clazz.getSuperclass();
+ }
+ return null;
+ }
+
+ /*
+ * Error handler for {@link AuthenticationException}
+ */
+ private CompletableFuture handleAuthException(RequestContext ctx,
+ Throwable ex) {
- if (kvRequest.getNumRetries() == 0) {
- retryable = true;
+ if (authProvider instanceof StoreAccessTokenProvider) {
+ authProvider.flushCache();
+ return retryRequest(ctx, 0, ex);
+ } else {
+ logInfo(logger, "Unexpected authentication exception: " + ex);
+ return failRequest(ctx, new NoSQLException(
+ "Unexpected exception: " + ex.getMessage(), ex));
+ }
+ }
+
+ /*
+ * Error handler for {@link InvalidAuthorizationException}
+ */
+ private CompletableFuture handleInvalidAuthError(RequestContext ctx,
+ Throwable ex) {
+ Request kvRequest = ctx.kvRequest;
+ if (kvRequest.getNumRetries() > 0) {
+ return failRequest(ctx, ex);
+ }
+ authProvider.flushCache();
+ logFine(logger,
+ "Client retrying on InvalidAuthorizationException: "
+ + ex.getMessage());
+ return retryRequest(ctx, 0, ex);
+ }
+
+ /*
+ * Error handler for {@link SecurityInfoNotReadyException}
+ */
+ private CompletableFuture handleSecurityNotReadyError(
+ RequestContext ctx, Throwable ex) {
+
+ Request kvRequest = ctx.kvRequest;
+ int delayMs = SEC_ERROR_DELAY_MS;
+ if (kvRequest.getNumRetries() > 10) {
+ delayMs = DefaultRetryHandler.computeBackoffDelay(kvRequest, 0);
+ if (delayMs <= 0) {
+ return failRequest(ctx,
+ new RequestTimeoutException(ctx.timeoutMs,
+ ctx.requestClass + " timed out:" +
+ (ctx.requestId.isEmpty() ? "" :
+ " requestId=" + ctx.requestId) +
+ " nextRequestId=" + nextRequestId() +
+ (kvRequest.getRetryStats() != null ?
+ kvRequest.getRetryStats() : ""), ctx.exception));
}
- } catch (TimeoutException te) {
- //TODO handle this
- } catch (Throwable t) {
- /*
- * this is likely an exception from Netty, perhaps a bad
- * connection. Retry.
- */
- /* Maybe make this logFine */
- String name = t.getClass().getName();
- logInfo(logger, "Client execute Throwable, name: " +
- name + "message: " + t.getMessage());
}
+ return retryRequest(ctx, delayMs, ex);
+ }
+
+
+ /*
+ * Error handler for {@link RetryableException}
+ */
+ private CompletableFuture handleRetryableError(RequestContext ctx,
+ Throwable ex) {
+ Request kvRequest = ctx.kvRequest;
- if (retryable) {
- if (!(actualCause instanceof UnsupportedProtocolException)
- && !(actualCause instanceof UnsupportedQueryVersionException)) {
- kvRequest.addRetryException(err.getClass());
- kvRequest.incrementRetries();
+ if (ex instanceof WriteThrottlingException && ctx.writeLimiter != null) {
+ /* ensure we check write limits next retry */
+ ctx.checkWriteUnits = true;
+ /* set limiter to its limit, if not over already */
+ if (ctx.writeLimiter.getCurrentRate() < 100.0) {
+ ctx.writeLimiter.setCurrentRate(100.0);
}
+ }
+ if (ex instanceof ReadThrottlingException && ctx.readLimiter != null) {
+ /* ensure we check read limits next loop */
+ ctx.checkReadUnits = true;
+ /* set limiter to its limit, if not over already */
+ if (ctx.readLimiter.getCurrentRate() < 100.0) {
+ ctx.readLimiter.setCurrentRate(100.0);
+ }
+ }
+ logFine(logger, "Retryable exception: " + ex.getMessage());
+ /*
+ * Handle automatic retries. If this does not throw an error,
+ * then the delay (if any) will have been performed and the
+ * request should be retried.
+ *
+ * If there have been too many retries this method will
+ * throw the original exception.
+ */
+ int delayMs = handleRetry((RetryableException) ex, kvRequest);
+ return retryRequest(ctx, delayMs, ex);
+ }
+ /*
+ * Error handler for {@link UnsupportedQueryVersionException}
+ */
+ private CompletableFuture handleQueryVerError(RequestContext ctx,
+ Throwable ex) {
+ if (decrementQueryVersion(ctx.queryVersionUsed)) {
+ logFine(logger, "Got unsupported query version error " +
+ "from server: decrementing query version to " +
+ queryVersion + " and trying again.");
+ return retryRequest(ctx, 0, ex);
+ }
+ return failRequest(ctx, ex);
+ }
- CompletableFuture retryFuture = new CompletableFuture<>();
- int finalDelayMs = delayMs;
- taskExecutor.schedule(() -> {
- /* Increment request-id for retry */
- ctx.requestId = String.valueOf(ctx.nextIdSupplier.get());
- executeWithRetry(ctx)
- .whenComplete((res,e) -> {
- kvRequest.addRetryDelayMs(finalDelayMs);
- if (e != null) {
- retryFuture.completeExceptionally(e);
- } else {
- retryFuture.complete(res);
- }
- });
- }, delayMs, TimeUnit.MILLISECONDS);
- return retryFuture;
+ /*
+ * Error handler for {@link UnsupportedProtocolException}
+ */
+ private CompletableFuture handleProtocolVerError(RequestContext ctx,
+ Throwable ex) {
+ if (decrementSerialVersion(ctx.serialVersionUsed)) {
+ logFine(logger, "Got unsupported protocol error " +
+ "from server: decrementing serial version to " +
+ serialVersion + " and trying again.");
+ return retryRequest(ctx, 0, ex);
}
- /* No retry, complete with exception */
- kvRequest.setRateLimitDelayedMs(ctx.rateDelayedMs.get());
- statsControl.observeError(kvRequest);
- return CompletableFuture.failedFuture(actualCause);
+ return failRequest(ctx, ex);
+ }
+
+ /*
+ * Error handler for {@link IOException}
+ */
+ private CompletableFuture handleIOError(RequestContext ctx,
+ Throwable ex) {
+ Request kvRequest = ctx.kvRequest;
+ String name = ex.getClass().getName();
+ logFine(logger, "Client execution IOException, name: " +
+ name + ", message: " + ex.getMessage());
+ if (kvRequest.getNumRetries() > 10) {
+ return failRequest(ctx, ex);
+ }
+ return retryRequest(ctx, 10, ex);
+ }
+
+ private CompletableFuture handleInterruptedError(RequestContext ctx,
+ Throwable ex) {
+ logInfo(logger, "Interrupted: " + ex.getMessage());
+ return failRequest(ctx,
+ new NoSQLException("Request interrupted: " + ex.getMessage()));
+ }
+
+ private CompletableFuture handleExecutionError(RequestContext ctx,
+ Throwable ex) {
+ /*
+ * This can happen if a channel is bad in HttpClient.getChannel.
+ * This happens if the channel is shut down by the server side
+ * or the server (proxy) is restarted, etc. Treat it like
+ * IOException above, but retry without waiting
+ */
+ String name = ex.getCause().getClass().getName();
+ logFine(logger, "Client ExecutionException, name: " +
+ name + ", message: " + ex.getMessage() + ", retrying");
+ return retryRequest(ctx, 10, ex);
+ }
+
+ private CompletableFuture handleTimeoutError(RequestContext ctx,
+ Throwable ex) {
+ logInfo(logger, "Timeout exception: " + ex);
+ return failRequest(ctx,
+ new RequestTimeoutException(
+ ctx.timeoutMs,
+ ctx.requestClass + " timed out:" +
+ (ctx.requestId.isEmpty() ? "" : " requestId=" + ctx.requestId) +
+ " nextRequestId=" + nextRequestId() +
+ (ctx.kvRequest.getRetryStats() != null ?
+ ctx.kvRequest.getRetryStats() : ""),
+ ctx.exception));
}
/**
@@ -1108,12 +1253,32 @@ private CompletableFuture createDelayFuture(int delayMs) {
CompletableFuture delayFuture = new CompletableFuture<>();
if (delayMs > 0) {
taskExecutor.schedule(() -> delayFuture.complete(delayMs), delayMs,
- TimeUnit.MILLISECONDS);
+ TimeUnit.MILLISECONDS);
} else {
delayFuture.complete(delayMs); // Complete immediately if no delay
}
return delayFuture;
}
+
+ private CompletableFuture scheduleRetry(RequestContext ctx,
+ int delayMs) {
+ //TODO check for overall timeout before schedule
+ CompletableFuture retryFuture = new CompletableFuture<>();
+ taskExecutor.schedule(() -> {
+ /* Increment request-id for retry */
+ ctx.requestId = String.valueOf(ctx.nextIdSupplier.get());
+ executeWithRetry(ctx)
+ .whenComplete((res, e) -> {
+ ctx.kvRequest.addRetryDelayMs(delayMs);
+ if (e != null) {
+ retryFuture.completeExceptionally(e);
+ } else {
+ retryFuture.complete(res);
+ }
+ });
+ }, delayMs, TimeUnit.MILLISECONDS);
+ return retryFuture;
+ }
/**
* Calculate the timeout for the next iteration.
* This is basically the given timeout minus the time
@@ -1150,7 +1315,7 @@ private RateLimiter getQueryRateLimiter(Request request, boolean read) {
* the tablename from the request and apply rate limiting.
*/
String tableName = ((QueryRequest)request).getTableName();
- if (tableName == null || tableName.isEmpty()) {
+ if (tableName == null || tableName == "") {
return null;
}
@@ -1230,10 +1395,11 @@ public boolean updateRateLimiters(String tableName, TableLimits limits) {
*/
/* allow tests to override this hardcoded setting */
- int durationSeconds = Integer.getInteger("test.rldurationsecs", 30);
+ int durationSeconds = Integer.getInteger("test.rldurationsecs", 30)
+ .intValue();
- double RUs = limits.getReadUnits();
- double WUs = limits.getWriteUnits();
+ double RUs = (double)limits.getReadUnits();
+ double WUs = (double)limits.getWriteUnits();
/* if there's a specified rate limiter percentage, use that */
double rlPercent = config.getDefaultRateLimitingPercentage();
@@ -1278,6 +1444,7 @@ boolean timeoutRequest(long startNanos,
*/
private void writeContent(ByteBuf content, RequestContext ctx)
throws IOException {
+
final Request kvRequest = ctx.kvRequest;
final NettyByteOutputStream bos = new NettyByteOutputStream(content);
ctx.serialVersionUsed = serialVersion;
@@ -1509,18 +1676,18 @@ private void setTableNeedsRefresh(String tableName, boolean needsRefresh) {
private void backgroundUpdateLimiters(String tableName,
String compartmentId) {
ConcurrentUtil.synchronizedCall(this.lock, () -> {
- if (!tableNeedsRefresh(tableName)) {
- return;
- }
- setTableNeedsRefresh(tableName, false);
+ if (tableNeedsRefresh(tableName) == false) {
+ return;
+ }
+ setTableNeedsRefresh(tableName, false);
- try {
- threadPool.execute(() -> {
- updateTableLimiters(tableName, compartmentId);
- });
- } catch (RejectedExecutionException e) {
- setTableNeedsRefresh(tableName, true);
- }
+ try {
+ threadPool.execute(() -> {
+ updateTableLimiters(tableName, compartmentId);
+ });
+ } catch (RejectedExecutionException e) {
+ setTableNeedsRefresh(tableName, true);
+ }
});
}
@@ -1537,7 +1704,7 @@ private void updateTableLimiters(String tableName, String compartmentId) {
try {
logFine(logger, "Starting GetTableRequest for table '" +
tableName + "'");
- res = (TableResult) this.execute(gtr).get();
+ res = (TableResult) ConcurrentUtil.awaitFuture(this.execute(gtr));
} catch (Exception e) {
logFine(logger, "GetTableRequest for table '" +
tableName + "' returned exception: " + e.getMessage());
@@ -1696,20 +1863,19 @@ StatsControl getStatsControl() {
* false: already at lowest version number.
*/
private boolean decrementSerialVersion(short versionUsed) {
- return ConcurrentUtil.synchronizedCall(this.lock,
- () -> {
- if (serialVersion != versionUsed) {
- return true;
- }
- if (serialVersion == V4) {
- serialVersion = V3;
- return true;
- }
- if (serialVersion == V3) {
- serialVersion = V2;
- return true;
- }
- return false;
+ return ConcurrentUtil.synchronizedCall(this.lock, () -> {
+ if (serialVersion != versionUsed) {
+ return true;
+ }
+ if (serialVersion == V4) {
+ serialVersion = V3;
+ return true;
+ }
+ if (serialVersion == V3) {
+ serialVersion = V2;
+ return true;
+ }
+ return false;
});
}
@@ -1721,18 +1887,17 @@ private boolean decrementSerialVersion(short versionUsed) {
* false: already at lowest version number.
*/
private boolean decrementQueryVersion(short versionUsed) {
- return ConcurrentUtil.synchronizedCall(this.lock,
- () -> {
- if (queryVersion != versionUsed) {
- return true;
- }
+ return ConcurrentUtil.synchronizedCall(this.lock, () -> {
+ if (queryVersion != versionUsed) {
+ return true;
+ }
- if (queryVersion == QueryDriver.QUERY_V3) {
- return false;
- }
+ if (queryVersion == QueryDriver.QUERY_V3) {
+ return false;
+ }
- --queryVersion;
- return true;
+ --queryVersion;
+ return true;
});
}
@@ -1868,28 +2033,27 @@ private boolean stringsEqualOrNull(String s1, String s2) {
* This is synchronized to avoid 2 requests adding the same table
*/
private void addRequestToRefreshList(Request request) {
- ConcurrentUtil.synchronizedCall(this.lock,
- () -> {
- logFine(logger, "Adding table to request list: " +
- request.getCompartment() + ":" + request.getTableName());
- PutRequest pr =
- new PutRequest().setTableName(request.getTableName());
- pr.setCompartmentInternal(request.getCompartment());
- pr.setValue(badValue);
- pr.setIsRefresh(true);
- authRefreshRequests.add(pr);
- GetRequest gr =
- new GetRequest().setTableName(request.getTableName());
- gr.setCompartmentInternal(request.getCompartment());
- gr.setKey(badValue);
- gr.setIsRefresh(true);
- authRefreshRequests.add(gr);
- DeleteRequest dr =
- new DeleteRequest().setTableName(request.getTableName());
- dr.setCompartmentInternal(request.getCompartment());
- dr.setKey(badValue);
- dr.setIsRefresh(true);
- authRefreshRequests.add(dr);
+ ConcurrentUtil.synchronizedCall(this.lock, () -> {
+ logFine(logger, "Adding table to request list: " +
+ request.getCompartment() + ":" + request.getTableName());
+ PutRequest pr =
+ new PutRequest().setTableName(request.getTableName());
+ pr.setCompartmentInternal(request.getCompartment());
+ pr.setValue(badValue);
+ pr.setIsRefresh(true);
+ authRefreshRequests.add(pr);
+ GetRequest gr =
+ new GetRequest().setTableName(request.getTableName());
+ gr.setCompartmentInternal(request.getCompartment());
+ gr.setKey(badValue);
+ gr.setIsRefresh(true);
+ authRefreshRequests.add(gr);
+ DeleteRequest dr =
+ new DeleteRequest().setTableName(request.getTableName());
+ dr.setCompartmentInternal(request.getCompartment());
+ dr.setKey(badValue);
+ dr.setIsRefresh(true);
+ authRefreshRequests.add(dr);
});
}
@@ -1898,12 +2062,11 @@ private void addRequestToRefreshList(Request request) {
* for internal use
*/
public void oneTimeMessage(String msg) {
- ConcurrentUtil.synchronizedCall(this.lock,
- () -> {
- if (oneTimeMessages.add(msg) == false) {
- return;
- }
- logWarning(logger, msg);
+ ConcurrentUtil.synchronizedCall(this.lock, () -> {
+ if (oneTimeMessages.add(msg) == false) {
+ return;
+ }
+ logWarning(logger, msg);
});
}
@@ -2009,22 +2172,21 @@ public TopologyInfo getTopology() {
}
private int getTopoSeqNum() {
- return ConcurrentUtil.synchronizedCall(this.lock,
- () -> (topology == null ? -1 : topology.getSeqNum()));
+ return ConcurrentUtil.synchronizedCall(this.lock, () ->
+ (topology == null ? -1 : topology.getSeqNum()));
}
private void setTopology(TopologyInfo topo) {
- ConcurrentUtil.synchronizedCall(this.lock,
- () -> {
- if (topo == null) {
- return;
- }
+ ConcurrentUtil.synchronizedCall(this.lock, () -> {
+ if (topo == null) {
+ return;
+ }
- if (topology == null || topology.getSeqNum() < topo.getSeqNum()) {
- topology = topo;
- trace("New topology: " + topo, 1);
- }
+ if (topology == null || topology.getSeqNum() < topo.getSeqNum()) {
+ topology = topo;
+ trace("New topology: " + topo, 1);
+ }
});
}
diff --git a/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
index 07230b57..d40abb17 100644
--- a/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
+++ b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
@@ -9,13 +9,11 @@
import java.util.ArrayList;
import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
import java.util.logging.Logger;
import javax.net.ssl.SSLException;
import oracle.nosql.driver.AuthorizationProvider;
-import oracle.nosql.driver.NoSQLException;
import oracle.nosql.driver.NoSQLHandle;
import oracle.nosql.driver.NoSQLHandleConfig;
import oracle.nosql.driver.StatsControl;
@@ -55,6 +53,7 @@
import oracle.nosql.driver.ops.TableUsageResult;
import oracle.nosql.driver.ops.WriteMultipleRequest;
import oracle.nosql.driver.ops.WriteMultipleResult;
+import oracle.nosql.driver.util.ConcurrentUtil;
import oracle.nosql.driver.values.FieldValue;
import oracle.nosql.driver.values.JsonUtils;
import oracle.nosql.driver.values.MapValue;
@@ -206,23 +205,14 @@ public MultiDeleteResult multiDelete(MultiDeleteRequest request) {
@Override
public QueryResult query(QueryRequest request) {
- try {
- return queryAsync(request).get();
- } catch (InterruptedException ie) {
- throw new NoSQLException("Request interrupted: " + ie.getMessage(), ie);
- } catch (ExecutionException e) {
- if (e.getCause() instanceof RuntimeException) {
- throw ((RuntimeException) e.getCause());
- }
- throw new NoSQLException("ExecutionException: " + e.getMessage(),
- e.getCause());
- }
+ return ConcurrentUtil.awaitFuture(queryAsync(request));
}
- public CompletableFuture queryAsync(QueryRequest request) {
-
+ private CompletableFuture queryAsync(QueryRequest request) {
+ checkClient();
return client.execute(request)
.thenCompose(result -> {
+ /* Complex queries need RCB, run asynchronously */
if (!request.isSimpleQuery()) {
// TODO supplyAsync runs in fork-join pool.
// Change to dedicated pool
@@ -233,7 +223,6 @@ public CompletableFuture queryAsync(QueryRequest request) {
.thenApply(result -> ((QueryResult) result));
}
-
@Override
public QueryIterableResult queryIterable(QueryRequest request) {
checkClient();
@@ -490,16 +479,6 @@ public void refresh(long refreshMs) {
@SuppressWarnings("unchecked")
private T executeSync(Request request) {
checkClient();
- try {
- return (T) client.execute(request).get();
- } catch (InterruptedException ie) {
- throw new NoSQLException("Request interrupted: " + ie.getMessage(), ie);
- } catch (ExecutionException e) {
- if (e.getCause() instanceof RuntimeException) {
- throw ((RuntimeException) e.getCause());
- }
- throw new NoSQLException("ExecutionException: " + e.getMessage(),
- e.getCause());
- }
+ return (T) ConcurrentUtil.awaitFuture(client.execute(request));
}
}
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java b/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
index b0696c41..4d323110 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
@@ -10,13 +10,11 @@
import static oracle.nosql.driver.util.LogUtil.logFine;
import static oracle.nosql.driver.util.LogUtil.logInfo;
-import java.io.IOException;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.logging.Logger;
@@ -29,7 +27,6 @@
import io.netty.channel.pool.ChannelPoolHandler;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.Promise;
-import io.netty.util.concurrent.ScheduledFuture;
/**
* A class to manage and pool Netty Channels (connections). This is used
@@ -471,10 +468,9 @@ int doKeepAlive(int keepAlivePeriod) {
continue;
}
logFine(logger,
- "Sending keepalive on channel " + ch + ", stats: " + cs);
+ "Sending keepalive on channel " + ch + ", stats: " + cs);
keepAlive.keepAlive(ch).handle((didKeepalive, err) -> {
if (err != null) {
- // TODO log err
logFine(logger,
"Keepalive failed on channel "
+ ch
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
index b7e684b1..e7eccd91 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
@@ -11,10 +11,8 @@
import static io.netty.handler.codec.http.HttpMethod.HEAD;
import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
import static oracle.nosql.driver.util.HttpConstants.CONNECTION;
-import static oracle.nosql.driver.util.LogUtil.isFineEnabled;
import static oracle.nosql.driver.util.LogUtil.logFine;
import static oracle.nosql.driver.util.LogUtil.logInfo;
-import static oracle.nosql.driver.util.LogUtil.logWarning;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
@@ -34,11 +32,11 @@
import io.netty.handler.ssl.SslContext;
import io.netty.handler.timeout.ReadTimeoutHandler;
import io.netty.util.AttributeKey;
-import io.netty.util.concurrent.Future;
/*
* If this code is ever made generic, the proxy information obtained
* from this config needs to be abstracted to a generic class.
*/
+import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.FutureListener;
import oracle.nosql.driver.NoSQLHandleConfig;
@@ -90,8 +88,8 @@ public class HttpClient {
/* AttributeKey to attach a CompletableFuture to the Channel,
* allowing the HttpResponseHandler to signal completion.
*/
- public static final AttributeKey> STATE_KEY =
- AttributeKey.valueOf("rqstate");
+ public static final AttributeKey>
+ STATE_KEY = AttributeKey.valueOf("rqstate");
//private final FixedChannelPool pool;
private final ConnectionPool pool;
@@ -394,74 +392,6 @@ public void shutdown() {
syncUninterruptibly();
}
-// public Channel getChannel(int timeoutMs)
-// throws InterruptedException, ExecutionException, TimeoutException {
-//
-// long startMs = System.currentTimeMillis();
-// long now = startMs;
-// int retries = 0;
-//
-// while (true) {
-// long msDiff = now - startMs;
-//
-// /* retry loop with at most (retryInterval) ms timeouts */
-// long thisTimeoutMs = (timeoutMs - msDiff);
-// if (thisTimeoutMs <= 0) {
-// String msg = "Timed out trying to acquire channel";
-// logInfo(logger, "HttpClient " + name + " " + msg);
-// throw new TimeoutException(msg);
-// }
-// if (thisTimeoutMs > acquireRetryIntervalMs) {
-// thisTimeoutMs = acquireRetryIntervalMs;
-// }
-// Future fut = pool.acquire();
-// Channel retChan = null;
-// try {
-// retChan = fut.get(thisTimeoutMs, TimeUnit.MILLISECONDS);
-// } catch (TimeoutException e) {
-// if (retries == 0) {
-// logFine(logger, "Timed out after " +
-// (System.currentTimeMillis() - startMs) +
-// "ms trying to acquire channel: retrying");
-// }
-// /* fall through */
-// }
-//
-// /*
-// * Ensure that the channel is in good shape. retChan is null
-// * on a timeout exception from above; that path will retry.
-// */
-// if (retChan != null) {
-// if (fut.isSuccess() && retChan.isActive()) {
-// /*
-// * Clear out any previous state. The channel should not
-// * have any state associated with it, but this code is here
-// * just in case it does.
-// */
-// if (retChan.attr(STATE_KEY).get() != null) {
-// if (isFineEnabled(logger)) {
-// logFine(logger,
-// "HttpClient acquired a channel with " +
-// "a still-active state: clearing.");
-// }
-// retChan.attr(STATE_KEY).set(null);
-// }
-// return retChan;
-// }
-// logFine(logger,
-// "HttpClient " + name + ", acquired an inactive " +
-// "channel, releasing it and retrying, reason: " +
-// fut.cause());
-// releaseChannel(retChan);
-// }
-// /* reset "now" and increment retries */
-// now = System.currentTimeMillis();
-// retries++;
-// }
-// }
-
-
-
public void releaseChannel(Channel channel) {
/* Clear any response handler state from channel before releasing it */
channel.attr(STATE_KEY).set(null);
@@ -503,22 +433,30 @@ public CompletableFuture runRequest(HttpRequest request,
if (isTimeout(deadlineNs)) {
String msg = "Timed out trying to acquire channel";
logInfo(logger, "HttpClient " + name + " " + msg);
- responseFuture.completeExceptionally(new TimeoutException(msg));
+ /* release channel and request Bytebuf */
releaseChannel(channel);
+ ReferenceCountUtil.release(request);
+ responseFuture.completeExceptionally(
+ new TimeoutException(msg));
return;
}
long remainingTimeoutNs = deadlineNs - System.nanoTime();
- long remainingTimeoutMs = Math.max(1, TimeUnit.NANOSECONDS.toMillis(remainingTimeoutNs));
+ long remainingTimeoutMs = Math.max(1,
+ TimeUnit.NANOSECONDS.toMillis(remainingTimeoutNs));
- // Execute the request on the acquired channel
+ /* Execute the request on the acquired channel */
CompletableFuture requestExecutionFuture =
runRequest(request, channel, remainingTimeoutMs);
- // When the request execution future completes (either successfully or exceptionally),
- // complete the public responseFuture and ensure the channel is released back to the pool.
+ /* When the request execution future completes (either
+ * successfully or exceptionally),
+ * complete the public responseFuture and ensure the channel
+ * is released back to the pool.
+ */
requestExecutionFuture.whenComplete((response, throwable) -> {
- pool.release(channel); // Always release the channel
+ /* Always release the channel */
+ releaseChannel(channel);
if (throwable != null) {
responseFuture.completeExceptionally(throwable);
} else {
@@ -537,19 +475,20 @@ public CompletableFuture runRequest(HttpRequest request,
public CompletableFuture runRequest(HttpRequest request,
Channel channel,
long timeoutMs) {
- CompletableFuture responseFuture = new CompletableFuture<>();
- // Attach the CompletableFuture to the channel's attributes
+ CompletableFuture
+ responseFuture = new CompletableFuture<>();
+ /* Attach the CompletableFuture to the channel's attributes */
channel.attr(STATE_KEY).set(responseFuture);
- // Add timeout handler to the pipeline
+ /* Add timeout handler to the pipeline */
channel.pipeline().addFirst(
new ReadTimeoutHandler(timeoutMs, TimeUnit.MILLISECONDS));
- // Write the request to the channel and flush it
+ /* Write the request to the channel and flush it */
channel.writeAndFlush(request)
.addListener((ChannelFutureListener) writeFuture -> {
if (!writeFuture.isSuccess()) {
- // If write fails, complete the future exceptionally
+ /* If write fails, complete the future exceptionally */
channel.attr(STATE_KEY).set(null);
responseFuture.completeExceptionally(writeFuture.cause());
}
@@ -566,7 +505,6 @@ private static boolean isTimeout(long deadlineNs) {
*/
CompletableFuture doKeepAlive(Channel ch) {
CompletableFuture future = new CompletableFuture<>();
- // TODO Handle timeout
final int keepAliveTimeout = 3000; /* ms */
final HttpRequest request =
new DefaultFullHttpRequest(HTTP_1_1, HEAD, "/");
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClientHandler.java b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClientHandler.java
index ba8626da..a41c8087 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClientHandler.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClientHandler.java
@@ -49,6 +49,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) {
* o redirects
*/
+ /* Remove timeout handler upon response arrival */
if (ctx.pipeline().get(ReadTimeoutHandler.class) != null) {
ctx.pipeline().remove(ReadTimeoutHandler.class);
}
@@ -69,17 +70,17 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) {
requestId = "(none)";
}
logFine(logger, "Discarding message with no response " +
- "handler. requestId=" + requestId);
+ "handler. requestId=" + requestId);
}
fhr.release();
- } else {
- responseFuture.complete(fhr);
+ return;
}
- } else {
- logWarning(logger,
- "HttpClientHandler, response not FullHttpResponse: " +
- msg.getClass());
+ responseFuture.complete(fhr);
+ return;
}
+ logWarning(logger,
+ "HttpClientHandler, response not FullHttpResponse: " +
+ msg.getClass());
}
@Override
@@ -89,7 +90,12 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
if (responseFuture != null) {
/* handleException logs */
logFine(logger, "HttpClientHandler read failed, cause: " + cause);
- responseFuture.completeExceptionally(cause);
+ Throwable err = cause;
+ if (err instanceof ReadTimeoutException) {
+ err = new TimeoutException("Request timed out while waiting "
+ + "for the response from the server");
+ }
+ responseFuture.completeExceptionally(err);
}
ctx.close();
}
diff --git a/driver/src/main/java/oracle/nosql/driver/iam/FederationRequestHelper.java b/driver/src/main/java/oracle/nosql/driver/iam/FederationRequestHelper.java
index cff84e2a..51ab9219 100644
--- a/driver/src/main/java/oracle/nosql/driver/iam/FederationRequestHelper.java
+++ b/driver/src/main/java/oracle/nosql/driver/iam/FederationRequestHelper.java
@@ -18,7 +18,6 @@
import java.nio.charset.StandardCharsets;
import java.util.Date;
import java.util.Set;
-import java.util.concurrent.ExecutionException;
import java.util.logging.Logger;
import oracle.nosql.driver.httpclient.HttpClient;
@@ -54,17 +53,10 @@ static String getSecurityToken(HttpClient client,
byte[] payloadByte = new byte[buf.remaining()];
buf.get(payloadByte);
- HttpResponse response = null;
- try {
- response = HttpRequestUtil.doPostRequest(
- client, endpoint.toString(),
- headers(tenantId, endpoint, payloadByte, pair, logger),
- payloadByte, timeoutMs, logger).get();
- } catch (InterruptedException e) {
- throw new RuntimeException(e);
- } catch (ExecutionException e) {
- throw new RuntimeException(e);
- }
+ HttpResponse response = HttpRequestUtil.doPostRequest(
+ client, endpoint.toString(),
+ headers(tenantId, endpoint, payloadByte, pair, logger),
+ payloadByte, timeoutMs, logger);
int responseCode = response.getStatusCode();
if (responseCode > 299) {
diff --git a/driver/src/main/java/oracle/nosql/driver/iam/InstanceMetadataHelper.java b/driver/src/main/java/oracle/nosql/driver/iam/InstanceMetadataHelper.java
index 8bf90d4d..f3e79f58 100644
--- a/driver/src/main/java/oracle/nosql/driver/iam/InstanceMetadataHelper.java
+++ b/driver/src/main/java/oracle/nosql/driver/iam/InstanceMetadataHelper.java
@@ -13,7 +13,6 @@
import static oracle.nosql.driver.util.HttpConstants.CONTENT_TYPE;
import java.io.IOException;
-import java.util.concurrent.ExecutionException;
import java.util.logging.Logger;
import oracle.nosql.driver.httpclient.HttpClient;
@@ -68,7 +67,7 @@ static InstanceMetadata fetchMetadata(int timeout, Logger logger) {
logger);
HttpRequestUtil.HttpResponse response = HttpRequestUtil.doGetRequest
- (client, instanceMDURL, headers(), timeout, logger).get();
+ (client, instanceMDURL, headers(), timeout, logger);
int status = response.getStatusCode();
if (status == 404) {
@@ -77,14 +76,14 @@ static InstanceMetadata fetchMetadata(int timeout, Logger logger) {
baseMetadataURL = FALLBACK_METADATA_SERVICE_URL;
instanceMDURL = getInstanceMetadaURL(baseMetadataURL);
response = HttpRequestUtil.doGetRequest
- (client, instanceMDURL, headers(), timeout, logger).get();
+ (client, instanceMDURL, headers(), timeout, logger);
if (response.getStatusCode() != 200) {
throw new IllegalStateException(
String.format("Unable to get federation URL from" +
"instance metadata " + METADATA_SERVICE_BASE_URL +
" or fallback to " + FALLBACK_METADATA_SERVICE_URL +
", status code: %d, output: %s",
- response.getStatusCode(), response.getOutput()));
+ response.getOutput()));
}
} else if (status != 200) {
throw new IllegalStateException(
@@ -99,10 +98,6 @@ static InstanceMetadata fetchMetadata(int timeout, Logger logger) {
String insRegion = findRegion(response.getOutput());
logTrace(logger, "Instance region " + insRegion);
return new InstanceMetadata(insRegion, baseMetadataURL);
- } catch (ExecutionException e) {
- throw new RuntimeException(e);
- } catch (InterruptedException e) {
- throw new RuntimeException(e);
} finally {
if (client != null) {
client.shutdown();
diff --git a/driver/src/main/java/oracle/nosql/driver/iam/OkeWorkloadIdentityProvider.java b/driver/src/main/java/oracle/nosql/driver/iam/OkeWorkloadIdentityProvider.java
index 6ad8ddb3..39b7a55a 100644
--- a/driver/src/main/java/oracle/nosql/driver/iam/OkeWorkloadIdentityProvider.java
+++ b/driver/src/main/java/oracle/nosql/driver/iam/OkeWorkloadIdentityProvider.java
@@ -26,7 +26,6 @@
import java.security.interfaces.RSAPublicKey;
import java.util.Base64;
import java.util.Map;
-import java.util.concurrent.ExecutionException;
import java.util.logging.Logger;
import javax.net.ssl.SSLException;
@@ -234,16 +233,9 @@ private String getSecurityToken() {
private String getSecurityTokenFromProxymux(String requestId,
String saToken,
byte[] payloadByte) {
- HttpRequestUtil.HttpResponse response = null;
- try {
- response = HttpRequestUtil.doPostRequest(
- okeTokenClient, tokenURL.toString(), headers(saToken, requestId),
- payloadByte, timeoutMs, logger).get();
- } catch (InterruptedException e) {
- throw new RuntimeException(e);
- } catch (ExecutionException e) {
- throw new RuntimeException(e);
- }
+ HttpRequestUtil.HttpResponse response = HttpRequestUtil.doPostRequest(
+ okeTokenClient, tokenURL.toString(), headers(saToken, requestId),
+ payloadByte, timeoutMs, logger);
final int responseCode = response.getStatusCode();
String responseOutput = response.getOutput();
diff --git a/driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java b/driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java
index 10c9afe9..99c0733f 100644
--- a/driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java
+++ b/driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java
@@ -27,7 +27,6 @@
import java.util.Timer;
import java.util.TimerTask;
import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.locks.ReentrantLock;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -863,28 +862,23 @@ public SignatureProvider(AuthenticationProfileProvider profileProvider,
@Override
public String getAuthorizationString(Request request) {
- try {
- return getAuthorizationStringAsync(request).get();
- } catch (InterruptedException e) {
- throw new RuntimeException(e);
- } catch (ExecutionException e) {
- throw new RuntimeException(e);
- }
+ return ConcurrentUtil.awaitFuture(getAuthorizationStringAsync(request));
}
@Override
- public CompletableFuture getAuthorizationStringAsync(Request request) {
+ public CompletableFuture getAuthorizationStringAsync(
+ Request request) {
if (serviceHost == null) {
CompletableFuture.failedFuture(new IllegalArgumentException(
"Unable to find service host, use setServiceHost " +
"to load from NoSQLHandleConfig"));
}
- return getSignatureDetails(request).thenApply(signatureDetails -> {
- if (signatureDetails == null) {
- return null;
+ return getSignatureDetails(request).thenApply(sigDetails -> {
+ if (sigDetails != null) {
+ return sigDetails.getSignatureHeader();
}
- return signatureDetails.getSignatureHeader();
+ return null;
});
}
@@ -893,13 +887,9 @@ public void setRequiredHeaders(String authString,
Request request,
HttpHeaders headers,
byte[] content) {
- try {
- setRequiredHeadersAsync(authString, request, headers, content).get();
- } catch (InterruptedException e) {
- throw new RuntimeException(e);
- } catch (ExecutionException e) {
- throw new RuntimeException(e);
- }
+
+ ConcurrentUtil.awaitFuture(
+ setRequiredHeadersAsync(authString, request, headers, content));
}
@Override
@@ -910,7 +900,8 @@ public CompletableFuture setRequiredHeadersAsync(String authString,
CompletableFuture sigDetailsFuture;
if (content != null) {
- sigDetailsFuture = getSignatureWithContent(request, headers, content);
+ sigDetailsFuture = getSignatureWithContent(
+ request, headers, content);
} else {
sigDetailsFuture = getSignatureDetails(request);
}
@@ -919,6 +910,7 @@ public CompletableFuture setRequiredHeadersAsync(String authString,
if (sigDetails != null) {
headers.add(AUTHORIZATION, sigDetails.getSignatureHeader());
headers.add(DATE, sigDetails.getDate());
+
final String token = getDelegationToken(request);
if (token != null) {
headers.add(OBO_TOKEN_HEADER, token);
@@ -926,20 +918,21 @@ public CompletableFuture setRequiredHeadersAsync(String authString,
String compartment = request.getCompartment();
if (compartment == null) {
/*
- * If request doesn't has compartment id, set the tenant id as the
- * default compartment, which is the root compartment in IAM if
- * using user principal. If using an instance principal this
- * value is null.
+ * If request doesn't has compartment id, set the tenant id
+ * as the default compartment, which is the root compartment
+ * in IAM if using user principal. If using an instance
+ * principal this value is null.
*/
compartment = getTenantOCID();
}
+
if (compartment != null) {
headers.add(REQUEST_COMPARTMENT_ID, compartment);
} else {
throw new IllegalArgumentException(
- "Compartment is null. When authenticating using an " +
- "Instance Principal the compartment for the operation " +
- "must be specified.");
+ "Compartment is null. When authenticating using an " +
+ "Instance Principal the compartment for the operation " +
+ "must be specified.");
}
}
});
@@ -947,10 +940,9 @@ public CompletableFuture setRequiredHeadersAsync(String authString,
@Override
public void flushCache() {
- ConcurrentUtil.synchronizedCall(lock,
- () -> {
- currentSigDetails = null;
- refreshSigDetails = null;
+ ConcurrentUtil.synchronizedCall(lock, () -> {
+ currentSigDetails = null;
+ refreshSigDetails = null;
});
}
@@ -1016,7 +1008,11 @@ public SignatureProvider prepare(NoSQLHandleConfig config) {
}
/* creates and caches a signature as warm-up */
- getSignatureDetailsForCache(false);
+ getSignatureDetailsInternal(false, /* isRefresh */
+ null, /* request */
+ null, /* headers */
+ null /* content */
+ );
return this;
}
@@ -1078,7 +1074,8 @@ private void logMessage(Level level, String msg) {
}
}
- private CompletableFuture getSignatureDetails(Request request) {
+ private CompletableFuture
+ getSignatureDetails(Request request) {
SignatureDetails sigDetails =
(request.getIsRefresh() ? refreshSigDetails : currentSigDetails);
if (sigDetails != null) {
@@ -1096,26 +1093,28 @@ private CompletableFuture getSignatureDetails(Request request)
return getSignatureDetailsForCache(false);
}
- private CompletableFuture getSignatureWithContent(Request request,
- HttpHeaders headers,
- byte[] content) {
- return getSignatureDetailsAsync(false, request, headers, content);
- }
-
- private CompletableFuture getSignatureDetailsForCache(boolean isRefresh) {
- return getSignatureDetailsAsync(isRefresh,
- null /* request */,
- null /* headers */,
- null /* content */);
+ private CompletableFuture
+ getSignatureWithContent(Request request,
+ HttpHeaders headers,
+ byte[] content) {
+ /* TODO: supplyAsync runs in JVM common fork-join pool.
+ * Do we need a separate executor?
+ */
+ return CompletableFuture.supplyAsync(() ->
+ getSignatureDetailsInternal(false, request, headers, content));
}
private CompletableFuture
- getSignatureDetailsAsync(boolean isRefresh,
- Request request,
- HttpHeaders headers,
- byte[] content) {
+ getSignatureDetailsForCache(boolean isRefresh) {
+ /* TODO: supplyAsync runs in JVM common fork-join pool.
+ * Do we need a separate executor?
+ */
return CompletableFuture.supplyAsync(() ->
- getSignatureDetailsInternal(isRefresh, request, headers, content));
+ getSignatureDetailsInternal(isRefresh,
+ null /* request */,
+ null /* headers */,
+ null /* content */)
+ );
}
/* visible for testing */
@@ -1124,82 +1123,79 @@ private CompletableFuture getSignatureDetailsForCache(boolean
Request request,
HttpHeaders headers,
byte[] content) {
- lock.lock();
+ return ConcurrentUtil.synchronizedCall(lock, () -> {
+ /*
+ * add one minute to the current time, so that any caching is
+ * effective over a more valid time period.
+ */
+ long nowPlus = System.currentTimeMillis() + 60_000L;
+ String date = createFormatter().format(new Date(nowPlus));
+ String keyId = provider.getKeyId();
+
+ /*
+ * Security token based providers may refresh the security token
+ * and associated private key in above getKeyId() method, reload
+ * private key to PrivateKeyProvider to avoid a mismatch, which
+ * will create an invalid signature, cause authentication error.
+ */
+ if (provider instanceof SecurityTokenBasedProvider) {
+ privateKeyProvider.reload(provider.getPrivateKey(),
+ provider.getPassphraseCharacters());
+ }
+ String signature;
try {
- /*
- * add one minute to the current time, so that any caching is
- * effective over a more valid time period.
- */
- long nowPlus = System.currentTimeMillis() + 60_000L;
- String date = createFormatter().format(new Date(nowPlus));
- String keyId = provider.getKeyId();
+ signature = sign(signingContent(date, request, headers, content),
+ privateKeyProvider.getKey());
+ } catch (Exception e) {
+ logMessage(Level.SEVERE, "Error signing request " + e.getMessage());
+ return null;
+ }
- /*
- * Security token based providers may refresh the security token
- * and associated private key in above getKeyId() method, reload
- * private key to PrivateKeyProvider to avoid a mismatch, which
- * will create an invalid signature, cause authentication error.
- */
- if (provider instanceof SecurityTokenBasedProvider) {
- privateKeyProvider.reload(provider.getPrivateKey(),
- provider.getPassphraseCharacters());
- }
- String signature;
- try {
- signature = sign(signingContent(date, request, headers, content),
- privateKeyProvider.getKey());
- } catch (Exception e) {
- logMessage(Level.SEVERE, "Error signing request " + e.getMessage());
- return null;
- }
+ String token = getDelegationToken(request);
+ String signingHeader;
+ if (content != null) {
+ signingHeader = (token == null)
+ ? SIGNING_HEADERS_WITH_CONTENT :
+ SIGNING_HEADERS_WITH_CONTENT_OBO;
+ } else {
+ signingHeader = (token == null)
+ ? SIGNING_HEADERS : SIGNING_HEADERS_WITH_OBO;
+ }
- String token = getDelegationToken(request);
- String signingHeader;
- if (content != null) {
- signingHeader = (token == null)
- ? SIGNING_HEADERS_WITH_CONTENT :
- SIGNING_HEADERS_WITH_CONTENT_OBO;
- } else {
- signingHeader = (token == null)
- ? SIGNING_HEADERS : SIGNING_HEADERS_WITH_OBO;
- }
+ String sigHeader = String.format(SIGNATURE_HEADER_FORMAT,
+ signingHeader,
+ keyId,
+ RSA,
+ signature,
+ SINGATURE_VERSION);
+ SignatureDetails sigDetails = new SignatureDetails(sigHeader, date);
- String sigHeader = String.format(SIGNATURE_HEADER_FORMAT,
- signingHeader,
- keyId,
- RSA,
- signature,
- SINGATURE_VERSION);
- SignatureDetails sigDetails = new SignatureDetails(sigHeader, date);
+ /*
+ * Don't cache the signature generated with content, which
+ * needs to be associated with its request
+ */
+ if (content != null) {
+ return sigDetails;
+ }
+ if (!isRefresh) {
/*
- * Don't cache the signature generated with content, which
- * needs to be associated with its request
+ * if this is not a refresh, use the normal key and schedule a
+ * refresh
*/
- if (content != null) {
- return sigDetails;
- }
-
- if (!isRefresh) {
- /*
- * if this is not a refresh, use the normal key and schedule a
- * refresh
- */
- currentSigDetails = sigDetails;
- scheduleRefresh();
- } else {
- /*
- * If this is a refresh put the object in a temporary key.
- * The caller (the refresh task) will:
- * 1. perform callbacks if needed and when done,
- * 2. move the object to the normal key and schedule a refresh
- */
- refreshSigDetails = sigDetails;
- }
- return sigDetails;
- } finally {
- lock.unlock();
+ currentSigDetails = sigDetails;
+ scheduleRefresh();
+ } else {
+ /*
+ * If this is a refresh put the object in a temporary key.
+ * The caller (the refresh task) will:
+ * 1. perform callbacks if needed and when done,
+ * 2. move the object to the normal key and schedule a refresh
+ */
+ refreshSigDetails = sigDetails;
}
+ return sigDetails;
+ });
}
/*
@@ -1216,13 +1212,11 @@ private String getDelegationToken(Request req) {
}
private void setRefreshKey() {
- ConcurrentUtil.synchronizedCall(lock,
- () -> {
- if (refreshSigDetails != null) {
- currentSigDetails = refreshSigDetails;
- refreshSigDetails = null;
- }
- });
+ ConcurrentUtil.synchronizedCall(lock, () -> {
+ if (refreshSigDetails != null) {
+ currentSigDetails = refreshSigDetails;
+ refreshSigDetails = null;
+ }});
}
private String signingContent(String date,
@@ -1318,7 +1312,10 @@ public void run() {
Exception lastException;
do {
try {
- getSignatureDetailsForCache(true);
+ getSignatureDetailsInternal(true, /* isRefresh */
+ null /* request */,
+ null /* headers */,
+ null /* content */);
handleRefreshCallback(refreshAheadMs);
return;
} catch (SecurityInfoNotReadyException se) {
diff --git a/driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java b/driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java
index 67b3e644..12bacea5 100644
--- a/driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java
+++ b/driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java
@@ -16,7 +16,6 @@
import java.util.Timer;
import java.util.TimerTask;
import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReentrantLock;
import java.util.logging.Logger;
@@ -175,7 +174,6 @@ public class StoreAccessTokenProvider implements AuthorizationProvider {
public static boolean disableSSLHook;
private final ReentrantLock lock = new ReentrantLock();
- private volatile CompletableFuture tokenRefreshInProgress = null;
/**
* This method is used for access to a store without security enabled.
@@ -229,28 +227,15 @@ public StoreAccessTokenProvider(String userName,
*
* Bootstrap login using the provided credentials
*/
- public CompletableFuture bootstrapLogin(Request request) {
+ public void bootstrapLogin(Request request) {
- lock.lock();
- try {
- if (!isSecure || isClosed ) {
- return CompletableFuture.completedFuture(null);
- }
-
- /* re-check the authString in case of a race */
- if (authString.get() != null) {
- return CompletableFuture.completedFuture(authString.get());
- }
-
- if (tokenRefreshInProgress != null && !tokenRefreshInProgress.isDone()) {
- return tokenRefreshInProgress;
- }
-
- if (timer != null) {
- timer.cancel();
- timer = null;
- }
+ ConcurrentUtil.synchronizedCall(lock, () -> {
+ /* re-check the authString in case of a race */
+ if (!isSecure || isClosed || authString.get() != null) {
+ return;
+ }
+ try {
/*
* Convert the user:password pair in base 64 format with
* Basic prefix
@@ -267,38 +252,37 @@ public CompletableFuture bootstrapLogin(Request request) {
/*
* Send request to server for login token
*/
- CompletableFuture refreshFuture =
- sendRequest(BASIC_PREFIX + encoded, LOGIN_SERVICE, timeoutMs)
- .thenApply(response -> {
- /* login fail */
- if (response.getStatusCode() != HttpResponseStatus.OK.code()) {
- throw new InvalidAuthorizationException(
- "Fail to login to service: "
- + response.getOutput());
- }
- if (isClosed) {
- return null;
- }
- /*
- * Generate the authentication string using login token
- */
- authString.set(BEARER_PREFIX + parseJsonResult(response.getOutput()));
- /*
- * Schedule login token renew thread
- */
- scheduleRefresh();
- return authString.get();
- }).exceptionally(err -> {
- if (!(err instanceof InvalidAuthorizationException)) {
- throw new NoSQLException("Bootstrap login fail", err);
- }
- return null;
- });
- tokenRefreshInProgress = refreshFuture;
- return refreshFuture;
- } finally {
- lock.unlock();
- }
+ HttpResponse response = sendRequest(BASIC_PREFIX + encoded,
+ LOGIN_SERVICE, timeoutMs);
+
+ /*
+ * login fail
+ */
+ if (response.getStatusCode() != HttpResponseStatus.OK.code()) {
+ throw new InvalidAuthorizationException(
+ "Fail to login to service: " + response.getOutput());
+ }
+
+ if (isClosed) {
+ return;
+ }
+
+ /*
+ * Generate the authentication string using login token
+ */
+ authString.set(BEARER_PREFIX +
+ parseJsonResult(response.getOutput()));
+
+ /*
+ * Schedule login token renew thread
+ */
+ scheduleRefresh();
+
+ } catch (InvalidAuthorizationException iae) {
+ throw iae;
+ } catch (Exception e) {
+ throw new NoSQLException("Bootstrap login fail", e);
+ }});
}
/**
@@ -306,29 +290,30 @@ public CompletableFuture bootstrapLogin(Request request) {
*/
@Override
public String getAuthorizationString(Request request) {
- try {
- return getAuthorizationStringAsync(request).get();
- } catch (InterruptedException e) {
- throw new RuntimeException(e);
- } catch (ExecutionException e) {
- throw new RuntimeException(e);
- }
+ return ConcurrentUtil.awaitFuture(getAuthorizationStringAsync(request));
}
@Override
- public CompletableFuture getAuthorizationStringAsync(Request request) {
+ public CompletableFuture
+ getAuthorizationStringAsync(Request request) {
+
if (!isSecure || isClosed) {
return CompletableFuture.completedFuture(null);
}
- /*
- * If there is no cached auth string, re-authentication to retrieve
- * the login token and generate the auth string.
- */
- if (authString.get() == null) {
- return bootstrapLogin(request);
+ String token = authString.get();
+ if (token != null) {
+ return CompletableFuture.completedFuture(token);
}
- return CompletableFuture.completedFuture(authString.get());
+
+ /* Run bootstrap login asynchronously, reusing existing sync logic. */
+ /* TODO: supplyAsync runs in JVM common fork-join pool.
+ * Do we need a separate executor?
+ */
+ return CompletableFuture.supplyAsync(() -> {
+ bootstrapLogin(request);
+ return authString.get();
+ });
}
/**
@@ -343,27 +328,42 @@ public void validateAuthString(String input) {
}
}
+ @Override
+ public void flushCache() {
+ ConcurrentUtil.synchronizedCall(lock,
+ () -> {
+ if (!isSecure || isClosed) {
+ return;
+ }
+ authString.set(null);
+ expirationTime = 0;
+ if (timer != null) {
+ timer.cancel();
+ timer = null;
+ }
+ });
+ }
+
/**
* Closes the provider, releasing resources such as a stored login
* token.
*/
@Override
public void close() {
-
- lock.lock();
+ ConcurrentUtil.synchronizedCall(lock, () -> {
+ /*
+ * Don't do anything for non-secure case
+ */
+ if (!isSecure || isClosed) {
+ return;
+ }
/*
* Send request for logout
*/
try {
- /*
- * Don't do anything for non-secure case
- */
- if (!isSecure || isClosed) {
- return;
- }
final HttpResponse response =
- sendRequest(authString.get(), LOGOUT_SERVICE, 0).get();
+ sendRequest(authString.get(), LOGOUT_SERVICE, 0);
if (response.getStatusCode() != HttpResponseStatus.OK.code()) {
if (logger != null) {
logger.info("Failed to logout user " + userName +
@@ -375,31 +375,19 @@ public void close() {
logger.info("Failed to logout user " + userName +
": " + e);
}
- /*
- * Clean up
- */
- isClosed = true;
- authString = null;
- expirationTime = 0;
- Arrays.fill(password, ' ');
- if (timer != null) {
- timer.cancel();
- timer = null;
- }
- } finally {
- lock.unlock();
}
- }
- @Override
- public void flushCache() {
- ConcurrentUtil.synchronizedCall(lock,
- () -> {
- authString.set(null);
- if (timer != null) {
- timer.cancel();
- timer = null;
- }
+ /*
+ * Clean up
+ */
+ isClosed = true;
+ authString = null;
+ expirationTime = 0;
+ Arrays.fill(password, ' ');
+ if (timer != null) {
+ timer.cancel();
+ timer = null;
+ }
});
}
@@ -542,9 +530,9 @@ private String parseJsonResult(String jsonResult) {
* Send HTTPS request to login/renew/logout service location with proper
* authentication information.
*/
- private CompletableFuture sendRequest(String authHeader,
+ private HttpResponse sendRequest(String authHeader,
String serviceName,
- int timeoutMs) {
+ int timeoutMs) throws Exception {
HttpClient client = null;
try {
final HttpHeaders headers = new DefaultHttpHeaders();
@@ -611,7 +599,7 @@ public void run() {
final String oldAuth = authString.get();
HttpResponse response = sendRequest(oldAuth,
RENEW_SERVICE,
- 0).get();
+ 0);
final String token = parseJsonResult(response.getOutput());
if (response.getStatusCode() != HttpResponseStatus.OK.code()) {
throw new InvalidAuthorizationException(token);
diff --git a/driver/src/main/java/oracle/nosql/driver/query/ReceiveIter.java b/driver/src/main/java/oracle/nosql/driver/query/ReceiveIter.java
index bb0fece0..4fe29f62 100644
--- a/driver/src/main/java/oracle/nosql/driver/query/ReceiveIter.java
+++ b/driver/src/main/java/oracle/nosql/driver/query/ReceiveIter.java
@@ -12,12 +12,14 @@
import java.util.HashSet;
import java.util.List;
import java.util.TreeSet;
-import java.util.concurrent.ExecutionException;
+import java.util.concurrent.CompletableFuture;
import oracle.nosql.driver.NoSQLException;
import oracle.nosql.driver.RetryableException;
import oracle.nosql.driver.ops.QueryRequest;
import oracle.nosql.driver.ops.QueryResult;
+import oracle.nosql.driver.ops.Result;
+import oracle.nosql.driver.util.ConcurrentUtil;
import oracle.nosql.driver.values.BinaryValue;
import oracle.nosql.driver.values.FieldValue;
import oracle.nosql.driver.values.MapValue;
@@ -593,13 +595,10 @@ private QueryResult execute(RuntimeControlBlock rcb,
NoSQLException e = null;
QueryResult result = null;
try {
- result = (QueryResult)rcb.getClient().execute(reqCopy).get();
+ CompletableFuture fut = rcb.getClient().execute(reqCopy);
+ result = (QueryResult) ConcurrentUtil.awaitFuture(fut);
} catch (NoSQLException qe) {
e = qe;
- } catch (ExecutionException ex) {
- throw new RuntimeException(ex);
- } catch (InterruptedException ex) {
- throw new RuntimeException(ex);
}
/*
* Copy values back to original request, even when the execute()
diff --git a/driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java b/driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java
index 38a6a9a2..6d3b5b00 100644
--- a/driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java
+++ b/driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java
@@ -1,5 +1,10 @@
package oracle.nosql.driver.util;
+import oracle.nosql.driver.NoSQLException;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Supplier;
@@ -29,4 +34,38 @@ public static void synchronizedCall(ReentrantLock lock,
lock.unlock();
}
}
+
+ /**
+ * A helper function to wait for the future to complete.
+ */
+ public static T awaitFuture(CompletableFuture future) {
+ try {
+ return future.get();
+ } catch (ExecutionException e) {
+ final Throwable cause = e.getCause();
+ if (cause instanceof RuntimeException) {
+ throw ((RuntimeException) cause);
+ }
+ throw new NoSQLException("ExecutionException: "
+ + e.getMessage(), e.getCause());
+ } catch (InterruptedException ie) {
+ throw new NoSQLException("Request interrupted: "
+ + ie.getMessage(), ie);
+ }
+ }
+
+ /**
+ * Returns the cause if the exception is a CompletionException, otherwise
+ * returns the exception.
+ */
+ public static Throwable unwrapCompletionException(Throwable t) {
+ Throwable actual = t;
+ while (true) {
+ if (!(actual instanceof CompletionException)
+ || (actual.getCause() == null)) {
+ return actual;
+ }
+ actual = actual.getCause();
+ }
+ }
}
diff --git a/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java b/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
index 897bf98f..d90f747b 100644
--- a/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
+++ b/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
@@ -26,23 +26,20 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeoutException;
-import java.util.function.Function;
import java.util.logging.Logger;
import javax.net.ssl.SSLException;
import io.netty.buffer.Unpooled;
+import io.netty.handler.codec.http.HttpResponseStatus;
import oracle.nosql.driver.RequestTimeoutException;
import oracle.nosql.driver.httpclient.HttpClient;
-import oracle.nosql.driver.httpclient.ResponseHandler;
import io.netty.buffer.ByteBuf;
-import io.netty.channel.Channel;
import io.netty.handler.codec.http.DefaultFullHttpRequest;
import io.netty.handler.codec.http.FullHttpRequest;
import io.netty.handler.codec.http.HttpHeaderNames;
import io.netty.handler.codec.http.HttpHeaders;
import io.netty.handler.codec.http.HttpMethod;
-import io.netty.handler.codec.http.HttpResponseStatus;
/**
* Utility to issue HTTP request using {@link HttpClient}.
@@ -77,7 +74,7 @@ public class HttpRequestUtil {
*
* @return HTTP response, a object encapsulate status code and response
*/
- public static CompletableFuture doGetRequest(HttpClient httpClient,
+ public static HttpResponse doGetRequest(HttpClient httpClient,
String uri,
HttpHeaders headers,
int timeoutMs,
@@ -115,7 +112,7 @@ public static CompletableFuture doGetRequest(HttpClient httpClient
*
* @return HTTP response, a object encapsulate status code and response
*/
- public static CompletableFuture doPostRequest(HttpClient httpClient,
+ public static HttpResponse doPostRequest(HttpClient httpClient,
String uri,
HttpHeaders headers,
byte[] payload,
@@ -154,7 +151,7 @@ public static CompletableFuture doPostRequest(HttpClient httpClien
*
* @return HTTP response, a object encapsulate status code and response
*/
- public static CompletableFuture doPutRequest(HttpClient httpClient,
+ public static HttpResponse doPutRequest(HttpClient httpClient,
String uri,
HttpHeaders headers,
byte[] payload,
@@ -191,7 +188,7 @@ public static CompletableFuture doPutRequest(HttpClient httpClient
*
* @return HTTP response, a object encapsulate status code and response
*/
- public static CompletableFuture doDeleteRequest(HttpClient httpClient,
+ public static HttpResponse doDeleteRequest(HttpClient httpClient,
String uri,
HttpHeaders headers,
int timeoutMs,
@@ -201,7 +198,7 @@ public static CompletableFuture doDeleteRequest(HttpClient httpCli
timeoutMs, logger);
}
- private static CompletableFuture doRequest(HttpClient httpClient,
+ private static HttpResponse doRequest(HttpClient httpClient,
String uri,
HttpHeaders headers,
HttpMethod method,
@@ -213,46 +210,109 @@ private static CompletableFuture doRequest(HttpClient httpClient,
int numRetries = 0;
Throwable exception = null;
- FullHttpRequest request;
- if (payload == null) {
- request = buildRequest(uri, method, headers);
- } else {
- request = buildRequest(uri, headers, method, payload);
- }
- addRequiredHeaders(request);
- logFine(logger, request.headers().toString());
-
-
- return httpClient.runRequest(request, timeoutMs)
- .thenApply(fhr -> {
- if (fhr.status() == null) {
- throw new IllegalStateException("Invalid null response");
+ do {
+ if (numRetries > 0) {
+ logInfo(logger, "Client, doing retry: " + numRetries +
+ (exception != null ? ", exception: " + exception : ""));
}
try {
- return processResponse(fhr.status().code(), fhr.content());
- } finally {
- fhr.release();
- }
- })
- .handle((res, err) -> {
- if (err != null) {
- return handleError(res, err);
- } else if(res.getStatusCode() >= 500) {
- logFine(logger,
- "Remote server temporarily unavailable," +
+ FullHttpRequest request;
+ if (payload == null) {
+ request = buildRequest(uri, method, headers);
+ } else {
+ request = buildRequest(uri, headers, method, payload);
+ }
+ addRequiredHeaders(request);
+ logFine(logger, request.headers().toString());
+ CompletableFuture httpResponse =
+ httpClient.runRequest(request, timeoutMs)
+ .thenApply(fhr -> {
+ if (fhr.status() == null) {
+ throw new IllegalStateException(
+ "Invalid null response");
+ }
+ try {
+ final int code = fhr.status().code();
+ return processResponse(code, fhr.content());
+ } finally {
+ fhr.release();
+ }
+ });
+ HttpResponse res = httpResponse.get();
+
+ /*
+ * Retry upon status code larger than 500, in general,
+ * this indicates server internal error.
+ */
+ if (res.getStatusCode() >= 500) {
+ logFine(logger,
+ "Remote server temporarily unavailable," +
" status code " + res.getStatusCode() +
" , response " + res.getOutput());
- return handleError(res, err);
+ delay();
+ ++numRetries;
+ continue;
+ }
+ return res;
+ } catch (ExecutionException ee) {
+ Throwable cause = ee.getCause();
+ if (cause instanceof IOException) {
+ IOException ioe = (IOException) cause;
+ String name = ioe.getClass().getName();
+ logFine(logger, "Client execute IOException, name: " +
+ name + ", message: " + ioe.getMessage());
+ /*
+ * An exception in the channel, e.g. the server may have
+ * disconnected. Retry.
+ */
+ exception = ioe;
+ ++numRetries;
+ if (ioe instanceof SSLException) {
+ /* disconnect the channel to force a new one */
+ /*if (channel != null) {
+ logFine(logger,
+ "Client disconnecting channel due to: " + ioe);
+ channel.disconnect();
+ }*/
+ //TODO what to do?
+ } else {
+ delay();
+ }
+ continue;
+ } else if (cause instanceof TimeoutException) {
+ throw new RuntimeException("Timeout exception: host=" +
+ httpClient.getHost() + " port=" +
+ httpClient.getPort() + " uri=" +
+ uri, cause);
+ }
+ throw new RuntimeException("Unable to execute request: ", ee);
+
+ } catch (RuntimeException e) {
+ logFine(logger, "Client execute runtime exception: " +
+ e.getMessage());
+ throw e;
+ } catch (InterruptedException ie) {
+ throw new RuntimeException("Client interrupted exception: ", ie);
+ } catch (Throwable t) {
+ /*
+ * this is likely an exception from Netty, perhaps a bad
+ * connection. Retry.
+ */
+ String name = t.getClass().getName();
+ logFine(logger, "Client execute Throwable, name: " +
+ name + "message: " + t.getMessage());
+
+ exception = t;
+ delay();
+ ++numRetries;
+ continue;
}
- return CompletableFuture.completedFuture(res);
- })
- .thenCompose(Function.identity());
- }
+ } while ((System.currentTimeMillis()- startTime) < timeoutMs);
- static CompletableFuture handleError(HttpResponse response,
- Throwable err) {
- //TODO handle error
- return null;
+ throw new RequestTimeoutException(timeoutMs,
+ "Request timed out after " + numRetries +
+ (numRetries == 1 ? " retry." : " retries."),
+ exception);
}
private static FullHttpRequest buildRequest(String requestURI,
diff --git a/driver/src/main/java/oracle/nosql/driver/util/LogUtil.java b/driver/src/main/java/oracle/nosql/driver/util/LogUtil.java
index 25710c02..5aff93ae 100644
--- a/driver/src/main/java/oracle/nosql/driver/util/LogUtil.java
+++ b/driver/src/main/java/oracle/nosql/driver/util/LogUtil.java
@@ -7,6 +7,8 @@
package oracle.nosql.driver.util;
+import java.io.PrintWriter;
+import java.io.StringWriter;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -67,4 +69,19 @@ public static void logTrace(Logger logger, String msg) {
public static boolean isLoggable(Logger logger, Level level) {
return (logger != null && logger.isLoggable(level));
}
+
+ /**
+ * Returns the stack trace.
+ *
+ * @param t the exception
+ */
+ public static String getStackTrace(Throwable t) {
+ if (t == null) {
+ return null;
+ }
+ final StringWriter sw = new StringWriter();
+ final PrintWriter pw = new PrintWriter(sw);
+ t.printStackTrace(pw);
+ return sw.toString();
+ }
}
diff --git a/examples/src/main/java/Common.java b/examples/src/main/java/Common.java
index ae301572..cc4bcd89 100644
--- a/examples/src/main/java/Common.java
+++ b/examples/src/main/java/Common.java
@@ -360,7 +360,8 @@ public String getAuthorizationString(Request request) {
}
@Override
- public CompletableFuture getAuthorizationStringAsync(Request request) {
+ public CompletableFuture
+ getAuthorizationStringAsync(Request request) {
return CompletableFuture.completedFuture(id);
}
From a67c2d75f3824f41a1fa60650f98c612e79d7516 Mon Sep 17 00:00:00 2001
From: Akshay Sundarraj
Date: Tue, 2 Sep 2025 20:03:19 +0530
Subject: [PATCH 04/11] Async refactor changes-III
- synchronous keep-alive
o. Keep-alive used to happen on Netty event-loop threads. Introduced a new single thread for keep-alive and also made the keep-alive synchronous
o. Fixed an issue in Connection pool stats collection where ChannelStats is added to the Map even for removal
- Async NoSQL Handle
o. Added new NoSQLHandleAsync interface and corresponding NoSQLHandleAsyncImpl implementation
o. Added a new method NoSQLHandleAsync#queryPaginator, which returns Flow.Publisher
o. queryPaginator is an async version of queryIterable
o. Updated NoSQLHandleImpl to internally use NoSQLHandleAsyncImpl
o. Added static method to create NoSQLHandleAsync in NoSQLHandleFactory
o. Added an async method TableResult#waitForCompletionAsync in TableResult
o. Added an async method SystemResult#waitForCompletionAsync in SystemResult
Files:
A driver/src/main/java/oracle/nosql/driver/NoSQLHandleAsync.java
A driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleAsyncImpl.java
- Async NoSQL handle addition
A driver/src/main/java/oracle/nosql/driver/ops/QueryPublisher.java
- Added a Publisher for query paginator. The Publisher is backpressure aware.
It will contact the server only when there is more demand from
downstream subscriber. The Publisher also closes the QueryRequest internally
M driver/src/main/java/oracle/nosql/driver/NoSQLHandleFactory.java
- Added a static method to create NoSQLHandleAsync
M driver/src/main/java/oracle/nosql/driver/http/Client.java
- Explicitly pass task executor to use
M driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
- Changed to be wrapper around async handle
M driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
- Added new single thread pool to run keep-alive in background and reverted async keep-alive
- In updateStats() channel has been added to map even for remove case. Added a check for isAcquire
- Reverted the signature of keepALive interface to boolean from CompletableFuture
M driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
M driver/src/test/java/oracle/nosql/driver/httpclient/ConnectionPoolTest.java
- Reverted the signature of keepALive interface to boolean from CompletableFuture
M driver/src/main/java/oracle/nosql/driver/ops/SystemResult.java
M driver/src/main/java/oracle/nosql/driver/ops/TableResult.java
- Added async version of waitForCompletion method
---
.../oracle/nosql/driver/NoSQLHandleAsync.java | 966 ++++++++++++++++++
.../nosql/driver/NoSQLHandleFactory.java | 30 +
.../java/oracle/nosql/driver/http/Client.java | 35 +-
.../driver/http/NoSQLHandleAsyncImpl.java | 504 +++++++++
.../nosql/driver/http/NoSQLHandleImpl.java | 254 +----
.../driver/httpclient/ConnectionPool.java | 65 +-
.../nosql/driver/httpclient/HttpClient.java | 52 +-
.../nosql/driver/ops/QueryPublisher.java | 106 ++
.../oracle/nosql/driver/ops/SystemResult.java | 65 ++
.../oracle/nosql/driver/ops/TableResult.java | 82 ++
.../driver/httpclient/ConnectionPoolTest.java | 5 +-
11 files changed, 1839 insertions(+), 325 deletions(-)
create mode 100644 driver/src/main/java/oracle/nosql/driver/NoSQLHandleAsync.java
create mode 100644 driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleAsyncImpl.java
create mode 100644 driver/src/main/java/oracle/nosql/driver/ops/QueryPublisher.java
diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleAsync.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleAsync.java
new file mode 100644
index 00000000..8e07f9af
--- /dev/null
+++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleAsync.java
@@ -0,0 +1,966 @@
+/*-
+ * Copyright (c) 2011, 2025 Oracle and/or its affiliates. All rights reserved.
+ *
+ * Licensed under the Universal Permissive License v 1.0 as shown at
+ * https://oss.oracle.com/licenses/upl/
+ */
+
+package oracle.nosql.driver;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Flow;
+
+import oracle.nosql.driver.ops.AddReplicaRequest;
+import oracle.nosql.driver.ops.DeleteRequest;
+import oracle.nosql.driver.ops.DeleteResult;
+import oracle.nosql.driver.ops.DropReplicaRequest;
+import oracle.nosql.driver.ops.GetIndexesRequest;
+import oracle.nosql.driver.ops.GetIndexesResult;
+import oracle.nosql.driver.ops.GetRequest;
+import oracle.nosql.driver.ops.GetResult;
+import oracle.nosql.driver.ops.GetTableRequest;
+import oracle.nosql.driver.ops.ListTablesRequest;
+import oracle.nosql.driver.ops.ListTablesResult;
+import oracle.nosql.driver.ops.MultiDeleteRequest;
+import oracle.nosql.driver.ops.MultiDeleteResult;
+import oracle.nosql.driver.ops.PrepareRequest;
+import oracle.nosql.driver.ops.PrepareResult;
+import oracle.nosql.driver.ops.PutRequest;
+import oracle.nosql.driver.ops.PutResult;
+import oracle.nosql.driver.ops.QueryIterableResult;
+import oracle.nosql.driver.ops.QueryRequest;
+import oracle.nosql.driver.ops.QueryResult;
+import oracle.nosql.driver.ops.ReplicaStatsRequest;
+import oracle.nosql.driver.ops.ReplicaStatsResult;
+import oracle.nosql.driver.ops.Request;
+import oracle.nosql.driver.ops.Result;
+import oracle.nosql.driver.ops.SystemRequest;
+import oracle.nosql.driver.ops.SystemResult;
+import oracle.nosql.driver.ops.SystemStatusRequest;
+import oracle.nosql.driver.ops.TableRequest;
+import oracle.nosql.driver.ops.TableResult;
+import oracle.nosql.driver.ops.TableUsageRequest;
+import oracle.nosql.driver.ops.TableUsageResult;
+import oracle.nosql.driver.ops.WriteMultipleRequest;
+import oracle.nosql.driver.ops.WriteMultipleResult;
+import oracle.nosql.driver.values.MapValue;
+
+/**
+ * NoSQLHandleAsync is an asynchronous handle that can be used to access Oracle
+ * NoSQL tables. To create a connection represented by NoSQLHandleAsync,
+ * request an instance using {@link NoSQLHandleFactory#createNoSQLHandleAsync}
+ * and {@link NoSQLHandleConfig}, which allows an application to specify
+ * default values and other configuration information to be used by the handle.
+ *
+ * The same interface is available to both users of the Oracle NoSQL Database
+ * Cloud Service and the on-premises Oracle NoSQL Database; however, some
+ * methods and/or parameters are specific to each environment. The
+ * documentation has notes about whether a class, method, or parameter is
+ * environment-specific. Unless otherwise noted they are applicable to both
+ * environments.
+ *
+ * A handle has memory and network resources associated with it. Consequently,
+ * the {@link NoSQLHandleAsync#close} method must be invoked to free up the
+ * resources when the application is done using the handle.
+ *
+ * To minimize network activity as well as resource allocation and
+ * deallocation overheads, it's best to avoid repeated creation and closing of
+ * handles. For example, creating and closing a handle around each operation,
+ * would incur large resource allocation overheads resulting in poor
+ * application performance.
+ *
+ *
+ * A handle permits concurrent operations, so a single handle is sufficient to
+ * access tables in a multi-threaded application. The creation of multiple
+ * handles incurs additional resource overheads without providing any
+ * performance benefit.
+ *
+ *
+ * With the exception of {@link #close} the operations on this interface follow
+ * a similar pattern. They accept a {@link Request} object containing
+ * parameters, both required and optional. They return a {@link CompletableFuture}
+ * which returns a {@link Result} object containing results. Operation failures
+ * throw exceptions. Unique subclasses of {@link Request} and {@link Result}
+ * exist for most operations, containing information specific to the operation.
+ * All of these operations result in remote calls across a network.
+ *
+ *
+ * All {@link Request} instances support specification of parameters for the
+ * operation as well as the ability to override default parameters which may
+ * have been specified in {@link NoSQLHandleConfig}, such as request timeouts,
+ * {@link Consistency}, etc.
+ *
+ *
+ * {@link Request} objects
+ * are not copied and must not be modified by the application while a method
+ * on this interface is using them.
+ *
+ *
Error and Exception Handling
+ *
+ * On success all methods in this interface return {@link CompletableFuture}
+ * which completes with {@link Result} objects.
+ * On Error, return {@link CompletableFuture} completes with exceptions.
+ * Some Java exceptions, such as {@link IllegalArgumentException} and
+ * {@link NullPointerException} are thrown directly. All other exceptions are
+ * instances of {@link NoSQLException}, which serves as a base class for NoSQL
+ * Database exceptions.
+ *
+ *
+ * {@link NoSQLException} instances are split into two broad categories:
+ *
+ *
Exceptions that may be retried with the expectation that they
+ * may succeed on retry. These are instances of {@link RetryableException}
+ *
Exceptions that may not be retried and if retried, will fail again
+ *
+ *
+ * Exceptions that may be retried return true for
+ * {@link NoSQLException#okToRetry} while those that may not will return false.
+ * Examples of retryable exceptions are those which indicate resource
+ * consumption violations such as {@link ThrottlingException}.
+ * Examples of exceptions that should not be
+ * retried are {@link IllegalArgumentException},
+ * {@link TableNotFoundException}, and any other exception indicating a
+ * syntactic or semantic error.
+ *
+ *
+ * Instances of NoSQLHandleAsync are thread-safe and expected to be shared among
+ * threads.
+ *
+ */
+public interface NoSQLHandleAsync extends AutoCloseable {
+
+ /**
+ * Deletes a row from a table asynchronously. The row is identified using a
+ * primary key value supplied in {@link DeleteRequest#setKey}
+ *
+ * By default, a delete operation is unconditional and will succeed if the
+ * specified row exists. Delete operations can be made conditional based
+ * on whether the {@link Version} of an existing row matches that supplied
+ * by {@link DeleteRequest#setMatchVersion}.
+ *
+ * It is also possible to return information about the existing
+ * row. The row, including its {@link Version} and modification time
+ * can be optionally returned.
+ * The existing row information will only be returned if
+ * {@link DeleteRequest#setReturnRow} is true and one of the following
+ * occurs:
+ *
+ *
The {@link DeleteRequest#setMatchVersion} is used and the operation
+ * fails because the row exists and its version does not match.
+ *
+ *
The {@link DeleteRequest#setMatchVersion} is not used and the
+ * operation succeeds provided that the server supports providing the
+ * existing row.
+ *
+ *
+ * Use of {@link DeleteRequest#setReturnRow} may result in additional
+ * consumed read capacity.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ *
+ */
+ CompletableFuture delete(DeleteRequest request);
+
+ /**
+ * Gets the row associated with a primary key asynchronously. On success the
+ * value of the row is available using the {@link GetResult#getValue}
+ * operation. If there are no matching rows that method will return null.
+ *
+ * The default {@link Consistency} used for the operation is
+ * {@link Consistency#EVENTUAL} unless an explicit value is has been set
+ * using {@link NoSQLHandleConfig#setConsistency} or
+ * {@link GetRequest#setConsistency}. Use of {@link Consistency#ABSOLUTE}
+ * may affect latency of the operation and may result in additional cost
+ * for the operation.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The CompletableFuture returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ *
+ */
+ CompletableFuture get(GetRequest request);
+
+ /**
+ * Puts a row into a table asynchronously. This method creates a new row or
+ * overwrites an existing row entirely. The value used for the put is in
+ * the {@link PutRequest} object and must contain a complete primary key and
+ * all required fields.
+ *
+ * It is not possible to put part of a row. Any fields that are not
+ * provided will be defaulted, overwriting any existing value. Fields that
+ * are not nullable or defaulted must be provided or an exception will be
+ * thrown.
+ *
+ * By default a put operation is unconditional, but put operations can be
+ * conditional based on existence, or not, of a previous value as well as
+ * conditional on the {@link Version} of the existing value.
+ *
+ *
Use {@link PutRequest.Option#IfAbsent} to do a put only if there is
+ * no existing row that matches the primary key
+ *
Use {@link PutRequest.Option#IfPresent} to do a put only if there
+ * is an existing row that matches the primary key
+ *
Use {@link PutRequest.Option#IfVersion} to do a put only if there is
+ * an existing row that matches the primary key and its
+ * {@link Version} matches that provided
+ *
+ *
+ * It is also possible to return information about the existing
+ * row. The existing row, including its {@link Version} and modification
+ * time can be optionally returned.
+ * The existing row information will only be returned if
+ * {@link PutRequest#setReturnRow} is true and one of the following occurs:
+ *
+ *
The {@link PutRequest.Option#IfAbsent} is used and the operation
+ * fails because the row already exists.
+ *
The {@link PutRequest.Option#IfVersion} is used and the operation
+ * fails because the row exists and its version does not match.
+ *
The {@link PutRequest.Option#IfPresent} is used and the operation
+ * succeeds provided that the server supports providing the existing row.
+ *
The {@link PutRequest.Option} is not used and put operation replaces
+ * the existing row provided that the server supports providing the existing
+ * row.
+ *
+ * Use of {@link PutRequest#setReturnRow} may result in additional consumed
+ * read capacity.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ */
+ CompletableFuture put(PutRequest request);
+
+ /**
+ * Executes a sequence of operations associated with a table that share the
+ * same shard key portion of their primary keys, all the specified
+ * operations are executed within the scope of a single transaction.
+ * {@link WriteMultipleRequest}.
+ *
+ * There are some size-based limitations on this operation:
+ *
+ *
The max number of individual operations (put, delete) in a single
+ * WriteMultiple request is 50.
+ *
The total request size is limited to 25MB.
+ *
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link RowSizeLimitException} if the size of the request exceeds
+ * the maximum limit.
+ *
+ *
+ * {@link BatchOperationNumberLimitException} if the number of operations in
+ * the request exceeds the maximum limit.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ *
+ */
+ CompletableFuture writeMultiple(WriteMultipleRequest request);
+
+ /**
+ * Deletes multiple rows from a table in an atomic operation asynchronously.
+ * The key used may be partial but must contain all of the fields that are
+ * in the shard key. A range may be specified to delete a range of keys.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ *
+ */
+ CompletableFuture multiDelete(MultiDeleteRequest request);
+
+ /**
+ * Queries a table based on the query statement specified in the
+ * {@link QueryRequest} asynchronously.
+ *
+ * Queries that include a full shard key will execute much more efficiently
+ * than more distributed queries that must go to multiple shards.
+ *
+ * Table- and system-style queries such as "CREATE TABLE ..." or "DROP TABLE .."
+ * are not supported by this interface. Those operations must be performed
+ * using {@link #tableRequest} or {@link #systemRequest} as appropriate.
+ *
+ * The amount of data read by a single query request is limited by a system
+ * default and can be further limited using
+ * {@link QueryRequest#setMaxReadKB}. This limits the amount of data
+ * read and not the amount of data returned, which means
+ * that a query can return zero results but still have more data to read.
+ * This situation is detected by checking if the {@link QueryResult} has a
+ * continuation key, using {@link QueryResult#getContinuationKey}. For this
+ * reason queries should always operate in a loop, acquiring more results,
+ * until the continuation key is null, indicating that the query is done.
+ * Inside the loop the continuation key is applied to the
+ * {@link QueryRequest} using {@link QueryRequest#setContinuationKey}.
+ *
+ * Note: Since Query might use resources until they reach the end, it
+ * is necessary to close the QueryRequest or use the
+ * try-with-resources statement:
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ *
+ */
+ CompletableFuture query(QueryRequest request);
+
+ /**
+ * Queries a table based on the query statement specified in the
+ * {@link QueryRequest} while returning an {@link java.util.concurrent.Flow.Publisher}.
+ * The return {@link Flow.Publisher} can be subscribed only once.
+ *
+ * Queries that include a full shard key will execute much more efficiently
+ * than more distributed queries that must go to multiple shards.
+ *
+ * Remote calls, including preparation of a query statement, will not
+ * occur until the subscription happens.
+ *
+ * Table- and system-style queries such as "CREATE TABLE ..." or "DROP TABLE .."
+ * are not supported by this interface. Those operations must be performed using
+ * {@link #tableRequest} or {@link #systemRequest} as appropriate.
+ *
+ * Note: Publisher will close the {@link QueryRequest}
+ *
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return The {@link Flow.Publisher} of {@link MapValue}
+ *
+ * @throws IllegalArgumentException if any of the parameters are invalid or
+ * required parameters are missing
+ *
+ * @throws NoSQLException if the operation cannot be performed for any other
+ * reason
+ */
+ Flow.Publisher queryPaginator(QueryRequest request);
+ /**
+ * Prepares a query for execution and reuse asynchronously. See
+ * {@link #query} for general information and restrictions. It is
+ * recommended that prepared queries are used when the same query will run
+ * multiple times as execution is much more efficient than starting with a
+ * query string every time. The query language and API support query
+ * variables to assist with re-use.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ *
+ */
+ CompletableFuture prepare(PrepareRequest request);
+
+ /**
+ * Performs an operation on a table asynchronously. This method is used for
+ * creating and dropping tables and indexes as well as altering tables.
+ * Only one operation is allowed on a table at any one time.
+ *
+ * This operation is implicitly asynchronous. The caller must poll using
+ * methods on {@link TableResult} to determine when it has completed.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ *
+ */
+ CompletableFuture tableRequest(TableRequest request);
+
+ /**
+ * A convenience method that performs a TableRequest and waits for
+ * completion of the operation. This is the same as calling
+ * {@link #tableRequest} then calling {@link TableResult#waitForCompletion}.
+ * If the operation fails an exception is thrown. All parameters are
+ * required.
+ *
+ * @param request the {@link TableRequest} to perform.
+ *
+ * @param timeoutMs the amount of time to wait for completion, in
+ * milliseconds.
+ *
+ * @param pollIntervalMs the polling interval for the wait operation.
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link RequestTimeoutException} if the operation times out.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ * Performs a system operation on the system asynchronously, such as
+ * administrative operations that don't affect a specific table. For
+ * table-specific operations use {@link #tableRequest} or
+ * {@link #doTableRequest}.
+ *
+ * Examples of statements in the {@link SystemRequest} passed to this
+ * method include:
+ *
+ *
CREATE NAMESPACE mynamespace
+ *
CREATE USER some_user IDENTIFIED BY password
+ *
CREATE ROLE some_role
+ *
GRANT ROLE some_role TO USER some_user
+ *
+ *
+ * This operation is implicitly asynchronous. The caller must poll using
+ * methods on {@link SystemResult} to determine when it has completed.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ * Checks the status of an operation previously performed using
+ * {@link #systemRequest} asynchronously.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ *
+ */
+ CompletableFuture systemStatus(SystemStatusRequest request);
+
+ /**
+ * Gets static information about the specified table asynchronously
+ * including its state, provisioned throughput and capacity and schema.
+ * Dynamic information such as usage is obtained using {@link #getTableUsage}.
+ * Throughput, capacity and usage information is only available when using
+ * the Cloud Service and will be null or not defined on-premises.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link TableNotFoundException} if the specified table does not exist.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ * Gets dynamic information about the specified table asynchronously such as
+ * the current throughput usage. Usage information is collected in time
+ * slices and returned in individual usage records. It is possible to
+ * specify a time-based range of usage records using input parameters.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link TableNotFoundException} if the specified table does not exist.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ *
+ */
+ CompletableFuture getTableUsage(TableUsageRequest request);
+
+ /**
+ * Lists tables asynchronously, returning table names. If further information
+ * about a specific table is desired the {@link #getTable} interface may be
+ * used. If a given identity has access to a large number of tables the
+ * list may be paged using input parameters.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ *
+ */
+ CompletableFuture listTables(ListTablesRequest request);
+
+ /**
+ * Returns information about an index, or indexes on a table asynchronously.
+ * If no index name is specified in the {@link GetIndexesRequest}, then
+ * information on all indexes is returned.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ * Returns the namespaces in a store as an array of String.
+ *
+ * @return a {@link CompletableFuture} which completes with the namespaces
+ * or null if none are found.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ * Returns the roles in a store as an array of String.
+ *
+ * @return a {@link CompletableFuture} which completes with the list of
+ * roles or null if none are found.
+ *
+ * The CompletableFuture returned by this method can be completed exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ * Returns the users in a store as an array of {@link UserInfo}.
+ *
+ * @return a {@link CompletableFuture} which completes with the users
+ * or null if none are found.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ * A convenience method that performs a SystemRequest and waits for
+ * completion of the operation. This is the same as calling {@link
+ * #systemRequest} then calling {@link SystemResult#waitForCompletion}. If
+ * the operation fails an exception is thrown. All parameters are required.
+ *
+ * System requests are those related to namespaces and security and are
+ * generally independent of specific tables. Examples of statements include:
+ *
+ *
CREATE NAMESPACE mynamespace
+ *
CREATE USER some_user IDENTIFIED BY password
+ *
CREATE ROLE some_role
+ *
GRANT ROLE some_role TO USER some_user
+ *
+ *
+ * @param statement the system statement for the operation.
+ *
+ * @param timeoutMs the amount of time to wait for completion, in
+ * milliseconds.
+ *
+ * @param pollIntervalMs the polling interval for the wait operation.
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link RequestTimeoutException} if the operation times out.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ *
+ */
+ CompletableFuture doSystemRequest(String statement,
+ int timeoutMs,
+ int pollIntervalMs);
+
+ /**
+ * Cloud service only.
+ *
+ * Add replica to a table asynchronously.
+ *
+ * This operation is implicitly asynchronous. The caller must poll using
+ * methods on {@link TableResult} to determine when it has completed.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ * This operation is implicitly asynchronous. The caller must poll using
+ * methods on {@link TableResult} to determine when it has completed.
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ *
+ * TODO remove since
+ * @since 5.4.13
+ */
+ CompletableFuture dropReplica(DropReplicaRequest request);
+
+ /**
+ * Cloud service only.
+ *
+ * Gets replica statistics information asynchronously
+ *
+ * @param request the input parameters for the operation
+ *
+ * @return a {@link CompletableFuture} which completes with the result of
+ * the operation.
+ *
+ * The {@link CompletableFuture} returned by this method can be completed
+ * exceptionally with the following exceptions.
+ *
+ *
+ * {@link IllegalArgumentException} if any of the parameters are invalid or
+ * required parameters are missing.
+ *
+ *
+ * {@link TableNotFoundException} if the specified table does not exist.
+ *
+ *
+ * {@link NoSQLException} if the operation cannot be performed for
+ * any other reason.
+ *
+ *
+ *
+ * TODO remove since
+ * @since 5.4.13
+ */
+ CompletableFuture getReplicaStats(ReplicaStatsRequest request);
+
+ /**
+ * Returns an object that allows control over how SDK statistics
+ * are collected.
+ *
+ * @return the StatsControl object
+ *
+ * @since 5.2.30
+ */
+ StatsControl getStatsControl();
+
+ /**
+ * Closes the handle, releasing its memory and network resources. Once
+ * this method is closed the handle is no longer usable. Any attempt to
+ * use a closed handle will throw {@link IllegalArgumentException}.
+ */
+ void close();
+}
diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleFactory.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleFactory.java
index a6bf2033..2538d4aa 100644
--- a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleFactory.java
+++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleFactory.java
@@ -9,6 +9,7 @@
import static oracle.nosql.driver.util.CheckNull.requireNonNull;
+import oracle.nosql.driver.http.NoSQLHandleAsyncImpl;
import oracle.nosql.driver.http.NoSQLHandleImpl;
/**
@@ -43,4 +44,33 @@ public static NoSQLHandle createNoSQLHandle(NoSQLHandleConfig config) {
}
return new NoSQLHandleImpl(configCopy);
}
+
+ /**
+ * Creates a async handle that can be used to access tables.
+ * The application must invoke {@link NoSQLHandleAsync#close},
+ * when it is done accessing the system to
+ * free up resources associated with the handle.
+ *
+ * @param config the NoSQLHandle configuration parameters
+ *
+ * @return a valid {@link NoSQLHandleAsync} instance, ready for use
+ *
+ * @throws IllegalArgumentException if an illegal configuration parameter
+ * is specified.
+ *
+ * @see NoSQLHandleAsync#close
+ */
+ public static NoSQLHandleAsync createNoSQLHandleAsync(NoSQLHandleConfig config) {
+ requireNonNull(
+ config,
+ "NoSQLHandleFactory.createNoSQLHandle: config cannot be null");
+ NoSQLHandleConfig configCopy = config.clone();
+ if (configCopy.getRetryHandler() == null) {
+ /*
+ * Default retry handler: 10 retries, default backoff
+ */
+ configCopy.configureDefaultRetryHandler(10, 0);
+ }
+ return new NoSQLHandleAsyncImpl(configCopy);
+ }
}
diff --git a/driver/src/main/java/oracle/nosql/driver/http/Client.java b/driver/src/main/java/oracle/nosql/driver/http/Client.java
index 93903d10..52b054e9 100644
--- a/driver/src/main/java/oracle/nosql/driver/http/Client.java
+++ b/driver/src/main/java/oracle/nosql/driver/http/Client.java
@@ -54,8 +54,6 @@
import java.util.concurrent.Executors;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -141,8 +139,6 @@
public final class Client {
public static int traceLevel = 0;
- private static final int cores = Runtime.getRuntime().availableProcessors();
-
private final NoSQLHandleConfig config;
private final SerializerFactory v3factory = new BinarySerializerFactory();
@@ -311,7 +307,8 @@ private static class RequestContext {
}
public Client(Logger logger,
- NoSQLHandleConfig httpConfig) {
+ NoSQLHandleConfig httpConfig,
+ ScheduledExecutorService taskExecutor) {
this.logger = logger;
this.config = httpConfig;
@@ -397,33 +394,7 @@ public Client(Logger logger,
/* for internal testing */
prepareFilename = System.getProperty("test.preparefilename");
-
- taskExecutor = new ScheduledThreadPoolExecutor(cores /* core threads */,
- new ThreadFactory() {
- private final AtomicInteger threadNumber = new AtomicInteger(1);
- @Override
- public Thread newThread(Runnable r) {
- final Thread t = Executors.defaultThreadFactory().newThread(r);
- t.setName(String.format("nosql-task-executor-%s",
- threadNumber.getAndIncrement()));
- t.setDaemon(true);
- t.setUncaughtExceptionHandler((thread, error) -> {
- if (ConcurrentUtil.unwrapCompletionException(error)
- instanceof RejectedExecutionException) {
- /*
- * Ignore uncaught error for rejected exception
- * since that is expected to happen during
- * executor shut down.
- */
- return;
- }
- logger.warning(() -> String.format(
- "Uncaught exception from %s: %s",
- error, LogUtil.getStackTrace(error)));
- });
- return t;
- }
- });
+ this.taskExecutor = taskExecutor;
initErrorHandlers();
}
diff --git a/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleAsyncImpl.java b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleAsyncImpl.java
new file mode 100644
index 00000000..a985939b
--- /dev/null
+++ b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleAsyncImpl.java
@@ -0,0 +1,504 @@
+package oracle.nosql.driver.http;
+
+import io.netty.handler.ssl.SslContextBuilder;
+import io.netty.util.internal.logging.InternalLoggerFactory;
+import io.netty.util.internal.logging.JdkLoggerFactory;
+import oracle.nosql.driver.AuthorizationProvider;
+import oracle.nosql.driver.NoSQLHandleAsync;
+import oracle.nosql.driver.NoSQLHandleConfig;
+import oracle.nosql.driver.StatsControl;
+import oracle.nosql.driver.UserInfo;
+import oracle.nosql.driver.iam.SignatureProvider;
+import oracle.nosql.driver.kv.StoreAccessTokenProvider;
+import oracle.nosql.driver.ops.AddReplicaRequest;
+import oracle.nosql.driver.ops.DeleteRequest;
+import oracle.nosql.driver.ops.DeleteResult;
+import oracle.nosql.driver.ops.DropReplicaRequest;
+import oracle.nosql.driver.ops.GetIndexesRequest;
+import oracle.nosql.driver.ops.GetIndexesResult;
+import oracle.nosql.driver.ops.GetRequest;
+import oracle.nosql.driver.ops.GetResult;
+import oracle.nosql.driver.ops.GetTableRequest;
+import oracle.nosql.driver.ops.ListTablesRequest;
+import oracle.nosql.driver.ops.ListTablesResult;
+import oracle.nosql.driver.ops.MultiDeleteRequest;
+import oracle.nosql.driver.ops.MultiDeleteResult;
+import oracle.nosql.driver.ops.PrepareRequest;
+import oracle.nosql.driver.ops.PrepareResult;
+import oracle.nosql.driver.ops.PutRequest;
+import oracle.nosql.driver.ops.PutResult;
+import oracle.nosql.driver.ops.QueryPublisher;
+import oracle.nosql.driver.ops.QueryRequest;
+import oracle.nosql.driver.ops.QueryResult;
+import oracle.nosql.driver.ops.ReplicaStatsRequest;
+import oracle.nosql.driver.ops.ReplicaStatsResult;
+import oracle.nosql.driver.ops.Request;
+import oracle.nosql.driver.ops.Result;
+import oracle.nosql.driver.ops.SystemRequest;
+import oracle.nosql.driver.ops.SystemResult;
+import oracle.nosql.driver.ops.SystemStatusRequest;
+import oracle.nosql.driver.ops.TableRequest;
+import oracle.nosql.driver.ops.TableResult;
+import oracle.nosql.driver.ops.TableUsageRequest;
+import oracle.nosql.driver.ops.TableUsageResult;
+import oracle.nosql.driver.ops.WriteMultipleRequest;
+import oracle.nosql.driver.ops.WriteMultipleResult;
+import oracle.nosql.driver.util.ConcurrentUtil;
+import oracle.nosql.driver.util.LogUtil;
+import oracle.nosql.driver.values.FieldValue;
+import oracle.nosql.driver.values.JsonUtils;
+import oracle.nosql.driver.values.MapValue;
+
+import javax.net.ssl.SSLException;
+import java.util.ArrayList;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Flow;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.logging.Logger;
+
+public class NoSQLHandleAsyncImpl implements NoSQLHandleAsync {
+ private static final int cores = Runtime.getRuntime().availableProcessors();
+
+ /*
+ * The HTTP client. This is not final so that it can be nulled upon
+ * close.
+ */
+ private final Client client;
+ private final AtomicBoolean isClosed = new AtomicBoolean(false);
+
+ /* thread-pool for scheduling tasks */
+ private final ScheduledExecutorService taskExecutor;
+
+ public NoSQLHandleAsyncImpl(NoSQLHandleConfig config) {
+ configNettyLogging();
+ final Logger logger = getLogger(config);
+ /*
+ * config SslContext first, on-prem authorization provider
+ * will reuse the context in NoSQLHandleConfig
+ */
+ configSslContext(config);
+ taskExecutor = new ScheduledThreadPoolExecutor(cores /* core threads */,
+ new ThreadFactory() {
+ private final AtomicInteger threadNumber = new AtomicInteger(1);
+ @Override
+ public Thread newThread(Runnable r) {
+ final Thread t = Executors.defaultThreadFactory().newThread(r);
+ t.setName(String.format("nosql-task-executor-%s",
+ threadNumber.getAndIncrement()));
+ t.setDaemon(true);
+ t.setUncaughtExceptionHandler((thread, error) -> {
+ if (ConcurrentUtil.unwrapCompletionException(error)
+ instanceof RejectedExecutionException) {
+ /*
+ * Ignore uncaught error for rejected exception
+ * since that is expected to happen during
+ * executor shut down.
+ */
+ return;
+ }
+ logger.warning(() -> String.format(
+ "Uncaught exception from %s: %s",
+ error, LogUtil.getStackTrace(error)));
+ });
+ return t;
+ }
+ });
+ client = new Client(logger, config, taskExecutor);
+ try {
+ /* configAuthProvider may use client */
+ configAuthProvider(logger, config);
+ } catch (RuntimeException re) {
+ /* cleanup client */
+ client.shutdown();
+ taskExecutor.shutdown();
+ throw re;
+ }
+ }
+
+ /**
+ * Returns the logger used for the driver. If no logger is specified
+ * create one based on this class name.
+ */
+ private Logger getLogger(NoSQLHandleConfig config) {
+ if (config.getLogger() != null) {
+ return config.getLogger();
+ }
+
+ /*
+ * The default logger logs at INFO. If this is too verbose users
+ * must create a logger and pass it in.
+ */
+ Logger logger = Logger.getLogger(getClass().getName());
+ return logger;
+ }
+
+ /**
+ * Configures the logging of Netty library.
+ */
+ private void configNettyLogging() {
+ /*
+ * Configure default Netty logging using Jdk Logger.
+ */
+ InternalLoggerFactory.setDefaultFactory(JdkLoggerFactory.INSTANCE);
+ }
+
+ private void configSslContext(NoSQLHandleConfig config) {
+ if (config.getSslContext() != null) {
+ return;
+ }
+ if (config.getServiceURL().getProtocol().equalsIgnoreCase("HTTPS")) {
+ try {
+ SslContextBuilder builder = SslContextBuilder.forClient();
+ if (config.getSSLCipherSuites() != null) {
+ builder.ciphers(config.getSSLCipherSuites());
+ }
+ if (config.getSSLProtocols() != null) {
+ builder.protocols(config.getSSLProtocols());
+ }
+ builder.sessionTimeout(config.getSSLSessionTimeout());
+ builder.sessionCacheSize(config.getSSLSessionCacheSize());
+ config.setSslContext(builder.build());
+ } catch (SSLException se) {
+ throw new IllegalStateException(
+ "Unable to start handle with SSL", se);
+ }
+ }
+ }
+
+ private void configAuthProvider(Logger logger, NoSQLHandleConfig config) {
+ final AuthorizationProvider ap = config.getAuthorizationProvider();
+ if (ap instanceof StoreAccessTokenProvider) {
+ final StoreAccessTokenProvider stProvider =
+ (StoreAccessTokenProvider) ap;
+ if (stProvider.getLogger() == null) {
+ stProvider.setLogger(logger);
+ }
+ if (stProvider.isSecure() &&
+ stProvider.getEndpoint() == null) {
+ String endpoint = config.getServiceURL().toString();
+ if (endpoint.endsWith("/")) {
+ endpoint = endpoint.substring(0, endpoint.length() - 1);
+ }
+ stProvider.setEndpoint(endpoint)
+ .setSslContext(config.getSslContext())
+ .setSslHandshakeTimeout(
+ config.getSSLHandshakeTimeout());
+ }
+
+ } else if (ap instanceof SignatureProvider) {
+ SignatureProvider sigProvider = (SignatureProvider) ap;
+ if (sigProvider.getLogger() == null) {
+ sigProvider.setLogger(logger);
+ }
+ sigProvider.prepare(config);
+ if (config.getAuthRefresh()) {
+ sigProvider.setOnSignatureRefresh(new SigRefresh());
+ client.createAuthRefreshList();
+ }
+ }
+ }
+
+ @Override
+ public CompletableFuture delete(DeleteRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public CompletableFuture get(GetRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public CompletableFuture put(PutRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public CompletableFuture writeMultiple(WriteMultipleRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public CompletableFuture multiDelete(MultiDeleteRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public CompletableFuture query(QueryRequest request) {
+ return queryAsync(request);
+ }
+
+ CompletableFuture queryAsync(QueryRequest request) {
+ return executeASync(request)
+ .thenCompose(result -> {
+ /* Complex queries need RCB, run asynchronously */
+ if (!request.isSimpleQuery()) {
+ // TODO supplyAsync runs in fork-join pool.
+ // Change to dedicated pool
+ return CompletableFuture.supplyAsync(() -> result);
+ }
+ return CompletableFuture.completedFuture(result);
+ })
+ .thenApply(result -> ((QueryResult) result));
+ }
+
+ @Override
+ public Flow.Publisher queryPaginator(QueryRequest request) {
+ return new QueryPublisher(this, request);
+ }
+
+ @Override
+ public CompletableFuture prepare(PrepareRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public CompletableFuture tableRequest(TableRequest request) {
+ return executeASync(request).thenApply(tres -> {
+ TableResult res = (TableResult) tres;
+ /* update rate limiters, if table has limits */
+ client.updateRateLimiters(res.getTableName(), res.getTableLimits());
+ return res;
+ });
+ }
+
+ @Override
+ public CompletableFuture getTable(GetTableRequest request) {
+ return executeASync(request).thenApply(tres -> {
+ TableResult res = (TableResult) tres;
+ /* update rate limiters, if table has limits */
+ client.updateRateLimiters(res.getTableName(), res.getTableLimits());
+ return res;
+ });
+ }
+
+ @Override
+ public CompletableFuture systemRequest(SystemRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public CompletableFuture systemStatus(SystemStatusRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public CompletableFuture getTableUsage(TableUsageRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public CompletableFuture listTables(ListTablesRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public CompletableFuture getIndexes(GetIndexesRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public CompletableFuture addReplica(AddReplicaRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public CompletableFuture dropReplica(DropReplicaRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public CompletableFuture getReplicaStats(ReplicaStatsRequest request) {
+ return executeASync(request);
+ }
+
+ @Override
+ public void close() {
+ if (isClosed.compareAndSet(false, true)) {
+ client.shutdown();
+ taskExecutor.shutdown();
+ }
+ }
+
+ @Override
+ public CompletableFuture listNamespaces() {
+ return doSystemRequest("show as json namespaces")
+ .thenApply((SystemResult dres )-> {
+ String jsonResult = dres.getResultString();
+ if (jsonResult == null) {
+ return null;
+ }
+ MapValue root = JsonUtils.createValueFromJson(jsonResult, null).asMap();
+
+ FieldValue namespaces = root.get("namespaces");
+ if (namespaces == null) {
+ return null;
+ }
+
+ ArrayList results = new ArrayList(
+ namespaces.asArray().size());
+ for (FieldValue val : namespaces.asArray()) {
+ results.add(val.getString());
+ }
+ return results.toArray(new String[0]);
+ });
+ }
+
+ @Override
+ public CompletableFuture listUsers() {
+ return doSystemRequest("show as json users")
+ .thenApply((SystemResult dres) -> {
+ String jsonResult = dres.getResultString();
+ if (jsonResult == null) {
+ return null;
+ }
+
+ MapValue root = JsonUtils.createValueFromJson(
+ jsonResult, null).asMap();
+
+ FieldValue users = root.get("users");
+ if (users == null) {
+ return null;
+ }
+
+ ArrayList results = new ArrayList(
+ users.asArray().size());
+
+ for (FieldValue val : users.asArray()) {
+ String id = val.asMap().getString("id");
+ String name = val.asMap().getString("name");
+ results.add(new UserInfo(id, name));
+ }
+ return results.toArray(new UserInfo[0]);
+ });
+ }
+
+ @Override
+ public CompletableFuture listRoles() {
+ return doSystemRequest("show as json roles")
+ .thenApply((SystemResult dres) -> {
+ String jsonResult = dres.getResultString();
+ if (jsonResult == null) {
+ return null;
+ }
+ MapValue root = JsonUtils.createValueFromJson(
+ jsonResult, null).asMap();
+
+ FieldValue roles = root.get("roles");
+ if (roles == null) {
+ return null;
+ }
+
+ ArrayList results = new ArrayList(
+ roles.asArray().size());
+ for (FieldValue val : roles.asArray()) {
+ String role = val.asMap().getString("name");
+ results.add(role);
+ }
+ return results.toArray(new String[0]);
+ });
+ }
+
+ /**
+ * Internal method used by list* methods that defaults timeouts.
+ */
+ private CompletableFuture doSystemRequest(String statement) {
+ return doSystemRequest(statement, 30000, 1000);
+ }
+
+ @Override
+ public CompletableFuture doTableRequest(TableRequest request,
+ int timeoutMs,
+ int pollIntervalMs) {
+
+ return tableRequest(request).thenCompose((TableResult res) ->
+ res.waitForCompletionAsync(this, timeoutMs, pollIntervalMs)
+ .thenApply(v -> res));
+ }
+
+ @Override
+ public CompletableFuture doSystemRequest(String statement,
+ int timeoutMs,
+ int pollIntervalMs) {
+ checkClient();
+ SystemRequest dreq =
+ new SystemRequest().setStatement(statement.toCharArray());
+ return systemRequest(dreq).thenCompose(
+ (SystemResult dres) ->
+ dres.waitForCompletionAsync(this, timeoutMs, pollIntervalMs)
+ .thenApply(v -> dres));
+ }
+
+ @Override
+ public StatsControl getStatsControl() {
+ return client.getStatsControl();
+ }
+
+ void checkClient() {
+ if (isClosed.get()) {
+ throw new IllegalStateException("NoSQLHandle has been closed");
+ }
+ }
+
+ /**
+ * @hidden
+ * For testing use
+ */
+ public Client getClient() {
+ return client;
+ }
+
+ /**
+ * @hidden
+ * For testing use
+ */
+ public short getSerialVersion() {
+ return client.getSerialVersion();
+ }
+
+ /**
+ * @hidden
+ *
+ * Testing use only.
+ */
+ public void setDefaultNamespace(String ns) {
+ client.setDefaultNamespace(ns);
+ }
+
+
+ @SuppressWarnings("unchecked")
+ CompletableFuture executeASync(Request request) {
+ checkClient();
+ return client.execute(request).thenApply(result -> (T) result);
+ }
+
+ public ScheduledExecutorService getTaskExecutor() {
+ return taskExecutor;
+ }
+
+ /**
+ * Cloud service only.
+ * The refresh method of this class is called when a Signature is refreshed
+ * in SignatureProvider. This happens every 4 minutes or so. This mechanism
+ * allows the authentication and authorization information cached by the
+ * server to be refreshed out of band with the normal request path.
+ */
+ private class SigRefresh implements SignatureProvider.OnSignatureRefresh {
+
+ /*
+ * Attempt to refresh the server's authentication and authorization
+ * information for a new signature.
+ */
+ @Override
+ public void refresh(long refreshMs) {
+ client.doRefresh(refreshMs);
+ }
+ }
+}
diff --git a/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
index d40abb17..c284b21f 100644
--- a/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
+++ b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
@@ -7,19 +7,12 @@
package oracle.nosql.driver.http;
-import java.util.ArrayList;
import java.util.concurrent.CompletableFuture;
-import java.util.logging.Logger;
-import javax.net.ssl.SSLException;
-
-import oracle.nosql.driver.AuthorizationProvider;
import oracle.nosql.driver.NoSQLHandle;
import oracle.nosql.driver.NoSQLHandleConfig;
import oracle.nosql.driver.StatsControl;
import oracle.nosql.driver.UserInfo;
-import oracle.nosql.driver.iam.SignatureProvider;
-import oracle.nosql.driver.kv.StoreAccessTokenProvider;
import oracle.nosql.driver.ops.AddReplicaRequest;
import oracle.nosql.driver.ops.DeleteRequest;
import oracle.nosql.driver.ops.DeleteResult;
@@ -54,13 +47,6 @@
import oracle.nosql.driver.ops.WriteMultipleRequest;
import oracle.nosql.driver.ops.WriteMultipleResult;
import oracle.nosql.driver.util.ConcurrentUtil;
-import oracle.nosql.driver.values.FieldValue;
-import oracle.nosql.driver.values.JsonUtils;
-import oracle.nosql.driver.values.MapValue;
-
-import io.netty.handler.ssl.SslContextBuilder;
-import io.netty.util.internal.logging.InternalLoggerFactory;
-import io.netty.util.internal.logging.JdkLoggerFactory;
/**
* The methods in this class require non-null arguments. Because they all
@@ -68,114 +54,14 @@
* single place.
*/
public class NoSQLHandleImpl implements NoSQLHandle {
-
/*
* The HTTP client. This is not final so that it can be nulled upon
* close.
*/
- private Client client;
+ private final NoSQLHandleAsyncImpl asyncHandle;
public NoSQLHandleImpl(NoSQLHandleConfig config) {
-
- configNettyLogging();
- final Logger logger = getLogger(config);
-
- /*
- * config SslContext first, on-prem authorization provider
- * will reuse the context in NoSQLHandleConfig
- */
- configSslContext(config);
- client = new Client(logger, config);
- try {
- /* configAuthProvider may use client */
- configAuthProvider(logger, config);
- } catch (RuntimeException re) {
- /* cleanup client */
- client.shutdown();
- throw re;
- }
- }
-
- /**
- * Returns the logger used for the driver. If no logger is specified
- * create one based on this class name.
- */
- private Logger getLogger(NoSQLHandleConfig config) {
- if (config.getLogger() != null) {
- return config.getLogger();
- }
-
- /*
- * The default logger logs at INFO. If this is too verbose users
- * must create a logger and pass it in.
- */
- Logger logger = Logger.getLogger(getClass().getName());
- return logger;
- }
-
- /**
- * Configures the logging of Netty library.
- */
- private void configNettyLogging() {
- /*
- * Configure default Netty logging using Jdk Logger.
- */
- InternalLoggerFactory.setDefaultFactory(JdkLoggerFactory.INSTANCE);
- }
-
- private void configSslContext(NoSQLHandleConfig config) {
- if (config.getSslContext() != null) {
- return;
- }
- if (config.getServiceURL().getProtocol().equalsIgnoreCase("HTTPS")) {
- try {
- SslContextBuilder builder = SslContextBuilder.forClient();
- if (config.getSSLCipherSuites() != null) {
- builder.ciphers(config.getSSLCipherSuites());
- }
- if (config.getSSLProtocols() != null) {
- builder.protocols(config.getSSLProtocols());
- }
- builder.sessionTimeout(config.getSSLSessionTimeout());
- builder.sessionCacheSize(config.getSSLSessionCacheSize());
- config.setSslContext(builder.build());
- } catch (SSLException se) {
- throw new IllegalStateException(
- "Unable to start handle with SSL", se);
- }
- }
- }
-
- private void configAuthProvider(Logger logger, NoSQLHandleConfig config) {
- final AuthorizationProvider ap = config.getAuthorizationProvider();
- if (ap instanceof StoreAccessTokenProvider) {
- final StoreAccessTokenProvider stProvider =
- (StoreAccessTokenProvider) ap;
- if (stProvider.getLogger() == null) {
- stProvider.setLogger(logger);
- }
- if (stProvider.isSecure() &&
- stProvider.getEndpoint() == null) {
- String endpoint = config.getServiceURL().toString();
- if (endpoint.endsWith("/")) {
- endpoint = endpoint.substring(0, endpoint.length() - 1);
- }
- stProvider.setEndpoint(endpoint)
- .setSslContext(config.getSslContext())
- .setSslHandshakeTimeout(
- config.getSSLHandshakeTimeout());
- }
- } else if (ap instanceof SignatureProvider) {
- SignatureProvider sigProvider = (SignatureProvider) ap;
- if (sigProvider.getLogger() == null) {
- sigProvider.setLogger(logger);
- }
- sigProvider.prepare(config);
- if (config.getAuthRefresh()) {
- sigProvider.setOnSignatureRefresh(new SigRefresh());
- client.createAuthRefreshList();
- }
- }
+ asyncHandle = new NoSQLHandleAsyncImpl(config);
}
@Override
@@ -205,27 +91,12 @@ public MultiDeleteResult multiDelete(MultiDeleteRequest request) {
@Override
public QueryResult query(QueryRequest request) {
- return ConcurrentUtil.awaitFuture(queryAsync(request));
- }
-
- private CompletableFuture queryAsync(QueryRequest request) {
- checkClient();
- return client.execute(request)
- .thenCompose(result -> {
- /* Complex queries need RCB, run asynchronously */
- if (!request.isSimpleQuery()) {
- // TODO supplyAsync runs in fork-join pool.
- // Change to dedicated pool
- return CompletableFuture.supplyAsync(() -> result);
- }
- return CompletableFuture.completedFuture(result);
- })
- .thenApply(result -> ((QueryResult) result));
+ return ConcurrentUtil.awaitFuture(asyncHandle.queryAsync(request));
}
@Override
public QueryIterableResult queryIterable(QueryRequest request) {
- checkClient();
+ asyncHandle.checkClient();
return new QueryIterableResult(request, this);
}
@@ -236,20 +107,12 @@ public PrepareResult prepare(PrepareRequest request) {
@Override
public TableResult tableRequest(TableRequest request) {
- checkClient();
- TableResult res = executeSync(request);
- /* update rate limiters, if table has limits */
- client.updateRateLimiters(res.getTableName(), res.getTableLimits());
- return res;
+ return executeSync(request);
}
@Override
public TableResult getTable(GetTableRequest request) {
- checkClient();
- TableResult res = executeSync(request);
- /* update rate limiters, if table has limits */
- client.updateRateLimiters(res.getTableName(), res.getTableLimits());
- return res;
+ return executeSync(request);
}
@Override
@@ -294,10 +157,8 @@ public ReplicaStatsResult getReplicaStats(ReplicaStatsRequest request) {
}
@Override
- synchronized public void close() {
- checkClient();
- client.shutdown();
- client = null;
+ public void close() {
+ asyncHandle.close();
}
/**
@@ -307,25 +168,7 @@ synchronized public void close() {
*/
@Override
public String[] listNamespaces() {
- SystemResult dres = doSystemRequest("show as json namespaces");
-
- String jsonResult = dres.getResultString();
- if (jsonResult == null) {
- return null;
- }
- MapValue root = JsonUtils.createValueFromJson(jsonResult, null).asMap();
-
- FieldValue namespaces = root.get("namespaces");
- if (namespaces == null) {
- return null;
- }
-
- ArrayList results = new ArrayList(
- namespaces.asArray().size());
- for (FieldValue val : namespaces.asArray()) {
- results.add(val.getString());
- }
- return results.toArray(new String[0]);
+ return ConcurrentUtil.awaitFuture(asyncHandle.listNamespaces());
}
/**
@@ -335,29 +178,7 @@ public String[] listNamespaces() {
*/
@Override
public UserInfo[] listUsers() {
- SystemResult dres = doSystemRequest("show as json users");
-
- String jsonResult = dres.getResultString();
- if (jsonResult == null) {
- return null;
- }
-
- MapValue root = JsonUtils.createValueFromJson(jsonResult, null).asMap();
-
- FieldValue users = root.get("users");
- if (users == null) {
- return null;
- }
-
- ArrayList results = new ArrayList(
- users.asArray().size());
-
- for (FieldValue val : users.asArray()) {
- String id = val.asMap().getString("id");
- String name = val.asMap().getString("name");
- results.add(new UserInfo(id, name));
- }
- return results.toArray(new UserInfo[0]);
+ return ConcurrentUtil.awaitFuture(asyncHandle.listUsers());
}
/**
@@ -367,26 +188,7 @@ public UserInfo[] listUsers() {
*/
@Override
public String[] listRoles() {
- SystemResult dres = doSystemRequest("show as json roles");
-
- String jsonResult = dres.getResultString();
- if (jsonResult == null) {
- return null;
- }
- MapValue root = JsonUtils.createValueFromJson(jsonResult, null).asMap();
-
- FieldValue roles = root.get("roles");
- if (roles == null) {
- return null;
- }
-
- ArrayList results = new ArrayList(
- roles.asArray().size());
- for (FieldValue val : roles.asArray()) {
- String role = val.asMap().getString("name");
- results.add(role);
- }
- return results.toArray(new String[0]);
+ return ConcurrentUtil.awaitFuture(asyncHandle.listRoles());
}
@@ -420,16 +222,14 @@ public SystemResult doSystemRequest(String statement,
@Override
public StatsControl getStatsControl() {
- return client.getStatsControl();
+ return asyncHandle.getStatsControl();
}
/**
* Ensure that the client exists and hasn't been closed;
*/
private void checkClient() {
- if (client == null) {
- throw new IllegalStateException("NoSQLHandle has been closed");
- }
+ asyncHandle.checkClient();
}
/**
@@ -437,7 +237,7 @@ private void checkClient() {
* For testing use
*/
public Client getClient() {
- return client;
+ return asyncHandle.getClient();
}
/**
@@ -445,7 +245,7 @@ public Client getClient() {
* For testing use
*/
public short getSerialVersion() {
- return client.getSerialVersion();
+ return asyncHandle.getSerialVersion();
}
/**
@@ -454,31 +254,11 @@ public short getSerialVersion() {
* Testing use only.
*/
public void setDefaultNamespace(String ns) {
- client.setDefaultNamespace(ns);
- }
-
- /**
- * Cloud service only.
- * The refresh method of this class is called when a Signature is refreshed
- * in SignatureProvider. This happens every 4 minutes or so. This mechanism
- * allows the authentication and authorization information cached by the
- * server to be refreshed out of band with the normal request path.
- */
- private class SigRefresh implements SignatureProvider.OnSignatureRefresh {
-
- /*
- * Attempt to refresh the server's authentication and authorization
- * information for a new signature.
- */
- @Override
- public void refresh(long refreshMs) {
- client.doRefresh(refreshMs);
- }
+ asyncHandle.setDefaultNamespace(ns);
}
@SuppressWarnings("unchecked")
private T executeSync(Request request) {
- checkClient();
- return (T) ConcurrentUtil.awaitFuture(client.execute(request));
+ return (T) ConcurrentUtil.awaitFuture(asyncHandle.executeASync(request));
}
}
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java b/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
index 4d323110..96d444fb 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
@@ -11,11 +11,11 @@
import static oracle.nosql.driver.util.LogUtil.logInfo;
import java.util.Map;
-import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
import java.util.logging.Logger;
import io.netty.bootstrap.Bootstrap;
@@ -102,12 +102,15 @@ class ConnectionPool {
private final Map stats;
private int acquiredChannelCount;
+ /* Executor to run keep-alive task periodically */
+ private final ScheduledExecutorService keepAlivescheduler;
+
/**
* Keepalive callback interface
*/
@FunctionalInterface
interface KeepAlive {
- CompletableFuture keepAlive(Channel ch);
+ boolean keepAlive(Channel ch);
}
/**
@@ -178,10 +181,17 @@ protected void initChannel(Channel ch) throws Exception {
DEFAULT_REFRESH_PERIOD_SECS :
Math.min(DEFAULT_REFRESH_PERIOD_SECS,
this.inactivityPeriodSeconds);
- this.bootstrap.config().group().next()
- .scheduleAtFixedRate(new RefreshTask(),
- refreshPeriod, refreshPeriod,
- TimeUnit.SECONDS);
+ this.keepAlivescheduler =
+ Executors.newSingleThreadScheduledExecutor(r -> {
+ Thread t = new Thread(r, "nosql-keep-alive");
+ t.setDaemon(true);
+ return t;
+ });
+ keepAlivescheduler.scheduleAtFixedRate(new RefreshTask(),
+ refreshPeriod, refreshPeriod,
+ TimeUnit.SECONDS);
+ } else {
+ this.keepAlivescheduler = null;
}
}
@@ -281,6 +291,9 @@ public void removeChannel(Channel channel) {
*/
void close() {
logFine(logger, "Closing pool, stats " + getStats());
+ if (keepAlivescheduler != null) {
+ keepAlivescheduler.shutdown();
+ }
/* TODO: do this cleanly */
validatePool("close1");
Channel ch = queue.pollFirst();
@@ -451,7 +464,7 @@ int doKeepAlive(int keepAlivePeriod) {
* Don't remove a channel from the queue until it's clear that
* it will be used
*/
- AtomicInteger numSent = new AtomicInteger();
+ int numSent = 0;
for (Channel ch : queue) {
if (!ch.isActive()) {
continue;
@@ -468,22 +481,18 @@ int doKeepAlive(int keepAlivePeriod) {
continue;
}
logFine(logger,
- "Sending keepalive on channel " + ch + ", stats: " + cs);
- keepAlive.keepAlive(ch).handle((didKeepalive, err) -> {
- if (err != null) {
- logFine(logger,
- "Keepalive failed on channel "
- + ch
- + ", removing from pool");
- removeChannel(ch);
- } else {
- cs.acquired(); /* update lastAcquired time */
- numSent.getAndIncrement();
- //TODO is this operation safe inside for-each?
- queue.addFirst(ch);
- }
- return null;
- });
+ "Sending keepalive on channel " + ch + ", stats: " + cs);
+ boolean didKeepalive = keepAlive.keepAlive(ch);
+ if (!didKeepalive) {
+ logFine(logger,
+ "Keepalive failed on channel " + ch +
+ ", removing from pool");
+ removeChannel(ch);
+ continue;
+ }
+ cs.acquired(); /* update lastAcquired time */
+ numSent++;
+ queue.addFirst(ch);
}
/*
* channels that are not used but have been used within the
@@ -494,9 +503,9 @@ int doKeepAlive(int keepAlivePeriod) {
break;
}
}
- //TODO how to validate this without blocking?
- //validatePool("doKeepAlive");
- return numSent.get();
+ validatePool("doKeepAlive");
+
+ return numSent;
}
private void validatePool(final String caller) {
@@ -517,7 +526,7 @@ private void validatePool(final String caller) {
*/
private void updateStats(Channel channel, boolean isAcquire) {
ChannelStats cstats = stats.get(channel);
- if (cstats == null) {
+ if (cstats == null && isAcquire) {
cstats = new ChannelStats();
stats.put(channel, cstats);
}
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
index e7eccd91..8832ffa7 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
@@ -39,6 +39,7 @@
import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.FutureListener;
import oracle.nosql.driver.NoSQLHandleConfig;
+import oracle.nosql.driver.util.ConcurrentUtil;
/**
* Netty HTTP client. Initialization process:
@@ -272,7 +273,7 @@ private HttpClient(String host,
/* this is the main request client */
pool.setKeepAlive(new ConnectionPool.KeepAlive() {
@Override
- public CompletableFuture keepAlive(Channel ch) {
+ public boolean keepAlive(Channel ch) {
return doKeepAlive(ch);
}
});
@@ -503,38 +504,39 @@ private static boolean isTimeout(long deadlineNs) {
/**
* Use HTTP HEAD method to refresh the channel
*/
- CompletableFuture doKeepAlive(Channel ch) {
- CompletableFuture future = new CompletableFuture<>();
+ boolean doKeepAlive(Channel ch) {
final int keepAliveTimeout = 3000; /* ms */
- final HttpRequest request =
+ try {
+ final HttpRequest request =
new DefaultFullHttpRequest(HTTP_1_1, HEAD, "/");
- /*
- * All requests need a HOST header or the LBaaS (nginx) or
- * other server may reject them and close the connection
- */
- request.headers().add(HOST, host);
- runRequest(request, ch, keepAliveTimeout)
- .thenApply(fullHttpResponse -> {
+
+ /*
+ * All requests need a HOST header or the LBaaS (nginx) or
+ * other server may reject them and close the connection
+ */
+ request.headers().add(HOST, host);
+ FullHttpResponse response = ConcurrentUtil.awaitFuture(
+ runRequest(request, ch, keepAliveTimeout));
/*
* LBaaS will return a non-200 status but that is expected as the
* path "/" does not map to the service. This is ok because all that
* matters is that the connection remain alive.
*/
- String conn = fullHttpResponse.headers().get(CONNECTION);
- if (!"keep-alive".equalsIgnoreCase(conn)) {
+ String conn = response.headers().get(CONNECTION);
+ if (conn == null || !"keep-alive".equalsIgnoreCase(conn)) {
logFine(logger,
- "Keepalive HEAD request did not return keep-alive "
- + "in connection header, is: " + conn);
+ "Keepalive HEAD request did not return keep-alive " +
+ "in connection header, is: " + conn);
}
- return fullHttpResponse;
- }).whenComplete((res, err) -> {
- res.release();
- if (err != null) {
- future.complete(false);
- } else {
- future.complete(true);
- }
- });
- return future;
+
+ return true;
+ } catch (Throwable t) {
+ String msg = String.format(
+ "Exception sending keepalive on [channel:%s] error:%s",
+ ch.id(), t.getMessage());
+ logFine(logger, msg);
+ }
+ /* something went wrong, caller is responsible for disposition */
+ return false;
}
}
diff --git a/driver/src/main/java/oracle/nosql/driver/ops/QueryPublisher.java b/driver/src/main/java/oracle/nosql/driver/ops/QueryPublisher.java
new file mode 100644
index 00000000..c5abe85e
--- /dev/null
+++ b/driver/src/main/java/oracle/nosql/driver/ops/QueryPublisher.java
@@ -0,0 +1,106 @@
+package oracle.nosql.driver.ops;
+
+import oracle.nosql.driver.http.NoSQLHandleAsyncImpl;
+import oracle.nosql.driver.values.MapValue;
+
+import java.util.List;
+import java.util.concurrent.Flow;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Publisher for query pagination.
+ */
+public class QueryPublisher implements Flow.Publisher {
+
+ private final NoSQLHandleAsyncImpl handle;
+ private final QueryRequest request;
+ private final AtomicBoolean subscribed = new AtomicBoolean(false);
+
+ public QueryPublisher(NoSQLHandleAsyncImpl handle, QueryRequest request) {
+ this.handle = handle;
+ this.request = request;
+ }
+
+ @Override
+ public void subscribe(Flow.Subscriber super MapValue> subscriber) {
+ /* only allow one subscriber */
+ if (!subscribed.compareAndSet(false, true)) {
+ subscriber.onSubscribe(new Flow.Subscription() {
+ @Override public void request(long n) {}
+ @Override public void cancel() {}
+ });
+ subscriber.onError(new IllegalStateException(
+ "already subscribed"));
+ return;
+ }
+
+ subscriber.onSubscribe(new Flow.Subscription() {
+ private final AtomicBoolean cancelled = new AtomicBoolean(false);
+ private final AtomicLong demand = new AtomicLong(0);
+ private int currentIndex = 0;
+ private List currentBatch = List.of();
+ /* first run triggered? */
+ private boolean started = false;
+
+ @Override
+ public void request(long n) {
+ if (n <= 0 || cancelled.get()) return;
+ demand.addAndGet(n);
+ fetchNext();
+ }
+
+ @Override
+ public void cancel() {
+ cancelled.set(true);
+ /* close the query request */
+ request.close();
+ }
+
+ private void fetchNext() {
+ if (cancelled.get()) return;
+
+ /* If batch exhausted, fetch next Result */
+ if (currentIndex >= currentBatch.size()) {
+ if (started && request.isDone()) {
+ /* close the query request */
+ request.close();
+ subscriber.onComplete();
+ return;
+ }
+ started = true;
+ handle.query(request).whenComplete((result, error) -> {
+ if (cancelled.get()) return;
+ if (error != null) {
+ request.close();
+ subscriber.onError(error);
+ } else {
+ currentBatch = result.getResults();
+ currentIndex = 0;
+ fetchNext(); /* continue with new batch */
+ }
+ });
+ return;
+ }
+
+ /* Emit items while demand > 0 and we still have rows */
+ while (demand.get() > 0
+ && currentIndex < currentBatch.size()
+ && !cancelled.get()) {
+ subscriber.onNext(currentBatch.get(currentIndex++));
+ demand.decrementAndGet();
+ }
+
+ // If demand still positive but batch finished, fetch more
+ if (demand.get() > 0
+ && currentIndex >= currentBatch.size()
+ && !cancelled.get()) {
+ fetchNext();
+ }
+ }
+ });
+ }
+}
+
+
+
diff --git a/driver/src/main/java/oracle/nosql/driver/ops/SystemResult.java b/driver/src/main/java/oracle/nosql/driver/ops/SystemResult.java
index 45bf9361..4aac347f 100644
--- a/driver/src/main/java/oracle/nosql/driver/ops/SystemResult.java
+++ b/driver/src/main/java/oracle/nosql/driver/ops/SystemResult.java
@@ -10,6 +10,11 @@
import oracle.nosql.driver.NoSQLException;
import oracle.nosql.driver.NoSQLHandle;
import oracle.nosql.driver.RequestTimeoutException;
+import oracle.nosql.driver.http.NoSQLHandleAsyncImpl;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
/**
* On-premises only.
@@ -233,4 +238,64 @@ public void waitForCompletion(NoSQLHandle handle,
}
} while (!state.equals(State.COMPLETE));
}
+
+ public CompletableFuture waitForCompletionAsync(
+ NoSQLHandleAsyncImpl handle, int waitMillis, int delayMillis) {
+
+ if (state.equals(State.COMPLETE)) {
+ return CompletableFuture.completedFuture(null);
+ }
+
+ final int DELAY_MS = 500;
+
+ final int delayMS = (delayMillis != 0 ? delayMillis : DELAY_MS);
+ if (waitMillis < delayMillis) {
+ Throwable t = new IllegalArgumentException(
+ "Wait milliseconds must be a minimum of " +
+ DELAY_MS + " and greater than delay milliseconds");
+ return CompletableFuture.failedFuture(t);
+ }
+ final long startTime = System.currentTimeMillis();
+ SystemStatusRequest ds = new SystemStatusRequest()
+ .setOperationId(operationId);
+
+ final CompletableFuture resultFuture = new CompletableFuture<>();
+ final ScheduledExecutorService taskExecutor = handle.getTaskExecutor();
+
+ Runnable poll = new Runnable() {
+ @Override
+ public void run() {
+ final long curTime = System.currentTimeMillis();
+ if ((curTime - startTime) > waitMillis) {
+ Throwable t = new RequestTimeoutException(
+ waitMillis,
+ "Operation not completed within timeout: " +
+ statement);
+ resultFuture.completeExceptionally(t);
+ return;
+ }
+ handle.systemStatus(ds)
+ .whenComplete((res, ex) -> {
+ if (ex != null) {
+ resultFuture.completeExceptionally(ex);
+ return;
+ }
+ /* Update state */
+ resultString = res.resultString;
+ state = res.state;
+
+ if (state.equals(State.COMPLETE)) {
+ resultFuture.complete(null);
+ } else {
+ /* Schedule next poll */
+ taskExecutor.schedule(this, delayMS,
+ TimeUnit.MILLISECONDS);
+ }
+ });
+ }
+ };
+ /* Kick off the first poll immediately */
+ taskExecutor.execute(poll);
+ return resultFuture;
+ }
}
diff --git a/driver/src/main/java/oracle/nosql/driver/ops/TableResult.java b/driver/src/main/java/oracle/nosql/driver/ops/TableResult.java
index 428fec61..dcdfd829 100644
--- a/driver/src/main/java/oracle/nosql/driver/ops/TableResult.java
+++ b/driver/src/main/java/oracle/nosql/driver/ops/TableResult.java
@@ -12,8 +12,13 @@
import oracle.nosql.driver.NoSQLException;
import oracle.nosql.driver.NoSQLHandle;
import oracle.nosql.driver.RequestTimeoutException;
+import oracle.nosql.driver.http.NoSQLHandleAsyncImpl;
import oracle.nosql.driver.ops.TableLimits.CapacityMode;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
/**
* TableResult is returned from {@link NoSQLHandle#getTable} and
* {@link NoSQLHandle#tableRequest} operations. It encapsulates the
@@ -784,6 +789,83 @@ public void waitForCompletion(NoSQLHandle handle,
}
}
+ public CompletableFuture waitForCompletionAsync
+ (NoSQLHandleAsyncImpl handle, int waitMillis, int delayMillis) {
+
+ if (isTerminal()) {
+ return CompletableFuture.completedFuture(null);
+ }
+
+ if (operationId == null) {
+ Throwable t = new IllegalArgumentException(
+ "Operation state must not be null");
+ return CompletableFuture.failedFuture(t);
+ }
+
+ /* TODO: try to share code with waitForState? */
+ final int DELAY_MS = 500;
+
+ final int delayMS = (delayMillis != 0 ? delayMillis : DELAY_MS);
+ if (waitMillis < delayMillis) {
+ Throwable t = new IllegalArgumentException(
+ "Wait milliseconds must be a minimum of " +
+ DELAY_MS + " and greater than delay milliseconds");
+ return CompletableFuture.failedFuture(t);
+ }
+
+ final long startTime = System.currentTimeMillis();
+ final CompletableFuture resultFuture = new CompletableFuture<>();
+ final ScheduledExecutorService taskExecutor = handle.getTaskExecutor();
+
+ GetTableRequest getTable =
+ new GetTableRequest().setTableName(tableName).
+ setOperationId(operationId).setCompartment(
+ compartmentOrNamespace);
+
+ Runnable poll = new Runnable() {
+ @Override
+ public void run() {
+ long curTime = System.currentTimeMillis();
+ if ((curTime - startTime) > waitMillis) {
+ Throwable t = new RequestTimeoutException(
+ waitMillis,
+ "Operation not completed in expected time");
+ resultFuture.completeExceptionally(t);
+ return;
+ }
+ handle.getTable(getTable).whenComplete((res, ex) -> {
+ if (ex != null) {
+ resultFuture.completeExceptionally(ex);
+ return;
+ }
+ /*
+ * partial "copy" of possibly modified state. Don't modify
+ * operationId as that is what we are waiting to complete
+ */
+ state = res.getTableState();
+ limits = res.getTableLimits();
+ schema = res.getSchema();
+ matchETag = res.getMatchETag();
+ ddl = res.getDdl();
+ isFrozen = res.isFrozen();
+ isLocalReplicaInitialized = res.isLocalReplicaInitialized();
+ replicas = res.getReplicas();
+
+ if (isTerminal()) {
+ resultFuture.complete(null);
+ } else {
+ /* Schedule next poll */
+ taskExecutor.schedule(this, delayMS,
+ TimeUnit.MILLISECONDS);
+ }
+ });
+ }
+ };
+ /* Kick off the first poll immediately */
+ taskExecutor.execute(poll);
+ return resultFuture;
+ }
+
private boolean isTerminal() {
return state == State.ACTIVE || state == State.DROPPED;
}
diff --git a/driver/src/test/java/oracle/nosql/driver/httpclient/ConnectionPoolTest.java b/driver/src/test/java/oracle/nosql/driver/httpclient/ConnectionPoolTest.java
index e12b11a5..48295fb4 100644
--- a/driver/src/test/java/oracle/nosql/driver/httpclient/ConnectionPoolTest.java
+++ b/driver/src/test/java/oracle/nosql/driver/httpclient/ConnectionPoolTest.java
@@ -9,7 +9,6 @@
import static org.junit.Assert.assertEquals;
-import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Future;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -78,7 +77,7 @@ public void poolTest() throws Exception {
pool.setKeepAlive(new ConnectionPool.KeepAlive() {
@Override
- public CompletableFuture keepAlive(Channel ch) {
+ public boolean keepAlive(Channel ch) {
return client.doKeepAlive(ch);
}
});
@@ -177,7 +176,7 @@ public void testCloudTimeout() throws Exception {
pool.setKeepAlive(new ConnectionPool.KeepAlive() {
@Override
- public CompletableFuture keepAlive(Channel ch) {
+ public boolean keepAlive(Channel ch) {
return client.doKeepAlive(ch);
}
});
From 7311fe479865ba2b1a6dde6cae497f91e7b7c545 Mon Sep 17 00:00:00 2001
From: Akshay Sundarraj
Date: Sat, 11 Oct 2025 10:38:55 +0530
Subject: [PATCH 05/11] Async refactor changes-IV
- Fix review comments by Jin
Files:
M driver/src/main/java/oracle/nosql/driver/AuthorizationProvider.java
M driver/src/main/java/oracle/nosql/driver/NoSQLHandleFactory.java
M driver/src/main/java/oracle/nosql/driver/http/Client.java
M driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleAsyncImpl.java
M driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
M driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
M driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
M driver/src/main/java/oracle/nosql/driver/httpclient/RequestState.java
M driver/src/main/java/oracle/nosql/driver/httpclient/ResponseHandler.java
M driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java
M driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java
M driver/src/main/java/oracle/nosql/driver/ops/QueryPublisher.java
M driver/src/main/java/oracle/nosql/driver/ops/SystemResult.java
M driver/src/main/java/oracle/nosql/driver/ops/TableResult.java
M driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java
M driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
- Fix indentation, limit line width to 80, remove unused imports, add copyrights
M driver/src/main/java/oracle/nosql/driver/http/Client.java
- Remove duplicate entry of kvRequest.addRetryDelayMs() before retrying the request
M driver/src/main/java/oracle/nosql/driver/ops/SystemResult.java
M driver/src/main/java/oracle/nosql/driver/ops/TableResult.java
M driver/src/main/java/oracle/nosql/driver/RetryHandler.java
- Added javadocs for public APIs
M driver/src/main/java/oracle/nosql/driver/package-info.java
- Added TODO to include doc about Async client
---
.../nosql/driver/AuthorizationProvider.java | 6 +-
.../nosql/driver/NoSQLHandleFactory.java | 5 +-
.../oracle/nosql/driver/RetryHandler.java | 15 ++
.../java/oracle/nosql/driver/http/Client.java | 208 +++++++++---------
.../driver/http/NoSQLHandleAsyncImpl.java | 160 ++++++++------
.../nosql/driver/http/NoSQLHandleImpl.java | 3 -
.../driver/httpclient/ConnectionPool.java | 6 +-
.../nosql/driver/httpclient/HttpClient.java | 49 +++--
.../nosql/driver/httpclient/RequestState.java | 1 +
.../driver/httpclient/ResponseHandler.java | 1 +
.../nosql/driver/iam/SignatureProvider.java | 167 +++++++-------
.../driver/kv/StoreAccessTokenProvider.java | 3 +
.../nosql/driver/ops/QueryPublisher.java | 21 +-
.../oracle/nosql/driver/ops/SystemResult.java | 20 ++
.../oracle/nosql/driver/ops/TableResult.java | 28 +++
.../oracle/nosql/driver/package-info.java | 1 +
.../nosql/driver/util/ConcurrentUtil.java | 7 +
.../nosql/driver/util/HttpRequestUtil.java | 1 -
18 files changed, 404 insertions(+), 298 deletions(-)
diff --git a/driver/src/main/java/oracle/nosql/driver/AuthorizationProvider.java b/driver/src/main/java/oracle/nosql/driver/AuthorizationProvider.java
index 2cc7dfbd..d01730a1 100644
--- a/driver/src/main/java/oracle/nosql/driver/AuthorizationProvider.java
+++ b/driver/src/main/java/oracle/nosql/driver/AuthorizationProvider.java
@@ -104,9 +104,9 @@ public default void setRequiredHeaders(String authString,
* @param content the request content bytes
*/
default CompletableFuture setRequiredHeadersAsync(String authString,
- Request request,
- HttpHeaders headers,
- byte[] content) {
+ Request request,
+ HttpHeaders headers,
+ byte[] content) {
if (authString != null) {
headers.set(AUTHORIZATION, authString);
}
diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleFactory.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleFactory.java
index 2538d4aa..c3edfd8b 100644
--- a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleFactory.java
+++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleFactory.java
@@ -60,10 +60,11 @@ public static NoSQLHandle createNoSQLHandle(NoSQLHandleConfig config) {
*
* @see NoSQLHandleAsync#close
*/
- public static NoSQLHandleAsync createNoSQLHandleAsync(NoSQLHandleConfig config) {
+ public static NoSQLHandleAsync createNoSQLHandleAsync(
+ NoSQLHandleConfig config) {
requireNonNull(
config,
- "NoSQLHandleFactory.createNoSQLHandle: config cannot be null");
+ "NoSQLHandleFactory.createNoSQLHandleAsync: config cannot be null");
NoSQLHandleConfig configCopy = config.clone();
if (configCopy.getRetryHandler() == null) {
/*
diff --git a/driver/src/main/java/oracle/nosql/driver/RetryHandler.java b/driver/src/main/java/oracle/nosql/driver/RetryHandler.java
index c3b841b0..11842169 100644
--- a/driver/src/main/java/oracle/nosql/driver/RetryHandler.java
+++ b/driver/src/main/java/oracle/nosql/driver/RetryHandler.java
@@ -70,5 +70,20 @@ public interface RetryHandler {
*/
void delay(Request request, int numRetries, RetryableException re);
+ /**
+ * This method is called when a {@link RetryableException} is thrown and it
+ * is determined that the request will be retried based on the return value
+ * of {@link #doRetry}. It returns the number of milliseconds to delay
+ * before retrying the request.
+ *
+ * @param request the Request that has triggered the exception
+ *
+ * @param numRetries the number of retries that have occurred for the
+ * operation
+ *
+ * @param re the exception that was thrown
+ *
+ * @return Retry delay time in milliseconds
+ */
int delayTime(Request request, int numRetries, RetryableException re);
}
diff --git a/driver/src/main/java/oracle/nosql/driver/http/Client.java b/driver/src/main/java/oracle/nosql/driver/http/Client.java
index 52b054e9..88cefee4 100644
--- a/driver/src/main/java/oracle/nosql/driver/http/Client.java
+++ b/driver/src/main/java/oracle/nosql/driver/http/Client.java
@@ -117,7 +117,6 @@
import oracle.nosql.driver.util.ByteInputStream;
import oracle.nosql.driver.util.ConcurrentUtil;
import oracle.nosql.driver.util.HttpConstants;
-import oracle.nosql.driver.util.LogUtil;
import oracle.nosql.driver.util.NettyByteInputStream;
import oracle.nosql.driver.util.NettyByteOutputStream;
import oracle.nosql.driver.util.RateLimiterMap;
@@ -286,7 +285,7 @@ private static class RequestContext {
private volatile short serialVersionUsed;
private volatile short queryVersionUsed;
private volatile long latencyNanos;
- public volatile long networkLatency;
+ private volatile long networkLatency;
RequestContext(Request kvRequest, long startNanos, int timeoutMs,
Supplier nextIdSupplier, RateLimiter readLimiter,
@@ -358,7 +357,7 @@ public Client(Logger logger,
httpClient.configureProxy(httpConfig);
}
- authProvider= config.getAuthorizationProvider();
+ authProvider = config.getAuthorizationProvider();
if (authProvider == null) {
throw new IllegalArgumentException(
"Must configure AuthorizationProvider to use HttpClient");
@@ -644,21 +643,21 @@ private CompletableFuture executeWithRetry(RequestContext ctx) {
}
return handlePreRateLimit(ctx)
- .thenCompose( (Integer delay) -> getAuthString(ctx, authProvider))
- .thenCompose((String authString) -> createRequest(ctx, authString))
- .thenCompose((FullHttpRequest request) -> submitRequest(ctx,request))
- .thenApply((FullHttpResponse response) -> handleResponse(ctx, response))
- .thenApply((Result result) -> handleResult(ctx, result))
- .thenCompose((Result result) -> handlePostRateLimit(ctx, result))
- .handle((Result result, Throwable err) -> {
- /* Handle error and retry */
- if (err != null) {
- return handleError(ctx, err);
- } else {
- return CompletableFuture.completedFuture(result);
- }
- })
- .thenCompose(Function.identity());
+ .thenCompose((Integer delay) -> getAuthString(ctx, authProvider))
+ .thenCompose((String authString) -> createRequest(ctx, authString))
+ .thenCompose((FullHttpRequest request) -> submitRequest(ctx, request))
+ .thenApply((FullHttpResponse response) -> handleResponse(ctx, response))
+ .thenApply((Result result) -> handleResult(ctx, result))
+ .thenCompose((Result result) -> handlePostRateLimit(ctx, result))
+ .handle((Result result, Throwable err) -> {
+ /* Handle error and retry */
+ if (err != null) {
+ return handleError(ctx, err);
+ } else {
+ return CompletableFuture.completedFuture(result);
+ }
+ })
+ .thenCompose(Function.identity());
}
private CompletableFuture handlePreRateLimit(RequestContext ctx) {
@@ -705,7 +704,8 @@ private CompletableFuture getAuthString(RequestContext ctx,
.thenApply(authString -> {
/* Check whether timed out while acquiring the auth token */
if (timeoutRequest(kvRequest.getStartNanos(),
- kvRequest.getTimeoutInternal(), null)) {
+ kvRequest.getTimeoutInternal(),
+ null /* exception */)) {
TimeoutException ex = new TimeoutException(
"timed out during auth token acquisition");
throw new CompletionException(ex);
@@ -805,18 +805,18 @@ private CompletableFuture createRequest(RequestContext ctx,
* with content
*/
byte[] content = signContent ? getBodyBytes(buffer) : null;
- return authProvider.
- setRequiredHeadersAsync(authString, kvRequest, headers, content)
- .thenApply(n -> {
- String namespace = kvRequest.getNamespace();
- if (namespace == null) {
- namespace = config.getDefaultNamespace();
- }
- if (namespace != null) {
- headers.add(REQUEST_NAMESPACE_HEADER, namespace);
- }
- return request;
- });
+ return authProvider.setRequiredHeadersAsync(authString, kvRequest,
+ headers, content)
+ .thenApply(n -> {
+ String namespace = kvRequest.getNamespace();
+ if (namespace == null) {
+ namespace = config.getDefaultNamespace();
+ }
+ if (namespace != null) {
+ headers.add(REQUEST_NAMESPACE_HEADER, namespace);
+ }
+ return request;
+ });
} catch (Throwable e) {
/* Release the buffer on error */
if (buffer != null) {
@@ -829,8 +829,9 @@ private CompletableFuture createRequest(RequestContext ctx,
/**
* Send the HTTP request to server and get the response back.
*/
- private CompletableFuture submitRequest(RequestContext ctx,
- HttpRequest request) {
+ private CompletableFuture submitRequest(
+ RequestContext ctx, HttpRequest request) {
+
final Request kvRequest = ctx.kvRequest;
if (isLoggable(logger, Level.FINE) && !kvRequest.getIsRefresh()) {
logTrace(logger, "Request: " + ctx.requestClass +
@@ -840,10 +841,10 @@ private CompletableFuture submitRequest(RequestContext ctx,
int timeoutMs = getIterationTimeoutMs(ctx.timeoutMs, ctx.startNanos);
return httpClient.runRequest(request, timeoutMs)
- .whenComplete((res, err) -> {
- ctx.networkLatency =
- (System.nanoTime() - ctx.latencyNanos) / 1_000_000;
- });
+ .whenComplete((res, err) -> {
+ ctx.networkLatency =
+ (System.nanoTime() - ctx.latencyNanos) / 1_000_000;
+ });
}
/**
@@ -1172,6 +1173,9 @@ private CompletableFuture handleIOError(RequestContext ctx,
String name = ex.getClass().getName();
logFine(logger, "Client execution IOException, name: " +
name + ", message: " + ex.getMessage());
+ /* Retry only 10 times. We shouldn't be retrying till timeout occurs
+ * as this can consume a lot of async resources.
+ */
if (kvRequest.getNumRetries() > 10) {
return failRequest(ctx, ex);
}
@@ -1231,8 +1235,8 @@ private CompletableFuture createDelayFuture(int delayMs) {
return delayFuture;
}
- private CompletableFuture scheduleRetry(RequestContext ctx,
- int delayMs) {
+ private CompletableFuture scheduleRetry(RequestContext ctx,
+ int delayMs) {
//TODO check for overall timeout before schedule
CompletableFuture retryFuture = new CompletableFuture<>();
taskExecutor.schedule(() -> {
@@ -1240,7 +1244,6 @@ private CompletableFuture scheduleRetry(RequestContext ctx,
ctx.requestId = String.valueOf(ctx.nextIdSupplier.get());
executeWithRetry(ctx)
.whenComplete((res, e) -> {
- ctx.kvRequest.addRetryDelayMs(delayMs);
if (e != null) {
retryFuture.completeExceptionally(e);
} else {
@@ -1647,18 +1650,18 @@ private void setTableNeedsRefresh(String tableName, boolean needsRefresh) {
private void backgroundUpdateLimiters(String tableName,
String compartmentId) {
ConcurrentUtil.synchronizedCall(this.lock, () -> {
- if (tableNeedsRefresh(tableName) == false) {
- return;
- }
- setTableNeedsRefresh(tableName, false);
+ if (tableNeedsRefresh(tableName) == false) {
+ return;
+ }
+ setTableNeedsRefresh(tableName, false);
- try {
- threadPool.execute(() -> {
- updateTableLimiters(tableName, compartmentId);
- });
- } catch (RejectedExecutionException e) {
- setTableNeedsRefresh(tableName, true);
- }
+ try {
+ threadPool.execute(() -> {
+ updateTableLimiters(tableName, compartmentId);
+ });
+ } catch (RejectedExecutionException e) {
+ setTableNeedsRefresh(tableName, true);
+ }
});
}
@@ -1835,18 +1838,18 @@ StatsControl getStatsControl() {
*/
private boolean decrementSerialVersion(short versionUsed) {
return ConcurrentUtil.synchronizedCall(this.lock, () -> {
- if (serialVersion != versionUsed) {
- return true;
- }
- if (serialVersion == V4) {
- serialVersion = V3;
- return true;
- }
- if (serialVersion == V3) {
- serialVersion = V2;
- return true;
- }
- return false;
+ if (serialVersion != versionUsed) {
+ return true;
+ }
+ if (serialVersion == V4) {
+ serialVersion = V3;
+ return true;
+ }
+ if (serialVersion == V3) {
+ serialVersion = V2;
+ return true;
+ }
+ return false;
});
}
@@ -1859,16 +1862,16 @@ private boolean decrementSerialVersion(short versionUsed) {
*/
private boolean decrementQueryVersion(short versionUsed) {
return ConcurrentUtil.synchronizedCall(this.lock, () -> {
- if (queryVersion != versionUsed) {
- return true;
- }
+ if (queryVersion != versionUsed) {
+ return true;
+ }
- if (queryVersion == QueryDriver.QUERY_V3) {
- return false;
- }
+ if (queryVersion == QueryDriver.QUERY_V3) {
+ return false;
+ }
- --queryVersion;
- return true;
+ --queryVersion;
+ return true;
});
}
@@ -2005,26 +2008,26 @@ private boolean stringsEqualOrNull(String s1, String s2) {
*/
private void addRequestToRefreshList(Request request) {
ConcurrentUtil.synchronizedCall(this.lock, () -> {
- logFine(logger, "Adding table to request list: " +
- request.getCompartment() + ":" + request.getTableName());
- PutRequest pr =
- new PutRequest().setTableName(request.getTableName());
- pr.setCompartmentInternal(request.getCompartment());
- pr.setValue(badValue);
- pr.setIsRefresh(true);
- authRefreshRequests.add(pr);
- GetRequest gr =
- new GetRequest().setTableName(request.getTableName());
- gr.setCompartmentInternal(request.getCompartment());
- gr.setKey(badValue);
- gr.setIsRefresh(true);
- authRefreshRequests.add(gr);
- DeleteRequest dr =
- new DeleteRequest().setTableName(request.getTableName());
- dr.setCompartmentInternal(request.getCompartment());
- dr.setKey(badValue);
- dr.setIsRefresh(true);
- authRefreshRequests.add(dr);
+ logFine(logger, "Adding table to request list: " +
+ request.getCompartment() + ":" + request.getTableName());
+ PutRequest pr =
+ new PutRequest().setTableName(request.getTableName());
+ pr.setCompartmentInternal(request.getCompartment());
+ pr.setValue(badValue);
+ pr.setIsRefresh(true);
+ authRefreshRequests.add(pr);
+ GetRequest gr =
+ new GetRequest().setTableName(request.getTableName());
+ gr.setCompartmentInternal(request.getCompartment());
+ gr.setKey(badValue);
+ gr.setIsRefresh(true);
+ authRefreshRequests.add(gr);
+ DeleteRequest dr =
+ new DeleteRequest().setTableName(request.getTableName());
+ dr.setCompartmentInternal(request.getCompartment());
+ dr.setKey(badValue);
+ dr.setIsRefresh(true);
+ authRefreshRequests.add(dr);
});
}
@@ -2034,10 +2037,10 @@ private void addRequestToRefreshList(Request request) {
*/
public void oneTimeMessage(String msg) {
ConcurrentUtil.synchronizedCall(this.lock, () -> {
- if (oneTimeMessages.add(msg) == false) {
- return;
- }
- logWarning(logger, msg);
+ if (oneTimeMessages.add(msg) == false) {
+ return;
+ }
+ logWarning(logger, msg);
});
}
@@ -2148,16 +2151,15 @@ private int getTopoSeqNum() {
}
private void setTopology(TopologyInfo topo) {
-
ConcurrentUtil.synchronizedCall(this.lock, () -> {
- if (topo == null) {
- return;
- }
+ if (topo == null) {
+ return;
+ }
- if (topology == null || topology.getSeqNum() < topo.getSeqNum()) {
- topology = topo;
- trace("New topology: " + topo, 1);
- }
+ if (topology == null || topology.getSeqNum() < topo.getSeqNum()) {
+ topology = topo;
+ trace("New topology: " + topo, 1);
+ }
});
}
diff --git a/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleAsyncImpl.java b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleAsyncImpl.java
index a985939b..d792027f 100644
--- a/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleAsyncImpl.java
+++ b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleAsyncImpl.java
@@ -1,3 +1,10 @@
+/*-
+ * Copyright (c) 2011, 2025 Oracle and/or its affiliates. All rights reserved.
+ *
+ * Licensed under the Universal Permissive License v 1.0 as shown at
+ * https://oss.oracle.com/licenses/upl/
+ */
+
package oracle.nosql.driver.http;
import io.netty.handler.ssl.SslContextBuilder;
@@ -60,7 +67,6 @@
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
import java.util.logging.Logger;
public class NoSQLHandleAsyncImpl implements NoSQLHandleAsync {
@@ -89,9 +95,10 @@ public NoSQLHandleAsyncImpl(NoSQLHandleConfig config) {
private final AtomicInteger threadNumber = new AtomicInteger(1);
@Override
public Thread newThread(Runnable r) {
- final Thread t = Executors.defaultThreadFactory().newThread(r);
+ final Thread t = Executors.defaultThreadFactory()
+ .newThread(r);
t.setName(String.format("nosql-task-executor-%s",
- threadNumber.getAndIncrement()));
+ threadNumber.getAndIncrement()));
t.setDaemon(true);
t.setUncaughtExceptionHandler((thread, error) -> {
if (ConcurrentUtil.unwrapCompletionException(error)
@@ -221,12 +228,14 @@ public CompletableFuture put(PutRequest request) {
}
@Override
- public CompletableFuture writeMultiple(WriteMultipleRequest request) {
+ public CompletableFuture writeMultiple(
+ WriteMultipleRequest request) {
return executeASync(request);
}
@Override
- public CompletableFuture multiDelete(MultiDeleteRequest request) {
+ public CompletableFuture multiDelete(
+ MultiDeleteRequest request) {
return executeASync(request);
}
@@ -280,42 +289,50 @@ public CompletableFuture getTable(GetTableRequest request) {
}
@Override
- public CompletableFuture systemRequest(SystemRequest request) {
+ public CompletableFuture systemRequest(
+ SystemRequest request) {
return executeASync(request);
}
@Override
- public CompletableFuture systemStatus(SystemStatusRequest request) {
+ public CompletableFuture systemStatus(
+ SystemStatusRequest request) {
return executeASync(request);
}
@Override
- public CompletableFuture getTableUsage(TableUsageRequest request) {
+ public CompletableFuture getTableUsage(
+ TableUsageRequest request) {
return executeASync(request);
}
@Override
- public CompletableFuture listTables(ListTablesRequest request) {
+ public CompletableFuture listTables(
+ ListTablesRequest request) {
return executeASync(request);
}
@Override
- public CompletableFuture getIndexes(GetIndexesRequest request) {
+ public CompletableFuture getIndexes(
+ GetIndexesRequest request) {
return executeASync(request);
}
@Override
- public CompletableFuture addReplica(AddReplicaRequest request) {
+ public CompletableFuture addReplica(
+ AddReplicaRequest request) {
return executeASync(request);
}
@Override
- public CompletableFuture dropReplica(DropReplicaRequest request) {
+ public CompletableFuture dropReplica(
+ DropReplicaRequest request) {
return executeASync(request);
}
@Override
- public CompletableFuture getReplicaStats(ReplicaStatsRequest request) {
+ public CompletableFuture getReplicaStats(
+ ReplicaStatsRequest request) {
return executeASync(request);
}
@@ -330,80 +347,81 @@ public void close() {
@Override
public CompletableFuture listNamespaces() {
return doSystemRequest("show as json namespaces")
- .thenApply((SystemResult dres )-> {
- String jsonResult = dres.getResultString();
- if (jsonResult == null) {
- return null;
- }
- MapValue root = JsonUtils.createValueFromJson(jsonResult, null).asMap();
+ .thenApply((SystemResult dres )-> {
+ String jsonResult = dres.getResultString();
+ if (jsonResult == null) {
+ return null;
+ }
+ MapValue root = JsonUtils.createValueFromJson(jsonResult, null)
+ .asMap();
- FieldValue namespaces = root.get("namespaces");
- if (namespaces == null) {
- return null;
- }
+ FieldValue namespaces = root.get("namespaces");
+ if (namespaces == null) {
+ return null;
+ }
- ArrayList results = new ArrayList(
+ ArrayList results = new ArrayList(
namespaces.asArray().size());
- for (FieldValue val : namespaces.asArray()) {
- results.add(val.getString());
- }
- return results.toArray(new String[0]);
- });
+ for (FieldValue val : namespaces.asArray()) {
+ results.add(val.getString());
+ }
+ return results.toArray(new String[0]);
+ });
}
@Override
public CompletableFuture listUsers() {
return doSystemRequest("show as json users")
- .thenApply((SystemResult dres) -> {
- String jsonResult = dres.getResultString();
- if (jsonResult == null) {
- return null;
- }
+ .thenApply((SystemResult dres) -> {
+ String jsonResult = dres.getResultString();
+ if (jsonResult == null) {
+ return null;
+ }
- MapValue root = JsonUtils.createValueFromJson(
- jsonResult, null).asMap();
+ MapValue root = JsonUtils.createValueFromJson(
+ jsonResult, null).asMap();
- FieldValue users = root.get("users");
- if (users == null) {
- return null;
- }
+ FieldValue users = root.get("users");
+ if (users == null) {
+ return null;
+ }
- ArrayList results = new ArrayList(
- users.asArray().size());
+ ArrayList results = new ArrayList(
+ users.asArray().size());
- for (FieldValue val : users.asArray()) {
- String id = val.asMap().getString("id");
- String name = val.asMap().getString("name");
- results.add(new UserInfo(id, name));
- }
- return results.toArray(new UserInfo[0]);
- });
+ for (FieldValue val : users.asArray()) {
+ String id = val.asMap().getString("id");
+ String name = val.asMap().getString("name");
+ results.add(new UserInfo(id, name));
+ }
+ return results.toArray(new UserInfo[0]);
+ });
}
@Override
public CompletableFuture listRoles() {
return doSystemRequest("show as json roles")
- .thenApply((SystemResult dres) -> {
- String jsonResult = dres.getResultString();
- if (jsonResult == null) {
- return null;
- }
- MapValue root = JsonUtils.createValueFromJson(
- jsonResult, null).asMap();
+ .thenApply((SystemResult dres) -> {
+ String jsonResult = dres.getResultString();
+ if (jsonResult == null) {
+ return null;
+ }
+ MapValue root = JsonUtils.createValueFromJson(
+ jsonResult, null).asMap();
- FieldValue roles = root.get("roles");
- if (roles == null) {
- return null;
- }
+ FieldValue roles = root.get("roles");
+ if (roles == null) {
+ return null;
+ }
- ArrayList results = new ArrayList(
- roles.asArray().size());
- for (FieldValue val : roles.asArray()) {
- String role = val.asMap().getString("name");
- results.add(role);
- }
- return results.toArray(new String[0]);
- });
+ ArrayList results = new ArrayList(
+ roles.asArray().size());
+ for (FieldValue val : roles.asArray()) {
+ String role = val.asMap().getString("name");
+ results.add(role);
+ }
+ return results.toArray(new String[0]);
+ });
}
/**
@@ -430,10 +448,9 @@ public CompletableFuture doSystemRequest(String statement,
checkClient();
SystemRequest dreq =
new SystemRequest().setStatement(statement.toCharArray());
- return systemRequest(dreq).thenCompose(
- (SystemResult dres) ->
+ return systemRequest(dreq).thenCompose((SystemResult dres) ->
dres.waitForCompletionAsync(this, timeoutMs, pollIntervalMs)
- .thenApply(v -> dres));
+ .thenApply(v -> dres));
}
@Override
@@ -472,7 +489,6 @@ public void setDefaultNamespace(String ns) {
client.setDefaultNamespace(ns);
}
-
@SuppressWarnings("unchecked")
CompletableFuture executeASync(Request request) {
checkClient();
diff --git a/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
index c284b21f..edc4a706 100644
--- a/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
+++ b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java
@@ -7,8 +7,6 @@
package oracle.nosql.driver.http;
-import java.util.concurrent.CompletableFuture;
-
import oracle.nosql.driver.NoSQLHandle;
import oracle.nosql.driver.NoSQLHandleConfig;
import oracle.nosql.driver.StatsControl;
@@ -117,7 +115,6 @@ public TableResult getTable(GetTableRequest request) {
@Override
public SystemResult systemRequest(SystemRequest request) {
- checkClient();
return executeSync(request);
}
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java b/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
index 96d444fb..beafebbf 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
@@ -226,10 +226,10 @@ final Future acquire(final Promise promise) {
Bootstrap bs = bootstrap.clone();
ChannelFuture fut = bs.connect();
if (fut.isDone()) {
- notifyOnConnect(fut,promise);
+ notifyOnConnect(fut, promise);
} else {
fut.addListener((ChannelFutureListener) future ->
- notifyOnConnect(future,promise));
+ notifyOnConnect(future, promise));
}
} else {
/*
@@ -327,7 +327,7 @@ private void notifyOnConnect(ChannelFuture future,
}
private void checkChannel(final Channel channel,
- final Promise promise) {
+ final Promise promise) {
try {
/*
* If channel isn't healthy close it. It's been removed from
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
index 8832ffa7..875bfe60 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
@@ -58,26 +58,32 @@
* use by requests.
*
*
- * Using the client to send request and get a synchronous response. The
+ * Using the client to send request. The
* request must be an instance of HttpRequest:
+ *
+ * For synchronous calls, wait for a response:
+ *
+ * response.join() or response.get();
+ *
+ *
+ *
+ * For asynchronous calls, consume the response future.
+ *
+ *
+ * If there was a problem with the send or receive, future completes
+ * with exception.
+ *
+ *
*
- * 1. Get a Channel.
- * Channel channel = client.getChannel(timeoutMs);
- * 2. Create a ResponseHandler to handle a response.
- * ResponseHandler rhandler = new ResponseHandler(client, logger, channel);
- * Note that the ResponseHandler will release the Channel.
- * 3. Call runRequest to send the request.
- * client.runRequest(request, rhandler, channel);
- * 4. For synchronous calls, wait for a response:
- * rhandler.await(timeoutMs);
- * If there was a problem with the send or receive this call will throw a
- * Throwable with the relevant information. If successful the response
- * information can be extracted from the ResponseHandler.
- * ResponseHandler instances must be closed using the close() method. This
- * releases resources associated with the request/response dialog such as the
- * channel and the HttpResponse itself.
- *
- * TODO: asynchronous handler
*/
public class HttpClient {
@@ -424,9 +430,10 @@ public void removeChannel(Channel channel) {
*/
public CompletableFuture runRequest(HttpRequest request,
int timeoutMs) {
-
- CompletableFuture responseFuture = new CompletableFuture<>();
- long deadlineNs = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(timeoutMs);
+ CompletableFuture responseFuture =
+ new CompletableFuture<>();
+ long deadlineNs = System.nanoTime() +
+ TimeUnit.MILLISECONDS.toNanos(timeoutMs);
pool.acquire().addListener((FutureListener) channelFuture -> {
if (channelFuture.isSuccess()) {
Channel channel = channelFuture.getNow();
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/RequestState.java b/driver/src/main/java/oracle/nosql/driver/httpclient/RequestState.java
index 185399d5..d84a507a 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/RequestState.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/RequestState.java
@@ -28,6 +28,7 @@
* desired here it can be added using a CompositeByteBuf and calls to add
* content incrementally.
*/
+/* TODO: this class is no longer used. Remove this once testing is completed */
class RequestState {
private final ResponseHandler handler;
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/ResponseHandler.java b/driver/src/main/java/oracle/nosql/driver/httpclient/ResponseHandler.java
index 12b38216..7b4a9d34 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/ResponseHandler.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/ResponseHandler.java
@@ -34,6 +34,7 @@
*
* TODO: examples of both sync and async usage
*/
+/* TODO: this class is no longer used. Remove this once testing is completed */
public class ResponseHandler implements Closeable {
private HttpResponseStatus status;
diff --git a/driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java b/driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java
index 99c0733f..81817d36 100644
--- a/driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java
+++ b/driver/src/main/java/oracle/nosql/driver/iam/SignatureProvider.java
@@ -920,7 +920,7 @@ public CompletableFuture setRequiredHeadersAsync(String authString,
/*
* If request doesn't has compartment id, set the tenant id
* as the default compartment, which is the root compartment
- * in IAM if using user principal. If using an instance
+ * in IAM if using user principal. If using an instance
* principal this value is null.
*/
compartment = getTenantOCID();
@@ -941,8 +941,8 @@ public CompletableFuture setRequiredHeadersAsync(String authString,
@Override
public void flushCache() {
ConcurrentUtil.synchronizedCall(lock, () -> {
- currentSigDetails = null;
- refreshSigDetails = null;
+ currentSigDetails = null;
+ refreshSigDetails = null;
});
}
@@ -1009,10 +1009,9 @@ public SignatureProvider prepare(NoSQLHandleConfig config) {
/* creates and caches a signature as warm-up */
getSignatureDetailsInternal(false, /* isRefresh */
- null, /* request */
- null, /* headers */
- null /* content */
- );
+ null, /* request */
+ null, /* headers */
+ null /* content */);
return this;
}
@@ -1101,7 +1100,7 @@ private void logMessage(Level level, String msg) {
* Do we need a separate executor?
*/
return CompletableFuture.supplyAsync(() ->
- getSignatureDetailsInternal(false, request, headers, content));
+ getSignatureDetailsInternal(false, request, headers, content));
}
private CompletableFuture
@@ -1111,9 +1110,9 @@ private void logMessage(Level level, String msg) {
*/
return CompletableFuture.supplyAsync(() ->
getSignatureDetailsInternal(isRefresh,
- null /* request */,
- null /* headers */,
- null /* content */)
+ null /* request */,
+ null /* headers */,
+ null /* content */)
);
}
@@ -1124,77 +1123,78 @@ private void logMessage(Level level, String msg) {
HttpHeaders headers,
byte[] content) {
return ConcurrentUtil.synchronizedCall(lock, () -> {
- /*
- * add one minute to the current time, so that any caching is
- * effective over a more valid time period.
- */
- long nowPlus = System.currentTimeMillis() + 60_000L;
- String date = createFormatter().format(new Date(nowPlus));
- String keyId = provider.getKeyId();
-
- /*
- * Security token based providers may refresh the security token
- * and associated private key in above getKeyId() method, reload
- * private key to PrivateKeyProvider to avoid a mismatch, which
- * will create an invalid signature, cause authentication error.
- */
- if (provider instanceof SecurityTokenBasedProvider) {
- privateKeyProvider.reload(provider.getPrivateKey(),
- provider.getPassphraseCharacters());
- }
- String signature;
- try {
- signature = sign(signingContent(date, request, headers, content),
- privateKeyProvider.getKey());
- } catch (Exception e) {
- logMessage(Level.SEVERE, "Error signing request " + e.getMessage());
- return null;
- }
+ /*
+ * add one minute to the current time, so that any caching is
+ * effective over a more valid time period.
+ */
+ long nowPlus = System.currentTimeMillis() + 60_000L;
+ String date = createFormatter().format(new Date(nowPlus));
+ String keyId = provider.getKeyId();
- String token = getDelegationToken(request);
- String signingHeader;
- if (content != null) {
- signingHeader = (token == null)
- ? SIGNING_HEADERS_WITH_CONTENT :
- SIGNING_HEADERS_WITH_CONTENT_OBO;
- } else {
- signingHeader = (token == null)
- ? SIGNING_HEADERS : SIGNING_HEADERS_WITH_OBO;
- }
+ /*
+ * Security token based providers may refresh the security token
+ * and associated private key in above getKeyId() method, reload
+ * private key to PrivateKeyProvider to avoid a mismatch, which
+ * will create an invalid signature, cause authentication error.
+ */
+ if (provider instanceof SecurityTokenBasedProvider) {
+ privateKeyProvider.reload(provider.getPrivateKey(),
+ provider.getPassphraseCharacters());
+ }
+ String signature;
+ try {
+ signature = sign(signingContent(date, request, headers, content),
+ privateKeyProvider.getKey());
+ } catch (Exception e) {
+ logMessage(Level.SEVERE, "Error signing request " +
+ e.getMessage());
+ return null;
+ }
- String sigHeader = String.format(SIGNATURE_HEADER_FORMAT,
- signingHeader,
- keyId,
- RSA,
- signature,
- SINGATURE_VERSION);
- SignatureDetails sigDetails = new SignatureDetails(sigHeader, date);
+ String token = getDelegationToken(request);
+ String signingHeader;
+ if (content != null) {
+ signingHeader = (token == null)
+ ? SIGNING_HEADERS_WITH_CONTENT :
+ SIGNING_HEADERS_WITH_CONTENT_OBO;
+ } else {
+ signingHeader = (token == null)
+ ? SIGNING_HEADERS : SIGNING_HEADERS_WITH_OBO;
+ }
- /*
- * Don't cache the signature generated with content, which
- * needs to be associated with its request
- */
- if (content != null) {
- return sigDetails;
- }
+ String sigHeader = String.format(SIGNATURE_HEADER_FORMAT,
+ signingHeader,
+ keyId,
+ RSA,
+ signature,
+ SINGATURE_VERSION);
+ SignatureDetails sigDetails = new SignatureDetails(sigHeader, date);
- if (!isRefresh) {
- /*
- * if this is not a refresh, use the normal key and schedule a
- * refresh
- */
- currentSigDetails = sigDetails;
- scheduleRefresh();
- } else {
/*
- * If this is a refresh put the object in a temporary key.
- * The caller (the refresh task) will:
- * 1. perform callbacks if needed and when done,
- * 2. move the object to the normal key and schedule a refresh
+ * Don't cache the signature generated with content, which
+ * needs to be associated with its request
*/
- refreshSigDetails = sigDetails;
- }
- return sigDetails;
+ if (content != null) {
+ return sigDetails;
+ }
+
+ if (!isRefresh) {
+ /*
+ * if this is not a refresh, use the normal key and schedule a
+ * refresh
+ */
+ currentSigDetails = sigDetails;
+ scheduleRefresh();
+ } else {
+ /*
+ * If this is a refresh put the object in a temporary key.
+ * The caller (the refresh task) will:
+ * 1. perform callbacks if needed and when done,
+ * 2. move the object to the normal key and schedule a refresh
+ */
+ refreshSigDetails = sigDetails;
+ }
+ return sigDetails;
});
}
@@ -1213,10 +1213,11 @@ private String getDelegationToken(Request req) {
private void setRefreshKey() {
ConcurrentUtil.synchronizedCall(lock, () -> {
- if (refreshSigDetails != null) {
- currentSigDetails = refreshSigDetails;
- refreshSigDetails = null;
- }});
+ if (refreshSigDetails != null) {
+ currentSigDetails = refreshSigDetails;
+ refreshSigDetails = null;
+ }
+ });
}
private String signingContent(String date,
@@ -1313,9 +1314,9 @@ public void run() {
do {
try {
getSignatureDetailsInternal(true, /* isRefresh */
- null /* request */,
- null /* headers */,
- null /* content */);
+ null /* request */,
+ null /* headers */,
+ null /* content */);
handleRefreshCallback(refreshAheadMs);
return;
} catch (SecurityInfoNotReadyException se) {
diff --git a/driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java b/driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java
index 12bacea5..596098eb 100644
--- a/driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java
+++ b/driver/src/main/java/oracle/nosql/driver/kv/StoreAccessTokenProvider.java
@@ -293,6 +293,9 @@ public String getAuthorizationString(Request request) {
return ConcurrentUtil.awaitFuture(getAuthorizationStringAsync(request));
}
+ /**
+ * @hidden
+ */
@Override
public CompletableFuture
getAuthorizationStringAsync(Request request) {
diff --git a/driver/src/main/java/oracle/nosql/driver/ops/QueryPublisher.java b/driver/src/main/java/oracle/nosql/driver/ops/QueryPublisher.java
index c5abe85e..f3fe752a 100644
--- a/driver/src/main/java/oracle/nosql/driver/ops/QueryPublisher.java
+++ b/driver/src/main/java/oracle/nosql/driver/ops/QueryPublisher.java
@@ -1,3 +1,10 @@
+/*-
+ * Copyright (c) 2011, 2025 Oracle and/or its affiliates. All rights reserved.
+ *
+ * Licensed under the Universal Permissive License v 1.0 as shown at
+ * https://oss.oracle.com/licenses/upl/
+ */
+
package oracle.nosql.driver.ops;
import oracle.nosql.driver.http.NoSQLHandleAsyncImpl;
@@ -27,11 +34,14 @@ public void subscribe(Flow.Subscriber super MapValue> subscriber) {
/* only allow one subscriber */
if (!subscribed.compareAndSet(false, true)) {
subscriber.onSubscribe(new Flow.Subscription() {
- @Override public void request(long n) {}
- @Override public void cancel() {}
+ @Override
+ public void request(long n) {
+ }
+ @Override
+ public void cancel() {
+ }
});
- subscriber.onError(new IllegalStateException(
- "already subscribed"));
+ subscriber.onError(new IllegalStateException("already subscribed"));
return;
}
@@ -101,6 +111,3 @@ private void fetchNext() {
});
}
}
-
-
-
diff --git a/driver/src/main/java/oracle/nosql/driver/ops/SystemResult.java b/driver/src/main/java/oracle/nosql/driver/ops/SystemResult.java
index 4aac347f..ea332c14 100644
--- a/driver/src/main/java/oracle/nosql/driver/ops/SystemResult.java
+++ b/driver/src/main/java/oracle/nosql/driver/ops/SystemResult.java
@@ -239,6 +239,26 @@ public void waitForCompletion(NoSQLHandle handle,
} while (!state.equals(State.COMPLETE));
}
+ /**
+ * Asynchronously waits for the operation to be complete.
+ * This is a polling style wait that delays for the specified number of
+ * milliseconds between each polling operation.
+ *
+ * This instance is modified with any changes in state.
+ *
+ * @param handle the Async NoSQLHandle to use
+ * @param waitMillis the total amount of time to wait, in milliseconds. This
+ * value must be non-zero and greater than delayMillis
+ * @param delayMillis the amount of time to wait between polling attempts,
+ * in milliseconds. If 0 it will default to 500.
+ *
+ * @return Returns a {@link CompletableFuture} which completes
+ * successfully when operation is completed within waitMillis otherwise
+ * completes exceptionally with {@link IllegalArgumentException}
+ * if the operation times out or the parameters are not valid.
+ * Completes exceptionally with {@link NoSQLException}
+ * if the operation id used is unknown or the operation has failed.
+ */
public CompletableFuture waitForCompletionAsync(
NoSQLHandleAsyncImpl handle, int waitMillis, int delayMillis) {
diff --git a/driver/src/main/java/oracle/nosql/driver/ops/TableResult.java b/driver/src/main/java/oracle/nosql/driver/ops/TableResult.java
index dcdfd829..78fdf6b2 100644
--- a/driver/src/main/java/oracle/nosql/driver/ops/TableResult.java
+++ b/driver/src/main/java/oracle/nosql/driver/ops/TableResult.java
@@ -789,6 +789,34 @@ public void waitForCompletion(NoSQLHandle handle,
}
}
+ /**
+ * Asynchronously waits for a table operation to complete. Table operations
+ * are asynchronous. This is a polling style wait that delays for
+ * the specified number of milliseconds between each polling operation.
+ * The returned future completes when the table reaches a
+ * terminal state,
+ * which is either {@link State#ACTIVE} or {@link State#DROPPED}.
+ *
+ * This instance must be the return value of a previous
+ * {@link NoSQLHandle#tableRequest} and contain a non-null operation id
+ * representing the in-progress operation unless the operation has
+ * already completed.
+ *
+ * This instance is modified with any change in table state or metadata.
+ *
+ * @param handle the Async NoSQLHandle to use
+ * @param waitMillis the total amount of time to wait, in milliseconds. This
+ * value must be non-zero and greater than delayMillis
+ * @param delayMillis the amount of time to wait between polling attempts,
+ * in milliseconds. If 0 it will default to 500.
+ *
+ * @return Returns a {@link CompletableFuture} which completes
+ * successfully when operation is completed within waitMillis otherwise
+ * completes exceptionally with {@link IllegalArgumentException}
+ * if the parameters are not valid.
+ * Completes exceptionally with {@link RequestTimeoutException}
+ * if the operation times out.
+ */
public CompletableFuture waitForCompletionAsync
(NoSQLHandleAsyncImpl handle, int waitMillis, int delayMillis) {
diff --git a/driver/src/main/java/oracle/nosql/driver/package-info.java b/driver/src/main/java/oracle/nosql/driver/package-info.java
index 27557cb0..a3dff30b 100644
--- a/driver/src/main/java/oracle/nosql/driver/package-info.java
+++ b/driver/src/main/java/oracle/nosql/driver/package-info.java
@@ -4,6 +4,7 @@
* Licensed under the Universal Permissive License v 1.0 as shown at
* https://oss.oracle.com/licenses/upl/
*/
+ /* TODO: need add NoSQLHandleAsync? */
/**
* Contains the public API for using the Oracle NoSQL Database
* as well as configuration and common parameter classes used in
diff --git a/driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java b/driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java
index 6d3b5b00..cf68f5f4 100644
--- a/driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java
+++ b/driver/src/main/java/oracle/nosql/driver/util/ConcurrentUtil.java
@@ -1,3 +1,10 @@
+/*-
+ * Copyright (c) 2011, 2025 Oracle and/or its affiliates. All rights reserved.
+ *
+ * Licensed under the Universal Permissive License v 1.0 as shown at
+ * https://oss.oracle.com/licenses/upl/
+ */
+
package oracle.nosql.driver.util;
import oracle.nosql.driver.NoSQLException;
diff --git a/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java b/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
index d90f747b..82416a22 100644
--- a/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
+++ b/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
@@ -30,7 +30,6 @@
import javax.net.ssl.SSLException;
import io.netty.buffer.Unpooled;
-import io.netty.handler.codec.http.HttpResponseStatus;
import oracle.nosql.driver.RequestTimeoutException;
import oracle.nosql.driver.httpclient.HttpClient;
From 38203b09114099eb75cab7e731284b0ae6ba3569 Mon Sep 17 00:00:00 2001
From: Akshay Sundarraj
Date: Wed, 15 Oct 2025 13:34:01 +0530
Subject: [PATCH 06/11] Async refactor changes-V
- Change package version to 6.0.0
- Updated javadocs of NoSQLHandleAsync to mention the CompletionException.
When CompletableFuture returned by the async APIs completes exceptionally,
the thrown exception is CompletionException. The actual cause is wrapped in
CompletionException
- Added CHANGELOG.md entry
- Added a new smoke test for async APIs
Files:
M CHANGELOG.md
- Added an entry for unreleased 6.0.0 version
M README.md
M driver/pom.xml
M examples/pom.xml
M pom.xml
M driver/src/main/java/oracle/nosql/driver/SDKVersion.java
- Changed version to 6.0.0
M driver/src/main/java/oracle/nosql/driver/NoSQLHandleAsync.java
- Updated javadocs for CompletionException
M driver/src/test/java/oracle/nosql/driver/BasicAsyncTest.java
M driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java
- Added new async tests
---
CHANGELOG.md | 4 +
README.md | 2 +-
driver/pom.xml | 2 +-
.../oracle/nosql/driver/NoSQLHandleAsync.java | 139 ++++++++----
.../java/oracle/nosql/driver/SDKVersion.java | 2 +-
.../oracle/nosql/driver/BasicAsyncTest.java | 201 ++++++++++++++++++
.../oracle/nosql/driver/ProxyTestBase.java | 79 +++++++
examples/pom.xml | 4 +-
pom.xml | 2 +-
9 files changed, 383 insertions(+), 52 deletions(-)
create mode 100644 driver/src/test/java/oracle/nosql/driver/BasicAsyncTest.java
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 15f68fa4..692c88a1 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -2,6 +2,10 @@
All notable changes to this project will be documented in this file.
The format is based on [Keep a Changelog](http://keepachangelog.com/).
+## [6.0.0] [Unreleased]
+
+### TODO
+
## [5.4.18] 2025-10-01
### Added
diff --git a/README.md b/README.md
index cff205cb..cb24ece5 100644
--- a/README.md
+++ b/README.md
@@ -37,7 +37,7 @@ project. The version changes with each release.
com.oracle.nosql.sdknosqldriver
- 5.4.18
+ 6.0.0
```
diff --git a/driver/pom.xml b/driver/pom.xml
index 7b7af82d..7c92425e 100644
--- a/driver/pom.xml
+++ b/driver/pom.xml
@@ -29,7 +29,7 @@
com.oracle.nosql.sdknosqldriver
- 5.4.18
+ 6.0.0jar
diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleAsync.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleAsync.java
index 8e07f9af..1fa9ca73 100644
--- a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleAsync.java
+++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleAsync.java
@@ -8,6 +8,7 @@
package oracle.nosql.driver;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
import java.util.concurrent.Flow;
import oracle.nosql.driver.ops.AddReplicaRequest;
@@ -27,7 +28,6 @@
import oracle.nosql.driver.ops.PrepareResult;
import oracle.nosql.driver.ops.PutRequest;
import oracle.nosql.driver.ops.PutResult;
-import oracle.nosql.driver.ops.QueryIterableResult;
import oracle.nosql.driver.ops.QueryRequest;
import oracle.nosql.driver.ops.QueryResult;
import oracle.nosql.driver.ops.ReplicaStatsRequest;
@@ -99,7 +99,9 @@
*
* On success all methods in this interface return {@link CompletableFuture}
* which completes with {@link Result} objects.
- * On Error, return {@link CompletableFuture} completes with exceptions.
+ * On Error, return {@link CompletableFuture} completes with
+ * {@link java.util.concurrent.CompletionException} that wraps the original
+ * exception as its cause.
* Some Java exceptions, such as {@link IllegalArgumentException} and
* {@link NullPointerException} are thrown directly. All other exceptions are
* instances of {@link NoSQLException}, which serves as a base class for NoSQL
@@ -161,8 +163,10 @@ public interface NoSQLHandleAsync extends AutoCloseable {
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -194,8 +198,10 @@ public interface NoSQLHandleAsync extends AutoCloseable {
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The CompletableFuture returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -258,8 +264,10 @@ public interface NoSQLHandleAsync extends AutoCloseable {
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -291,8 +299,10 @@ public interface NoSQLHandleAsync extends AutoCloseable {
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -325,8 +335,10 @@ public interface NoSQLHandleAsync extends AutoCloseable {
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -382,8 +394,10 @@ public interface NoSQLHandleAsync extends AutoCloseable {
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -469,8 +483,10 @@ public interface NoSQLHandleAsync extends AutoCloseable {
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -498,8 +514,10 @@ public interface NoSQLHandleAsync extends AutoCloseable {
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -531,8 +549,10 @@ public interface NoSQLHandleAsync extends AutoCloseable {
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -577,8 +597,10 @@ CompletableFuture doTableRequest(TableRequest request,
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -604,8 +626,10 @@ CompletableFuture doTableRequest(TableRequest request,
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -632,8 +656,10 @@ CompletableFuture doTableRequest(TableRequest request,
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -664,8 +690,10 @@ CompletableFuture doTableRequest(TableRequest request,
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -694,8 +722,10 @@ CompletableFuture doTableRequest(TableRequest request,
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -720,8 +750,10 @@ CompletableFuture doTableRequest(TableRequest request,
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -744,8 +776,10 @@ CompletableFuture doTableRequest(TableRequest request,
* @return a {@link CompletableFuture} which completes with the namespaces
* or null if none are found.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -767,8 +801,11 @@ CompletableFuture doTableRequest(TableRequest request,
*
* @return a {@link CompletableFuture} which completes with the list of
* roles or null if none are found.
- *
- * The CompletableFuture returned by this method can be completed exceptionally with the following exceptions.
+ *
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -790,8 +827,10 @@ CompletableFuture doTableRequest(TableRequest request,
* @return a {@link CompletableFuture} which completes with the users
* or null if none are found.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -833,8 +872,10 @@ CompletableFuture doTableRequest(TableRequest request,
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -867,8 +908,10 @@ CompletableFuture doSystemRequest(String statement,
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -898,8 +941,10 @@ CompletableFuture doSystemRequest(String statement,
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
@@ -926,8 +971,10 @@ CompletableFuture doSystemRequest(String statement,
* @return a {@link CompletableFuture} which completes with the result of
* the operation.
*
- * The {@link CompletableFuture} returned by this method can be completed
- * exceptionally with the following exceptions.
+ * The returned {@link CompletableFuture} may complete exceptionally with
+ * the following exceptions. The underlying exception is wrapped in a
+ * {@link CompletionException}; use {@link Throwable#getCause()} to
+ * retrieve it.
*
*
* {@link IllegalArgumentException} if any of the parameters are invalid or
diff --git a/driver/src/main/java/oracle/nosql/driver/SDKVersion.java b/driver/src/main/java/oracle/nosql/driver/SDKVersion.java
index 5fad9116..77d294d5 100644
--- a/driver/src/main/java/oracle/nosql/driver/SDKVersion.java
+++ b/driver/src/main/java/oracle/nosql/driver/SDKVersion.java
@@ -12,5 +12,5 @@ public class SDKVersion {
/**
* The full X.Y.Z version of the current SDK
*/
- public static final String VERSION = "5.4.18";
+ public static final String VERSION = "6.0.0";
}
diff --git a/driver/src/test/java/oracle/nosql/driver/BasicAsyncTest.java b/driver/src/test/java/oracle/nosql/driver/BasicAsyncTest.java
new file mode 100644
index 00000000..a065ceec
--- /dev/null
+++ b/driver/src/test/java/oracle/nosql/driver/BasicAsyncTest.java
@@ -0,0 +1,201 @@
+/*-
+ * Copyright (c) 2011, 2025 Oracle and/or its affiliates. All rights reserved.
+ *
+ * Licensed under the Universal Permissive License v 1.0 as shown at
+ * https://oss.oracle.com/licenses/upl/
+ */
+
+package oracle.nosql.driver;
+
+import oracle.nosql.driver.ops.DeleteRequest;
+import oracle.nosql.driver.ops.GetRequest;
+import oracle.nosql.driver.ops.ListTablesRequest;
+import oracle.nosql.driver.ops.PutRequest;
+import oracle.nosql.driver.ops.TableLimits;
+import oracle.nosql.driver.ops.TableResult;
+import oracle.nosql.driver.ops.TableUsageRequest;
+import oracle.nosql.driver.values.MapValue;
+import org.junit.Test;
+
+import java.util.concurrent.CompletionException;
+
+import static oracle.nosql.driver.util.BinaryProtocol.V4;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class BasicAsyncTest extends ProxyTestBase {
+
+ @Test
+ public void smokeTest() {
+
+ try {
+ MapValue key = new MapValue().put("id", 10);
+ MapValue value = new MapValue().put("id", 10).put("name", "jane");
+
+ /* drop a table */
+ tableOperationAsync(asyncHandle,
+ "drop table if exists testusers",
+ null)
+ .whenComplete((tres, err) -> {
+ assertNotNull(tres.getTableName());
+ assertNull(tres.getTableLimits());
+ })
+ .thenCompose(ignored -> {
+ /* drop again without if exists -- should throw */
+ return tableOperationAsync(asyncHandle,
+ "drop table testusers",
+ null)
+ .handle((tres, err) -> {
+ assertNotNull("operation should have thrown", err);
+ assertTrue(err instanceof CompletionException);
+ assertTrue("Expecting TableNotFoundException",
+ err.getCause() instanceof TableNotFoundException);
+ return null;
+ });
+ })
+ .thenCompose(ignored -> {
+ /* Create a table */
+ return tableOperationAsync(
+ asyncHandle,
+ "create table if not exists testusers(id integer, " +
+ "name string, primary key(id))",
+ new TableLimits(500, 500, 50))
+ .thenAccept(tres -> {
+ assertNotNull(tres);
+ assertEquals(TableResult.State.ACTIVE, tres.getTableState());
+ });
+ })
+ .thenCompose(ignored -> {
+ /* Create an index */
+ return tableOperationAsync(
+ asyncHandle,
+ "create index if not exists Name on testusers(name)",
+ null)
+ .thenAccept(tres -> {
+ assertNotNull(tres);
+ assertEquals(TableResult.State.ACTIVE, tres.getTableState());
+ });
+ })
+ .thenCompose(ignored -> {
+ /* list tables */
+ ListTablesRequest listTables = new ListTablesRequest();
+ return asyncHandle.listTables(listTables)
+ .thenApply(lres -> {
+ assertNotNull(lres);
+ /*
+ * the test cases don't yet clean up so there
+ * may be additional tables present, be
+ * flexible in this assertion.
+ */
+ assertTrue(lres.getTables().length >= 1);
+ assertNotNull(lres.toString());
+ return lres;
+ });
+ })
+ .thenCompose(ignored -> {
+ /* getTableUsage. It won't return much in test mode */
+ if (!onprem) {
+ TableUsageRequest gtu = new TableUsageRequest()
+ .setTableName("testusers").setLimit(2)
+ .setEndTime(System.currentTimeMillis());
+ return asyncHandle.getTableUsage(gtu)
+ .thenAccept(gtuRes -> {
+ assertNotNull(gtuRes);
+ assertNotNull(gtuRes.getUsageRecords());
+ });
+ }
+ return null;
+ })
+ .thenCompose(ignored -> {
+ /* PUT */
+ PutRequest putRequest = new PutRequest()
+ .setValue(value)
+ .setTableName("testusers");
+ return asyncHandle.put(putRequest)
+ .thenAccept(res -> {
+ assertNotNull(res.getVersion());
+ assertWriteKB(res);
+ });
+ })
+ .thenCompose(ignored -> {
+ /* GET */
+ GetRequest getRequest = new GetRequest()
+ .setKey(key)
+ .setTableName("testusers");
+
+ return asyncHandle.get(getRequest)
+ .whenComplete((gres, err) -> {
+ assertNotNull(gres);
+ assertNotNull(gres.getJsonValue());
+ assertEquals("jane",
+ gres.getValue().getString("name"));
+ assertReadKB(gres);
+ });
+ })
+ .thenCompose(ignored -> {
+ /* DELETE */
+ DeleteRequest deleteRequest = new DeleteRequest()
+ .setKey(key)
+ .setTableName("testusers")
+ .setReturnRow(true);
+ return asyncHandle.delete(deleteRequest)
+ .whenComplete((dres, err) -> {
+ assertNotNull(dres);
+ assertTrue(dres.getSuccess());
+ assertWriteKB(dres);
+ if (proxySerialVersion <= V4) {
+ assertNull(dres.getExistingVersion());
+ } else {
+ assertEquals(value, dres.getExistingValue());
+ }
+ });
+ })
+ .thenCompose(ignored -> {
+ /* GET -- no row, it was removed above */
+ GetRequest getRequest = new GetRequest()
+ .setTableName("testusers")
+ .setKey(key);
+ return asyncHandle.get(getRequest)
+ .whenComplete((gres, err) -> {
+ assertNotNull(gres);
+ assertNull(gres.getValue());
+ });
+ }).join();
+
+ /* GET -- no table */
+ GetRequest getRequest = new GetRequest()
+ .setTableName("not_a_table")
+ .setKey(key);
+ asyncHandle.get(getRequest)
+ .handle((gres, err) -> {
+ assertTrue(err instanceof CompletionException);
+ assertTrue(
+ "Attempt to access missing table should "
+ + "have thrown",
+ err.getCause() instanceof TableNotFoundException);
+ return null;
+ }).join();
+
+ /* PUT -- invalid row -- this will throw */
+ value.remove("id");
+ value.put("not_a_field", 1);
+ PutRequest putRequest = new PutRequest()
+ .setValue(value)
+ .setTableName("testusers");
+ asyncHandle.put(putRequest)
+ .handle((pres, err) -> {
+ assertTrue(err instanceof CompletionException);
+ assertTrue(
+ "Attempt to put invalid row should have thrown",
+ err.getCause() instanceof IllegalArgumentException);
+ return null;
+ }).join();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail("Exception in test");
+ }
+ }
+}
diff --git a/driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java b/driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java
index 87b47650..2b45e574 100644
--- a/driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java
+++ b/driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java
@@ -123,6 +123,8 @@ public class ProxyTestBase {
protected NoSQLHandle handle;
+ protected NoSQLHandleAsync asyncHandle;
+
/* serial version used at the proxy server */
protected int proxySerialVersion;
@@ -249,6 +251,35 @@ protected static TableResult tableOperation(NoSQLHandle handle,
handle.doTableRequest(tableRequest, waitMillis, 1000);
return tres;
}
+
+ protected static CompletableFuture
+ tableOperationAsync(NoSQLHandleAsync handle,
+ String statement,
+ TableLimits limits) {
+ return tableOperationAsync(handle,
+ statement,
+ limits,
+ DEFAULT_DDL_TIMEOUT);
+ }
+
+ /**
+ * run the statement, assumes success, exception is thrown on error
+ */
+ protected static CompletableFuture
+ tableOperationAsync(NoSQLHandleAsync handle,
+ String statement,
+ TableLimits limits,
+ int waitMillis) {
+ assertTrue(waitMillis > 500);
+ TableRequest tableRequest = new TableRequest()
+ .setStatement(statement)
+ .setTableLimits(limits)
+ .setTimeout(DEFAULT_DDL_TIMEOUT);
+
+ CompletableFuture tres =
+ handle.doTableRequest(tableRequest, waitMillis, 1000);
+ return tres;
+ }
/**
* run the statement, assumes success, exception is thrown on error
*/
@@ -272,6 +303,7 @@ public void beforeTest() throws Exception {
* Configure and get the handle
*/
handle = getHandle(endpoint);
+ asyncHandle = getAsyncHandle(endpoint);
/* track existing tables and don't drop them */
existingTables = new HashSet();
@@ -310,6 +342,9 @@ public void afterTest() throws Exception {
}
handle.close();
}
+ if (asyncHandle != null) {
+ asyncHandle.close();
+ }
}
protected static void dropAllTables(NoSQLHandle nosqlHandle,
@@ -425,6 +460,12 @@ protected NoSQLHandle getHandle(String ep) {
return setupHandle(config);
}
+ protected NoSQLHandleAsync getAsyncHandle(String ep) {
+ NoSQLHandleConfig config = new NoSQLHandleConfig(ep);
+ serviceURL = config.getServiceURL();
+ return setupAsyncHandle(config);
+ }
+
/* Set configuration values for the handle */
protected NoSQLHandle setupHandle(NoSQLHandleConfig config) {
/*
@@ -447,6 +488,23 @@ protected NoSQLHandle setupHandle(NoSQLHandleConfig config) {
return h;
}
+ /* Set configuration values for the aysnc handle */
+ protected NoSQLHandleAsync setupAsyncHandle(NoSQLHandleConfig config) {
+ /*
+ * 5 retries, default retry algorithm
+ */
+ config.configureDefaultRetryHandler(5, 0);
+ config.setRequestTimeout(30000);
+
+ /* remove idle connections after this many seconds */
+ config.setConnectionPoolInactivityPeriod(INACTIVITY_PERIOD_SECS);
+ configAuth(config);
+
+ /* allow test cases to add/modify handle config */
+ perTestHandleConfig(config);
+ return getAsyncHandle(config);
+ }
+
/**
* sub classes can override this to affect the handle config
*/
@@ -475,6 +533,27 @@ protected NoSQLHandle getHandle(NoSQLHandleConfig config) {
return NoSQLHandleFactory.createNoSQLHandle(config);
}
+ /**
+ * get a handle based on the config
+ */
+ protected NoSQLHandleAsync getAsyncHandle(NoSQLHandleConfig config) {
+ /*
+ * Create a Logger, set to WARNING by default.
+ */
+ Logger logger = Logger.getLogger(getClass().getName());
+ String level = System.getProperty("test.loglevel");
+ if (level == null) {
+ level = "WARNING";
+ }
+ logger.setLevel(Level.parse(level));
+ config.setLogger(logger);
+
+ /*
+ * Open the handle
+ */
+ return NoSQLHandleFactory.createNoSQLHandleAsync(config);
+ }
+
void assertReadKB(Result res) {
if (onprem) {
return;
diff --git a/examples/pom.xml b/examples/pom.xml
index 75a94468..355af68d 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -3,7 +3,7 @@
4.0.0com.oracle.nosql.sdk
- 5.4.18
+ 6.0.0nosql-java-sdk-examplesOracle NoSQL Database Java ExamplesJava examples for Oracle NoSQL Database
@@ -25,7 +25,7 @@
com.oracle.nosql.sdknosqldriver
- 5.4.18
+ 6.0.0
diff --git a/pom.xml b/pom.xml
index 464ef673..3b10c327 100644
--- a/pom.xml
+++ b/pom.xml
@@ -6,7 +6,7 @@
com.oracle.nosql.sdknosql-java-sdk
- 5.4.18
+ 6.0.0pomOracle NoSQL SDK
From 79db68bc1084fca6ebb84d4589ceee5aaad29f88 Mon Sep 17 00:00:00 2001
From: Akshay Sundarraj
Date: Fri, 31 Oct 2025 20:22:54 +0530
Subject: [PATCH 07/11] Async refactor changes-VI
- Fixed netty bytebuf leak
- Fixed NullPointer exception in BasicAsyncTest
- Fixed double writing to bytebuf in HttpRequestUtil
- Updated javadoc for HttpClient.runRequest
Files:
M driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
- Added javadoc for runRequest
- Release HttpResponse in keep-alive path
M driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
- Fix double writing to bytebuf
M driver/src/test/java/oracle/nosql/driver/BasicAsyncTest.java
- Fix NullPointer issue
---
.../nosql/driver/httpclient/HttpClient.java | 35 ++++++++++++++-----
.../nosql/driver/util/HttpRequestUtil.java | 1 -
.../oracle/nosql/driver/BasicAsyncTest.java | 3 +-
3 files changed, 29 insertions(+), 10 deletions(-)
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
index 875bfe60..d271c8cf 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
@@ -418,15 +418,17 @@ public void removeChannel(Channel channel) {
pool.removeChannel(channel);
}
-
/**
- * Sends an HttpRequest, setting up the ResponseHandler as the handler to
- * use for the (asynchronous) response.
- *
- * @param request the request
+ * Sends an HttpRequest to the server.
*
- * @throws IOException if there is a network problem (bad channel). Such
- * exceptions can be retried.
+ * @param request HttpRequest
+ * @param timeoutMs Time to wait for the response from the server.
+ * Returned future completes with {@link TimeoutException}
+ * in case of timeout
+ * @return {@link CompletableFuture} holding the response from the server.
+ * @apiNote The caller must release the response by calling
+ * {@link FullHttpResponse#release()} or
+ * {@link ReferenceCountUtil#release(Object)}
*/
public CompletableFuture runRequest(HttpRequest request,
int timeoutMs) {
@@ -480,6 +482,19 @@ public CompletableFuture runRequest(HttpRequest request,
return responseFuture;
}
+ /**
+ * Sends an HttpRequest to the server on a given netty channel.
+ *
+ * @param request HttpRequest
+ * @param channel Netty channel
+ * @param timeoutMs Time to wait for the response from the server.
+ * Returned future completes with {@link TimeoutException}
+ * in case of timeout
+ * @return {@link CompletableFuture} holding the response from the server.
+ * @apiNote The caller must release the response by calling
+ * {@link FullHttpResponse#release()} or
+ * {@link ReferenceCountUtil#release(Object)}
+ */
public CompletableFuture runRequest(HttpRequest request,
Channel channel,
long timeoutMs) {
@@ -513,6 +528,7 @@ private static boolean isTimeout(long deadlineNs) {
*/
boolean doKeepAlive(Channel ch) {
final int keepAliveTimeout = 3000; /* ms */
+ FullHttpResponse response = null;
try {
final HttpRequest request =
new DefaultFullHttpRequest(HTTP_1_1, HEAD, "/");
@@ -522,7 +538,7 @@ boolean doKeepAlive(Channel ch) {
* other server may reject them and close the connection
*/
request.headers().add(HOST, host);
- FullHttpResponse response = ConcurrentUtil.awaitFuture(
+ response = ConcurrentUtil.awaitFuture(
runRequest(request, ch, keepAliveTimeout));
/*
* LBaaS will return a non-200 status but that is expected as the
@@ -542,6 +558,9 @@ boolean doKeepAlive(Channel ch) {
"Exception sending keepalive on [channel:%s] error:%s",
ch.id(), t.getMessage());
logFine(logger, msg);
+ } finally {
+ /* Release response */
+ ReferenceCountUtil.release(response);
}
/* something went wrong, caller is responsible for disposition */
return false;
diff --git a/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java b/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
index 82416a22..62903523 100644
--- a/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
+++ b/driver/src/main/java/oracle/nosql/driver/util/HttpRequestUtil.java
@@ -328,7 +328,6 @@ private static FullHttpRequest buildRequest(String requestURI,
HttpMethod method,
byte[] payload) {
final ByteBuf buffer = Unpooled.wrappedBuffer(payload);
- buffer.writeBytes(payload);
final FullHttpRequest request =
new DefaultFullHttpRequest(HTTP_1_1, method, requestURI,
diff --git a/driver/src/test/java/oracle/nosql/driver/BasicAsyncTest.java b/driver/src/test/java/oracle/nosql/driver/BasicAsyncTest.java
index a065ceec..7da08411 100644
--- a/driver/src/test/java/oracle/nosql/driver/BasicAsyncTest.java
+++ b/driver/src/test/java/oracle/nosql/driver/BasicAsyncTest.java
@@ -17,6 +17,7 @@
import oracle.nosql.driver.values.MapValue;
import org.junit.Test;
+import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import static oracle.nosql.driver.util.BinaryProtocol.V4;
@@ -107,7 +108,7 @@ public void smokeTest() {
assertNotNull(gtuRes.getUsageRecords());
});
}
- return null;
+ return CompletableFuture.completedFuture(null);
})
.thenCompose(ignored -> {
/* PUT */
From bdf01cfeff680b87f1484a2bf3fd861079e46761 Mon Sep 17 00:00:00 2001
From: Akshay Sundarraj
Date: Wed, 26 Nov 2025 14:11:30 +0530
Subject: [PATCH 08/11] Async refactor changes-VII
Connection pooling changes
- Added new connection pool `IdleEvictFixedChannelPool` to limit the maximum
number of connections. This is necessary in the async world, where a single
thread can fire huge number of requests in a very short period of time.
Existing unbounded connections put pressure on system resources and also
eventually lead to too many files(sockets) open error. To tackle this, two new
parameters 'connectionPoolSize' and 'poolMaxPending' are re-introduced. These
configs were present earlier and later removed
- connectionPoolSize - The maximum number of connections (upper bound) in the
connection pool, if this number is reached, further connection acquisition
requests are queued up to poolMaxPending
- poolMaxPending - The maximum number of pending acquires for the pool.
If this exceeds, new connection acquires result in error until connections
are released back to the pool
- The IdleEvictFixedChannelPool class wraps Netty's FixedChannelPool to track the
pool metrics and to evict idle channels from the pool after idle timeout.
This class simplifies the existing ConnectionPool implementation by removing
keep-alive on minimum number of channels and using Netty's native
IdleChannelHandler to remove the idle channels instead of running a dedicated
periodic job to modify the internal state of the pool which is not possible for
Netty FixedChannelPool
- Channel keep-alive task is removed. In the existing implementation keep-alive job
periodically runs keep-alive on two idle connections to retain the connections for
future requests and there by reducing the latency. This hardly reduces P95 latency
and difficult to implement on top of Netty FixedChannelPool. However, Idle time out
and evict of channels is still retained. Now, all the idle channels are evicted.
- Added unit tests for IdleEvictFixedChannelPool and performance tests for async APIs
Files:
A driver/src/main/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPool.java
A driver/src/test/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPoolTest.java
- New connection pool impl for fixed number of connections
- Unit tests
M driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
- Added a new constructor to take maxConnections and maxPending
- Replace ConnectionPool with IdleEvictFixedChannelPool
- Removed acquireRetryIntervalMs
- connectionPoolMinSize and isMinimalClient are no longer used as keep-alive is unnecessary
- Added timeout for channel acquisition from the pool. This timeout is set to request timeout
M driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java
- Added default value for connectionPoolSize and poolMaxPending
- Removed deprecate javadoc for pool size and pending size
- Updated javadoc to deprecate min pool size
M driver/src/main/java/oracle/nosql/driver/http/Client.java
- Pass max connection and pending connection to HttpClient constructor
M driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java
- Changed default request time for test to 5000 ms. This is the default used by production code
A driver/src/test/java/oracle/nosql/driver/PerformanceTest.java
- Async PerformanceTest
---
.../nosql/driver/NoSQLHandleConfig.java | 63 ++--
.../java/oracle/nosql/driver/http/Client.java | 4 +-
.../nosql/driver/httpclient/HttpClient.java | 232 +++++++------
.../httpclient/IdleEvictFixedChannelPool.java | 237 ++++++++++++++
.../oracle/nosql/driver/PerformanceTest.java | 239 ++++++++++++++
.../oracle/nosql/driver/ProxyTestBase.java | 5 +-
.../IdleEvictFixedChannelPoolTest.java | 308 ++++++++++++++++++
7 files changed, 972 insertions(+), 116 deletions(-)
create mode 100644 driver/src/main/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPool.java
create mode 100644 driver/src/test/java/oracle/nosql/driver/PerformanceTest.java
create mode 100644 driver/src/test/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPoolTest.java
diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java
index 28ea4769..1d49449e 100644
--- a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java
+++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java
@@ -102,6 +102,8 @@ public class NoSQLHandleConfig implements Cloneable {
*/
public static final boolean DEFAULT_ENABLE_LOG = true;
+ private static final int DEFAULT_CONNECTION_POOL_SIZE = 100;
+ private static final int DEFAULT_CONNECTION_PENDING_SIZE = 10_000;
/*
* The url used to contact an HTTP proxy
@@ -277,6 +279,16 @@ public class NoSQLHandleConfig implements Cloneable {
*/
private String extensionUserAgent;
+ /**
+ * Maximum size of the connection pool
+ */
+ private int connectionPoolSize = DEFAULT_CONNECTION_POOL_SIZE;
+
+ /**
+ * The maximum number of pending acquires for the pool
+ */
+ private int poolMaxPending = DEFAULT_CONNECTION_PENDING_SIZE;
+
/**
* Specifies an endpoint or region id to use to connect to the Oracle
* NoSQL Database Cloud Service or, if on-premise, the Oracle NoSQL
@@ -724,19 +736,23 @@ public NoSQLHandleConfig setNumThreads(int numThreads) {
* Sets the maximum number of individual connections to use to connect
* to the service. Each request/response pair uses a connection. The
* pool exists to allow concurrent requests and will bound the number of
- * concurrent requests. Additional requests will wait for a connection to
- * become available. If requests need to wait for a significant time
- * additional connections may be created regardless of the pool size.
- * The default value if not set is number of available CPUs * 2.
+ * concurrent requests. Additional requests upto
+ * {@link NoSQLHandleConfig#poolMaxPending} will wait for a connection
+ * to become available.
+ * Default value is {@value DEFAULT_CONNECTION_POOL_SIZE}
*
* @param poolSize the pool size
*
* @return this
- * @deprecated The connection pool no longer supports a size setting.
- * It will expand as needed based on concurrent demand.
+ *
+ * @since 6.0.0
*/
- @Deprecated
public NoSQLHandleConfig setConnectionPoolSize(int poolSize) {
+ if (poolSize <= 0) {
+ throw new IllegalArgumentException(
+ "Connection pool size must be positive");
+ }
+ this.connectionPoolSize = poolSize;
return this;
}
@@ -760,7 +776,12 @@ public NoSQLHandleConfig setConnectionPoolSize(int poolSize) {
* @return this
*
* @since 5.3.2
+ * @deprecated since 6.0.0 - The connection pool no longer supports minimum
+ * size. Idle connections will be removed based on
+ * {@link NoSQLHandleConfig#connectionPoolInactivityPeriod}
+ * @see NoSQLHandleConfig#connectionPoolInactivityPeriod
*/
+ @Deprecated
public NoSQLHandleConfig setConnectionPoolMinSize(int poolMinSize) {
this.connectionPoolMinSize = poolMinSize;
return this;
@@ -789,16 +810,20 @@ public NoSQLHandleConfig setConnectionPoolInactivityPeriod(
/**
* Sets the maximum number of pending acquire operations allowed on the
* connection pool. This number is used if the degree of concurrency
- * desired exceeds the size of the connection pool temporarily. The
- * default value is 3.
+ * desired exceeds the size of the connection pool temporarily.
+ * Default value is {@value DEFAULT_CONNECTION_PENDING_SIZE}.
*
* @param poolMaxPending the maximum number allowed
*
* @return this
- * @deprecated The connection pool no longer supports pending requests.
+ * @since 6.0.0
*/
- @Deprecated
public NoSQLHandleConfig setPoolMaxPending(int poolMaxPending) {
+ if (poolMaxPending <= 0) {
+ throw new IllegalArgumentException("pool max pending value must " +
+ "be positive");
+ }
+ this.poolMaxPending = poolMaxPending;
return this;
}
@@ -869,13 +894,12 @@ public int getMaxChunkSize() {
* concurrent requests. Additional requests will wait for a connection to
* become available.
*
- * @return 0
- * @deprecated The connection pool no longer supports a size setting.
- * It will expand as needed based on concurrent demand.
+ * @return the pool size
+ *
+ * @since 6.0.0
*/
- @Deprecated
public int getConnectionPoolSize() {
- return 0;
+ return connectionPoolSize;
}
/**
@@ -908,12 +932,11 @@ public int getConnectionPoolInactivityPeriod() {
* Returns the maximum number of pending acquire operations allowed on
* the connection pool.
*
- * @return 0
- * @deprecated The connection pool no longer supports pending requests.
+ * @return the max pool pending
+ * @since 6.0.0
*/
- @Deprecated
public int getPoolMaxPending() {
- return 0;
+ return poolMaxPending;
}
/**
diff --git a/driver/src/main/java/oracle/nosql/driver/http/Client.java b/driver/src/main/java/oracle/nosql/driver/http/Client.java
index a79940aa..989329ce 100644
--- a/driver/src/main/java/oracle/nosql/driver/http/Client.java
+++ b/driver/src/main/java/oracle/nosql/driver/http/Client.java
@@ -410,7 +410,9 @@ protected HttpClient createHttpClient(URL url,
sslCtx,
httpConfig.getSSLHandshakeTimeout(),
"NoSQL Driver",
- logger);
+ logger,
+ httpConfig.getConnectionPoolSize(),
+ httpConfig.getPoolMaxPending());
}
/**
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
index d271c8cf..4b9dc82a 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
@@ -12,10 +12,9 @@
import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
import static oracle.nosql.driver.util.HttpConstants.CONNECTION;
import static oracle.nosql.driver.util.LogUtil.logFine;
-import static oracle.nosql.driver.util.LogUtil.logInfo;
-import java.io.IOException;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.logging.Logger;
@@ -25,6 +24,7 @@
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelOption;
import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.pool.ChannelPoolHandler;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.handler.codec.http.DefaultFullHttpRequest;
import io.netty.handler.codec.http.FullHttpResponse;
@@ -98,10 +98,7 @@ public class HttpClient {
public static final AttributeKey>
STATE_KEY = AttributeKey.valueOf("rqstate");
- //private final FixedChannelPool pool;
- private final ConnectionPool pool;
- private final HttpClientChannelPoolHandler poolHandler;
-
+ private final IdleEvictFixedChannelPool pool;
private final int maxContentLength;
private final int maxChunkSize;
@@ -109,12 +106,6 @@ public class HttpClient {
private final int port;
private final String name;
- /*
- * Amount of time to wait for acquiring a channel before timing
- * out and possibly retrying
- */
- private final int acquireRetryIntervalMs;
-
/*
* Non-null if using SSL
*/
@@ -164,7 +155,9 @@ public static HttpClient createMinimalClient(String host,
true, /* minimal client */
DEFAULT_MAX_CONTENT_LENGTH,
DEFAULT_MAX_CHUNK_SIZE,
- sslCtx, handshakeTimeoutMs, name, logger);
+ sslCtx, handshakeTimeoutMs, name, logger,
+ 1, /* max connections */
+ 1 /* max pending connections */);
}
/**
@@ -208,7 +201,60 @@ public HttpClient(String host,
this(host, port, numThreads, connectionPoolMinSize,
inactivityPeriodSeconds, false /* not minimal */,
- maxContentLength, maxChunkSize, sslCtx, handshakeTimeoutMs, name, logger);
+ maxContentLength, maxChunkSize, sslCtx, handshakeTimeoutMs, name,
+ logger,
+ 100 /* max connections */,
+ 10_000 /* max pending connections */);
+ }
+
+ /**
+ * Creates a new HttpClient class capable of sending Netty HttpRequest
+ * instances and receiving replies. This is a concurrent, asynchronous
+ * interface capable of sending and receiving on multiple HTTP channels
+ * at the same time.
+ *
+ * @param host the hostname for the HTTP server
+ * @param port the port for the HTTP server
+ * @param numThreads the number of async threads to use for Netty
+ * notifications. If 0, a default value is used based on the number of
+ * cores
+ * @param connectionPoolMinSize the number of connections to keep in the
+ * pool and keep alive using a minimal HTTP request. If 0, none are kept
+ * alive
+ * @param inactivityPeriodSeconds the number of seconds to keep an
+ * inactive channel/connection before removing it. 0 means use the default,
+ * a negative number means there is no timeout and channels are not
+ * removed
+ * @param maxContentLength maximum size in bytes of requests/responses.
+ * If 0, a default value is used (32MB).
+ * @param maxChunkSize maximum size in bytes of chunked response messages.
+ * If 0, a default value is used (64KB).
+ * @param sslCtx if non-null, SSL context to use for connections.
+ * @param handshakeTimeoutMs if not zero, timeout to use for SSL handshake
+ * @param name A name to use in logging messages for this client.
+ * @param logger A logger to use for logging messages.
+ * @param maxConnections Maximum size of the connection pool
+ * @param maxPendingConnections The maximum number of pending acquires
+ * for the pool
+ */
+ public HttpClient(String host,
+ int port,
+ int numThreads,
+ int connectionPoolMinSize,
+ int inactivityPeriodSeconds,
+ int maxContentLength,
+ int maxChunkSize,
+ SslContext sslCtx,
+ int handshakeTimeoutMs,
+ String name,
+ Logger logger,
+ int maxConnections,
+ int maxPendingConnections) {
+
+ this(host, port, numThreads, connectionPoolMinSize,
+ inactivityPeriodSeconds, false /* not minimal */,
+ maxContentLength, maxChunkSize, sslCtx, handshakeTimeoutMs, name,
+ logger, maxConnections, maxPendingConnections);
}
/*
@@ -225,7 +271,9 @@ private HttpClient(String host,
SslContext sslCtx,
int handshakeTimeoutMs,
String name,
- Logger logger) {
+ Logger logger,
+ int maxConnections,
+ int maxPendingConnections) {
this.logger = logger;
this.sslCtx = sslCtx;
@@ -247,13 +295,6 @@ private HttpClient(String host,
numThreads = cores*2;
}
- /* default pool min */
- if (connectionPoolMinSize == 0) {
- connectionPoolMinSize = DEFAULT_MIN_POOL_SIZE;
- } else if (connectionPoolMinSize < 0) {
- connectionPoolMinSize = 0; // no min size
- }
-
workerGroup = new NioEventLoopGroup(numThreads);
Bootstrap b = new Bootstrap();
@@ -263,32 +304,11 @@ private HttpClient(String host,
b.option(ChannelOption.TCP_NODELAY, true);
b.remoteAddress(host, port);
- poolHandler =
- new HttpClientChannelPoolHandler(this);
- pool = new ConnectionPool(b, poolHandler, logger,
- isMinimalClient,
- connectionPoolMinSize,
- inactivityPeriodSeconds);
-
- /*
- * Don't do keepalive if min size is not set. That configuration
- * doesn't care about keep connections alive. Also don't set for
- * minimal clients.
- */
- if (!isMinimalClient && connectionPoolMinSize > 0) {
- /* this is the main request client */
- pool.setKeepAlive(new ConnectionPool.KeepAlive() {
- @Override
- public boolean keepAlive(Channel ch) {
- return doKeepAlive(ch);
- }
- });
- }
-
- /* TODO: eventually add this to Config? */
- acquireRetryIntervalMs = Integer.getInteger(
- "oracle.nosql.driver.acquire.retryinterval",
- 1000);
+ ChannelPoolHandler handler = new HttpClientChannelPoolHandler(this);
+ pool = new IdleEvictFixedChannelPool(b, handler,
+ maxConnections,
+ maxPendingConnections,
+ inactivityPeriodSeconds);
}
SslContext getSslContext() {
@@ -366,20 +386,24 @@ public int getProxyPort() {
}
public int getAcquiredChannelCount() {
- return pool.getAcquiredChannelCount();
+ return pool.getStats().acquired;
}
public int getTotalChannelCount() {
- return pool.getTotalChannels();
+ return pool.getStats().total;
}
public int getFreeChannelCount() {
- return pool.getFreeChannels();
+ return pool.getStats().idle;
+ }
+
+ public int getPendingAcquires() {
+ return pool.getStats().pending;
}
/* available for testing */
ConnectionPool getConnectionPool() {
- return pool;
+ return null;
}
/**
@@ -399,6 +423,22 @@ public void shutdown() {
syncUninterruptibly();
}
+ private CompletableFuture getChannel() {
+ CompletableFuture acquireFuture = new CompletableFuture<>();
+ pool.acquire().addListener((FutureListener) channelFuture -> {
+ if (channelFuture.isSuccess()) {
+ Channel channel = channelFuture.getNow();
+ if (!acquireFuture.complete(channel)) {
+ /* future already completed release channel back to pool */
+ pool.release(channel);
+ }
+ } else {
+ acquireFuture.completeExceptionally(channelFuture.cause());
+ }
+ });
+ return acquireFuture;
+ }
+
public void releaseChannel(Channel channel) {
/* Clear any response handler state from channel before releasing it */
channel.attr(STATE_KEY).set(null);
@@ -415,7 +455,8 @@ public void releaseChannel(Channel channel) {
*/
public void removeChannel(Channel channel) {
logFine(logger, "closing and removing channel " + channel);
- pool.removeChannel(channel);
+ /* below line is unnecessary. Pool will take care of closed channels */
+ //pool.removeChannel(channel);
}
/**
@@ -436,48 +477,56 @@ public CompletableFuture runRequest(HttpRequest request,
new CompletableFuture<>();
long deadlineNs = System.nanoTime() +
TimeUnit.MILLISECONDS.toNanos(timeoutMs);
- pool.acquire().addListener((FutureListener) channelFuture -> {
- if (channelFuture.isSuccess()) {
- Channel channel = channelFuture.getNow();
- if (isTimeout(deadlineNs)) {
- String msg = "Timed out trying to acquire channel";
- logInfo(logger, "HttpClient " + name + " " + msg);
- /* release channel and request Bytebuf */
- releaseChannel(channel);
- ReferenceCountUtil.release(request);
+ /* Acquire a channel from the pool */
+ CompletableFuture acuireFuture = getChannel();
+
+ /* setup timeout on channel acquisition */
+ acuireFuture.orTimeout(timeoutMs, TimeUnit.MILLISECONDS);
+
+ /* when acquire future completes exceptionally, release request bytebuf
+ * and complete the response future
+ */
+ acuireFuture.whenComplete((ch, err) -> {
+ if (err != null) {
+ ReferenceCountUtil.release(request);
+ /* Unwrap to check the real cause */
+ Throwable cause = err instanceof CompletionException ?
+ err.getCause() : err;
+ if (cause instanceof TimeoutException) {
+ final String msg = "Timed out trying to acquire channel";
responseFuture.completeExceptionally(
- new TimeoutException(msg));
- return;
+ new CompletionException(new TimeoutException(msg)));
}
+ /* Re-throw original if it wasn't a timeout */
+ responseFuture.completeExceptionally(cause);
+ }
+ });
- long remainingTimeoutNs = deadlineNs - System.nanoTime();
- long remainingTimeoutMs = Math.max(1,
- TimeUnit.NANOSECONDS.toMillis(remainingTimeoutNs));
+ /* send request on acquired channel */
+ acuireFuture.thenAccept(channel -> {
+ long remainingTimeoutNs = deadlineNs - System.nanoTime();
+ long remainingTimeoutMs = Math.max(1,
+ TimeUnit.NANOSECONDS.toMillis(remainingTimeoutNs));
- /* Execute the request on the acquired channel */
- CompletableFuture requestExecutionFuture =
+ /* Execute the request on the acquired channel */
+ CompletableFuture requestExecutionFuture =
runRequest(request, channel, remainingTimeoutMs);
- /* When the request execution future completes (either
- * successfully or exceptionally),
- * complete the public responseFuture and ensure the channel
- * is released back to the pool.
- */
- requestExecutionFuture.whenComplete((response, throwable) -> {
- /* Always release the channel */
- releaseChannel(channel);
- if (throwable != null) {
- responseFuture.completeExceptionally(throwable);
- } else {
- responseFuture.complete(response);
- }
- });
- } else {
- /* channel acquisition failed */
- //TODO send proper exception
- responseFuture.completeExceptionally(channelFuture.cause());
- }
+ /* When the request execution future completes (either
+ * successfully or exceptionally),
+ * complete the public responseFuture and ensure the channel
+ * is released back to the pool.
+ */
+ requestExecutionFuture.whenComplete((response, throwable) -> {
+ /* Always release the channel */
+ releaseChannel(channel);
+ if (throwable != null) {
+ responseFuture.completeExceptionally(throwable);
+ } else {
+ responseFuture.complete(response);
+ }
+ });
});
return responseFuture;
}
@@ -519,12 +568,9 @@ public CompletableFuture runRequest(HttpRequest request,
return responseFuture;
}
- private static boolean isTimeout(long deadlineNs) {
- return System.nanoTime() >= deadlineNs;
- }
-
/**
- * Use HTTP HEAD method to refresh the channel
+ * Use HTTP HEAD method to refresh the channel.
+ * TODO Remove this once approved
*/
boolean doKeepAlive(Channel ch) {
final int keepAliveTimeout = 3000; /* ms */
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPool.java b/driver/src/main/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPool.java
new file mode 100644
index 00000000..860b5a21
--- /dev/null
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPool.java
@@ -0,0 +1,237 @@
+/*-
+ * Copyright (c) 2011, 2025 Oracle and/or its affiliates. All rights reserved.
+ *
+ * Licensed under the Universal Permissive License v 1.0 as shown at
+ * https://oss.oracle.com/licenses/upl/
+ */
+
+package oracle.nosql.driver.httpclient;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.channel.group.ChannelGroup;
+import io.netty.channel.group.DefaultChannelGroup;
+import io.netty.channel.pool.ChannelPool;
+import io.netty.channel.pool.ChannelPoolHandler;
+import io.netty.channel.pool.FixedChannelPool;
+import io.netty.handler.timeout.IdleStateEvent;
+import io.netty.handler.timeout.IdleStateHandler;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.ImmediateEventExecutor;
+import io.netty.util.concurrent.Promise;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * A FixedChannelPool that comes pre-wired with:
+ * 1. Metrics (Total, Acquired, Idle, Pending)
+ * 2. Idle Eviction (Close connections unused for X seconds)
+ */
+public class IdleEvictFixedChannelPool implements ChannelPool {
+ private final FixedChannelPool delegate;
+ /* Tracks all active connections. Auto-removes them when they close */
+ private final ChannelGroup allChannels;
+ private final AtomicInteger acquiredCount = new AtomicInteger(0);
+ private final AtomicInteger pendingAcquireCount = new AtomicInteger(0);
+
+ /**
+ * Create new instance
+ * @param bootstrap the {@link Bootstrap} that is used for connections
+ * @param handler the {@link ChannelPoolHandler} that will be notified for
+ * the different pool actions
+ * @param maxConnections the number of maximal active connections,
+ * once this is reached new tries to acquire a {@link Channel} will be
+ * delayed until a connection is returned to the pool again.
+ * @param maxPendingAcquires the maximum number of pending acquires.
+ * Once this is exceed acquire tries will be failed.
+ * @param idleTimeoutSeconds The duration in seconds that a channel
+ * sits unused in the pool before being automatically closed and evicted.
+ * If this value is less than or equal to zero, idle channels won't be
+ * evicted.
+ */
+ IdleEvictFixedChannelPool(Bootstrap bootstrap,
+ ChannelPoolHandler handler,
+ int maxConnections,
+ int maxPendingAcquires,
+ int idleTimeoutSeconds) {
+
+ this.allChannels = new DefaultChannelGroup(ImmediateEventExecutor.INSTANCE);
+
+ /* Wrap the user's handler with our Idle handler */
+ ChannelPoolHandler idleHandler =
+ new IdleAwarePoolHandler(handler, idleTimeoutSeconds);
+
+ this.delegate = new FixedChannelPool(bootstrap,
+ idleHandler,
+ maxConnections,
+ maxPendingAcquires);
+ }
+
+ @Override
+ public Future acquire() {
+ return acquire(ImmediateEventExecutor.INSTANCE.newPromise());
+ }
+
+ @Override
+ public Future acquire(Promise promise) {
+ pendingAcquireCount.incrementAndGet();
+
+ /* Create an INTERNAL Promise that Netty's pool will complete */
+ Promise internalPromise =
+ ImmediateEventExecutor.INSTANCE.newPromise();
+
+ /* Ask the delegate pool to notify our INTERNAL promise, not the user
+ * passed promise
+ */
+ delegate.acquire(internalPromise);
+
+ internalPromise.addListener(f -> {
+ pendingAcquireCount.decrementAndGet();
+ if (f.isSuccess()) {
+ Channel ch = (Channel) f.getNow();
+ acquiredCount.incrementAndGet();
+ /* ChannelGroup acts as a Set.
+ * Adding the same channel multiple times (reuse) is
+ * safe/ignored.
+ */
+ allChannels.add(ch);
+ promise.setSuccess(ch);
+ } else {
+ /* Propagate failure */
+ promise.setFailure(f.cause());
+ }
+ });
+ return promise;
+ }
+
+ @Override
+ public Future release(Channel channel) {
+ return release(channel, ImmediateEventExecutor.INSTANCE.newPromise());
+ }
+
+ @Override
+ public Future release(Channel channel, Promise promise) {
+ acquiredCount.decrementAndGet();
+ return delegate.release(channel, promise);
+ }
+
+ @Override
+ public void close() {
+ delegate.close();
+ allChannels.close();
+ }
+
+ public PoolStats getStats() {
+ int total = allChannels.size();
+ int acquired = acquiredCount.get();
+ int pending = pendingAcquireCount.get();
+ /* Safety: Idle cannot be negative (handles race conditions) */
+ int idle = Math.max(0, total - acquired);
+ return new PoolStats(total, acquired, idle, pending);
+ }
+
+ /* Simple DTO for the stats
+ * Pool stats are not atomic consistent view of the pool.
+ */
+ public static class PoolStats {
+ public final int total;
+ public final int acquired;
+ public final int idle;
+ public final int pending;
+
+ public PoolStats(int total, int acquired, int idle, int pending) {
+ this.total = total;
+ this.acquired = acquired;
+ this.idle = idle;
+ this.pending = pending;
+ }
+
+ @Override
+ public String toString() {
+ return String.format("PoolStats{Total=%d, " +
+ "Acquired=%d, Idle=%d, Pending=%d}",
+ total, acquired, idle, pending);
+ }
+ }
+
+ /**
+ * A {@link ChannelPoolHandler} which detects idle channels and closes them
+ * when the channel is idle for configured time.
+ * This class wraps another {@link ChannelPoolHandler}
+ * typically {@link HttpClientChannelPoolHandler}.
+ */
+ public static class IdleAwarePoolHandler implements ChannelPoolHandler {
+ private static final String IDLE_HANDLER = "idleWatcher";
+ private static final String EVICT_HANDLER = "idleEvictor";
+ private final int idleSeconds;
+ private final ChannelPoolHandler delegate;
+
+ IdleAwarePoolHandler(ChannelPoolHandler delegate,
+ int idleSeconds) {
+ this.delegate = delegate;
+ this.idleSeconds = idleSeconds;
+ }
+
+ @Override
+ public void channelCreated(Channel ch) throws Exception {
+ assert ch.eventLoop().inEventLoop();
+ delegate.channelCreated(ch);
+ }
+
+ @Override
+ public void channelAcquired(Channel ch) throws Exception {
+ /* channel is in use now, remove Idle handler */
+ assert ch.eventLoop().inEventLoop();
+ if (idleSeconds > 0) {
+ if (ch.pipeline().get(IDLE_HANDLER) != null) {
+ ch.pipeline().remove(IDLE_HANDLER);
+ }
+ if (ch.pipeline().get(EVICT_HANDLER) != null) {
+ ch.pipeline().remove(EVICT_HANDLER);
+ }
+ }
+ delegate.channelAcquired(ch);
+ }
+
+ @Override
+ public void channelReleased(Channel ch) throws Exception {
+ /* Channel returned to pool: START the idle timer */
+ if (idleSeconds > 0) {
+ if (ch.pipeline().get(IDLE_HANDLER) == null) {
+ ch.pipeline().addFirst(IDLE_HANDLER,
+ new IdleStateHandler(0, 0,
+ idleSeconds, TimeUnit.SECONDS));
+ ch.pipeline().addAfter(IDLE_HANDLER, EVICT_HANDLER,
+ new EvictionHandler());
+ }
+ }
+ delegate.channelReleased(ch);
+ }
+
+ /* class to handle the idle timeout event */
+ static class EvictionHandler extends ChannelInboundHandlerAdapter {
+ @Override
+ public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception {
+ if (evt instanceof IdleStateEvent) {
+ /* Close the channel.
+ * The FixedChannelPool will detect this and remove it from
+ the queue.
+ * The MonitoredChannelPool's ChannelGroup will detect this
+ and decrement 'Total'.
+ */
+ ctx.close();
+ } else {
+ super.userEventTriggered(ctx, evt);
+ }
+ }
+
+ @Override
+ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+ super.exceptionCaught(ctx, cause);
+ }
+ }
+ }
+}
diff --git a/driver/src/test/java/oracle/nosql/driver/PerformanceTest.java b/driver/src/test/java/oracle/nosql/driver/PerformanceTest.java
new file mode 100644
index 00000000..b6af5902
--- /dev/null
+++ b/driver/src/test/java/oracle/nosql/driver/PerformanceTest.java
@@ -0,0 +1,239 @@
+/*-
+ * Copyright (c) 2011, 2025 Oracle and/or its affiliates. All rights reserved.
+ *
+ * Licensed under the Universal Permissive License v 1.0 as shown at
+ * https://oss.oracle.com/licenses/upl/
+ */
+
+package oracle.nosql.driver;
+
+import oracle.nosql.driver.http.Client;
+import oracle.nosql.driver.http.NoSQLHandleAsyncImpl;
+import oracle.nosql.driver.ops.DeleteRequest;
+import oracle.nosql.driver.ops.DeleteResult;
+import oracle.nosql.driver.ops.GetRequest;
+import oracle.nosql.driver.ops.GetResult;
+import oracle.nosql.driver.ops.PutRequest;
+import oracle.nosql.driver.ops.PutResult;
+import oracle.nosql.driver.ops.QueryRequest;
+import oracle.nosql.driver.ops.QueryResult;
+import oracle.nosql.driver.ops.Result;
+import oracle.nosql.driver.ops.TableLimits;
+import oracle.nosql.driver.ops.TableResult;
+import oracle.nosql.driver.values.MapValue;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Performance test for async APIs.
+ * The test has two phases, warm-up and load phase
+ * warm-up phase is to warm-up the netty connections
+ * load phase to randomly run one of put, get, delete and query
+ */
+public class PerformanceTest extends ProxyTestBase {
+ private static final String table = "perf_test";
+ private static final String ddl = "create table if not exists " + table +
+ "(id long, name string, primary key(id))";
+ private static final String dropDdl = "drop table if exists "+ table;
+ private static final int WARMUP_OPS = 100;
+ private static final int TOTAL_OPS = 100000;
+ private static final int THREADS = 100;
+ private static ExecutorService executor;
+
+ private static final int pipelineDepth = 100;
+
+ @BeforeClass
+ public static void setupTest() {
+ executor = Executors.newFixedThreadPool(THREADS);
+ }
+ @Before
+ public void setup() {
+ TableResult tres = tableOperationAsync(asyncHandle, ddl,
+ new TableLimits(1000, 1000,1)).join();
+ assertNotNull(tres.getTableName());
+ }
+
+ @After
+ public void teardown() {
+ TableResult tres =
+ tableOperationAsync(asyncHandle, dropDdl, null).join();
+ assertNotNull(tres.getTableName());
+ }
+
+ @Test
+ public void test() throws Exception {
+ Client client = ((NoSQLHandleAsyncImpl) asyncHandle).getClient();
+ client.enableRateLimiting(true, 100);
+
+ System.out.println("Warm-up phase");
+ //runOpsAsync(WARMUP_OPS, pipelineDepth);
+ runOpsAsync(WARMUP_OPS, pipelineDepth);
+
+ StatsControl statsControl = asyncHandle.getStatsControl();
+ statsControl.setProfile(StatsControl.Profile.ALL).setPrettyPrint(true);
+ statsControl.start();
+
+
+ System.out.println("Load phase");
+ long start = System.nanoTime();
+ //runOpsAsync(TOTAL_OPS, pipelineDepth);
+ runOpsAsync(TOTAL_OPS, pipelineDepth);
+ long end = System.nanoTime();
+
+
+ Duration duration = Duration.ofNanos(end - start);
+ double throughput = TOTAL_OPS / (duration.toMillis() / 1000.0);
+
+ System.out.println("Completed " + TOTAL_OPS + " operations");
+ System.out.println("Time = " + duration);
+ System.out.println("Throughput = " + throughput + " ops/sec");
+ statsControl.stop();
+ }
+
+ private void runOps(int count) throws Exception {
+ List> futures = new ArrayList<>(count);
+ Random random = new Random();
+ AtomicInteger failures = new AtomicInteger();
+ MapValue row = new MapValue()
+ .put("id", 1)
+ .put("name", "oracle");
+ MapValue key = new MapValue().put("id", 1);
+
+ for (int i = 0; i < count; i++) {
+ futures.add(CompletableFuture.runAsync(() -> {
+ try {
+ int op = random.nextInt(4);
+ switch (op) {
+ case 0 : {
+ //put op
+ PutRequest pr = new PutRequest()
+ .setTableName(table)
+ .setValue(row);
+ Result res = asyncHandle.put(pr).join();
+ assertNotNull(res);
+ break;
+ }
+ case 1 : {
+ GetRequest gr = new GetRequest()
+ .setTableName(table)
+ .setKey(key);
+ Result res = asyncHandle.get(gr).join();
+ assertNotNull(res);
+ break;
+ }
+ case 2 : {
+ DeleteRequest dr = new DeleteRequest()
+ .setTableName(table)
+ .setKey(key);
+ Result res = asyncHandle.delete(dr).join();
+ assertNotNull(res);
+ break;
+ } default : {
+ try(QueryRequest qr =
+ new QueryRequest()
+ .setStatement(
+ "select * from " + table + " where id=1")) {
+ Result res = asyncHandle.query(qr).join();
+ assertNotNull(res);
+ }
+ }
+ }
+ } catch (Exception e) {
+ failures.incrementAndGet();
+ }
+ }, executor));
+ }
+ CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get();
+ System.out.println("Failures = " + failures.get());
+ }
+
+ private void runOpsAsync(int count, int pipelineDepth) throws Exception {
+ final Semaphore semaphore = new Semaphore(pipelineDepth);
+ final List> futures = new ArrayList<>(count);
+ Random random = new Random();
+ AtomicInteger failures = new AtomicInteger();
+ MapValue row = new MapValue()
+ .put("id", 1)
+ .put("name", "oracle");
+ MapValue key = new MapValue().put("id", 1);
+
+ for (int i = 0; i < count; i++) {
+ try {
+ semaphore.acquire();
+ int op = random.nextInt(4);
+ switch (op) {
+ case 0 : {
+ //put op
+ PutRequest pr = new PutRequest()
+ .setTableName(table)
+ .setValue(row);
+ CompletableFuture fut =
+ asyncHandle.put(pr).whenComplete((res, err) -> {
+ assertNotNull(res);
+ semaphore.release();
+ });
+ futures.add(fut.thenRun(() -> {}));
+ break;
+ }
+ case 1 : {
+ GetRequest gr = new GetRequest()
+ .setTableName(table)
+ .setKey(key);
+ CompletableFuture fut =
+ asyncHandle.get(gr).whenComplete((res, err) -> {
+ assertNotNull(res);
+ semaphore.release();
+ });
+ futures.add(fut.thenRun(() -> {}));
+ break;
+ }
+ case 2 : {
+ DeleteRequest dr = new DeleteRequest()
+ .setTableName(table)
+ .setKey(key);
+ CompletableFuture fut =
+ asyncHandle.delete(dr).whenComplete((res, err) -> {
+ assertNotNull(res);
+ semaphore.release();
+ });
+ futures.add(fut.thenRun(() -> {}));
+ break;
+ } default : {
+ try(QueryRequest qr =
+ new QueryRequest()
+ .setStatement(
+ "select * from " + table + " where id=1")) {
+ CompletableFuture fut =
+ asyncHandle.query(qr)
+ .whenComplete((res, err) -> {
+ assertNotNull(res);
+ semaphore.release();
+ });
+ futures.add(fut.thenRun(() -> {}));
+ }
+ }
+ }
+ } catch (Exception e) {
+ failures.incrementAndGet();
+ }
+ }
+ CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get();
+ System.out.println("Failures = " + failures.get());
+ }
+}
diff --git a/driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java b/driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java
index 2b45e574..80fd72b5 100644
--- a/driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java
+++ b/driver/src/test/java/oracle/nosql/driver/ProxyTestBase.java
@@ -80,6 +80,7 @@ public class ProxyTestBase {
protected static String TRACE = "test.trace";
protected static int DEFAULT_DDL_TIMEOUT = 15000;
protected static int DEFAULT_DML_TIMEOUT = 5000;
+ protected static int DEFAULT_REQUEST_TIMEOUT = 5000;
protected static String TEST_TABLE_NAME = "drivertest";
protected static int INACTIVITY_PERIOD_SECS = 2;
protected static String NETTY_LEAK_PROP="test.detectleaks";
@@ -472,7 +473,7 @@ protected NoSQLHandle setupHandle(NoSQLHandleConfig config) {
* 5 retries, default retry algorithm
*/
config.configureDefaultRetryHandler(5, 0);
- config.setRequestTimeout(30000);
+ config.setRequestTimeout(DEFAULT_REQUEST_TIMEOUT);
/* remove idle connections after this many seconds */
config.setConnectionPoolInactivityPeriod(INACTIVITY_PERIOD_SECS);
@@ -494,7 +495,7 @@ protected NoSQLHandleAsync setupAsyncHandle(NoSQLHandleConfig config) {
* 5 retries, default retry algorithm
*/
config.configureDefaultRetryHandler(5, 0);
- config.setRequestTimeout(30000);
+ config.setRequestTimeout(DEFAULT_REQUEST_TIMEOUT);
/* remove idle connections after this many seconds */
config.setConnectionPoolInactivityPeriod(INACTIVITY_PERIOD_SECS);
diff --git a/driver/src/test/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPoolTest.java b/driver/src/test/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPoolTest.java
new file mode 100644
index 00000000..fa6bfb08
--- /dev/null
+++ b/driver/src/test/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPoolTest.java
@@ -0,0 +1,308 @@
+/*-
+ * Copyright (c) 2011, 2025 Oracle and/or its affiliates. All rights reserved.
+ *
+ * Licensed under the Universal Permissive License v 1.0 as shown at
+ * https://oss.oracle.com/licenses/upl/
+ */
+
+package oracle.nosql.driver.httpclient;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.local.LocalAddress;
+import io.netty.channel.local.LocalChannel;
+import io.netty.channel.local.LocalServerChannel;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.pool.ChannelPoolHandler;
+import io.netty.handler.timeout.IdleStateEvent;
+import io.netty.util.concurrent.Future;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class IdleEvictFixedChannelPoolTest {
+ private EventLoopGroup group;
+ private Channel serverChannel;
+ private LocalAddress address;
+
+ @Before
+ public void setUp() throws InterruptedException {
+ group = new NioEventLoopGroup();
+ address = new LocalAddress("test-port");
+
+ /* Start a fake Local Server so the pool can connect */
+ ServerBootstrap sb = new ServerBootstrap()
+ .group(group)
+ .channel(LocalServerChannel.class)
+ .childHandler(new ChannelInitializer() {
+ @Override
+ protected void initChannel(Channel ch) {
+ ch.pipeline().addLast(new ChannelInboundHandlerAdapter());
+ }
+ });
+ serverChannel = sb.bind(address).sync().channel();
+ }
+
+ @After
+ public void tearDown() {
+ serverChannel.close();
+ group.shutdownGracefully();
+ }
+
+ @Test
+ public void testMetricsAndReuse() throws Exception {
+ /* Create Pool */
+ Bootstrap bootstrap = new Bootstrap()
+ .group(group)
+ .channel(LocalChannel.class)
+ .remoteAddress(address);
+
+ /* A dummy user handler (noop) */
+ ChannelPoolHandler noopHandler = new ChannelPoolHandler() {
+ public void channelReleased(Channel ch) {}
+ public void channelAcquired(Channel ch) {}
+ public void channelCreated(Channel ch) {}
+ };
+
+ IdleEvictFixedChannelPool pool =
+ new IdleEvictFixedChannelPool(bootstrap,
+ noopHandler,
+ 2,
+ 5,
+ 5);
+
+ /* CHECK 1: Initial */
+ assertStats(pool, 0, 0, 0, 0);
+
+ /* CHECK 2: Acquire */
+ Channel ch1 = pool.acquire().sync().getNow();
+ /* Total:1, Acquired:1, Idle:0 */
+ assertStats(pool, 1, 1, 0, 0);
+
+ /* CHECK 3: Release (Idle Logic should kick in) */
+ pool.release(ch1).sync();
+ /* Total:1, Acquired:0, Idle:1 */
+ assertStats(pool, 1, 0, 1, 0);
+
+ /* CHECK 4: Reuse */
+ Channel ch2 = pool.acquire().sync().getNow();
+ /* Should be the SAME channel object (reused) */
+ assertEquals(ch1.id(), ch2.id());
+ /* Stats: Total:1, Acquired:1, Idle:0 */
+ assertStats(pool, 1, 1, 0, 0);
+
+ /* Cleanup */
+ pool.close();
+ }
+
+ @Test
+ public void testIdleEvictionInPool() throws InterruptedException {
+ /* Create Pool */
+ Bootstrap bootstrap = new Bootstrap()
+ .group(group)
+ .channel(LocalChannel.class)
+ .remoteAddress(address);
+
+ /* A dummy user handler (noop) */
+ ChannelPoolHandler noopHandler = new ChannelPoolHandler() {
+ public void channelReleased(Channel ch) {}
+ public void channelAcquired(Channel ch) {}
+ public void channelCreated(Channel ch) {}
+ };
+
+ try(IdleEvictFixedChannelPool pool =
+ new IdleEvictFixedChannelPool(bootstrap,
+ noopHandler,
+ 2,
+ 5,
+ 60)) {
+
+ /* 1. Acquire a channel */
+ Channel ch = pool.acquire().sync().getNow();
+
+ /* 2. Release it back to the pool (This starts the Idle Timer) */
+ pool.release(ch).sync();
+
+ /* Verify it's currently Idle */
+ assertTrue(ch.isOpen());
+ assertEquals(1, pool.getStats().idle);
+
+ /* 3. SIMULATE the timeout Event
+ * Instead of waiting 1 minute, we force the event specifically on
+ * this channel
+ */
+ ch.pipeline().
+ fireUserEventTriggered(IdleStateEvent.ALL_IDLE_STATE_EVENT);
+
+
+ /* 4. Assert Eviction - The channel should be closed */
+ Thread.sleep(100);
+ assertFalse("Channel should be closed after idle event",
+ ch.isOpen());
+
+ /* The metrics should update (Total drops to 0) */
+ assertEquals("Total count should drop to 0",
+ 0, pool.getStats().total);
+ assertEquals("Idle count should drop to 0",
+ 0, pool.getStats().idle);
+ }
+ }
+
+ @Test
+ public void testNoEvictionWhenAcquired() throws InterruptedException {
+ /* Create Pool */
+ Bootstrap bootstrap = new Bootstrap()
+ .group(group)
+ .channel(LocalChannel.class)
+ .remoteAddress(address);
+
+ /* A dummy user handler (noop) */
+ ChannelPoolHandler noopHandler = new ChannelPoolHandler() {
+ public void channelReleased(Channel ch) {}
+ public void channelAcquired(Channel ch) {}
+ public void channelCreated(Channel ch) {}
+ };
+
+ IdleEvictFixedChannelPool pool =
+ new IdleEvictFixedChannelPool(bootstrap,
+ noopHandler,
+ 2,
+ 5,
+ 60);
+
+ /* 1. Acquire a channel */
+ Channel ch = pool.acquire().sync().getNow();
+
+ /* 2. Release it (Timer starts) */
+ pool.release(ch).sync();
+
+ /* 3. Re-acquire it immediately (Timer should STOP) */
+ Channel reusedCh = pool.acquire().sync().getNow();
+ assertEquals( "Should reuse the same channel", ch, reusedCh);
+
+ /* 4. Simulate the Timeout Event
+ * Even though we fire the event, the handler should not have been
+ * REMOVED
+ */
+ ch.pipeline().fireUserEventTriggered(IdleStateEvent.ALL_IDLE_STATE_EVENT);
+
+ /* 5. Assert Safety */
+ assertTrue("Active channel should NOT be closed", ch.isOpen());
+ assertEquals("Channel should remain acquired",
+ 1, pool.getStats().acquired);
+ pool.release(ch).sync();
+ pool.close();
+ assertStats(pool, 0, 0, 0, 0);
+ }
+
+ @Test
+ public void testMaxConnectionsAndPendingQueue() throws InterruptedException {
+ int numberOfRequests = 5;
+ int maxConnections = 2;
+
+ /* Thread-safe list to hold the channels we successfully acquire */
+ List heldChannels =
+ Collections.synchronizedList(new ArrayList<>());
+
+ /* Latch to wait ONLY for the allowed connections to succeed */
+ CountDownLatch acquiredLatch = new CountDownLatch(maxConnections);
+
+ /* Create Pool */
+ Bootstrap bootstrap = new Bootstrap()
+ .group(group)
+ .channel(LocalChannel.class)
+ .remoteAddress(address);
+
+ /* A dummy user handler (noop) */
+ ChannelPoolHandler noopHandler = new ChannelPoolHandler() {
+ public void channelReleased(Channel ch) {}
+ public void channelAcquired(Channel ch) {}
+ public void channelCreated(Channel ch) {}
+ };
+
+ IdleEvictFixedChannelPool pool =
+ new IdleEvictFixedChannelPool(bootstrap,
+ noopHandler,
+ maxConnections,
+ numberOfRequests + 1,
+ 60);
+ ExecutorService threadPool = Executors.newFixedThreadPool(10);
+
+ /* PHASE 1: Bombard the pool */
+ for (int i = 0; i < numberOfRequests; i++) {
+ threadPool.submit(() -> {
+ Future future = pool.acquire();
+ future.addListener(f -> {
+ if (f.isSuccess()) {
+ heldChannels.add((Channel) f.getNow());
+ acquiredLatch.countDown();
+ }
+ });
+ });
+ }
+ /* Wait for the pool to fill up (Max maxConnections) */
+ boolean success = acquiredLatch.await(5, TimeUnit.SECONDS);
+ if (!success) {
+ throw new RuntimeException("Timeout waiting for initial connections");
+ }
+
+ /* Give a tiny buffer for metrics to settle */
+ Thread.sleep(50);
+
+ /* PHASE 2: Assert Saturation */
+ IdleEvictFixedChannelPool.PoolStats stats = pool.getStats();
+ assertEquals("Total should be capped at max",
+ maxConnections, stats.total);
+ assertEquals("Acquired should be capped at max",
+ maxConnections, stats.acquired);
+ assertEquals("Excess requests should be pending",
+ numberOfRequests - maxConnections, stats.pending);
+
+ /* PHASE 3: Drain the Queue
+ * Now we manually release the channels we were holding.
+ * This should trigger the Pending requests to proceed.
+ */
+
+ /* We need a new latch to verify the REMAINING 3 requests finish
+ * (But we can't easily attach listeners now, so we just check stats)
+ */
+ for (Channel ch : heldChannels) {
+ pool.release(ch);
+ }
+
+ /* Wait a moment for the pending queue to drain */
+ Thread.sleep(200);
+ stats = pool.getStats();
+
+ /* Expect: Pending should be one now. */
+ assertEquals("Pending queue should have 1", 1, stats.pending);
+
+ threadPool.shutdown();
+ pool.close();
+ }
+
+ private void assertStats(IdleEvictFixedChannelPool pool, int t,
+ int a, int i, int p) {
+ IdleEvictFixedChannelPool.PoolStats s = pool.getStats();
+ assertEquals("Total", t, s.total);
+ assertEquals("Acquired", a, s.acquired);
+ assertEquals("Idle", i, s.idle);
+ assertEquals("Pending", p, s.pending);
+ }
+}
From 1b7d683f1872275aeeb3171afc4651fd5c304c56 Mon Sep 17 00:00:00 2001
From: Akshay Sundarraj
Date: Fri, 28 Nov 2025 15:45:42 +0530
Subject: [PATCH 09/11] Async refactor changes-VIII
Connection pooling changes
- Updated ConnectionPool to support maxConnections and maxPending.
Modified acquire and release flow to honor maxConnections and maxPending connections.
Existing keep-alive and idle channel evict should work out of the box.
- Removed IdleEvictFixedChannelPool.java as found it difficult to add keep-alive for this implementation
- Removed IdleEvictFixedChannelPoolTest.java
- Removed unused ResponseHandler.java and RequestState.java
- Disable PerformanceTest for unit testing. Jenkins kvserver is set to maxActiveRequests=100
Files:
M .gitignore
- Added .idea and .oca to ignore list
M driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java
- Reverted deprecation of minPoolSize
M driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
M driver/src/test/java/oracle/nosql/driver/httpclient/ConnectionPoolTest.java
- Added maxConnection and maxPending
M driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java
- Reverted minPool removal
M driver/src/test/java/oracle/nosql/driver/PerformanceTest.java
- Disabled for unit testing
D driver/src/main/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPool.java
D driver/src/main/java/oracle/nosql/driver/httpclient/RequestState.java
D driver/src/main/java/oracle/nosql/driver/httpclient/ResponseHandler.java
D driver/src/test/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPoolTest.java
- Removed unused classes
---
.gitignore | 2 +
.../nosql/driver/NoSQLHandleConfig.java | 5 -
.../driver/httpclient/ConnectionPool.java | 285 ++++++++++----
.../nosql/driver/httpclient/HttpClient.java | 59 ++-
.../httpclient/IdleEvictFixedChannelPool.java | 237 ------------
.../nosql/driver/httpclient/RequestState.java | 81 ----
.../driver/httpclient/ResponseHandler.java | 240 ------------
.../oracle/nosql/driver/PerformanceTest.java | 2 +
.../driver/httpclient/ConnectionPoolTest.java | 355 ++++++++++++++++--
.../IdleEvictFixedChannelPoolTest.java | 308 ---------------
10 files changed, 592 insertions(+), 982 deletions(-)
delete mode 100644 driver/src/main/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPool.java
delete mode 100644 driver/src/main/java/oracle/nosql/driver/httpclient/RequestState.java
delete mode 100644 driver/src/main/java/oracle/nosql/driver/httpclient/ResponseHandler.java
delete mode 100644 driver/src/test/java/oracle/nosql/driver/httpclient/IdleEvictFixedChannelPoolTest.java
diff --git a/.gitignore b/.gitignore
index 4ab57f89..32f9c2ea 100644
--- a/.gitignore
+++ b/.gitignore
@@ -3,3 +3,5 @@ target
*.diff
Fortify*
logging.properties
+.idea
+.oca
diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java
index 1d49449e..a1a7a75e 100644
--- a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java
+++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java
@@ -776,12 +776,7 @@ public NoSQLHandleConfig setConnectionPoolSize(int poolSize) {
* @return this
*
* @since 5.3.2
- * @deprecated since 6.0.0 - The connection pool no longer supports minimum
- * size. Idle connections will be removed based on
- * {@link NoSQLHandleConfig#connectionPoolInactivityPeriod}
- * @see NoSQLHandleConfig#connectionPoolInactivityPeriod
*/
- @Deprecated
public NoSQLHandleConfig setConnectionPoolMinSize(int poolMinSize) {
this.connectionPoolMinSize = poolMinSize;
return this;
diff --git a/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java b/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
index beafebbf..c480cd17 100644
--- a/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
+++ b/driver/src/main/java/oracle/nosql/driver/httpclient/ConnectionPool.java
@@ -8,14 +8,16 @@
package oracle.nosql.driver.httpclient;
import static oracle.nosql.driver.util.LogUtil.logFine;
-import static oracle.nosql.driver.util.LogUtil.logInfo;
import java.util.Map;
+import java.util.Queue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.logging.Logger;
import io.netty.bootstrap.Bootstrap;
@@ -34,11 +36,16 @@
* and tracking of Channels.
*
* Configuration:
- * minSize - actively keep this many alive, even after inactivity, by default
- * this is the number of cores
+ * minSize - actively keep this many alive, even after inactivity, by default,
+ * this is set to 2
* inactivityPeriod - remove inactive channels after this many seconds.
* If negative, don't ever remove them
* Logger
+ * maxSize - Maximum number of connections to create. Once these many channels
+ * are acquired, further channel acquires are put into the pending queue
+ * maxPending - Maximum number of pending acquires. Once pending queue is full
+ * further acquires will fail till channels are released back to the pool
+ *
*
* Usage
* o acquire()
@@ -53,6 +60,9 @@
* release
* o if no Channels are in the queue for acquire a new one is created and
* placed in the queue on release
+ * o During release, if there are pending acquire requests in pending queue,
+ * released channel is used to serve pending request instead of putting back
+ * to the queue
*
* Keep-alive and minimum size
* o if a pool is not a minimal pool a refresh task is created on construction.
@@ -100,11 +110,23 @@ class ConnectionPool {
* closed.
*/
private final Map stats;
- private int acquiredChannelCount;
+ private final AtomicInteger acquiredChannelCount = new AtomicInteger();
/* Executor to run keep-alive task periodically */
private final ScheduledExecutorService keepAlivescheduler;
+ private final int maxPoolSize;
+ private final int maxPending;
+
+ /* State to ensure to maxConnections */
+ private final AtomicInteger currentConnectionCount;
+
+ /* State to ensure to maxPending */
+ private final AtomicInteger pendingAcquireCount;
+
+ /* Queue to track pending acquires */
+ private final Queue> pendingAcquires;
+
/**
* Keepalive callback interface
*/
@@ -122,7 +144,7 @@ interface KeepAlive {
*
* @param bootstrap (netty)
* @param handler the handler, mostly used for event callbacks
- * @param logger
+ * @param logger logger
* @param isMinimalPool set to true if this is a one-time, or minimal time
* use. In this case no refresh task is created
* @param poolMin the minimum size at which the pool should be maintained.
@@ -133,13 +155,17 @@ interface KeepAlive {
* to the minimum (if set). This allows bursty behavior to automatically
* clean up when channels are no longer required. This is more for on-prem
* than the cloud service but applies to both.
+ * @param maxPoolSize maximum number of connections in the pool
+ * @param maxPending maximum number of pending acquires
*/
ConnectionPool(Bootstrap bootstrap,
ChannelPoolHandler handler,
Logger logger,
boolean isMinimalPool,
int poolMin,
- int inactivityPeriodSeconds) {
+ int inactivityPeriodSeconds,
+ int maxPoolSize,
+ int maxPending) {
/* clone bootstrap to set handler */
this.bootstrap = bootstrap.clone();
@@ -166,6 +192,12 @@ protected void initChannel(Channel ch) throws Exception {
queue = new ConcurrentLinkedDeque();
stats = new ConcurrentHashMap();
+ this.maxPoolSize = maxPoolSize;
+ this.maxPending = maxPending;
+ this.currentConnectionCount = new AtomicInteger(0);
+ this.pendingAcquireCount = new AtomicInteger(0);
+ this.pendingAcquires = new ConcurrentLinkedDeque<>();
+
/*
* If not creating a minimal pool run RefreshTask every 30s. A
* minimal pool is short-lived so don't create the overhead.
@@ -215,32 +247,34 @@ final Future acquire() {
* significant time sink in terms of affecting overall latency of this call
*
* Acquired channels are removed from the queue and are "owned" by the
- * caller until released, at which time they are put back on the queue.
+ * caller until released, at which time they are put back on the queue or
+ * serve pending acquires
*/
final Future acquire(final Promise promise) {
try {
- /* this *removes* the channel from the queue */
- final Channel channel = queue.pollFirst();
- if (channel == null) {
- /* need a new Channel */
- Bootstrap bs = bootstrap.clone();
- ChannelFuture fut = bs.connect();
- if (fut.isDone()) {
- notifyOnConnect(fut, promise);
- } else {
- fut.addListener((ChannelFutureListener) future ->
- notifyOnConnect(future, promise));
+ /* 1. Try to get a free channel from the idle pool (LIFO) */
+ Channel channel = queue.pollFirst();
+ if (channel != null) {
+ activateChannel(channel, promise);
+ return promise;
+ }
+
+ /* 2. Pool is empty.
+ * Try to create a new connection respecting maxPoolSize.
+ */
+ while (true) {
+ int current = currentConnectionCount.get();
+ if (current >= maxPoolSize) {
+ /* Pool is full. Enqueue the request and return */
+ enqueueRequest(promise);
+ return promise;
}
- } else {
- /*
- * This logic must happen in the event loop
- */
- EventLoop loop = channel.eventLoop();
- if (loop.inEventLoop()) {
- checkChannel(channel, promise);
- } else {
- loop.execute(() -> checkChannel(channel, promise));
+ /* CAS (Compare-And-Swap) to reserve a slot */
+ if (currentConnectionCount.compareAndSet(current, current + 1)) {
+ createConnection(promise);
+ return promise;
}
+ /* If CAS failed, loop retry */
}
} catch (Throwable t) {
promise.tryFailure(t);
@@ -249,21 +283,77 @@ final Future acquire(final Promise promise) {
}
/**
- * Release a channel. This is not async. The channel is added to the
- * front of the queue. This class implements a LIFO algorithm to ensure
- * that the first, or first few channels on the queue remain active and
- * are not subject to inactivity timeouts from the server side.
- * Note that inactive released channels will be closed and not
- * re-added to the queue.
+ * Helper to safely enqueue pending requests.
+ */
+ private void enqueueRequest(Promise promise) {
+ /* Atomic check-then-act */
+ if (pendingAcquireCount.incrementAndGet() > maxPending) {
+ /* Rollback and fail */
+ pendingAcquireCount.decrementAndGet();
+ promise.tryFailure(new IllegalStateException(
+ "Pending acquire queue has reached its maximum size of "
+ + maxPending));
+ } else {
+ pendingAcquires.add(promise);
+ }
+ }
+
+ /**
+ * Helper to create a new connection.
+ */
+ private void createConnection(Promise promise) {
+ Bootstrap bs = bootstrap.clone();
+ ChannelFuture fut = bs.connect();
+ if (fut.isDone()) {
+ notifyOnConnect(fut, promise);
+ } else {
+ fut.addListener((ChannelFutureListener) future ->
+ notifyOnConnect(future, promise));
+ }
+ }
+
+ /**
+ * Release a channel. This is not async.
+ *