avoid including non-queried nodes in rangeslice read repair patch by jbellis; reviewed by Vijay for CASSANDRA-3843
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e5be9520 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e5be9520 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e5be9520 Branch: refs/heads/cassandra-1.0 Commit: e5be9520b7f9f7f18739f00d4f265a3a3e0adfe2 Parents: 0183dc0 Author: Jonathan Ellis <jbel...@apache.org> Authored: Wed Feb 8 22:28:47 2012 -0600 Committer: Jonathan Ellis <jbel...@apache.org> Committed: Thu Feb 9 15:38:11 2012 -0600 ---------------------------------------------------------------------- CHANGES.txt | 7 ++++--- .../service/RangeSliceResponseResolver.java | 10 +++++++--- .../org/apache/cassandra/service/StorageProxy.java | 3 ++- 3 files changed, 13 insertions(+), 7 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5be9520/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index fe04571..8b97c73 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -69,12 +69,13 @@ * fix system tests (CASSANDRA-3827) * CQL support for altering key_validation_class in ALTER TABLE (CASSANDRA-3781) * turn compression on by default (CASSANDRA-3871) -Merged from 1.0: - * Only snapshot CF being compacted for snapshot_before_compaction - (CASSANDRA-3803) 1.0.8 + * avoid including non-queried nodes in rangeslice read repair + (CASSANDRA-3843) + * Only snapshot CF being compacted for snapshot_before_compaction + (CASSANDRA-3803) * Log active compactions in StatusLogger (CASSANDRA-3703) * Compute more accurate compaction score per level (CASSANDRA-3790) * Return InvalidRequest when using a keyspace that doesn't exist http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5be9520/src/java/org/apache/cassandra/service/RangeSliceResponseResolver.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/RangeSliceResponseResolver.java b/src/java/org/apache/cassandra/service/RangeSliceResponseResolver.java index 3be61d1..a870d5c 100644 --- a/src/java/org/apache/cassandra/service/RangeSliceResponseResolver.java +++ b/src/java/org/apache/cassandra/service/RangeSliceResponseResolver.java @@ -56,16 +56,20 @@ public class RangeSliceResponseResolver implements IResponseResolver<Iterable<Ro }; private final String table; - private final List<InetAddress> sources; + private List<InetAddress> sources; protected final Collection<Message> responses = new LinkedBlockingQueue<Message>();; public final List<IAsyncResult> repairResults = new ArrayList<IAsyncResult>(); - public RangeSliceResponseResolver(String table, List<InetAddress> sources) + public RangeSliceResponseResolver(String table) { - this.sources = sources; this.table = table; } + public void setSources(List<InetAddress> endpoints) + { + this.sources = endpoints; + } + public List<Row> getData() throws IOException { Message response = responses.iterator().next(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5be9520/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 532992a..310836e 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -861,9 +861,10 @@ public class StorageProxy implements StorageProxyMBean else { // collect replies and resolve according to consistency level - RangeSliceResponseResolver resolver = new RangeSliceResponseResolver(nodeCmd.keyspace, liveEndpoints); + RangeSliceResponseResolver resolver = new RangeSliceResponseResolver(nodeCmd.keyspace); ReadCallback<Iterable<Row>> handler = getReadCallback(resolver, nodeCmd, consistency_level, liveEndpoints); handler.assureSufficientLiveNodes(); + resolver.setSources(handler.endpoints); for (InetAddress endpoint : handler.endpoints) { MessagingService.instance().sendRR(nodeCmd, endpoint, handler);