future) {
.getMetricUpdater()
.incrementCounter(DefaultNodeMetric.UNSENT_REQUESTS, executionProfile.getName());
recordError(node, error);
- trackNodeError(node, error.getCause());
+ trackNodeError(error.getCause(), null);
sendRequest(statement, null, executionIndex, retryCount, scheduleSpeculativeExecution);
}
} else {
@@ -739,7 +741,8 @@ private void onPageTimeout(int expectedPage) {
* Invoked when a continuous paging response is received, either a successful or failed one.
*
* Delegates further processing to appropriate methods: {@link #processResultResponse(Result,
- * Frame)} if the response was successful, or {@link #processErrorResponse(Error)} if it wasn't.
+ * Frame)} if the response was successful, or {@link #processErrorResponse(Error, Frame)} if it
+ * wasn't.
*
* @param response the received {@link Frame}.
*/
@@ -760,15 +763,15 @@ public void onResponse(@NonNull Frame response) {
processResultResponse((Result) responseMessage, response);
} else if (responseMessage instanceof Error) {
LOG.trace("[{}] Got error response", logPrefix);
- processErrorResponse((Error) responseMessage);
+ processErrorResponse((Error) responseMessage, response);
} else {
IllegalStateException error =
new IllegalStateException("Unexpected response " + responseMessage);
- trackNodeError(node, error);
+ trackNodeError(error, response);
abort(error, false);
}
} catch (Throwable t) {
- trackNodeError(node, t);
+ trackNodeError(t, response);
abort(t, false);
}
} finally {
@@ -831,7 +834,8 @@ public void onFailure(@NonNull Throwable error) {
private void processResultResponse(@NonNull Result result, @Nullable Frame frame) {
assert lock.isHeldByCurrentThread();
try {
- ExecutionInfo executionInfo = createExecutionInfo(result, frame);
+ ExecutionInfo executionInfo =
+ createExecutionInfo(null).withServerResponse(result, frame).build();
if (result instanceof Rows) {
DseRowsMetadata rowsMetadata = (DseRowsMetadata) ((Rows) result).getMetadata();
if (columnDefinitions == null) {
@@ -902,7 +906,7 @@ private void processResultResponse(@NonNull Result result, @Nullable Frame frame
* @param errorMessage the error message received.
*/
@SuppressWarnings("GuardedBy") // this method is only called with the lock held
- private void processErrorResponse(@NonNull Error errorMessage) {
+ private void processErrorResponse(@NonNull Error errorMessage, @NonNull Frame frame) {
assert lock.isHeldByCurrentThread();
if (errorMessage instanceof Unprepared) {
processUnprepared((Unprepared) errorMessage);
@@ -911,7 +915,7 @@ private void processErrorResponse(@NonNull Error errorMessage) {
if (error instanceof BootstrappingException) {
LOG.trace("[{}] {} is bootstrapping, trying next node", logPrefix, node);
recordError(node, error);
- trackNodeError(node, error);
+ trackNodeError(error, frame);
sendRequest(statement, null, executionIndex, retryCount, false);
} else if (error instanceof QueryValidationException
|| error instanceof FunctionFailureException
@@ -923,7 +927,7 @@ private void processErrorResponse(@NonNull Error errorMessage) {
NodeMetricUpdater metricUpdater = ((DefaultNode) node).getMetricUpdater();
metricUpdater.incrementCounter(
DefaultNodeMetric.OTHER_ERRORS, executionProfile.getName());
- trackNodeError(node, error);
+ trackNodeError(error, frame);
abort(error, true);
} else {
try {
@@ -1062,7 +1066,7 @@ private void processUnprepared(@NonNull Unprepared errorMessage) {
+ "This usually happens when you run a 'USE...' query after "
+ "the statement was prepared.",
Bytes.toHexString(idToReprepare), Bytes.toHexString(repreparedId)));
- trackNodeError(node, illegalStateException);
+ trackNodeError(illegalStateException, null);
fatalError = illegalStateException;
} else {
LOG.trace(
@@ -1081,18 +1085,18 @@ private void processUnprepared(@NonNull Unprepared errorMessage) {
|| prepareError instanceof FunctionFailureException
|| prepareError instanceof ProtocolError) {
LOG.trace("[{}] Unrecoverable error on re-prepare, rethrowing", logPrefix);
- trackNodeError(node, prepareError);
+ trackNodeError(prepareError, null);
fatalError = prepareError;
}
}
} else if (exception instanceof RequestThrottlingException) {
- trackNodeError(node, exception);
+ trackNodeError(exception, null);
fatalError = exception;
}
if (fatalError == null) {
LOG.trace("[{}] Re-prepare failed, trying next node", logPrefix);
recordError(node, exception);
- trackNodeError(node, exception);
+ trackNodeError(exception, null);
sendRequest(statement, null, executionIndex, retryCount, false);
}
}
@@ -1120,18 +1124,18 @@ private void processRetryVerdict(@NonNull RetryVerdict verdict, @NonNull Throwab
switch (verdict.getRetryDecision()) {
case RETRY_SAME:
recordError(node, error);
- trackNodeError(node, error);
+ trackNodeError(error, null);
sendRequest(
verdict.getRetryRequest(statement), node, executionIndex, retryCount + 1, false);
break;
case RETRY_NEXT:
recordError(node, error);
- trackNodeError(node, error);
+ trackNodeError(error, null);
sendRequest(
verdict.getRetryRequest(statement), null, executionIndex, retryCount + 1, false);
break;
case RETHROW:
- trackNodeError(node, error);
+ trackNodeError(error, null);
abort(error, true);
break;
case IGNORE:
@@ -1444,12 +1448,15 @@ private void reenableAutoReadIfNeeded() {
// ERROR HANDLING
- private void trackNodeError(@NonNull Node node, @NonNull Throwable error) {
+ private void trackNodeError(@NonNull Throwable error, @Nullable Frame frame) {
if (nodeErrorReported.compareAndSet(false, true)) {
long latencyNanos = System.nanoTime() - this.messageStartTimeNanos;
context
.getRequestTracker()
- .onNodeError(this.statement, error, latencyNanos, executionProfile, node, logPrefix);
+ .onNodeError(
+ latencyNanos,
+ createExecutionInfo(error).withServerResponse(frame).build(),
+ logPrefix);
}
}
@@ -1563,21 +1570,25 @@ private void completeResultSetFuture(
if (resultSetClass.isInstance(pageOrError)) {
if (future.complete(resultSetClass.cast(pageOrError))) {
throttler.signalSuccess(ContinuousRequestHandlerBase.this);
+
+ ExecutionInfo executionInfo = null;
+ if (pageOrError instanceof AsyncPagingIterable) {
+ executionInfo = ((AsyncPagingIterable, ?>) pageOrError).getExecutionInfo();
+ } else if (pageOrError instanceof AsyncGraphResultSet) {
+ executionInfo = ((AsyncGraphResultSet) pageOrError).getRequestExecutionInfo();
+ }
+
if (nodeSuccessReported.compareAndSet(false, true)) {
- context
- .getRequestTracker()
- .onNodeSuccess(statement, nodeLatencyNanos, executionProfile, node, logPrefix);
+ context.getRequestTracker().onNodeSuccess(nodeLatencyNanos, executionInfo, logPrefix);
}
- context
- .getRequestTracker()
- .onSuccess(statement, totalLatencyNanos, executionProfile, node, logPrefix);
+ context.getRequestTracker().onSuccess(totalLatencyNanos, executionInfo, logPrefix);
}
} else {
Throwable error = (Throwable) pageOrError;
if (future.completeExceptionally(error)) {
context
.getRequestTracker()
- .onError(statement, error, totalLatencyNanos, executionProfile, node, logPrefix);
+ .onError(totalLatencyNanos, createExecutionInfo(error).build(), logPrefix);
if (error instanceof DriverTimeoutException) {
throttler.signalTimeout(ContinuousRequestHandlerBase.this);
session
@@ -1591,18 +1602,14 @@ private void completeResultSetFuture(
}
@NonNull
- private ExecutionInfo createExecutionInfo(@NonNull Result result, @Nullable Frame response) {
- ByteBuffer pagingState =
- result instanceof Rows ? ((Rows) result).getMetadata().pagingState : null;
- return new DefaultExecutionInfo(
+ private DefaultExecutionInfo.Builder createExecutionInfo(Throwable error) {
+ return DefaultExecutionInfo.builder(
statement,
node,
startedSpeculativeExecutionsCount.get(),
executionIndex,
+ error,
errors,
- pagingState,
- response,
- true,
session,
context,
executionProfile);
diff --git a/core/src/main/java/com/datastax/dse/driver/internal/core/graph/GraphExecutionInfoConverter.java b/core/src/main/java/com/datastax/dse/driver/internal/core/graph/GraphExecutionInfoConverter.java
index b6472f690d3..0c35b302577 100644
--- a/core/src/main/java/com/datastax/dse/driver/internal/core/graph/GraphExecutionInfoConverter.java
+++ b/core/src/main/java/com/datastax/dse/driver/internal/core/graph/GraphExecutionInfoConverter.java
@@ -18,6 +18,7 @@
package com.datastax.dse.driver.internal.core.graph;
import com.datastax.dse.driver.api.core.graph.GraphStatement;
+import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
import com.datastax.oss.driver.api.core.cql.QueryTrace;
import com.datastax.oss.driver.api.core.cql.Statement;
@@ -62,6 +63,11 @@ public Statement> getStatement() {
throw new ClassCastException("GraphStatement cannot be cast to Statement");
}
+ @Override
+ public DriverExecutionProfile getExecutionProfile() {
+ return graphExecutionInfo.getExecutionProfile();
+ }
+
@Nullable
@Override
public Node getCoordinator() {
@@ -78,6 +84,12 @@ public int getSuccessfulExecutionIndex() {
return graphExecutionInfo.getSuccessfulExecutionIndex();
}
+ @Nullable
+ @Override
+ public Throwable getDriverError() {
+ return graphExecutionInfo.getDriverError();
+ }
+
@NonNull
@Override
public List> getErrors() {
@@ -146,6 +158,11 @@ public GraphStatement> getStatement() {
return (GraphStatement>) executionInfo.getRequest();
}
+ @Override
+ public DriverExecutionProfile getExecutionProfile() {
+ return executionInfo.getExecutionProfile();
+ }
+
@Override
public Node getCoordinator() {
return executionInfo.getCoordinator();
@@ -161,6 +178,11 @@ public int getSuccessfulExecutionIndex() {
return executionInfo.getSuccessfulExecutionIndex();
}
+ @Override
+ public Throwable getDriverError() {
+ return executionInfo.getDriverError();
+ }
+
@Override
public List> getErrors() {
return executionInfo.getErrors();
diff --git a/core/src/main/java/com/datastax/dse/driver/internal/core/graph/GraphRequestHandler.java b/core/src/main/java/com/datastax/dse/driver/internal/core/graph/GraphRequestHandler.java
index 5c9ceb00df2..5bb5fee8ad4 100644
--- a/core/src/main/java/com/datastax/dse/driver/internal/core/graph/GraphRequestHandler.java
+++ b/core/src/main/java/com/datastax/dse/driver/internal/core/graph/GraphRequestHandler.java
@@ -330,7 +330,19 @@ private void cancelScheduledTasks() {
private void setFinalResult(
Result resultMessage, Frame responseFrame, NodeResponseCallback callback) {
try {
- ExecutionInfo executionInfo = buildExecutionInfo(callback, responseFrame);
+ ExecutionInfo executionInfo =
+ DefaultExecutionInfo.builder(
+ callback.statement,
+ callback.node,
+ startedSpeculativeExecutionsCount.get(),
+ callback.execution,
+ null,
+ errors,
+ session,
+ context,
+ callback.executionProfile)
+ .withServerResponse(resultMessage, responseFrame)
+ .build();
DriverExecutionProfile executionProfile =
Conversions.resolveExecutionProfile(callback.statement, context);
GraphProtocol subProtocol =
@@ -360,10 +372,8 @@ private void setFinalResult(
completionTimeNanos = System.nanoTime();
totalLatencyNanos = completionTimeNanos - startTimeNanos;
long nodeLatencyNanos = completionTimeNanos - callback.nodeStartTimeNanos;
- requestTracker.onNodeSuccess(
- callback.statement, nodeLatencyNanos, executionProfile, callback.node, logPrefix);
- requestTracker.onSuccess(
- callback.statement, totalLatencyNanos, executionProfile, callback.node, logPrefix);
+ requestTracker.onNodeSuccess(nodeLatencyNanos, executionInfo, logPrefix);
+ requestTracker.onSuccess(totalLatencyNanos, executionInfo, logPrefix);
}
if (sessionMetricUpdater.isEnabled(
DseSessionMetric.GRAPH_REQUESTS, executionProfile.getName())) {
@@ -417,23 +427,6 @@ private void logServerWarnings(GraphStatement> statement, List warning
LOG.warn("Query '{}' generated server side warning(s): {}", statementString, warning));
}
- private ExecutionInfo buildExecutionInfo(NodeResponseCallback callback, Frame responseFrame) {
- DriverExecutionProfile executionProfile =
- Conversions.resolveExecutionProfile(callback.statement, context);
- return new DefaultExecutionInfo(
- callback.statement,
- callback.node,
- startedSpeculativeExecutionsCount.get(),
- callback.execution,
- errors,
- null,
- responseFrame,
- true,
- session,
- context,
- executionProfile);
- }
-
@Override
public void onThrottleFailure(@NonNull RequestThrottlingException error) {
DriverExecutionProfile executionProfile =
@@ -447,27 +440,26 @@ private void setFinalError(
GraphStatement> statement, Throwable error, Node node, int execution) {
DriverExecutionProfile executionProfile =
Conversions.resolveExecutionProfile(statement, context);
+ ExecutionInfo executionInfo =
+ DefaultExecutionInfo.builder(
+ statement,
+ node,
+ startedSpeculativeExecutionsCount.get(),
+ execution,
+ error,
+ errors,
+ session,
+ context,
+ executionProfile)
+ .build();
if (error instanceof DriverException) {
- ((DriverException) error)
- .setExecutionInfo(
- new DefaultExecutionInfo(
- statement,
- node,
- startedSpeculativeExecutionsCount.get(),
- execution,
- errors,
- null,
- null,
- true,
- session,
- context,
- executionProfile));
+ ((DriverException) error).setExecutionInfo(executionInfo);
}
if (result.completeExceptionally(error)) {
cancelScheduledTasks();
if (!(requestTracker instanceof NoopRequestTracker)) {
long latencyNanos = System.nanoTime() - startTimeNanos;
- requestTracker.onError(statement, error, latencyNanos, executionProfile, node, logPrefix);
+ requestTracker.onError(latencyNanos, executionInfo, logPrefix);
}
if (error instanceof DriverTimeoutException) {
throttler.signalTimeout(this);
@@ -529,7 +521,7 @@ public void operationComplete(Future future) {
Throwable error = future.cause();
if (error instanceof EncoderException
&& error.getCause() instanceof FrameTooLongException) {
- trackNodeError(node, error.getCause(), NANOTIME_NOT_MEASURED_YET);
+ trackNodeError(this, error.getCause(), NANOTIME_NOT_MEASURED_YET, null);
setFinalError(statement, error.getCause(), node, execution);
} else {
LOG.trace(
@@ -538,7 +530,7 @@ public void operationComplete(Future future) {
channel,
error);
recordError(node, error);
- trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
+ trackNodeError(this, error, NANOTIME_NOT_MEASURED_YET, null);
((DefaultNode) node)
.getMetricUpdater()
.incrementCounter(DefaultNodeMetric.UNSENT_REQUESTS, executionProfile.getName());
@@ -646,12 +638,13 @@ public void onResponse(Frame responseFrame) {
setFinalResult((Result) responseMessage, responseFrame, this);
} else if (responseMessage instanceof Error) {
LOG.trace("[{}] Got error response, processing", logPrefix);
- processErrorResponse((Error) responseMessage);
+ processErrorResponse((Error) responseMessage, responseFrame);
} else {
trackNodeError(
- node,
+ this,
new IllegalStateException("Unexpected response " + responseMessage),
- nodeResponseTimeNanos);
+ nodeResponseTimeNanos,
+ responseFrame);
setFinalError(
statement,
new IllegalStateException("Unexpected response " + responseMessage),
@@ -659,25 +652,25 @@ public void onResponse(Frame responseFrame) {
execution);
}
} catch (Throwable t) {
- trackNodeError(node, t, nodeResponseTimeNanos);
+ trackNodeError(this, t, nodeResponseTimeNanos, responseFrame);
setFinalError(statement, t, node, execution);
}
}
- private void processErrorResponse(Error errorMessage) {
+ private void processErrorResponse(Error errorMessage, Frame responseFrame) {
CoordinatorException error = Conversions.toThrowable(node, errorMessage, context);
NodeMetricUpdater metricUpdater = ((DefaultNode) node).getMetricUpdater();
if (error instanceof BootstrappingException) {
LOG.trace("[{}] {} is bootstrapping, trying next node", logPrefix, node);
recordError(node, error);
- trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
+ trackNodeError(this, error, NANOTIME_NOT_MEASURED_YET, responseFrame);
sendRequest(statement, null, queryPlan, execution, retryCount, false);
} else if (error instanceof QueryValidationException
|| error instanceof FunctionFailureException
|| error instanceof ProtocolError) {
LOG.trace("[{}] Unrecoverable error, rethrowing", logPrefix);
metricUpdater.incrementCounter(DefaultNodeMetric.OTHER_ERRORS, executionProfile.getName());
- trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
+ trackNodeError(this, error, NANOTIME_NOT_MEASURED_YET, responseFrame);
setFinalError(statement, error, node, execution);
} else {
RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(context, executionProfile);
@@ -752,7 +745,7 @@ private void processRetryVerdict(RetryVerdict verdict, Throwable error) {
switch (verdict.getRetryDecision()) {
case RETRY_SAME:
recordError(node, error);
- trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
+ trackNodeError(this, error, NANOTIME_NOT_MEASURED_YET, null);
sendRequest(
verdict.getRetryRequest(statement),
node,
@@ -763,7 +756,7 @@ private void processRetryVerdict(RetryVerdict verdict, Throwable error) {
break;
case RETRY_NEXT:
recordError(node, error);
- trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
+ trackNodeError(this, error, NANOTIME_NOT_MEASURED_YET, null);
sendRequest(
verdict.getRetryRequest(statement),
null,
@@ -773,7 +766,7 @@ private void processRetryVerdict(RetryVerdict verdict, Throwable error) {
false);
break;
case RETHROW:
- trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
+ trackNodeError(this, error, NANOTIME_NOT_MEASURED_YET, null);
setFinalError(statement, error, node, execution);
break;
case IGNORE:
@@ -852,15 +845,29 @@ void cancel() {
* measured. If {@link #NANOTIME_NOT_MEASURED_YET}, it hasn't and we need to measure it now
* (this is to avoid unnecessary calls to System.nanoTime)
*/
- private void trackNodeError(Node node, Throwable error, long nodeResponseTimeNanos) {
+ private void trackNodeError(
+ NodeResponseCallback callback, Throwable error, long nodeResponseTimeNanos, Frame frame) {
if (requestTracker instanceof NoopRequestTracker) {
return;
}
if (nodeResponseTimeNanos == NANOTIME_NOT_MEASURED_YET) {
nodeResponseTimeNanos = System.nanoTime();
}
+ ExecutionInfo executionInfo =
+ DefaultExecutionInfo.builder(
+ callback.statement,
+ callback.node,
+ startedSpeculativeExecutionsCount.get(),
+ callback.execution,
+ error,
+ errors,
+ session,
+ context,
+ callback.executionProfile)
+ .withServerResponse(null, frame)
+ .build();
long latencyNanos = nodeResponseTimeNanos - this.nodeStartTimeNanos;
- requestTracker.onNodeError(statement, error, latencyNanos, executionProfile, node, logPrefix);
+ requestTracker.onNodeError(latencyNanos, executionInfo, logPrefix);
}
@Override
diff --git a/core/src/main/java/com/datastax/oss/driver/api/core/cql/ExecutionInfo.java b/core/src/main/java/com/datastax/oss/driver/api/core/cql/ExecutionInfo.java
index 40cfca827d1..a6ac06ca230 100644
--- a/core/src/main/java/com/datastax/oss/driver/api/core/cql/ExecutionInfo.java
+++ b/core/src/main/java/com/datastax/oss/driver/api/core/cql/ExecutionInfo.java
@@ -21,6 +21,7 @@
import com.datastax.oss.driver.api.core.DriverException;
import com.datastax.oss.driver.api.core.RequestThrottlingException;
import com.datastax.oss.driver.api.core.config.DefaultDriverOption;
+import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
import com.datastax.oss.driver.api.core.detach.AttachmentPoint;
import com.datastax.oss.driver.api.core.metadata.Node;
import com.datastax.oss.driver.api.core.retry.RetryDecision;
@@ -66,6 +67,12 @@ default Request getRequest() {
@Deprecated
Statement> getStatement();
+ /** @return Execution profile applied when executing given request. */
+ @Nullable
+ default DriverExecutionProfile getExecutionProfile() {
+ return null;
+ }
+
/**
* The node that acted as a coordinator for the query.
*
@@ -113,6 +120,12 @@ default Request getRequest() {
@NonNull
List> getErrors();
+ /** @return Exception raised by the driver to the application. */
+ @Nullable
+ default Throwable getDriverError() {
+ return null;
+ }
+
/**
* The paging state of the query, in its raw form.
*
diff --git a/core/src/main/java/com/datastax/oss/driver/api/core/tracker/RequestTracker.java b/core/src/main/java/com/datastax/oss/driver/api/core/tracker/RequestTracker.java
index d29ee48d352..de95edd151f 100644
--- a/core/src/main/java/com/datastax/oss/driver/api/core/tracker/RequestTracker.java
+++ b/core/src/main/java/com/datastax/oss/driver/api/core/tracker/RequestTracker.java
@@ -18,6 +18,7 @@
package com.datastax.oss.driver.api.core.tracker;
import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
+import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
import com.datastax.oss.driver.api.core.metadata.Node;
import com.datastax.oss.driver.api.core.session.Request;
import com.datastax.oss.driver.api.core.session.Session;
@@ -36,38 +37,166 @@
public interface RequestTracker extends AutoCloseable {
/**
- * @deprecated This method only exists for backward compatibility. Override {@link
- * #onSuccess(Request, long, DriverExecutionProfile, Node, String)} instead.
+ * Invoked each time new request is created.
+ *
+ * @param request the request to be executed
+ * @param executionProfile the execution profile of this request
+ * @param requestLogPrefix the dedicated log prefix for this request
*/
- @Deprecated
- default void onSuccess(
+ default void onRequestCreated(
@NonNull Request request,
- long latencyNanos,
@NonNull DriverExecutionProfile executionProfile,
- @NonNull Node node) {}
+ @NonNull String requestLogPrefix) {}
+
+ /**
+ * Invoked each time a new request is created and sent to next node. Due to retry policy, this
+ * method can be triggered multiple times while processing one logical request.
+ *
+ * @param request the request to be executed
+ * @param executionProfile the execution profile of this request
+ * @param node the node which will receive the request
+ * @param requestLogPrefix the dedicated log prefix for this request
+ */
+ default void onRequestCreatedForNode(
+ @NonNull Request request,
+ @NonNull DriverExecutionProfile executionProfile,
+ @NonNull Node node,
+ @NonNull String requestLogPrefix) {}
/**
* Invoked each time a request succeeds.
*
* @param latencyNanos the overall execution time (from the {@link Session#execute(Request,
* GenericType) session.execute} call until the result is made available to the client).
- * @param executionProfile the execution profile of this request.
- * @param node the node that returned the successful response.
+ * @param executionInfo the execution info containing the results of this request
* @param requestLogPrefix the dedicated log prefix for this request
*/
+ default void onSuccess(
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestLogPrefix) {
+ // delegate call to the old method
+ onSuccess(
+ executionInfo.getRequest(),
+ latencyNanos,
+ executionInfo.getExecutionProfile(),
+ executionInfo.getCoordinator(),
+ requestLogPrefix);
+ }
+
+ /**
+ * Invoked each time a request fails.
+ *
+ * @param latencyNanos the overall execution time (from the {@link Session#execute(Request,
+ * GenericType) session.execute} call until the error is propagated to the client).
+ * @param executionInfo the execution info being returned to the client for this request if
+ * available
+ * @param requestLogPrefix the dedicated log prefix for this request
+ */
+ default void onError(
+ // TODO: Shall we expose start and end timestamp so that users do not need to call nanoTime()?
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestLogPrefix) {
+ // delegate call to the old method
+ onError(
+ executionInfo.getRequest(),
+ executionInfo.getDriverError(),
+ latencyNanos,
+ executionInfo.getExecutionProfile(),
+ executionInfo.getCoordinator(),
+ requestLogPrefix);
+ }
+
+ /**
+ * Invoked each time a request succeeds at the node level. Similar to {@link #onSuccess(Request,
+ * long, DriverExecutionProfile, Node, String)} but at per node level.
+ *
+ * @param latencyNanos the overall execution time (from the {@link Session#execute(Request,
+ * GenericType) session.execute} call until the result is made available to the client).
+ * @param executionInfo the execution info containing the results of this request
+ * @param requestLogPrefix the dedicated log prefix for this request
+ */
+ default void onNodeSuccess(
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestLogPrefix) {
+ // delegate call to the old method
+ onNodeSuccess(
+ executionInfo.getRequest(),
+ latencyNanos,
+ executionInfo.getExecutionProfile(),
+ executionInfo.getCoordinator(),
+ requestLogPrefix);
+ }
+
+ /**
+ * Invoked each time a request fails at the node level. Similar to {@link #onError(Request,
+ * Throwable, long, DriverExecutionProfile, Node, String)} but at a per node level.
+ *
+ * @param latencyNanos the overall execution time (from the {@link Session#execute(Request,
+ * GenericType) session.execute} call until the error is propagated to the client).
+ * @param executionInfo the execution info containing the results of this request if available
+ * @param requestLogPrefix the dedicated log prefix for this request
+ */
+ default void onNodeError(
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestLogPrefix) {
+ // delegate call to the old method
+ onNodeError(
+ executionInfo.getRequest(),
+ executionInfo.getDriverError(),
+ latencyNanos,
+ executionInfo.getExecutionProfile(),
+ executionInfo.getCoordinator(),
+ requestLogPrefix);
+ }
+
+ /**
+ * Invoked when the session is ready to process user requests.
+ *
+ * WARNING: if you use {@code session.execute()} in your tracker implementation, keep in
+ * mind that those requests will in turn recurse back into {@code onSuccess} / {@code onError}
+ * methods. Make sure you don't trigger an infinite loop; one way to do that is to use a
+ * custom execution profile for internal requests.
+ *
+ *
This corresponds to the moment when {@link SessionBuilder#build()} returns, or the future
+ * returned by {@link SessionBuilder#buildAsync()} completes. If the session initialization fails,
+ * this method will not get called.
+ *
+ *
Listener methods are invoked from different threads; if you store the session in a field,
+ * make it at least volatile to guarantee proper publication.
+ *
+ *
This method is guaranteed to be the first one invoked on this object.
+ *
+ *
The default implementation is empty.
+ */
+ default void onSessionReady(@NonNull Session session) {}
+
+ // ----- Below methods have been deprecated and will be removed in next releases -----
+
+ /**
+ * @deprecated This method only exists for backward compatibility. Override {@link
+ * #onSuccess(long, ExecutionInfo, String)} instead.
+ */
+ @Deprecated
+ default void onSuccess(
+ @NonNull Request request,
+ long latencyNanos,
+ @NonNull DriverExecutionProfile executionProfile,
+ @NonNull Node node) {}
+
+ /**
+ * @deprecated This method only exists for backward compatibility. Override {@link
+ * #onSuccess(long, ExecutionInfo, String)} instead.
+ */
+ @Deprecated
default void onSuccess(
@NonNull Request request,
long latencyNanos,
@NonNull DriverExecutionProfile executionProfile,
@NonNull Node node,
@NonNull String requestLogPrefix) {
- // If client doesn't override onSuccess with requestLogPrefix delegate call to the old method
+ // delegate call to the old method
onSuccess(request, latencyNanos, executionProfile, node);
}
/**
- * @deprecated This method only exists for backward compatibility. Override {@link
- * #onError(Request, Throwable, long, DriverExecutionProfile, Node, String)} instead.
+ * @deprecated This method only exists for backward compatibility. Override {@link #onError(long,
+ * ExecutionInfo, String)} instead.
*/
@Deprecated
default void onError(
@@ -78,14 +207,10 @@ default void onError(
@Nullable Node node) {}
/**
- * Invoked each time a request fails.
- *
- * @param latencyNanos the overall execution time (from the {@link Session#execute(Request,
- * GenericType) session.execute} call until the error is propagated to the client).
- * @param executionProfile the execution profile of this request.
- * @param node the node that returned the error response, or {@code null} if the error occurred
- * @param requestLogPrefix the dedicated log prefix for this request
+ * @deprecated This method only exists for backward compatibility. Override {@link #onError(long,
+ * ExecutionInfo, String)} instead.
*/
+ @Deprecated
default void onError(
@NonNull Request request,
@NonNull Throwable error,
@@ -93,13 +218,13 @@ default void onError(
@NonNull DriverExecutionProfile executionProfile,
@Nullable Node node,
@NonNull String requestLogPrefix) {
- // If client doesn't override onError with requestLogPrefix delegate call to the old method
+ // delegate call to the old method
onError(request, error, latencyNanos, executionProfile, node);
}
/**
* @deprecated This method only exists for backward compatibility. Override {@link
- * #onNodeError(Request, Throwable, long, DriverExecutionProfile, Node, String)} instead.
+ * #onNodeError(long, ExecutionInfo, String)} instead.
*/
@Deprecated
default void onNodeError(
@@ -110,15 +235,10 @@ default void onNodeError(
@NonNull Node node) {}
/**
- * Invoked each time a request fails at the node level. Similar to {@link #onError(Request,
- * Throwable, long, DriverExecutionProfile, Node, String)} but at a per node level.
- *
- * @param latencyNanos the overall execution time (from the {@link Session#execute(Request,
- * GenericType) session.execute} call until the error is propagated to the client).
- * @param executionProfile the execution profile of this request.
- * @param node the node that returned the error response.
- * @param requestLogPrefix the dedicated log prefix for this request
+ * @deprecated This method only exists for backward compatibility. Override {@link
+ * #onNodeError(long, ExecutionInfo, String)} instead.
*/
+ @Deprecated
default void onNodeError(
@NonNull Request request,
@NonNull Throwable error,
@@ -126,13 +246,13 @@ default void onNodeError(
@NonNull DriverExecutionProfile executionProfile,
@NonNull Node node,
@NonNull String requestLogPrefix) {
- // If client doesn't override onNodeError with requestLogPrefix delegate call to the old method
+ // delegate call to the old method
onNodeError(request, error, latencyNanos, executionProfile, node);
}
/**
* @deprecated This method only exists for backward compatibility. Override {@link
- * #onNodeSuccess(Request, long, DriverExecutionProfile, Node, String)} instead.
+ * #onNodeSuccess(long, ExecutionInfo, String)} instead.
*/
@Deprecated
default void onNodeSuccess(
@@ -142,44 +262,17 @@ default void onNodeSuccess(
@NonNull Node node) {}
/**
- * Invoked each time a request succeeds at the node level. Similar to {@link #onSuccess(Request,
- * long, DriverExecutionProfile, Node, String)} but at per node level.
- *
- * @param latencyNanos the overall execution time (from the {@link Session#execute(Request,
- * GenericType) session.execute} call until the result is made available to the client).
- * @param executionProfile the execution profile of this request.
- * @param node the node that returned the successful response.
- * @param requestLogPrefix the dedicated log prefix for this request
+ * @deprecated This method only exists for backward compatibility. Override {@link
+ * #onNodeSuccess(long, ExecutionInfo, String)} instead.
*/
+ @Deprecated
default void onNodeSuccess(
@NonNull Request request,
long latencyNanos,
@NonNull DriverExecutionProfile executionProfile,
@NonNull Node node,
@NonNull String requestLogPrefix) {
- // If client doesn't override onNodeSuccess with requestLogPrefix delegate call to the old
- // method
+ // delegate call to the old method
onNodeSuccess(request, latencyNanos, executionProfile, node);
}
-
- /**
- * Invoked when the session is ready to process user requests.
- *
- *
WARNING: if you use {@code session.execute()} in your tracker implementation, keep in
- * mind that those requests will in turn recurse back into {@code onSuccess} / {@code onError}
- * methods. Make sure you don't trigger an infinite loop; one way to do that is to use a
- * custom execution profile for internal requests.
- *
- *
This corresponds to the moment when {@link SessionBuilder#build()} returns, or the future
- * returned by {@link SessionBuilder#buildAsync()} completes. If the session initialization fails,
- * this method will not get called.
- *
- *
Listener methods are invoked from different threads; if you store the session in a field,
- * make it at least volatile to guarantee proper publication.
- *
- *
This method is guaranteed to be the first one invoked on this object.
- *
- *
The default implementation is empty.
- */
- default void onSessionReady(@NonNull Session session) {}
}
diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/cql/Conversions.java b/core/src/main/java/com/datastax/oss/driver/internal/core/cql/Conversions.java
index ff9384b3e24..0afb182b365 100644
--- a/core/src/main/java/com/datastax/oss/driver/internal/core/cql/Conversions.java
+++ b/core/src/main/java/com/datastax/oss/driver/internal/core/cql/Conversions.java
@@ -74,6 +74,7 @@
import com.datastax.oss.protocol.internal.ProtocolConstants;
import com.datastax.oss.protocol.internal.request.Batch;
import com.datastax.oss.protocol.internal.request.Execute;
+import com.datastax.oss.protocol.internal.request.Prepare;
import com.datastax.oss.protocol.internal.request.Query;
import com.datastax.oss.protocol.internal.request.query.QueryOptions;
import com.datastax.oss.protocol.internal.response.Error;
@@ -359,6 +360,10 @@ public static ColumnDefinitions getResultDefinitions(
}
}
+ public static PrepareRequest toPrepareRequest(Prepare request) {
+ return new DefaultPrepareRequest(SimpleStatement.newInstance(request.cqlQuery));
+ }
+
public static DefaultPreparedStatement toPreparedStatement(
Prepared response, PrepareRequest request, InternalDriverContext context) {
ColumnDefinitions variableDefinitions =
diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandler.java b/core/src/main/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandler.java
index 1ee1f303ab2..6204c65abfe 100644
--- a/core/src/main/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandler.java
+++ b/core/src/main/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandler.java
@@ -25,6 +25,7 @@
import com.datastax.oss.driver.api.core.RequestThrottlingException;
import com.datastax.oss.driver.api.core.config.DefaultDriverOption;
import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
+import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
import com.datastax.oss.driver.api.core.cql.PrepareRequest;
import com.datastax.oss.driver.api.core.cql.PreparedStatement;
import com.datastax.oss.driver.api.core.metadata.Node;
@@ -37,8 +38,10 @@
import com.datastax.oss.driver.api.core.servererrors.FunctionFailureException;
import com.datastax.oss.driver.api.core.servererrors.ProtocolError;
import com.datastax.oss.driver.api.core.servererrors.QueryValidationException;
+import com.datastax.oss.driver.api.core.session.Request;
import com.datastax.oss.driver.api.core.session.throttling.RequestThrottler;
import com.datastax.oss.driver.api.core.session.throttling.Throttled;
+import com.datastax.oss.driver.api.core.tracker.RequestTracker;
import com.datastax.oss.driver.internal.core.DefaultProtocolFeature;
import com.datastax.oss.driver.internal.core.ProtocolVersionRegistry;
import com.datastax.oss.driver.internal.core.adminrequest.ThrottledAdminRequestHandler;
@@ -46,6 +49,7 @@
import com.datastax.oss.driver.internal.core.channel.ResponseCallback;
import com.datastax.oss.driver.internal.core.context.InternalDriverContext;
import com.datastax.oss.driver.internal.core.session.DefaultSession;
+import com.datastax.oss.driver.internal.core.tracker.NoopRequestTracker;
import com.datastax.oss.driver.internal.core.util.Loggers;
import com.datastax.oss.driver.internal.core.util.concurrent.CompletableFutures;
import com.datastax.oss.protocol.internal.Frame;
@@ -90,6 +94,7 @@ public class CqlPrepareHandler implements Throttled {
protected final CompletableFuture result;
private final Timer timer;
private final Timeout scheduledTimeout;
+ private final RequestTracker requestTracker;
private final RequestThrottler throttler;
private final Boolean prepareOnAllNodes;
private final DriverExecutionProfile executionProfile;
@@ -112,7 +117,7 @@ protected CqlPrepareHandler(
this.initialRequest = request;
this.session = session;
this.context = context;
- executionProfile = Conversions.resolveExecutionProfile(request, context);
+ this.executionProfile = Conversions.resolveExecutionProfile(request, context);
this.queryPlan =
context
.getLoadBalancingPolicyWrapper()
@@ -137,14 +142,15 @@ protected CqlPrepareHandler(
this.scheduledTimeout = scheduleTimeout(timeout);
this.prepareOnAllNodes = executionProfile.getBoolean(DefaultDriverOption.PREPARE_ON_ALL_NODES);
+ this.requestTracker = context.getRequestTracker();
+ trackStart();
+
this.throttler = context.getRequestThrottler();
this.throttler.register(this);
}
@Override
public void onThrottleReady(boolean wasDelayed) {
- DriverExecutionProfile executionProfile =
- Conversions.resolveExecutionProfile(initialRequest, context);
if (wasDelayed) {
session
.getMetricUpdater()
@@ -165,7 +171,8 @@ private Timeout scheduleTimeout(Duration timeoutDuration) {
if (timeoutDuration.toNanos() > 0) {
return this.timer.newTimeout(
(Timeout timeout1) -> {
- setFinalError(new DriverTimeoutException("Query timed out after " + timeoutDuration));
+ setFinalError(
+ new DriverTimeoutException("Query timed out after " + timeoutDuration), null);
if (initialCallback != null) {
initialCallback.cancel();
}
@@ -199,13 +206,12 @@ private void sendRequest(PrepareRequest request, Node node, int retryCount) {
}
}
if (channel == null) {
- setFinalError(AllNodesFailedException.fromErrors(this.errors));
+ setFinalError(AllNodesFailedException.fromErrors(this.errors), node);
} else {
InitialPrepareCallback initialPrepareCallback =
new InitialPrepareCallback(request, node, channel, retryCount);
-
Prepare message = toPrepareMessage(request);
-
+ trackNodeStart(request, node);
channel
.write(message, false, request.getCustomPayload(), initialPrepareCallback)
.addListener(initialPrepareCallback);
@@ -226,7 +232,7 @@ private Prepare toPrepareMessage(PrepareRequest request) {
}
private void recordError(Node node, Throwable error) {
- // Use a local variable to do only a single single volatile read in the nominal case
+ // Use a local variable to do only a single volatile read in the nominal case
List> errorsSnapshot = this.errors;
if (errorsSnapshot == null) {
synchronized (CqlPrepareHandler.this) {
@@ -239,7 +245,8 @@ private void recordError(Node node, Throwable error) {
errorsSnapshot.add(new AbstractMap.SimpleEntry<>(node, error));
}
- private void setFinalResult(PrepareRequest request, Prepared response) {
+ private void setFinalResult(
+ PrepareRequest request, Prepared response, InitialPrepareCallback callback) {
// Whatever happens below, we're done with this stream id
throttler.signalSuccess(this);
@@ -247,6 +254,8 @@ private void setFinalResult(PrepareRequest request, Prepared response) {
DefaultPreparedStatement preparedStatement =
Conversions.toPreparedStatement(response, request, context);
+ trackNodeEnd(request, callback.node, null, callback.nodeStartTimeNanos);
+
session
.getRepreparePayloads()
.put(preparedStatement.getId(), preparedStatement.getRepreparePayload());
@@ -257,15 +266,18 @@ private void setFinalResult(PrepareRequest request, Prepared response) {
LOG.trace(
"[{}] Done repreparing on other nodes, completing the request", logPrefix);
result.complete(preparedStatement);
+ trackEnd(callback.node, null);
})
.exceptionally(
error -> {
result.completeExceptionally(error);
+ trackEnd(callback.node, error);
return null;
});
} else {
LOG.trace("[{}] Prepare on all nodes is disabled, completing the request", logPrefix);
result.complete(preparedStatement);
+ trackEnd(callback.node, null);
}
}
@@ -298,15 +310,19 @@ private CompletionStage prepareOnOtherNode(PrepareRequest request, Node no
throttler,
session.getMetricUpdater(),
logPrefix);
+ long nodeStartTimeNanos = System.nanoTime();
+ trackNodeStart(request, node);
return handler
.start()
.handle(
(result, error) -> {
if (error == null) {
LOG.trace("[{}] Successfully reprepared on {}", logPrefix, node);
+ trackNodeEnd(request, node, null, nodeStartTimeNanos);
} else {
Loggers.warnWithException(
LOG, "[{}] Error while repreparing on {}", node, logPrefix, error);
+ trackNodeEnd(request, node, error, nodeStartTimeNanos);
}
return null;
});
@@ -320,12 +336,13 @@ public void onThrottleFailure(@NonNull RequestThrottlingException error) {
session
.getMetricUpdater()
.incrementCounter(DefaultSessionMetric.THROTTLING_ERRORS, executionProfile.getName());
- setFinalError(error);
+ setFinalError(error, null);
}
- private void setFinalError(Throwable error) {
+ private void setFinalError(Throwable error, Node node) {
if (result.completeExceptionally(error)) {
cancelTimeout();
+ trackEnd(node, error);
if (error instanceof DriverTimeoutException) {
throttler.signalTimeout(this);
} else if (!(error instanceof RequestThrottlingException)) {
@@ -336,6 +353,7 @@ private void setFinalError(Throwable error) {
private class InitialPrepareCallback
implements ResponseCallback, GenericFutureListener> {
+ private final long nodeStartTimeNanos = System.nanoTime();
private final PrepareRequest request;
private final Node node;
private final DriverChannel channel;
@@ -361,6 +379,7 @@ public void operationComplete(Future future) {
node,
future.cause().toString());
recordError(node, future.cause());
+ trackNodeEnd(request, node, future.cause(), nodeStartTimeNanos);
sendRequest(request, null, retryCount); // try next host
} else {
if (result.isDone()) {
@@ -382,15 +401,15 @@ public void onResponse(Frame responseFrame) {
Message responseMessage = responseFrame.message;
if (responseMessage instanceof Prepared) {
LOG.trace("[{}] Got result, completing", logPrefix);
- setFinalResult(request, (Prepared) responseMessage);
+ setFinalResult(request, (Prepared) responseMessage, this);
} else if (responseMessage instanceof Error) {
LOG.trace("[{}] Got error response, processing", logPrefix);
processErrorResponse((Error) responseMessage);
} else {
- setFinalError(new IllegalStateException("Unexpected response " + responseMessage));
+ setFinalError(new IllegalStateException("Unexpected response " + responseMessage), node);
}
} catch (Throwable t) {
- setFinalError(t);
+ setFinalError(t, node);
}
}
@@ -404,20 +423,21 @@ private void processErrorResponse(Error errorMessage) {
|| errorMessage.code == ProtocolConstants.ErrorCode.UNAVAILABLE
|| errorMessage.code == ProtocolConstants.ErrorCode.TRUNCATE_ERROR) {
setFinalError(
- new IllegalStateException(
- "Unexpected server error for a PREPARE query" + errorMessage));
+ new IllegalStateException("Unexpected server error for a PREPARE query" + errorMessage),
+ node);
return;
}
CoordinatorException error = Conversions.toThrowable(node, errorMessage, context);
if (error instanceof BootstrappingException) {
LOG.trace("[{}] {} is bootstrapping, trying next node", logPrefix, node);
recordError(node, error);
+ trackNodeEnd(request, node, error, nodeStartTimeNanos);
sendRequest(request, null, retryCount);
} else if (error instanceof QueryValidationException
|| error instanceof FunctionFailureException
|| error instanceof ProtocolError) {
LOG.trace("[{}] Unrecoverable error, rethrowing", logPrefix);
- setFinalError(error);
+ setFinalError(error, node);
} else {
// Because prepare requests are known to always be idempotent, we call the retry policy
// directly, without checking the flag.
@@ -433,20 +453,24 @@ private void processRetryVerdict(RetryVerdict verdict, Throwable error) {
switch (decision) {
case RETRY_SAME:
recordError(node, error);
+ trackNodeEnd(request, node, error, nodeStartTimeNanos);
sendRequest(verdict.getRetryRequest(request), node, retryCount + 1);
break;
case RETRY_NEXT:
recordError(node, error);
+ trackNodeEnd(request, node, error, nodeStartTimeNanos);
sendRequest(verdict.getRetryRequest(request), null, retryCount + 1);
break;
case RETHROW:
- setFinalError(error);
+ trackNodeEnd(request, node, error, nodeStartTimeNanos);
+ setFinalError(error, node);
break;
case IGNORE:
setFinalError(
new IllegalArgumentException(
"IGNORE decisions are not allowed for prepare requests, "
- + "please fix your retry policy."));
+ + "please fix your retry policy."),
+ node);
break;
}
}
@@ -463,7 +487,8 @@ public void onFailure(Throwable error) {
verdict = retryPolicy.onRequestAbortedVerdict(request, error, retryCount);
} catch (Throwable cause) {
setFinalError(
- new IllegalStateException("Unexpected error while invoking the retry policy", cause));
+ new IllegalStateException("Unexpected error while invoking the retry policy", cause),
+ node);
return;
}
processRetryVerdict(verdict, error);
@@ -484,4 +509,60 @@ public String toString() {
return logPrefix;
}
}
+
+ /** Notify request tracker that processing of initial statement starts. */
+ private void trackStart() {
+ if (requestTracker instanceof NoopRequestTracker) {
+ return;
+ }
+ requestTracker.onRequestCreated(initialRequest, executionProfile, logPrefix);
+ }
+
+ /**
+ * Notify request tracker that processing of statement starts at a given node. Statement is passed
+ * as a separate parameter, because it might have been changed by custom retry policy.
+ */
+ private void trackNodeStart(Request request, Node node) {
+ if (requestTracker instanceof NoopRequestTracker) {
+ return;
+ }
+ requestTracker.onRequestCreatedForNode(request, executionProfile, node, logPrefix);
+ }
+
+ /** Notify request tracker that processing of statement has been completed by a given node. */
+ private void trackNodeEnd(Request request, Node node, Throwable error, long startTimeNanos) {
+ if (requestTracker instanceof NoopRequestTracker) {
+ return;
+ }
+ long latencyNanos = System.nanoTime() - startTimeNanos;
+ ExecutionInfo executionInfo = defaultExecutionInfo(request, node, error).build();
+ if (error == null) {
+ requestTracker.onNodeSuccess(latencyNanos, executionInfo, logPrefix);
+ } else {
+ requestTracker.onNodeError(latencyNanos, executionInfo, logPrefix);
+ }
+ }
+
+ /**
+ * Notify request tracker that processing of initial statement has been completed (successfully or
+ * with error).
+ */
+ private void trackEnd(Node node, Throwable error) {
+ if (requestTracker instanceof NoopRequestTracker) {
+ return;
+ }
+ long latencyNanos = System.nanoTime() - this.startTimeNanos;
+ ExecutionInfo executionInfo = defaultExecutionInfo(initialRequest, node, error).build();
+ if (error == null) {
+ requestTracker.onSuccess(latencyNanos, executionInfo, logPrefix);
+ } else {
+ requestTracker.onError(latencyNanos, executionInfo, logPrefix);
+ }
+ }
+
+ private DefaultExecutionInfo.Builder defaultExecutionInfo(
+ Request statement, Node node, Throwable error) {
+ return new DefaultExecutionInfo.Builder(
+ statement, node, -1, 0, error, null, session, context, executionProfile);
+ }
}
diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/cql/CqlRequestHandler.java b/core/src/main/java/com/datastax/oss/driver/internal/core/cql/CqlRequestHandler.java
index 0808bdce63f..b56eecc898e 100644
--- a/core/src/main/java/com/datastax/oss/driver/internal/core/cql/CqlRequestHandler.java
+++ b/core/src/main/java/com/datastax/oss/driver/internal/core/cql/CqlRequestHandler.java
@@ -28,6 +28,7 @@
import com.datastax.oss.driver.api.core.connection.FrameTooLongException;
import com.datastax.oss.driver.api.core.cql.AsyncResultSet;
import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
+import com.datastax.oss.driver.api.core.cql.PrepareRequest;
import com.datastax.oss.driver.api.core.cql.Statement;
import com.datastax.oss.driver.api.core.metadata.Node;
import com.datastax.oss.driver.api.core.metrics.DefaultNodeMetric;
@@ -42,6 +43,7 @@
import com.datastax.oss.driver.api.core.servererrors.ReadTimeoutException;
import com.datastax.oss.driver.api.core.servererrors.UnavailableException;
import com.datastax.oss.driver.api.core.servererrors.WriteTimeoutException;
+import com.datastax.oss.driver.api.core.session.Request;
import com.datastax.oss.driver.api.core.session.throttling.RequestThrottler;
import com.datastax.oss.driver.api.core.session.throttling.Throttled;
import com.datastax.oss.driver.api.core.tracker.RequestTracker;
@@ -66,7 +68,6 @@
import com.datastax.oss.protocol.internal.response.Error;
import com.datastax.oss.protocol.internal.response.Result;
import com.datastax.oss.protocol.internal.response.error.Unprepared;
-import com.datastax.oss.protocol.internal.response.result.Rows;
import com.datastax.oss.protocol.internal.response.result.SchemaChange;
import com.datastax.oss.protocol.internal.response.result.SetKeyspace;
import com.datastax.oss.protocol.internal.response.result.Void;
@@ -100,6 +101,7 @@ public class CqlRequestHandler implements Throttled {
private static final long NANOTIME_NOT_MEASURED_YET = -1;
private final long startTimeNanos;
+ private long endTimeNanos = NANOTIME_NOT_MEASURED_YET;
private final String logPrefix;
private final Statement> initialStatement;
private final DefaultSession session;
@@ -173,6 +175,7 @@ protected CqlRequestHandler(
Duration timeout = Conversions.resolveRequestTimeout(statement, executionProfile);
this.scheduledTimeout = scheduleTimeout(timeout);
+ trackStart();
this.throttler = context.getRequestThrottler();
this.throttler.register(this);
}
@@ -206,12 +209,13 @@ private Timeout scheduleTimeout(Duration timeoutDuration) {
if (timeoutDuration.toNanos() > 0) {
try {
return this.timer.newTimeout(
- (Timeout timeout1) ->
- setFinalError(
- initialStatement,
- new DriverTimeoutException("Query timed out after " + timeoutDuration),
- null,
- -1),
+ (Timeout timeout1) -> {
+ DriverTimeoutException timeoutException =
+ new DriverTimeoutException("Query timed out after " + timeoutDuration);
+ ExecutionInfo executionInfo =
+ failedExecutionInfoNoRequestSent(timeoutException).build();
+ setFinalError(executionInfo);
+ },
timeoutDuration.toNanos(),
TimeUnit.NANOSECONDS);
} catch (IllegalStateException e) {
@@ -264,7 +268,11 @@ private void sendRequest(
// We've reached the end of the query plan without finding any node to write to
if (!result.isDone() && activeExecutionsCount.decrementAndGet() == 0) {
// We're the last execution so fail the result
- setFinalError(statement, AllNodesFailedException.fromErrors(this.errors), null, -1);
+ ExecutionInfo executionInfo =
+ failedExecutionInfoNoRequestSent(
+ statement, AllNodesFailedException.fromErrors(this.errors))
+ .build();
+ setFinalError(executionInfo);
}
} else {
NodeResponseCallback nodeResponseCallback =
@@ -278,6 +286,7 @@ private void sendRequest(
scheduleNextExecution,
logPrefix);
Message message = Conversions.toMessage(statement, executionProfile, context);
+ trackNodeStart(statement, node, nodeResponseCallback.logPrefix);
channel
.write(message, statement.isTracing(), statement.getCustomPayload(), nodeResponseCallback)
.addListener(nodeResponseCallback);
@@ -285,7 +294,7 @@ private void sendRequest(
}
private void recordError(Node node, Throwable error) {
- // Use a local variable to do only a single single volatile read in the nominal case
+ // Use a local variable to do only a single volatile read in the nominal case
List> errorsSnapshot = this.errors;
if (errorsSnapshot == null) {
synchronized (CqlRequestHandler.this) {
@@ -319,32 +328,25 @@ private void setFinalResult(
NodeResponseCallback callback) {
try {
ExecutionInfo executionInfo =
- buildExecutionInfo(callback, resultMessage, responseFrame, schemaInAgreement);
+ defaultExecutionInfo(callback, null)
+ .withServerResponse(resultMessage, responseFrame)
+ .withSchemaInAgreement(schemaInAgreement)
+ .build();
AsyncResultSet resultSet =
Conversions.toResultSet(resultMessage, executionInfo, session, context);
if (result.complete(resultSet)) {
cancelScheduledTasks();
throttler.signalSuccess(this);
- // Only call nanoTime() if we're actually going to use it
- long completionTimeNanos = NANOTIME_NOT_MEASURED_YET,
- totalLatencyNanos = NANOTIME_NOT_MEASURED_YET;
-
- if (!(requestTracker instanceof NoopRequestTracker)) {
- completionTimeNanos = System.nanoTime();
- totalLatencyNanos = completionTimeNanos - startTimeNanos;
- long nodeLatencyNanos = completionTimeNanos - callback.nodeStartTimeNanos;
- requestTracker.onNodeSuccess(
- callback.statement, nodeLatencyNanos, executionProfile, callback.node, logPrefix);
- requestTracker.onSuccess(
- callback.statement, totalLatencyNanos, executionProfile, callback.node, logPrefix);
- }
+ long endTimeNanos = trackNodeEnd(callback, executionInfo);
+ trackEnd(executionInfo);
if (sessionMetricUpdater.isEnabled(
DefaultSessionMetric.CQL_REQUESTS, executionProfile.getName())) {
- if (completionTimeNanos == NANOTIME_NOT_MEASURED_YET) {
- completionTimeNanos = System.nanoTime();
- totalLatencyNanos = completionTimeNanos - startTimeNanos;
+ // Only call nanoTime() if we're actually going to use it
+ if (endTimeNanos == NANOTIME_NOT_MEASURED_YET) {
+ endTimeNanos = System.nanoTime();
}
+ long totalLatencyNanos = endTimeNanos - startTimeNanos;
sessionMetricUpdater.updateTimer(
DefaultSessionMetric.CQL_REQUESTS,
executionProfile.getName(),
@@ -359,7 +361,9 @@ private void setFinalResult(
logServerWarnings(callback.statement, executionProfile, executionInfo.getWarnings());
}
} catch (Throwable error) {
- setFinalError(callback.statement, error, callback.node, -1);
+ // something unpredictable unexpected happened here that we can't blame on the request itself
+ ExecutionInfo executionInfo = defaultExecutionInfo(callback, -1, error).build();
+ setFinalError(executionInfo);
}
}
@@ -390,61 +394,28 @@ private void logServerWarnings(
LOG.warn("Query '{}' generated server side warning(s): {}", statementString, warning));
}
- private ExecutionInfo buildExecutionInfo(
- NodeResponseCallback callback,
- Result resultMessage,
- Frame responseFrame,
- boolean schemaInAgreement) {
- ByteBuffer pagingState =
- (resultMessage instanceof Rows) ? ((Rows) resultMessage).getMetadata().pagingState : null;
- return new DefaultExecutionInfo(
- callback.statement,
- callback.node,
- startedSpeculativeExecutionsCount.get(),
- callback.execution,
- errors,
- pagingState,
- responseFrame,
- schemaInAgreement,
- session,
- context,
- executionProfile);
- }
-
@Override
public void onThrottleFailure(@NonNull RequestThrottlingException error) {
sessionMetricUpdater.incrementCounter(
DefaultSessionMetric.THROTTLING_ERRORS, executionProfile.getName());
- setFinalError(initialStatement, error, null, -1);
+ ExecutionInfo executionInfo = failedExecutionInfoNoRequestSent(error).build();
+ setFinalError(executionInfo);
}
- private void setFinalError(Statement> statement, Throwable error, Node node, int execution) {
- if (error instanceof DriverException) {
- ((DriverException) error)
- .setExecutionInfo(
- new DefaultExecutionInfo(
- statement,
- node,
- startedSpeculativeExecutionsCount.get(),
- execution,
- errors,
- null,
- null,
- true,
- session,
- context,
- executionProfile));
+ private void setFinalError(ExecutionInfo executionInfo) {
+ Throwable error = executionInfo.getDriverError();
+ if (executionInfo.getDriverError() instanceof DriverException) {
+ // ExecutionInfo is not exposed for retried errors
+ ((DriverException) executionInfo.getDriverError()).setExecutionInfo(executionInfo);
}
if (result.completeExceptionally(error)) {
cancelScheduledTasks();
- if (!(requestTracker instanceof NoopRequestTracker)) {
- long latencyNanos = System.nanoTime() - startTimeNanos;
- requestTracker.onError(statement, error, latencyNanos, executionProfile, node, logPrefix);
- }
+ trackEnd(executionInfo);
if (error instanceof DriverTimeoutException) {
throttler.signalTimeout(this);
sessionMetricUpdater.incrementCounter(
- DefaultSessionMetric.CQL_CLIENT_TIMEOUTS, executionProfile.getName());
+ DefaultSessionMetric.CQL_CLIENT_TIMEOUTS,
+ executionInfo.getExecutionProfile().getName());
} else if (!(error instanceof RequestThrottlingException)) {
throttler.signalError(this, error);
}
@@ -460,6 +431,7 @@ private class NodeResponseCallback
implements ResponseCallback, GenericFutureListener> {
private final long nodeStartTimeNanos = System.nanoTime();
+ private long nodeEndTimeNanos = NANOTIME_NOT_MEASURED_YET;
private final Statement> statement;
private final Node node;
private final Queue queryPlan;
@@ -492,23 +464,28 @@ private NodeResponseCallback(
this.logPrefix = logPrefix + "|" + execution;
}
- // this gets invoked once the write completes.
+ // this gets invoked once the write request completes.
@Override
public void operationComplete(Future future) throws Exception {
if (!future.isSuccess()) {
Throwable error = future.cause();
if (error instanceof EncoderException
&& error.getCause() instanceof FrameTooLongException) {
- trackNodeError(node, error.getCause(), NANOTIME_NOT_MEASURED_YET);
- setFinalError(statement, error.getCause(), node, execution);
+ ExecutionInfo executionInfo =
+ CqlRequestHandler.this.defaultExecutionInfo(this, error.getCause()).build();
+ trackNodeEnd(this, executionInfo);
+ setFinalError(executionInfo);
} else {
LOG.trace(
"[{}] Failed to send request on {}, trying next node (cause: {})",
logPrefix,
channel,
+ error.getMessage(),
error);
+ ExecutionInfo executionInfo =
+ CqlRequestHandler.this.defaultExecutionInfo(this, error).build();
recordError(node, error);
- trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
+ trackNodeEnd(this, executionInfo);
((DefaultNode) node)
.getMetricUpdater()
.incrementCounter(DefaultNodeMetric.UNSENT_REQUESTS, executionProfile.getName());
@@ -598,7 +575,7 @@ public void onResponse(Frame responseFrame) {
NodeMetricUpdater nodeMetricUpdater = ((DefaultNode) node).getMetricUpdater();
if (nodeMetricUpdater.isEnabled(DefaultNodeMetric.CQL_MESSAGES, executionProfile.getName())) {
nodeResponseTimeNanos = System.nanoTime();
- long nodeLatency = System.nanoTime() - nodeStartTimeNanos;
+ long nodeLatency = nodeResponseTimeNanos - nodeStartTimeNanos;
nodeMetricUpdater.updateTimer(
DefaultNodeMetric.CQL_MESSAGES,
executionProfile.getName(),
@@ -642,25 +619,22 @@ public void onResponse(Frame responseFrame) {
setFinalResult((Result) responseMessage, responseFrame, true, this);
} else if (responseMessage instanceof Error) {
LOG.trace("[{}] Got error response, processing", logPrefix);
- processErrorResponse((Error) responseMessage);
+ processErrorResponse((Error) responseMessage, responseFrame);
} else {
- trackNodeError(
- node,
- new IllegalStateException("Unexpected response " + responseMessage),
- nodeResponseTimeNanos);
- setFinalError(
- statement,
- new IllegalStateException("Unexpected response " + responseMessage),
- node,
- execution);
+ IllegalStateException error =
+ new IllegalStateException("Unexpected response " + responseMessage);
+ ExecutionInfo executionInfo = defaultExecutionInfo(error).build();
+ trackNodeEnd(this, executionInfo);
+ setFinalError(executionInfo);
}
} catch (Throwable t) {
- trackNodeError(node, t, nodeResponseTimeNanos);
- setFinalError(statement, t, node, execution);
+ ExecutionInfo executionInfo = defaultExecutionInfo(t).build();
+ trackNodeEnd(this, executionInfo);
+ setFinalError(executionInfo);
}
}
- private void processErrorResponse(Error errorMessage) {
+ private void processErrorResponse(Error errorMessage, Frame errorFrame) {
if (errorMessage.code == ProtocolConstants.ErrorCode.UNPREPARED) {
ByteBuffer idToReprepare = ByteBuffer.wrap(((Unprepared) errorMessage).id);
LOG.trace(
@@ -686,6 +660,14 @@ private void processErrorResponse(Error errorMessage) {
throttler,
sessionMetricUpdater,
logPrefix);
+ PrepareRequest reprepareRequest = Conversions.toPrepareRequest(reprepareMessage);
+ long reprepareStartNanos = System.nanoTime();
+ IllegalStateException driverError =
+ new IllegalStateException("Unexpected response " + errorMessage);
+ trackNodeEnd(
+ this, defaultExecutionInfo(driverError).withServerResponse(errorFrame).build());
+ // TODO: Shall we have different logPrefix?
+ trackReprepareStatementStart(reprepareRequest, this, logPrefix);
reprepareHandler
.start()
.handle(
@@ -702,18 +684,28 @@ private void processErrorResponse(Error errorMessage) {
|| prepareError instanceof FunctionFailureException
|| prepareError instanceof ProtocolError) {
LOG.trace("[{}] Unrecoverable error on reprepare, rethrowing", logPrefix);
- trackNodeError(node, prepareError, NANOTIME_NOT_MEASURED_YET);
- setFinalError(statement, prepareError, node, execution);
+ trackReprepareStatementEnd(
+ reprepareRequest, this, prepareError, reprepareStartNanos, logPrefix);
+ ExecutionInfo executionInfo = defaultExecutionInfo(prepareError).build();
+ trackNodeEnd(this, executionInfo);
+ setFinalError(executionInfo);
return null;
}
}
} else if (exception instanceof RequestThrottlingException) {
- trackNodeError(node, exception, NANOTIME_NOT_MEASURED_YET);
- setFinalError(statement, exception, node, execution);
+ trackReprepareStatementEnd(
+ reprepareRequest, this, exception, reprepareStartNanos, logPrefix);
+ ExecutionInfo executionInfo = defaultExecutionInfo(exception).build();
+ trackNodeEnd(this, executionInfo);
+ setFinalError(executionInfo);
return null;
}
recordError(node, exception);
- trackNodeError(node, exception, NANOTIME_NOT_MEASURED_YET);
+ trackReprepareStatementEnd(
+ reprepareRequest, this, exception, reprepareStartNanos, logPrefix);
+ ExecutionInfo executionInfo = defaultExecutionInfo(exception).build();
+ trackNodeEnd(this, executionInfo);
+ setFinalError(executionInfo);
LOG.trace("[{}] Reprepare failed, trying next node", logPrefix);
sendRequest(statement, null, queryPlan, execution, retryCount, false);
} else {
@@ -727,10 +719,18 @@ private void processErrorResponse(Error errorMessage) {
+ "the statement was prepared.",
Bytes.toHexString(idToReprepare),
Bytes.toHexString(repreparedId)));
- trackNodeError(node, illegalStateException, NANOTIME_NOT_MEASURED_YET);
- setFinalError(statement, illegalStateException, node, execution);
+ // notify error in initial statement execution
+ ExecutionInfo executionInfo =
+ defaultExecutionInfo(illegalStateException).build();
+ trackNodeEnd(this, executionInfo);
+ setFinalError(executionInfo);
}
- LOG.trace("[{}] Reprepare sucessful, retrying", logPrefix);
+ LOG.trace("[{}] Reprepare successful, retrying", logPrefix);
+ // notify statement preparation as successful
+ trackReprepareStatementEnd(
+ reprepareRequest, this, null, reprepareStartNanos, logPrefix);
+ // do not report to onRequestStart(), because we already did during first
+ // attempt
sendRequest(statement, node, queryPlan, execution, retryCount, false);
}
return null;
@@ -741,16 +741,18 @@ private void processErrorResponse(Error errorMessage) {
NodeMetricUpdater metricUpdater = ((DefaultNode) node).getMetricUpdater();
if (error instanceof BootstrappingException) {
LOG.trace("[{}] {} is bootstrapping, trying next node", logPrefix, node);
+ ExecutionInfo executionInfo = defaultExecutionInfo(error).build();
recordError(node, error);
- trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
+ trackNodeEnd(this, executionInfo);
sendRequest(statement, null, queryPlan, execution, retryCount, false);
} else if (error instanceof QueryValidationException
|| error instanceof FunctionFailureException
|| error instanceof ProtocolError) {
LOG.trace("[{}] Unrecoverable error, rethrowing", logPrefix);
metricUpdater.incrementCounter(DefaultNodeMetric.OTHER_ERRORS, executionProfile.getName());
- trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
- setFinalError(statement, error, node, execution);
+ ExecutionInfo executionInfo = defaultExecutionInfo(error).build();
+ trackNodeEnd(this, executionInfo);
+ setFinalError(executionInfo);
} else {
RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(context, executionProfile);
RetryVerdict verdict;
@@ -821,10 +823,11 @@ private void processErrorResponse(Error errorMessage) {
private void processRetryVerdict(RetryVerdict verdict, Throwable error) {
LOG.trace("[{}] Processing retry decision {}", logPrefix, verdict);
+ ExecutionInfo executionInfo = defaultExecutionInfo(error).build();
switch (verdict.getRetryDecision()) {
case RETRY_SAME:
recordError(node, error);
- trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
+ trackNodeEnd(this, executionInfo);
sendRequest(
verdict.getRetryRequest(statement),
node,
@@ -835,7 +838,7 @@ private void processRetryVerdict(RetryVerdict verdict, Throwable error) {
break;
case RETRY_NEXT:
recordError(node, error);
- trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
+ trackNodeEnd(this, executionInfo);
sendRequest(
verdict.getRetryRequest(statement),
null,
@@ -845,8 +848,8 @@ private void processRetryVerdict(RetryVerdict verdict, Throwable error) {
false);
break;
case RETHROW:
- trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
- setFinalError(statement, error, node, execution);
+ trackNodeEnd(this, executionInfo);
+ setFinalError(executionInfo);
break;
case IGNORE:
setFinalResult(Void.INSTANCE, null, true, this);
@@ -872,7 +875,7 @@ private void updateErrorMetrics(
metricUpdater.incrementCounter(ignoresOnError, executionProfile.getName());
break;
case RETHROW:
- // nothing do do
+ // nothing to do
}
}
@@ -882,7 +885,7 @@ public void onFailure(Throwable error) {
if (result.isDone()) {
return;
}
- LOG.trace("[{}] Request failure, processing: {}", logPrefix, error);
+ LOG.trace("[{}] Request failure, processing: {}", logPrefix, error.getMessage(), error);
RetryVerdict verdict;
if (!Conversions.resolveIdempotence(statement, executionProfile)
|| error instanceof FrameTooLongException) {
@@ -892,11 +895,10 @@ public void onFailure(Throwable error) {
RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(context, executionProfile);
verdict = retryPolicy.onRequestAbortedVerdict(statement, error, retryCount);
} catch (Throwable cause) {
- setFinalError(
- statement,
- new IllegalStateException("Unexpected error while invoking the retry policy", cause),
- null,
- execution);
+ IllegalStateException driverError =
+ new IllegalStateException("Unexpected error while invoking the retry policy", cause);
+ ExecutionInfo executionInfo = defaultExecutionInfo(driverError).build();
+ setFinalError(executionInfo);
return;
}
}
@@ -919,20 +921,8 @@ public void cancel() {
}
}
- /**
- * @param nodeResponseTimeNanos the time we received the response, if it's already been
- * measured. If {@link #NANOTIME_NOT_MEASURED_YET}, it hasn't and we need to measure it now
- * (this is to avoid unnecessary calls to System.nanoTime)
- */
- private void trackNodeError(Node node, Throwable error, long nodeResponseTimeNanos) {
- if (requestTracker instanceof NoopRequestTracker) {
- return;
- }
- if (nodeResponseTimeNanos == NANOTIME_NOT_MEASURED_YET) {
- nodeResponseTimeNanos = System.nanoTime();
- }
- long latencyNanos = nodeResponseTimeNanos - this.nodeStartTimeNanos;
- requestTracker.onNodeError(statement, error, latencyNanos, executionProfile, node, logPrefix);
+ private DefaultExecutionInfo.Builder defaultExecutionInfo(Throwable error) {
+ return CqlRequestHandler.this.defaultExecutionInfo(this, execution, error);
}
@Override
@@ -940,4 +930,154 @@ public String toString() {
return logPrefix;
}
}
+
+ private DefaultExecutionInfo.Builder defaultExecutionInfo(
+ NodeResponseCallback callback, Throwable error) {
+ return defaultExecutionInfo(callback, callback.execution, error);
+ }
+
+ private DefaultExecutionInfo.Builder defaultExecutionInfo(
+ NodeResponseCallback callback, int execution, Throwable error) {
+ return new DefaultExecutionInfo.Builder(
+ callback.statement,
+ callback.node,
+ startedSpeculativeExecutionsCount.get(),
+ execution,
+ error,
+ errors,
+ session,
+ context,
+ executionProfile);
+ }
+
+ private DefaultExecutionInfo.Builder failedExecutionInfoNoRequestSent(Throwable error) {
+ return failedExecutionInfoNoRequestSent(initialStatement, error);
+ }
+
+ private DefaultExecutionInfo.Builder failedExecutionInfoNoRequestSent(
+ Statement> statement, Throwable error) {
+ return new DefaultExecutionInfo.Builder(
+ statement,
+ null,
+ startedSpeculativeExecutionsCount.get(),
+ -1,
+ error,
+ errors,
+ session,
+ context,
+ executionProfile);
+ }
+
+ /** Notify request tracker that processing of initial statement starts. */
+ private void trackStart() {
+ trackStart(initialStatement, logPrefix);
+ }
+
+ /** Notify request tracker that processing of given statement starts. */
+ private void trackStart(Request request, String logPrefix) {
+ if (requestTracker instanceof NoopRequestTracker) {
+ return;
+ }
+ requestTracker.onRequestCreated(request, executionProfile, logPrefix);
+ }
+
+ /** Notify request tracker that processing of given statement starts at a certain node. */
+ private void trackNodeStart(Request request, Node node, String logPrefix) {
+ if (requestTracker instanceof NoopRequestTracker) {
+ return;
+ }
+ requestTracker.onRequestCreatedForNode(request, executionProfile, node, logPrefix);
+ }
+
+ /** Utility method to trigger {@link RequestTracker} based on {@link NodeResponseCallback}. */
+ private long trackNodeEnd(NodeResponseCallback callback, ExecutionInfo executionInfo) {
+ callback.nodeEndTimeNanos =
+ trackNodeEndInternal(
+ executionInfo,
+ callback.nodeStartTimeNanos,
+ callback.nodeEndTimeNanos,
+ callback.logPrefix);
+ return callback.nodeEndTimeNanos;
+ }
+
+ /**
+ * Notify request tracker that processing of initial statement has been completed (successfully or
+ * with error).
+ */
+ private void trackEnd(ExecutionInfo executionInfo) {
+ endTimeNanos = trackEndInternal(executionInfo, startTimeNanos, endTimeNanos, logPrefix);
+ }
+
+ /**
+ * Notify request tracker that processing of statement has been completed by a given node. To
+ * minimalize number of calls to {@code System#nanoTime()} caller may pass end timestamp. If
+ * passed timestamp equals {@code NANOTIME_NOT_MEASURED_YET}, method returns current end timestamp
+ * for further reuse.
+ */
+ private long trackNodeEndInternal(
+ ExecutionInfo executionInfo, long startTimeNanos, long endTimeNanos, String logPrefix) {
+ if (requestTracker instanceof NoopRequestTracker) {
+ return NANOTIME_NOT_MEASURED_YET;
+ }
+ endTimeNanos = endTimeNanos == -1 ? System.nanoTime() : endTimeNanos;
+ long latencyNanos = endTimeNanos - startTimeNanos;
+ if (executionInfo.getDriverError() == null) {
+ requestTracker.onNodeSuccess(latencyNanos, executionInfo, logPrefix);
+ } else {
+ requestTracker.onNodeError(latencyNanos, executionInfo, logPrefix);
+ }
+ return endTimeNanos;
+ }
+
+ /**
+ * Notify request tracker that processing of given statement has been completed (successfully or
+ * with error).
+ */
+ private long trackEndInternal(
+ ExecutionInfo executionInfo, long startTimeNanos, long endTimeNanos, String logPrefix) {
+ if (requestTracker instanceof NoopRequestTracker) {
+ return NANOTIME_NOT_MEASURED_YET;
+ }
+ endTimeNanos = endTimeNanos == NANOTIME_NOT_MEASURED_YET ? System.nanoTime() : endTimeNanos;
+ long latencyNanos = endTimeNanos - startTimeNanos;
+ if (executionInfo.getDriverError() == null) {
+ requestTracker.onSuccess(latencyNanos, executionInfo, logPrefix);
+ } else {
+ requestTracker.onError(latencyNanos, executionInfo, logPrefix);
+ }
+ return endTimeNanos;
+ }
+
+ /**
+ * Utility method to notify request tracker about start execution of re-prepating prepared
+ * statement.
+ */
+ private void trackReprepareStatementStart(
+ Request reprepareRequest, NodeResponseCallback callback, String logPrefix) {
+ trackStart(reprepareRequest, logPrefix);
+ trackNodeStart(reprepareRequest, callback.node, logPrefix);
+ }
+
+ /**
+ * Utility method to notify request tracker about completed execution of re-prepating prepared
+ * statement.
+ */
+ private void trackReprepareStatementEnd(
+ Request statement,
+ NodeResponseCallback callback,
+ Throwable error,
+ long startTimeNanos,
+ String logPrefix) {
+ ExecutionInfo executionInfo =
+ defaultReprepareExecutionInfo(statement, callback.node, error).build();
+ long endTimeNanos =
+ trackNodeEndInternal(executionInfo, startTimeNanos, NANOTIME_NOT_MEASURED_YET, logPrefix);
+ trackEndInternal(executionInfo, startTimeNanos, endTimeNanos, logPrefix);
+ }
+
+ private DefaultExecutionInfo.Builder defaultReprepareExecutionInfo(
+ Request statement, Node node, Throwable error) {
+ return new DefaultExecutionInfo.Builder(
+ statement, node, -1, 0, error, null, session, context, executionProfile);
+ }
}
diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/cql/DefaultExecutionInfo.java b/core/src/main/java/com/datastax/oss/driver/internal/core/cql/DefaultExecutionInfo.java
index 3ab57ddc598..fed8cf8e8b8 100644
--- a/core/src/main/java/com/datastax/oss/driver/internal/core/cql/DefaultExecutionInfo.java
+++ b/core/src/main/java/com/datastax/oss/driver/internal/core/cql/DefaultExecutionInfo.java
@@ -28,6 +28,8 @@
import com.datastax.oss.driver.internal.core.session.DefaultSession;
import com.datastax.oss.driver.internal.core.util.concurrent.CompletableFutures;
import com.datastax.oss.protocol.internal.Frame;
+import com.datastax.oss.protocol.internal.response.Result;
+import com.datastax.oss.protocol.internal.response.result.Rows;
import edu.umd.cs.findbugs.annotations.NonNull;
import edu.umd.cs.findbugs.annotations.Nullable;
import java.nio.ByteBuffer;
@@ -45,6 +47,7 @@ public class DefaultExecutionInfo implements ExecutionInfo {
private final Node coordinator;
private final int speculativeExecutionCount;
private final int successfulExecutionIndex;
+ private final Throwable driverError;
private final List> errors;
private final ByteBuffer pagingState;
private final UUID tracingId;
@@ -57,11 +60,12 @@ public class DefaultExecutionInfo implements ExecutionInfo {
private final InternalDriverContext context;
private final DriverExecutionProfile executionProfile;
- public DefaultExecutionInfo(
+ private DefaultExecutionInfo(
Request request,
Node coordinator,
int speculativeExecutionCount,
int successfulExecutionIndex,
+ Throwable driverError,
List> errors,
ByteBuffer pagingState,
Frame frame,
@@ -74,6 +78,7 @@ public DefaultExecutionInfo(
this.coordinator = coordinator;
this.speculativeExecutionCount = speculativeExecutionCount;
this.successfulExecutionIndex = successfulExecutionIndex;
+ this.driverError = driverError;
this.errors = errors;
this.pagingState = pagingState;
@@ -102,6 +107,11 @@ public Request getRequest() {
return request;
}
+ @Override
+ public DriverExecutionProfile getExecutionProfile() {
+ return executionProfile;
+ }
+
@Nullable
@Override
public Node getCoordinator() {
@@ -118,6 +128,12 @@ public int getSuccessfulExecutionIndex() {
return successfulExecutionIndex;
}
+ @Nullable
+ @Override
+ public Throwable getDriverError() {
+ return driverError;
+ }
+
@NonNull
@Override
public List> getErrors() {
@@ -189,4 +205,99 @@ public int getResponseSizeInBytes() {
public int getCompressedResponseSizeInBytes() {
return compressedResponseSizeInBytes;
}
+
+ public static Builder builder(
+ Request request,
+ Node coordinator,
+ int speculativeExecutionCount,
+ int successfulExecutionIndex,
+ Throwable driverError,
+ List> errors,
+ DefaultSession session,
+ InternalDriverContext context,
+ DriverExecutionProfile executionProfile) {
+ return new Builder(
+ request,
+ coordinator,
+ speculativeExecutionCount,
+ successfulExecutionIndex,
+ driverError,
+ errors,
+ session,
+ context,
+ executionProfile);
+ }
+
+ public static class Builder {
+ private final Request request;
+ private final Node coordinator;
+ private final int speculativeExecutionCount;
+ private final int successfulExecutionIndex;
+ private final Throwable driverError;
+ private final List> errors;
+ private final DefaultSession session;
+ private final InternalDriverContext context;
+ private final DriverExecutionProfile executionProfile;
+
+ private Result response;
+ private Frame frame;
+ private boolean schemaInAgreement = true;
+
+ public Builder(
+ Request request,
+ Node coordinator,
+ int speculativeExecutionCount,
+ int successfulExecutionIndex,
+ Throwable driverError,
+ List> errors,
+ DefaultSession session,
+ InternalDriverContext context,
+ DriverExecutionProfile executionProfile) {
+ this.request = request;
+ this.coordinator = coordinator;
+ this.speculativeExecutionCount = speculativeExecutionCount;
+ this.successfulExecutionIndex = successfulExecutionIndex;
+ this.driverError = driverError;
+ this.errors = errors;
+ this.session = session;
+ this.context = context;
+ this.executionProfile = executionProfile;
+ }
+
+ public Builder withServerResponse(Result response, Frame frame) {
+ this.response = response;
+ this.frame = frame;
+ return this;
+ }
+
+ /** Client received a response, but it could not be parsed to expected message. */
+ public Builder withServerResponse(Frame frame) {
+ return withServerResponse(null, frame);
+ }
+
+ public Builder withSchemaInAgreement(boolean schemaInAgreement) {
+ this.schemaInAgreement = schemaInAgreement;
+ return this;
+ }
+
+ public DefaultExecutionInfo build() {
+ final ByteBuffer pagingState =
+ (response instanceof Rows) ? ((Rows) response).getMetadata().pagingState : null;
+ DefaultExecutionInfo executionInfo =
+ new DefaultExecutionInfo(
+ request,
+ coordinator,
+ speculativeExecutionCount,
+ successfulExecutionIndex,
+ driverError,
+ errors,
+ pagingState,
+ frame,
+ schemaInAgreement,
+ session,
+ context,
+ executionProfile);
+ return executionInfo;
+ }
+ }
}
diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicy.java b/core/src/main/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicy.java
index 0f03cbb3643..7c2210ee1d5 100644
--- a/core/src/main/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicy.java
+++ b/core/src/main/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicy.java
@@ -21,8 +21,8 @@
import static java.util.concurrent.TimeUnit.MINUTES;
import com.datastax.oss.driver.api.core.config.DefaultDriverOption;
-import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
import com.datastax.oss.driver.api.core.context.DriverContext;
+import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
import com.datastax.oss.driver.api.core.metadata.Node;
import com.datastax.oss.driver.api.core.session.Request;
import com.datastax.oss.driver.api.core.session.Session;
@@ -241,23 +241,14 @@ > getInFlight((Node) currentNodes[1], session)) {
@Override
public void onNodeSuccess(
- @NonNull Request request,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- @NonNull Node node,
- @NonNull String logPrefix) {
- updateResponseTimes(node);
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String logPrefix) {
+ updateResponseTimes(executionInfo.getCoordinator());
}
@Override
public void onNodeError(
- @NonNull Request request,
- @NonNull Throwable error,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- @NonNull Node node,
- @NonNull String logPrefix) {
- updateResponseTimes(node);
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String logPrefix) {
+ updateResponseTimes(executionInfo.getCoordinator());
}
/** Exposed as a protected method so that it can be accessed by tests */
diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/tracker/MultiplexingRequestTracker.java b/core/src/main/java/com/datastax/oss/driver/internal/core/tracker/MultiplexingRequestTracker.java
index d4d20f3eb78..cfa212b6c48 100644
--- a/core/src/main/java/com/datastax/oss/driver/internal/core/tracker/MultiplexingRequestTracker.java
+++ b/core/src/main/java/com/datastax/oss/driver/internal/core/tracker/MultiplexingRequestTracker.java
@@ -18,13 +18,13 @@
package com.datastax.oss.driver.internal.core.tracker;
import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
+import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
import com.datastax.oss.driver.api.core.metadata.Node;
import com.datastax.oss.driver.api.core.session.Request;
import com.datastax.oss.driver.api.core.session.Session;
import com.datastax.oss.driver.api.core.tracker.RequestTracker;
import com.datastax.oss.driver.internal.core.util.Loggers;
import edu.umd.cs.findbugs.annotations.NonNull;
-import edu.umd.cs.findbugs.annotations.Nullable;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
@@ -77,56 +77,59 @@ public void register(@NonNull RequestTracker tracker) {
}
@Override
- public void onSuccess(
+ public void onRequestCreated(
+ @NonNull Request request,
+ @NonNull DriverExecutionProfile executionProfile,
+ @NonNull String requestLogPrefix) {
+ invokeTrackers(
+ tracker -> tracker.onRequestCreated(request, executionProfile, requestLogPrefix),
+ requestLogPrefix,
+ "onRequestStart");
+ }
+
+ @Override
+ public void onRequestCreatedForNode(
@NonNull Request request,
- long latencyNanos,
@NonNull DriverExecutionProfile executionProfile,
@NonNull Node node,
- @NonNull String logPrefix) {
+ @NonNull String requestLogPrefix) {
invokeTrackers(
- tracker -> tracker.onSuccess(request, latencyNanos, executionProfile, node, logPrefix),
+ tracker ->
+ tracker.onRequestCreatedForNode(request, executionProfile, node, requestLogPrefix),
+ requestLogPrefix,
+ "onRequestNodeStart");
+ }
+
+ @Override
+ public void onSuccess(
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String logPrefix) {
+ invokeTrackers(
+ tracker -> tracker.onSuccess(latencyNanos, executionInfo, logPrefix),
logPrefix,
"onSuccess");
}
@Override
public void onError(
- @NonNull Request request,
- @NonNull Throwable error,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- @Nullable Node node,
- @NonNull String logPrefix) {
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String logPrefix) {
invokeTrackers(
- tracker -> tracker.onError(request, error, latencyNanos, executionProfile, node, logPrefix),
- logPrefix,
- "onError");
+ tracker -> tracker.onError(latencyNanos, executionInfo, logPrefix), logPrefix, "onError");
}
@Override
public void onNodeSuccess(
- @NonNull Request request,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- @NonNull Node node,
- @NonNull String logPrefix) {
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String logPrefix) {
invokeTrackers(
- tracker -> tracker.onNodeSuccess(request, latencyNanos, executionProfile, node, logPrefix),
+ tracker -> tracker.onNodeSuccess(latencyNanos, executionInfo, logPrefix),
logPrefix,
"onNodeSuccess");
}
@Override
public void onNodeError(
- @NonNull Request request,
- @NonNull Throwable error,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- @NonNull Node node,
- @NonNull String logPrefix) {
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String logPrefix) {
invokeTrackers(
- tracker ->
- tracker.onNodeError(request, error, latencyNanos, executionProfile, node, logPrefix),
+ tracker -> tracker.onNodeError(latencyNanos, executionInfo, logPrefix),
logPrefix,
"onNodeError");
}
diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/tracker/NoopRequestTracker.java b/core/src/main/java/com/datastax/oss/driver/internal/core/tracker/NoopRequestTracker.java
index 09ac27e5e75..09b7dec4b47 100644
--- a/core/src/main/java/com/datastax/oss/driver/internal/core/tracker/NoopRequestTracker.java
+++ b/core/src/main/java/com/datastax/oss/driver/internal/core/tracker/NoopRequestTracker.java
@@ -17,10 +17,8 @@
*/
package com.datastax.oss.driver.internal.core.tracker;
-import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
import com.datastax.oss.driver.api.core.context.DriverContext;
-import com.datastax.oss.driver.api.core.metadata.Node;
-import com.datastax.oss.driver.api.core.session.Request;
+import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
import com.datastax.oss.driver.api.core.tracker.RequestTracker;
import edu.umd.cs.findbugs.annotations.NonNull;
import net.jcip.annotations.ThreadSafe;
@@ -38,43 +36,25 @@ public NoopRequestTracker(@SuppressWarnings("unused") DriverContext context) {
@Override
public void onSuccess(
- @NonNull Request request,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- @NonNull Node node,
- @NonNull String requestPrefix) {
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestPrefix) {
// nothing to do
}
@Override
public void onError(
- @NonNull Request request,
- @NonNull Throwable error,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- Node node,
- @NonNull String requestPrefix) {
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestPrefix) {
// nothing to do
}
@Override
public void onNodeError(
- @NonNull Request request,
- @NonNull Throwable error,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- @NonNull Node node,
- @NonNull String requestPrefix) {
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestPrefix) {
// nothing to do
}
@Override
public void onNodeSuccess(
- @NonNull Request request,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- @NonNull Node node,
- @NonNull String requestPrefix) {
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestPrefix) {
// nothing to do
}
diff --git a/core/src/main/java/com/datastax/oss/driver/internal/core/tracker/RequestLogger.java b/core/src/main/java/com/datastax/oss/driver/internal/core/tracker/RequestLogger.java
index 235ef051b40..635407049fd 100644
--- a/core/src/main/java/com/datastax/oss/driver/internal/core/tracker/RequestLogger.java
+++ b/core/src/main/java/com/datastax/oss/driver/internal/core/tracker/RequestLogger.java
@@ -20,6 +20,7 @@
import com.datastax.oss.driver.api.core.config.DefaultDriverOption;
import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
import com.datastax.oss.driver.api.core.context.DriverContext;
+import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
import com.datastax.oss.driver.api.core.metadata.Node;
import com.datastax.oss.driver.api.core.session.Request;
import com.datastax.oss.driver.api.core.session.SessionBuilder;
@@ -82,12 +83,9 @@ protected RequestLogger(RequestLogFormatter formatter) {
@Override
public void onSuccess(
- @NonNull Request request,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- @NonNull Node node,
- @NonNull String logPrefix) {
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String logPrefix) {
+ DriverExecutionProfile executionProfile = executionInfo.getExecutionProfile();
boolean successEnabled =
executionProfile.getBoolean(DefaultDriverOption.REQUEST_LOGGER_SUCCESS_ENABLED, false);
boolean slowEnabled =
@@ -121,10 +119,10 @@ public void onSuccess(
DEFAULT_REQUEST_LOGGER_MAX_VALUE_LENGTH);
logSuccess(
- request,
+ executionInfo.getRequest(),
latencyNanos,
isSlow,
- node,
+ executionInfo.getCoordinator(),
maxQueryLength,
showValues,
maxValues,
@@ -134,13 +132,9 @@ public void onSuccess(
@Override
public void onError(
- @NonNull Request request,
- @NonNull Throwable error,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- Node node,
- @NonNull String logPrefix) {
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String logPrefix) {
+ DriverExecutionProfile executionProfile = executionInfo.getExecutionProfile();
if (!executionProfile.getBoolean(DefaultDriverOption.REQUEST_LOGGER_ERROR_ENABLED, false)) {
return;
}
@@ -164,10 +158,10 @@ public void onError(
executionProfile.getBoolean(DefaultDriverOption.REQUEST_LOGGER_STACK_TRACES, false);
logError(
- request,
- error,
+ executionInfo.getRequest(),
+ executionInfo.getDriverError(),
latencyNanos,
- node,
+ executionInfo.getCoordinator(),
maxQueryLength,
showValues,
maxValues,
@@ -178,22 +172,13 @@ public void onError(
@Override
public void onNodeError(
- @NonNull Request request,
- @NonNull Throwable error,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- @NonNull Node node,
- @NonNull String logPrefix) {
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String logPrefix) {
// Nothing to do
}
@Override
public void onNodeSuccess(
- @NonNull Request request,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- @NonNull Node node,
- @NonNull String logPrefix) {
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String logPrefix) {
// Nothing to do
}
diff --git a/core/src/test/java/com/datastax/dse/driver/internal/core/cql/continuous/ContinuousCqlRequestHandlerTest.java b/core/src/test/java/com/datastax/dse/driver/internal/core/cql/continuous/ContinuousCqlRequestHandlerTest.java
index a816183e9ee..2c9ed24e0cf 100644
--- a/core/src/test/java/com/datastax/dse/driver/internal/core/cql/continuous/ContinuousCqlRequestHandlerTest.java
+++ b/core/src/test/java/com/datastax/dse/driver/internal/core/cql/continuous/ContinuousCqlRequestHandlerTest.java
@@ -23,13 +23,13 @@
import static com.datastax.dse.protocol.internal.DseProtocolConstants.RevisionType.MORE_CONTINUOUS_PAGES;
import static com.datastax.oss.driver.Assertions.assertThat;
import static com.datastax.oss.driver.Assertions.assertThatStage;
+import static com.datastax.oss.driver.internal.core.cql.CqlRequestHandlerTrackerTest.execInfoMatcher;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.anyMap;
import static org.mockito.ArgumentMatchers.argThat;
-import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.ArgumentMatchers.matches;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
@@ -46,7 +46,6 @@
import com.datastax.oss.driver.api.core.DriverTimeoutException;
import com.datastax.oss.driver.api.core.NoNodeAvailableException;
import com.datastax.oss.driver.api.core.ProtocolVersion;
-import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
import com.datastax.oss.driver.api.core.cql.Row;
import com.datastax.oss.driver.api.core.servererrors.BootstrappingException;
@@ -493,25 +492,22 @@ public void should_invoke_request_tracker(DseProtocolVersion version) {
verify(requestTracker)
.onNodeError(
- eq(UNDEFINED_IDEMPOTENCE_STATEMENT),
- any(BootstrappingException.class),
anyLong(),
- any(DriverExecutionProfile.class),
- eq(node1),
+ argThat(
+ execInfoMatcher(
+ node1,
+ UNDEFINED_IDEMPOTENCE_STATEMENT,
+ BootstrappingException.class)),
matches(LOG_PREFIX_PER_REQUEST));
verify(requestTracker)
.onNodeSuccess(
- eq(UNDEFINED_IDEMPOTENCE_STATEMENT),
anyLong(),
- any(DriverExecutionProfile.class),
- eq(node2),
+ argThat(execInfoMatcher(node2, UNDEFINED_IDEMPOTENCE_STATEMENT, null)),
matches(LOG_PREFIX_PER_REQUEST));
verify(requestTracker)
.onSuccess(
- eq(UNDEFINED_IDEMPOTENCE_STATEMENT),
anyLong(),
- any(DriverExecutionProfile.class),
- eq(node2),
+ argThat(execInfoMatcher(node2, UNDEFINED_IDEMPOTENCE_STATEMENT, null)),
matches(LOG_PREFIX_PER_REQUEST));
verifyNoMoreInteractions(requestTracker);
});
diff --git a/core/src/test/java/com/datastax/dse/driver/internal/core/graph/GraphRequestHandlerTest.java b/core/src/test/java/com/datastax/dse/driver/internal/core/graph/GraphRequestHandlerTest.java
index 9f325003610..1989477561d 100644
--- a/core/src/test/java/com/datastax/dse/driver/internal/core/graph/GraphRequestHandlerTest.java
+++ b/core/src/test/java/com/datastax/dse/driver/internal/core/graph/GraphRequestHandlerTest.java
@@ -26,9 +26,11 @@
import static com.datastax.dse.driver.internal.core.graph.GraphTestUtils.singleGraphRow;
import static com.datastax.oss.driver.api.core.type.codec.TypeCodecs.BIGINT;
import static com.datastax.oss.driver.api.core.type.codec.TypeCodecs.TEXT;
+import static com.datastax.oss.driver.internal.core.cql.CqlRequestHandlerTrackerTest.execInfoMatcher;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.ArgumentMatchers.argThat;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.ArgumentMatchers.matches;
import static org.mockito.Mockito.mock;
@@ -517,17 +519,13 @@ public void should_invoke_request_tracker_and_update_metrics(
verify(requestTracker)
.onSuccess(
- eq(graphStatement),
anyLong(),
- any(DriverExecutionProfile.class),
- eq(node),
+ argThat(execInfoMatcher(node, graphStatement, null)),
matches(LOG_PREFIX_PER_REQUEST));
verify(requestTracker)
.onNodeSuccess(
- eq(graphStatement),
anyLong(),
- any(DriverExecutionProfile.class),
- eq(node),
+ argThat(execInfoMatcher(node, graphStatement, null)),
matches(LOG_PREFIX_PER_REQUEST));
verifyNoMoreInteractions(requestTracker);
diff --git a/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandlerTest.java b/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandlerTest.java
index 1924ef5a9af..9d3a7fd249f 100644
--- a/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandlerTest.java
+++ b/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandlerTest.java
@@ -31,7 +31,6 @@
import com.datastax.oss.driver.api.core.NodeUnavailableException;
import com.datastax.oss.driver.api.core.config.DefaultDriverOption;
import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
-import com.datastax.oss.driver.api.core.cql.ColumnDefinitions;
import com.datastax.oss.driver.api.core.cql.PreparedStatement;
import com.datastax.oss.driver.api.core.cql.SimpleStatement;
import com.datastax.oss.driver.api.core.metadata.Node;
@@ -39,42 +38,15 @@
import com.datastax.oss.driver.api.core.retry.RetryVerdict;
import com.datastax.oss.driver.api.core.servererrors.OverloadedException;
import com.datastax.oss.driver.internal.core.channel.ResponseCallback;
-import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableList;
-import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableMap;
-import com.datastax.oss.protocol.internal.Message;
import com.datastax.oss.protocol.internal.ProtocolConstants;
import com.datastax.oss.protocol.internal.request.Prepare;
import com.datastax.oss.protocol.internal.response.Error;
-import com.datastax.oss.protocol.internal.response.result.ColumnSpec;
-import com.datastax.oss.protocol.internal.response.result.Prepared;
-import com.datastax.oss.protocol.internal.response.result.RawType;
-import com.datastax.oss.protocol.internal.response.result.RowsMetadata;
-import com.datastax.oss.protocol.internal.util.Bytes;
-import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletionStage;
-import org.junit.Before;
import org.junit.Test;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-public class CqlPrepareHandlerTest {
-
- private static final DefaultPrepareRequest PREPARE_REQUEST =
- new DefaultPrepareRequest("mock query");
-
- @Mock private Node node1;
- @Mock private Node node2;
- @Mock private Node node3;
-
- private final Map payload =
- ImmutableMap.of("key1", ByteBuffer.wrap(new byte[] {1, 2, 3, 4}));
-
- @Before
- public void setup() {
- MockitoAnnotations.initMocks(this);
- }
+public class CqlPrepareHandlerTest extends CqlPrepareHandlerTestBase {
@Test
public void should_prepare_on_first_node_and_reprepare_on_others() {
@@ -355,45 +327,4 @@ public void should_propagate_custom_payload_on_all_nodes() {
assertThatStage(prepareFuture).isSuccess(CqlPrepareHandlerTest::assertMatchesSimplePrepared);
}
}
-
- private static Message simplePrepared() {
- RowsMetadata variablesMetadata =
- new RowsMetadata(
- ImmutableList.of(
- new ColumnSpec(
- "ks",
- "table",
- "key",
- 0,
- RawType.PRIMITIVES.get(ProtocolConstants.DataType.VARCHAR))),
- null,
- new int[] {0},
- null);
- RowsMetadata resultMetadata =
- new RowsMetadata(
- ImmutableList.of(
- new ColumnSpec(
- "ks",
- "table",
- "message",
- 0,
- RawType.PRIMITIVES.get(ProtocolConstants.DataType.VARCHAR))),
- null,
- new int[] {},
- null);
- return new Prepared(
- Bytes.fromHexString("0xffff").array(), null, variablesMetadata, resultMetadata);
- }
-
- private static void assertMatchesSimplePrepared(PreparedStatement statement) {
- assertThat(Bytes.toHexString(statement.getId())).isEqualTo("0xffff");
-
- ColumnDefinitions variableDefinitions = statement.getVariableDefinitions();
- assertThat(variableDefinitions).hasSize(1);
- assertThat(variableDefinitions.get(0).getName().asInternal()).isEqualTo("key");
-
- ColumnDefinitions resultSetDefinitions = statement.getResultSetDefinitions();
- assertThat(resultSetDefinitions).hasSize(1);
- assertThat(resultSetDefinitions.get(0).getName().asInternal()).isEqualTo("message");
- }
}
diff --git a/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandlerTestBase.java b/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandlerTestBase.java
new file mode 100644
index 00000000000..cf824ed0a10
--- /dev/null
+++ b/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandlerTestBase.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.datastax.oss.driver.internal.core.cql;
+
+import static com.datastax.oss.driver.Assertions.assertThat;
+
+import com.datastax.oss.driver.api.core.cql.ColumnDefinitions;
+import com.datastax.oss.driver.api.core.cql.PreparedStatement;
+import com.datastax.oss.driver.api.core.metadata.Node;
+import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableList;
+import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableMap;
+import com.datastax.oss.protocol.internal.Message;
+import com.datastax.oss.protocol.internal.ProtocolConstants;
+import com.datastax.oss.protocol.internal.response.result.ColumnSpec;
+import com.datastax.oss.protocol.internal.response.result.Prepared;
+import com.datastax.oss.protocol.internal.response.result.RawType;
+import com.datastax.oss.protocol.internal.response.result.RowsMetadata;
+import com.datastax.oss.protocol.internal.util.Bytes;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import org.junit.Before;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+public abstract class CqlPrepareHandlerTestBase {
+
+ protected static final DefaultPrepareRequest PREPARE_REQUEST =
+ new DefaultPrepareRequest("mock query");
+
+ @Mock protected Node node1;
+ @Mock protected Node node2;
+ @Mock protected Node node3;
+
+ protected final Map payload =
+ ImmutableMap.of("key1", ByteBuffer.wrap(new byte[] {1, 2, 3, 4}));
+
+ @Before
+ public void setup() {
+ MockitoAnnotations.initMocks(this);
+ }
+
+ protected static Message simplePrepared() {
+ RowsMetadata variablesMetadata =
+ new RowsMetadata(
+ ImmutableList.of(
+ new ColumnSpec(
+ "ks",
+ "table",
+ "key",
+ 0,
+ RawType.PRIMITIVES.get(ProtocolConstants.DataType.VARCHAR))),
+ null,
+ new int[] {0},
+ null);
+ RowsMetadata resultMetadata =
+ new RowsMetadata(
+ ImmutableList.of(
+ new ColumnSpec(
+ "ks",
+ "table",
+ "message",
+ 0,
+ RawType.PRIMITIVES.get(ProtocolConstants.DataType.VARCHAR))),
+ null,
+ new int[] {},
+ null);
+ return new Prepared(
+ Bytes.fromHexString("0xffff").array(), null, variablesMetadata, resultMetadata);
+ }
+
+ protected static void assertMatchesSimplePrepared(PreparedStatement statement) {
+ assertThat(Bytes.toHexString(statement.getId())).isEqualTo("0xffff");
+
+ ColumnDefinitions variableDefinitions = statement.getVariableDefinitions();
+ assertThat(variableDefinitions).hasSize(1);
+ assertThat(variableDefinitions.get(0).getName().asInternal()).isEqualTo("key");
+
+ ColumnDefinitions resultSetDefinitions = statement.getResultSetDefinitions();
+ assertThat(resultSetDefinitions).hasSize(1);
+ assertThat(resultSetDefinitions.get(0).getName().asInternal()).isEqualTo("message");
+ }
+}
diff --git a/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandlerTrackerTest.java b/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandlerTrackerTest.java
new file mode 100644
index 00000000000..48833ab9534
--- /dev/null
+++ b/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlPrepareHandlerTrackerTest.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.datastax.oss.driver.internal.core.cql;
+
+import static com.datastax.oss.driver.Assertions.assertThatStage;
+import static com.datastax.oss.driver.internal.core.cql.CqlRequestHandlerTestBase.defaultFrameOf;
+import static com.datastax.oss.driver.internal.core.cql.CqlRequestHandlerTrackerTest.execInfoMatcher;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyLong;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
+import com.datastax.oss.driver.api.core.cql.PreparedStatement;
+import com.datastax.oss.driver.api.core.servererrors.BootstrappingException;
+import com.datastax.oss.driver.api.core.tracker.RequestTracker;
+import com.datastax.oss.driver.internal.core.tracker.NoopRequestTracker;
+import com.datastax.oss.protocol.internal.ProtocolConstants;
+import com.datastax.oss.protocol.internal.response.Error;
+import java.util.concurrent.CompletionStage;
+import org.junit.Test;
+
+public class CqlPrepareHandlerTrackerTest extends CqlPrepareHandlerTestBase {
+
+ @Test
+ public void should_invoke_request_tracker() {
+ try (RequestHandlerTestHarness harness =
+ RequestHandlerTestHarness.builder()
+ .withDefaultIdempotence(true)
+ .withResponse(
+ node1,
+ defaultFrameOf(
+ new Error(ProtocolConstants.ErrorCode.IS_BOOTSTRAPPING, "mock message")))
+ .withResponse(node2, defaultFrameOf(CqlPrepareHandlerTest.simplePrepared()))
+ .build()) {
+
+ RequestTracker requestTracker = mock(RequestTracker.class);
+ when(harness.getContext().getRequestTracker()).thenReturn(requestTracker);
+
+ CompletionStage resultSetFuture =
+ new CqlPrepareHandler(PREPARE_REQUEST, harness.getSession(), harness.getContext(), "test")
+ .handle();
+
+ assertThatStage(resultSetFuture)
+ .isSuccess(
+ resultSet -> {
+ verify(requestTracker)
+ .onRequestCreated(
+ eq(PREPARE_REQUEST), any(DriverExecutionProfile.class), any(String.class));
+ verify(requestTracker)
+ .onRequestCreatedForNode(
+ eq(PREPARE_REQUEST),
+ any(DriverExecutionProfile.class),
+ eq(node1),
+ any(String.class));
+ verify(requestTracker)
+ .onNodeError(
+ anyLong(),
+ argThat(
+ execInfoMatcher(node1, PREPARE_REQUEST, BootstrappingException.class)),
+ any(String.class));
+ verify(requestTracker)
+ .onRequestCreatedForNode(
+ eq(PREPARE_REQUEST),
+ any(DriverExecutionProfile.class),
+ eq(node2),
+ any(String.class));
+ verify(requestTracker)
+ .onNodeSuccess(
+ anyLong(),
+ argThat(execInfoMatcher(node2, PREPARE_REQUEST, null)),
+ any(String.class));
+ verify(requestTracker)
+ .onSuccess(
+ anyLong(),
+ argThat(execInfoMatcher(node2, PREPARE_REQUEST, null)),
+ any(String.class));
+ verifyNoMoreInteractions(requestTracker);
+ });
+ }
+ }
+
+ @Test
+ public void should_not_invoke_noop_request_tracker() {
+ try (RequestHandlerTestHarness harness =
+ RequestHandlerTestHarness.builder()
+ .withDefaultIdempotence(true)
+ .withResponse(
+ node1,
+ defaultFrameOf(
+ new Error(ProtocolConstants.ErrorCode.IS_BOOTSTRAPPING, "mock message")))
+ .withResponse(node2, defaultFrameOf(CqlPrepareHandlerTest.simplePrepared()))
+ .build()) {
+
+ RequestTracker requestTracker = spy(new NoopRequestTracker(harness.getContext()));
+ when(harness.getContext().getRequestTracker()).thenReturn(requestTracker);
+
+ CompletionStage resultSetFuture =
+ new CqlPrepareHandler(PREPARE_REQUEST, harness.getSession(), harness.getContext(), "test")
+ .handle();
+
+ assertThatStage(resultSetFuture)
+ .isSuccess(resultSet -> verifyNoMoreInteractions(requestTracker));
+ }
+ }
+}
diff --git a/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlRequestHandlerTrackerTest.java b/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlRequestHandlerTrackerTest.java
index ecc087fb8ac..e25726e8e4b 100644
--- a/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlRequestHandlerTrackerTest.java
+++ b/core/src/test/java/com/datastax/oss/driver/internal/core/cql/CqlRequestHandlerTrackerTest.java
@@ -18,10 +18,13 @@
package com.datastax.oss.driver.internal.core.cql;
import static com.datastax.oss.driver.Assertions.assertThatStage;
+import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.ArgumentMatchers.argThat;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.mockingDetails;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
@@ -29,15 +32,43 @@
import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
import com.datastax.oss.driver.api.core.cql.AsyncResultSet;
+import com.datastax.oss.driver.api.core.cql.BoundStatement;
+import com.datastax.oss.driver.api.core.cql.ColumnDefinitions;
+import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
+import com.datastax.oss.driver.api.core.cql.PreparedStatement;
+import com.datastax.oss.driver.api.core.cql.Statement;
+import com.datastax.oss.driver.api.core.metadata.Node;
import com.datastax.oss.driver.api.core.servererrors.BootstrappingException;
+import com.datastax.oss.driver.api.core.session.Request;
import com.datastax.oss.driver.api.core.tracker.RequestTracker;
+import com.datastax.oss.driver.internal.core.session.RepreparePayload;
import com.datastax.oss.driver.internal.core.tracker.NoopRequestTracker;
import com.datastax.oss.protocol.internal.ProtocolConstants;
+import com.datastax.oss.protocol.internal.request.Prepare;
import com.datastax.oss.protocol.internal.response.Error;
+import com.datastax.oss.protocol.internal.response.error.Unprepared;
+import com.datastax.oss.protocol.internal.response.result.Prepared;
+import com.datastax.oss.protocol.internal.util.Bytes;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.regex.Pattern;
import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+import org.mockito.invocation.Invocation;
public class CqlRequestHandlerTrackerTest extends CqlRequestHandlerTestBase {
+ private static final String ON_REQUEST_CREATED = "onRequestCreated";
+ private static final String ON_REQUEST_CREATED_FOR_NODE = "onRequestCreatedForNode";
+ private static final String ON_NODE_SUCCESS = "onNodeSuccess";
+ private static final String ON_NODE_ERROR = "onNodeError";
+ private static final String ON_SUCCESS = "onSuccess";
+ private static final Pattern LOG_PREFIX_PER_REQUEST = Pattern.compile("(test)\\|\\d*");
+ private static final Pattern LOG_PREFIX_WITH_EXECUTION_NUMBER =
+ Pattern.compile("(test)\\|\\d*\\|\\d*");
@Test
public void should_invoke_request_tracker() {
@@ -66,26 +97,40 @@ public void should_invoke_request_tracker() {
.isSuccess(
resultSet -> {
verify(requestTracker)
- .onNodeError(
+ .onRequestCreated(
eq(UNDEFINED_IDEMPOTENCE_STATEMENT),
- any(BootstrappingException.class),
- anyLong(),
any(DriverExecutionProfile.class),
- eq(node1),
any(String.class));
verify(requestTracker)
- .onNodeSuccess(
+ .onRequestCreatedForNode(
eq(UNDEFINED_IDEMPOTENCE_STATEMENT),
+ any(DriverExecutionProfile.class),
+ eq(node1),
+ any(String.class));
+ verify(requestTracker)
+ .onNodeError(
anyLong(),
+ argThat(
+ execInfoMatcher(
+ node1,
+ UNDEFINED_IDEMPOTENCE_STATEMENT,
+ BootstrappingException.class)),
+ any(String.class));
+ verify(requestTracker)
+ .onRequestCreatedForNode(
+ eq(UNDEFINED_IDEMPOTENCE_STATEMENT),
any(DriverExecutionProfile.class),
eq(node2),
any(String.class));
+ verify(requestTracker)
+ .onNodeSuccess(
+ anyLong(),
+ argThat(execInfoMatcher(node2, UNDEFINED_IDEMPOTENCE_STATEMENT, null)),
+ any(String.class));
verify(requestTracker)
.onSuccess(
- eq(UNDEFINED_IDEMPOTENCE_STATEMENT),
anyLong(),
- any(DriverExecutionProfile.class),
- eq(node2),
+ argThat(execInfoMatcher(node2, UNDEFINED_IDEMPOTENCE_STATEMENT, null)),
any(String.class));
verifyNoMoreInteractions(requestTracker);
});
@@ -119,4 +164,143 @@ public void should_not_invoke_noop_request_tracker() {
.isSuccess(resultSet -> verifyNoMoreInteractions(requestTracker));
}
}
+
+ @Test
+ public void should_invoke_implicit_prepare_request_tracker() {
+ ByteBuffer mockId = Bytes.fromHexString("0xffff");
+
+ PreparedStatement preparedStatement = mock(PreparedStatement.class);
+ when(preparedStatement.getId()).thenReturn(mockId);
+ ColumnDefinitions columnDefinitions = mock(ColumnDefinitions.class);
+ when(columnDefinitions.size()).thenReturn(0);
+ when(preparedStatement.getResultSetDefinitions()).thenReturn(columnDefinitions);
+ BoundStatement boundStatement = mock(BoundStatement.class);
+ when(boundStatement.getPreparedStatement()).thenReturn(preparedStatement);
+ when(boundStatement.getValues()).thenReturn(Collections.emptyList());
+ when(boundStatement.getNowInSeconds()).thenReturn(Statement.NO_NOW_IN_SECONDS);
+
+ RequestHandlerTestHarness.Builder harnessBuilder = RequestHandlerTestHarness.builder();
+ // For the first attempt that gets the UNPREPARED response
+ PoolBehavior node1Behavior = harnessBuilder.customBehavior(node1);
+ // For the second attempt that succeeds
+ harnessBuilder.withResponse(node1, defaultFrameOf(singleRow()));
+
+ try (RequestHandlerTestHarness harness = harnessBuilder.build()) {
+ RequestTracker requestTracker = mock(RequestTracker.class);
+ when(harness.getContext().getRequestTracker()).thenReturn(requestTracker);
+
+ // The handler will look for the info to reprepare in the session's cache, put it there
+ ConcurrentMap repreparePayloads = new ConcurrentHashMap<>();
+ repreparePayloads.put(
+ mockId, new RepreparePayload(mockId, "mock query", null, Collections.emptyMap()));
+ when(harness.getSession().getRepreparePayloads()).thenReturn(repreparePayloads);
+
+ CompletionStage resultSetFuture =
+ new CqlRequestHandler(
+ UNDEFINED_IDEMPOTENCE_STATEMENT,
+ harness.getSession(),
+ harness.getContext(),
+ "test")
+ .handle();
+
+ // Before we proceed, mock the PREPARE exchange that will occur as soon as we complete the
+ // first response.
+ node1Behavior.mockFollowupRequest(
+ Prepare.class, defaultFrameOf(new Prepared(Bytes.getArray(mockId), null, null, null)));
+
+ node1Behavior.setWriteSuccess();
+ node1Behavior.setResponseSuccess(
+ defaultFrameOf(new Unprepared("mock message", Bytes.getArray(mockId))));
+
+ assertThatStage(resultSetFuture)
+ .isSuccess(
+ resultSet -> {
+ List invocations =
+ (List) mockingDetails(requestTracker).getInvocations();
+ assertThat(invocations).hasSize(10);
+ // start processing CQL statement
+ checkOnCreateInvocation(
+ invocations.get(0),
+ ON_REQUEST_CREATED,
+ DefaultSimpleStatement.class,
+ LOG_PREFIX_PER_REQUEST);
+ checkOnCreateInvocation(
+ invocations.get(1),
+ ON_REQUEST_CREATED_FOR_NODE,
+ DefaultSimpleStatement.class,
+ LOG_PREFIX_WITH_EXECUTION_NUMBER);
+ checkOnEndInvocation(
+ invocations.get(2),
+ ON_NODE_ERROR,
+ DefaultSimpleStatement.class,
+ LOG_PREFIX_WITH_EXECUTION_NUMBER);
+ // implicit reprepare statement
+ checkOnCreateInvocation(
+ invocations.get(3),
+ ON_REQUEST_CREATED,
+ DefaultPrepareRequest.class,
+ LOG_PREFIX_WITH_EXECUTION_NUMBER);
+ checkOnCreateInvocation(
+ invocations.get(4),
+ ON_REQUEST_CREATED_FOR_NODE,
+ DefaultPrepareRequest.class,
+ LOG_PREFIX_WITH_EXECUTION_NUMBER);
+ checkOnEndInvocation(
+ invocations.get(5),
+ ON_NODE_SUCCESS,
+ DefaultPrepareRequest.class,
+ LOG_PREFIX_WITH_EXECUTION_NUMBER);
+ checkOnEndInvocation(
+ invocations.get(6),
+ ON_SUCCESS,
+ DefaultPrepareRequest.class,
+ LOG_PREFIX_WITH_EXECUTION_NUMBER);
+ // send new statement and process it
+ checkOnCreateInvocation(
+ invocations.get(7),
+ ON_REQUEST_CREATED_FOR_NODE,
+ DefaultSimpleStatement.class,
+ LOG_PREFIX_WITH_EXECUTION_NUMBER);
+ checkOnEndInvocation(
+ invocations.get(8),
+ ON_NODE_SUCCESS,
+ DefaultSimpleStatement.class,
+ LOG_PREFIX_WITH_EXECUTION_NUMBER);
+ checkOnEndInvocation(
+ invocations.get(9),
+ ON_SUCCESS,
+ DefaultSimpleStatement.class,
+ LOG_PREFIX_PER_REQUEST);
+ });
+ }
+ }
+
+ private void checkOnCreateInvocation(
+ Invocation invocation, String methodName, Class> firstParameter, Pattern logPrefixPattern) {
+ assertThat(invocation.getMethod().getName()).isEqualTo(methodName);
+ assertThat(invocation.getArguments()[0]).isInstanceOf(firstParameter);
+ String logPrefix = invocation.getArguments()[invocation.getArguments().length - 1].toString();
+ assertThat(logPrefix).matches(logPrefixPattern);
+ }
+
+ private void checkOnEndInvocation(
+ Invocation invocation, String methodName, Class> firstParameter, Pattern logPrefixPattern) {
+ assertThat(invocation.getMethod().getName()).isEqualTo(methodName);
+ assertThat(((ExecutionInfo) invocation.getArguments()[1]).getRequest())
+ .isInstanceOf(firstParameter);
+ String logPrefix = invocation.getArguments()[invocation.getArguments().length - 1].toString();
+ assertThat(logPrefix).matches(logPrefixPattern);
+ }
+
+ public static ArgumentMatcher execInfoMatcher(
+ Node node, Request request, Class extends Throwable> errorClass) {
+ return executionInfo ->
+ node.equals(executionInfo.getCoordinator())
+ && request.equals(executionInfo.getRequest())
+ && (errorClass != null
+ ? executionInfo.getDriverError() != null
+ && executionInfo.getDriverError().getClass().isAssignableFrom(errorClass)
+ : executionInfo.getDriverError() == null)
+ && executionInfo.getExecutionProfile() != null;
+ }
}
diff --git a/core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicyRequestTrackerTest.java b/core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicyRequestTrackerTest.java
index 757af43ef67..ebcb21faaf8 100644
--- a/core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicyRequestTrackerTest.java
+++ b/core/src/test/java/com/datastax/oss/driver/internal/core/loadbalancing/DefaultLoadBalancingPolicyRequestTrackerTest.java
@@ -23,6 +23,10 @@
import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
import com.datastax.oss.driver.api.core.session.Request;
+import com.datastax.oss.driver.internal.core.context.InternalDriverContext;
+import com.datastax.oss.driver.internal.core.cql.DefaultExecutionInfo;
+import com.datastax.oss.driver.internal.core.metadata.DefaultNode;
+import com.datastax.oss.driver.internal.core.session.DefaultSession;
import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableMap;
import com.datastax.oss.driver.shaded.guava.common.collect.ImmutableSet;
import java.util.UUID;
@@ -34,6 +38,8 @@ public class DefaultLoadBalancingPolicyRequestTrackerTest extends LoadBalancingP
@Mock Request request;
@Mock DriverExecutionProfile profile;
+ @Mock DefaultSession session;
+ @Mock InternalDriverContext driverContext;
final String logPrefix = "lbp-test-log-prefix";
private DefaultLoadBalancingPolicy policy;
@@ -65,7 +71,7 @@ public void should_record_first_response_time_on_node_success() {
nextNanoTime = 123;
// When
- policy.onNodeSuccess(request, 0, profile, node1, logPrefix);
+ policy.onNodeSuccess(0, createExecutionInfo(node1, null), logPrefix);
// Then
assertThat(policy.responseTimes)
@@ -83,7 +89,7 @@ public void should_record_second_response_time_on_node_success() {
nextNanoTime = 456;
// When
- policy.onNodeSuccess(request, 0, profile, node1, logPrefix);
+ policy.onNodeSuccess(0, createExecutionInfo(node1, null), logPrefix);
// Then
assertThat(policy.responseTimes)
@@ -107,8 +113,8 @@ public void should_record_further_response_times_on_node_success() {
nextNanoTime = 789;
// When
- policy.onNodeSuccess(request, 0, profile, node1, logPrefix);
- policy.onNodeSuccess(request, 0, profile, node2, logPrefix);
+ policy.onNodeSuccess(0, createExecutionInfo(node1, null), logPrefix);
+ policy.onNodeSuccess(0, createExecutionInfo(node2, null), logPrefix);
// Then
assertThat(policy.responseTimes)
@@ -133,7 +139,7 @@ public void should_record_first_response_time_on_node_error() {
Throwable iae = new IllegalArgumentException();
// When
- policy.onNodeError(request, iae, 0, profile, node1, logPrefix);
+ policy.onNodeError(0, createExecutionInfo(node1, iae), logPrefix);
// Then
assertThat(policy.responseTimes)
@@ -152,7 +158,7 @@ public void should_record_second_response_time_on_node_error() {
Throwable iae = new IllegalArgumentException();
// When
- policy.onNodeError(request, iae, 0, profile, node1, logPrefix);
+ policy.onNodeError(0, createExecutionInfo(node1, iae), logPrefix);
// Then
assertThat(policy.responseTimes)
@@ -177,8 +183,8 @@ public void should_record_further_response_times_on_node_error() {
Throwable iae = new IllegalArgumentException();
// When
- policy.onNodeError(request, iae, 0, profile, node1, logPrefix);
- policy.onNodeError(request, iae, 0, profile, node2, logPrefix);
+ policy.onNodeError(0, createExecutionInfo(node1, iae), logPrefix);
+ policy.onNodeError(0, createExecutionInfo(node2, iae), logPrefix);
// Then
assertThat(policy.responseTimes)
@@ -195,4 +201,10 @@ public void should_record_further_response_times_on_node_error() {
assertThat(policy.isResponseRateInsufficient(node2, nextNanoTime)).isFalse();
assertThat(policy.isResponseRateInsufficient(node3, nextNanoTime)).isFalse();
}
+
+ private DefaultExecutionInfo createExecutionInfo(DefaultNode node, Throwable error) {
+ return new DefaultExecutionInfo.Builder(
+ request, node, -1, 0, error, null, session, driverContext, profile)
+ .build();
+ }
}
diff --git a/core/src/test/java/com/datastax/oss/driver/internal/core/tracker/MultiplexingRequestTrackerTest.java b/core/src/test/java/com/datastax/oss/driver/internal/core/tracker/MultiplexingRequestTrackerTest.java
index 8dcad99b459..0f262c8bd8b 100644
--- a/core/src/test/java/com/datastax/oss/driver/internal/core/tracker/MultiplexingRequestTrackerTest.java
+++ b/core/src/test/java/com/datastax/oss/driver/internal/core/tracker/MultiplexingRequestTrackerTest.java
@@ -17,7 +17,11 @@
*/
package com.datastax.oss.driver.internal.core.tracker;
+import static com.datastax.oss.driver.internal.core.cql.CqlRequestHandlerTrackerTest.execInfoMatcher;
import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.BDDMockito.given;
import static org.mockito.BDDMockito.willThrow;
import static org.mockito.Mockito.verify;
@@ -30,8 +34,10 @@
import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
import com.datastax.oss.driver.api.core.metadata.Node;
import com.datastax.oss.driver.api.core.session.Request;
-import com.datastax.oss.driver.api.core.session.Session;
import com.datastax.oss.driver.api.core.tracker.RequestTracker;
+import com.datastax.oss.driver.internal.core.context.InternalDriverContext;
+import com.datastax.oss.driver.internal.core.cql.DefaultExecutionInfo;
+import com.datastax.oss.driver.internal.core.session.DefaultSession;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -50,7 +56,8 @@ public class MultiplexingRequestTrackerTest {
@Mock private Request request;
@Mock private DriverExecutionProfile profile;
@Mock private Node node;
- @Mock private Session session;
+ @Mock private DefaultSession session;
+ @Mock private InternalDriverContext driverContext;
@Mock private Appender appender;
@Captor private ArgumentCaptor loggingEventCaptor;
@@ -109,14 +116,16 @@ public void should_flatten_child_multiplexing_tracker_via_register() {
public void should_notify_onSuccess() {
// given
MultiplexingRequestTracker tracker = new MultiplexingRequestTracker(child1, child2);
- willThrow(new NullPointerException())
- .given(child1)
- .onSuccess(request, 123456L, profile, node, "test");
+ willThrow(new NullPointerException()).given(child1).onSuccess(eq(123456L), any(), eq("test"));
// when
- tracker.onSuccess(request, 123456L, profile, node, "test");
+ tracker.onSuccess(123456L, createExecutionInfo(), "test");
// then
- verify(child1).onSuccess(request, 123456L, profile, node, "test");
- verify(child2).onSuccess(request, 123456L, profile, node, "test");
+ verify(child1)
+ .onSuccess(
+ eq(123456L), argThat(execInfoMatcher(node, request, error.getClass())), eq("test"));
+ verify(child2)
+ .onSuccess(
+ eq(123456L), argThat(execInfoMatcher(node, request, error.getClass())), eq("test"));
verify(appender).doAppend(loggingEventCaptor.capture());
assertThat(loggingEventCaptor.getAllValues().stream().map(ILoggingEvent::getFormattedMessage))
.contains(
@@ -127,14 +136,16 @@ public void should_notify_onSuccess() {
public void should_notify_onError() {
// given
MultiplexingRequestTracker tracker = new MultiplexingRequestTracker(child1, child2);
- willThrow(new NullPointerException())
- .given(child1)
- .onError(request, error, 123456L, profile, node, "test");
+ willThrow(new NullPointerException()).given(child1).onError(eq(123456L), any(), eq("test"));
// when
- tracker.onError(request, error, 123456L, profile, node, "test");
+ tracker.onError(123456L, createExecutionInfo(), "test");
// then
- verify(child1).onError(request, error, 123456L, profile, node, "test");
- verify(child2).onError(request, error, 123456L, profile, node, "test");
+ verify(child1)
+ .onError(
+ eq(123456L), argThat(execInfoMatcher(node, request, error.getClass())), eq("test"));
+ verify(child2)
+ .onError(
+ eq(123456L), argThat(execInfoMatcher(node, request, error.getClass())), eq("test"));
verify(appender).doAppend(loggingEventCaptor.capture());
assertThat(loggingEventCaptor.getAllValues().stream().map(ILoggingEvent::getFormattedMessage))
.contains(
@@ -147,12 +158,16 @@ public void should_notify_onNodeSuccess() {
MultiplexingRequestTracker tracker = new MultiplexingRequestTracker(child1, child2);
willThrow(new NullPointerException())
.given(child1)
- .onNodeSuccess(request, 123456L, profile, node, "test");
+ .onNodeSuccess(eq(123456L), any(), eq("test"));
// when
- tracker.onNodeSuccess(request, 123456L, profile, node, "test");
+ tracker.onNodeSuccess(123456L, createExecutionInfo(), "test");
// then
- verify(child1).onNodeSuccess(request, 123456L, profile, node, "test");
- verify(child2).onNodeSuccess(request, 123456L, profile, node, "test");
+ verify(child1)
+ .onNodeSuccess(
+ eq(123456L), argThat(execInfoMatcher(node, request, error.getClass())), eq("test"));
+ verify(child2)
+ .onNodeSuccess(
+ eq(123456L), argThat(execInfoMatcher(node, request, error.getClass())), eq("test"));
verify(appender).doAppend(loggingEventCaptor.capture());
assertThat(loggingEventCaptor.getAllValues().stream().map(ILoggingEvent::getFormattedMessage))
.contains(
@@ -163,14 +178,16 @@ public void should_notify_onNodeSuccess() {
public void should_notify_onNodeError() {
// given
MultiplexingRequestTracker tracker = new MultiplexingRequestTracker(child1, child2);
- willThrow(new NullPointerException())
- .given(child1)
- .onNodeError(request, error, 123456L, profile, node, "test");
+ willThrow(new NullPointerException()).given(child1).onNodeError(eq(123456L), any(), eq("test"));
// when
- tracker.onNodeError(request, error, 123456L, profile, node, "test");
+ tracker.onNodeError(123456L, createExecutionInfo(), "test");
// then
- verify(child1).onNodeError(request, error, 123456L, profile, node, "test");
- verify(child2).onNodeError(request, error, 123456L, profile, node, "test");
+ verify(child1)
+ .onNodeError(
+ eq(123456L), argThat(execInfoMatcher(node, request, error.getClass())), eq("test"));
+ verify(child2)
+ .onNodeError(
+ eq(123456L), argThat(execInfoMatcher(node, request, error.getClass())), eq("test"));
verify(appender).doAppend(loggingEventCaptor.capture());
assertThat(loggingEventCaptor.getAllValues().stream().map(ILoggingEvent::getFormattedMessage))
.contains(
@@ -210,4 +227,10 @@ public void should_notify_close() throws Exception {
.contains(
"Unexpected error while closing request tracker child1. (NullPointerException: null)");
}
+
+ private DefaultExecutionInfo createExecutionInfo() {
+ return new DefaultExecutionInfo.Builder(
+ request, node, -1, 0, error, null, session, driverContext, profile)
+ .build();
+ }
}
diff --git a/integration-tests/src/test/java/com/datastax/oss/driver/core/cql/PagingIterableSpliteratorIT.java b/integration-tests/src/test/java/com/datastax/oss/driver/core/cql/PagingIterableSpliteratorIT.java
index 02078b683db..fa2d201323a 100644
--- a/integration-tests/src/test/java/com/datastax/oss/driver/core/cql/PagingIterableSpliteratorIT.java
+++ b/integration-tests/src/test/java/com/datastax/oss/driver/core/cql/PagingIterableSpliteratorIT.java
@@ -22,24 +22,35 @@
import com.datastax.oss.driver.api.core.CqlSession;
import com.datastax.oss.driver.api.core.config.DefaultDriverOption;
import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
+import com.datastax.oss.driver.api.core.context.DriverContext;
import com.datastax.oss.driver.api.core.cql.BatchStatement;
import com.datastax.oss.driver.api.core.cql.BatchStatementBuilder;
import com.datastax.oss.driver.api.core.cql.DefaultBatchType;
+import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
import com.datastax.oss.driver.api.core.cql.PreparedStatement;
import com.datastax.oss.driver.api.core.cql.ResultSet;
import com.datastax.oss.driver.api.core.cql.Row;
import com.datastax.oss.driver.api.core.cql.SimpleStatement;
+import com.datastax.oss.driver.api.core.metadata.Node;
+import com.datastax.oss.driver.api.core.session.Request;
+import com.datastax.oss.driver.api.core.tracker.RequestTracker;
import com.datastax.oss.driver.api.testinfra.ccm.CcmRule;
import com.datastax.oss.driver.api.testinfra.session.SessionRule;
+import com.datastax.oss.driver.api.testinfra.session.SessionUtils;
import com.datastax.oss.driver.categories.ParallelizableTests;
+import com.datastax.oss.driver.internal.core.tracker.RequestLogFormatter;
import com.datastax.oss.driver.shaded.guava.common.collect.Lists;
import com.tngtech.java.junit.dataprovider.DataProvider;
import com.tngtech.java.junit.dataprovider.DataProviderRunner;
import com.tngtech.java.junit.dataprovider.UseDataProvider;
+import edu.umd.cs.findbugs.annotations.NonNull;
import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
import java.util.List;
import java.util.Spliterator;
import java.util.stream.StreamSupport;
+import org.apache.commons.lang3.tuple.Pair;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Test;
@@ -54,7 +65,15 @@ public class PagingIterableSpliteratorIT {
private static final CcmRule CCM_RULE = CcmRule.getInstance();
- private static final SessionRule SESSION_RULE = SessionRule.builder(CCM_RULE).build();
+ private static final SessionRule SESSION_RULE =
+ SessionRule.builder(CCM_RULE)
+ .withConfigLoader(
+ SessionUtils.configLoaderBuilder()
+ .withClassList(
+ DefaultDriverOption.REQUEST_TRACKER_CLASSES,
+ Collections.singletonList(RecordingRequestTracker.class))
+ .build())
+ .build();
@ClassRule
public static final TestRule CHAIN = RuleChain.outerRule(CCM_RULE).around(SESSION_RULE);
@@ -78,6 +97,27 @@ public static void setupSchema() {
}
SESSION_RULE.session().execute(batch.setExecutionProfile(SESSION_RULE.slowProfile()).build());
}
+ RecordingRequestTracker.reset();
+ }
+
+ @Test
+ public void should_notify_request_tracker_during_pagination() throws Exception {
+ String query = "SELECT v FROM test where k0 = 0";
+ RecordingRequestTracker.query = query;
+ CqlSession session = SESSION_RULE.session();
+ ResultSet result = session.execute(SimpleStatement.newInstance(query));
+ Iterator iterator = result.iterator();
+ while (iterator.hasNext()) {
+ Row row = iterator.next();
+ assertThat(row.getInt("v")).isGreaterThanOrEqualTo(0);
+ }
+ int expectedFetches = 20_000 / 5_000 + 1; // +1 to retrieve empty page
+ assertThat(RecordingRequestTracker.startedRequests).hasSize(expectedFetches);
+ assertThat(RecordingRequestTracker.startedRequestsAtNode).hasSize(expectedFetches);
+ assertThat(RecordingRequestTracker.successfulRequestsAtNode).hasSize(expectedFetches);
+ assertThat(RecordingRequestTracker.successfulRequests).hasSize(expectedFetches);
+ assertThat(RecordingRequestTracker.errorRequestsAtNode).hasSize(0);
+ assertThat(RecordingRequestTracker.errorRequests).hasSize(0);
}
@Test
@@ -140,4 +180,100 @@ public static Iterable> pageSizes() {
arguments.add(Lists.newArrayList(19_995, true));
return arguments;
}
+
+ public static class RecordingRequestTracker implements RequestTracker {
+
+ private static volatile String query = "none";
+ private static final List startedRequests = new ArrayList<>();
+ private static final List> startedRequestsAtNode = new ArrayList<>();
+ private static final List> successfulRequestsAtNode = new ArrayList<>();
+ private static final List successfulRequests = new ArrayList<>();
+ private static final List> errorRequestsAtNode = new ArrayList<>();
+ private static final List errorRequests = new ArrayList<>();
+
+ private final RequestLogFormatter formatter;
+
+ public RecordingRequestTracker(DriverContext context) {
+ formatter = new RequestLogFormatter(context);
+ }
+
+ @Override
+ public synchronized void onRequestCreated(
+ @NonNull Request request,
+ @NonNull DriverExecutionProfile executionProfile,
+ @NonNull String requestLogPrefix) {
+ if (shouldRecord(request)) {
+ startedRequests.add(request);
+ }
+ }
+
+ @Override
+ public synchronized void onRequestCreatedForNode(
+ @NonNull Request request,
+ @NonNull DriverExecutionProfile executionProfile,
+ @NonNull Node node,
+ @NonNull String requestLogPrefix) {
+ if (shouldRecord(request)) {
+ startedRequestsAtNode.add(Pair.of(request, node));
+ }
+ }
+
+ @Override
+ public synchronized void onNodeSuccess(
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestLogPrefix) {
+ if (shouldRecord(executionInfo.getRequest())) {
+ successfulRequestsAtNode.add(
+ Pair.of(executionInfo.getRequest(), executionInfo.getCoordinator()));
+ }
+ }
+
+ @Override
+ public synchronized void onSuccess(
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestLogPrefix) {
+ if (shouldRecord(executionInfo.getRequest())) {
+ successfulRequests.add(executionInfo.getRequest());
+ }
+ }
+
+ @Override
+ public synchronized void onNodeError(
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestLogPrefix) {
+ if (shouldRecord(executionInfo.getRequest())) {
+ errorRequestsAtNode.add(
+ Pair.of(executionInfo.getRequest(), executionInfo.getCoordinator()));
+ }
+ }
+
+ @Override
+ public synchronized void onError(
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestLogPrefix) {
+ if (shouldRecord(executionInfo.getRequest())) {
+ errorRequests.add(executionInfo.getRequest());
+ }
+ }
+
+ private boolean shouldRecord(Request request) {
+ if (query == null) {
+ return true;
+ }
+ StringBuilder builder = new StringBuilder();
+ formatter.appendRequest(request, 1000, true, 1000, 1000, builder);
+ return builder.toString().contains(query);
+ }
+
+ @Override
+ public void close() throws Exception {
+ reset();
+ }
+
+ public static void reset() {
+ query = "none";
+ startedRequests.clear();
+ startedRequestsAtNode.clear();
+ successfulRequestsAtNode.clear();
+ successfulRequests.clear();
+ errorRequestsAtNode.clear();
+ errorRequests.clear();
+ }
+ }
}
diff --git a/integration-tests/src/test/java/com/datastax/oss/driver/core/tracker/RequestLoggerIT.java b/integration-tests/src/test/java/com/datastax/oss/driver/core/tracker/RequestLoggerIT.java
index ae2c46fe4a0..a53d558cb38 100644
--- a/integration-tests/src/test/java/com/datastax/oss/driver/core/tracker/RequestLoggerIT.java
+++ b/integration-tests/src/test/java/com/datastax/oss/driver/core/tracker/RequestLoggerIT.java
@@ -369,7 +369,7 @@ public void should_log_failed_nodes_on_successful_request() {
.matches(WITH_EXECUTION_PREFIX);
assertThat(events.get(1).getFormattedMessage())
.contains("Success", "[0 values]", QUERY)
- .matches(WITH_PER_REQUEST_PREFIX);
+ .matches(WITH_EXECUTION_PREFIX);
assertThat(events.get(2).getFormattedMessage())
.contains("Success", "[0 values]", QUERY)
.matches(WITH_PER_REQUEST_PREFIX);
@@ -399,7 +399,7 @@ public void should_log_successful_nodes_on_successful_request() {
List events = loggingEventCaptor.getAllValues();
assertThat(events.get(0).getFormattedMessage())
.contains("Success", "[0 values]", QUERY)
- .matches(WITH_PER_REQUEST_PREFIX);
+ .matches(WITH_EXECUTION_PREFIX);
assertThat(events.get(1).getFormattedMessage())
.contains("Success", "[0 values]", QUERY)
.matches(WITH_PER_REQUEST_PREFIX);
diff --git a/integration-tests/src/test/java/com/datastax/oss/driver/core/tracker/RequestNodeLoggerExample.java b/integration-tests/src/test/java/com/datastax/oss/driver/core/tracker/RequestNodeLoggerExample.java
index eae98339637..53393af278b 100644
--- a/integration-tests/src/test/java/com/datastax/oss/driver/core/tracker/RequestNodeLoggerExample.java
+++ b/integration-tests/src/test/java/com/datastax/oss/driver/core/tracker/RequestNodeLoggerExample.java
@@ -20,8 +20,7 @@
import com.datastax.oss.driver.api.core.config.DefaultDriverOption;
import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
import com.datastax.oss.driver.api.core.context.DriverContext;
-import com.datastax.oss.driver.api.core.metadata.Node;
-import com.datastax.oss.driver.api.core.session.Request;
+import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
import com.datastax.oss.driver.internal.core.tracker.RequestLogFormatter;
import com.datastax.oss.driver.internal.core.tracker.RequestLogger;
import edu.umd.cs.findbugs.annotations.NonNull;
@@ -34,12 +33,8 @@ public RequestNodeLoggerExample(DriverContext context) {
@Override
public void onNodeError(
- @NonNull Request request,
- @NonNull Throwable error,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- @NonNull Node node,
- @NonNull String logPrefix) {
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String logPrefix) {
+ DriverExecutionProfile executionProfile = executionInfo.getExecutionProfile();
if (!executionProfile.getBoolean(DefaultDriverOption.REQUEST_LOGGER_ERROR_ENABLED)) {
return;
}
@@ -57,10 +52,10 @@ public void onNodeError(
executionProfile.getBoolean(DefaultDriverOption.REQUEST_LOGGER_STACK_TRACES);
logError(
- request,
- error,
+ executionInfo.getRequest(),
+ executionInfo.getDriverError(),
latencyNanos,
- node,
+ executionInfo.getCoordinator(),
maxQueryLength,
showValues,
maxValues,
@@ -71,11 +66,8 @@ public void onNodeError(
@Override
public void onNodeSuccess(
- @NonNull Request request,
- long latencyNanos,
- @NonNull DriverExecutionProfile executionProfile,
- @NonNull Node node,
- @NonNull String logPrefix) {
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String logPrefix) {
+ DriverExecutionProfile executionProfile = executionInfo.getExecutionProfile();
boolean successEnabled =
executionProfile.getBoolean(DefaultDriverOption.REQUEST_LOGGER_SUCCESS_ENABLED);
boolean slowEnabled =
@@ -106,10 +98,10 @@ public void onNodeSuccess(
: 0;
logSuccess(
- request,
+ executionInfo.getRequest(),
latencyNanos,
isSlow,
- node,
+ executionInfo.getCoordinator(),
maxQueryLength,
showValues,
maxValues,
diff --git a/manual/open_telemetry/README.md b/manual/open_telemetry/README.md
new file mode 100644
index 00000000000..7ed51c71372
--- /dev/null
+++ b/manual/open_telemetry/README.md
@@ -0,0 +1,99 @@
+
+
+# OpenTelemetry
+The driver provides support for session and node level traces using OpenTelemetry.
+
+## Including OpenTelemetry instrumentation in your code
+
+You need to include the `java-driver-open-telemetry` module in your project's dependency.
+```xml
+
+ org.apache.cassandra
+ java-driver-open-telemetry
+
+```
+
+You also need to instantiate an `OtelRequestTracker` and pass it to the `CqlSessionBuilder` when building the session.
+
+```java
+CqlSession session = CqlSession.builder()
+ .addContactPoint(new InetSocketAddress("127.0.0.1", 9042))
+ .withLocalDatacenter("datacenter1")
+ .withRequestTracker(new OtelRequestTracker(initOpenTelemetry()))
+ .build();
+```
+
+The constructor of `OtelRequestTracker` needs an argument of `OpenTelemetry` instance. This instance will contain the configuration for the resource and the exporter.
+This is an example of how to initialize the `OpenTelemetry` instance with Jaeger exporter.
+
+```java
+static OpenTelemetry initOpenTelemetry() {
+
+ ManagedChannel jaegerChannel =
+ ManagedChannelBuilder.forAddress("localhost", 14250).usePlaintext().build();
+
+ JaegerGrpcSpanExporter jaegerExporter =
+ JaegerGrpcSpanExporter.builder()
+ .setChannel(jaegerChannel)
+ .setTimeout(30, TimeUnit.SECONDS)
+ .build();
+
+ Resource serviceNameResource =
+ Resource.create(Attributes.of(ResourceAttributes.SERVICE_NAME, "Demo App"));
+
+ SdkTracerProvider tracerProvider =
+ SdkTracerProvider.builder()
+ .addSpanProcessor(SimpleSpanProcessor.create(jaegerExporter))
+ .setResource(Resource.getDefault().merge(serviceNameResource))
+ .build();
+ OpenTelemetrySdk openTelemetry =
+ OpenTelemetrySdk.builder().setTracerProvider(tracerProvider).build();
+
+ return openTelemetry;
+}
+```
+
+You can also find an demo system [here](https://github.com/SiyaoIsHiding/java-driver-otel-example), which includes an Apache Cassandra database, a Spring server using the Apache Cassandra Java Driver, an HTTP client simulating browser behavior, and a Jaeger the Opentelemetry collector.
+The example demonstrates how to use OpenTelemetry to trace the queries and achieve context propagation between distributed components.
+
+## Attributes
+
+| Attribute | Description | Output Values |
+|---------------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------|
+| db.system.name | Always "cassandra" | "cassandra" |
+| db.namespace | The keyspace associated with the session. | "mykeyspace" |
+| db.operation.name | The name of the operation or command being executed. `Session_Request({RequestType})` for session level calls and Node_Request({RequestType}) for node level calls | "Node_Request(DefaultBoundStatement)" |
+| error.type | Describes a class of error the operation ended with | "NodeUnavailableException" |
+| server.port | Server port number. | 9042 |
+| cassandra.consistency.level | The consistency level of the query. Based on consistency values from [CQL](https://docs.datastax.com/en/cassandra-oss/3.0/cassandra/dml/dmlConfigConsistency.html). | "QUORUM" |
+| cassandra.coordinator.dc | The data center of the coordinating node for a query. | "datacenter1" |
+| cassandra.coordinator.id | The ID of the coordinating node for a query. | "be13faa2-8574-4d71-926d-27f16cf8a7af" |
+| cassandra.page.size | The fetch size used for paging, i.e. how many rows will be returned at once. | 5000 |
+| cassandra.query.idempotent | Whether or not the query is idempotent. | true |
+| cassandra.query.id | The query ID to correlate with logs. `{sessionId}\|{sessionRequestId}` for a session request and `{sessionId}\|{sessionRequestId}\|{nodeRequestCount}` for a node request | "s0\|229540037\|0" |
+| cassandra.speculative_execution.count | The number of times a query was speculatively executed. Not set or 0 if the query was not executed speculatively. | 0 |
+| db.operation.batch.size | The number of queries included in the case of a batch operation. | 2 |
+| db.query.text | The database query being executed. | "SELECT * FROM ingredients WHERE id=? LIMIT 1 \[id='COTO'\]" |
+| server.address | Name of the database host. | "127.0.0.1" |
+| db.operation.parameter. | In the case of a `BoundStatement`, this is a database operation parameter, with being the parameter name, and the attribute value being a string representation of the parameter value. | "someval" |
+
+## Apache Cassandra Internal Traces
+When using `OtelRequestTracker`, If the statement has `setTracing()` enabled to turn on Apache Cassandra built-in Query Trace feature,
+the driver will retrieve the Query Trace information about the events happened in Apache Cassandra internally and export them to the OpenTelemetry collector, too.
\ No newline at end of file
diff --git a/open-telemetry/pom.xml b/open-telemetry/pom.xml
new file mode 100644
index 00000000000..86af50c7be5
--- /dev/null
+++ b/open-telemetry/pom.xml
@@ -0,0 +1,143 @@
+
+
+
+ 4.0.0
+
+ org.apache.cassandra
+ java-driver-parent
+ 4.18.2-SNAPSHOT
+ ../
+
+ java-driver-open-telemetry
+ bundle
+ Apache Cassandra Java Driver - OpenTelemetry
+
+
+
+ ${project.groupId}
+ java-driver-bom
+ ${project.version}
+ pom
+ import
+
+
+
+
+
+ org.apache.cassandra
+ java-driver-core
+
+
+ com.github.stephenc.jcip
+ jcip-annotations
+ provided
+
+
+ com.github.spotbugs
+ spotbugs-annotations
+ provided
+
+
+ ch.qos.logback
+ logback-classic
+ test
+
+
+ junit
+ junit
+ test
+
+
+ io.opentelemetry
+ opentelemetry-api
+ 1.36.0
+
+
+ com.tngtech.java
+ junit-dataprovider
+ test
+
+
+ org.assertj
+ assertj-core
+ test
+
+
+ org.mockito
+ mockito-core
+ test
+
+
+ org.apache.cassandra
+ java-driver-core
+ test
+ test-jar
+
+
+
+
+
+ src/main/resources
+
+
+ ${project.basedir}/..
+
+ LICENSE
+ NOTICE_binary.txt
+ NOTICE.txt
+
+ META-INF
+
+
+
+
+ maven-jar-plugin
+
+
+
+ javadoc-jar
+ package
+
+ jar
+
+
+ javadoc
+
+ **
+
+
+
+
+
+
+ org.revapi
+ revapi-maven-plugin
+
+
+ true
+
+
+
+
+
diff --git a/open-telemetry/src/main/java/com/datastax/oss/driver/internal/core/tracker/OtelRequestTracker.java b/open-telemetry/src/main/java/com/datastax/oss/driver/internal/core/tracker/OtelRequestTracker.java
new file mode 100644
index 00000000000..de531b6c1d3
--- /dev/null
+++ b/open-telemetry/src/main/java/com/datastax/oss/driver/internal/core/tracker/OtelRequestTracker.java
@@ -0,0 +1,466 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.datastax.oss.driver.internal.core.tracker;
+
+import com.datastax.oss.driver.api.core.config.DefaultDriverOption;
+import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
+import com.datastax.oss.driver.api.core.cql.BatchStatement;
+import com.datastax.oss.driver.api.core.cql.BoundStatement;
+import com.datastax.oss.driver.api.core.cql.ColumnDefinitions;
+import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
+import com.datastax.oss.driver.api.core.cql.QueryTrace;
+import com.datastax.oss.driver.api.core.cql.Statement;
+import com.datastax.oss.driver.api.core.metadata.EndPoint;
+import com.datastax.oss.driver.api.core.metadata.Node;
+import com.datastax.oss.driver.api.core.session.Request;
+import com.datastax.oss.driver.api.core.session.Session;
+import com.datastax.oss.driver.api.core.tracker.RequestTracker;
+import com.datastax.oss.driver.api.core.type.DataType;
+import com.datastax.oss.driver.internal.core.channel.DriverChannel;
+import com.datastax.oss.driver.internal.core.context.DefaultDriverContext;
+import com.datastax.oss.driver.internal.core.cql.CqlRequestHandler;
+import com.datastax.oss.driver.internal.core.metadata.DefaultEndPoint;
+import com.datastax.oss.driver.internal.core.metadata.SniEndPoint;
+import com.datastax.oss.driver.internal.core.util.concurrent.LazyReference;
+import com.datastax.oss.driver.shaded.guava.common.util.concurrent.ThreadFactoryBuilder;
+import edu.umd.cs.findbugs.annotations.NonNull;
+import io.opentelemetry.api.OpenTelemetry;
+import io.opentelemetry.api.common.AttributeKey;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.api.trace.Tracer;
+import io.opentelemetry.context.Context;
+import java.lang.reflect.Field;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Queue;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OtelRequestTracker implements RequestTracker {
+
+ private final Map logPrefixToTracingInfoMap = new ConcurrentHashMap<>();
+
+ private final Tracer tracer;
+
+ private final Logger LOG = LoggerFactory.getLogger(OtelRequestTracker.class);
+
+ private final LazyReference threadPool;
+
+ private RequestLogFormatter formatter;
+ private DefaultDriverContext context;
+ private final Field proxyAddressField = getProxyAddressField();
+
+ /**
+ * Attributes that are "conditionally required" or "recommended" but we cannot provide: 1.
+ * db.collection.name 2. db.response.status_code
+ */
+ private static final AttributeKey DB_SYSTEM_NAME =
+ AttributeKey.stringKey("db.system.name");
+
+ private static final AttributeKey DB_NAMESPACE = AttributeKey.stringKey("db.namespace");
+ private static final AttributeKey DB_OPERATION_NAME =
+ AttributeKey.stringKey("db.operation.name");
+ private static final AttributeKey ERROR_TYPE = AttributeKey.stringKey("error.type");
+ private static final AttributeKey SERVER_PORT = AttributeKey.longKey("server.port");
+ private static final AttributeKey CASSANDRA_CONSISTENCY_LEVEL =
+ AttributeKey.stringKey("cassandra.consistency.level");
+ private static final AttributeKey CASSANDRA_COORDINATOR_DC =
+ AttributeKey.stringKey("cassandra.coordinator.dc");
+ private static final AttributeKey CASSANDRA_COORDINATOR_ID =
+ AttributeKey.stringKey("cassandra.coordinator.id");
+ private static final AttributeKey CASSANDRA_PAGE_SIZE =
+ AttributeKey.longKey("cassandra.page.size");
+ private static final AttributeKey CASSANDRA_QUERY_IDEMPOTENT =
+ AttributeKey.booleanKey("cassandra.query.idempotent");
+ private static final AttributeKey CASSANDRA_QUERY_ID =
+ AttributeKey.stringKey("cassandra.query.id");
+ private static final AttributeKey CASSANDRA_SPECULATIVE_EXECUTION_COUNT =
+ AttributeKey.longKey("cassandra.speculative_execution.count");
+ private static final AttributeKey DB_OPERATION_BATCH_SIZE =
+ AttributeKey.longKey("db.operation.batch.size");
+ private static final AttributeKey DB_QUERY_TEXT = AttributeKey.stringKey("db.query.text");
+ private static final AttributeKey SERVER_ADDRESS =
+ AttributeKey.stringKey("server.address");
+
+ public OtelRequestTracker(OpenTelemetry openTelemetry) {
+ this.tracer =
+ openTelemetry.getTracer("com.datastax.oss.driver.internal.core.tracker.OtelRequestTracker");
+ this.threadPool =
+ new LazyReference<>(
+ () ->
+ new ThreadPoolExecutor(
+ 1,
+ Math.max(Runtime.getRuntime().availableProcessors(), 1),
+ 10,
+ TimeUnit.SECONDS,
+ new ArrayBlockingQueue<>(1000),
+ new ThreadFactoryBuilder().setNameFormat("otel-thread-%d").build(),
+ new ThreadPoolExecutor.AbortPolicy()));
+ }
+
+ @Override
+ public void close() throws Exception {
+ threadPool.get().shutdown();
+ threadPool.get().awaitTermination(10, TimeUnit.SECONDS);
+ logPrefixToTracingInfoMap.clear();
+ }
+
+ @Override
+ public void onRequestCreated(
+ @NonNull Request request,
+ @NonNull DriverExecutionProfile executionProfile,
+ @NonNull String requestLogPrefix) {
+ Span parentSpan = tracer.spanBuilder("Cassandra Java Driver Session Request").startSpan();
+ TracingInfo tracingInfo = new TracingInfo(parentSpan);
+ logPrefixToTracingInfoMap.put(requestLogPrefix, tracingInfo);
+ addRequestAttributesToSpan(request, parentSpan, false);
+ LOG.debug("Request created: {}", requestLogPrefix);
+ }
+
+ @Override
+ public void onRequestCreatedForNode(
+ @NonNull Request request,
+ @NonNull DriverExecutionProfile executionProfile,
+ @NonNull Node node,
+ @NonNull String requestLogPrefix) {
+ logPrefixToTracingInfoMap.computeIfPresent(
+ nodePrefixToRequestPrefix(requestLogPrefix),
+ (k, v) -> {
+ Span parentSpan = v.parentSpan;
+ Span span =
+ tracer
+ .spanBuilder("Cassandra Java Driver Node Request")
+ .setParent(Context.current().with(parentSpan))
+ .startSpan();
+ addRequestAttributesToSpan(request, span, true);
+ v.addNodeSpan(requestLogPrefix, span);
+ return v;
+ });
+ LOG.debug("Request created for node: {}", requestLogPrefix);
+ }
+
+ @Override
+ public void onSuccess(
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestLogPrefix) {
+ logPrefixToTracingInfoMap.computeIfPresent(
+ requestLogPrefix,
+ (k, v) -> {
+ Span span = v.parentSpan;
+ span.setAttribute(CASSANDRA_QUERY_ID, requestLogPrefix);
+ span.setStatus(StatusCode.OK);
+ addRequestAttributesToSpan(executionInfo.getRequest(), span, false);
+ addExecutionInfoToSpan(executionInfo, span);
+ span.end();
+ return null;
+ });
+ }
+
+ @Override
+ public void onError(
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestLogPrefix) {
+ logPrefixToTracingInfoMap.computeIfPresent(
+ requestLogPrefix,
+ (k, v) -> {
+ Span span = v.parentSpan;
+ span.setAttribute(CASSANDRA_QUERY_ID, requestLogPrefix);
+ if (!executionInfo.getErrors().isEmpty()) {
+ span.recordException(executionInfo.getErrors().get(0).getValue());
+ }
+ span.setStatus(StatusCode.ERROR);
+ addRequestAttributesToSpan(executionInfo.getRequest(), span, false);
+ addExecutionInfoToSpan(executionInfo, span);
+ span.end();
+ return null;
+ });
+ }
+
+ @Override
+ public void onNodeSuccess(
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestLogPrefix) {
+ logPrefixToTracingInfoMap.computeIfPresent(
+ nodePrefixToRequestPrefix(requestLogPrefix),
+ (k, v) -> {
+ Span span = v.getNodeSpan(requestLogPrefix);
+ span.setAttribute(CASSANDRA_QUERY_ID, requestLogPrefix);
+ span.setStatus(StatusCode.OK);
+ addRequestAttributesToSpan(executionInfo.getRequest(), span, true);
+ addExecutionInfoToSpan(executionInfo, span);
+ span.end();
+ if (executionInfo.getTracingId() != null) {
+ threadPool
+ .get()
+ .submit(
+ () -> {
+ QueryTrace queryTrace = executionInfo.getQueryTrace();
+ addCassandraQueryTraceToSpan(span, queryTrace);
+ });
+ }
+ return v;
+ });
+ }
+
+ @Override
+ public void onNodeError(
+ long latencyNanos, @NonNull ExecutionInfo executionInfo, @NonNull String requestLogPrefix) {
+ logPrefixToTracingInfoMap.computeIfPresent(
+ nodePrefixToRequestPrefix(requestLogPrefix),
+ (k, v) -> {
+ Span span = v.getNodeSpan(requestLogPrefix);
+ span.setAttribute(CASSANDRA_QUERY_ID, requestLogPrefix);
+ if (!executionInfo.getErrors().isEmpty()) {
+ /*
+ Find the first error for this node. Because a node can appear twice in the errors list due
+ to retry policy, the error for this NodeRequest may not actually be the first error in
+ this list. In that scenario, the wrong error may be attached to this span, but this is the best we can do.
+ */
+ executionInfo
+ .getErrors()
+ .forEach(
+ entry -> {
+ if (entry
+ .getKey()
+ .getHostId()
+ .equals(executionInfo.getCoordinator().getHostId())) {
+ span.recordException(entry.getValue());
+ }
+ });
+ }
+ span.setStatus(StatusCode.ERROR);
+ addRequestAttributesToSpan(executionInfo.getRequest(), span, true);
+ addExecutionInfoToSpan(executionInfo, span);
+ span.end();
+ if (executionInfo.getTracingId() != null) {
+ threadPool
+ .get()
+ .submit(
+ () -> {
+ QueryTrace queryTrace = executionInfo.getQueryTrace();
+ addCassandraQueryTraceToSpan(span, queryTrace);
+ });
+ }
+ return v;
+ });
+ }
+
+ @Override
+ public void onSessionReady(@NonNull Session session) {
+ this.context = (DefaultDriverContext) session.getContext();
+ this.formatter = this.context.getRequestLogFormatter();
+ }
+
+ private static class TracingInfo {
+ private final Span parentSpan;
+ private final Map nodeSpans = new ConcurrentHashMap<>(); // logPrefix -> span
+
+ private TracingInfo(Span parentSpan) {
+ this.parentSpan = parentSpan;
+ }
+
+ private void addNodeSpan(String logPrefix, Span span) {
+ nodeSpans.put(logPrefix, span);
+ }
+
+ private Span getNodeSpan(String logPrefix) {
+ return nodeSpans.get(logPrefix);
+ }
+ }
+
+ private void addRequestAttributesToSpan(Request request, Span span, boolean isNodeRequest) {
+ span.setAttribute(DB_SYSTEM_NAME, "cassandra");
+ String operationName =
+ String.format(
+ "%s(%s)",
+ isNodeRequest ? "Node_Request" : "Session_Request", request.getClass().getSimpleName());
+ span.setAttribute(DB_OPERATION_NAME, operationName);
+ if (request.getKeyspace() != null)
+ span.setAttribute(DB_NAMESPACE, request.getKeyspace().asCql(true));
+
+ if (request instanceof Statement>) {
+ String consistencyLevel;
+ if (((Statement>) request).getConsistencyLevel() != null) {
+ consistencyLevel = ((Statement>) request).getConsistencyLevel().name();
+ } else {
+ consistencyLevel =
+ context
+ .getConfig()
+ .getDefaultProfile()
+ .getString(DefaultDriverOption.REQUEST_CONSISTENCY);
+ }
+ span.setAttribute(CASSANDRA_CONSISTENCY_LEVEL, consistencyLevel);
+
+ int pageSize;
+ if (((Statement>) request).getPageSize() > 0) {
+ pageSize = ((Statement>) request).getPageSize();
+ } else {
+ pageSize =
+ context.getConfig().getDefaultProfile().getInt(DefaultDriverOption.REQUEST_PAGE_SIZE);
+ }
+ span.setAttribute(CASSANDRA_PAGE_SIZE, pageSize);
+ }
+
+ span.setAttribute(DB_QUERY_TEXT, requestToString(request));
+ if (request.isIdempotent() != null)
+ span.setAttribute(CASSANDRA_QUERY_IDEMPOTENT, request.isIdempotent());
+
+ if (request instanceof BatchStatement) {
+ span.setAttribute(DB_OPERATION_BATCH_SIZE, ((BatchStatement) request).size());
+ }
+
+ if (request instanceof BoundStatement) {
+ addParametersOfBoundStatementToSpan(span, (BoundStatement) request);
+ }
+ }
+
+ private void addExecutionInfoToSpan(ExecutionInfo executionInfo, Span span) {
+ Node node = executionInfo.getCoordinator();
+ if (node != null) {
+ addServerAddressAndPortToSpan(span, node);
+ span.setAttribute(CASSANDRA_COORDINATOR_ID, node.getHostId().toString());
+ span.setAttribute(CASSANDRA_COORDINATOR_DC, node.getDatacenter());
+ }
+
+ /*
+ Find the first error for this node. Because a node can appear twice in the errors list due
+ to retry policy, the error for this NodeRequest may not actually be the first error in
+ this list. In that scenario, the wrong error may be attached to this span, but this is the best we can do.
+ */
+ executionInfo
+ .getErrors()
+ .forEach(
+ entry -> {
+ if (entry.getKey().getHostId().equals(node.getHostId())) {
+ span.setAttribute(ERROR_TYPE, entry.getValue().getClass().getSimpleName());
+ }
+ });
+
+ span.setAttribute(
+ CASSANDRA_SPECULATIVE_EXECUTION_COUNT, executionInfo.getSpeculativeExecutionCount());
+ }
+
+ private String requestToString(Request request) {
+ StringBuilder builder = new StringBuilder();
+ assert this.formatter != null;
+ this.formatter.appendQueryString(
+ request, RequestLogger.DEFAULT_REQUEST_LOGGER_MAX_QUERY_LENGTH, builder);
+ this.formatter.appendValues(
+ request,
+ RequestLogger.DEFAULT_REQUEST_LOGGER_MAX_VALUES,
+ RequestLogger.DEFAULT_REQUEST_LOGGER_MAX_VALUE_LENGTH,
+ true,
+ builder);
+ return builder.toString();
+ }
+
+ private void addParametersOfBoundStatementToSpan(Span span, BoundStatement statement) {
+ ColumnDefinitions definitions = statement.getPreparedStatement().getVariableDefinitions();
+ List values = statement.getValues();
+ assert definitions.size() == values.size();
+ for (int i = 0; i < definitions.size(); i++) {
+ String key = "db.operation.parameter." + definitions.get(i).getName().asCql(true);
+ StringBuilder valueBuilder = new StringBuilder();
+ if (!statement.isSet(i)) {
+ valueBuilder.append("");
+ } else {
+ ByteBuffer value = values.get(i);
+ DataType type = definitions.get(i).getType();
+ this.formatter.appendValue(
+ value, type, RequestLogger.DEFAULT_REQUEST_LOGGER_MAX_VALUE_LENGTH, valueBuilder);
+ }
+ span.setAttribute(key, valueBuilder.toString());
+ }
+ }
+
+ private void addServerAddressAndPortToSpan(Span span, Node coordinator) {
+ EndPoint endPoint = coordinator.getEndPoint();
+ if (endPoint instanceof DefaultEndPoint) {
+ InetSocketAddress address = ((DefaultEndPoint) endPoint).resolve();
+ span.setAttribute(SERVER_ADDRESS, address.getHostString());
+ span.setAttribute(SERVER_PORT, address.getPort());
+ } else if (endPoint instanceof SniEndPoint && proxyAddressField != null) {
+ SniEndPoint sniEndPoint = (SniEndPoint) endPoint;
+ Object object = null;
+ try {
+ object = proxyAddressField.get(sniEndPoint);
+ } catch (Exception e) {
+ this.LOG.trace(
+ "Error when accessing the private field proxyAddress of SniEndPoint using reflection.");
+ }
+ if (object instanceof InetSocketAddress) {
+ InetSocketAddress address = (InetSocketAddress) object;
+ span.setAttribute(SERVER_ADDRESS, address.getHostString());
+ span.setAttribute(SERVER_PORT, address.getPort());
+ }
+ }
+ }
+
+ /**
+ * This depends on the implementation of {@link
+ * CqlRequestHandler.NodeResponseCallback#NodeResponseCallback(Statement, Node, Queue,
+ * DriverChannel, int, int, boolean, String) NodeResponseCallback}
+ *
+ * @param nodePrefix s0|1716164115|0
+ * @return the request prefix, like s0|1716164115
+ */
+ private static String nodePrefixToRequestPrefix(String nodePrefix) {
+ int lastSeparatorIndex = nodePrefix.lastIndexOf("|");
+ return nodePrefix.substring(0, lastSeparatorIndex);
+ }
+
+ @Nullable
+ private Field getProxyAddressField() {
+ try {
+ Field field = SniEndPoint.class.getDeclaredField("proxyAddress");
+ field.setAccessible(true);
+ return field;
+ } catch (Exception e) {
+ return null;
+ }
+ }
+
+ private void addCassandraQueryTraceToSpan(Span parentSpan, QueryTrace queryTrace) {
+ Span span =
+ this.tracer
+ .spanBuilder("Cassandra Internal")
+ .setStartTimestamp(Instant.ofEpochMilli(queryTrace.getStartedAt()))
+ .setParent(Context.current().with(parentSpan))
+ .startSpan();
+ queryTrace
+ .getEvents()
+ .forEach(
+ event -> {
+ span.addEvent(
+ Objects.requireNonNull(event.getActivity()),
+ // Why are these timestamps later than the span's end time?
+ Instant.ofEpochMilli(event.getTimestamp()));
+ });
+
+ span.end(
+ Instant.ofEpochMilli(queryTrace.getStartedAt() + queryTrace.getDurationMicros() / 1000));
+ }
+}
diff --git a/open-telemetry/src/test/java/com/datastax/oss/driver/internal/core/tracker/OtelRequestTrackerTest.java b/open-telemetry/src/test/java/com/datastax/oss/driver/internal/core/tracker/OtelRequestTrackerTest.java
new file mode 100644
index 00000000000..70036827074
--- /dev/null
+++ b/open-telemetry/src/test/java/com/datastax/oss/driver/internal/core/tracker/OtelRequestTrackerTest.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.datastax.oss.driver.internal.core.tracker;
+
+import static org.mockito.BDDMockito.given;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import com.datastax.oss.driver.api.core.CqlSession;
+import com.datastax.oss.driver.api.core.config.DriverConfig;
+import com.datastax.oss.driver.api.core.config.DriverExecutionProfile;
+import com.datastax.oss.driver.api.core.cql.ExecutionInfo;
+import com.datastax.oss.driver.api.core.cql.SimpleStatement;
+import com.datastax.oss.driver.api.core.metadata.Node;
+import com.datastax.oss.driver.internal.core.config.typesafe.DefaultDriverConfigLoader;
+import com.datastax.oss.driver.internal.core.config.typesafe.TypesafeDriverConfig;
+import com.datastax.oss.driver.internal.core.context.DefaultDriverContext;
+import io.opentelemetry.api.OpenTelemetry;
+import io.opentelemetry.api.common.AttributeKey;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.api.trace.SpanBuilder;
+import io.opentelemetry.api.trace.Tracer;
+import io.opentelemetry.context.Context;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.Strict.class)
+public class OtelRequestTrackerTest {
+ @Mock private OpenTelemetry openTelemetry;
+ @Mock private Tracer tracer;
+ @Mock private DriverExecutionProfile profile;
+ @Mock private SpanBuilder sessionSpanBuilder;
+ @Mock private SpanBuilder nodeSpanBuilder;
+ @Mock private Span sessionSpan;
+ @Mock private Span nodeSpan;
+ @Mock private CqlSession session;
+ @Mock private DefaultDriverContext context;
+ private DriverConfig config =
+ new TypesafeDriverConfig(DefaultDriverConfigLoader.DEFAULT_CONFIG_SUPPLIER.get());
+ private RequestLogFormatter requestLogFormatter = new RequestLogFormatter(context);
+ private String sessionLogPrefix = "s0|229540037";
+ private String nodeLogPrefix = "s0|229540037|0";
+ @Mock private Node node1;
+ @Mock private ExecutionInfo executionInfo;
+ private SimpleStatement statement = SimpleStatement.newInstance("SELECT * FROM test");
+
+ @Before
+ public void setup() {
+ given(
+ openTelemetry.getTracer(
+ "com.datastax.oss.driver.internal.core.tracker.OtelRequestTracker"))
+ .willReturn(tracer);
+ given(tracer.spanBuilder("Cassandra Java Driver Session Request"))
+ .willReturn(sessionSpanBuilder);
+ given(tracer.spanBuilder("Cassandra Java Driver Node Request")).willReturn(nodeSpanBuilder);
+ given(sessionSpanBuilder.startSpan()).willReturn(sessionSpan);
+ given(nodeSpanBuilder.startSpan()).willReturn(nodeSpan);
+ given(nodeSpanBuilder.setParent(Context.current().with(sessionSpan)))
+ .willReturn(nodeSpanBuilder);
+ given(session.getContext()).willReturn(context);
+ given(context.getRequestLogFormatter()).willReturn(requestLogFormatter);
+ given(context.getConfig()).willReturn(config);
+ given(executionInfo.getRequest()).willReturn(statement);
+ }
+
+ @Test
+ public void should_send_trace_on_request_created() {
+ // Given
+ OtelRequestTracker tracker = new OtelRequestTracker(openTelemetry);
+ tracker.onSessionReady(session);
+ // When
+ tracker.onRequestCreated(statement, profile, sessionLogPrefix);
+ // Then
+ verify(tracer).spanBuilder("Cassandra Java Driver Session Request");
+ verify(sessionSpan).setAttribute(AttributeKey.stringKey("db.query.text"), "SELECT * FROM test");
+
+ // when
+ tracker.onRequestCreatedForNode(statement, profile, node1, nodeLogPrefix);
+ // Then
+ verify(tracer).spanBuilder("Cassandra Java Driver Node Request");
+ verify(nodeSpan).setAttribute(AttributeKey.stringKey("db.query.text"), "SELECT * FROM test");
+ }
+
+ @Test
+ public void should_send_trace_on_success() {
+ // Given
+ OtelRequestTracker tracker = new OtelRequestTracker(openTelemetry);
+ tracker.onSessionReady(session);
+ tracker.onRequestCreated(statement, profile, sessionLogPrefix);
+ tracker.onRequestCreatedForNode(statement, profile, node1, nodeLogPrefix);
+ // When
+ tracker.onNodeSuccess(1L, executionInfo, nodeLogPrefix);
+ // Then
+ verify(nodeSpan).end();
+ verify(nodeSpan, times(2))
+ .setAttribute(
+ AttributeKey.stringKey("db.operation.name"), "Node_Request(DefaultSimpleStatement)");
+ // When
+ tracker.onSuccess(1L, executionInfo, sessionLogPrefix);
+ // Then
+ verify(sessionSpan).end();
+ verify(sessionSpan, times(2))
+ .setAttribute(
+ AttributeKey.stringKey("db.operation.name"), "Session_Request(DefaultSimpleStatement)");
+ }
+
+ @Test
+ public void should_send_trace_on_error() {
+ // Given
+ OtelRequestTracker tracker = new OtelRequestTracker(openTelemetry);
+ tracker.onSessionReady(session);
+ tracker.onRequestCreated(statement, profile, sessionLogPrefix);
+ tracker.onRequestCreatedForNode(statement, profile, node1, nodeLogPrefix);
+ // When
+ tracker.onNodeError(1L, executionInfo, nodeLogPrefix);
+ // Then
+ verify(nodeSpan).end();
+ // When
+ tracker.onError(1L, executionInfo, sessionLogPrefix);
+ // Then
+ verify(sessionSpan).end();
+ }
+}
diff --git a/pom.xml b/pom.xml
index c61e6485fd3..1e3a911cd3b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -40,6 +40,7 @@
mapper-processor
metrics/micrometer
metrics/microprofile
+ open-telemetry
guava-shaded
test-infra
integration-tests