This is an automated email from the ASF dual-hosted git repository.

absurdfarce pushed a commit to branch 4.x
in repository https://gitbox.apache.org/repos/asf/cassandra-java-driver.git


The following commit(s) were added to refs/heads/4.x by this push:
     new 6d7b1950a Limit calls to Conversions.resolveExecutionProfile
6d7b1950a is described below

commit 6d7b1950a5430a73ff0662056b9e9a7e2b3701a9
Author: Benoit Tellier <btell...@linagora.com>
AuthorDate: Sun Jan 22 13:58:19 2023 +0700

    Limit calls to Conversions.resolveExecutionProfile
    
    Those repeated calls account for a non-negligible portion of my application
    CPU (0.6%) and can definitly be a final field so that it gets resolved only
    once per CqlRequestHandler.
---
 .../continuous/ContinuousRequestHandlerBase.java   | 15 +++---
 .../internal/core/graph/GraphRequestHandler.java   | 15 +++---
 .../oss/driver/internal/core/cql/Conversions.java  | 38 +++++++++++++--
 .../internal/core/cql/CqlPrepareHandler.java       | 11 +++--
 .../internal/core/cql/CqlRequestHandler.java       | 54 ++++++++--------------
 5 files changed, 75 insertions(+), 58 deletions(-)

diff --git 
a/core/src/main/java/com/datastax/dse/driver/internal/core/cql/continuous/ContinuousRequestHandlerBase.java
 
