From 21a29525569bd8f50ddd10a4227fcf1076b8f7dc Mon Sep 17 00:00:00 2001 From: Cezar Andrei Date: Thu, 26 Aug 2021 18:28:46 -0500 Subject: [PATCH 01/13] Add client statistics. Stats are collected on the client application, when system property -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile=[regular|full] is used, for an interval of time, set by -Dcom.oracle.nosql.sdk.nosqldriver.stats.interval=600 in seconds. The following stats are collected for each type of request: count, wire latencies, request and response sizes, errors, retries, retry delay, rate limit delay. Also if rateLimiting is enabled and concurent connections number. StatsConfig also provides an API to programaticaly configure the statistics collection, to selectively start and stop collection and get access to collected statistics. --- .../java/oracle/nosql/driver/NoSQLHandle.java | 9 + .../java/oracle/nosql/driver/StatsConfig.java | 143 +++++++ .../java/oracle/nosql/driver/http/Client.java | 57 ++- .../nosql/driver/http/NoSQLHandleImpl.java | 6 + .../java/oracle/nosql/driver/http/Stats.java | 370 ++++++++++++++++++ .../nosql/driver/http/StatsConfigImpl.java | 230 +++++++++++ .../nosql/driver/httpclient/HttpClient.java | 4 + .../java/oracle/nosql/driver/ops/Request.java | 9 + 8 files changed, 819 insertions(+), 9 deletions(-) create mode 100644 driver/src/main/java/oracle/nosql/driver/StatsConfig.java create mode 100644 driver/src/main/java/oracle/nosql/driver/http/Stats.java create mode 100644 driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java index 62b75e3d..2f467a7c 100644 --- a/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java +++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java @@ -571,6 +571,15 @@ public SystemResult doSystemRequest(String statement, int timeoutMs, int pollIntervalMs); + /** + * Returns configuration object for client side statistics. + * Client side statistics, when enabled, are INFO generated entries on the + * logger at regular time interval. + * + * @return configuration object for client side statistics + */ + public StatsConfig getStatsConfig(); + /** * Closes the handle, releasing its memory and network resources. Once * this method is closed the handle is no longer usable. Any attempt to diff --git a/driver/src/main/java/oracle/nosql/driver/StatsConfig.java b/driver/src/main/java/oracle/nosql/driver/StatsConfig.java new file mode 100644 index 00000000..4ecde9df --- /dev/null +++ b/driver/src/main/java/oracle/nosql/driver/StatsConfig.java @@ -0,0 +1,143 @@ +package oracle.nosql.driver; + +import java.util.function.Consumer; +import java.util.logging.Logger; + +import oracle.nosql.driver.values.FieldValue; + +/** + *

This interface allows user to setup the collection of statistics on the + * client side.

+ * + * The statistics data is collected for an interval of time. At the end of the + * interval, the stats data is logged in a specified JSON format that can be + * filtered and parsed. After the logging, the counters are cleared and + * collection of data resumes.

+ * + * Collection intervals are aligned to the top of the hour. This means first + * interval logs may contain stats for a shorter interval.

+ * + * Collection of stats are controlled by the following system properties:

  • + * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile=[none|regular|full] + * Specifies the stats profile: none - disabled, + * regular - all except 95th and 99th percentile and + * full - all including 95th and 99th percentile.
  • + * -Dcom.oracle.nosql.sdk.nosqldriver.stats.interval=600 Interval in + * seconds to log the stats, by default is 10 minutes.
  • + * -Dcom.oracle.nosql.sdk.nosqldriver.stats.pretty-print=true Option + * to enable pretty printing of the JSON data, default value is false
  • + * + * Collection of stats can also be used by using the API: + * NoSQLHandleConfig config = new NoSQLHandleConfig( endpoint ); + * NoSQLHandle handle = NoSQLHandleFactory.createNoSQLHandle(config); + * + * StatsConfig statsConfig = handle.getStatsConfig(); + * statsConfig.setProfile(StatsConfig.Profile.REGULAR); + * statsConfig.setInterval(600); + * statsConfig.setPrettyPrint(false); + * statsConfig.registerHandler( + * new StatsConfig.StatsHandler() { + * @Override + * public void accept(FieldValue jsonStats) { + * System.out.println("!!! Got a stat: " + jsonStats); + * } + * }); + * statsConfig.start(); + * + * //... application code + * + * statsConfig.stop(); + * handle.close(); + *

    + */ +public interface StatsConfig { + + /** + * The following semantics are attached to the Profile: + * - NONE: no stats are logged. + * - REGULAR: all stats except 95th and 99th percentile latencies. + * - FULL: all stats including 95th and 99th percentile latencies. + */ + enum Profile { + NONE, REGULAR, FULL; + } + + /** + * Handler interface that user can register to get access to stats at + * the end of the interval. + */ + interface StatsHandler extends Consumer { + /** Stats are encoded in JSON format using the FieldValue API. */ + void accept(FieldValue jsonStats); + } + + /** + * Sets the logger to be used. + */ + StatsConfig setLogger(Logger logger); + + /** + * Returns the current logger. + */ + Logger getLogger(); + + /** + * Sets interval size in seconds. + * Default interval is 600 seconds, i.e. 10 min. + */ + StatsConfig setInterval(int interval); + + + /** + * Returns the current collection interval. + * Default interval is 600 seconds, i.e. 10 min. + */ + int getInterval(); + + /** + * Set the collection profile. + * Default profile is NONE. + */ + StatsConfig setProfile(Profile profile); + + /** + * Returns the collection profile. + * Default profile is NONE. + */ + Profile getProfile(); + + /** + * Enable JSON pretty print for easier human reading. + * Default is disabled. + */ + StatsConfig setPrettyPrint(boolean enablePrettyPrint); + + /** + * Returns the current JSON pretty print flag. + * Default is disabled. + */ + boolean getPrettyPrint(); + + /** + * Registers a stats handler. + * @param handler User defined StatsHandler. + */ + void registerHandler(StatsHandler handler); + + /** + * Collection of stats is enabled only between start and stop or from the + * beginning if system property + * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile= is "regular" or "full". + */ + void start(); + + /** + * Stops collection of stats. + */ + void stop(); + + /** + * Returns true if collection of stats is enabled, otherwise returns false. + */ + boolean isStarted(); +} \ No newline at end of file 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 17f73ee6..487361bb 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/Client.java +++ b/driver/src/main/java/oracle/nosql/driver/http/Client.java @@ -11,10 +11,6 @@ import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1; import static java.nio.charset.StandardCharsets.UTF_8; import static oracle.nosql.driver.util.CheckNull.requireNonNull; -import static oracle.nosql.driver.util.LogUtil.isLoggable; -import static oracle.nosql.driver.util.LogUtil.logFine; -import static oracle.nosql.driver.util.LogUtil.logInfo; -import static oracle.nosql.driver.util.LogUtil.logTrace; import static oracle.nosql.driver.util.HttpConstants.ACCEPT; import static oracle.nosql.driver.util.HttpConstants.CONNECTION; import static oracle.nosql.driver.util.HttpConstants.CONTENT_LENGTH; @@ -22,14 +18,18 @@ import static oracle.nosql.driver.util.HttpConstants.NOSQL_DATA_PATH; import static oracle.nosql.driver.util.HttpConstants.REQUEST_ID_HEADER; import static oracle.nosql.driver.util.HttpConstants.USER_AGENT; +import static oracle.nosql.driver.util.LogUtil.isLoggable; +import static oracle.nosql.driver.util.LogUtil.logFine; +import static oracle.nosql.driver.util.LogUtil.logInfo; +import static oracle.nosql.driver.util.LogUtil.logTrace; import java.io.IOException; import java.net.URL; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executors; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; @@ -49,6 +49,7 @@ import oracle.nosql.driver.RetryHandler; import oracle.nosql.driver.RetryableException; import oracle.nosql.driver.SecurityInfoNotReadyException; +import oracle.nosql.driver.StatsConfig; import oracle.nosql.driver.WriteThrottlingException; import oracle.nosql.driver.httpclient.HttpClient; import oracle.nosql.driver.httpclient.ResponseHandler; @@ -150,6 +151,11 @@ public class Client { */ private ExecutorService threadPool; + /** + * config for statistics + */ + private StatsConfigImpl statsConfig; + public Client(Logger logger, NoSQLHandleConfig httpConfig) { @@ -218,6 +224,9 @@ public Client(Logger logger, tableLimitUpdateMap = null; threadPool = null; } + + statsConfig = new StatsConfigImpl(config.getLibraryVersion(), logger, + httpClient, httpConfig.getRateLimitingEnabled()); } /** @@ -231,6 +240,7 @@ public void shutdown() { return; } httpClient.shutdown(); + statsConfig.shutdown(); if (authProvider != null) { authProvider.close(); } @@ -289,7 +299,7 @@ public Result execute(Request kvRequest) { /* * The following "if" may be true for advanced queries only. For - * such qyeries, the "if" will be true (i.e., the QueryRequest will + * such queries, the "if" will be true (i.e., the QueryRequest will * be bound with a QueryDriver) if and only if this is not the 1st * execute() call for this query. In this case we just return a new, * empty QueryResult. Actual computation of a result batch will take @@ -373,6 +383,7 @@ public Result execute(Request kvRequest) { final long startTime = System.currentTimeMillis(); kvRequest.setStartTimeMs(startTime); + final String requestClass = kvRequest.getClass().getSimpleName(); do { long thisTime = System.currentTimeMillis(); @@ -426,6 +437,7 @@ public Result execute(Request kvRequest) { ResponseHandler responseHandler = null; ByteBuf buffer = null; + long wireTime; try { /* * NOTE: the ResponseHandler will release the Channel @@ -476,9 +488,10 @@ public Result execute(Request kvRequest) { false /* Don't validate hdrs */); HttpHeaders headers = request.headers(); addCommonHeaders(headers); + int contentLength = buffer.readableBytes(); headers.add(HttpHeaderNames.HOST, host) .add(REQUEST_ID_HEADER, requestId) - .setInt(CONTENT_LENGTH, buffer.readableBytes()); + .setInt(CONTENT_LENGTH, contentLength); /* * If the request doesn't set an explicit compartment, use @@ -490,10 +503,10 @@ public Result execute(Request kvRequest) { } authProvider.setRequiredHeaders(authString, kvRequest, headers); - final String requestClass = kvRequest.getClass().getSimpleName(); if (isLoggable(logger, Level.FINE)) { logTrace(logger, "Request: " + requestClass); } + wireTime = System.currentTimeMillis(); httpClient.runRequest(request, responseHandler, channel); boolean isTimeout = @@ -508,9 +521,12 @@ public Result execute(Request kvRequest) { responseHandler.getStatus()); } + ByteBuf wireContent = responseHandler.getContent(); Result res = processResponse(responseHandler.getStatus(), - responseHandler.getContent(), + wireContent, kvRequest); + int resSize = wireContent.readerIndex(); + wireTime = System.currentTimeMillis() - wireTime; if (res instanceof TableResult && rateLimiterMap != null) { /* update rate limiter settings for table */ @@ -536,6 +552,10 @@ public Result execute(Request kvRequest) { /* copy retry stats to Result on successful operation */ res.setRetryStats(kvRequest.getRetryStats()); + kvRequest.setRateLimitDelayedMs(rateDelayedMs); + + statsConfig.logReqStat(kvRequest, wireTime, + contentLength, resSize); return res; @@ -550,6 +570,8 @@ public Result execute(Request kvRequest) { exception = rae; continue; } + kvRequest.setRateLimitDelayedMs(rateDelayedMs); + statsConfig.logReqStatError(kvRequest); logInfo(logger, "Unexpected authentication exception: " + rae); throw new NoSQLException("Unexpected exception: " + @@ -611,10 +633,14 @@ public Result execute(Request kvRequest) { exception = re; continue; } catch (NoSQLException nse) { + kvRequest.setRateLimitDelayedMs(rateDelayedMs); + statsConfig.logReqStatError(kvRequest); logFine(logger, "Client execute NoSQLException: " + nse.getMessage()); throw nse; /* pass through */ } catch (RuntimeException e) { + kvRequest.setRateLimitDelayedMs(rateDelayedMs); + statsConfig.logReqStatError(kvRequest); logFine(logger, "Client execute runtime exception: " + e.getMessage()); throw e; @@ -637,12 +663,16 @@ public Result execute(Request kvRequest) { continue; } catch (InterruptedException ie) { + kvRequest.setRateLimitDelayedMs(rateDelayedMs); + statsConfig.logReqStatError(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) { + kvRequest.setRateLimitDelayedMs(rateDelayedMs); + statsConfig.logReqStatError(kvRequest); logInfo(logger, "Unable to execute request: " + ee.getCause().getMessage()); /* is there a better exception? */ @@ -680,6 +710,8 @@ public Result execute(Request kvRequest) { } } while (! timeoutRequest(startTime, timeoutMs, exception)); + kvRequest.setRateLimitDelayedMs(rateDelayedMs); + statsConfig.logReqStatError(kvRequest); throw new RequestTimeoutException(timeoutMs, "Request timed out after " + kvRequest.getNumRetries() + (kvRequest.getNumRetries() == 1 ? " retry. " : " retries. ") + @@ -1152,4 +1184,11 @@ public void enableRateLimiting(boolean enable, double usePercent) { } } } + + /** + * Returns the config object for client statistics. + */ + StatsConfig getStatsConfig() { + return statsConfig; + } } 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 cace4a86..289a0265 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java +++ b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java @@ -15,6 +15,7 @@ import oracle.nosql.driver.AuthorizationProvider; import oracle.nosql.driver.NoSQLHandle; import oracle.nosql.driver.NoSQLHandleConfig; +import oracle.nosql.driver.StatsConfig; import oracle.nosql.driver.UserInfo; import oracle.nosql.driver.iam.SignatureProvider; import oracle.nosql.driver.kv.StoreAccessTokenProvider; @@ -368,6 +369,11 @@ public SystemResult doSystemRequest(String statement, return dres; } + @Override + public StatsConfig getStatsConfig() { + return client.getStatsConfig(); + } + /** * Ensure that the client exists and hasn't been closed; */ diff --git a/driver/src/main/java/oracle/nosql/driver/http/Stats.java b/driver/src/main/java/oracle/nosql/driver/http/Stats.java new file mode 100644 index 00000000..f228eb83 --- /dev/null +++ b/driver/src/main/java/oracle/nosql/driver/http/Stats.java @@ -0,0 +1,370 @@ +package oracle.nosql.driver.http; + +import java.sql.Timestamp; +import java.time.LocalTime; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.logging.Level; + +import oracle.nosql.driver.StatsConfig; +import oracle.nosql.driver.values.ArrayValue; +import oracle.nosql.driver.values.FieldValue; +import oracle.nosql.driver.values.JsonOptions; +import oracle.nosql.driver.values.MapValue; +import oracle.nosql.driver.values.StringValue; +import oracle.nosql.driver.values.TimestampValue; + +public class Stats { + + private ScheduledExecutorService service; + private StatsConfigImpl statsConfig; + + private long startTime; + private long endTime; + private Map requests; + private ConnectionStats connectionStats; + + private static class ReqStats { + long count; + long errors; + int reqSizeMin = Integer.MAX_VALUE; + int reqSizeMax; + float reqSizeAvg; + int resSizeMin = Integer.MAX_VALUE; + int resSizeMax; + float resSizeAvg; + int retryAuthCount; + int retryThrottleCouunt; + int retryTotalCount; + int retryTotalDelay; + int totalRateLimitDelay; + long wireLatencyMin = Long.MAX_VALUE; + long wireLatencyMax; + float wireLatencyAvg; + Percentile wireLatencyPercentile; + } + + static class Percentile { + List values; + + void addValue(long value) { + if (values == null) { + values = new ArrayList<>(); + } + values.add(value); + } + + long getPercentile(double percentile) { + if (values == null || values.size() == 0) { + return -1; + } + + if (values.size() == 1) { + return values.get(0); + } + + values.sort(Comparator.comparingLong(Long::longValue)); + int index = (int)Math.round(percentile * values.size()); + if (index >= values.size()) { + index = values.size() -1; + } + return values.get(index); + } + + long get95thPercentile() { + return getPercentile(0.95d); + } + + long get99thPercentile() { + return getPercentile(0.99d); + } + } + + private static class ConnectionStats { + long count; + int min = Integer.MAX_VALUE; + int max; + float avg; + + public void clear() { + count = 0; + min = Integer.MAX_VALUE; + max = 0; + avg = 0; + } + } + + Stats(StatsConfigImpl statsConfig) { + this.statsConfig = statsConfig; + + requests = new HashMap<>(); + connectionStats = new ConnectionStats(); + + Runnable runnable = new Runnable() { + public void run() { + try { + logClientStats(); + } catch (RuntimeException re) { + statsConfig.getLogger().log(Level.INFO, + re.getMessage()); + } + } + }; + + LocalTime localTime = LocalTime.now(); + + long delay = 1000l * statsConfig.getInterval() - + (( 1000l * 60l * localTime.getMinute() + + 1000l * localTime.getSecond() + + localTime.getNano() / 1000000l) % + (1000l * statsConfig.getInterval())); + + service = Executors + .newSingleThreadScheduledExecutor(); + service.scheduleAtFixedRate(runnable, + delay, + 1000l * statsConfig.getInterval(), + TimeUnit.MILLISECONDS); + + startTime = System.currentTimeMillis(); + } + + void logClientStats() { + endTime = System.currentTimeMillis(); + FieldValue fvStats = generateFieldValueStats(); + String json = fvStats.toJson(statsConfig.getPrettyPrint() ? + JsonOptions.PRETTY : null); + + /* Call user handle if configured. */ + StatsConfig.StatsHandler statsHandler = + statsConfig.getHandler(); + if (statsHandler != null) { + statsHandler.accept(fvStats); + } + + /* start from scratch in the new interval */ + clearStats(); + statsConfig.getLogger().log(Level.INFO, statsConfig.LOG_PREFIX + json); + } + + private FieldValue generateFieldValueStats() { + MapValue root = new MapValue(); + Timestamp ts = new Timestamp(startTime); + ts.setNanos(0); + root.put("startTime", new TimestampValue(ts)); + ts = new Timestamp(endTime); + ts.setNanos(0); + root.put("endTime", new TimestampValue(ts)); + root.put("clientId", new StringValue(statsConfig.getId())); + + if (connectionStats.count > 0) { + MapValue connections = new MapValue(); + connections.put("min", connectionStats.min); + connections.put("max", connectionStats.max); + connections.put("avg", connectionStats.avg); + root.put("connections", connections); + } + + Set> entries; + synchronized (requests) { + entries = new HashSet<>(requests.entrySet()); + } + + if (entries.size() > 0) { + root.put("rateLimitingEnabled", + statsConfig.isRateLimitingEnabled()); + + ArrayValue reqArray = new ArrayValue(); + root.put("requests", reqArray); + + entries.forEach(e -> { + + String k = e.getKey(); + ReqStats v = e.getValue(); + + MapValue req = new MapValue(); + req.put("name", k); + req.put("count", v.count); + req.put("errors", v.errors); + + MapValue retry = new MapValue(); + retry.put("totalCount", v.retryTotalCount); + retry.put("totalDelay", v.retryTotalDelay); + retry.put("authCount", v.retryAuthCount); + retry.put("throttleCount", v.retryThrottleCouunt); + req.put("retry", retry); + req.put("totalRateLimitDelay", v.totalRateLimitDelay); + + if (v.wireLatencyMax > 0) { + MapValue latency = new MapValue(); + latency.put("min", v.wireLatencyMin); + latency.put("max", v.wireLatencyMax); + latency.put("avg", v.wireLatencyAvg); + if (v.wireLatencyPercentile != null) { + latency + .put("95th", + v.wireLatencyPercentile.get95thPercentile()); + latency + .put("99th", + v.wireLatencyPercentile.get99thPercentile()); + } + req.put("wireLatency", latency); + } + + if (v.reqSizeMax > 0) { + MapValue reqSize = new MapValue(); + reqSize.put("min", v.reqSizeMin); + reqSize.put("max", v.reqSizeMax); + reqSize.put("avg", v.reqSizeAvg); + req.put("reqSize", reqSize); + } + + if (v.resSizeMax > 0) { + MapValue resSize = new MapValue(); + resSize.put("min", v.resSizeMin); + resSize.put("max", v.resSizeMax); + resSize.put("avg", v.resSizeAvg); + req.put("resSize", resSize); + } + + reqArray.add(req); + }); + } + + return root; + } + + void clearStats() { + synchronized (requests) { + startTime = System.currentTimeMillis(); + endTime = 0; + requests.clear(); + connectionStats.clear(); + } + } + + void addReqStatError(String type, int retryCount, + int retryDelay, int retryAuth, int retryThrotle, int rateLimitDelay, + int connections) { + addReqStat(type, true, retryCount, retryDelay, retryAuth, retryThrotle, + rateLimitDelay, connections, -1, -1,-1); + } + + /** + * Adds a new statistic entry. Can be of 2 types: successful or error. + * Request, result sizes and wireLatency are not registered for error entries. + * + * @param requestClass Type of request. + * @param error Hard error, ie. return error to user. + * @param retryAuth Number of retryAuth errors. + * @param retryThrottle Number of retryThrottle errors. + * @param reqSize Request size in bytes. + * @param resSize Result size in bytes. + * @param wireLatency Latency on the wire, in milliseconds, it doesn't + * include retry delay or rate limit delay. + */ + void addReqStat(String requestClass, boolean error, int retries, + int retryDelay, int rateLimitDelay, int retryAuth, int retryThrottle, + int connections, int reqSize, int resSize, long wireLatency) { + + ReqStats rStat; + requestClass = requestClass != null && requestClass.endsWith("Request") + ? requestClass.substring(0, requestClass.length() - 7) : + requestClass; + synchronized (requests) { + rStat = requests.get(requestClass); + if (rStat == null) { + rStat = new ReqStats(); + requests.put(requestClass, rStat); + + if (statsConfig.getProfile() == StatsConfig.Profile.FULL) { + rStat.wireLatencyPercentile = new Percentile(); + } + } + } + + synchronized (rStat) { + rStat.count++; + rStat.retryTotalCount += retries; + rStat.retryTotalDelay += retryDelay; + rStat.retryAuthCount += retryAuth; + rStat.retryThrottleCouunt += retryThrottle; + rStat.totalRateLimitDelay += rateLimitDelay; + if (error) { + rStat.errors ++; + } else { + if (reqSize < rStat.reqSizeMin) { + rStat.reqSizeMin = reqSize; + } + if (reqSize > rStat.reqSizeMax) { + rStat.reqSizeMax = reqSize; + } + + if (resSize < rStat.resSizeMin) { + rStat.resSizeMin = resSize; + } + if (resSize > rStat.resSizeMax) { + rStat.resSizeMax = resSize; + } + + if (wireLatency < rStat.wireLatencyMin) { + rStat.wireLatencyMin = wireLatency; + } + if (wireLatency > rStat.wireLatencyMax) { + rStat.wireLatencyMax = wireLatency; + } + + if (rStat.wireLatencyPercentile != null) { + rStat.wireLatencyPercentile.addValue(wireLatency); + } + + if (rStat.count == 0) { + rStat.reqSizeAvg = reqSize; + rStat.resSizeAvg = resSize; + rStat.wireLatencyAvg = wireLatency; + } else { + rStat.reqSizeAvg = rStat.reqSizeAvg + + (reqSize - rStat.reqSizeAvg) / + (rStat.count - rStat.errors); + + rStat.resSizeAvg = rStat.resSizeAvg + + (resSize - rStat.resSizeAvg) / + (rStat.count - rStat.errors); + + rStat.wireLatencyAvg = rStat.wireLatencyAvg + + (wireLatency - rStat.wireLatencyAvg) / + (rStat.count - rStat.errors); + } + } + } + + synchronized (connectionStats) { + if (connections < connectionStats.min) { + connectionStats.min = connections; + } + if (connections > connectionStats.max) { + connectionStats.max = connections; + } + if (connectionStats.count == 0) { + connectionStats.avg = connections; + } else { + connectionStats.avg = connectionStats.avg + + (connections - connectionStats.avg) / + connectionStats.count; + } + connectionStats.count++; + } + } + + void shutdown() { + service.shutdown(); + } +} diff --git a/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java b/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java new file mode 100644 index 00000000..f9465554 --- /dev/null +++ b/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java @@ -0,0 +1,230 @@ +package oracle.nosql.driver.http; + +import java.util.UUID; +import java.util.logging.Level; +import java.util.logging.Logger; + +import oracle.nosql.driver.SecurityInfoNotReadyException; +import oracle.nosql.driver.StatsConfig; +import oracle.nosql.driver.ThrottlingException; +import oracle.nosql.driver.httpclient.HttpClient; +import oracle.nosql.driver.kv.AuthenticationException; +import oracle.nosql.driver.ops.Request; +import oracle.nosql.driver.ops.RetryStats; + +public class StatsConfigImpl + implements StatsConfig { + + private final static String PROFILE_PROPERTY = + "com.oracle.nosql.sdk.nosqldriver.stats.profile"; + private final static String INTERVAL_PROPERTY = + "com.oracle.nosql.sdk.nosqldriver.stats.interval"; + private final static String PRETTY_PRINT_PROPERTY = + "com.oracle.nosql.sdk.nosqldriver.stats.pretty-print"; + final static String LOG_PREFIX = "ONJS:Monitoring stats|"; + + private StatsConfig.Profile profile = Profile.NONE; + /* Time interval to log in seconds. Default 600, ie. 10 minutes. */ + private int interval = 600; + private boolean prettyPrint = false; + + private Logger logger; + private HttpClient httpClient; /* required for connections */ + private boolean rateLimitingEnabled; + private String id = Integer.toHexString(UUID.randomUUID().hashCode()); + private StatsHandler statsHandler; + private boolean enableCollection = false; + private Stats stats; + + StatsConfigImpl(String libraryVersion, Logger logger, + HttpClient httpClient, boolean rateLimitingEnabled) { + this.logger = logger; + this.httpClient = httpClient; + this.rateLimitingEnabled = rateLimitingEnabled; + + String profileProp = System.getProperty(PROFILE_PROPERTY); + if (profileProp != null) { + try { + setProfile(Profile.valueOf(profileProp.toUpperCase())); + } catch (IllegalArgumentException iae) { + logger.log(Level.SEVERE, LOG_PREFIX + "Invalid profile " + + "value for system property " + PROFILE_PROPERTY + ": " + + profileProp); + } + } + + String intervalProp = System.getProperty(INTERVAL_PROPERTY); + if (intervalProp != null) { + try { + setInterval(Integer.valueOf(intervalProp)); + } catch (NumberFormatException nfe) { + logger.log(Level.SEVERE, "Invalid integer value for system " + + "property " + INTERVAL_PROPERTY + ": " + intervalProp); + } + } + + String ppProp = System.getProperty(PRETTY_PRINT_PROPERTY); + if (ppProp != null && ("true".equals(ppProp.toLowerCase()) || "1".equals(ppProp) || + "on".equals(ppProp.toLowerCase()))) { + prettyPrint = Boolean.valueOf(ppProp); + } + + if (profile != Profile.NONE) { + logger.setLevel(Level.INFO); + logger.log(Level.INFO, LOG_PREFIX + + "{\"sdkName\"=\"Oracle NoSQL SDK for Java\", " + + "\"sdkVersion\":\"" + libraryVersion + "\", " + + "clientId=\"" + id + "\",\"profile\":\"" + profile + "\", " + + "\"intervalSec\"=" + interval + " \"prettyPrint\"=" + + prettyPrint + "}"); + + start(); + } + } + + @Override + public StatsConfig setLogger(Logger logger) { + this.logger = logger; + return this; + } + + @Override + public Logger getLogger() { + return logger; + } + + @Override + public StatsConfig setInterval(int interval) { + if (interval < 1) { + throw new IllegalArgumentException("Stats interval can not be " + + "less than 1 second."); + } + this.interval = interval; + return this; + } + + @Override + public int getInterval() { + return interval; + } + + @Override + public StatsConfig setProfile(Profile profile) { + this.profile = profile; + return this; + } + + @Override + public Profile getProfile() { + return profile; + } + + @Override + public StatsConfig setPrettyPrint(boolean enablePrettyPrint) { + this.prettyPrint = enablePrettyPrint; + return this; + } + + @Override + public boolean getPrettyPrint() { + return prettyPrint; + } + + @Override + public void registerHandler(StatsHandler statsHandler) { + this.statsHandler = statsHandler; + } + + public StatsHandler getHandler() { + return statsHandler; + } + + @Override + public void start() { + if (profile == Profile.NONE) { + stats = null; + } else if (stats == null) { + stats = new Stats(this); + enableCollection = true; + } + } + + @Override + public void stop() { + enableCollection = false; + } + + @Override + public boolean isStarted() { + return enableCollection; + } + + public String getId() { + return id; + } + + public void shutdown() { + if (stats != null) { + stats.shutdown(); + } + } + + public void logReqStat(Request kvRequest, long wireTime, + int reqSize, int resSize) { + if (stats != null && enableCollection) { + String requestClass = kvRequest.getClass().getSimpleName(); + int auth = 0, throttle = 0, retries = 0, retryDelay = 0; + RetryStats retryStats = kvRequest.getRetryStats(); + if (retryStats != null) { + + auth = retryStats.getNumExceptions( + AuthenticationException.class); + auth += retryStats.getNumExceptions( + SecurityInfoNotReadyException.class); + + throttle = retryStats.getNumExceptions( + ThrottlingException.class); + + retries = retryStats.getRetries(); + retryDelay = retryStats.getDelayMs(); + } + + int rateLimitDelay = kvRequest.getRateLimitDelayedMs(); + + // using direct measured http call instead of total - delays + stats.addReqStat(requestClass, false, retries, retryDelay, + rateLimitDelay, auth, throttle, + httpClient.getAcquiredChannelCount(), + reqSize, resSize, wireTime); + } + } + + public void logReqStatError(Request kvRequest) { + if (stats != null && enableCollection) { + String requestClass = kvRequest.getClass().getSimpleName(); + int auth = 0, throttle = 0, retryCount = 0, retryDelay = 0; + RetryStats retryStats = kvRequest.getRetryStats(); + if (retryStats != null) { + auth = retryStats.getNumExceptions( + AuthenticationException.class); + auth += retryStats.getNumExceptions( + SecurityInfoNotReadyException.class); + + throttle = retryStats.getNumExceptions( + ThrottlingException.class); + + retryCount = retryStats.getRetries(); + retryDelay = retryStats.getDelayMs(); + } + + int rateLimitDelay = kvRequest.getRateLimitDelayedMs(); + + stats.addReqStatError(requestClass, retryCount, retryDelay, auth, + throttle, rateLimitDelay, httpClient.getAcquiredChannelCount()); + } + } + + public boolean isRateLimitingEnabled() { + return rateLimitingEnabled; + } +} 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 9076e4a2..14a8b0f9 100644 --- a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java +++ b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java @@ -363,4 +363,8 @@ public void operationComplete(ChannelFuture future) { } }); } + + public int getAcquiredChannelCount() { + return pool.acquiredChannelCount(); + } } 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 0b8f0f7c..6be80b3c 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/Request.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/Request.java @@ -64,6 +64,7 @@ public abstract class Request { * @hidden */ private RateLimiter writeRateLimiter; + private int rateLimitDelayedMs; protected Request() {} @@ -407,4 +408,12 @@ public void setStartTimeMs(long ms) { public long getStartTimeMs() { return startTimeMs; } + + public void setRateLimitDelayedMs(int rateLimitDelayedMs) { + this.rateLimitDelayedMs = rateLimitDelayedMs; + } + + public int getRateLimitDelayedMs() { + return rateLimitDelayedMs; + } } From 6f5d0b23e48b785f2203088106c14687ba4b65ff Mon Sep 17 00:00:00 2001 From: Cezar Andrei Date: Fri, 27 Aug 2021 14:56:35 -0500 Subject: [PATCH 02/13] Moved rateLimitingEnabled from interval log to initial log output. Fixed throttle counter bug. Misc javadoc and clearer param and var names. --- .../java/oracle/nosql/driver/http/Stats.java | 38 ++++++++++++------- .../nosql/driver/http/StatsConfigImpl.java | 23 +++++------ 2 files changed, 34 insertions(+), 27 deletions(-) diff --git a/driver/src/main/java/oracle/nosql/driver/http/Stats.java b/driver/src/main/java/oracle/nosql/driver/http/Stats.java index f228eb83..0a3b4fa0 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/Stats.java +++ b/driver/src/main/java/oracle/nosql/driver/http/Stats.java @@ -42,7 +42,7 @@ private static class ReqStats { int resSizeMax; float resSizeAvg; int retryAuthCount; - int retryThrottleCouunt; + int retryThrottleCount; int retryTotalCount; int retryTotalDelay; int totalRateLimitDelay; @@ -179,8 +179,6 @@ private FieldValue generateFieldValueStats() { } if (entries.size() > 0) { - root.put("rateLimitingEnabled", - statsConfig.isRateLimitingEnabled()); ArrayValue reqArray = new ArrayValue(); root.put("requests", reqArray); @@ -199,7 +197,7 @@ private FieldValue generateFieldValueStats() { retry.put("totalCount", v.retryTotalCount); retry.put("totalDelay", v.retryTotalDelay); retry.put("authCount", v.retryAuthCount); - retry.put("throttleCount", v.retryThrottleCouunt); + retry.put("throttleCount", v.retryThrottleCount); req.put("retry", retry); req.put("totalRateLimitDelay", v.totalRateLimitDelay); @@ -242,6 +240,9 @@ private FieldValue generateFieldValueStats() { return root; } + /** + * Clear all collected stats. + */ void clearStats() { synchronized (requests) { startTime = System.currentTimeMillis(); @@ -251,11 +252,19 @@ void clearStats() { } } - void addReqStatError(String type, int retryCount, - int retryDelay, int retryAuth, int retryThrotle, int rateLimitDelay, + /** + * Adds a new error statistic entry. When error we don't track request, + * response sizes and latency. + * + * @param requestClass Type of request. + * @param authCount Number of authCount errors. + * @param throttleCount Number of throttleCount errors. + */ + void addReqStatError(String requestClass, int retryCount, + int retryDelay, int authCount, int throttleCount, int rateLimitDelay, int connections) { - addReqStat(type, true, retryCount, retryDelay, retryAuth, retryThrotle, - rateLimitDelay, connections, -1, -1,-1); + addReqStat(requestClass, true, retryCount, retryDelay, rateLimitDelay, + authCount, throttleCount, connections, -1, -1,-1); } /** @@ -264,15 +273,15 @@ void addReqStatError(String type, int retryCount, * * @param requestClass Type of request. * @param error Hard error, ie. return error to user. - * @param retryAuth Number of retryAuth errors. - * @param retryThrottle Number of retryThrottle errors. + * @param authCount Number of authCount errors. + * @param throttleCount Number of throttleCount errors. * @param reqSize Request size in bytes. * @param resSize Result size in bytes. * @param wireLatency Latency on the wire, in milliseconds, it doesn't * include retry delay or rate limit delay. */ void addReqStat(String requestClass, boolean error, int retries, - int retryDelay, int rateLimitDelay, int retryAuth, int retryThrottle, + int retryDelay, int rateLimitDelay, int authCount, int throttleCount, int connections, int reqSize, int resSize, long wireLatency) { ReqStats rStat; @@ -295,8 +304,8 @@ void addReqStat(String requestClass, boolean error, int retries, rStat.count++; rStat.retryTotalCount += retries; rStat.retryTotalDelay += retryDelay; - rStat.retryAuthCount += retryAuth; - rStat.retryThrottleCouunt += retryThrottle; + rStat.retryAuthCount += authCount; + rStat.retryThrottleCount += throttleCount; rStat.totalRateLimitDelay += rateLimitDelay; if (error) { rStat.errors ++; @@ -364,6 +373,9 @@ void addReqStat(String requestClass, boolean error, int retries, } } + /** + * Shuts down the time scheduler. + */ void shutdown() { service.shutdown(); } diff --git a/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java b/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java index f9465554..1d06875b 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java +++ b/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java @@ -30,7 +30,6 @@ public class StatsConfigImpl private Logger logger; private HttpClient httpClient; /* required for connections */ - private boolean rateLimitingEnabled; private String id = Integer.toHexString(UUID.randomUUID().hashCode()); private StatsHandler statsHandler; private boolean enableCollection = false; @@ -40,7 +39,6 @@ public class StatsConfigImpl HttpClient httpClient, boolean rateLimitingEnabled) { this.logger = logger; this.httpClient = httpClient; - this.rateLimitingEnabled = rateLimitingEnabled; String profileProp = System.getProperty(PROFILE_PROPERTY); if (profileProp != null) { @@ -75,8 +73,9 @@ public class StatsConfigImpl "{\"sdkName\"=\"Oracle NoSQL SDK for Java\", " + "\"sdkVersion\":\"" + libraryVersion + "\", " + "clientId=\"" + id + "\",\"profile\":\"" + profile + "\", " + - "\"intervalSec\"=" + interval + " \"prettyPrint\"=" + - prettyPrint + "}"); + "\"intervalSec\"=" + interval + + ", \"prettyPrint\"=" + prettyPrint + + ", \"rateLimitingEnabled\"=" + rateLimitingEnabled + "}"); start(); } @@ -202,15 +201,15 @@ public void logReqStat(Request kvRequest, long wireTime, public void logReqStatError(Request kvRequest) { if (stats != null && enableCollection) { String requestClass = kvRequest.getClass().getSimpleName(); - int auth = 0, throttle = 0, retryCount = 0, retryDelay = 0; + int authCount = 0, throttleCount = 0, retryCount = 0, retryDelay = 0; RetryStats retryStats = kvRequest.getRetryStats(); if (retryStats != null) { - auth = retryStats.getNumExceptions( + authCount = retryStats.getNumExceptions( AuthenticationException.class); - auth += retryStats.getNumExceptions( + authCount += retryStats.getNumExceptions( SecurityInfoNotReadyException.class); - throttle = retryStats.getNumExceptions( + throttleCount = retryStats.getNumExceptions( ThrottlingException.class); retryCount = retryStats.getRetries(); @@ -219,12 +218,8 @@ public void logReqStatError(Request kvRequest) { int rateLimitDelay = kvRequest.getRateLimitDelayedMs(); - stats.addReqStatError(requestClass, retryCount, retryDelay, auth, - throttle, rateLimitDelay, httpClient.getAcquiredChannelCount()); + stats.addReqStatError(requestClass, retryCount, retryDelay, authCount, + throttleCount, rateLimitDelay, httpClient.getAcquiredChannelCount()); } } - - public boolean isRateLimitingEnabled() { - return rateLimitingEnabled; - } } From d05f5d8e89d5da52a81c1865db1f921bf42daba1 Mon Sep 17 00:00:00 2001 From: Cezar Andrei Date: Wed, 8 Sep 2021 14:43:33 -0500 Subject: [PATCH 03/13] Various fixes: - renamed methods and fields - create full stats structure at the beginning to avoid sync locks. - switched avg calculation based on sum to be more accurate for smaller number of samples. - encapsulated writes and reads into their classes for moving sync on methods. --- .../java/oracle/nosql/driver/http/Client.java | 18 +- .../java/oracle/nosql/driver/http/Stats.java | 418 ++++++++++-------- .../nosql/driver/http/StatsConfigImpl.java | 18 +- 3 files changed, 248 insertions(+), 206 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 487361bb..f4368522 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/Client.java +++ b/driver/src/main/java/oracle/nosql/driver/http/Client.java @@ -225,8 +225,8 @@ public Client(Logger logger, threadPool = null; } - statsConfig = new StatsConfigImpl(config.getLibraryVersion(), logger, - httpClient, httpConfig.getRateLimitingEnabled()); + statsConfig = new StatsConfigImpl(NoSQLHandleConfig.getLibraryVersion(), + logger, httpClient, httpConfig.getRateLimitingEnabled()); } /** @@ -554,7 +554,7 @@ public Result execute(Request kvRequest) { res.setRetryStats(kvRequest.getRetryStats()); kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.logReqStat(kvRequest, wireTime, + statsConfig.observe(kvRequest, Math.toIntExact(wireTime), contentLength, resSize); return res; @@ -571,7 +571,7 @@ public Result execute(Request kvRequest) { continue; } kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.logReqStatError(kvRequest); + statsConfig.observeError(kvRequest); logInfo(logger, "Unexpected authentication exception: " + rae); throw new NoSQLException("Unexpected exception: " + @@ -634,13 +634,13 @@ public Result execute(Request kvRequest) { continue; } catch (NoSQLException nse) { kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.logReqStatError(kvRequest); + statsConfig.observeError(kvRequest); logFine(logger, "Client execute NoSQLException: " + nse.getMessage()); throw nse; /* pass through */ } catch (RuntimeException e) { kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.logReqStatError(kvRequest); + statsConfig.observeError(kvRequest); logFine(logger, "Client execute runtime exception: " + e.getMessage()); throw e; @@ -664,7 +664,7 @@ public Result execute(Request kvRequest) { continue; } catch (InterruptedException ie) { kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.logReqStatError(kvRequest); + statsConfig.observeError(kvRequest); logInfo(logger, "Client interrupted exception: " + ie.getMessage()); /* this exception shouldn't retry -- direct throw */ @@ -672,7 +672,7 @@ public Result execute(Request kvRequest) { ie.getMessage()); } catch (ExecutionException ee) { kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.logReqStatError(kvRequest); + statsConfig.observeError(kvRequest); logInfo(logger, "Unable to execute request: " + ee.getCause().getMessage()); /* is there a better exception? */ @@ -711,7 +711,7 @@ public Result execute(Request kvRequest) { } while (! timeoutRequest(startTime, timeoutMs, exception)); kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.logReqStatError(kvRequest); + statsConfig.observeError(kvRequest); throw new RequestTimeoutException(timeoutMs, "Request timed out after " + kvRequest.getNumRetries() + (kvRequest.getNumRetries() == 1 ? " retry. " : " retries. ") + diff --git a/driver/src/main/java/oracle/nosql/driver/http/Stats.java b/driver/src/main/java/oracle/nosql/driver/http/Stats.java index 0a3b4fa0..269792ec 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/Stats.java +++ b/driver/src/main/java/oracle/nosql/driver/http/Stats.java @@ -22,47 +22,160 @@ import oracle.nosql.driver.values.StringValue; import oracle.nosql.driver.values.TimestampValue; -public class Stats { +class Stats { + + private static String[] REQUEST_KEYS = new String[] { + "Delete", "Get", "GetIndexes", "GetTable", + "ListTables", "MultiDelete", "Prepare", "Put", "Query", "Read", + "System", "SystemStatus", "Table", "TableUsage", + "WriteMultiple", "Write"}; private ScheduledExecutorService service; private StatsConfigImpl statsConfig; private long startTime; private long endTime; - private Map requests; - private ConnectionStats connectionStats; + private Map requests = new HashMap<>(); + private ConnectionStats connectionStats = new ConnectionStats(); private static class ReqStats { - long count; - long errors; - int reqSizeMin = Integer.MAX_VALUE; - int reqSizeMax; - float reqSizeAvg; - int resSizeMin = Integer.MAX_VALUE; - int resSizeMax; - float resSizeAvg; - int retryAuthCount; - int retryThrottleCount; - int retryTotalCount; - int retryTotalDelay; - int totalRateLimitDelay; - long wireLatencyMin = Long.MAX_VALUE; - long wireLatencyMax; - float wireLatencyAvg; - Percentile wireLatencyPercentile; + private long count = 0; + private long errors = 0; + private int reqSizeMin = Integer.MAX_VALUE; + private int reqSizeMax = 0; + private long reqSizeSum = 0; + private int resSizeMin = Integer.MAX_VALUE; + private int resSizeMax = 0; + private long resSizeSum = 0; + private int retryAuthCount = 0; + private int retryThrottleCount = 0; + private int retryCount = 0; + private int retryDelay = 0; + private int rateLimitDelay = 0; + private int wireLatencyMin = Integer.MAX_VALUE; + private int wireLatencyMax = 0; + private long wireLatencySum = 0; + private Percentile wireLatencyPercentile; + + synchronized void observe(boolean error, int retries, int retryDelay, + int rateLimitDelay, int authCount, int throttleCount, int reqSize, + int resSize, int wireLatency) { + + this.count++; + this.retryCount += retries; + this.retryDelay += retryDelay; + this.retryAuthCount += authCount; + this.retryThrottleCount += throttleCount; + this.rateLimitDelay += rateLimitDelay; + + if (error) { + this.errors++; + } + else { + this.reqSizeMin = Math.min(this.reqSizeMin, reqSize); + this.reqSizeMax = Math.max(this.reqSizeMax, reqSize); + this.reqSizeSum += reqSize; + + this.resSizeMin = Math.min(this.resSizeMin, resSize); + this.resSizeMax = Math.max(this.resSizeMax, resSize); + this.resSizeSum += resSize; + + this.wireLatencyMin = + Math.min(this.wireLatencyMin, wireLatency); + this.wireLatencyMax = + Math.max(this.wireLatencyMax, wireLatency); + this.wireLatencySum += wireLatency; + + if (this.wireLatencyPercentile != null) { + this.wireLatencyPercentile.addValue(wireLatency); + } + } + } + + synchronized void toJSON(String requestName, ArrayValue reqArray) { + if (count > 0) { + MapValue req = new MapValue(); + req.put("name", requestName); + req.put("count", count); + req.put("errors", errors); + + MapValue retry = new MapValue(); + retry.put("count", retryCount); + retry.put("delay", retryDelay); + retry.put("authCount", retryAuthCount); + retry.put("throttleCount", retryThrottleCount); + req.put("retry", retry); + req.put("rateLimitDelay", rateLimitDelay); + + if (wireLatencyMax > 0) { + MapValue latency = new MapValue(); + latency.put("min", wireLatencyMin); + latency.put("max", wireLatencyMax); + latency.put("avg", + 1.0 * wireLatencySum / (count - errors)); + if (wireLatencyPercentile != null) { + latency.put("95th", + wireLatencyPercentile.get95thPercentile()); + latency.put("99th", + wireLatencyPercentile.get99thPercentile()); + } + req.put("wireLatency", latency); + } + + if (reqSizeMax > 0) { + MapValue reqSize = new MapValue(); + reqSize.put("min", reqSizeMin); + reqSize.put("max", reqSizeMax); + reqSize.put("avg", 1.0 * reqSizeSum / (count - errors)); + req.put("reqSize", reqSize); + } + + if (resSizeMax > 0) { + MapValue resSize = new MapValue(); + resSize.put("min", resSizeMin); + resSize.put("max", resSizeMax); + resSize.put("avg", 1.0 * resSizeSum / (count - errors)); + req.put("resSize", resSize); + } + + reqArray.add(req); + } + } + + synchronized void clear() { + count = 0; + errors = 0; + reqSizeMin = Integer.MAX_VALUE; + reqSizeMax = 0; + reqSizeSum = 0; + resSizeMin = Integer.MAX_VALUE; + resSizeMax = 0; + resSizeSum = 0; + retryAuthCount = 0; + retryThrottleCount = 0; + retryCount = 0; + retryDelay = 0; + rateLimitDelay = 0; + wireLatencyMin = Integer.MAX_VALUE; + wireLatencyMax = 0; + wireLatencySum = 0; + if (wireLatencyPercentile != null) { + wireLatencyPercentile.clear(); + } + } } - static class Percentile { - List values; + private static class Percentile { + private List values; - void addValue(long value) { + synchronized void addValue(long value) { if (values == null) { values = new ArrayList<>(); } values.add(value); } - long getPercentile(double percentile) { + synchronized long getPercentile(double percentile) { if (values == null || values.size() == 0) { return -1; } @@ -86,41 +199,78 @@ long get95thPercentile() { long get99thPercentile() { return getPercentile(0.99d); } + + synchronized void clear() { + if (values != null) { + values.clear(); + } + } } private static class ConnectionStats { - long count; - int min = Integer.MAX_VALUE; - int max; - float avg; + private long count; + private int min = Integer.MAX_VALUE; + private int max; + private long sum; + + synchronized void observe(int connections) { + if (connections < min) { + min = connections; + } + if (connections > max) { + max = connections; + } + sum += connections; + count++; + } + + synchronized void toJSON(MapValue root) { + if (count > 0) { + MapValue connections = new MapValue(); + connections.put("min", min); + connections.put("max", max); + connections.put("avg", 1.0 * sum / count); + root.put("connections", connections); + } + } - public void clear() { + synchronized void clear() { count = 0; min = Integer.MAX_VALUE; max = 0; - avg = 0; + sum = 0; } } + Stats(StatsConfigImpl statsConfig) { this.statsConfig = statsConfig; - requests = new HashMap<>(); - connectionStats = new ConnectionStats(); + // Fill in the stats objects + for (String key : REQUEST_KEYS) { + ReqStats reqStats = new ReqStats(); + requests.put(key, reqStats); - Runnable runnable = new Runnable() { - public void run() { - try { - logClientStats(); - } catch (RuntimeException re) { - statsConfig.getLogger().log(Level.INFO, - re.getMessage()); - } + if (statsConfig.getProfile() == StatsConfig.Profile.FULL) { + reqStats.wireLatencyPercentile = new Percentile(); + } + } + + // Setup the scheduler for interval logging + Runnable runnable = () -> { + try { + logClientStats(); + } catch (RuntimeException re) { + statsConfig.getLogger().log(Level.INFO, + "Stats exception:" + re.getMessage()); + re.printStackTrace(); } }; LocalTime localTime = LocalTime.now(); + // To log stats at the top of the hour, calculate delay until first + // occurrence. Note: First interval can be smaller than the rest. long delay = 1000l * statsConfig.getInterval() - (( 1000l * 60l * localTime.getMinute() + 1000l * localTime.getSecond() + @@ -137,21 +287,23 @@ public void run() { startTime = System.currentTimeMillis(); } - void logClientStats() { + private void logClientStats() { endTime = System.currentTimeMillis(); + FieldValue fvStats = generateFieldValueStats(); - String json = fvStats.toJson(statsConfig.getPrettyPrint() ? - JsonOptions.PRETTY : null); + // Start from scratch in the new interval. + clearStats(); - /* Call user handle if configured. */ + // Call user handle if configured. StatsConfig.StatsHandler statsHandler = statsConfig.getHandler(); if (statsHandler != null) { statsHandler.accept(fvStats); } - /* start from scratch in the new interval */ - clearStats(); + // Output stats to logger. + String json = fvStats.toJson(statsConfig.getPrettyPrint() ? + JsonOptions.PRETTY : null); statsConfig.getLogger().log(Level.INFO, statsConfig.LOG_PREFIX + json); } @@ -165,75 +317,19 @@ private FieldValue generateFieldValueStats() { root.put("endTime", new TimestampValue(ts)); root.put("clientId", new StringValue(statsConfig.getId())); - if (connectionStats.count > 0) { - MapValue connections = new MapValue(); - connections.put("min", connectionStats.min); - connections.put("max", connectionStats.max); - connections.put("avg", connectionStats.avg); - root.put("connections", connections); - } + connectionStats.toJSON(root); Set> entries; - synchronized (requests) { - entries = new HashSet<>(requests.entrySet()); - } + entries = new HashSet<>(requests.entrySet()); if (entries.size() > 0) { - ArrayValue reqArray = new ArrayValue(); root.put("requests", reqArray); entries.forEach(e -> { - String k = e.getKey(); ReqStats v = e.getValue(); - - MapValue req = new MapValue(); - req.put("name", k); - req.put("count", v.count); - req.put("errors", v.errors); - - MapValue retry = new MapValue(); - retry.put("totalCount", v.retryTotalCount); - retry.put("totalDelay", v.retryTotalDelay); - retry.put("authCount", v.retryAuthCount); - retry.put("throttleCount", v.retryThrottleCount); - req.put("retry", retry); - req.put("totalRateLimitDelay", v.totalRateLimitDelay); - - if (v.wireLatencyMax > 0) { - MapValue latency = new MapValue(); - latency.put("min", v.wireLatencyMin); - latency.put("max", v.wireLatencyMax); - latency.put("avg", v.wireLatencyAvg); - if (v.wireLatencyPercentile != null) { - latency - .put("95th", - v.wireLatencyPercentile.get95thPercentile()); - latency - .put("99th", - v.wireLatencyPercentile.get99thPercentile()); - } - req.put("wireLatency", latency); - } - - if (v.reqSizeMax > 0) { - MapValue reqSize = new MapValue(); - reqSize.put("min", v.reqSizeMin); - reqSize.put("max", v.reqSizeMax); - reqSize.put("avg", v.reqSizeAvg); - req.put("reqSize", reqSize); - } - - if (v.resSizeMax > 0) { - MapValue resSize = new MapValue(); - resSize.put("min", v.resSizeMin); - resSize.put("max", v.resSizeMax); - resSize.put("avg", v.resSizeAvg); - req.put("resSize", resSize); - } - - reqArray.add(req); + v.toJSON(k , reqArray); }); } @@ -244,12 +340,12 @@ private FieldValue generateFieldValueStats() { * Clear all collected stats. */ void clearStats() { - synchronized (requests) { - startTime = System.currentTimeMillis(); - endTime = 0; - requests.clear(); - connectionStats.clear(); + for (String key : REQUEST_KEYS) { + requests.get(key).clear(); } + connectionStats.clear(); + startTime = System.currentTimeMillis(); + endTime = 0; } /** @@ -260,10 +356,10 @@ void clearStats() { * @param authCount Number of authCount errors. * @param throttleCount Number of throttleCount errors. */ - void addReqStatError(String requestClass, int retryCount, + void observeError(String requestClass, int retryCount, int retryDelay, int authCount, int throttleCount, int rateLimitDelay, int connections) { - addReqStat(requestClass, true, retryCount, retryDelay, rateLimitDelay, + observe(requestClass, true, retryCount, retryDelay, rateLimitDelay, authCount, throttleCount, connections, -1, -1,-1); } @@ -280,97 +376,37 @@ void addReqStatError(String requestClass, int retryCount, * @param wireLatency Latency on the wire, in milliseconds, it doesn't * include retry delay or rate limit delay. */ - void addReqStat(String requestClass, boolean error, int retries, + void observe(String requestClass, boolean error, int retries, int retryDelay, int rateLimitDelay, int authCount, int throttleCount, - int connections, int reqSize, int resSize, long wireLatency) { + int connections, int reqSize, int resSize, int wireLatency) { ReqStats rStat; requestClass = requestClass != null && requestClass.endsWith("Request") ? requestClass.substring(0, requestClass.length() - 7) : requestClass; - synchronized (requests) { - rStat = requests.get(requestClass); - if (rStat == null) { - rStat = new ReqStats(); - requests.put(requestClass, rStat); - - if (statsConfig.getProfile() == StatsConfig.Profile.FULL) { - rStat.wireLatencyPercentile = new Percentile(); - } - } - } - - synchronized (rStat) { - rStat.count++; - rStat.retryTotalCount += retries; - rStat.retryTotalDelay += retryDelay; - rStat.retryAuthCount += authCount; - rStat.retryThrottleCount += throttleCount; - rStat.totalRateLimitDelay += rateLimitDelay; - if (error) { - rStat.errors ++; - } else { - if (reqSize < rStat.reqSizeMin) { - rStat.reqSizeMin = reqSize; - } - if (reqSize > rStat.reqSizeMax) { - rStat.reqSizeMax = reqSize; - } - - if (resSize < rStat.resSizeMin) { - rStat.resSizeMin = resSize; - } - if (resSize > rStat.resSizeMax) { - rStat.resSizeMax = resSize; - } - - if (wireLatency < rStat.wireLatencyMin) { - rStat.wireLatencyMin = wireLatency; - } - if (wireLatency > rStat.wireLatencyMax) { - rStat.wireLatencyMax = wireLatency; - } - if (rStat.wireLatencyPercentile != null) { - rStat.wireLatencyPercentile.addValue(wireLatency); - } + rStat = requests.get(requestClass); - if (rStat.count == 0) { - rStat.reqSizeAvg = reqSize; - rStat.resSizeAvg = resSize; - rStat.wireLatencyAvg = wireLatency; - } else { - rStat.reqSizeAvg = rStat.reqSizeAvg + - (reqSize - rStat.reqSizeAvg) / - (rStat.count - rStat.errors); - - rStat.resSizeAvg = rStat.resSizeAvg + - (resSize - rStat.resSizeAvg) / - (rStat.count - rStat.errors); - - rStat.wireLatencyAvg = rStat.wireLatencyAvg + - (wireLatency - rStat.wireLatencyAvg) / - (rStat.count - rStat.errors); + // This will not happen unless a new request type is added but not + // registered in the REQUEST_KEYS array. + if (rStat == null) { + ReqStats newStat = new ReqStats(); + if (statsConfig.getProfile() == StatsConfig.Profile.FULL) { + newStat.wireLatencyPercentile = new Percentile(); + } + synchronized (requests) { + rStat = requests.get(requestClass); + if (rStat == null) { + requests.put(requestClass, newStat); + rStat = newStat; } } } - synchronized (connectionStats) { - if (connections < connectionStats.min) { - connectionStats.min = connections; - } - if (connections > connectionStats.max) { - connectionStats.max = connections; - } - if (connectionStats.count == 0) { - connectionStats.avg = connections; - } else { - connectionStats.avg = connectionStats.avg + - (connections - connectionStats.avg) / - connectionStats.count; - } - connectionStats.count++; - } + rStat.observe(error, retries, retryDelay, rateLimitDelay, authCount, + throttleCount, reqSize, resSize, wireLatency); + + connectionStats.observe(connections); } /** diff --git a/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java b/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java index 1d06875b..b62c3aad 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java +++ b/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java @@ -4,9 +4,11 @@ import java.util.logging.Level; import java.util.logging.Logger; +import oracle.nosql.driver.ReadThrottlingException; import oracle.nosql.driver.SecurityInfoNotReadyException; import oracle.nosql.driver.StatsConfig; import oracle.nosql.driver.ThrottlingException; +import oracle.nosql.driver.WriteThrottlingException; import oracle.nosql.driver.httpclient.HttpClient; import oracle.nosql.driver.kv.AuthenticationException; import oracle.nosql.driver.ops.Request; @@ -168,7 +170,7 @@ public void shutdown() { } } - public void logReqStat(Request kvRequest, long wireTime, + void observe(Request kvRequest, int wireTime, int reqSize, int resSize) { if (stats != null && enableCollection) { String requestClass = kvRequest.getClass().getSimpleName(); @@ -190,15 +192,14 @@ public void logReqStat(Request kvRequest, long wireTime, int rateLimitDelay = kvRequest.getRateLimitDelayedMs(); - // using direct measured http call instead of total - delays - stats.addReqStat(requestClass, false, retries, retryDelay, + stats.observe(requestClass, false, retries, retryDelay, rateLimitDelay, auth, throttle, httpClient.getAcquiredChannelCount(), reqSize, resSize, wireTime); } } - public void logReqStatError(Request kvRequest) { + void observeError(Request kvRequest) { if (stats != null && enableCollection) { String requestClass = kvRequest.getClass().getSimpleName(); int authCount = 0, throttleCount = 0, retryCount = 0, retryDelay = 0; @@ -211,6 +212,10 @@ public void logReqStatError(Request kvRequest) { throttleCount = retryStats.getNumExceptions( ThrottlingException.class); + throttleCount += retryStats.getNumExceptions( + ReadThrottlingException.class); + throttleCount += retryStats.getNumExceptions( + WriteThrottlingException.class); retryCount = retryStats.getRetries(); retryDelay = retryStats.getDelayMs(); @@ -218,8 +223,9 @@ public void logReqStatError(Request kvRequest) { int rateLimitDelay = kvRequest.getRateLimitDelayedMs(); - stats.addReqStatError(requestClass, retryCount, retryDelay, authCount, - throttleCount, rateLimitDelay, httpClient.getAcquiredChannelCount()); + stats.observeError(requestClass, retryCount, retryDelay, + authCount, throttleCount, rateLimitDelay, + httpClient.getAcquiredChannelCount()); } } } From 2c2f41cbd63072e1bb9df40d15ac72113e46c3d8 Mon Sep 17 00:00:00 2001 From: Cezar Andrei <675678+cezarfx@users.noreply.github.com> Date: Mon, 13 Sep 2021 19:20:01 -0500 Subject: [PATCH 04/13] Update NoSQLHandle.java Updated javadoc. --- .../src/main/java/oracle/nosql/driver/NoSQLHandle.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java index 2f467a7c..0af264ab 100644 --- a/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java +++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java @@ -572,11 +572,10 @@ public SystemResult doSystemRequest(String statement, int pollIntervalMs); /** - * Returns configuration object for client side statistics. - * Client side statistics, when enabled, are INFO generated entries on the - * logger at regular time interval. - * - * @return configuration object for client side statistics + * Returns configuration object for statistics. + * Statistics, when enabled, contain information like errors, delays, retries + * and wire latency agregated on types of requests. + * @return configuration object for statistics */ public StatsConfig getStatsConfig(); From 007e1bfc598ce5e262d5949c628676b5e818cf47 Mon Sep 17 00:00:00 2001 From: Cezar Andrei <675678+cezarfx@users.noreply.github.com> Date: Mon, 13 Sep 2021 19:24:00 -0500 Subject: [PATCH 05/13] Update StatsConfig.java Updated javadoc. --- driver/src/main/java/oracle/nosql/driver/StatsConfig.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/driver/src/main/java/oracle/nosql/driver/StatsConfig.java b/driver/src/main/java/oracle/nosql/driver/StatsConfig.java index 4ecde9df..e832f9cb 100644 --- a/driver/src/main/java/oracle/nosql/driver/StatsConfig.java +++ b/driver/src/main/java/oracle/nosql/driver/StatsConfig.java @@ -6,9 +6,8 @@ import oracle.nosql.driver.values.FieldValue; /** - *

    This interface allows user to setup the collection of statistics on the - * client side.

    - * + *

    This interface allows user to setup the collection of driver statistics.

    + * * The statistics data is collected for an interval of time. At the end of the * interval, the stats data is logged in a specified JSON format that can be * filtered and parsed. After the logging, the counters are cleared and @@ -140,4 +139,4 @@ interface StatsHandler extends Consumer { * Returns true if collection of stats is enabled, otherwise returns false. */ boolean isStarted(); -} \ No newline at end of file +} From e589fbc90192928e8aeeb00b7111d008e89fa08c Mon Sep 17 00:00:00 2001 From: Cezar Andrei Date: Wed, 22 Sep 2021 12:15:17 -0500 Subject: [PATCH 06/13] Introduced a new profile level. Adding detailed per query stats into the new profile. Renamed wireLatency to networkLatency, included Ms suffix for latency and delay entries to describe millisecond unit. Renamed reqSize and resSize to requestSize and resultSize. Add a stats log entry at end, just before shutdown. Fixed several javadoc params and added a sample output. Avoid NPE for Prepared statements without a driverPlan. Get statement from QueryRequest when available inside the prepared statement. --- .../java/oracle/nosql/driver/StatsConfig.java | 154 +++++++- .../java/oracle/nosql/driver/http/Client.java | 14 +- .../java/oracle/nosql/driver/http/Stats.java | 351 ++++++++++++++---- .../nosql/driver/http/StatsConfigImpl.java | 64 +--- .../nosql/driver/ops/PreparedStatement.java | 5 +- .../oracle/nosql/driver/ops/QueryRequest.java | 5 + 6 files changed, 443 insertions(+), 150 deletions(-) diff --git a/driver/src/main/java/oracle/nosql/driver/StatsConfig.java b/driver/src/main/java/oracle/nosql/driver/StatsConfig.java index e832f9cb..6c208d87 100644 --- a/driver/src/main/java/oracle/nosql/driver/StatsConfig.java +++ b/driver/src/main/java/oracle/nosql/driver/StatsConfig.java @@ -17,12 +17,15 @@ * interval logs may contain stats for a shorter interval.

    * * Collection of stats are controlled by the following system properties:

  • - * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile=[none|regular|full] + * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile=[none|regular|more|all] * Specifies the stats profile: none - disabled, - * regular - all except 95th and 99th percentile and - * full - all including 95th and 99th percentile.
  • + * regular - per request: counters, errors, latencies, delays, retries + * more - stats above with 95th and 99th percentile latencies + * all - stats above with per query information.
  • + * * -Dcom.oracle.nosql.sdk.nosqldriver.stats.interval=600 Interval in * seconds to log the stats, by default is 10 minutes.
  • + * * -Dcom.oracle.nosql.sdk.nosqldriver.stats.pretty-print=true Option * to enable pretty printing of the JSON data, default value is false
  • * @@ -36,7 +39,6 @@ * statsConfig.setPrettyPrint(false); * statsConfig.registerHandler( * new StatsConfig.StatsHandler() { - * @Override * public void accept(FieldValue jsonStats) { * System.out.println("!!! Got a stat: " + jsonStats); * } @@ -47,18 +49,152 @@ * * statsConfig.stop(); * handle.close(); - *

    + *

    + * + * The following is an example of stats log entry using the ALL + * profile: + * INFO: ONJS:Monitoring stats|{ + * "clientId" : "b7bc7734", + * "startTime" : "2021-09-20T20:11:42Z", + * "endTime" : "2021-09-20T20:11:47Z", + * "requests" : [{ + * "name" : "Get", + * "count" : 2, + * "errors" : 0, + * "networkLatencyMs" : { + * "min" : 4, + * "avg" : 4.5, + * "max" : 5, + * "95th" : 5, + * "99th" : 5 + * }, + * "requestSize" : { + * "min" : 42, + * "avg" : 42.5, + * "max" : 43 + * }, + * "resultSize" : { + * "min" : 193, + * "avg" : 206.5, + * "max" : 220 + * }, + * "rateLimitDelayMs" : 0, + * "retry" : { + * "delayMs" : 0, + * "authCount" : 0, + * "throttleCount" : 0, + * "count" : 0 + * } + * }, { + * "name" : "Query", + * "count" : 14, + * "errors" : 0, + * "networkLatencyMs" : { + * "min" : 3, + * "avg" : 13.0, + * "max" : 32, + * "95th" : 32, + * "99th" : 32 + * }, + * "resultSize" : { + * "min" : 146, + * "avg" : 7379.71, + * "max" : 10989 + * }, + * "requestSize" : { + * "min" : 65, + * "avg" : 709.85, + * "max" : 799 + * }, + * "rateLimitDelayMs" : 0, + * "retry" : { + * "delayMs" : 0, + * "authCount" : 0, + * "throttleCount" : 0, + * "count" : 0 + * } + * }, { + * "name" : "Put", + * "count" : 1002, + * "errors" : 0, + * "networkLatencyMs" : { + * "min" : 1, + * "avg" : 4.41, + * "max" : 80, + * "95th" : 8, + * "99th" : 20 + * }, + * "requestSize" : { + * "min" : 90, + * "avg" : 90.16, + * "max" : 187 + * }, + * "resultSize" : { + * "min" : 58, + * "avg" : 58.0, + * "max" : 58 + * }, + * "rateLimitDelayMs" : 0, + * "retry" : { + * "delayMs" : 0, + * "authCount" : 0, + * "throttleCount" : 0, + * "count" : 0 + * } + * }], + * "queries" : [{ + * "stmt" : "SELECT * FROM audienceData ORDER BY cookie_id", + * "plan" : "SFW([6])\n[\n FROM:\n RECV([3])\n [\n DistributionKind : ALL_PARTITIONS,\n Sort Fields : sort_gen,\n\n ] as $from-0\n\n SELECT:\n FIELD_STEP([6])\n [\n VAR_REF($from-0)([3]),\n audienceData\n ]\n]", + * "doesWrites" : false, + * "count" : 12, + * "unprepared" : 1, + * "simple" : 0, + * "countAPI" : 20, + * "errors" : 0, + * "networkLatencyMs" : { + * "min" : 8, + * "avg" : 14.58, + * "max" : 32, + * "95th" : 32, + * "99th" : 32 + * }, + * "requestSize" : { + * "min" : 65, + * "avg" : 732.5, + * "max" : 799 + * }, + * "resultSize" : { + * "min" : 914, + * "avg" : 8585.33, + * "max" : 10989 + * }, + * "rateLimitDelayMs" : 0, + * "retry" : { + * "delayMs" : 0, + * "authCount" : 0, + * "throttleCount" : 0, + * "count" : 0 + * } + * }], + * "connections" : { + * "min" : 1, + * "avg" : 9.58, + * "max" : 10 + * } + * } + *

    */ public interface StatsConfig { /** * The following semantics are attached to the Profile: * - NONE: no stats are logged. - * - REGULAR: all stats except 95th and 99th percentile latencies. - * - FULL: all stats including 95th and 99th percentile latencies. + * - REGULAR: per request: counters, errors, latencies, delays, retries + * - MORE: stats above with 95th and 99th percentile latencies. + * - ALL: stats above with per query information */ enum Profile { - NONE, REGULAR, FULL; + NONE, REGULAR, MORE, ALL; } /** @@ -126,7 +262,7 @@ interface StatsHandler extends Consumer { /** * Collection of stats is enabled only between start and stop or from the * beginning if system property - * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile= is "regular" or "full". + * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile= is not "none". */ void start(); 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 f4368522..639eba3a 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/Client.java +++ b/driver/src/main/java/oracle/nosql/driver/http/Client.java @@ -297,6 +297,8 @@ public Result execute(Request kvRequest) { if (kvRequest.isQueryRequest()) { QueryRequest qreq = (QueryRequest)kvRequest; + statsConfig.observeQuery(qreq); + /* * The following "if" may be true for advanced queries only. For * such queries, the "if" will be true (i.e., the QueryRequest will @@ -437,7 +439,7 @@ public Result execute(Request kvRequest) { ResponseHandler responseHandler = null; ByteBuf buffer = null; - long wireTime; + long networkLatency; try { /* * NOTE: the ResponseHandler will release the Channel @@ -506,7 +508,7 @@ public Result execute(Request kvRequest) { if (isLoggable(logger, Level.FINE)) { logTrace(logger, "Request: " + requestClass); } - wireTime = System.currentTimeMillis(); + networkLatency = System.currentTimeMillis(); httpClient.runRequest(request, responseHandler, channel); boolean isTimeout = @@ -526,7 +528,7 @@ public Result execute(Request kvRequest) { wireContent, kvRequest); int resSize = wireContent.readerIndex(); - wireTime = System.currentTimeMillis() - wireTime; + networkLatency = System.currentTimeMillis() - networkLatency; if (res instanceof TableResult && rateLimiterMap != null) { /* update rate limiter settings for table */ @@ -554,7 +556,7 @@ public Result execute(Request kvRequest) { res.setRetryStats(kvRequest.getRetryStats()); kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.observe(kvRequest, Math.toIntExact(wireTime), + statsConfig.observe(kvRequest, Math.toIntExact(networkLatency), contentLength, resSize); return res; @@ -880,7 +882,7 @@ void writeContent(ByteBuf content, Request kvRequest) * Processes the httpResponse object converting it into a suitable * return value. * - * @param httpResponse the response from the service + * @param content the response from the service * * @return the programmatic response object */ @@ -960,7 +962,7 @@ Result processOKResponse(ByteInputStream in, Request kvRequest) { * * @param status the http response code it must not be OK * - * @param in the input stream representing the failure response + * @param payload the payload representing the failure response */ private void processNotOKResponse(HttpResponseStatus status, ByteBuf payload) { diff --git a/driver/src/main/java/oracle/nosql/driver/http/Stats.java b/driver/src/main/java/oracle/nosql/driver/http/Stats.java index 269792ec..0373b9a2 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/Stats.java +++ b/driver/src/main/java/oracle/nosql/driver/http/Stats.java @@ -14,7 +14,15 @@ import java.util.concurrent.TimeUnit; import java.util.logging.Level; +import oracle.nosql.driver.SecurityInfoNotReadyException; import oracle.nosql.driver.StatsConfig; +import oracle.nosql.driver.ThrottlingException; +import oracle.nosql.driver.kv.AuthenticationException; +import oracle.nosql.driver.ops.PreparedStatement; +import oracle.nosql.driver.ops.QueryRequest; +import oracle.nosql.driver.ops.Request; +import oracle.nosql.driver.ops.RetryStats; +import oracle.nosql.driver.query.PlanIter; import oracle.nosql.driver.values.ArrayValue; import oracle.nosql.driver.values.FieldValue; import oracle.nosql.driver.values.JsonOptions; @@ -37,6 +45,7 @@ class Stats { private long endTime; private Map requests = new HashMap<>(); private ConnectionStats connectionStats = new ConnectionStats(); + private ExtraQueryStats extraQueryStats; private static class ReqStats { private long count = 0; @@ -50,23 +59,23 @@ private static class ReqStats { private int retryAuthCount = 0; private int retryThrottleCount = 0; private int retryCount = 0; - private int retryDelay = 0; - private int rateLimitDelay = 0; - private int wireLatencyMin = Integer.MAX_VALUE; - private int wireLatencyMax = 0; - private long wireLatencySum = 0; + private int retryDelayMs = 0; + private int rateLimitDelayMs = 0; + private int networkLatencyMin = Integer.MAX_VALUE; + private int networkLatencyMax = 0; + private long networkLatencySum = 0; private Percentile wireLatencyPercentile; synchronized void observe(boolean error, int retries, int retryDelay, int rateLimitDelay, int authCount, int throttleCount, int reqSize, - int resSize, int wireLatency) { + int resSize, int networkLatency) { this.count++; this.retryCount += retries; - this.retryDelay += retryDelay; + this.retryDelayMs += retryDelay; this.retryAuthCount += authCount; this.retryThrottleCount += throttleCount; - this.rateLimitDelay += rateLimitDelay; + this.rateLimitDelayMs += rateLimitDelay; if (error) { this.errors++; @@ -80,65 +89,69 @@ synchronized void observe(boolean error, int retries, int retryDelay, this.resSizeMax = Math.max(this.resSizeMax, resSize); this.resSizeSum += resSize; - this.wireLatencyMin = - Math.min(this.wireLatencyMin, wireLatency); - this.wireLatencyMax = - Math.max(this.wireLatencyMax, wireLatency); - this.wireLatencySum += wireLatency; + this.networkLatencyMin = + Math.min(this.networkLatencyMin, networkLatency); + this.networkLatencyMax = + Math.max(this.networkLatencyMax, networkLatency); + this.networkLatencySum += networkLatency; if (this.wireLatencyPercentile != null) { - this.wireLatencyPercentile.addValue(wireLatency); + this.wireLatencyPercentile.addValue(networkLatency); } } } synchronized void toJSON(String requestName, ArrayValue reqArray) { if (count > 0) { - MapValue req = new MapValue(); - req.put("name", requestName); - req.put("count", count); - req.put("errors", errors); - - MapValue retry = new MapValue(); - retry.put("count", retryCount); - retry.put("delay", retryDelay); - retry.put("authCount", retryAuthCount); - retry.put("throttleCount", retryThrottleCount); - req.put("retry", retry); - req.put("rateLimitDelay", rateLimitDelay); - - if (wireLatencyMax > 0) { - MapValue latency = new MapValue(); - latency.put("min", wireLatencyMin); - latency.put("max", wireLatencyMax); - latency.put("avg", - 1.0 * wireLatencySum / (count - errors)); - if (wireLatencyPercentile != null) { - latency.put("95th", - wireLatencyPercentile.get95thPercentile()); - latency.put("99th", - wireLatencyPercentile.get99thPercentile()); - } - req.put("wireLatency", latency); - } + MapValue mapValue = new MapValue(); + mapValue.put("name", requestName); - if (reqSizeMax > 0) { - MapValue reqSize = new MapValue(); - reqSize.put("min", reqSizeMin); - reqSize.put("max", reqSizeMax); - reqSize.put("avg", 1.0 * reqSizeSum / (count - errors)); - req.put("reqSize", reqSize); - } + toMapValue(mapValue); + reqArray.add(mapValue); + } + } - if (resSizeMax > 0) { - MapValue resSize = new MapValue(); - resSize.put("min", resSizeMin); - resSize.put("max", resSizeMax); - resSize.put("avg", 1.0 * resSizeSum / (count - errors)); - req.put("resSize", resSize); + private void toMapValue(MapValue mapValue) { + mapValue.put("count", count); + mapValue.put("errors", errors); + + MapValue retry = new MapValue(); + retry.put("count", retryCount); + retry.put("delayMs", retryDelayMs); + retry.put("authCount", retryAuthCount); + retry.put("throttleCount", retryThrottleCount); + mapValue.put("retry", retry); + mapValue.put("rateLimitDelayMs", rateLimitDelayMs); + + if (networkLatencyMax > 0) { + MapValue latency = new MapValue(); + latency.put("min", networkLatencyMin); + latency.put("max", networkLatencyMax); + latency.put("avg", + 1.0 * networkLatencySum / (count - errors)); + if (wireLatencyPercentile != null) { + latency.put("95th", + wireLatencyPercentile.get95thPercentile()); + latency.put("99th", + wireLatencyPercentile.get99thPercentile()); } + mapValue.put("networkLatencyMs", latency); + } + + if (reqSizeMax > 0) { + MapValue reqSize = new MapValue(); + reqSize.put("min", reqSizeMin); + reqSize.put("max", reqSizeMax); + reqSize.put("avg", 1.0 * reqSizeSum / (count - errors)); + mapValue.put("requestSize", reqSize); + } - reqArray.add(req); + if (resSizeMax > 0) { + MapValue resSize = new MapValue(); + resSize.put("min", resSizeMin); + resSize.put("max", resSizeMax); + resSize.put("avg", 1.0 * resSizeSum / (count - errors)); + mapValue.put("resultSize", resSize); } } @@ -154,11 +167,11 @@ synchronized void clear() { retryAuthCount = 0; retryThrottleCount = 0; retryCount = 0; - retryDelay = 0; - rateLimitDelay = 0; - wireLatencyMin = Integer.MAX_VALUE; - wireLatencyMax = 0; - wireLatencySum = 0; + retryDelayMs = 0; + rateLimitDelayMs = 0; + networkLatencyMin = Integer.MAX_VALUE; + networkLatencyMax = 0; + networkLatencySum = 0; if (wireLatencyPercentile != null) { wireLatencyPercentile.clear(); } @@ -242,6 +255,141 @@ synchronized void clear() { } } + static class ExtraQueryStats { + + static class QueryEntryStat { + long countAPI; + long unprepared; + long simple; + boolean doesWrites; + ReqStats reqStats; + String plan; + + QueryEntryStat(StatsConfig statsConfig, QueryRequest queryRequest) { + reqStats = new ReqStats(); + + if (statsConfig.getProfile().ordinal() >= + StatsConfig.Profile.MORE.ordinal()) { + reqStats.wireLatencyPercentile = new Percentile(); + } + + PreparedStatement pStmt = queryRequest.getPreparedStatement(); + if (pStmt != null) { + plan = pStmt.printDriverPlan(); + doesWrites = pStmt.doesWrites(); + } + } + } + + private Map queries = new HashMap<>(); + private StatsConfig statsConfig; + + ExtraQueryStats(StatsConfig statsConfig) { + this.statsConfig = statsConfig; + } + + synchronized void observeQuery(QueryRequest queryRequest) { + QueryEntryStat qStat = getExtraQueryStat(queryRequest); + + qStat.countAPI++; + if (!queryRequest.isPrepared()) { + qStat.unprepared++; + } + if (queryRequest.isPrepared() && queryRequest.isSimpleQuery()) { + qStat.simple++; + } + + if (queryRequest.getPreparedStatement() != null && + queryRequest.getPreparedStatement().driverPlan() != null) { + PlanIter drvPlan = queryRequest.getPreparedStatement().driverPlan(); + drvPlan.getKind(); + } + } + + synchronized void observeQuery(QueryRequest queryRequest, boolean error, + int retries, int retryDelay, int rateLimitDelay, + int authCount, int throttleCount, int reqSize, int resSize, + int wireLatency) { + + QueryEntryStat qStat = getExtraQueryStat(queryRequest); + + qStat.reqStats.count++; + if (error) { + qStat.reqStats.errors++; + } + qStat.reqStats.retryCount += retries; + qStat.reqStats.retryDelayMs += retryDelay; + qStat.reqStats.rateLimitDelayMs += rateLimitDelay; + qStat.reqStats.retryAuthCount += authCount; + qStat.reqStats.retryThrottleCount += throttleCount; + qStat.reqStats.reqSizeSum += reqSize; + qStat.reqStats.reqSizeMin = Math.min(qStat.reqStats.reqSizeMin, + reqSize); + qStat.reqStats.reqSizeMax = Math.max(qStat.reqStats.reqSizeMax, + reqSize); + qStat.reqStats.resSizeSum += resSize; + qStat.reqStats.resSizeMin = Math.min(qStat.reqStats.resSizeMin, + resSize); + qStat.reqStats.resSizeMax = Math.max(qStat.reqStats.resSizeMax, + resSize); + qStat.reqStats.networkLatencySum += wireLatency; + qStat.reqStats.networkLatencyMin = + Math.min(qStat.reqStats.networkLatencyMin, wireLatency); + qStat.reqStats.networkLatencyMax = + Math.max(qStat.reqStats.networkLatencyMax, wireLatency); + if (qStat.reqStats.wireLatencyPercentile != null) { + qStat.reqStats.wireLatencyPercentile.addValue(wireLatency); + } + } + + private QueryEntryStat getExtraQueryStat( + QueryRequest queryRequest) { + String sql = queryRequest.getStatement(); + + QueryEntryStat qStat = queries.get(sql); + + if (qStat == null) { + qStat = new QueryEntryStat(statsConfig, queryRequest); + queries.put(sql, qStat); + } + + if (qStat.plan == null) { + PreparedStatement pStmt = queryRequest.getPreparedStatement(); + if (pStmt != null) { + qStat.plan = pStmt.printDriverPlan(); + qStat.doesWrites = pStmt.doesWrites(); + } + } + return qStat; + } + + synchronized void toJSON(MapValue root) { + if (queries.size() > 0) { + ArrayValue queryArr = new ArrayValue(); + root.put("queries", queryArr); + + queries.forEach((key, val) -> { + + MapValue queryVal = new MapValue(); + queryArr.add(queryVal); + + queryVal.put("stmt", key == null ? "null" : key); + queryVal.put("countAPI", val.countAPI); + queryVal.put("unprepared", val.unprepared); + queryVal.put("simple", val.simple); + queryVal.put("doesWrites", val.doesWrites); + if (val.plan != null) { + queryVal.put("plan", val.plan); + } + val.reqStats.toMapValue(queryVal); + }); + } + } + + synchronized void clear() { + queries.clear(); + } + } Stats(StatsConfigImpl statsConfig) { this.statsConfig = statsConfig; @@ -251,18 +399,24 @@ synchronized void clear() { ReqStats reqStats = new ReqStats(); requests.put(key, reqStats); - if (statsConfig.getProfile() == StatsConfig.Profile.FULL) { + if (statsConfig.getProfile().ordinal() >= + StatsConfig.Profile.MORE.ordinal()) { reqStats.wireLatencyPercentile = new Percentile(); } } + if (statsConfig.getProfile().ordinal() >= + StatsConfig.Profile.ALL.ordinal()) { + extraQueryStats = new ExtraQueryStats(statsConfig); + } + // Setup the scheduler for interval logging Runnable runnable = () -> { try { logClientStats(); } catch (RuntimeException re) { statsConfig.getLogger().log(Level.INFO, - "Stats exception:" + re.getMessage()); + "Stats exception: " + re.getMessage()); re.printStackTrace(); } }; @@ -318,6 +472,9 @@ private FieldValue generateFieldValueStats() { root.put("clientId", new StringValue(statsConfig.getId())); connectionStats.toJSON(root); + if (extraQueryStats != null) { + extraQueryStats.toJSON(root); + } Set> entries; entries = new HashSet<>(requests.entrySet()); @@ -344,6 +501,10 @@ void clearStats() { requests.get(key).clear(); } connectionStats.clear(); + if (extraQueryStats != null) { + extraQueryStats.clear(); + } + startTime = System.currentTimeMillis(); endTime = 0; } @@ -351,37 +512,47 @@ void clearStats() { /** * Adds a new error statistic entry. When error we don't track request, * response sizes and latency. - * - * @param requestClass Type of request. - * @param authCount Number of authCount errors. - * @param throttleCount Number of throttleCount errors. */ - void observeError(String requestClass, int retryCount, - int retryDelay, int authCount, int throttleCount, int rateLimitDelay, + void observeError(Request kvRequest, int connections) { - observe(requestClass, true, retryCount, retryDelay, rateLimitDelay, - authCount, throttleCount, connections, -1, -1,-1); + observe(kvRequest, true, connections, -1, -1, -1); } /** * Adds a new statistic entry. Can be of 2 types: successful or error. - * Request, result sizes and wireLatency are not registered for error entries. + * Request, result sizes and networkLatency are not registered for error entries. * - * @param requestClass Type of request. + * @param kvRequest The request object. * @param error Hard error, ie. return error to user. - * @param authCount Number of authCount errors. - * @param throttleCount Number of throttleCount errors. + * @param connections The number of active connections in the pool. * @param reqSize Request size in bytes. * @param resSize Result size in bytes. - * @param wireLatency Latency on the wire, in milliseconds, it doesn't + * @param networkLatency Latency on the wire, in milliseconds, it doesn't * include retry delay or rate limit delay. */ - void observe(String requestClass, boolean error, int retries, - int retryDelay, int rateLimitDelay, int authCount, int throttleCount, - int connections, int reqSize, int resSize, int wireLatency) { + void observe(Request kvRequest, boolean error, + int connections, int reqSize, int resSize, int networkLatency) { + + int authCount = 0, throttleCount = 0, retries = 0, retryDelay = 0; + RetryStats retryStats = kvRequest.getRetryStats(); + if (retryStats != null) { + authCount = retryStats.getNumExceptions( + AuthenticationException.class); + authCount += retryStats.getNumExceptions( + SecurityInfoNotReadyException.class); + + throttleCount = retryStats.getNumExceptions( + ThrottlingException.class); + + retries = retryStats.getRetries(); + retryDelay = retryStats.getDelayMs(); + } + + int rateLimitDelay = kvRequest.getRateLimitDelayedMs(); ReqStats rStat; - requestClass = requestClass != null && requestClass.endsWith("Request") + String requestClass = kvRequest.getClass().getSimpleName(); + requestClass = requestClass.endsWith("Request") ? requestClass.substring(0, requestClass.length() - 7) : requestClass; @@ -391,7 +562,8 @@ void observe(String requestClass, boolean error, int retries, // registered in the REQUEST_KEYS array. if (rStat == null) { ReqStats newStat = new ReqStats(); - if (statsConfig.getProfile() == StatsConfig.Profile.FULL) { + if (statsConfig.getProfile().ordinal() >= + StatsConfig.Profile.MORE.ordinal()) { newStat.wireLatencyPercentile = new Percentile(); } synchronized (requests) { @@ -404,15 +576,32 @@ void observe(String requestClass, boolean error, int retries, } rStat.observe(error, retries, retryDelay, rateLimitDelay, authCount, - throttleCount, reqSize, resSize, wireLatency); + throttleCount, reqSize, resSize, networkLatency); connectionStats.observe(connections); + + if (extraQueryStats != null) { + if (kvRequest instanceof QueryRequest) { + QueryRequest queryRequest = (QueryRequest)kvRequest; + + extraQueryStats.observeQuery(queryRequest, error, retries, + retryDelay, rateLimitDelay, authCount, throttleCount, + reqSize, resSize, networkLatency); + } + } + } + + public void observeQuery(QueryRequest qreq) { + if (extraQueryStats != null) { + extraQueryStats.observeQuery(qreq); + } } /** * Shuts down the time scheduler. */ void shutdown() { + logClientStats(); service.shutdown(); } } diff --git a/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java b/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java index b62c3aad..9b74ce4e 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java +++ b/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java @@ -4,15 +4,10 @@ import java.util.logging.Level; import java.util.logging.Logger; -import oracle.nosql.driver.ReadThrottlingException; -import oracle.nosql.driver.SecurityInfoNotReadyException; import oracle.nosql.driver.StatsConfig; -import oracle.nosql.driver.ThrottlingException; -import oracle.nosql.driver.WriteThrottlingException; import oracle.nosql.driver.httpclient.HttpClient; -import oracle.nosql.driver.kv.AuthenticationException; +import oracle.nosql.driver.ops.QueryRequest; import oracle.nosql.driver.ops.Request; -import oracle.nosql.driver.ops.RetryStats; public class StatsConfigImpl implements StatsConfig { @@ -170,62 +165,25 @@ public void shutdown() { } } - void observe(Request kvRequest, int wireTime, + void observe(Request kvRequest, int networkLatency, int reqSize, int resSize) { if (stats != null && enableCollection) { - String requestClass = kvRequest.getClass().getSimpleName(); - int auth = 0, throttle = 0, retries = 0, retryDelay = 0; - RetryStats retryStats = kvRequest.getRetryStats(); - if (retryStats != null) { - - auth = retryStats.getNumExceptions( - AuthenticationException.class); - auth += retryStats.getNumExceptions( - SecurityInfoNotReadyException.class); - - throttle = retryStats.getNumExceptions( - ThrottlingException.class); - - retries = retryStats.getRetries(); - retryDelay = retryStats.getDelayMs(); - } - - int rateLimitDelay = kvRequest.getRateLimitDelayedMs(); - - stats.observe(requestClass, false, retries, retryDelay, - rateLimitDelay, auth, throttle, + stats.observe(kvRequest, false, httpClient.getAcquiredChannelCount(), - reqSize, resSize, wireTime); + reqSize, resSize, networkLatency); } } void observeError(Request kvRequest) { if (stats != null && enableCollection) { - String requestClass = kvRequest.getClass().getSimpleName(); - int authCount = 0, throttleCount = 0, retryCount = 0, retryDelay = 0; - RetryStats retryStats = kvRequest.getRetryStats(); - if (retryStats != null) { - authCount = retryStats.getNumExceptions( - AuthenticationException.class); - authCount += retryStats.getNumExceptions( - SecurityInfoNotReadyException.class); - - throttleCount = retryStats.getNumExceptions( - ThrottlingException.class); - throttleCount += retryStats.getNumExceptions( - ReadThrottlingException.class); - throttleCount += retryStats.getNumExceptions( - WriteThrottlingException.class); - - retryCount = retryStats.getRetries(); - retryDelay = retryStats.getDelayMs(); - } - - int rateLimitDelay = kvRequest.getRateLimitDelayedMs(); - - stats.observeError(requestClass, retryCount, retryDelay, - authCount, throttleCount, rateLimitDelay, + stats.observeError(kvRequest, httpClient.getAcquiredChannelCount()); } } + + public void observeQuery(QueryRequest qreq) { + if (stats != null && enableCollection) { + stats.observeQuery(qreq); + } + } } diff --git a/driver/src/main/java/oracle/nosql/driver/ops/PreparedStatement.java b/driver/src/main/java/oracle/nosql/driver/ops/PreparedStatement.java index cb104870..5c5c930c 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/PreparedStatement.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/PreparedStatement.java @@ -302,7 +302,10 @@ public final byte[] getStatement() { * @return the driver portion of the query plan as a string */ public String printDriverPlan() { - return driverQueryPlan.display(); + if (driverQueryPlan != null) { + return driverQueryPlan.display(); + } + return null; } /** diff --git a/driver/src/main/java/oracle/nosql/driver/ops/QueryRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/QueryRequest.java index accc038e..1179ede8 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/QueryRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/QueryRequest.java @@ -454,6 +454,11 @@ public QueryRequest setMathContext(MathContext mathContext) { * @return the statement, or null if it has not been set */ public String getStatement() { + + if (statement == null && preparedStatement != null) { + return preparedStatement.getSQLText(); + } + return statement; } From 16a315ad0cb9caaf43049f4fdcfdaefe27158aee Mon Sep 17 00:00:00 2001 From: Cezar Andrei Date: Wed, 22 Sep 2021 13:01:40 -0500 Subject: [PATCH 07/13] Remove double getAcquiredChannelCount(). --- .../main/java/oracle/nosql/driver/httpclient/HttpClient.java | 4 ---- 1 file changed, 4 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 f03ba7a2..3ef19725 100644 --- a/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java +++ b/driver/src/main/java/oracle/nosql/driver/httpclient/HttpClient.java @@ -374,8 +374,4 @@ public void operationComplete(ChannelFuture future) { } }); } - - public int getAcquiredChannelCount() { - return pool.acquiredChannelCount(); - } } From 84f5b584f7d818e3a1d37bb089c7cc7ccf343e66 Mon Sep 17 00:00:00 2001 From: Cezar Andrei Date: Wed, 6 Oct 2021 14:35:32 -0500 Subject: [PATCH 08/13] Fix Percentile to be more exact and avoid errors for negative values. Renamed wireLatency to networkLatency. Using capital L for long numerals. Fix initial message to conform to JSON. --- .../java/oracle/nosql/driver/http/Stats.java | 47 ++++++++----------- .../nosql/driver/http/StatsConfigImpl.java | 13 ++--- .../oracle/nosql/driver/ops/QueryResult.java | 1 - 3 files changed, 27 insertions(+), 34 deletions(-) diff --git a/driver/src/main/java/oracle/nosql/driver/http/Stats.java b/driver/src/main/java/oracle/nosql/driver/http/Stats.java index 0373b9a2..a6aa68cd 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/Stats.java +++ b/driver/src/main/java/oracle/nosql/driver/http/Stats.java @@ -22,7 +22,6 @@ import oracle.nosql.driver.ops.QueryRequest; import oracle.nosql.driver.ops.Request; import oracle.nosql.driver.ops.RetryStats; -import oracle.nosql.driver.query.PlanIter; import oracle.nosql.driver.values.ArrayValue; import oracle.nosql.driver.values.FieldValue; import oracle.nosql.driver.values.JsonOptions; @@ -31,7 +30,7 @@ import oracle.nosql.driver.values.TimestampValue; -class Stats { +public class Stats { private static String[] REQUEST_KEYS = new String[] { "Delete", "Get", "GetIndexes", "GetTable", @@ -193,14 +192,14 @@ synchronized long getPercentile(double percentile) { return -1; } - if (values.size() == 1) { - return values.get(0); - } - values.sort(Comparator.comparingLong(Long::longValue)); - int index = (int)Math.round(percentile * values.size()); + int index = (int)Math.round(percentile * values.size() - 1); + + if (index < 0) { + index = 0; + } if (index >= values.size()) { - index = values.size() -1; + index = values.size() - 1; } return values.get(index); } @@ -298,18 +297,12 @@ synchronized void observeQuery(QueryRequest queryRequest) { if (queryRequest.isPrepared() && queryRequest.isSimpleQuery()) { qStat.simple++; } - - if (queryRequest.getPreparedStatement() != null && - queryRequest.getPreparedStatement().driverPlan() != null) { - PlanIter drvPlan = queryRequest.getPreparedStatement().driverPlan(); - drvPlan.getKind(); - } } synchronized void observeQuery(QueryRequest queryRequest, boolean error, int retries, int retryDelay, int rateLimitDelay, int authCount, int throttleCount, int reqSize, int resSize, - int wireLatency) { + int networkLatency) { QueryEntryStat qStat = getExtraQueryStat(queryRequest); @@ -332,13 +325,13 @@ synchronized void observeQuery(QueryRequest queryRequest, boolean error, resSize); qStat.reqStats.resSizeMax = Math.max(qStat.reqStats.resSizeMax, resSize); - qStat.reqStats.networkLatencySum += wireLatency; + qStat.reqStats.networkLatencySum += networkLatency; qStat.reqStats.networkLatencyMin = - Math.min(qStat.reqStats.networkLatencyMin, wireLatency); + Math.min(qStat.reqStats.networkLatencyMin, networkLatency); qStat.reqStats.networkLatencyMax = - Math.max(qStat.reqStats.networkLatencyMax, wireLatency); + Math.max(qStat.reqStats.networkLatencyMax, networkLatency); if (qStat.reqStats.wireLatencyPercentile != null) { - qStat.reqStats.wireLatencyPercentile.addValue(wireLatency); + qStat.reqStats.wireLatencyPercentile.addValue(networkLatency); } } @@ -425,17 +418,17 @@ synchronized void clear() { // To log stats at the top of the hour, calculate delay until first // occurrence. Note: First interval can be smaller than the rest. - long delay = 1000l * statsConfig.getInterval() - - (( 1000l * 60l * localTime.getMinute() + - 1000l * localTime.getSecond() + - localTime.getNano() / 1000000l) % - (1000l * statsConfig.getInterval())); + long delay = 1000L * statsConfig.getInterval() - + ((1000L * 60L * localTime.getMinute() + + 1000L * localTime.getSecond() + + localTime.getNano() / 1000000L) % + (1000L * statsConfig.getInterval())); service = Executors .newSingleThreadScheduledExecutor(); service.scheduleAtFixedRate(runnable, delay, - 1000l * statsConfig.getInterval(), + 1000L * statsConfig.getInterval(), TimeUnit.MILLISECONDS); startTime = System.currentTimeMillis(); @@ -458,7 +451,7 @@ private void logClientStats() { // Output stats to logger. String json = fvStats.toJson(statsConfig.getPrettyPrint() ? JsonOptions.PRETTY : null); - statsConfig.getLogger().log(Level.INFO, statsConfig.LOG_PREFIX + json); + statsConfig.getLogger().log(Level.INFO, StatsConfigImpl.LOG_PREFIX + json); } private FieldValue generateFieldValueStats() { @@ -496,7 +489,7 @@ private FieldValue generateFieldValueStats() { /** * Clear all collected stats. */ - void clearStats() { + private void clearStats() { for (String key : REQUEST_KEYS) { requests.get(key).clear(); } diff --git a/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java b/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java index 9b74ce4e..7d1c3a3b 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java +++ b/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java @@ -67,12 +67,13 @@ public class StatsConfigImpl if (profile != Profile.NONE) { logger.setLevel(Level.INFO); logger.log(Level.INFO, LOG_PREFIX + - "{\"sdkName\"=\"Oracle NoSQL SDK for Java\", " + - "\"sdkVersion\":\"" + libraryVersion + "\", " + - "clientId=\"" + id + "\",\"profile\":\"" + profile + "\", " + - "\"intervalSec\"=" + interval + - ", \"prettyPrint\"=" + prettyPrint + - ", \"rateLimitingEnabled\"=" + rateLimitingEnabled + "}"); + "{\"sdkName\" : \"Oracle NoSQL SDK for Java" + + "\", \"sdkVersion\" : \"" + libraryVersion + + "\", \"clientId\" : \"" + id + + "\", \"profile\" : \"" + profile + + "\", \"intervalSec\" : " + interval + + ", \"prettyPrint\" : " + prettyPrint + + ", \"rateLimitingEnabled\" : " + rateLimitingEnabled + "}"); start(); } diff --git a/driver/src/main/java/oracle/nosql/driver/ops/QueryResult.java b/driver/src/main/java/oracle/nosql/driver/ops/QueryResult.java index b0fb4ff2..15e875a9 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/QueryResult.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/QueryResult.java @@ -12,7 +12,6 @@ import oracle.nosql.driver.Consistency; import oracle.nosql.driver.NoSQLHandle; import oracle.nosql.driver.RateLimiter; -import oracle.nosql.driver.ops.Result; import oracle.nosql.driver.query.QueryDriver; import oracle.nosql.driver.values.MapValue; From 5c34f853771603eeca38749b480e5964b5a0f246 Mon Sep 17 00:00:00 2001 From: Cezar Andrei Date: Wed, 27 Oct 2021 11:12:31 -0500 Subject: [PATCH 09/13] Added stats config options to NoSQLConfig and renamed StatsConfig into StatsControl. Added copyright comments to new files. Renamed: count to httpRequestCount, networkLatencyMs to httpRequestLatencyMs, countAPI to count for query requests and stmt to query. Added stacktrace to log when there is an exception during stats logging. Using request.getName() for all request types. --- .../java/oracle/nosql/driver/NoSQLHandle.java | 7 +- .../nosql/driver/NoSQLHandleConfig.java | 142 ++++++++- .../java/oracle/nosql/driver/StatsConfig.java | 278 ------------------ .../oracle/nosql/driver/StatsControl.java | 162 ++++++++++ .../java/oracle/nosql/driver/http/Client.java | 30 +- .../nosql/driver/http/NoSQLHandleImpl.java | 7 +- .../java/oracle/nosql/driver/http/Stats.java | 228 +++++++------- ...sConfigImpl.java => StatsControlImpl.java} | 87 ++---- .../nosql/driver/ops/DeleteRequest.java | 5 + .../nosql/driver/ops/GetIndexesRequest.java | 5 + .../oracle/nosql/driver/ops/GetRequest.java | 5 + .../nosql/driver/ops/GetTableRequest.java | 5 + .../nosql/driver/ops/ListTablesRequest.java | 5 + .../nosql/driver/ops/MultiDeleteRequest.java | 5 + .../nosql/driver/ops/PrepareRequest.java | 5 + .../oracle/nosql/driver/ops/PutRequest.java | 5 + .../oracle/nosql/driver/ops/QueryRequest.java | 5 + .../java/oracle/nosql/driver/ops/Request.java | 5 + .../nosql/driver/ops/SystemRequest.java | 5 + .../nosql/driver/ops/SystemStatusRequest.java | 5 + .../oracle/nosql/driver/ops/TableRequest.java | 5 + .../nosql/driver/ops/TableUsageRequest.java | 5 + .../driver/ops/WriteMultipleRequest.java | 5 + .../oracle/nosql/driver/package-info.java | 215 ++++++++++++++ 24 files changed, 767 insertions(+), 464 deletions(-) delete mode 100644 driver/src/main/java/oracle/nosql/driver/StatsConfig.java create mode 100644 driver/src/main/java/oracle/nosql/driver/StatsControl.java rename driver/src/main/java/oracle/nosql/driver/http/{StatsConfigImpl.java => StatsControlImpl.java} (54%) diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java index 0af264ab..b10b236a 100644 --- a/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java +++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java @@ -572,12 +572,9 @@ public SystemResult doSystemRequest(String statement, int pollIntervalMs); /** - * Returns configuration object for statistics. - * Statistics, when enabled, contain information like errors, delays, retries - * and wire latency agregated on types of requests. - * @return configuration object for statistics + * Returns an object that allows control over how statistics are collected. */ - public StatsConfig getStatsConfig(); + public StatsControl getStatsControl(); /** * Closes the handle, releasing its memory and network resources. Once diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java index 4a2231b9..12616dc0 100644 --- a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java +++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java @@ -16,12 +16,15 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.logging.Level; import java.util.logging.Logger; -import io.netty.handler.ssl.SslContext; import oracle.nosql.driver.Region.RegionProvider; +import oracle.nosql.driver.http.StatsControlImpl; import oracle.nosql.driver.iam.SignatureProvider; +import io.netty.handler.ssl.SslContext; + /** * NoSQLHandleConfig groups parameters used to configure a {@link * NoSQLHandle}. It also provides a way to default common parameters for use by @@ -196,6 +199,21 @@ public class NoSQLHandleConfig implements Cloneable { private String proxyUsername; private String proxyPassword; + /** + * Statistics configuration, optional. + */ + private final static String PROFILE_PROPERTY = + "com.oracle.nosql.sdk.nosqldriver.stats.profile"; + private final static String INTERVAL_PROPERTY = + "com.oracle.nosql.sdk.nosqldriver.stats.interval"; + private final static String PRETTY_PRINT_PROPERTY = + "com.oracle.nosql.sdk.nosqldriver.stats.pretty-print"; + /* Statistics logging interval in seconds. Default 600 sec, ie. 10 min. */ + private int statsInterval = 600; + private StatsControl.Profile statsProfile = StatsControl.Profile.NONE; + private boolean statsPrettyPrint = false; + private StatsControl.StatsHandler statsHandler = null; + /** * Specifies an endpoint or region id to use to connect to the Oracle * NoSQL Database Cloud Service or, if on-premise, the Oracle NoSQL @@ -240,6 +258,7 @@ public NoSQLHandleConfig(String endpoint) { super(); endpoint = checkRegionId(endpoint, null); this.serviceURL = createURL(endpoint, "/"); + setConfigFromEnvironment(); } /** @@ -290,6 +309,7 @@ public NoSQLHandleConfig(String endpoint, AuthorizationProvider provider) { endpoint = checkRegionId(endpoint, provider); this.serviceURL = createURL(endpoint, "/"); this.authProvider = provider; + setConfigFromEnvironment(); } /** @@ -313,6 +333,7 @@ public NoSQLHandleConfig(Region region, AuthorizationProvider provider) { this.serviceURL = createURL(region.endpoint(), "/"); this.region = region; this.authProvider = provider; + setConfigFromEnvironment(); } /** @@ -339,6 +360,7 @@ public NoSQLHandleConfig(AuthorizationProvider provider) { } this.serviceURL = createURL(region.endpoint(), "/"); this.authProvider = provider; + setConfigFromEnvironment(); } /** @@ -1184,6 +1206,90 @@ public SslContext getSslContext() { return sslCtx; } + /** + * Sets interval size in seconds for logging statistics. + * Default interval is 600 seconds, i.e. 10 min. + * + * @ return this + */ + public NoSQLHandleConfig setStatsInterval(int statsInterval) { + if (statsInterval < 1) { + throw new IllegalArgumentException("Stats interval can not be " + + "less than 1 second."); + } + this.statsInterval = statsInterval; + return this; + } + + + /** + * Returns the current interval for logging statistics. + * Default interval is 600 seconds, i.e. 10 min. + */ + public int getStatsInterval() { + return this.statsInterval; + } + + /** + * Set the statistics collection profile. + * Default profile is NONE. + * + * @return this + */ + public NoSQLHandleConfig setStatsProfile(StatsControl.Profile statsProfile) { + this.statsProfile = statsProfile; + return this; + } + + /** + * Returns the statistics collection profile. + * Default profile is NONE. + */ + public StatsControl.Profile getStatsProfile() { + return this.statsProfile; + } + + /** + * Enable JSON pretty print for easier human reading when logging + * statistics. + * Default is disabled. + * + * @return this + */ + public NoSQLHandleConfig setStatsPrettyPrint(boolean statsPrettyPrint) { + this.statsPrettyPrint = statsPrettyPrint; + return this; + } + + /** + * Returns the current JSON pretty print flag for logging statistics. + * Default is disabled. + */ + public boolean getStatsPrettyPrint() { + return this.statsPrettyPrint; + } + + /** + * Registers a handler that is called every time the statistics are logged. + * @param statsHandler User defined StatsHandler. + * + * @return this + */ + public NoSQLHandleConfig registerStatsHandler( + StatsControl.StatsHandler statsHandler) { + this.statsHandler = statsHandler; + return this; + } + + /** + * Returns the registered statistics handler, otherwise null. + * + * @return this + */ + public StatsControl.StatsHandler getStatsHandler() { + return this.statsHandler; + } + @Override public NoSQLHandleConfig clone() { try { @@ -1220,4 +1326,38 @@ static private void checkRegion(Region region, } } + private void setConfigFromEnvironment() { + String profileProp = System.getProperty(PROFILE_PROPERTY); + if (profileProp != null) { + try { + setStatsProfile(StatsControl.Profile.valueOf( + profileProp.toUpperCase())); + } catch (IllegalArgumentException iae) { + if (logger != null) { + logger.log(Level.SEVERE, StatsControlImpl.LOG_PREFIX + + "Invalid profile value for system property " + + PROFILE_PROPERTY + ": " + profileProp); + } + } + } + + String intervalProp = System.getProperty(INTERVAL_PROPERTY); + if (intervalProp != null) { + try { + setStatsInterval(Integer.valueOf(intervalProp)); + } catch (NumberFormatException nfe) { + if (logger != null) { + logger.log(Level.SEVERE, "Invalid integer value for " + + "system property " + INTERVAL_PROPERTY + ": " + + intervalProp); + } + } + } + + String ppProp = System.getProperty(PRETTY_PRINT_PROPERTY); + if (ppProp != null && ("true".equals(ppProp.toLowerCase()) || "1".equals(ppProp) || + "on".equals(ppProp.toLowerCase()))) { + statsPrettyPrint = Boolean.valueOf(ppProp); + } + } } diff --git a/driver/src/main/java/oracle/nosql/driver/StatsConfig.java b/driver/src/main/java/oracle/nosql/driver/StatsConfig.java deleted file mode 100644 index 6c208d87..00000000 --- a/driver/src/main/java/oracle/nosql/driver/StatsConfig.java +++ /dev/null @@ -1,278 +0,0 @@ -package oracle.nosql.driver; - -import java.util.function.Consumer; -import java.util.logging.Logger; - -import oracle.nosql.driver.values.FieldValue; - -/** - *

    This interface allows user to setup the collection of driver statistics.

    - * - * The statistics data is collected for an interval of time. At the end of the - * interval, the stats data is logged in a specified JSON format that can be - * filtered and parsed. After the logging, the counters are cleared and - * collection of data resumes.

    - * - * Collection intervals are aligned to the top of the hour. This means first - * interval logs may contain stats for a shorter interval.

    - * - * Collection of stats are controlled by the following system properties:

  • - * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile=[none|regular|more|all] - * Specifies the stats profile: none - disabled, - * regular - per request: counters, errors, latencies, delays, retries - * more - stats above with 95th and 99th percentile latencies - * all - stats above with per query information.
  • - * - * -Dcom.oracle.nosql.sdk.nosqldriver.stats.interval=600 Interval in - * seconds to log the stats, by default is 10 minutes.
  • - * - * -Dcom.oracle.nosql.sdk.nosqldriver.stats.pretty-print=true Option - * to enable pretty printing of the JSON data, default value is false
  • - * - * Collection of stats can also be used by using the API: - * NoSQLHandleConfig config = new NoSQLHandleConfig( endpoint ); - * NoSQLHandle handle = NoSQLHandleFactory.createNoSQLHandle(config); - * - * StatsConfig statsConfig = handle.getStatsConfig(); - * statsConfig.setProfile(StatsConfig.Profile.REGULAR); - * statsConfig.setInterval(600); - * statsConfig.setPrettyPrint(false); - * statsConfig.registerHandler( - * new StatsConfig.StatsHandler() { - * public void accept(FieldValue jsonStats) { - * System.out.println("!!! Got a stat: " + jsonStats); - * } - * }); - * statsConfig.start(); - * - * //... application code - * - * statsConfig.stop(); - * handle.close(); - *

    - * - * The following is an example of stats log entry using the ALL - * profile: - * INFO: ONJS:Monitoring stats|{ - * "clientId" : "b7bc7734", - * "startTime" : "2021-09-20T20:11:42Z", - * "endTime" : "2021-09-20T20:11:47Z", - * "requests" : [{ - * "name" : "Get", - * "count" : 2, - * "errors" : 0, - * "networkLatencyMs" : { - * "min" : 4, - * "avg" : 4.5, - * "max" : 5, - * "95th" : 5, - * "99th" : 5 - * }, - * "requestSize" : { - * "min" : 42, - * "avg" : 42.5, - * "max" : 43 - * }, - * "resultSize" : { - * "min" : 193, - * "avg" : 206.5, - * "max" : 220 - * }, - * "rateLimitDelayMs" : 0, - * "retry" : { - * "delayMs" : 0, - * "authCount" : 0, - * "throttleCount" : 0, - * "count" : 0 - * } - * }, { - * "name" : "Query", - * "count" : 14, - * "errors" : 0, - * "networkLatencyMs" : { - * "min" : 3, - * "avg" : 13.0, - * "max" : 32, - * "95th" : 32, - * "99th" : 32 - * }, - * "resultSize" : { - * "min" : 146, - * "avg" : 7379.71, - * "max" : 10989 - * }, - * "requestSize" : { - * "min" : 65, - * "avg" : 709.85, - * "max" : 799 - * }, - * "rateLimitDelayMs" : 0, - * "retry" : { - * "delayMs" : 0, - * "authCount" : 0, - * "throttleCount" : 0, - * "count" : 0 - * } - * }, { - * "name" : "Put", - * "count" : 1002, - * "errors" : 0, - * "networkLatencyMs" : { - * "min" : 1, - * "avg" : 4.41, - * "max" : 80, - * "95th" : 8, - * "99th" : 20 - * }, - * "requestSize" : { - * "min" : 90, - * "avg" : 90.16, - * "max" : 187 - * }, - * "resultSize" : { - * "min" : 58, - * "avg" : 58.0, - * "max" : 58 - * }, - * "rateLimitDelayMs" : 0, - * "retry" : { - * "delayMs" : 0, - * "authCount" : 0, - * "throttleCount" : 0, - * "count" : 0 - * } - * }], - * "queries" : [{ - * "stmt" : "SELECT * FROM audienceData ORDER BY cookie_id", - * "plan" : "SFW([6])\n[\n FROM:\n RECV([3])\n [\n DistributionKind : ALL_PARTITIONS,\n Sort Fields : sort_gen,\n\n ] as $from-0\n\n SELECT:\n FIELD_STEP([6])\n [\n VAR_REF($from-0)([3]),\n audienceData\n ]\n]", - * "doesWrites" : false, - * "count" : 12, - * "unprepared" : 1, - * "simple" : 0, - * "countAPI" : 20, - * "errors" : 0, - * "networkLatencyMs" : { - * "min" : 8, - * "avg" : 14.58, - * "max" : 32, - * "95th" : 32, - * "99th" : 32 - * }, - * "requestSize" : { - * "min" : 65, - * "avg" : 732.5, - * "max" : 799 - * }, - * "resultSize" : { - * "min" : 914, - * "avg" : 8585.33, - * "max" : 10989 - * }, - * "rateLimitDelayMs" : 0, - * "retry" : { - * "delayMs" : 0, - * "authCount" : 0, - * "throttleCount" : 0, - * "count" : 0 - * } - * }], - * "connections" : { - * "min" : 1, - * "avg" : 9.58, - * "max" : 10 - * } - * } - *

    - */ -public interface StatsConfig { - - /** - * The following semantics are attached to the Profile: - * - NONE: no stats are logged. - * - REGULAR: per request: counters, errors, latencies, delays, retries - * - MORE: stats above with 95th and 99th percentile latencies. - * - ALL: stats above with per query information - */ - enum Profile { - NONE, REGULAR, MORE, ALL; - } - - /** - * Handler interface that user can register to get access to stats at - * the end of the interval. - */ - interface StatsHandler extends Consumer { - /** Stats are encoded in JSON format using the FieldValue API. */ - void accept(FieldValue jsonStats); - } - - /** - * Sets the logger to be used. - */ - StatsConfig setLogger(Logger logger); - - /** - * Returns the current logger. - */ - Logger getLogger(); - - /** - * Sets interval size in seconds. - * Default interval is 600 seconds, i.e. 10 min. - */ - StatsConfig setInterval(int interval); - - - /** - * Returns the current collection interval. - * Default interval is 600 seconds, i.e. 10 min. - */ - int getInterval(); - - /** - * Set the collection profile. - * Default profile is NONE. - */ - StatsConfig setProfile(Profile profile); - - /** - * Returns the collection profile. - * Default profile is NONE. - */ - Profile getProfile(); - - /** - * Enable JSON pretty print for easier human reading. - * Default is disabled. - */ - StatsConfig setPrettyPrint(boolean enablePrettyPrint); - - /** - * Returns the current JSON pretty print flag. - * Default is disabled. - */ - boolean getPrettyPrint(); - - /** - * Registers a stats handler. - * @param handler User defined StatsHandler. - */ - void registerHandler(StatsHandler handler); - - /** - * Collection of stats is enabled only between start and stop or from the - * beginning if system property - * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile= is not "none". - */ - void start(); - - /** - * Stops collection of stats. - */ - void stop(); - - /** - * Returns true if collection of stats is enabled, otherwise returns false. - */ - boolean isStarted(); -} diff --git a/driver/src/main/java/oracle/nosql/driver/StatsControl.java b/driver/src/main/java/oracle/nosql/driver/StatsControl.java new file mode 100644 index 00000000..a998842f --- /dev/null +++ b/driver/src/main/java/oracle/nosql/driver/StatsControl.java @@ -0,0 +1,162 @@ +/*- + * Copyright (c) 2011, 2021 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.function.Consumer; + +import oracle.nosql.driver.values.FieldValue; + +/** + *

    This interface allows user to setup the collection of driver statistics.

    + * + * The statistics data is collected for an interval of time. At the end of the + * interval, the stats data is logged in a specified JSON format that can be + * filtered and parsed. After the logging, the counters are cleared and + * collection of data resumes.

    + * + * Collection intervals are aligned to the top of the hour. This means first + * interval logs may contain stats for a shorter interval.

    + * + * Collection of stats are controlled by the following system properties:

  • + * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile=[none|regular|more|all] + * Specifies the stats profile: none - disabled, + * regular - per request: counters, errors, latencies, delays, retries + * more - stats above with 95th and 99th percentile latencies + * all - stats above with per query information.
  • + * + * -Dcom.oracle.nosql.sdk.nosqldriver.stats.interval=600 Interval in + * seconds to log the stats, by default is 10 minutes.
  • + * + * -Dcom.oracle.nosql.sdk.nosqldriver.stats.pretty-print=true Option + * to enable pretty printing of the JSON data, default value is false
  • + * + * Statistics can also be enabled by using the API: + * {@link NoSQLHandleConfig#setStatsProfile(Profile)} or + * {@link StatsControl#setProfile(Profile)}. At runtime stats + * collection can be used selectively by using {@link StatsControl#start()} and + * {@link StatsControl#stop()}. The following example shows how to use a stats + * handler:
    + *     NoSQLHandleConfig config = new NoSQLHandleConfig( endpoint );
    + *     config.setStatsProfile(StatsControl.Profile.REGULAR);
    + *     config.setStatsInterval(600);
    + *     config.setStatsPrettyPrint(false);
    + *     config.registerStatsHandler(
    + *         new StatsControl.StatsHandler() {
    + *             public void accept(FieldValue jsonStats) {
    + *                 System.out.println("!!! Got a stat: " + jsonStats);
    + *             }
    + *         });
    + *     NoSQLHandle handle = NoSQLHandleFactory.createNoSQLHandle(config);
    + *
    + *     StatsControl statsControl = handle.getStatsControl();
    + *
    + *     //... application code without stats
    + *
    + *     // enable observations
    + *     statsControl.start();
    + *
    + *     //... application code with REGULAR stats
    + *
    + *     // For particular parts of code profile can be changed collect more stats.
    + *     statsControl.setProfile(StatsControl.Profile.ALL)
    + *     //... more sensitive code with ALL stats
    + *     statsControl.setProfile(StatsControl.Profile.REGULAR)
    + *
    + *     //... application code with REGULAR stats
    + *
    + *     // disable observations
    + *     statsControl.stop();
    + *
    + *     // ... application code without stats
    + *     handle.close();
    + *     

    + * + * For a detailed statistics log entries structure and values see + * {@link oracle.nosql.driver}

    + */ +public interface StatsControl { + + /** + * The following semantics are attached to the Profile: + * - NONE: no stats are logged. + * - REGULAR: per request: counters, errors, latencies, delays, retries + * - MORE: stats above with 95th and 99th percentile latencies. + * - ALL: stats above with per query information + */ + enum Profile { + NONE, REGULAR, MORE, ALL; + } + + /** + * Handler interface that user can register to get access to stats at + * the end of the interval. + */ + interface StatsHandler extends Consumer { + /** Stats are encoded in JSON format using the FieldValue API. */ + void accept(FieldValue jsonStats); + } + + /** + * Returns the current collection interval. + * Default interval is 600 seconds, i.e. 10 min. + */ + int getInterval(); + + /** + * Set the collection profile. + * Default profile is NONE. + * + * @return this + */ + StatsControl setProfile(Profile profile); + + /** + * Returns the collection profile. + * Default profile is NONE. + */ + Profile getProfile(); + + /** + * Enable JSON pretty print for easier human reading. + * Default is disabled. + * + * @return this + */ + StatsControl setPrettyPrint(boolean enablePrettyPrint); + + /** + * Returns the current JSON pretty print flag. + * Default is disabled. + */ + boolean getPrettyPrint(); + + /** + * Registers a stats handler. + * @param handler User defined StatsHandler. + * + * @return this + */ + StatsControl registerStatsHandler(StatsHandler handler); + + /** + * Collection of stats is enabled only between start and stop or from the + * beginning if system property + * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile= is not "none". + */ + void start(); + + /** + * Stops collection of stats. + */ + void stop(); + + /** + * Returns true if collection of stats is enabled, otherwise returns false. + */ + boolean isStarted(); +} 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 98e26165..5eb38f90 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/Client.java +++ b/driver/src/main/java/oracle/nosql/driver/http/Client.java @@ -49,7 +49,7 @@ import oracle.nosql.driver.RetryHandler; import oracle.nosql.driver.RetryableException; import oracle.nosql.driver.SecurityInfoNotReadyException; -import oracle.nosql.driver.StatsConfig; +import oracle.nosql.driver.StatsControl; import oracle.nosql.driver.WriteThrottlingException; import oracle.nosql.driver.httpclient.HttpClient; import oracle.nosql.driver.httpclient.ResponseHandler; @@ -154,7 +154,7 @@ public class Client { /** * config for statistics */ - private StatsConfigImpl statsConfig; + private StatsControlImpl statsControl; public Client(Logger logger, NoSQLHandleConfig httpConfig) { @@ -225,7 +225,7 @@ public Client(Logger logger, threadPool = null; } - statsConfig = new StatsConfigImpl(NoSQLHandleConfig.getLibraryVersion(), + statsControl = new StatsControlImpl(config, logger, httpClient, httpConfig.getRateLimitingEnabled()); } @@ -240,7 +240,7 @@ public void shutdown() { return; } httpClient.shutdown(); - statsConfig.shutdown(); + statsControl.shutdown(); if (authProvider != null) { authProvider.close(); } @@ -301,7 +301,7 @@ public Result execute(Request kvRequest) { if (kvRequest.isQueryRequest()) { QueryRequest qreq = (QueryRequest)kvRequest; - statsConfig.observeQuery(qreq); + statsControl.observeQuery(qreq); /* * The following "if" may be true for advanced queries only. For @@ -560,7 +560,7 @@ public Result execute(Request kvRequest) { res.setRetryStats(kvRequest.getRetryStats()); kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.observe(kvRequest, Math.toIntExact(networkLatency), + statsControl.observe(kvRequest, Math.toIntExact(networkLatency), contentLength, resSize); return res; @@ -577,7 +577,7 @@ public Result execute(Request kvRequest) { continue; } kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.observeError(kvRequest); + statsControl.observeError(kvRequest); logInfo(logger, "Unexpected authentication exception: " + rae); throw new NoSQLException("Unexpected exception: " + @@ -640,13 +640,13 @@ public Result execute(Request kvRequest) { continue; } catch (NoSQLException nse) { kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.observeError(kvRequest); + statsControl.observeError(kvRequest); logFine(logger, "Client execute NoSQLException: " + nse.getMessage()); throw nse; /* pass through */ } catch (RuntimeException e) { kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.observeError(kvRequest); + statsControl.observeError(kvRequest); logFine(logger, "Client execute runtime exception: " + e.getMessage()); throw e; @@ -670,7 +670,7 @@ public Result execute(Request kvRequest) { continue; } catch (InterruptedException ie) { kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.observeError(kvRequest); + statsControl.observeError(kvRequest); logInfo(logger, "Client interrupted exception: " + ie.getMessage()); /* this exception shouldn't retry -- direct throw */ @@ -678,7 +678,7 @@ public Result execute(Request kvRequest) { ie.getMessage()); } catch (ExecutionException ee) { kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.observeError(kvRequest); + statsControl.observeError(kvRequest); logInfo(logger, "Unable to execute request: " + ee.getCause().getMessage()); /* is there a better exception? */ @@ -717,7 +717,7 @@ public Result execute(Request kvRequest) { } while (! timeoutRequest(startTime, timeoutMs, exception)); kvRequest.setRateLimitDelayedMs(rateDelayedMs); - statsConfig.observeError(kvRequest); + statsControl.observeError(kvRequest); throw new RequestTimeoutException(timeoutMs, "Request timed out after " + kvRequest.getNumRetries() + (kvRequest.getNumRetries() == 1 ? " retry. " : " retries. ") + @@ -1165,9 +1165,9 @@ public void enableRateLimiting(boolean enable, double usePercent) { } /** - * Returns the config object for client statistics. + * Returns the statistics control object. */ - StatsConfig getStatsConfig() { - return statsConfig; + StatsControl getStatsControl() { + return statsControl; } } 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 af6cdcf5..70c35bd7 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java +++ b/driver/src/main/java/oracle/nosql/driver/http/NoSQLHandleImpl.java @@ -8,14 +8,13 @@ package oracle.nosql.driver.http; import java.util.ArrayList; -import java.util.logging.Level; 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.StatsConfig; +import oracle.nosql.driver.StatsControl; import oracle.nosql.driver.UserInfo; import oracle.nosql.driver.iam.SignatureProvider; import oracle.nosql.driver.kv.StoreAccessTokenProvider; @@ -372,8 +371,8 @@ public SystemResult doSystemRequest(String statement, } @Override - public StatsConfig getStatsConfig() { - return client.getStatsConfig(); + public StatsControl getStatsControl() { + return client.getStatsControl(); } /** diff --git a/driver/src/main/java/oracle/nosql/driver/http/Stats.java b/driver/src/main/java/oracle/nosql/driver/http/Stats.java index a6aa68cd..ee2592a4 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/Stats.java +++ b/driver/src/main/java/oracle/nosql/driver/http/Stats.java @@ -1,5 +1,14 @@ +/*- + * Copyright (c) 2011, 2021 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 java.io.PrintWriter; +import java.io.StringWriter; import java.sql.Timestamp; import java.time.LocalTime; import java.util.ArrayList; @@ -15,7 +24,7 @@ import java.util.logging.Level; import oracle.nosql.driver.SecurityInfoNotReadyException; -import oracle.nosql.driver.StatsConfig; +import oracle.nosql.driver.StatsControl; import oracle.nosql.driver.ThrottlingException; import oracle.nosql.driver.kv.AuthenticationException; import oracle.nosql.driver.ops.PreparedStatement; @@ -29,16 +38,19 @@ import oracle.nosql.driver.values.StringValue; import oracle.nosql.driver.values.TimestampValue; - +/** + * Stores and logs all collected statistics. + */ public class Stats { + // The following keys must match the request names. private static String[] REQUEST_KEYS = new String[] { "Delete", "Get", "GetIndexes", "GetTable", - "ListTables", "MultiDelete", "Prepare", "Put", "Query", "Read", + "ListTables", "MultiDelete", "Prepare", "Put", "Query", "System", "SystemStatus", "Table", "TableUsage", "WriteMultiple", "Write"}; private ScheduledExecutorService service; - private StatsConfigImpl statsConfig; + private StatsControlImpl statsControl; private long startTime; private long endTime; @@ -46,8 +58,11 @@ public class Stats { private ConnectionStats connectionStats = new ConnectionStats(); private ExtraQueryStats extraQueryStats; + /** + * Stores per type of request statistics. + */ private static class ReqStats { - private long count = 0; + private long httpRequestCount = 0; private long errors = 0; private int reqSizeMin = Integer.MAX_VALUE; private int reqSizeMax = 0; @@ -60,16 +75,16 @@ private static class ReqStats { private int retryCount = 0; private int retryDelayMs = 0; private int rateLimitDelayMs = 0; - private int networkLatencyMin = Integer.MAX_VALUE; - private int networkLatencyMax = 0; - private long networkLatencySum = 0; - private Percentile wireLatencyPercentile; + private int requestLatencyMin = Integer.MAX_VALUE; + private int requestLatencyMax = 0; + private long requestLatencySum = 0; + private Percentile requestLatencyPercentile; synchronized void observe(boolean error, int retries, int retryDelay, int rateLimitDelay, int authCount, int throttleCount, int reqSize, - int resSize, int networkLatency) { + int resSize, int requestLatency) { - this.count++; + this.httpRequestCount++; this.retryCount += retries; this.retryDelayMs += retryDelay; this.retryAuthCount += authCount; @@ -88,20 +103,20 @@ synchronized void observe(boolean error, int retries, int retryDelay, this.resSizeMax = Math.max(this.resSizeMax, resSize); this.resSizeSum += resSize; - this.networkLatencyMin = - Math.min(this.networkLatencyMin, networkLatency); - this.networkLatencyMax = - Math.max(this.networkLatencyMax, networkLatency); - this.networkLatencySum += networkLatency; + this.requestLatencyMin = + Math.min(this.requestLatencyMin, requestLatency); + this.requestLatencyMax = + Math.max(this.requestLatencyMax, requestLatency); + this.requestLatencySum += requestLatency; - if (this.wireLatencyPercentile != null) { - this.wireLatencyPercentile.addValue(networkLatency); + if (this.requestLatencyPercentile != null) { + this.requestLatencyPercentile.addValue(requestLatency); } } } synchronized void toJSON(String requestName, ArrayValue reqArray) { - if (count > 0) { + if (httpRequestCount > 0) { MapValue mapValue = new MapValue(); mapValue.put("name", requestName); @@ -111,7 +126,7 @@ synchronized void toJSON(String requestName, ArrayValue reqArray) { } private void toMapValue(MapValue mapValue) { - mapValue.put("count", count); + mapValue.put("httpRequestCount", httpRequestCount); mapValue.put("errors", errors); MapValue retry = new MapValue(); @@ -122,26 +137,27 @@ private void toMapValue(MapValue mapValue) { mapValue.put("retry", retry); mapValue.put("rateLimitDelayMs", rateLimitDelayMs); - if (networkLatencyMax > 0) { + if (requestLatencyMax > 0) { MapValue latency = new MapValue(); - latency.put("min", networkLatencyMin); - latency.put("max", networkLatencyMax); - latency.put("avg", - 1.0 * networkLatencySum / (count - errors)); - if (wireLatencyPercentile != null) { + latency.put("min", requestLatencyMin); + latency.put("max", requestLatencyMax); + latency.put("avg", 1.0 * requestLatencySum / + (httpRequestCount - errors)); + if (requestLatencyPercentile != null) { latency.put("95th", - wireLatencyPercentile.get95thPercentile()); + requestLatencyPercentile.get95thPercentile()); latency.put("99th", - wireLatencyPercentile.get99thPercentile()); + requestLatencyPercentile.get99thPercentile()); } - mapValue.put("networkLatencyMs", latency); + mapValue.put("httpRequestLatencyMs", latency); } if (reqSizeMax > 0) { MapValue reqSize = new MapValue(); reqSize.put("min", reqSizeMin); reqSize.put("max", reqSizeMax); - reqSize.put("avg", 1.0 * reqSizeSum / (count - errors)); + reqSize.put("avg", 1.0 * reqSizeSum / + (httpRequestCount - errors)); mapValue.put("requestSize", reqSize); } @@ -149,13 +165,14 @@ private void toMapValue(MapValue mapValue) { MapValue resSize = new MapValue(); resSize.put("min", resSizeMin); resSize.put("max", resSizeMax); - resSize.put("avg", 1.0 * resSizeSum / (count - errors)); + resSize.put("avg", 1.0 * resSizeSum / + (httpRequestCount - errors)); mapValue.put("resultSize", resSize); } } synchronized void clear() { - count = 0; + httpRequestCount = 0; errors = 0; reqSizeMin = Integer.MAX_VALUE; reqSizeMax = 0; @@ -168,15 +185,18 @@ synchronized void clear() { retryCount = 0; retryDelayMs = 0; rateLimitDelayMs = 0; - networkLatencyMin = Integer.MAX_VALUE; - networkLatencyMax = 0; - networkLatencySum = 0; - if (wireLatencyPercentile != null) { - wireLatencyPercentile.clear(); + requestLatencyMin = Integer.MAX_VALUE; + requestLatencyMax = 0; + requestLatencySum = 0; + if (requestLatencyPercentile != null) { + requestLatencyPercentile.clear(); } } } + /** + * Percentile class helps storing and calculating percentiles. + */ private static class Percentile { private List values; @@ -219,6 +239,10 @@ synchronized void clear() { } } + /** + * Stores connection aggregated statistics. Min, max, avg show the number of + * simultaneously opened connections. + */ private static class ConnectionStats { private long count; private int min = Integer.MAX_VALUE; @@ -254,22 +278,28 @@ synchronized void clear() { } } + /** + * Stores more detailed per query statistics. + */ static class ExtraQueryStats { + /** + * Stores statistics for a query type. + */ static class QueryEntryStat { - long countAPI; + long count; long unprepared; - long simple; + boolean simple; boolean doesWrites; ReqStats reqStats; String plan; - QueryEntryStat(StatsConfig statsConfig, QueryRequest queryRequest) { + QueryEntryStat(StatsControl statsConfig, QueryRequest queryRequest) { reqStats = new ReqStats(); if (statsConfig.getProfile().ordinal() >= - StatsConfig.Profile.MORE.ordinal()) { - reqStats.wireLatencyPercentile = new Percentile(); + StatsControl.Profile.MORE.ordinal()) { + reqStats.requestLatencyPercentile = new Percentile(); } PreparedStatement pStmt = queryRequest.getPreparedStatement(); @@ -281,32 +311,31 @@ static class QueryEntryStat { } private Map queries = new HashMap<>(); - private StatsConfig statsConfig; + private StatsControl statsConfig; - ExtraQueryStats(StatsConfig statsConfig) { + ExtraQueryStats(StatsControl statsConfig) { this.statsConfig = statsConfig; } synchronized void observeQuery(QueryRequest queryRequest) { QueryEntryStat qStat = getExtraQueryStat(queryRequest); - qStat.countAPI++; + qStat.count++; if (!queryRequest.isPrepared()) { qStat.unprepared++; - } - if (queryRequest.isPrepared() && queryRequest.isSimpleQuery()) { - qStat.simple++; + } else { + qStat.simple = queryRequest.isSimpleQuery(); } } synchronized void observeQuery(QueryRequest queryRequest, boolean error, int retries, int retryDelay, int rateLimitDelay, int authCount, int throttleCount, int reqSize, int resSize, - int networkLatency) { + int requestLatency) { QueryEntryStat qStat = getExtraQueryStat(queryRequest); - qStat.reqStats.count++; + qStat.reqStats.httpRequestCount++; if (error) { qStat.reqStats.errors++; } @@ -325,13 +354,13 @@ synchronized void observeQuery(QueryRequest queryRequest, boolean error, resSize); qStat.reqStats.resSizeMax = Math.max(qStat.reqStats.resSizeMax, resSize); - qStat.reqStats.networkLatencySum += networkLatency; - qStat.reqStats.networkLatencyMin = - Math.min(qStat.reqStats.networkLatencyMin, networkLatency); - qStat.reqStats.networkLatencyMax = - Math.max(qStat.reqStats.networkLatencyMax, networkLatency); - if (qStat.reqStats.wireLatencyPercentile != null) { - qStat.reqStats.wireLatencyPercentile.addValue(networkLatency); + qStat.reqStats.requestLatencySum += requestLatency; + qStat.reqStats.requestLatencyMin = + Math.min(qStat.reqStats.requestLatencyMin, requestLatency); + qStat.reqStats.requestLatencyMax = + Math.max(qStat.reqStats.requestLatencyMax, requestLatency); + if (qStat.reqStats.requestLatencyPercentile != null) { + qStat.reqStats.requestLatencyPercentile.addValue(requestLatency); } } @@ -366,8 +395,8 @@ synchronized void toJSON(MapValue root) { MapValue queryVal = new MapValue(); queryArr.add(queryVal); - queryVal.put("stmt", key == null ? "null" : key); - queryVal.put("countAPI", val.countAPI); + queryVal.put("query", key == null ? "null" : key); + queryVal.put("count", val.count); queryVal.put("unprepared", val.unprepared); queryVal.put("simple", val.simple); queryVal.put("doesWrites", val.doesWrites); @@ -384,23 +413,23 @@ synchronized void clear() { } } - Stats(StatsConfigImpl statsConfig) { - this.statsConfig = statsConfig; + Stats(StatsControlImpl statsControl) { + this.statsControl = statsControl; // Fill in the stats objects for (String key : REQUEST_KEYS) { ReqStats reqStats = new ReqStats(); requests.put(key, reqStats); - if (statsConfig.getProfile().ordinal() >= - StatsConfig.Profile.MORE.ordinal()) { - reqStats.wireLatencyPercentile = new Percentile(); + if (statsControl.getProfile().ordinal() >= + StatsControl.Profile.MORE.ordinal()) { + reqStats.requestLatencyPercentile = new Percentile(); } } - if (statsConfig.getProfile().ordinal() >= - StatsConfig.Profile.ALL.ordinal()) { - extraQueryStats = new ExtraQueryStats(statsConfig); + if (statsControl.getProfile().ordinal() >= + StatsControl.Profile.ALL.ordinal()) { + extraQueryStats = new ExtraQueryStats(statsControl); } // Setup the scheduler for interval logging @@ -408,9 +437,11 @@ synchronized void clear() { try { logClientStats(); } catch (RuntimeException re) { - statsConfig.getLogger().log(Level.INFO, - "Stats exception: " + re.getMessage()); - re.printStackTrace(); + StringWriter stackTrace = new StringWriter(); + re.printStackTrace(new PrintWriter(stackTrace)); + statsControl.getLogger().log(Level.INFO, + "Stats exception: " + re.getMessage() + "\n" + + stackTrace); } }; @@ -418,17 +449,17 @@ synchronized void clear() { // To log stats at the top of the hour, calculate delay until first // occurrence. Note: First interval can be smaller than the rest. - long delay = 1000L * statsConfig.getInterval() - + long delay = 1000L * statsControl.getInterval() - ((1000L * 60L * localTime.getMinute() + 1000L * localTime.getSecond() + localTime.getNano() / 1000000L) % - (1000L * statsConfig.getInterval())); + (1000L * statsControl.getInterval())); service = Executors .newSingleThreadScheduledExecutor(); service.scheduleAtFixedRate(runnable, delay, - 1000L * statsConfig.getInterval(), + 1000L * statsControl.getInterval(), TimeUnit.MILLISECONDS); startTime = System.currentTimeMillis(); @@ -442,16 +473,17 @@ private void logClientStats() { clearStats(); // Call user handle if configured. - StatsConfig.StatsHandler statsHandler = - statsConfig.getHandler(); + StatsControl.StatsHandler statsHandler = + statsControl.getHandler(); if (statsHandler != null) { statsHandler.accept(fvStats); } // Output stats to logger. - String json = fvStats.toJson(statsConfig.getPrettyPrint() ? + String json = fvStats.toJson(statsControl.getPrettyPrint() ? JsonOptions.PRETTY : null); - statsConfig.getLogger().log(Level.INFO, StatsConfigImpl.LOG_PREFIX + json); + statsControl.getLogger().log(Level.INFO, + StatsControlImpl.LOG_PREFIX + json); } private FieldValue generateFieldValueStats() { @@ -462,7 +494,7 @@ private FieldValue generateFieldValueStats() { ts = new Timestamp(endTime); ts.setNanos(0); root.put("endTime", new TimestampValue(ts)); - root.put("clientId", new StringValue(statsConfig.getId())); + root.put("clientId", new StringValue(statsControl.getId())); connectionStats.toJSON(root); if (extraQueryStats != null) { @@ -490,7 +522,7 @@ private FieldValue generateFieldValueStats() { * Clear all collected stats. */ private void clearStats() { - for (String key : REQUEST_KEYS) { + for (String key : requests.keySet()) { requests.get(key).clear(); } connectionStats.clear(); @@ -513,18 +545,19 @@ void observeError(Request kvRequest, /** * Adds a new statistic entry. Can be of 2 types: successful or error. - * Request, result sizes and networkLatency are not registered for error entries. + * Request, result sizes and requestLatency are not registered for error + * entries. * * @param kvRequest The request object. * @param error Hard error, ie. return error to user. * @param connections The number of active connections in the pool. * @param reqSize Request size in bytes. * @param resSize Result size in bytes. - * @param networkLatency Latency on the wire, in milliseconds, it doesn't - * include retry delay or rate limit delay. + * @param requestLatency Latency on the wire, in milliseconds, it doesn't + * include retry delay or rate limit delay. */ void observe(Request kvRequest, boolean error, - int connections, int reqSize, int resSize, int networkLatency) { + int connections, int reqSize, int resSize, int requestLatency) { int authCount = 0, throttleCount = 0, retries = 0, retryDelay = 0; RetryStats retryStats = kvRequest.getRetryStats(); @@ -542,34 +575,28 @@ void observe(Request kvRequest, boolean error, } int rateLimitDelay = kvRequest.getRateLimitDelayedMs(); - - ReqStats rStat; - String requestClass = kvRequest.getClass().getSimpleName(); - requestClass = requestClass.endsWith("Request") - ? requestClass.substring(0, requestClass.length() - 7) : - requestClass; - - rStat = requests.get(requestClass); + String requestName = kvRequest.getTypeName(); + ReqStats rStat = requests.get(requestName); // This will not happen unless a new request type is added but not // registered in the REQUEST_KEYS array. if (rStat == null) { ReqStats newStat = new ReqStats(); - if (statsConfig.getProfile().ordinal() >= - StatsConfig.Profile.MORE.ordinal()) { - newStat.wireLatencyPercentile = new Percentile(); + if (statsControl.getProfile().ordinal() >= + StatsControl.Profile.MORE.ordinal()) { + newStat.requestLatencyPercentile = new Percentile(); } synchronized (requests) { - rStat = requests.get(requestClass); + rStat = requests.get(requestName); if (rStat == null) { - requests.put(requestClass, newStat); + requests.put(requestName, newStat); rStat = newStat; } } } rStat.observe(error, retries, retryDelay, rateLimitDelay, authCount, - throttleCount, reqSize, resSize, networkLatency); + throttleCount, reqSize, resSize, requestLatency); connectionStats.observe(connections); @@ -579,12 +606,15 @@ void observe(Request kvRequest, boolean error, extraQueryStats.observeQuery(queryRequest, error, retries, retryDelay, rateLimitDelay, authCount, throttleCount, - reqSize, resSize, networkLatency); + reqSize, resSize, requestLatency); } } } - public void observeQuery(QueryRequest qreq) { + /** + * Adds a new statistic entry for this query request. + */ + void observeQuery(QueryRequest qreq) { if (extraQueryStats != null) { extraQueryStats.observeQuery(qreq); } diff --git a/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java b/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java similarity index 54% rename from driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java rename to driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java index 7d1c3a3b..598c3941 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/StatsConfigImpl.java +++ b/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java @@ -1,29 +1,30 @@ +/*- + * Copyright (c) 2011, 2021 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 java.util.UUID; import java.util.logging.Level; import java.util.logging.Logger; -import oracle.nosql.driver.StatsConfig; +import oracle.nosql.driver.NoSQLHandleConfig; +import oracle.nosql.driver.StatsControl; import oracle.nosql.driver.httpclient.HttpClient; import oracle.nosql.driver.ops.QueryRequest; import oracle.nosql.driver.ops.Request; -public class StatsConfigImpl - implements StatsConfig { +public class StatsControlImpl + implements StatsControl { - private final static String PROFILE_PROPERTY = - "com.oracle.nosql.sdk.nosqldriver.stats.profile"; - private final static String INTERVAL_PROPERTY = - "com.oracle.nosql.sdk.nosqldriver.stats.interval"; - private final static String PRETTY_PRINT_PROPERTY = - "com.oracle.nosql.sdk.nosqldriver.stats.pretty-print"; - final static String LOG_PREFIX = "ONJS:Monitoring stats|"; + static public final String LOG_PREFIX = "ONJS:Monitoring stats|"; - private StatsConfig.Profile profile = Profile.NONE; - /* Time interval to log in seconds. Default 600, ie. 10 minutes. */ - private int interval = 600; - private boolean prettyPrint = false; + private StatsControl.Profile profile; + private int interval; + private boolean prettyPrint; private Logger logger; private HttpClient httpClient; /* required for connections */ @@ -32,43 +33,21 @@ public class StatsConfigImpl private boolean enableCollection = false; private Stats stats; - StatsConfigImpl(String libraryVersion, Logger logger, + StatsControlImpl(NoSQLHandleConfig config, Logger logger, HttpClient httpClient, boolean rateLimitingEnabled) { this.logger = logger; this.httpClient = httpClient; - String profileProp = System.getProperty(PROFILE_PROPERTY); - if (profileProp != null) { - try { - setProfile(Profile.valueOf(profileProp.toUpperCase())); - } catch (IllegalArgumentException iae) { - logger.log(Level.SEVERE, LOG_PREFIX + "Invalid profile " + - "value for system property " + PROFILE_PROPERTY + ": " + - profileProp); - } - } - - String intervalProp = System.getProperty(INTERVAL_PROPERTY); - if (intervalProp != null) { - try { - setInterval(Integer.valueOf(intervalProp)); - } catch (NumberFormatException nfe) { - logger.log(Level.SEVERE, "Invalid integer value for system " + - "property " + INTERVAL_PROPERTY + ": " + intervalProp); - } - } - - String ppProp = System.getProperty(PRETTY_PRINT_PROPERTY); - if (ppProp != null && ("true".equals(ppProp.toLowerCase()) || "1".equals(ppProp) || - "on".equals(ppProp.toLowerCase()))) { - prettyPrint = Boolean.valueOf(ppProp); - } + this.interval = config.getStatsInterval(); + this.profile = config.getStatsProfile(); + this.prettyPrint = config.getStatsPrettyPrint(); if (profile != Profile.NONE) { logger.setLevel(Level.INFO); logger.log(Level.INFO, LOG_PREFIX + "{\"sdkName\" : \"Oracle NoSQL SDK for Java" + - "\", \"sdkVersion\" : \"" + libraryVersion + + "\", \"sdkVersion\" : \"" + + NoSQLHandleConfig.getLibraryVersion() + "\", \"clientId\" : \"" + id + "\", \"profile\" : \"" + profile + "\", \"intervalSec\" : " + interval + @@ -79,34 +58,17 @@ public class StatsConfigImpl } } - @Override - public StatsConfig setLogger(Logger logger) { - this.logger = logger; - return this; - } - - @Override public Logger getLogger() { return logger; } - @Override - public StatsConfig setInterval(int interval) { - if (interval < 1) { - throw new IllegalArgumentException("Stats interval can not be " + - "less than 1 second."); - } - this.interval = interval; - return this; - } - @Override public int getInterval() { return interval; } @Override - public StatsConfig setProfile(Profile profile) { + public StatsControl setProfile(Profile profile) { this.profile = profile; return this; } @@ -117,7 +79,7 @@ public Profile getProfile() { } @Override - public StatsConfig setPrettyPrint(boolean enablePrettyPrint) { + public StatsControl setPrettyPrint(boolean enablePrettyPrint) { this.prettyPrint = enablePrettyPrint; return this; } @@ -128,8 +90,9 @@ public boolean getPrettyPrint() { } @Override - public void registerHandler(StatsHandler statsHandler) { + public StatsControl registerStatsHandler(StatsHandler statsHandler) { this.statsHandler = statsHandler; + return this; } public StatsHandler getHandler() { diff --git a/driver/src/main/java/oracle/nosql/driver/ops/DeleteRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/DeleteRequest.java index d501a292..09806058 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/DeleteRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/DeleteRequest.java @@ -238,6 +238,11 @@ public Serializer createDeserializer(SerializerFactory factory) { return factory.createDeleteDeserializer(); } + @Override + public String getTypeName() { + return "Delete"; + } + /** * @hidden */ diff --git a/driver/src/main/java/oracle/nosql/driver/ops/GetIndexesRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/GetIndexesRequest.java index 2e939c4c..d1071593 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/GetIndexesRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/GetIndexesRequest.java @@ -123,6 +123,11 @@ public Serializer createDeserializer(SerializerFactory factory) { return factory.createGetIndexesDeserializer(); } + @Override + public String getTypeName() { + return "GetIndexes"; + } + /** * @hidden */ diff --git a/driver/src/main/java/oracle/nosql/driver/ops/GetRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/GetRequest.java index ee2626d0..5662f6db 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/GetRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/GetRequest.java @@ -179,4 +179,9 @@ public Serializer createSerializer(SerializerFactory factory) { public Serializer createDeserializer(SerializerFactory factory) { return factory.createGetDeserializer(); } + + @Override + public String getTypeName() { + return "Get"; + } } diff --git a/driver/src/main/java/oracle/nosql/driver/ops/GetTableRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/GetTableRequest.java index 2950c0bb..ac298555 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/GetTableRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/GetTableRequest.java @@ -132,6 +132,11 @@ public Serializer createDeserializer(SerializerFactory factory) { return factory.createGetTableDeserializer(); } + @Override + public String getTypeName() { + return "GetTable"; + } + /** * @hidden */ diff --git a/driver/src/main/java/oracle/nosql/driver/ops/ListTablesRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/ListTablesRequest.java index 33066d32..78281e57 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/ListTablesRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/ListTablesRequest.java @@ -174,6 +174,11 @@ public Serializer createDeserializer(SerializerFactory factory) { return factory.createListTablesDeserializer(); } + @Override + public String getTypeName() { + return "ListTables"; + } + /** * @hidden */ diff --git a/driver/src/main/java/oracle/nosql/driver/ops/MultiDeleteRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/MultiDeleteRequest.java index bfbd13f3..6d0893a2 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/MultiDeleteRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/MultiDeleteRequest.java @@ -210,6 +210,11 @@ public Serializer createDeserializer(SerializerFactory factory) { return factory.createMultiDeleteDeserializer(); } + @Override + public String getTypeName() { + return "MultiDelete"; + } + /** * @hidden */ diff --git a/driver/src/main/java/oracle/nosql/driver/ops/PrepareRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/PrepareRequest.java index 8baa425a..18c19993 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/PrepareRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/PrepareRequest.java @@ -156,6 +156,11 @@ public Serializer createDeserializer(SerializerFactory factory) { return factory.createPrepareDeserializer(); } + @Override + public String getTypeName() { + return "Prepare"; + } + /** * @hidden */ diff --git a/driver/src/main/java/oracle/nosql/driver/ops/PutRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/PutRequest.java index 281c905e..229f162e 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/PutRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/PutRequest.java @@ -418,6 +418,11 @@ public Serializer createDeserializer(SerializerFactory factory) { return factory.createPutDeserializer(); } + @Override + public String getTypeName() { + return "Put"; + } + /** * @hidden * diff --git a/driver/src/main/java/oracle/nosql/driver/ops/QueryRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/QueryRequest.java index 1179ede8..eaca87b4 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/QueryRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/QueryRequest.java @@ -659,6 +659,11 @@ public Serializer createDeserializer(SerializerFactory factory) { return factory.createQueryDeserializer(); } + @Override + public String getTypeName() { + return "Query"; + } + /** * @hidden * 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 6be80b3c..5b731de4 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/Request.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/Request.java @@ -416,4 +416,9 @@ public void setRateLimitDelayedMs(int rateLimitDelayedMs) { public int getRateLimitDelayedMs() { return rateLimitDelayedMs; } + + /** + * Returns the type name of the request. This is used for stats. + */ + public abstract String getTypeName(); } diff --git a/driver/src/main/java/oracle/nosql/driver/ops/SystemRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/SystemRequest.java index 4a0bb2e4..2d917b1a 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/SystemRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/SystemRequest.java @@ -117,6 +117,11 @@ public Serializer createDeserializer(SerializerFactory factory) { return factory.createSystemOpDeserializer(); } + @Override + public String getTypeName() { + return "System"; + } + @Override public String toString() { StringBuilder sb = new StringBuilder(); diff --git a/driver/src/main/java/oracle/nosql/driver/ops/SystemStatusRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/SystemStatusRequest.java index 7072541e..c5a7982a 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/SystemStatusRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/SystemStatusRequest.java @@ -129,6 +129,11 @@ public Serializer createDeserializer(SerializerFactory factory) { return factory.createSystemStatusDeserializer(); } + @Override + public String getTypeName() { + return "SystemStatus"; + } + @Override public String toString() { StringBuilder sb = new StringBuilder(); diff --git a/driver/src/main/java/oracle/nosql/driver/ops/TableRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/TableRequest.java index 4a67e037..d4513043 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/TableRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/TableRequest.java @@ -189,6 +189,11 @@ public Serializer createDeserializer(SerializerFactory factory) { return factory.createTableOpDeserializer(); } + @Override + public String getTypeName() { + return "Table"; + } + @Override public String toString() { StringBuilder sb = new StringBuilder(); diff --git a/driver/src/main/java/oracle/nosql/driver/ops/TableUsageRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/TableUsageRequest.java index 9175679e..ad4ae796 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/TableUsageRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/TableUsageRequest.java @@ -252,6 +252,11 @@ public Serializer createDeserializer(SerializerFactory factory) { return factory.createGetTableUsageDeserializer(); } + @Override + public String getTypeName() { + return "TableUsage"; + } + /** * @hidden */ diff --git a/driver/src/main/java/oracle/nosql/driver/ops/WriteMultipleRequest.java b/driver/src/main/java/oracle/nosql/driver/ops/WriteMultipleRequest.java index 3556209d..ec05f99b 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/WriteMultipleRequest.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/WriteMultipleRequest.java @@ -177,6 +177,11 @@ public Serializer createDeserializer(SerializerFactory factory) { return factory.createWriteMultipleDeserializer(); } + @Override + public String getTypeName() { + return "WriteMultiple"; + } + /** * @hidden */ 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 7897fbaf..df96fb61 100644 --- a/driver/src/main/java/oracle/nosql/driver/package-info.java +++ b/driver/src/main/java/oracle/nosql/driver/package-info.java @@ -138,5 +138,220 @@ oracle.nosql.level=FINE io.netty.level=FINE * + * + *

    Logging internal SDK statistics

    + *

    There are 4 profiles of statistics information that can be logged: + * none, regular, more, all. For none, which is the default, no + * statistics are collected and there are no messages logged.

    + *

    For regular, more and all profiles, the statistics data is + * collected for an interval of time. At the end of the + * interval, the stats data is logged in a specified JSON format that can be + * filtered and parsed. At the end of each interval, after the logging, the + * counters are cleared and collection of data resumes.

    + * + * Collection intervals are aligned to the top of the hour. This means first + * interval logs may contain stats for a shorter interval.

    + * + * Collection of stats are controlled by the following system properties:

  • + * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile=[none|regular|more|all] + * Specifies the stats profile: none - disabled, + * regular - per request: counters, errors, latencies, delays, retries + * more - stats above with 95th and 99th percentile latencies + * all - stats above with per query information.
  • + * + * -Dcom.oracle.nosql.sdk.nosqldriver.stats.interval=600 Interval in + * seconds to log the stats, by default is 10 minutes.
  • + * + * -Dcom.oracle.nosql.sdk.nosqldriver.stats.pretty-print=true Option + * to enable pretty printing of the JSON data, default value is false
  • + * + * Statistics can also be enabled by using the API: + * {@link oracle.nosql.driver.NoSQLHandleConfig#setStatsProfile(StatsControl.Profile)} + * or {@link oracle.nosql.driver.StatsControl#setProfile(StatsControl.Profile)}. + * At runtime stats collection can be used selectively by using + * {@link oracle.nosql.driver.StatsControl#start()} and + * {@link oracle.nosql.driver.StatsControl#stop()}. The following example shows + * how to use a stats handler: + * + * NoSQLHandleConfig config = new NoSQLHandleConfig( endpoint ); + * config.setStatsProfile(StatsControl.Profile.REGULAR); + * config.setStatsInterval(600); + * config.setStatsPrettyPrint(false); + * config.registerStatsHandler( + * new StatsControl.StatsHandler() { + * public void accept(FieldValue jsonStats) { + * System.out.println("!!! Got a stat: " + jsonStats); + * } + * }); + * NoSQLHandle handle = NoSQLHandleFactory.createNoSQLHandle(config); + * + * StatsControl statsControl = handle.getStatsControl(); + * + * //... application code without stats + * + * // enable observations + * statsControl.start(); + * + * //... application code with REGULAR stats + * + * // For particular parts of code profile can be changed collect more stats. + * statsControl.setProfile(StatsControl.Profile.ALL) + * //... more sensitive code with ALL stats + * statsControl.setProfile(StatsControl.Profile.REGULAR) + * + * //... application code with REGULAR stats + * + * // disable observations + * statsControl.stop(); + * + * // ... application code without stats + * handle.close(); + *

    + * + *

      The following is an example of stats log entry using the ALL + * profile:
    1. + * A one time entry containing stats id and options: + *

      INFO: ONJS:Monitoring stats|{    // INFO log entry
      + *    "sdkName" : "Oracle NoSQL SDK for Java",  // SDK name
      + *    "sdkVersion" : "current",                 // SDK version
      + *    "clientId" : "f595b333",                  // NoSQLHandle id
      + *    "profile" : "ALL",                        // stats profile
      + *    "intervalSec" : 600,                      // interval length in seconds
      + *    "prettyPrint" : true,                     // JSON pretty print
      + *    "rateLimitingEnabled" : false}            // if rate limiting is
      + *    enabled

    2. + * An entry at the end of each interval containing the stats values: + *

      INFO: ONJS:Monitoring stats|{
      + *   "clientId" : "b7bc7734",              // id of NoSQLHandle object
      + *   "startTime" : "2021-09-20T20:11:42Z", // UTC start interval time
      + *   "endTime" : "2021-09-20T20:11:47Z",   // UTC end interval time
      + *   "requests" : [{                       // array of types of requests
      + *     "name" : "Get",                       // stats for GET request type
      + *     "httpRequestCount" : 2,               // count of http requests
      + *     "errors" : 0,                         // number of errors in interval
      + *     "httpRequestLatencyMs" : {            // response time of http requests
      + *       "min" : 4,                            // minimum value in interval
      + *       "avg" : 4.5,                          // average value in interval
      + *       "max" : 5,                            // maximum value in interval
      + *       "95th" : 5,                           // 95th percentile value
      + *       "99th" : 5                            // 99th percentile value
      + *     },
      + *     "requestSize" : {                     // http request size in bytes
      + *       "min" : 42,                           // minimum value in interval
      + *       "avg" : 42.5,                         // average value in interval
      + *       "max" : 43                            // maximum value in interval
      + *     },
      + *     "resultSize" : {                      // http result size in bytes
      + *       "min" : 193,                          // minimum value in interval
      + *       "avg" : 206.5,                        // average value in interval
      + *       "max" : 220                           // maximum value in interval
      + *     },
      + *     "rateLimitDelayMs" : 0,               // delay in milliseconds introduced by the rate limiter
      + *     "retry" : {                           // retries
      + *       "delayMs" : 0,                        // delay in milliseconds introduced by retries
      + *       "authCount" : 0,                      // no of auth retries
      + *       "throttleCount" : 0,                  // no of throttle retries
      + *       "count" : 0                           // total number of retries
      + *     }
      + *   }, {
      + *     "name" : "Query",                   // stats for all QUERY type requests
      + *     "httpRequestCount" : 14,
      + *     "errors" : 0,
      + *     "httpRequestLatencyMs" : {
      + *       "min" : 3,
      + *       "avg" : 13.0,
      + *       "max" : 32,
      + *       "95th" : 32,
      + *       "99th" : 32
      + *     },
      + *     "resultSize" : {
      + *       "min" : 146,
      + *       "avg" : 7379.71,
      + *       "max" : 10989
      + *     },
      + *     "requestSize" : {
      + *       "min" : 65,
      + *       "avg" : 709.85,
      + *       "max" : 799
      + *     },
      + *     "rateLimitDelayMs" : 0,
      + *     "retry" : {
      + *       "delayMs" : 0,
      + *       "authCount" : 0,
      + *       "throttleCount" : 0,
      + *       "count" : 0
      + *     }
      + *   }, {
      + *     "name" : "Put",                    // stats for PUT type requests
      + *     "httpRequestCount" : 1002,
      + *     "errors" : 0,
      + *     "httpRequestLatencyMs" : {
      + *       "min" : 1,
      + *       "avg" : 4.41,
      + *       "max" : 80,
      + *       "95th" : 8,
      + *       "99th" : 20
      + *     },
      + *     "requestSize" : {
      + *       "min" : 90,
      + *       "avg" : 90.16,
      + *       "max" : 187
      + *     },
      + *     "resultSize" : {
      + *       "min" : 58,
      + *       "avg" : 58.0,
      + *       "max" : 58
      + *     },
      + *     "rateLimitDelayMs" : 0,
      + *     "retry" : {
      + *       "delayMs" : 0,
      + *       "authCount" : 0,
      + *       "throttleCount" : 0,
      + *       "count" : 0
      + *     }
      + *   }],
      + *   "queries" : [{            // query stats aggregated by query statement
      + *                               // query statement
      + *     "query" : "SELECT * FROM audienceData ORDER BY cookie_id",
      + *                               // query plan description
      + *     "plan" : "SFW([6])\n[\n  FROM:\n  RECV([3])\n  [\n    DistributionKind : ALL_PARTITIONS,\n    Sort Fields : sort_gen,\n\n  ] as $from-0\n\n  SELECT:\n  FIELD_STEP([6])\n  [\n    VAR_REF($from-0)([3]),\n    audienceData\n  ]\n]",
      + *     "doesWrites" : false,
      + *     "httpRequestCount" : 12,  // number of http calls to the server
      + *     "unprepared" : 1,         // number of query requests without prepare
      + *     "simple" : false,         // type of query
      + *     "countAPI" : 20,          // number of handle.query() API calls
      + *     "errors" : 0,             // number of calls trowing exception
      + *     "httpRequestLatencyMs" : {// response time of http requests in milliseconds
      + *       "min" : 8,                // minimum value in interval
      + *       "avg" : 14.58,            // average value in interval
      + *       "max" : 32,               // maximum value in interval
      + *       "95th" : 32,              // 95th percentile value in interval
      + *       "99th" : 32               // 99th percentile value in interval
      + *     },
      + *     "requestSize" : {         // http request size in bytes
      + *       "min" : 65,               // minimum value in interval
      + *       "avg" : 732.5,            // average value in interval
      + *       "max" : 799               // maximum value in interval
      + *     },
      + *     "resultSize" : {          // http result size in bytes
      + *       "min" : 914,              // minimum value in interval
      + *       "avg" : 8585.33,          // average value in interval
      + *       "max" : 10989             // maximum value in interval
      + *     },
      + *     "rateLimitDelayMs" : 0,   // total delay introduced by rate limiter in milliseconds
      + *     "retry" : {               // automatic retries
      + *       "delayMs" : 0,            // delay introduced by retries
      + *       "authCount" : 0,          // count of auth related retries
      + *       "throttleCount" : 0,      // count of throttle related retries
      + *       "count" : 0               // total count of retries
      + *     }
      + *   }],
      + *   "connections" : {           // concurrent opened connections
      + *     "min" : 1,                  // minimum value in interval
      + *     "avg" : 9.58,               // average value in interval
      + *     "max" : 10                  // maximum value in interval
      + *   }
      + * }
      + *     

    */ package oracle.nosql.driver; From 6b7663bbc95b75cf9f03b55c8ec68e2dc3a31290 Mon Sep 17 00:00:00 2001 From: Cezar Andrei Date: Thu, 28 Oct 2021 14:05:07 -0500 Subject: [PATCH 10/13] Set use stats handler set in NoSQLHandleConfig. Fix javadoc errors and warnings. --- .../java/oracle/nosql/driver/NoSQLHandle.java | 2 ++ .../nosql/driver/NoSQLHandleConfig.java | 11 ++++++++- .../oracle/nosql/driver/StatsControl.java | 24 +++++++++++++------ .../nosql/driver/http/StatsControlImpl.java | 1 + .../java/oracle/nosql/driver/ops/Request.java | 2 ++ .../oracle/nosql/driver/package-info.java | 18 +++++++------- 6 files changed, 42 insertions(+), 16 deletions(-) diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java index b10b236a..818730c1 100644 --- a/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java +++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandle.java @@ -573,6 +573,8 @@ public SystemResult doSystemRequest(String statement, /** * Returns an object that allows control over how statistics are collected. + * + * @return the StatsControl object */ public StatsControl getStatsControl(); diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java index 12616dc0..34bc6612 100644 --- a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java +++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java @@ -1210,7 +1210,8 @@ public SslContext getSslContext() { * Sets interval size in seconds for logging statistics. * Default interval is 600 seconds, i.e. 10 min. * - * @ return this + * @param statsInterval stats logging interval in seconds + * @return this */ public NoSQLHandleConfig setStatsInterval(int statsInterval) { if (statsInterval < 1) { @@ -1225,6 +1226,8 @@ public NoSQLHandleConfig setStatsInterval(int statsInterval) { /** * Returns the current interval for logging statistics. * Default interval is 600 seconds, i.e. 10 min. + * + * @return the current interval in seconds */ public int getStatsInterval() { return this.statsInterval; @@ -1234,6 +1237,7 @@ public int getStatsInterval() { * Set the statistics collection profile. * Default profile is NONE. * + * @param statsProfile profile to use * @return this */ public NoSQLHandleConfig setStatsProfile(StatsControl.Profile statsProfile) { @@ -1244,6 +1248,8 @@ public NoSQLHandleConfig setStatsProfile(StatsControl.Profile statsProfile) { /** * Returns the statistics collection profile. * Default profile is NONE. + * + * @return the current profile */ public StatsControl.Profile getStatsProfile() { return this.statsProfile; @@ -1254,6 +1260,7 @@ public StatsControl.Profile getStatsProfile() { * statistics. * Default is disabled. * + * @param statsPrettyPrint flag to enable JSON pretty print * @return this */ public NoSQLHandleConfig setStatsPrettyPrint(boolean statsPrettyPrint) { @@ -1264,6 +1271,8 @@ public NoSQLHandleConfig setStatsPrettyPrint(boolean statsPrettyPrint) { /** * Returns the current JSON pretty print flag for logging statistics. * Default is disabled. + * + * @return the current JSON pretty print flag */ public boolean getStatsPrettyPrint() { return this.statsPrettyPrint; diff --git a/driver/src/main/java/oracle/nosql/driver/StatsControl.java b/driver/src/main/java/oracle/nosql/driver/StatsControl.java index a998842f..7b0cdea6 100644 --- a/driver/src/main/java/oracle/nosql/driver/StatsControl.java +++ b/driver/src/main/java/oracle/nosql/driver/StatsControl.java @@ -22,7 +22,7 @@ * Collection intervals are aligned to the top of the hour. This means first * interval logs may contain stats for a shorter interval.

    * - * Collection of stats are controlled by the following system properties:

  • + * Collection of stats are controlled by the following system properties:
    1. * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile=[none|regular|more|all] * Specifies the stats profile: none - disabled, * regular - per request: counters, errors, latencies, delays, retries @@ -33,11 +33,11 @@ * seconds to log the stats, by default is 10 minutes.
    2. * * -Dcom.oracle.nosql.sdk.nosqldriver.stats.pretty-print=true Option - * to enable pretty printing of the JSON data, default value is false
    3. + * to enable pretty printing of the JSON data, default value is false
    * * Statistics can also be enabled by using the API: - * {@link NoSQLHandleConfig#setStatsProfile(Profile)} or - * {@link StatsControl#setProfile(Profile)}. At runtime stats + * {@link NoSQLHandleConfig#setStatsProfile(StatsControl.Profile)} or + * {@link StatsControl#setProfile(StatsControl.Profile)}. At runtime stats * collection can be used selectively by using {@link StatsControl#start()} and * {@link StatsControl#stop()}. The following example shows how to use a stats * handler:
    @@ -74,7 +74,7 @@
      *
      *     // ... application code without stats
      *     handle.close();
    - *     

    + *

    * * For a detailed statistics log entries structure and values see * {@link oracle.nosql.driver}

    @@ -104,13 +104,16 @@ interface StatsHandler extends Consumer { /** * Returns the current collection interval. * Default interval is 600 seconds, i.e. 10 min. + * + * @return the current collection interval */ int getInterval(); /** - * Set the collection profile. + * Set the stats collection profile. * Default profile is NONE. * + * @param profile the stats collection profile * @return this */ StatsControl setProfile(Profile profile); @@ -118,6 +121,8 @@ interface StatsHandler extends Consumer { /** * Returns the collection profile. * Default profile is NONE. + * + * @return the current profile */ Profile getProfile(); @@ -125,6 +130,7 @@ interface StatsHandler extends Consumer { * Enable JSON pretty print for easier human reading. * Default is disabled. * + * @param enablePrettyPrint flag to enable JSON pretty print * @return this */ StatsControl setPrettyPrint(boolean enablePrettyPrint); @@ -132,13 +138,15 @@ interface StatsHandler extends Consumer { /** * Returns the current JSON pretty print flag. * Default is disabled. + * + * @return the current JSON pretty print flag */ boolean getPrettyPrint(); /** * Registers a stats handler. - * @param handler User defined StatsHandler. * + * @param handler User defined StatsHandler * @return this */ StatsControl registerStatsHandler(StatsHandler handler); @@ -157,6 +165,8 @@ interface StatsHandler extends Consumer { /** * Returns true if collection of stats is enabled, otherwise returns false. + * + * @return true if start() was called last, false if stop() was called last */ boolean isStarted(); } diff --git a/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java b/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java index 598c3941..a62e83b1 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java +++ b/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java @@ -41,6 +41,7 @@ public class StatsControlImpl this.interval = config.getStatsInterval(); this.profile = config.getStatsProfile(); this.prettyPrint = config.getStatsPrettyPrint(); + this.statsHandler = config.getStatsHandler(); if (profile != Profile.NONE) { logger.setLevel(Level.INFO); 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 5b731de4..070d6b70 100644 --- a/driver/src/main/java/oracle/nosql/driver/ops/Request.java +++ b/driver/src/main/java/oracle/nosql/driver/ops/Request.java @@ -419,6 +419,8 @@ public int getRateLimitDelayedMs() { /** * Returns the type name of the request. This is used for stats. + * + * @return the type name of the request */ public abstract String getTypeName(); } 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 df96fb61..07bd9681 100644 --- a/driver/src/main/java/oracle/nosql/driver/package-info.java +++ b/driver/src/main/java/oracle/nosql/driver/package-info.java @@ -152,7 +152,8 @@ * Collection intervals are aligned to the top of the hour. This means first * interval logs may contain stats for a shorter interval.

    * - * Collection of stats are controlled by the following system properties:

  • + * Collection of stats are controlled by the following system + * properties:
    1. * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile=[none|regular|more|all] * Specifies the stats profile: none - disabled, * regular - per request: counters, errors, latencies, delays, retries @@ -163,7 +164,8 @@ * seconds to log the stats, by default is 10 minutes.
    2. * * -Dcom.oracle.nosql.sdk.nosqldriver.stats.pretty-print=true Option - * to enable pretty printing of the JSON data, default value is false
    3. + * to enable pretty printing of the JSON data, default value is false + *

    * * Statistics can also be enabled by using the API: * {@link oracle.nosql.driver.NoSQLHandleConfig#setStatsProfile(StatsControl.Profile)} @@ -208,10 +210,10 @@ * handle.close(); *

    * - *

      The following is an example of stats log entry using the ALL - * profile:
    1. + * The following is an example of stats log entry using the ALL + * profile:
      1. * A one time entry containing stats id and options: - *

        INFO: ONJS:Monitoring stats|{    // INFO log entry
        + *    
        INFO: ONJS:Monitoring stats|{    // INFO log entry
          *    "sdkName" : "Oracle NoSQL SDK for Java",  // SDK name
          *    "sdkVersion" : "current",                 // SDK version
          *    "clientId" : "f595b333",                  // NoSQLHandle id
        @@ -219,9 +221,9 @@
          *    "intervalSec" : 600,                      // interval length in seconds
          *    "prettyPrint" : true,                     // JSON pretty print
          *    "rateLimitingEnabled" : false}            // if rate limiting is
        - *    enabled

      2. + * enabled
      3. * An entry at the end of each interval containing the stats values: - *

        INFO: ONJS:Monitoring stats|{
        + *    
        INFO: ONJS:Monitoring stats|{
          *   "clientId" : "b7bc7734",              // id of NoSQLHandle object
          *   "startTime" : "2021-09-20T20:11:42Z", // UTC start interval time
          *   "endTime" : "2021-09-20T20:11:47Z",   // UTC end interval time
        @@ -352,6 +354,6 @@
          *     "max" : 10                  // maximum value in interval
          *   }
          * }
        - *     

      + *
    */ package oracle.nosql.driver; From 4d8e81f9663f2e242f2893ff1553a195feab2fe0 Mon Sep 17 00:00:00 2001 From: Cezar Andrei Date: Tue, 9 Nov 2021 16:30:49 -0600 Subject: [PATCH 11/13] Added entry in CHANGELOG.md. Changes StatsHandler to accept MapValue instead of FieldValue. Fixed javadoc. --- CHANGELOG.md | 6 ++++ .../nosql/driver/NoSQLHandleConfig.java | 12 ++++--- .../oracle/nosql/driver/StatsControl.java | 34 +++++++++++-------- .../java/oracle/nosql/driver/http/Stats.java | 5 ++- .../nosql/driver/http/StatsControlImpl.java | 2 +- .../oracle/nosql/driver/package-info.java | 6 ++-- 6 files changed, 39 insertions(+), 26 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ba2c6dde..de90fbff 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,12 @@ All notable changes to this project will be documented in this file. The format is based on [Keep a Changelog](http://keepachangelog.com/). + +### Added +- Added client statistics. Users can enable internal driver statistics by +using system property -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile= +[none|regular|more|all] or by using the NoSQLConfig.setStatsProfile() api. + ## [5.2.28-SNAPSHOT] 2021-09-13 ### Fixed diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java index 34bc6612..424e6d07 100644 --- a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java +++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java @@ -1222,7 +1222,6 @@ public NoSQLHandleConfig setStatsInterval(int statsInterval) { return this; } - /** * Returns the current interval for logging statistics. * Default interval is 600 seconds, i.e. 10 min. @@ -1240,7 +1239,8 @@ public int getStatsInterval() { * @param statsProfile profile to use * @return this */ - public NoSQLHandleConfig setStatsProfile(StatsControl.Profile statsProfile) { + public NoSQLHandleConfig setStatsProfile(StatsControl.Profile statsProfile) + { this.statsProfile = statsProfile; return this; } @@ -1280,11 +1280,12 @@ public boolean getStatsPrettyPrint() { /** * Registers a handler that is called every time the statistics are logged. + * Note: setting a stats handler will not affect the stats log entries. * @param statsHandler User defined StatsHandler. * * @return this */ - public NoSQLHandleConfig registerStatsHandler( + public NoSQLHandleConfig setStatsHandler( StatsControl.StatsHandler statsHandler) { this.statsHandler = statsHandler; return this; @@ -1364,8 +1365,9 @@ private void setConfigFromEnvironment() { } String ppProp = System.getProperty(PRETTY_PRINT_PROPERTY); - if (ppProp != null && ("true".equals(ppProp.toLowerCase()) || "1".equals(ppProp) || - "on".equals(ppProp.toLowerCase()))) { + if (ppProp != null && + ("true".equals(ppProp.toLowerCase()) || "1".equals(ppProp) || + "on".equals(ppProp.toLowerCase()))) { statsPrettyPrint = Boolean.valueOf(ppProp); } } diff --git a/driver/src/main/java/oracle/nosql/driver/StatsControl.java b/driver/src/main/java/oracle/nosql/driver/StatsControl.java index 7b0cdea6..f4cb70d0 100644 --- a/driver/src/main/java/oracle/nosql/driver/StatsControl.java +++ b/driver/src/main/java/oracle/nosql/driver/StatsControl.java @@ -10,9 +10,11 @@ import java.util.function.Consumer; import oracle.nosql.driver.values.FieldValue; +import oracle.nosql.driver.values.MapValue; /** - *

    This interface allows user to setup the collection of driver statistics.

    + *

    This interface allows user to control the collection of driver + * statistics at runtime.

    * * The statistics data is collected for an interval of time. At the end of the * interval, the stats data is logged in a specified JSON format that can be @@ -22,7 +24,8 @@ * Collection intervals are aligned to the top of the hour. This means first * interval logs may contain stats for a shorter interval.

    * - * Collection of stats are controlled by the following system properties:

    1. + * Collection of stats are controlled by the following system + * properties:
      • * -Dcom.oracle.nosql.sdk.nosqldriver.stats.profile=[none|regular|more|all] * Specifies the stats profile: none - disabled, * regular - per request: counters, errors, latencies, delays, retries @@ -33,21 +36,22 @@ * seconds to log the stats, by default is 10 minutes.
      • * * -Dcom.oracle.nosql.sdk.nosqldriver.stats.pretty-print=true Option - * to enable pretty printing of the JSON data, default value is false
    + * to enable pretty printing of the JSON data, default value is + * false
  • * * Statistics can also be enabled by using the API: * {@link NoSQLHandleConfig#setStatsProfile(StatsControl.Profile)} or * {@link StatsControl#setProfile(StatsControl.Profile)}. At runtime stats - * collection can be used selectively by using {@link StatsControl#start()} and - * {@link StatsControl#stop()}. The following example shows how to use a stats - * handler:
    + * collection can be enabled selectively by using {@link StatsControl#start()}
    + * and {@link StatsControl#stop()}. The following example shows how to use a
    + * stats handler and control the stats at runtime: 
      *     NoSQLHandleConfig config = new NoSQLHandleConfig( endpoint );
      *     config.setStatsProfile(StatsControl.Profile.REGULAR);
      *     config.setStatsInterval(600);
      *     config.setStatsPrettyPrint(false);
    - *     config.registerStatsHandler(
    + *     config.setStatsHandler(
      *         new StatsControl.StatsHandler() {
    - *             public void accept(FieldValue jsonStats) {
    + *             public void accept(MapValue jsonStats) {
      *                 System.out.println("!!! Got a stat: " + jsonStats);
      *             }
      *         });
    @@ -62,7 +66,7 @@
      *
      *     //... application code with REGULAR stats
      *
    - *     // For particular parts of code profile can be changed collect more stats.
    + *     // For particular parts of code profile can be changed to collect more stats.
      *     statsControl.setProfile(StatsControl.Profile.ALL)
      *     //... more sensitive code with ALL stats
      *     statsControl.setProfile(StatsControl.Profile.REGULAR)
    @@ -96,9 +100,11 @@ enum Profile {
          * Handler interface that user can register to get access to stats at
          * the end of the interval.
          */
    -    interface StatsHandler extends Consumer {
    -        /** Stats are encoded in JSON format using the FieldValue API. */
    -        void accept(FieldValue jsonStats);
    +    interface StatsHandler extends Consumer {
    +        /** Stats are available in a MapValue instance that can be searched or
    +         * rendered in JSON format using {@link FieldValue#toString} or
    +         * {@link FieldValue#toJson}. */
    +        void accept(MapValue jsonStats);
         }
     
         /**
    @@ -149,7 +155,7 @@ interface StatsHandler extends Consumer {
          * @param handler User defined StatsHandler
          * @return this
          */
    -    StatsControl registerStatsHandler(StatsHandler handler);
    +    StatsControl setStatsHandler(StatsHandler handler);
     
         /**
          * Collection of stats is enabled only between start and stop or from the
    @@ -166,7 +172,7 @@ interface StatsHandler extends Consumer {
         /**
          * Returns true if collection of stats is enabled, otherwise returns false.
          *
    -     * @return true if start() was called last, false if stop() was called last
    +     * @return true if start() was called last, false otherwise.
          */
         boolean isStarted();
     }
    diff --git a/driver/src/main/java/oracle/nosql/driver/http/Stats.java b/driver/src/main/java/oracle/nosql/driver/http/Stats.java
    index ee2592a4..9e41add3 100644
    --- a/driver/src/main/java/oracle/nosql/driver/http/Stats.java
    +++ b/driver/src/main/java/oracle/nosql/driver/http/Stats.java
    @@ -32,7 +32,6 @@
     import oracle.nosql.driver.ops.Request;
     import oracle.nosql.driver.ops.RetryStats;
     import oracle.nosql.driver.values.ArrayValue;
    -import oracle.nosql.driver.values.FieldValue;
     import oracle.nosql.driver.values.JsonOptions;
     import oracle.nosql.driver.values.MapValue;
     import oracle.nosql.driver.values.StringValue;
    @@ -468,7 +467,7 @@ synchronized void clear() {
         private void logClientStats() {
             endTime = System.currentTimeMillis();
     
    -        FieldValue fvStats = generateFieldValueStats();
    +        MapValue fvStats = generateFieldValueStats();
             // Start from scratch in the new interval.
             clearStats();
     
    @@ -486,7 +485,7 @@ private void logClientStats() {
                 StatsControlImpl.LOG_PREFIX + json);
         }
     
    -    private FieldValue generateFieldValueStats() {
    +    private MapValue generateFieldValueStats() {
             MapValue root = new MapValue();
             Timestamp ts = new Timestamp(startTime);
             ts.setNanos(0);
    diff --git a/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java b/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java
    index a62e83b1..c5dd62f8 100644
    --- a/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java
    +++ b/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java
    @@ -91,7 +91,7 @@ public boolean getPrettyPrint() {
         }
     
         @Override
    -    public StatsControl registerStatsHandler(StatsHandler statsHandler) {
    +    public StatsControl setStatsHandler(StatsHandler statsHandler) {
             this.statsHandler = statsHandler;
             return this;
         }
    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 07bd9681..2a3486c4 100644
    --- a/driver/src/main/java/oracle/nosql/driver/package-info.java
    +++ b/driver/src/main/java/oracle/nosql/driver/package-info.java
    @@ -174,12 +174,12 @@
      * {@link oracle.nosql.driver.StatsControl#start()} and
      * {@link oracle.nosql.driver.StatsControl#stop()}. The following example shows
      * how to use a stats handler:
    - * 
    + * 
      *     NoSQLHandleConfig config = new NoSQLHandleConfig( endpoint );
      *     config.setStatsProfile(StatsControl.Profile.REGULAR);
      *     config.setStatsInterval(600);
      *     config.setStatsPrettyPrint(false);
    - *     config.registerStatsHandler(
    + *     config.setStatsHandler(
      *         new StatsControl.StatsHandler() {
      *             public void accept(FieldValue jsonStats) {
      *                 System.out.println("!!! Got a stat: " + jsonStats);
    @@ -208,7 +208,7 @@
      *
      *     // ... application code without stats
      *     handle.close();
    - * 

    + *

    * * The following is an example of stats log entry using the ALL * profile:

    1. From e4d60898e537b4fea347d5b7e84e278b0d1b6b20 Mon Sep 17 00:00:00 2001 From: Cezar Andrei Date: Wed, 10 Nov 2021 11:05:46 -0600 Subject: [PATCH 12/13] Added getStatsHandler in StatsControl along the setter. --- driver/src/main/java/oracle/nosql/driver/StatsControl.java | 7 +++++++ driver/src/main/java/oracle/nosql/driver/http/Stats.java | 2 +- .../java/oracle/nosql/driver/http/StatsControlImpl.java | 3 ++- 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/driver/src/main/java/oracle/nosql/driver/StatsControl.java b/driver/src/main/java/oracle/nosql/driver/StatsControl.java index f4cb70d0..e1822781 100644 --- a/driver/src/main/java/oracle/nosql/driver/StatsControl.java +++ b/driver/src/main/java/oracle/nosql/driver/StatsControl.java @@ -149,6 +149,13 @@ interface StatsHandler extends Consumer { */ boolean getPrettyPrint(); + /** + * Returns the registered stats handler. + * + * @return the current handler, null if no handler has been registered. + */ + StatsHandler getStatsHandler(); + /** * Registers a stats handler. * diff --git a/driver/src/main/java/oracle/nosql/driver/http/Stats.java b/driver/src/main/java/oracle/nosql/driver/http/Stats.java index 9e41add3..27e994db 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/Stats.java +++ b/driver/src/main/java/oracle/nosql/driver/http/Stats.java @@ -473,7 +473,7 @@ private void logClientStats() { // Call user handle if configured. StatsControl.StatsHandler statsHandler = - statsControl.getHandler(); + statsControl.getStatsHandler(); if (statsHandler != null) { statsHandler.accept(fvStats); } diff --git a/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java b/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java index c5dd62f8..ddf062ca 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java +++ b/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java @@ -96,7 +96,8 @@ public StatsControl setStatsHandler(StatsHandler statsHandler) { return this; } - public StatsHandler getHandler() { + @Override + public StatsHandler getStatsHandler() { return statsHandler; } From 7fe4557a7aeb7750ece6c76e313bf445857b984e Mon Sep 17 00:00:00 2001 From: Cezar Andrei Date: Thu, 11 Nov 2021 11:48:49 -0600 Subject: [PATCH 13/13] Moved LOG_PREFIX to StatsControl and changed it to Client stats. Fixed javadoc. --- .../main/java/oracle/nosql/driver/NoSQLHandleConfig.java | 3 +-- .../src/main/java/oracle/nosql/driver/StatsControl.java | 2 ++ driver/src/main/java/oracle/nosql/driver/http/Stats.java | 2 +- .../java/oracle/nosql/driver/http/StatsControlImpl.java | 2 -- .../src/main/java/oracle/nosql/driver/package-info.java | 8 ++++---- 5 files changed, 8 insertions(+), 9 deletions(-) diff --git a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java index 424e6d07..40455fbe 100644 --- a/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java +++ b/driver/src/main/java/oracle/nosql/driver/NoSQLHandleConfig.java @@ -20,7 +20,6 @@ import java.util.logging.Logger; import oracle.nosql.driver.Region.RegionProvider; -import oracle.nosql.driver.http.StatsControlImpl; import oracle.nosql.driver.iam.SignatureProvider; import io.netty.handler.ssl.SslContext; @@ -1344,7 +1343,7 @@ private void setConfigFromEnvironment() { profileProp.toUpperCase())); } catch (IllegalArgumentException iae) { if (logger != null) { - logger.log(Level.SEVERE, StatsControlImpl.LOG_PREFIX + + logger.log(Level.SEVERE, StatsControl.LOG_PREFIX + "Invalid profile value for system property " + PROFILE_PROPERTY + ": " + profileProp); } diff --git a/driver/src/main/java/oracle/nosql/driver/StatsControl.java b/driver/src/main/java/oracle/nosql/driver/StatsControl.java index e1822781..826bbbc5 100644 --- a/driver/src/main/java/oracle/nosql/driver/StatsControl.java +++ b/driver/src/main/java/oracle/nosql/driver/StatsControl.java @@ -85,6 +85,8 @@ */ public interface StatsControl { + String LOG_PREFIX = "Client stats|"; + /** * The following semantics are attached to the Profile: * - NONE: no stats are logged. diff --git a/driver/src/main/java/oracle/nosql/driver/http/Stats.java b/driver/src/main/java/oracle/nosql/driver/http/Stats.java index 27e994db..21cbc906 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/Stats.java +++ b/driver/src/main/java/oracle/nosql/driver/http/Stats.java @@ -482,7 +482,7 @@ private void logClientStats() { String json = fvStats.toJson(statsControl.getPrettyPrint() ? JsonOptions.PRETTY : null); statsControl.getLogger().log(Level.INFO, - StatsControlImpl.LOG_PREFIX + json); + StatsControl.LOG_PREFIX + json); } private MapValue generateFieldValueStats() { diff --git a/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java b/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java index ddf062ca..315376e1 100644 --- a/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java +++ b/driver/src/main/java/oracle/nosql/driver/http/StatsControlImpl.java @@ -20,8 +20,6 @@ public class StatsControlImpl implements StatsControl { - static public final String LOG_PREFIX = "ONJS:Monitoring stats|"; - private StatsControl.Profile profile; private int interval; private boolean prettyPrint; 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 2a3486c4..76f0389a 100644 --- a/driver/src/main/java/oracle/nosql/driver/package-info.java +++ b/driver/src/main/java/oracle/nosql/driver/package-info.java @@ -173,7 +173,7 @@ * At runtime stats collection can be used selectively by using * {@link oracle.nosql.driver.StatsControl#start()} and * {@link oracle.nosql.driver.StatsControl#stop()}. The following example shows - * how to use a stats handler: + * how to use a stats handler:

      *
        *     NoSQLHandleConfig config = new NoSQLHandleConfig( endpoint );
        *     config.setStatsProfile(StatsControl.Profile.REGULAR);
      @@ -208,12 +208,12 @@
        *
        *     // ... application code without stats
        *     handle.close();
      - * 

      + *

    * * The following is an example of stats log entry using the ALL * profile:

    1. * A one time entry containing stats id and options: - *
      INFO: ONJS:Monitoring stats|{    // INFO log entry
      + *    
      INFO: Client stats|{    // INFO log entry
        *    "sdkName" : "Oracle NoSQL SDK for Java",  // SDK name
        *    "sdkVersion" : "current",                 // SDK version
        *    "clientId" : "f595b333",                  // NoSQLHandle id
      @@ -223,7 +223,7 @@
        *    "rateLimitingEnabled" : false}            // if rate limiting is
        *    enabled
    2. * An entry at the end of each interval containing the stats values: - *
      INFO: ONJS:Monitoring stats|{
      + *    
      INFO: Client stats|{
        *   "clientId" : "b7bc7734",              // id of NoSQLHandle object
        *   "startTime" : "2021-09-20T20:11:42Z", // UTC start interval time
        *   "endTime" : "2021-09-20T20:11:47Z",   // UTC end interval time