[GitHub] cassandra pull request #276: Repair job tests

2018-10-08 Thread ifesdjeen
Github user ifesdjeen commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/276#discussion_r223013279
  
--- Diff: test/unit/org/apache/cassandra/repair/RepairJobTest.java ---
@@ -0,0 +1,569 @@
+/*
+ * 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 org.apache.cassandra.repair;
+
+import java.net.UnknownHostException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Predicate;
+
+import com.google.common.collect.Sets;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.MerkleTree;
+import org.apache.cassandra.utils.MerkleTrees;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class RepairJobTest
+{
+private static final IPartitioner PARTITIONER = 
ByteOrderedPartitioner.instance;
+
+static InetAddressAndPort addr1;
+static InetAddressAndPort addr2;
+static InetAddressAndPort addr3;
+static InetAddressAndPort addr4;
+static InetAddressAndPort addr5;
+
+static Range range1 = range(0, 1);
+static Range range2 = range(2, 3);
+static Range range3 = range(4, 5);
+static RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), 
UUID.randomUUID(), "ks", "cf", Arrays.asList());
+
+@AfterClass
+public static void reset()
+{
+FBUtilities.reset();
+}
+
+static
+{
+try
+{
+addr1 = InetAddressAndPort.getByName("127.0.0.1");
+addr2 = InetAddressAndPort.getByName("127.0.0.2");
+addr3 = InetAddressAndPort.getByName("127.0.0.3");
+addr4 = InetAddressAndPort.getByName("127.0.0.4");
+addr5 = InetAddressAndPort.getByName("127.0.0.5");
+DatabaseDescriptor.setBroadcastAddress(addr1.address);
+}
+catch (UnknownHostException e)
+{
+e.printStackTrace();
+}
+}
+
+@Test
+public void testCreateStandardSyncTasks()
+{
+testCreateStandardSyncTasks(false);
+}
+
+@Test
+public void testCreateStandardSyncTasksPullRepair()
+{
+testCreateStandardSyncTasks(true);
+}
+
+public static void testCreateStandardSyncTasks(boolean pullRepair)
+{
+List treeResponses = 
Arrays.asList(treeResponse(addr1, range1, "same",  range2, "same", range3, 
"same"),
+ 
treeResponse(addr2, range1, "different", range2, "same", range3, "different"),
+ 
treeResponse(addr3, range1, "same",  range2, "same", range3, "same"));
+
+Map tasks = 
toMap(RepairJob.createStandardSyncTasks(desc,
+   
 treeResponses,
+   
 addr1, // local
+   
 noTransient(), // transient
+  

[GitHub] cassandra pull request #278: Avoid running query to self through messaging s...

2018-10-08 Thread aweisberg
Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/278#discussion_r223398346
  
--- Diff: src/java/org/apache/cassandra/net/MessagingService.java ---
@@ -1078,7 +1078,7 @@ public void sendOneWay(MessageOut message, int id, 
InetAddressAndPort to)
 logger.trace("{} sending {} to {}@{}", 
FBUtilities.getBroadcastAddressAndPort(), message.verb, id, to);
 
 if (to.equals(FBUtilities.getBroadcastAddressAndPort()))
-logger.trace("Message-to-self {} going over MessagingService", 
message);
+logger.warn("Message-to-self {} going over MessagingService", 
message);
--- End diff --

My last thought is that as a user I can't do much with this information. 
It's mostly harmless, but I also can't fix it other than by filing a bug 
report. 

I think this is the biggest issue with this change. It will cause people 
worry when nothing is wrong. I think this should be debug not warn because no 
action is required on part of a user (as opposed to a developer).


---

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



[GitHub] cassandra pull request #278: Avoid running query to self through messaging s...

2018-10-08 Thread aweisberg
Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/278#discussion_r223396208
  
--- Diff: src/java/org/apache/cassandra/net/MessagingService.java ---
@@ -1078,7 +1078,7 @@ public void sendOneWay(MessageOut message, int id, 
InetAddressAndPort to)
 logger.trace("{} sending {} to {}@{}", 
FBUtilities.getBroadcastAddressAndPort(), message.verb, id, to);
 
 if (to.equals(FBUtilities.getBroadcastAddressAndPort()))
-logger.trace("Message-to-self {} going over MessagingService", 
message);
+logger.warn("Message-to-self {} going over MessagingService", 
message);
--- End diff --

Use NoSpamLogger and maybe for the key incorporate the stack trace so we 
rate limit every call site separately?


---

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



[GitHub] cassandra pull request #277: 12106 - blacklisting bad partitions for point r...

2018-10-08 Thread vinaykumarchella
Github user vinaykumarchella commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/277#discussion_r223517674
  
--- Diff: 
src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java ---
@@ -307,6 +320,42 @@ public static void setViewRemoved(String keyspaceName, 
String viewName)
 forceBlockingFlush(VIEW_BUILD_STATUS);
 }
 
+/**
+ * Reads blacklisted partitions from 
system_distributed.blacklisted_partitions table
+ * @return
+ */
+public static Set getBlacklistedPartitions()
+{
+String query = "SELECT keyspace_name, columnfamily_name, 
partition_key FROM %s.%s";
+UntypedResultSet results;
+try
+{
+results = QueryProcessor.execute(format(query, 
SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, BLACKLISTED_PARTITIONS),
+ ConsistencyLevel.ONE);
+}
+catch (Exception e)
+{
+logger.error("Error querying blacklisted partitions");
--- End diff --

You might want to log the exception also.


---

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



[GitHub] cassandra pull request #277: 12106 - blacklisting bad partitions for point r...

2018-10-08 Thread vinaykumarchella
Github user vinaykumarchella commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/277#discussion_r223516968
  
--- Diff: src/java/org/apache/cassandra/cache/BlacklistedPartition.java ---
@@ -0,0 +1,119 @@
+/*
+ * 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 org.apache.cassandra.cache;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+
+/**
+ * Class to represent a blacklisted partition
+ */
+public class BlacklistedPartition implements IMeasurableMemory
+{
+private static final long EMPTY_SIZE = ObjectSizes.measure(new 
BlacklistedPartition(null, new byte[0]));
+public final byte[] key;
+public final TableId tableId;
+
+public BlacklistedPartition(TableId tableId, DecoratedKey key)
+{
+this.tableId = tableId;
+this.key = ByteBufferUtil.getArray(key.getKey());
+}
+
+private BlacklistedPartition(TableId tableId, byte[] key)
+{
+this.tableId = tableId;
+this.key = key;
+}
+
+/**
+ * Creates an instance of BlacklistedPartition for a given keyspace, 
table and partition key
+ *
+ * @param keyspace
+ * @param table
+ * @param key
+ * @throws IllegalArgumentException
+ */
+public BlacklistedPartition(String keyspace, String table, String key) 
throws IllegalArgumentException
+{
+// Determine tableId from keyspace and table parameters. If 
tableId cannot be determined due to invalid
+// parameters, throw an exception
+KeyspaceMetadata ksMetaData = 
Schema.instance.getKeyspaceMetadata(keyspace);
+if (ksMetaData == null)
+{
+throw new IllegalArgumentException("Unknown keyspace '" + 
keyspace + "'");
+}
+
+TableMetadata metadata = ksMetaData.getTableOrViewNullable(table);
--- End diff --

You can simplify this by just checking `hasTable`


---

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



[GitHub] cassandra pull request #277: 12106 - blacklisting bad partitions for point r...

2018-10-08 Thread vinaykumarchella
Github user vinaykumarchella commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/277#discussion_r223518337
  
--- Diff: src/java/org/apache/cassandra/service/StorageProxy.java ---
@@ -1545,6 +1546,18 @@ public static PartitionIterator 
read(SinglePartitionReadCommand.Group group, Con
 throw new IsBootstrappingException();
 }
 
+// check if the partition in question is blacklisted; if yes, 
reject READ operation
+if (BlacklistedPartitionCache.instance.size() > 0)
+{
+for (SinglePartitionReadQuery query : group.queries)
+{
+if 
(BlacklistedPartitionCache.instance.contains(query.metadata().id, 
query.partitionKey()))
+{
+throw new InvalidRequestException("Cannot perform READ 
on a blacklisted partition");
--- End diff --

Do you want to expose the parition key here in the exception message?


---

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



[GitHub] cassandra pull request #277: 12106 - blacklisting bad partitions for point r...

2018-10-08 Thread vinaykumarchella
Github user vinaykumarchella commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/277#discussion_r223517840
  
--- Diff: 
src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java ---
@@ -307,6 +320,42 @@ public static void setViewRemoved(String keyspaceName, 
String viewName)
 forceBlockingFlush(VIEW_BUILD_STATUS);
 }
 
+/**
+ * Reads blacklisted partitions from 
system_distributed.blacklisted_partitions table
+ * @return
+ */
+public static Set getBlacklistedPartitions()
+{
+String query = "SELECT keyspace_name, columnfamily_name, 
partition_key FROM %s.%s";
+UntypedResultSet results;
+try
+{
+results = QueryProcessor.execute(format(query, 
SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, BLACKLISTED_PARTITIONS),
--- End diff --

I am assuming `BLACKLISTED_PARTITIONS` table wont be too big that it eats 
up your heap. Do you have any restrictions on that?


---

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



[GitHub] cassandra pull request #277: 12106 - blacklisting bad partitions for point r...

2018-10-08 Thread vinaykumarchella
Github user vinaykumarchella commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/277#discussion_r223517462
  
--- Diff: src/java/org/apache/cassandra/config/Config.java ---
@@ -257,6 +257,9 @@
 public volatile int counter_cache_save_period = 7200;
 public volatile int counter_cache_keys_to_save = Integer.MAX_VALUE;
 
+public long blacklisted_partitions_cache_size_warn_threshold_in_mb;
+public int blacklisted_partitions_cache_refresh_period_in_sec = 600; 
//10 minutes
--- End diff --

**Idea:** Do we need to expose this? If we can expose a nodetool and JMX 
interfaces, this extra config can be avoided. If anyone is interested in 
refreshing the cache quickly after adding new blacklisted partitions, they can 
use these interfaces. I am just trying to see if we can reduce the number of 
configs and keep it simple.


---

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



[GitHub] cassandra pull request #277: 12106 - blacklisting bad partitions for point r...

2018-10-08 Thread vinaykumarchella
Github user vinaykumarchella commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/277#discussion_r223518571
  
--- Diff: 
test/unit/org/apache/cassandra/cache/BlacklistedPartitionsCacheTest.java ---
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.cassandra.cache;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.InvalidQueryException;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.repair.SystemDistributedKeyspace;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class BlacklistedPartitionsCacheTest extends CQLTester
--- End diff --

Can you add a few test cases which can cover different types of partition 
keys (text, int, etc.,)


---

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



[GitHub] cassandra pull request #277: 12106 - blacklisting bad partitions for point r...

2018-10-08 Thread sumanth-pasupuleti
Github user sumanth-pasupuleti commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/277#discussion_r223545254
  
--- Diff: 
src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java ---
@@ -307,6 +320,42 @@ public static void setViewRemoved(String keyspaceName, 
String viewName)
 forceBlockingFlush(VIEW_BUILD_STATUS);
 }
 
+/**
+ * Reads blacklisted partitions from 
system_distributed.blacklisted_partitions table
+ * @return
+ */
+public static Set getBlacklistedPartitions()
+{
+String query = "SELECT keyspace_name, columnfamily_name, 
partition_key FROM %s.%s";
+UntypedResultSet results;
+try
+{
+results = QueryProcessor.execute(format(query, 
SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, BLACKLISTED_PARTITIONS),
--- End diff --

There is no restriction on how much heap the cache could take up; this is 
to make the blacklisting deterministic. Hoping the explicit mentions about this 
in the documentation I put in place, and the warn threshold on the cache size 
to help.


---

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



[GitHub] cassandra pull request #277: 12106 - blacklisting bad partitions for point r...

2018-10-08 Thread sumanth-pasupuleti
Github user sumanth-pasupuleti commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/277#discussion_r223544939
  
--- Diff: src/java/org/apache/cassandra/service/StorageProxy.java ---
@@ -1545,6 +1546,18 @@ public static PartitionIterator 
read(SinglePartitionReadCommand.Group group, Con
 throw new IsBootstrappingException();
 }
 
+// check if the partition in question is blacklisted; if yes, 
reject READ operation
+if (BlacklistedPartitionCache.instance.size() > 0)
+{
+for (SinglePartitionReadQuery query : group.queries)
+{
+if 
(BlacklistedPartitionCache.instance.contains(query.metadata().id, 
query.partitionKey()))
+{
+throw new InvalidRequestException("Cannot perform READ 
on a blacklisted partition");
--- End diff --

good point. added.


---

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



[GitHub] cassandra pull request #277: 12106 - blacklisting bad partitions for point r...

2018-10-08 Thread sumanth-pasupuleti
Github user sumanth-pasupuleti commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/277#discussion_r223545187
  
--- Diff: 
test/unit/org/apache/cassandra/cache/BlacklistedPartitionsCacheTest.java ---
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.cassandra.cache;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.InvalidQueryException;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.repair.SystemDistributedKeyspace;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class BlacklistedPartitionsCacheTest extends CQLTester
--- End diff --

@vinaykumarchella  I tried doing that to a certain extent in 
"testBlacklistingPartitionWithCompositeKey" where I test partition key of type 
blob, text and int. Do you suggest doing it for other types?


---

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



[GitHub] cassandra pull request #277: 12106 - blacklisting bad partitions for point r...

2018-10-08 Thread sumanth-pasupuleti
Github user sumanth-pasupuleti commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/277#discussion_r223544527
  
--- Diff: src/java/org/apache/cassandra/config/Config.java ---
@@ -257,6 +257,9 @@
 public volatile int counter_cache_save_period = 7200;
 public volatile int counter_cache_keys_to_save = Integer.MAX_VALUE;
 
+public long blacklisted_partitions_cache_size_warn_threshold_in_mb;
+public int blacklisted_partitions_cache_refresh_period_in_sec = 600; 
//10 minutes
--- End diff --

i've already added a nodetool command to force refresh a cache incase 
someone adds a new blacklisted partition to be cached. I am not sure of the 
likelyhood of someone wanting to change the frequency of cache refresh at run 
time. I see your point of reducing the config changes, but I am honestly not 
sure if we should shy away from adding a config.


---

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



[GitHub] cassandra-dtest pull request #39: Add no-read assert to read-repair test

2018-10-08 Thread aweisberg
Github user aweisberg commented on a diff in the pull request:

https://github.com/apache/cassandra-dtest/pull/39#discussion_r223397826
  
--- Diff: read_repair_test.py ---
@@ -495,6 +495,9 @@ def test_normal_read_repair(self):
 assert storage_proxy.speculated_rr_read == 0
 assert storage_proxy.speculated_rr_write == 0
 
+warn = node2.grep_log("Message-to-self TYPE:READ")
--- End diff --

Is this something we want to check for in all dtests all the time as part 
of checking the log for errors?

It might help us avoid leaving spammy log lines that don't mean much to 
users in the long run.


---

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



[GitHub] cassandra pull request #278: Avoid running query to self through messaging s...

2018-10-08 Thread ifesdjeen
Github user ifesdjeen commented on a diff in the pull request:

https://github.com/apache/cassandra/pull/278#discussion_r223421365
  
--- Diff: src/java/org/apache/cassandra/net/MessagingService.java ---
@@ -1078,7 +1078,7 @@ public void sendOneWay(MessageOut message, int id, 
InetAddressAndPort to)
 logger.trace("{} sending {} to {}@{}", 
FBUtilities.getBroadcastAddressAndPort(), message.verb, id, to);
 
 if (to.equals(FBUtilities.getBroadcastAddressAndPort()))
-logger.trace("Message-to-self {} going over MessagingService", 
message);
+logger.warn("Message-to-self {} going over MessagingService", 
message);
--- End diff --

Right, we can do debug here. I wanted to first throw in this case, but then 
thought that it's more useful to find all the cases where we still do that and 
eliminate those, since failing in that case brings more or less nothing.


---

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