b/core/src/main/java/com/datastax/dse/driver/internal/core/cql/continuous/ContinuousRequestHandlerBase.java
index 44df3b3a0..9a7be3447 100644
--- 
a/core/src/main/java/com/datastax/dse/driver/internal/core/cql/continuous/ContinuousRequestHandlerBase.java
+++ 
b/core/src/main/java/com/datastax/dse/driver/internal/core/cql/continuous/ContinuousRequestHandlerBase.java
@@ -648,12 +648,13 @@ public abstract class 
ContinuousRequestHandlerBase<StatementT extends Request, R
         }
       } else {
         LOG.trace("[{}] Request sent on {}", logPrefix, channel);
-        if (scheduleSpeculativeExecution && 
Conversions.resolveIdempotence(statement, context)) {
+        if (scheduleSpeculativeExecution
+            && Conversions.resolveIdempotence(statement, executionProfile)) {
           int nextExecution = executionIndex + 1;
           // Note that `node` is the first node of the execution, it might not 
be the "slow" one
           // if there were retries, but in practice retries are rare.
           long nextDelay =
-              Conversions.resolveSpeculativeExecutionPolicy(statement, context)
+              Conversions.resolveSpeculativeExecutionPolicy(context, 
executionProfile)
                   .nextExecution(node, keyspace, statement, nextExecution);
           if (nextDelay >= 0) {
             scheduleSpeculativeExecution(nextExecution, nextDelay);
@@ -787,12 +788,12 @@ public abstract class 
ContinuousRequestHandlerBase<StatementT extends Request, R
       cancelTimeout(pageTimeout);
       LOG.trace(String.format("[%s] Request failure", logPrefix), error);
       RetryVerdict verdict;
-      if (!Conversions.resolveIdempotence(statement, context)
+      if (!Conversions.resolveIdempotence(statement, executionProfile)
           || error instanceof FrameTooLongException) {
         verdict = RetryVerdict.RETHROW;
       } else {
         try {
-          RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(statement, 
context);
+          RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(context, 
executionProfile);
           verdict = retryPolicy.onRequestAbortedVerdict(statement, error, 
retryCount);
         } catch (Throwable cause) {
           abort(
@@ -945,7 +946,7 @@ public abstract class 
ContinuousRequestHandlerBase<StatementT extends Request, R
       assert lock.isHeldByCurrentThread();
       NodeMetricUpdater metricUpdater = ((DefaultNode) 
node).getMetricUpdater();
       RetryVerdict verdict;
-      RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(statement, 
context);
+      RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(context, 
executionProfile);
       if (error instanceof ReadTimeoutException) {
         ReadTimeoutException readTimeout = (ReadTimeoutException) error;
         verdict =
@@ -964,7 +965,7 @@ public abstract class 
ContinuousRequestHandlerBase<StatementT extends Request, R
             DefaultNodeMetric.IGNORES_ON_READ_TIMEOUT);
       } else if (error instanceof WriteTimeoutException) {
         WriteTimeoutException writeTimeout = (WriteTimeoutException) error;
-        if (Conversions.resolveIdempotence(statement, context)) {
+        if (Conversions.resolveIdempotence(statement, executionProfile)) {
           verdict =
               retryPolicy.onWriteTimeoutVerdict(
                   statement,
@@ -999,7 +1000,7 @@ public abstract class 
ContinuousRequestHandlerBase<StatementT extends Request, R
             DefaultNodeMetric.IGNORES_ON_UNAVAILABLE);
       } else {
         verdict =
-            Conversions.resolveIdempotence(statement, context)
+            Conversions.resolveIdempotence(statement, executionProfile)
                 ? retryPolicy.onErrorResponseVerdict(statement, error, 
retryCount)
                 : RetryVerdict.RETHROW;
         updateErrorMetrics(
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 c22984588..702da69b8 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
@@ -557,12 +557,13 @@ public class GraphRequestHandler implements Throttled {
           cancel();
         } else {
           inFlightCallbacks.add(this);
-          if (scheduleNextExecution && 
Conversions.resolveIdempotence(statement, context)) {
+          if (scheduleNextExecution
+              && Conversions.resolveIdempotence(statement, executionProfile)) {
             int nextExecution = execution + 1;
             long nextDelay;
             try {
               nextDelay =
-                  Conversions.resolveSpeculativeExecutionPolicy(statement, 
context)
+                  Conversions.resolveSpeculativeExecutionPolicy(context, 
executionProfile)
                       .nextExecution(node, null, statement, nextExecution);
             } catch (Throwable cause) {
               // This is a bug in the policy, but not fatal since we have at 
least one other
@@ -678,7 +679,7 @@ public class GraphRequestHandler implements Throttled {
         trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
         setFinalError(statement, error, node, execution);
       } else {
-        RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(statement, 
context);
+        RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(context, 
executionProfile);
         RetryVerdict verdict;
         if (error instanceof ReadTimeoutException) {
           ReadTimeoutException readTimeout = (ReadTimeoutException) error;
@@ -699,7 +700,7 @@ public class GraphRequestHandler implements Throttled {
         } else if (error instanceof WriteTimeoutException) {
           WriteTimeoutException writeTimeout = (WriteTimeoutException) error;
           verdict =
-              Conversions.resolveIdempotence(statement, context)
+              Conversions.resolveIdempotence(statement, executionProfile)
                   ? retryPolicy.onWriteTimeoutVerdict(
                       statement,
                       writeTimeout.getConsistencyLevel(),
@@ -731,7 +732,7 @@ public class GraphRequestHandler implements Throttled {
               DefaultNodeMetric.IGNORES_ON_UNAVAILABLE);
         } else {
           verdict =
-              Conversions.resolveIdempotence(statement, context)
+              Conversions.resolveIdempotence(statement, executionProfile)
                   ? retryPolicy.onErrorResponseVerdict(statement, error, 
retryCount)
                   : RetryVerdict.RETHROW;
           updateErrorMetrics(
@@ -810,12 +811,12 @@ public class GraphRequestHandler implements Throttled {
       }
       LOG.trace("[{}] Request failure, processing: {}", logPrefix, error);
       RetryVerdict verdict;
-      if (!Conversions.resolveIdempotence(statement, context)
+      if (!Conversions.resolveIdempotence(statement, executionProfile)
           || error instanceof FrameTooLongException) {
         verdict = RetryVerdict.RETHROW;
       } else {
         try {
-          RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(statement, 
context);
+          RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(context, 
executionProfile);
           verdict = retryPolicy.onRequestAbortedVerdict(statement, error, 
retryCount);
         } catch (Throwable cause) {
           setFinalError(
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 529664c66..ff9384b3e 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
@@ -535,29 +535,59 @@ public class Conversions {
     }
   }
 
+  /** Use {@link #resolveIdempotence(Request, DriverExecutionProfile)} 
instead. */
+  @Deprecated
   public static boolean resolveIdempotence(Request request, 
InternalDriverContext context) {
+    return resolveIdempotence(request, resolveExecutionProfile(request, 
context));
+  }
+
+  public static boolean resolveIdempotence(
+      Request request, DriverExecutionProfile executionProfile) {
     Boolean requestIsIdempotent = request.isIdempotent();
-    DriverExecutionProfile executionProfile = resolveExecutionProfile(request, 
context);
     return (requestIsIdempotent == null)
         ? 
executionProfile.getBoolean(DefaultDriverOption.REQUEST_DEFAULT_IDEMPOTENCE)
         : requestIsIdempotent;
   }
 
+  /** Use {@link #resolveRequestTimeout(Request, DriverExecutionProfile)} 
instead. */
+  @Deprecated
   public static Duration resolveRequestTimeout(Request request, 
InternalDriverContext context) {
-    DriverExecutionProfile executionProfile = resolveExecutionProfile(request, 
context);
-    return request.getTimeout() != null
-        ? request.getTimeout()
+    return resolveRequestTimeout(request, resolveExecutionProfile(request, 
context));
+  }
+
+  public static Duration resolveRequestTimeout(
+      Request request, DriverExecutionProfile executionProfile) {
+    Duration timeout = request.getTimeout();
+    return timeout != null
+        ? timeout
         : executionProfile.getDuration(DefaultDriverOption.REQUEST_TIMEOUT);
   }
 
+  /** Use {@link #resolveRetryPolicy(InternalDriverContext, 
DriverExecutionProfile)} instead. */
+  @Deprecated
   public static RetryPolicy resolveRetryPolicy(Request request, 
InternalDriverContext context) {
     DriverExecutionProfile executionProfile = resolveExecutionProfile(request, 
context);
     return context.getRetryPolicy(executionProfile.getName());
   }
 
+  public static RetryPolicy resolveRetryPolicy(
+      InternalDriverContext context, DriverExecutionProfile executionProfile) {
+    return context.getRetryPolicy(executionProfile.getName());
+  }
+
+  /**
+   * Use {@link #resolveSpeculativeExecutionPolicy(InternalDriverContext, 
DriverExecutionProfile)}
+   * instead.
+   */
+  @Deprecated
   public static SpeculativeExecutionPolicy resolveSpeculativeExecutionPolicy(
       Request request, InternalDriverContext context) {
     DriverExecutionProfile executionProfile = resolveExecutionProfile(request, 
context);
     return context.getSpeculativeExecutionPolicy(executionProfile.getName());
   }
+
+  public static SpeculativeExecutionPolicy resolveSpeculativeExecutionPolicy(
+      InternalDriverContext context, DriverExecutionProfile executionProfile) {
+    return context.getSpeculativeExecutionPolicy(executionProfile.getName());
+  }
 }
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 6faa8eee5..8fe1adb20 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
@@ -92,6 +92,7 @@ public class CqlPrepareHandler implements Throttled {
   private final Timeout scheduledTimeout;
   private final RequestThrottler throttler;
   private final Boolean prepareOnAllNodes;
+  private final DriverExecutionProfile executionProfile;
   private volatile InitialPrepareCallback initialCallback;
 
   // The errors on the nodes that were already tried (lazily initialized on 
the first error).
@@ -111,7 +112,7 @@ public class CqlPrepareHandler implements Throttled {
     this.initialRequest = request;
     this.session = session;
     this.context = context;
-    DriverExecutionProfile executionProfile = 
Conversions.resolveExecutionProfile(request, context);
+    executionProfile = Conversions.resolveExecutionProfile(request, context);
     this.queryPlan =
         context
             .getLoadBalancingPolicyWrapper()
@@ -131,7 +132,7 @@ public class CqlPrepareHandler implements Throttled {
         });
     this.timer = context.getNettyOptions().getTimer();
 
-    Duration timeout = Conversions.resolveRequestTimeout(request, context);
+    Duration timeout = Conversions.resolveRequestTimeout(request, 
executionProfile);
     this.scheduledTimeout = scheduleTimeout(timeout);
     this.prepareOnAllNodes = 
executionProfile.getBoolean(DefaultDriverOption.PREPARE_ON_ALL_NODES);
 
@@ -292,7 +293,7 @@ public class CqlPrepareHandler implements Throttled {
               false,
               toPrepareMessage(request),
               request.getCustomPayload(),
-              Conversions.resolveRequestTimeout(request, context),
+              Conversions.resolveRequestTimeout(request, executionProfile),
               throttler,
               session.getMetricUpdater(),
               logPrefix);
@@ -419,7 +420,7 @@ public class CqlPrepareHandler implements Throttled {
       } else {
         // Because prepare requests are known to always be idempotent, we call 
the retry policy
         // directly, without checking the flag.
-        RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(request, 
context);
+        RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(context, 
executionProfile);
         RetryVerdict verdict = retryPolicy.onErrorResponseVerdict(request, 
error, retryCount);
         processRetryVerdict(verdict, error);
       }
@@ -457,7 +458,7 @@ public class CqlPrepareHandler implements Throttled {
       LOG.trace("[{}] Request failure, processing: {}", logPrefix, 
error.toString());
       RetryVerdict verdict;
       try {
-        RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(request, 
context);
+        RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(context, 
executionProfile);
         verdict = retryPolicy.onRequestAbortedVerdict(request, error, 
retryCount);
       } catch (Throwable cause) {
         setFinalError(
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 e7e334d57..a1c6b0e54 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
@@ -126,6 +126,7 @@ public class CqlRequestHandler implements Throttled {
   private final RequestThrottler throttler;
   private final RequestTracker requestTracker;
   private final SessionMetricUpdater sessionMetricUpdater;
+  private final DriverExecutionProfile executionProfile;
 
   // The errors on the nodes that were already tried (lazily initialized on 
the first error).
   // We don't use a map because nodes can appear multiple times.
@@ -167,7 +168,8 @@ public class CqlRequestHandler implements Throttled {
     this.sessionMetricUpdater = session.getMetricUpdater();
 
     this.timer = context.getNettyOptions().getTimer();
-    Duration timeout = Conversions.resolveRequestTimeout(statement, context);
+    this.executionProfile = 
Conversions.resolveExecutionProfile(initialStatement, context);
+    Duration timeout = Conversions.resolveRequestTimeout(statement, 
executionProfile);
     this.scheduledTimeout = scheduleTimeout(timeout);
 
     this.throttler = context.getRequestThrottler();
@@ -176,8 +178,6 @@ public class CqlRequestHandler implements Throttled {
 
   @Override
   public void onThrottleReady(boolean wasDelayed) {
-    DriverExecutionProfile executionProfile =
-        Conversions.resolveExecutionProfile(initialStatement, context);
     if (wasDelayed
         // avoid call to nanoTime() if metric is disabled:
         && sessionMetricUpdater.isEnabled(
@@ -276,8 +276,6 @@ public class CqlRequestHandler implements Throttled {
               retryCount,
               scheduleNextExecution,
               logPrefix);
-      DriverExecutionProfile executionProfile =
-          Conversions.resolveExecutionProfile(statement, context);
       Message message = Conversions.toMessage(statement, executionProfile, 
context);
       channel
           .write(message, statement.isTracing(), statement.getCustomPayload(), 
nodeResponseCallback)
@@ -336,37 +334,28 @@ public class CqlRequestHandler implements Throttled {
           totalLatencyNanos = completionTimeNanos - startTimeNanos;
           long nodeLatencyNanos = completionTimeNanos - 
callback.nodeStartTimeNanos;
           requestTracker.onNodeSuccess(
-              callback.statement,
-              nodeLatencyNanos,
-              callback.executionProfile,
-              callback.node,
-              logPrefix);
+              callback.statement, nodeLatencyNanos, executionProfile, 
callback.node, logPrefix);
           requestTracker.onSuccess(
-              callback.statement,
-              totalLatencyNanos,
-              callback.executionProfile,
-              callback.node,
-              logPrefix);
+              callback.statement, totalLatencyNanos, executionProfile, 
callback.node, logPrefix);
         }
         if (sessionMetricUpdater.isEnabled(
-            DefaultSessionMetric.CQL_REQUESTS, 
callback.executionProfile.getName())) {
+            DefaultSessionMetric.CQL_REQUESTS, executionProfile.getName())) {
           if (completionTimeNanos == NANOTIME_NOT_MEASURED_YET) {
             completionTimeNanos = System.nanoTime();
             totalLatencyNanos = completionTimeNanos - startTimeNanos;
           }
           sessionMetricUpdater.updateTimer(
               DefaultSessionMetric.CQL_REQUESTS,
-              callback.executionProfile.getName(),
+              executionProfile.getName(),
               totalLatencyNanos,
               TimeUnit.NANOSECONDS);
         }
       }
       // log the warnings if they have NOT been disabled
       if (!executionInfo.getWarnings().isEmpty()
-          && 
callback.executionProfile.getBoolean(DefaultDriverOption.REQUEST_LOG_WARNINGS)
+          && 
executionProfile.getBoolean(DefaultDriverOption.REQUEST_LOG_WARNINGS)
           && LOG.isWarnEnabled()) {
-        logServerWarnings(
-            callback.statement, callback.executionProfile, 
executionInfo.getWarnings());
+        logServerWarnings(callback.statement, executionProfile, 
executionInfo.getWarnings());
       }
     } catch (Throwable error) {
       setFinalError(callback.statement, error, callback.node, -1);
@@ -418,21 +407,17 @@ public class CqlRequestHandler implements Throttled {
         schemaInAgreement,
         session,
         context,
-        callback.executionProfile);
+        executionProfile);
   }
 
   @Override
   public void onThrottleFailure(@NonNull RequestThrottlingException error) {
-    DriverExecutionProfile executionProfile =
-        Conversions.resolveExecutionProfile(initialStatement, context);
     sessionMetricUpdater.incrementCounter(
         DefaultSessionMetric.THROTTLING_ERRORS, executionProfile.getName());
     setFinalError(initialStatement, error, null, -1);
   }
 
   private void setFinalError(Statement<?> statement, Throwable error, Node 
node, int execution) {
-    DriverExecutionProfile executionProfile =
-        Conversions.resolveExecutionProfile(statement, context);
     if (error instanceof DriverException) {
       ((DriverException) error)
           .setExecutionInfo(
@@ -475,7 +460,6 @@ public class CqlRequestHandler implements Throttled {
 
     private final long nodeStartTimeNanos = System.nanoTime();
     private final Statement<?> statement;
-    private final DriverExecutionProfile executionProfile;
     private final Node node;
     private final Queue<Node> queryPlan;
     private final DriverChannel channel;
@@ -505,7 +489,6 @@ public class CqlRequestHandler implements Throttled {
       this.retryCount = retryCount;
       this.scheduleNextExecution = scheduleNextExecution;
       this.logPrefix = logPrefix + "|" + execution;
-      this.executionProfile = Conversions.resolveExecutionProfile(statement, 
context);
     }
 
     // this gets invoked once the write completes.
@@ -544,12 +527,13 @@ public class CqlRequestHandler implements Throttled {
           cancel();
         } else {
           inFlightCallbacks.add(this);
-          if (scheduleNextExecution && 
Conversions.resolveIdempotence(statement, context)) {
+          if (scheduleNextExecution
+              && Conversions.resolveIdempotence(statement, executionProfile)) {
             int nextExecution = execution + 1;
             long nextDelay;
             try {
               nextDelay =
-                  Conversions.resolveSpeculativeExecutionPolicy(statement, 
context)
+                  Conversions.resolveSpeculativeExecutionPolicy(context, 
executionProfile)
                       .nextExecution(node, keyspace, statement, nextExecution);
             } catch (Throwable cause) {
               // This is a bug in the policy, but not fatal since we have at 
least one other
@@ -697,7 +681,7 @@ public class CqlRequestHandler implements Throttled {
                 true,
                 reprepareMessage,
                 repreparePayload.customPayload,
-                Conversions.resolveRequestTimeout(statement, context),
+                Conversions.resolveRequestTimeout(statement, executionProfile),
                 throttler,
                 sessionMetricUpdater,
                 logPrefix);
@@ -767,7 +751,7 @@ public class CqlRequestHandler implements Throttled {
         trackNodeError(node, error, NANOTIME_NOT_MEASURED_YET);
         setFinalError(statement, error, node, execution);
       } else {
-        RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(statement, 
context);
+        RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(context, 
executionProfile);
         RetryVerdict verdict;
         if (error instanceof ReadTimeoutException) {
           ReadTimeoutException readTimeout = (ReadTimeoutException) error;
@@ -788,7 +772,7 @@ public class CqlRequestHandler implements Throttled {
         } else if (error instanceof WriteTimeoutException) {
           WriteTimeoutException writeTimeout = (WriteTimeoutException) error;
           verdict =
-              Conversions.resolveIdempotence(statement, context)
+              Conversions.resolveIdempotence(statement, executionProfile)
                   ? retryPolicy.onWriteTimeoutVerdict(
                       statement,
                       writeTimeout.getConsistencyLevel(),
@@ -820,7 +804,7 @@ public class CqlRequestHandler implements Throttled {
               DefaultNodeMetric.IGNORES_ON_UNAVAILABLE);
         } else {
           verdict =
-              Conversions.resolveIdempotence(statement, context)
+              Conversions.resolveIdempotence(statement, executionProfile)
                   ? retryPolicy.onErrorResponseVerdict(statement, error, 
retryCount)
                   : RetryVerdict.RETHROW;
           updateErrorMetrics(
@@ -899,12 +883,12 @@ public class CqlRequestHandler implements Throttled {
       }
       LOG.trace("[{}] Request failure, processing: {}", logPrefix, error);
       RetryVerdict verdict;
-      if (!Conversions.resolveIdempotence(statement, context)
+      if (!Conversions.resolveIdempotence(statement, executionProfile)
           || error instanceof FrameTooLongException) {
         verdict = RetryVerdict.RETHROW;
       } else {
         try {
-          RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(statement, 
context);
+          RetryPolicy retryPolicy = Conversions.resolveRetryPolicy(context, 
executionProfile);
           verdict = retryPolicy.onRequestAbortedVerdict(statement, error, 
retryCount);
         } catch (Throwable cause) {
           setFinalError(


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to