Change snapshot response message verb to make sure response do not get dropped
patch by yukim; reviewed by jbellis for CASSANDRA-6415 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c2024183 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c2024183 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c2024183 Branch: refs/heads/trunk Commit: c202418386c512dcb0410d9c50f696048e305d78 Parents: c63d068 Author: Yuki Morishita <yu...@apache.org> Authored: Mon Dec 2 12:31:38 2013 -0600 Committer: Yuki Morishita <yu...@apache.org> Committed: Mon Dec 2 12:31:38 2013 -0600 ---------------------------------------------------------------------- CHANGES.txt | 1 + src/java/org/apache/cassandra/service/SnapshotVerbHandler.java | 5 ++--- 2 files changed, 3 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2024183/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index 2119f77..c80a00a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -7,6 +7,7 @@ * Improve gossip performance for typical messages (CASSANDRA-6409) * Throw IRE if a prepared has more markers than supported (CASSANDRA-5598) * Expose Thread metrics for the native protocol server (CASSANDRA-6234) + * Change snapshot response message verb (CASSANDRA-6415) 1.2.12 http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2024183/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java index f15e8c5..8a80731 100644 --- a/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java +++ b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java @@ -26,7 +26,6 @@ import org.apache.cassandra.net.IVerbHandler; import org.apache.cassandra.net.MessageIn; import org.apache.cassandra.net.MessageOut; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.tracing.Tracing; public class SnapshotVerbHandler implements IVerbHandler<SnapshotCommand> { @@ -39,7 +38,7 @@ public class SnapshotVerbHandler implements IVerbHandler<SnapshotCommand> Table.open(command.keyspace).clearSnapshot(command.snapshot_name); else Table.open(command.keyspace).getColumnFamilyStore(command.column_family).snapshot(command.snapshot_name); - Tracing.trace("Enqueuing response to snapshot request {} to {}", command.snapshot_name, message.from); - MessagingService.instance().sendReply(new MessageOut(MessagingService.Verb.REQUEST_RESPONSE), id, message.from); + logger.debug("Enqueuing response to snapshot request {} to {}", command.snapshot_name, message.from); + MessagingService.instance().sendReply(new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE), id, message.from); } }