Updated Branches: refs/heads/cassandra-1.2 f57b89a9c -> 39a38e675 refs/heads/trunk 47ac42fdb -> b320cd9cd
Add timeout events to query traces patch by Tyler Hobbs; reviewed by jbellis for CASSANDRA-5520 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/39a38e67 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/39a38e67 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/39a38e67 Branch: refs/heads/cassandra-1.2 Commit: 39a38e675987d3bcb99a7d2cd8d5e606de6b56f1 Parents: f57b89a Author: Jonathan Ellis <jbel...@apache.org> Authored: Wed Jul 10 18:06:38 2013 -0500 Committer: Jonathan Ellis <jbel...@apache.org> Committed: Thu Jul 11 14:17:41 2013 -0500 ---------------------------------------------------------------------- CHANGES.txt | 2 + .../apache/cassandra/service/ReadCallback.java | 8 +++ .../apache/cassandra/service/StorageProxy.java | 60 +++++++++++++++++--- .../cassandra/thrift/CassandraServer.java | 4 -- 4 files changed, 63 insertions(+), 11 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/39a38e67/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 34235cf..e1e3154 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 1.2.7 + * Add timeout events to query traces (CASSANDRA-5520) * make starting native protocol server idempotent (CASSANDRA-5728) * Fix loading key cache when a saved entry is no longer valid (CASSANDRA-5706) * Fix serialization of the LEFT gossip value (CASSANDRA-5696) @@ -12,6 +13,7 @@ * cqlsh: fix handling of semicolons inside BATCH queries (CASSANDRA-5697) * Expose native protocol server status in nodetool info (CASSANDRA-5735) + 1.2.6 * Fix tracing when operation completes before all responses arrive (CASSANDRA-5668) http://git-wip-us.apache.org/repos/asf/cassandra/blob/39a38e67/src/java/org/apache/cassandra/service/ReadCallback.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/ReadCallback.java b/src/java/org/apache/cassandra/service/ReadCallback.java index 92032f2..ebda81b 100644 --- a/src/java/org/apache/cassandra/service/ReadCallback.java +++ b/src/java/org/apache/cassandra/service/ReadCallback.java @@ -125,6 +125,14 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallback<TMessag : true; } + /** + * @return the current number of received responses + */ + public int getReceivedCount() + { + return received.get(); + } + public void response(TMessage result) { MessageIn<TMessage> message = MessageIn.create(FBUtilities.getBroadcastAddress(), http://git-wip-us.apache.org/repos/asf/cassandra/blob/39a38e67/src/java/org/apache/cassandra/service/StorageProxy.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java index ee045eb..28e2af5 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -282,7 +282,7 @@ public class StorageProxy implements StorageProxyMBean { writeMetrics.timeouts.mark(); ClientRequestMetrics.writeTimeouts.inc(); - Tracing.trace("Write timeout"); + Tracing.trace("Write timeout; received {} of {} required replies", e.received, e.blockFor); throw e; } finally @@ -898,7 +898,7 @@ public class StorageProxy implements StorageProxyMBean } else { - Tracing.trace("Enqueuing digest request to {}", dataPoint); + Tracing.trace("Enqueuing digest request to {}", digestPoint); // (We lazy-construct the digest Message object since it may not be necessary if we // are doing a local digest read, or no digest reads at all.) if (message == null) @@ -926,8 +926,21 @@ public class StorageProxy implements StorageProxyMBean } catch (ReadTimeoutException ex) { - if (logger.isDebugEnabled()) - logger.debug("Read timeout: {}", ex.toString()); + int blockFor = consistency_level.blockFor(Table.open(command.getKeyspace())); + int responseCount = handler.getReceivedCount(); + String gotData = responseCount > 0 ? + gotData = handler.resolver.isDataPresent() ? " (including data)" : " (only digests)" : + ""; + + if (Tracing.isTracing()) + { + Tracing.trace("Timed out; received {} of {} responses{}", + new Object[]{ responseCount, blockFor, gotData }); + } + else if (logger.isDebugEnabled()) + { + logger.debug("Read timeout; received {} of {} responses{}", responseCount, blockFor, gotData); + } throw ex; } catch (DigestMismatchException ex) @@ -987,6 +1000,7 @@ public class StorageProxy implements StorageProxyMBean } catch (TimeoutException e) { + Tracing.trace("Timed out on digest mismatch retries"); int blockFor = consistency_level.blockFor(Table.open(command.getKeyspace())); throw new ReadTimeoutException(consistency_level, blockFor, blockFor, true); } @@ -1194,11 +1208,35 @@ public class StorageProxy implements StorageProxyMBean } FBUtilities.waitOnFutures(resolver.repairResults, DatabaseDescriptor.getWriteRpcTimeout()); } + catch (ReadTimeoutException ex) + { + // we timed out waiting for responses + int blockFor = consistency_level.blockFor(table); + int responseCount = resolver.responses.size(); + String gotData = responseCount > 0 ? + resolver.isDataPresent() ? " (including data)" : " (only digests)" : + ""; + + if (Tracing.isTracing()) + { + Tracing.trace("Timed out; received {} of {} responses{} for range {} of {}", + new Object[]{ responseCount, blockFor, gotData, i, ranges.size() }); + } + else if (logger.isDebugEnabled()) + { + logger.debug("Range slice timeout; received {} of {} responses{} for range {} of {}", + responseCount, blockFor, gotData, i, ranges.size()); + } + throw ex; + } catch (TimeoutException ex) { - logger.debug("Range slice timeout: {}", ex.toString()); - // We actually got all response at that point + // We got all responses, but timed out while repairing int blockFor = consistency_level.blockFor(table); + if (Tracing.isTracing()) + Tracing.trace("Timed out while read-repairing after receiving all {} data and digest responses", blockFor); + else + logger.debug("Range slice timeout while read-repairing after receiving all {} data and digest responses", blockFor); throw new ReadTimeoutException(consistency_level, blockFor, blockFor, true); } catch (DigestMismatchException e) @@ -1504,7 +1542,15 @@ public class StorageProxy implements StorageProxyMBean MessagingService.instance().sendRR(message, endpoint, responseHandler); // Wait for all - responseHandler.get(); + try + { + responseHandler.get(); + } + catch (TimeoutException e) + { + Tracing.trace("Timed out"); + throw e; + } } /** http://git-wip-us.apache.org/repos/asf/cassandra/blob/39a38e67/src/java/org/apache/cassandra/thrift/CassandraServer.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java index ba55195..1959815 100644 --- a/src/java/org/apache/cassandra/thrift/CassandraServer.java +++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java @@ -965,7 +965,6 @@ public class CassandraServer implements Cassandra.Iface } catch (ReadTimeoutException e) { - logger.debug("... timed out"); throw ThriftConversion.toThrift(e); } catch (org.apache.cassandra.exceptions.UnavailableException e) @@ -1049,7 +1048,6 @@ public class CassandraServer implements Cassandra.Iface } catch (ReadTimeoutException e) { - logger.debug("... timed out"); throw ThriftConversion.toThrift(e); } catch (org.apache.cassandra.exceptions.UnavailableException e) @@ -1125,7 +1123,6 @@ public class CassandraServer implements Cassandra.Iface } catch (ReadTimeoutException e) { - logger.debug("... timed out"); throw ThriftConversion.toThrift(e); } catch (org.apache.cassandra.exceptions.UnavailableException e) @@ -1457,7 +1454,6 @@ public class CassandraServer implements Cassandra.Iface } catch (TimeoutException e) { - logger.debug("... timed out"); throw new TimedOutException(); } catch (IOException e)