[jira] [Updated] (CASSANDRA-14720) SSTables may not be properly removed from original strategy when repair status changed

2018-09-11 Thread Marcus Eriksson (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14720?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Eriksson updated CASSANDRA-14720:

   Resolution: Fixed
Fix Version/s: 4.0
   Status: Resolved  (was: Patch Available)

nice catch, thanks!

committed as {{f100024eb3becf53042823ce1008d3d5ec4e5f86}}

> SSTables may not be properly removed from original strategy when repair 
> status changed
> --
>
> Key: CASSANDRA-14720
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14720
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: ZhaoYang
>Assignee: ZhaoYang
>Priority: Major
> Fix For: 4.0
>
>
> In {{CSM.handleRepairStatusChangedNotification()}}, CASSANDRA-14621 changed 
> the original semantic of {{removing sstables in repaired first}} to {{adding 
> sstables into unrepaired first}}...
> In case of LCS, adding sstables may modify their levels, so they won't be 
> removed from {{repaired}} which locates sstables by level. 
> | 
> [trunk|https://github.com/apache/cassandra/compare/trunk...jasonstack:CASSANDRA-14621-follow-up?expand=1]
>  | [circle|https://circleci.com/gh/jasonstack/cassandra/742] |



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



cassandra git commit: fix handleRepairStatusChangedNotification to remove first then add

2018-09-11 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk 0841353e9 -> f100024eb


fix handleRepairStatusChangedNotification to remove first then add

Patch by ZhaoYang reviewed by marcuse for CASSANDRA-14720


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f100024e
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f100024e
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f100024e

Branch: refs/heads/trunk
Commit: f100024eb3becf53042823ce1008d3d5ec4e5f86
Parents: 0841353
Author: Zhao Yang 
Authored: Wed Sep 12 11:32:46 2018 +0800
Committer: Marcus Eriksson 
Committed: Wed Sep 12 08:32:51 2018 +0200

--
 CHANGES.txt |  1 +
 .../compaction/CompactionStrategyManager.java   |  5 +-
 .../LongLeveledCompactionStrategyTest.java  | 99 ++--
 3 files changed, 73 insertions(+), 32 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f100024e/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index ef285e0..3cfdcff 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * fix handleRepairStatusChangedNotification to remove first then add 
(CASSANDRA-14720)
  * Allow transient node to serve as a repair coordinator (CASSANDRA-14693)
  * DecayingEstimatedHistogramReservoir.EstimatedHistogramReservoirSnapshot 
returns wrong value for size() and incorrectly calculates count 
(CASSANDRA-14696)
  * AbstractReplicaCollection equals and hash code should throw due to conflict 
between order sensitive/insensitive uses (CASSANDRA-14700)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f100024e/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
--
diff --git 
a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java 
b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index afe628b..45ccbe2 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@ -735,13 +735,16 @@ public class CompactionStrategyManager implements 
INotificationConsumer
 continue;
 
 AbstractStrategyHolder dstHolder = holders.get(i);
-dstHolder.addSSTables(group);
 
 for (AbstractStrategyHolder holder : holders)
 {
 if (holder != dstHolder)
 holder.removeSSTables(group);
 }
+
+// adding sstables into another strategy may change its level,
+// thus it won't be removed from original LCS. We have to 
remove sstables first
+dstHolder.addSSTables(group);
 }
 }
 finally

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f100024e/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
--
diff --git 
a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
 
b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
index 56eef17..3bcd9d1 100644
--- 
a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
+++ 
b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.compaction;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
+import java.util.stream.Collectors;
 
 import com.google.common.collect.Lists;
 
@@ -38,8 +39,11 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.CompactionParams;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 public class LongLeveledCompactionStrategyTest
@@ -75,22 +79,7 @@ public class LongLeveledCompactionStrategyTest
 
 ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB 
value, make it easy to have multiple files
 
-// Enough data to have a level 1 and 2
-int rows = 128;
-int columns = 10;
-
-// Adds enough data to trigger multiple sstable per level
-for (int r = 0; r < rows; r++)
-{
-DecoratedKey key = Util.dk(String.valueOf(r));
-UpdateBuilder builder = UpdateBu

[jira] [Updated] (CASSANDRA-14720) SSTables may not be properly removed from original strategy when repair status changed

2018-09-11 Thread Marcus Eriksson (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14720?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Eriksson updated CASSANDRA-14720:

Reviewer: Marcus Eriksson

> SSTables may not be properly removed from original strategy when repair 
> status changed
> --
>
> Key: CASSANDRA-14720
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14720
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: ZhaoYang
>Assignee: ZhaoYang
>Priority: Major
>
> In {{CSM.handleRepairStatusChangedNotification()}}, CASSANDRA-14621 changed 
> the original semantic of {{removing sstables in repaired first}} to {{adding 
> sstables into unrepaired first}}...
> In case of LCS, adding sstables may modify their levels, so they won't be 
> removed from {{repaired}} which locates sstables by level. 
> | 
> [trunk|https://github.com/apache/cassandra/compare/trunk...jasonstack:CASSANDRA-14621-follow-up?expand=1]
>  | [circle|https://circleci.com/gh/jasonstack/cassandra/742] |



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (CASSANDRA-14720) SSTables may not be properly removed from original strategy when repair status changed

2018-09-11 Thread ZhaoYang (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14720?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ZhaoYang updated CASSANDRA-14720:
-
Assignee: ZhaoYang
  Status: Patch Available  (was: Open)

> SSTables may not be properly removed from original strategy when repair 
> status changed
> --
>
> Key: CASSANDRA-14720
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14720
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: ZhaoYang
>Assignee: ZhaoYang
>Priority: Major
>
> In {{CSM.handleRepairStatusChangedNotification()}}, CASSANDRA-14621 changed 
> the original semantic of {{removing sstables in repaired first}} to {{adding 
> sstables into unrepaired first}}...
> In case of LCS, adding sstables may modify their levels, so they won't be 
> removed from {{repaired}} which locates sstables by level. 
> | 
> [trunk|https://github.com/apache/cassandra/compare/trunk...jasonstack:CASSANDRA-14621-follow-up?expand=1]
>  | [circle|https://circleci.com/gh/jasonstack/cassandra/742] |



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14720) SSTables may not be properly removed from original strategy when repair status changed

2018-09-11 Thread ZhaoYang (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14720?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16611553#comment-16611553
 ] 

ZhaoYang commented on CASSANDRA-14720:
--

[~bdeggleston] do you mind reviewing?

> SSTables may not be properly removed from original strategy when repair 
> status changed
> --
>
> Key: CASSANDRA-14720
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14720
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: ZhaoYang
>Priority: Major
>
> In {{CSM.handleRepairStatusChangedNotification()}}, CASSANDRA-14621 changed 
> the original semantic of {{removing sstables in repaired first}} to {{adding 
> sstables into unrepaired first}}...
> In case of LCS, adding sstables may modify their levels, so they won't be 
> removed from {{repaired}} which locates sstables by level. 
> | 
> [trunk|https://github.com/apache/cassandra/compare/trunk...jasonstack:CASSANDRA-14621-follow-up?expand=1]
>  | [circle|https://circleci.com/gh/jasonstack/cassandra/742] |



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (CASSANDRA-14720) SSTables may not be properly removed from original strategy when repair status changed

2018-09-11 Thread ZhaoYang (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14720?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ZhaoYang updated CASSANDRA-14720:
-
Description: 
In {{CSM.handleRepairStatusChangedNotification()}}, CASSANDRA-14621 changed the 
original semantic of {{removing sstables in repaired first}} to {{adding 
sstables into unrepaired first}}...

In case of LCS, adding sstables may modify their levels, so they won't be 
removed from {{repaired}} which locates sstables by level. 

| 
[trunk|https://github.com/apache/cassandra/compare/trunk...jasonstack:CASSANDRA-14621-follow-up?expand=1]
 | [circle|https://circleci.com/gh/jasonstack/cassandra/742] |

  was:
In {{CSM.handleRepairStatusChangedNotification()}}, CASSANDRA-14621 changed the 
original semantic of {{removing sstables in repaired first}} to {{adding 
sstables into unrepaired first}}...

In case of LCS, adding sstables may modify their levels, so they won't be 
removed from {{repaired}} which locates sstables by level. 

 

| 
[trunk|[https://github.com/apache/cassandra/compare/trunk...jasonstack:CASSANDRA-14621-follow-up?expand=1]]
 | [circle|https://circleci.com/gh/jasonstack/cassandra/742] |


> SSTables may not be properly removed from original strategy when repair 
> status changed
> --
>
> Key: CASSANDRA-14720
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14720
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: ZhaoYang
>Priority: Major
>
> In {{CSM.handleRepairStatusChangedNotification()}}, CASSANDRA-14621 changed 
> the original semantic of {{removing sstables in repaired first}} to {{adding 
> sstables into unrepaired first}}...
> In case of LCS, adding sstables may modify their levels, so they won't be 
> removed from {{repaired}} which locates sstables by level. 
> | 
> [trunk|https://github.com/apache/cassandra/compare/trunk...jasonstack:CASSANDRA-14621-follow-up?expand=1]
>  | [circle|https://circleci.com/gh/jasonstack/cassandra/742] |



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (CASSANDRA-14720) SSTables may not be properly removed from original strategy when repair status changed

2018-09-11 Thread ZhaoYang (JIRA)
ZhaoYang created CASSANDRA-14720:


 Summary: SSTables may not be properly removed from original 
strategy when repair status changed
 Key: CASSANDRA-14720
 URL: https://issues.apache.org/jira/browse/CASSANDRA-14720
 Project: Cassandra
  Issue Type: Bug
  Components: Compaction
Reporter: ZhaoYang


In {{CSM.handleRepairStatusChangedNotification()}}, CASSANDRA-14621 changed the 
original semantic of {{removing sstables in repaired first}} to {{adding 
sstables into unrepaired first}}...

In case of LCS, adding sstables may modify their levels, so they won't be 
removed from {{repaired}} which locates sstables by level. 

 

| 
[trunk|[https://github.com/apache/cassandra/compare/trunk...jasonstack:CASSANDRA-14621-follow-up?expand=1]]
 | [circle|https://circleci.com/gh/jasonstack/cassandra/742] |



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (CASSANDRA-14719) Auto-update C* documentation from CircleCI build

2018-09-11 Thread Sumanth Pasupuleti (JIRA)
Sumanth Pasupuleti created CASSANDRA-14719:
--

 Summary: Auto-update C* documentation from CircleCI build
 Key: CASSANDRA-14719
 URL: https://issues.apache.org/jira/browse/CASSANDRA-14719
 Project: Cassandra
  Issue Type: Improvement
  Components: Documentation and Website
Reporter: Sumanth Pasupuleti


Specifically  https://cassandra.apache.org/doc/latest/
Also update https://cassandra.apache.org/doc/X.Y upon patch release for that 
version.

Suggestion came out of mailing list thread 
https://lists.apache.org/thread.html/6ec86b6954a9138a0efb0ef73569cf1ef128a1125042c94b69ceaf72@%3Cdev.cassandra.apache.org%3E



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (CASSANDRA-14718) Add Circle CI job to validate artifacts generation

2018-09-11 Thread Sumanth Pasupuleti (JIRA)
Sumanth Pasupuleti created CASSANDRA-14718:
--

 Summary: Add Circle CI job to validate artifacts generation
 Key: CASSANDRA-14718
 URL: https://issues.apache.org/jira/browse/CASSANDRA-14718
 Project: Cassandra
  Issue Type: Improvement
  Components: Build
Reporter: Sumanth Pasupuleti


Specifically "ant artifacts" - suggestion came out of mailing list thread 
https://lists.apache.org/thread.html/6ec86b6954a9138a0efb0ef73569cf1ef128a1125042c94b69ceaf72@%3Cdev.cassandra.apache.org%3E



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Comment Edited] (CASSANDRA-14693) Follow-up: allow transient node to serve as repair coordinator

2018-09-11 Thread Alex Petrov (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16611250#comment-16611250
 ] 

Alex Petrov edited comment on CASSANDRA-14693 at 9/11/18 9:06 PM:
--

Thank you [~krummas], I've incorporated your comment on commit.

[~aweisberg] I'll open a separate issue: it's not that difficult to write tests 
for RepairJob streams, but there are quite a few cases, so it'd be good to work 
in it separately. [CASSANDRA-14717] will track this.

Committed as 
[0841353e90f1cc94dc47b435af87e4d5876478ea|https://github.com/apache/cassandra/commit/0841353e90f1cc94dc47b435af87e4d5876478ea]
 to trunk.


was (Author: ifesdjeen):
Thank you [~krummas], I've incorporated your comment on commit.

[~aweisberg] I'll open a separate issue: it's not that difficult to write tests 
for RepairJob streams, but there are quite a few cases, so it'd be good to work 
in it separately.

Committed as 
[0841353e90f1cc94dc47b435af87e4d5876478ea|https://github.com/apache/cassandra/commit/0841353e90f1cc94dc47b435af87e4d5876478ea]
 to trunk.

> Follow-up: allow transient node to serve as repair coordinator
> --
>
> Key: CASSANDRA-14693
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14693
> Project: Cassandra
>  Issue Type: Task
>Reporter: Alex Petrov
>Assignee: Alex Petrov
>Priority: Minor
>
> Allow transient node to serve as a coordinator. 
> |[trunk|https://github.com/apache/cassandra/pull/257]|[utest|https://circleci.com/gh/ifesdjeen/cassandra/329]|[dtest|https://circleci.com/gh/ifesdjeen/cassandra/330]|[dtest-novnode|https://circleci.com/gh/ifesdjeen/cassandra/328]|



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (CASSANDRA-14717) Write tests for RepairJob

2018-09-11 Thread Alex Petrov (JIRA)
Alex Petrov created CASSANDRA-14717:
---

 Summary: Write tests for RepairJob 
 Key: CASSANDRA-14717
 URL: https://issues.apache.org/jira/browse/CASSANDRA-14717
 Project: Cassandra
  Issue Type: Test
Reporter: Alex Petrov
Assignee: Alex Petrov


As a part of [CASSANDRA-14693], it was suggested to write additional unit tests 
for RepairJob.

Specifically, the unit-test candidates are {{standardSyncing}} and 
{{optimisedSynching}}. The rest of the class is slightly less important to test 
as it's mostly gathering ranges and collecting trees.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (CASSANDRA-14693) Follow-up: allow transient node to serve as repair coordinator

2018-09-11 Thread Alex Petrov (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14693?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Alex Petrov updated CASSANDRA-14693:

Resolution: Fixed
Status: Resolved  (was: Patch Available)

> Follow-up: allow transient node to serve as repair coordinator
> --
>
> Key: CASSANDRA-14693
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14693
> Project: Cassandra
>  Issue Type: Task
>Reporter: Alex Petrov
>Assignee: Alex Petrov
>Priority: Minor
>
> Allow transient node to serve as a coordinator. 
> |[trunk|https://github.com/apache/cassandra/pull/257]|[utest|https://circleci.com/gh/ifesdjeen/cassandra/329]|[dtest|https://circleci.com/gh/ifesdjeen/cassandra/330]|[dtest-novnode|https://circleci.com/gh/ifesdjeen/cassandra/328]|



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14693) Follow-up: allow transient node to serve as repair coordinator

2018-09-11 Thread Alex Petrov (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16611250#comment-16611250
 ] 

Alex Petrov commented on CASSANDRA-14693:
-

Thank you [~krummas], I've incorporated your comment on commit.

[~aweisberg] I'll open a separate issue: it's not that difficult to write tests 
for RepairJob streams, but there are quite a few cases, so it'd be good to work 
in it separately.

Committed as 
[0841353e90f1cc94dc47b435af87e4d5876478ea|https://github.com/apache/cassandra/commit/0841353e90f1cc94dc47b435af87e4d5876478ea]
 to trunk.

> Follow-up: allow transient node to serve as repair coordinator
> --
>
> Key: CASSANDRA-14693
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14693
> Project: Cassandra
>  Issue Type: Task
>Reporter: Alex Petrov
>Assignee: Alex Petrov
>Priority: Minor
>
> Allow transient node to serve as a coordinator. 
> |[trunk|https://github.com/apache/cassandra/pull/257]|[utest|https://circleci.com/gh/ifesdjeen/cassandra/329]|[dtest|https://circleci.com/gh/ifesdjeen/cassandra/330]|[dtest-novnode|https://circleci.com/gh/ifesdjeen/cassandra/328]|



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[1/2] cassandra git commit: Allow transient node to serve as a repair coordinator

2018-09-11 Thread ifesdjeen
Repository: cassandra
Updated Branches:
  refs/heads/trunk 2886cac38 -> 0841353e9


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0841353e/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
--
diff --git a/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java 
b/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
new file mode 100644
index 000..6e691f5
--- /dev/null
+++ b/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
@@ -0,0 +1,249 @@
+/*
+ * 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.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.InetAddressAndPort;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.StreamCoordinator;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.StreamPlan;
+import org.apache.cassandra.streaming.PreviewKind;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.MerkleTree;
+import org.apache.cassandra.utils.MerkleTrees;
+import org.apache.cassandra.utils.UUIDGen;
+
+import static 
org.apache.cassandra.service.ActiveRepairService.NO_PENDING_REPAIR;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class LocalSyncTaskTest extends AbstractRepairTest
+{
+private static final IPartitioner partitioner = 
Murmur3Partitioner.instance;
+private static final InetAddressAndPort local = 
FBUtilities.getBroadcastAddressAndPort();
+public static final String KEYSPACE1 = "DifferencerTest";
+public static final String CF_STANDARD = "Standard1";
+public static ColumnFamilyStore cfs;
+
+@BeforeClass
+public static void defineSchema()
+{
+SchemaLoader.prepareServer();
+SchemaLoader.createKeyspace(KEYSPACE1,
+KeyspaceParams.simple(1),
+SchemaLoader.standardCFMD(KEYSPACE1, 
CF_STANDARD));
+
+TableId tid = Schema.instance.getTableMetadata(KEYSPACE1, 
CF_STANDARD).id;
+cfs = Schema.instance.getColumnFamilyStoreInstance(tid);
+}
+
+/**
+ * When there is no difference between two, SymmetricLocalSyncTask should 
return stats with 0 difference.
+ */
+@Test
+public void testNoDifference() throws Throwable
+{
+final InetAddressAndPort ep2 = 
InetAddressAndPort.getByName("127.0.0.2");
+
+Range range = new Range<>(partitioner.getMinimumToken(), 
partitioner.getRandomToken());
+RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), 
UUID.randomUUID(), KEYSPACE1, "Standard1", Arrays.asList(range));
+
+MerkleTrees tree1 = createInitialTree(desc);
+
+MerkleTrees tree2 = createInitialTree(desc);
+
+// difference the trees
+// note: we reuse the same endpoint which is bogus in theory but fine 
here
+TreeResponse r1 = new TreeResponse(local, tree1);
+TreeResponse r2 = new TreeResponse(ep2, tree2);
+LocalSyncTask task = new LocalSyncTask(desc, r1, r2, 
NO_PENDING_REPAIR, true, true, PreviewKind.NONE);
+task.run();
+
+assertEquals(0, task.get().numberOfDifferences);
+}
+
+@Test
+public void t

[2/2] cassandra git commit: Allow transient node to serve as a repair coordinator

2018-09-11 Thread ifesdjeen
Allow transient node to serve as a repair coordinator

Patch by Alex Petrov and Blake Eggleston, reviewed by Ariel Weisberg, Blake 
Eggleston, Marcus Eriksson for CASSANDRA-14693

Co-authored-by: Blake Eggleston 


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0841353e
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0841353e
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0841353e

Branch: refs/heads/trunk
Commit: 0841353e90f1cc94dc47b435af87e4d5876478ea
Parents: 2886cac
Author: Alex Petrov 
Authored: Tue Sep 4 19:38:27 2018 +0200
Committer: Alex Petrov 
Committed: Tue Sep 11 22:58:01 2018 +0200

--
 CHANGES.txt |   1 +
 .../cassandra/db/DiskBoundaryManager.java   |  21 +-
 .../cassandra/repair/AbstractSyncTask.java  |  31 ---
 .../repair/AsymmetricLocalSyncTask.java | 105 
 .../repair/AsymmetricRemoteSyncTask.java|  19 +-
 .../cassandra/repair/AsymmetricSyncTask.java|  81 --
 .../apache/cassandra/repair/CommonRange.java|   8 +-
 .../apache/cassandra/repair/LocalSyncTask.java  | 158 
 .../org/apache/cassandra/repair/NodePair.java   |  91 ---
 .../org/apache/cassandra/repair/RepairJob.java  | 174 +++--
 .../apache/cassandra/repair/RepairRunnable.java |  27 +-
 .../apache/cassandra/repair/RepairSession.java  |  16 +-
 .../repair/SymmetricLocalSyncTask.java  | 142 ---
 .../repair/SymmetricRemoteSyncTask.java |  22 +-
 .../cassandra/repair/SymmetricSyncTask.java |  94 ---
 .../apache/cassandra/repair/SyncNodePair.java   |  91 +++
 .../org/apache/cassandra/repair/SyncStat.java   |   6 +-
 .../org/apache/cassandra/repair/SyncTask.java   |  96 +++
 .../cassandra/repair/messages/SyncComplete.java |  14 +-
 .../cassandra/service/ActiveRepairService.java  |   3 -
 .../service/reads/AbstractReadExecutor.java |   2 +-
 .../cassandra/repair/LocalSyncTaskTest.java | 249 +++
 .../repair/SymmetricLocalSyncTaskTest.java  | 232 -
 .../repair/SymmetricRemoteSyncTaskTest.java |   6 +-
 .../RepairMessageSerializationsTest.java|   6 +-
 .../cassandra/service/SerializationsTest.java   |   4 +-
 26 files changed, 766 insertions(+), 933 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0841353e/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index b7bc775..ef285e0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Allow transient node to serve as a repair coordinator (CASSANDRA-14693)
  * DecayingEstimatedHistogramReservoir.EstimatedHistogramReservoirSnapshot 
returns wrong value for size() and incorrectly calculates count 
(CASSANDRA-14696)
  * AbstractReplicaCollection equals and hash code should throw due to conflict 
between order sensitive/insensitive uses (CASSANDRA-14700)
  * Detect inconsistencies in repaired data on the read path (CASSANDRA-14145)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0841353e/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
--
diff --git a/src/java/org/apache/cassandra/db/DiskBoundaryManager.java 
b/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
index acfe71a..0961a42 100644
--- a/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
+++ b/src/java/org/apache/cassandra/db/DiskBoundaryManager.java
@@ -109,17 +109,7 @@ public class DiskBoundaryManager
 if (localRanges == null || localRanges.isEmpty())
 return new DiskBoundaries(dirs, null, ringVersion, 
directoriesVersion);
 
-// note that Range.sort unwraps any wraparound ranges, so we need to 
sort them here
-List> fullLocalRanges = Range.sort(localRanges.stream()
-   
.filter(Replica::isFull)
-   
.map(Replica::range)
-   
.collect(Collectors.toList()));
-List> transientLocalRanges = 
Range.sort(localRanges.stream()
-
.filter(Replica::isTransient)
-
.map(Replica::range)
-
.collect(Collectors.toList()));
-
-List positions = getDiskBoundaries(fullLocalRanges, 
transientLocalRanges, cfs.getPartitioner(), dirs);
+List positions = getDiskBoundaries(localRanges, 
cfs.getPartitioner(), dirs);
 
 return new DiskBoundaries(dirs, positions, ringV

[jira] [Comment Edited] (CASSANDRA-14694) add latency sample for speculative read repair writes

2018-09-11 Thread Chris Lohfink (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14694?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16611233#comment-16611233
 ] 

Chris Lohfink edited comment on CASSANDRA-14694 at 9/11/18 8:52 PM:


there is a coordinator write latency per table, can it just use that instead of 
introducing new metric?


was (Author: cnlwsu):
there is a coordinator write latency per table, can it just use that instead?

> add latency sample for speculative read repair writes
> -
>
> Key: CASSANDRA-14694
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14694
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Blake Eggleston
>Assignee: Blake Eggleston
>Priority: Major
> Fix For: 4.0
>
>
> Speculative read repair mutations shouldn't use read latencies to determine 
> when to send a speculative mutation. It should have it's own value based on 
> mutation latencies.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14694) add latency sample for speculative read repair writes

2018-09-11 Thread Chris Lohfink (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14694?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16611233#comment-16611233
 ] 

Chris Lohfink commented on CASSANDRA-14694:
---

there is a coordinator write latency per table, can it just use that instead?

> add latency sample for speculative read repair writes
> -
>
> Key: CASSANDRA-14694
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14694
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Blake Eggleston
>Assignee: Blake Eggleston
>Priority: Major
> Fix For: 4.0
>
>
> Speculative read repair mutations shouldn't use read latencies to determine 
> when to send a speculative mutation. It should have it's own value based on 
> mutation latencies.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14694) add latency sample for speculative read repair writes

2018-09-11 Thread Benedict (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14694?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16611015#comment-16611015
 ] 

Benedict commented on CASSANDRA-14694:
--

While we're at this patch, could we re-bikeshed the naming of 
speculative_write_threshold?

I had hoped we would call it transient_write_threshold, since the writes - once 
they happen - are no longer speculative.  They're just writes.  Admittedly, 
they're happening 'speculatively' in the expectation we may not reach 
consistency without them (even though we might).

Perhaps we should have a separate statistic for transient writes that includes 
those we may not through 'speculation' but because the failure detector informs 
us we need it to reach consistency, and retain the speculative_write.

Though these repair writes are also speculative, so we might at least need an 
extra component, e.g. speculative_transient_write and speculative_rr_write

I thought this would be a good spot to generally discuss our terminology around 
these kinds of actions, anyway.

> add latency sample for speculative read repair writes
> -
>
> Key: CASSANDRA-14694
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14694
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Blake Eggleston
>Assignee: Blake Eggleston
>Priority: Major
> Fix For: 4.0
>
>
> Speculative read repair mutations shouldn't use read latencies to determine 
> when to send a speculative mutation. It should have it's own value based on 
> mutation latencies.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (CASSANDRA-14716) Protocol frame checksumming options should not be case sensitive

2018-09-11 Thread Sam Tunnicliffe (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14716?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sam Tunnicliffe updated CASSANDRA-14716:

Status: Patch Available  (was: Open)

Patch + unit test: 
[here|https://github.com/beobal/cassandra/commit/f9cda7d55cc2b57ba5905f0635275f569d56d4c9]
||utests||dtests||
|[utests|https://circleci.com/gh/beobal/cassandra/437]|[vnodes|https://circleci.com/gh/beobal/cassandra/436]
 / [no vnodes|https://circleci.com/gh/beobal/cassandra/438]|

> Protocol frame checksumming options should not be case sensitive
> 
>
> Key: CASSANDRA-14716
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14716
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Sam Tunnicliffe
>Assignee: Sam Tunnicliffe
>Priority: Major
> Fix For: 4.0
>
>
> Protocol v5 adds support for checksumming of native protocol frame bodies. 
> The checksum type is negotiated per-connection via the \{{STARTUP}} message, 
> with two types currently supported, Adler32 and CRC32. The mapping of the 
> startup option value requested by the client to a \{{ChecksumType}} should 
> not be case sensitive, but currently it is.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (CASSANDRA-14688) Update protocol spec and class level doc with protocol checksumming details

2018-09-11 Thread Sam Tunnicliffe (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14688?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sam Tunnicliffe updated CASSANDRA-14688:

Status: Patch Available  (was: Open)

[Updated javadoc and protocol 
spec|https://github.com/beobal/cassandra/commit/6cce56fe92e9b469272773ec8df474818ef7c252]

> Update protocol spec and class level doc with protocol checksumming details
> ---
>
> Key: CASSANDRA-14688
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14688
> Project: Cassandra
>  Issue Type: Task
>  Components: Documentation and Website
>Reporter: Sam Tunnicliffe
>Assignee: Sam Tunnicliffe
>Priority: Major
> Fix For: 4.0
>
>
> CASSANDRA-13304 provides an option to add checksumming to the frame body of 
> native protocol messages. The native protocol spec needs to be updated to 
> reflect this ASAP. We should also verify that the javadoc comments describing 
> the on-wire format in 
> {{o.a.c.transport.frame.checksum.ChecksummingTransformer}} are up to date.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Created] (CASSANDRA-14716) Protocol frame checksumming options should not be case sensitive

2018-09-11 Thread Sam Tunnicliffe (JIRA)
Sam Tunnicliffe created CASSANDRA-14716:
---

 Summary: Protocol frame checksumming options should not be case 
sensitive
 Key: CASSANDRA-14716
 URL: https://issues.apache.org/jira/browse/CASSANDRA-14716
 Project: Cassandra
  Issue Type: Bug
Reporter: Sam Tunnicliffe
Assignee: Sam Tunnicliffe
 Fix For: 4.0


Protocol v5 adds support for checksumming of native protocol frame bodies. The 
checksum type is negotiated per-connection via the \{{STARTUP}} message, with 
two types currently supported, Adler32 and CRC32. The mapping of the startup 
option value requested by the client to a \{{ChecksumType}} should not be case 
sensitive, but currently it is.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14685) Incremental repair 4.0 : SSTables remain locked forever if the coordinator dies during streaming

2018-09-11 Thread Alexander Dejanovski (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14685?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610574#comment-16610574
 ] 

Alexander Dejanovski commented on CASSANDRA-14685:
--

[~jasobrown], sure thing, no worries.

For the "pending repair" part, I must add that it doesn't happen when a replica 
node goes down during repair, even if it comes back way after repair is over on 
the coordinator. Shortly after restart, SSTables are correctly released from 
the pending repair.

It's only when the coordinator goes down that replicas remain in pending repair 
state, even after a restart of the Cassandra process on these nodes.

> Incremental repair 4.0 : SSTables remain locked forever if the coordinator 
> dies during streaming 
> -
>
> Key: CASSANDRA-14685
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14685
> Project: Cassandra
>  Issue Type: Bug
>  Components: Repair
>Reporter: Alexander Dejanovski
>Assignee: Jason Brown
>Priority: Critical
>
> The changes in CASSANDRA-9143 modified the way incremental repair performs by 
> applying the following sequence of events : 
>  * Anticompaction is executed on all replicas for all SSTables overlapping 
> the repaired ranges
>  * Anticompacted SSTables are then marked as "Pending repair" and cannot be 
> compacted anymore, nor part of another repair session
>  * Merkle trees are generated and compared
>  * Streaming takes place if needed
>  * Anticompaction is committed and "pending repair" table are marked as 
> repaired if it succeeded, or they are released if the repair session failed.
> If the repair coordinator dies during the streaming phase, *the SSTables on 
> the replicas will remain in "pending repair" state and will never be eligible 
> for repair or compaction*, even after all the nodes in the cluster are 
> restarted. 
> Steps to reproduce (I've used Jason's 13938 branch that fixes streaming 
> errors) : 
> {noformat}
> ccm create inc-repair-issue -v github:jasobrown/13938 -n 3
> # Allow jmx access and remove all rpc_ settings in yaml
> for f in ~/.ccm/inc-repair-issue/node*/conf/cassandra-env.sh;
> do
>   sed -i'' -e 
> 's/com.sun.management.jmxremote.authenticate=true/com.sun.management.jmxremote.authenticate=false/g'
>  $f
> done
> for f in ~/.ccm/inc-repair-issue/node*/conf/cassandra.yaml;
> do
>   grep -v "rpc_" $f > ${f}.tmp
>   cat ${f}.tmp > $f
> done
> ccm start
> {noformat}
> I used [tlp-stress|https://github.com/thelastpickle/tlp-stress] to generate a 
> few 10s of MBs of data (killed it after some time). Obviously 
> cassandra-stress works as well :
> {noformat}
> bin/tlp-stress run BasicTimeSeries -i 1M -p 1M -t 2 --rate 5000  
> --replication "{'class':'SimpleStrategy', 'replication_factor':2}"   
> --compaction "{'class': 'SizeTieredCompactionStrategy'}"   --host 
> 127.0.0.1
> {noformat}
> Flush and delete all SSTables in node1 :
> {noformat}
> ccm node1 nodetool flush
> ccm node1 stop
> rm -f ~/.ccm/inc-repair-issue/node1/data0/tlp_stress/sensor*/*.*
> ccm node1 start{noformat}
> Then throttle streaming throughput to 1MB/s so we have time to take node1 
> down during the streaming phase and run repair:
> {noformat}
> ccm node1 nodetool setstreamthroughput 1
> ccm node2 nodetool setstreamthroughput 1
> ccm node3 nodetool setstreamthroughput 1
> ccm node1 nodetool repair tlp_stress
> {noformat}
> Once streaming starts, shut down node1 and start it again :
> {noformat}
> ccm node1 stop
> ccm node1 start
> {noformat}
> Run repair again :
> {noformat}
> ccm node1 nodetool repair tlp_stress
> {noformat}
> The command will return very quickly, showing that it skipped all sstables :
> {noformat}
> [2018-08-31 19:05:16,292] Repair completed successfully
> [2018-08-31 19:05:16,292] Repair command #1 finished in 2 seconds
> $ ccm node1 nodetool status
> Datacenter: datacenter1
> ===
> Status=Up/Down
> |/ State=Normal/Leaving/Joining/Moving
> --  AddressLoad   Tokens   OwnsHost ID
>Rack
> UN  127.0.0.1  228,64 KiB  256  ?   
> 437dc9cd-b1a1-41a5-961e-cfc99763e29f  rack1
> UN  127.0.0.2  60,09 MiB  256  ?   
> fbcbbdbb-e32a-4716-8230-8ca59aa93e62  rack1
> UN  127.0.0.3  57,59 MiB  256  ?   
> a0b1bcc6-0fad-405a-b0bf-180a0ca31dd0  rack1
> {noformat}
> sstablemetadata will then show that nodes 2 and 3 have SSTables still in 
> "pending repair" state :
> {noformat}
> ~/.ccm/repository/gitCOLONtrunk/tools/bin/sstablemetadata na-4-big-Data.db | 
> grep repair
> SSTable: 
> /Users/adejanovski/.ccm/inc-repair-4.0/node2/data0/tlp_stress/sensor_data-b7375660ad3111e8a0e59357ff9c9bda/na-4-big
> Pending repair: 3844a400-ad33-11e8-b5a7-6b8dd8f31b62
> 

[jira] [Commented] (CASSANDRA-14685) Incremental repair 4.0 : SSTables remain locked forever if the coordinator dies during streaming

2018-09-11 Thread Jason Brown (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14685?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610564#comment-16610564
 ] 

Jason Brown commented on CASSANDRA-14685:
-

[~adejanovski] ughh, missed the update on this. Definitely looks like something 
isn't timing out properly in streaming. I'll start digging into the streaming 
part of this. 

[~bdeggleston], can you comment about this part:

bq. replicas will remain in "pending repair" state and will never be eligible 
for repair or compaction, even after all the nodes in the cluster are 
restarted. 


> Incremental repair 4.0 : SSTables remain locked forever if the coordinator 
> dies during streaming 
> -
>
> Key: CASSANDRA-14685
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14685
> Project: Cassandra
>  Issue Type: Bug
>  Components: Repair
>Reporter: Alexander Dejanovski
>Assignee: Jason Brown
>Priority: Critical
>
> The changes in CASSANDRA-9143 modified the way incremental repair performs by 
> applying the following sequence of events : 
>  * Anticompaction is executed on all replicas for all SSTables overlapping 
> the repaired ranges
>  * Anticompacted SSTables are then marked as "Pending repair" and cannot be 
> compacted anymore, nor part of another repair session
>  * Merkle trees are generated and compared
>  * Streaming takes place if needed
>  * Anticompaction is committed and "pending repair" table are marked as 
> repaired if it succeeded, or they are released if the repair session failed.
> If the repair coordinator dies during the streaming phase, *the SSTables on 
> the replicas will remain in "pending repair" state and will never be eligible 
> for repair or compaction*, even after all the nodes in the cluster are 
> restarted. 
> Steps to reproduce (I've used Jason's 13938 branch that fixes streaming 
> errors) : 
> {noformat}
> ccm create inc-repair-issue -v github:jasobrown/13938 -n 3
> # Allow jmx access and remove all rpc_ settings in yaml
> for f in ~/.ccm/inc-repair-issue/node*/conf/cassandra-env.sh;
> do
>   sed -i'' -e 
> 's/com.sun.management.jmxremote.authenticate=true/com.sun.management.jmxremote.authenticate=false/g'
>  $f
> done
> for f in ~/.ccm/inc-repair-issue/node*/conf/cassandra.yaml;
> do
>   grep -v "rpc_" $f > ${f}.tmp
>   cat ${f}.tmp > $f
> done
> ccm start
> {noformat}
> I used [tlp-stress|https://github.com/thelastpickle/tlp-stress] to generate a 
> few 10s of MBs of data (killed it after some time). Obviously 
> cassandra-stress works as well :
> {noformat}
> bin/tlp-stress run BasicTimeSeries -i 1M -p 1M -t 2 --rate 5000  
> --replication "{'class':'SimpleStrategy', 'replication_factor':2}"   
> --compaction "{'class': 'SizeTieredCompactionStrategy'}"   --host 
> 127.0.0.1
> {noformat}
> Flush and delete all SSTables in node1 :
> {noformat}
> ccm node1 nodetool flush
> ccm node1 stop
> rm -f ~/.ccm/inc-repair-issue/node1/data0/tlp_stress/sensor*/*.*
> ccm node1 start{noformat}
> Then throttle streaming throughput to 1MB/s so we have time to take node1 
> down during the streaming phase and run repair:
> {noformat}
> ccm node1 nodetool setstreamthroughput 1
> ccm node2 nodetool setstreamthroughput 1
> ccm node3 nodetool setstreamthroughput 1
> ccm node1 nodetool repair tlp_stress
> {noformat}
> Once streaming starts, shut down node1 and start it again :
> {noformat}
> ccm node1 stop
> ccm node1 start
> {noformat}
> Run repair again :
> {noformat}
> ccm node1 nodetool repair tlp_stress
> {noformat}
> The command will return very quickly, showing that it skipped all sstables :
> {noformat}
> [2018-08-31 19:05:16,292] Repair completed successfully
> [2018-08-31 19:05:16,292] Repair command #1 finished in 2 seconds
> $ ccm node1 nodetool status
> Datacenter: datacenter1
> ===
> Status=Up/Down
> |/ State=Normal/Leaving/Joining/Moving
> --  AddressLoad   Tokens   OwnsHost ID
>Rack
> UN  127.0.0.1  228,64 KiB  256  ?   
> 437dc9cd-b1a1-41a5-961e-cfc99763e29f  rack1
> UN  127.0.0.2  60,09 MiB  256  ?   
> fbcbbdbb-e32a-4716-8230-8ca59aa93e62  rack1
> UN  127.0.0.3  57,59 MiB  256  ?   
> a0b1bcc6-0fad-405a-b0bf-180a0ca31dd0  rack1
> {noformat}
> sstablemetadata will then show that nodes 2 and 3 have SSTables still in 
> "pending repair" state :
> {noformat}
> ~/.ccm/repository/gitCOLONtrunk/tools/bin/sstablemetadata na-4-big-Data.db | 
> grep repair
> SSTable: 
> /Users/adejanovski/.ccm/inc-repair-4.0/node2/data0/tlp_stress/sensor_data-b7375660ad3111e8a0e59357ff9c9bda/na-4-big
> Pending repair: 3844a400-ad33-11e8-b5a7-6b8dd8f31b62
> {noformat}
> Restarting these nodes wouldn't help either.



--
This message wa

[jira] [Commented] (CASSANDRA-14692) join_ring=false populates wrong value into StorageServiceMB and prevents join by nodetool

2018-09-11 Thread Roland Johann (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14692?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610556#comment-16610556
 ] 

Roland Johann commented on CASSANDRA-14692:
---

{{nodetol status}} on the non joined node shows that it's in state {{UN}}, on 
other nodes it shows {{DN}}.

{quote}and the log output clearly describes the correct command to use 
instead{quote}
means that this won't be marked as bug?


> join_ring=false populates wrong value into StorageServiceMB and prevents join 
> by nodetool
> -
>
> Key: CASSANDRA-14692
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14692
> Project: Cassandra
>  Issue Type: Bug
>  Components: Lifecycle
>Reporter: Roland Johann
>Priority: Major
> Fix For: 3.11.3
>
> Attachments: Bildschirmfoto 2018-09-05 um 17.29.54.png, 
> cassandra1_log, cassandra1_nodetool_gossipinfo, cassandra1_nodetool_status, 
> cassandra2_nodetool_gossipinfo, cassandra2_nodetool_status
>
>
> Restarting a cassandra cluster member with option 
> {{-Dcassandra.join_ring=false}} populates wrong value to its 
> {{StorageServiceMB}} field {{Joined}} which causes the actual trigger to join 
> via {{nodetool join}} to abort due to check if {{Join}} in 
> {{StorageServiceMB}} is true. Via jconsole it's possible as there is no check.
> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/tools/nodetool/Join.java
> {{nodetool status}} also shows that the node is up and in normal node, on the 
> rest of the cluster node status is  {{DN}}. 
> {{nodetool gossipinfo}} states that the non joined node is in gossip state 
> {{hibernate}}.
> Came across this issue while evaluated the problem of zombies to integrate 
> into automation processes and the documentation states
> {quote}To avoid this problem, run {{nodetool repair}} on any restored node 
> before rejoining it to its cluster. 
> {quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14692) join_ring=false populates wrong value into StorageServiceMB and prevents join by nodetool

2018-09-11 Thread Stefan Podkowinski (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14692?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610548#comment-16610548
 ] 

Stefan Podkowinski commented on CASSANDRA-14692:


Ok, I didn't realize you're trying this with a node that has been successfully 
joined the ring before. This is in fact easy to reproduce. No idea why 
{{isJoined}} has been implemented like it is and {{nodetool join}} will not 
work for the described situation. But this doesn't seem to be a regression, and 
the log output clearly describes the correct command to use instead. Have there 
been any other issues related to that, except from the failing {{nodetool 
join}} command?

> join_ring=false populates wrong value into StorageServiceMB and prevents join 
> by nodetool
> -
>
> Key: CASSANDRA-14692
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14692
> Project: Cassandra
>  Issue Type: Bug
>  Components: Lifecycle
>Reporter: Roland Johann
>Priority: Major
> Fix For: 3.11.3
>
> Attachments: Bildschirmfoto 2018-09-05 um 17.29.54.png, 
> cassandra1_log, cassandra1_nodetool_gossipinfo, cassandra1_nodetool_status, 
> cassandra2_nodetool_gossipinfo, cassandra2_nodetool_status
>
>
> Restarting a cassandra cluster member with option 
> {{-Dcassandra.join_ring=false}} populates wrong value to its 
> {{StorageServiceMB}} field {{Joined}} which causes the actual trigger to join 
> via {{nodetool join}} to abort due to check if {{Join}} in 
> {{StorageServiceMB}} is true. Via jconsole it's possible as there is no check.
> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/tools/nodetool/Join.java
> {{nodetool status}} also shows that the node is up and in normal node, on the 
> rest of the cluster node status is  {{DN}}. 
> {{nodetool gossipinfo}} states that the non joined node is in gossip state 
> {{hibernate}}.
> Came across this issue while evaluated the problem of zombies to integrate 
> into automation processes and the documentation states
> {quote}To avoid this problem, run {{nodetool repair}} on any restored node 
> before rejoining it to its cluster. 
> {quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[Cassandra Wiki] Update of "Committers" by JasonBrown

2018-09-11 Thread Apache Wiki
Dear Wiki user,

You have subscribed to a wiki page or wiki category on "Cassandra Wiki" for 
change notification.

The "Committers" page has been changed by JasonBrown:
https://wiki.apache.org/cassandra/Committers?action=diff&rev1=82&rev2=83

  ||Alex Petrov ||February 2017 ||Apple || ||
  ||Joel Knighton ||February 2017 || Datastax || ||
  ||Philip Thompson ||June 2017 || Datastax || ||
- ||Jat Zhuang ||February 2018 || Uber || ||
+ ||Jay Zhuang ||February 2018 || Uber || ||
  ||Chris Lohfink ||August 2018 || Apple || ||
  
  {{https://c.statcounter.com/9397521/0/fe557aad/1/|stats}}

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



[Cassandra Wiki] Update of "Committers" by JasonBrown

2018-09-11 Thread Apache Wiki
Dear Wiki user,

You have subscribed to a wiki page or wiki category on "Cassandra Wiki" for 
change notification.

The "Committers" page has been changed by JasonBrown:
https://wiki.apache.org/cassandra/Committers?action=diff&rev1=81&rev2=82

  ||Alex Petrov ||February 2017 ||Apple || ||
  ||Joel Knighton ||February 2017 || Datastax || ||
  ||Philip Thompson ||June 2017 || Datastax || ||
+ ||Jat Zhuang ||February 2018 || Uber || ||
+ ||Chris Lohfink ||August 2018 || Apple || ||
  
  {{https://c.statcounter.com/9397521/0/fe557aad/1/|stats}}
  

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



[Cassandra Wiki] Update of "Committers" by JasonBrown

2018-09-11 Thread Apache Wiki
Dear Wiki user,

You have subscribed to a wiki page or wiki category on "Cassandra Wiki" for 
change notification.

The "Committers" page has been changed by JasonBrown:
https://wiki.apache.org/cassandra/Committers?action=diff&rev1=80&rev2=81

  ||Carl Yeksigian ||Jan 2016 ||Datastax ||Also a 
[[http://thrift.apache.org|Thrift]] committer ||
  ||Stefania Alborghetti ||Apr 2016 ||Datastax || ||
  ||Jeff Jirsa ||June 2016 ||Apple|| PMC member ||
- ||Nate !McCall ||June 2016 ||Last Pickle|| Project chair ||
+ ||Nate !McCall ||June 2016 ||Apple|| Project chair ||
  ||Jake Farrell ||June 2016 || || PMC member ||
  ||Michael Shuler ||June 2016 ||Datastax || PMC member, Release manager ||
  ||Michael Semb Wever ||June 2016 || Last Pickle || ||
@@ -40, +40 @@

  ||Branimir Lambov ||November 2016 ||Datastax || ||
  ||Paulo Motta || November 2016 ||Datastax || ||
  ||Sankalp Kohli || November 2016 ||Apple || PMC member ||
- ||Stefan Podkowinski ||February 2017 ||1&1 || ||
+ ||Stefan Podkowinski ||February 2017 ||1&1 ||PMC member ||
  ||Ariel Weisberg ||February 2017 ||Apple || ||
- ||Blake Eggleston ||February 2017 ||Apple || ||
+ ||Blake Eggleston ||February 2017 ||Apple ||PMC member ||
- ||Alex Petrov ||February 2017 ||Datastax || ||
+ ||Alex Petrov ||February 2017 ||Apple || ||
  ||Joel Knighton ||February 2017 || Datastax || ||
  ||Philip Thompson ||June 2017 || Datastax || ||
  

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



[jira] [Comment Edited] (CASSANDRA-14692) join_ring=false populates wrong value into StorageServiceMB and prevents join by nodetool

2018-09-11 Thread Roland Johann (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14692?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610524#comment-16610524
 ] 

Roland Johann edited comment on CASSANDRA-14692 at 9/11/18 12:26 PM:
-

Set this option at {{jvm.options}} and it shows up as part of process via {{ps 
aux}}.

Attached {{cassandra_log}} also shows that the property has been interpreted 
correctly:

{quote}INFO  [main] 2018-09-05 15:13:21,224 StorageService.java:694 - Not 
joining ring as requested. Use JMX (StorageService->joinRing()) to initiate 
ring joining{quote}

 The relevant code is in this line: 
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L1192
{code:java}
public boolean isJoined()
{
return tokenMetadata.isMember(FBUtilities.getBroadcastAddressAndPort()) && 
!isSurveyMode;
}
{code}
So cassandra internally is in state joined when connected at gossip without 
taking gossip state into account. There is a member {{joined}} which will be 
set when joning the ring, which can/should be used to expose the state: 
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L895


was (Author: rolandjohann):
Set this option at {{jvm.options}} and it shows up as part of process via {{ps 
aux}}.

Attached {{cassandra_log}} also shows that the property has been interpreted 
correctly:

{quote}INFO  [main] 2018-09-05 15:13:21,224 StorageService.java:694 - Not 
joining ring as requested. Use JMX (StorageService->joinRing()) to initiate 
ring joining{quote}

 The relevant code is in this line: 
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L1192
{code:java}
public boolean isJoined()
{
return tokenMetadata.isMember(FBUtilities.getBroadcastAddressAndPort()) && 
!isSurveyMode;
}
{code}
So cassandra internally is in state joined when connected at gossip without 
taking gossip state into account. There is a member {{joined}} which will be 
set when joning the ring, which can be used to expose the state: 
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L895

> join_ring=false populates wrong value into StorageServiceMB and prevents join 
> by nodetool
> -
>
> Key: CASSANDRA-14692
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14692
> Project: Cassandra
>  Issue Type: Bug
>  Components: Lifecycle
>Reporter: Roland Johann
>Priority: Major
> Fix For: 3.11.3
>
> Attachments: Bildschirmfoto 2018-09-05 um 17.29.54.png, 
> cassandra1_log, cassandra1_nodetool_gossipinfo, cassandra1_nodetool_status, 
> cassandra2_nodetool_gossipinfo, cassandra2_nodetool_status
>
>
> Restarting a cassandra cluster member with option 
> {{-Dcassandra.join_ring=false}} populates wrong value to its 
> {{StorageServiceMB}} field {{Joined}} which causes the actual trigger to join 
> via {{nodetool join}} to abort due to check if {{Join}} in 
> {{StorageServiceMB}} is true. Via jconsole it's possible as there is no check.
> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/tools/nodetool/Join.java
> {{nodetool status}} also shows that the node is up and in normal node, on the 
> rest of the cluster node status is  {{DN}}. 
> {{nodetool gossipinfo}} states that the non joined node is in gossip state 
> {{hibernate}}.
> Came across this issue while evaluated the problem of zombies to integrate 
> into automation processes and the documentation states
> {quote}To avoid this problem, run {{nodetool repair}} on any restored node 
> before rejoining it to its cluster. 
> {quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Comment Edited] (CASSANDRA-14692) join_ring=false populates wrong value into StorageServiceMB and prevents join by nodetool

2018-09-11 Thread Roland Johann (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14692?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610524#comment-16610524
 ] 

Roland Johann edited comment on CASSANDRA-14692 at 9/11/18 12:25 PM:
-

Set this option at {{jvm.options}} and it shows up as part of process via {{ps 
aux}}.

Attached {{cassandra_log}} also shows that the property has been interpreted 
correctly:

{quote}INFO  [main] 2018-09-05 15:13:21,224 StorageService.java:694 - Not 
joining ring as requested. Use JMX (StorageService->joinRing()) to initiate 
ring joining{quote}

 The relevant code is in this line: 
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L1192
{code:java}
public boolean isJoined()
{
return tokenMetadata.isMember(FBUtilities.getBroadcastAddressAndPort()) && 
!isSurveyMode;
}
{code}
So cassandra internally is in state joined when connected at gossip without 
taking gossip state into account. There is a member {{joined}} which will be 
set when joning the ring, which can be used to expose the state: 
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L895


was (Author: rolandjohann):
Set this option at {{jvm.options}} and it shows up as part of process via {{ps 
aux}}.

Attached {{cassandra_log}} also shows that the property has been interpreted 
correctly:

{quote}INFO  [main] 2018-09-05 15:13:21,224 StorageService.java:694 - Not 
joining ring as requested. Use JMX (StorageService->joinRing()) to initiate 
ring joining{quote}

 The relevant code is in this line: 
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L1192
{code:java}
public boolean isJoined()
{
return tokenMetadata.isMember(FBUtilities.getBroadcastAddressAndPort()) 
&& !isSurveyMode;
}
{code}
So cassandra internally is in state joined when connected at gossip without 
taking gossip state into account. There is a member {{joined}} which will be 
set when joning the ring, which can be used to expose the state: 
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L895

> join_ring=false populates wrong value into StorageServiceMB and prevents join 
> by nodetool
> -
>
> Key: CASSANDRA-14692
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14692
> Project: Cassandra
>  Issue Type: Bug
>  Components: Lifecycle
>Reporter: Roland Johann
>Priority: Major
> Fix For: 3.11.3
>
> Attachments: Bildschirmfoto 2018-09-05 um 17.29.54.png, 
> cassandra1_log, cassandra1_nodetool_gossipinfo, cassandra1_nodetool_status, 
> cassandra2_nodetool_gossipinfo, cassandra2_nodetool_status
>
>
> Restarting a cassandra cluster member with option 
> {{-Dcassandra.join_ring=false}} populates wrong value to its 
> {{StorageServiceMB}} field {{Joined}} which causes the actual trigger to join 
> via {{nodetool join}} to abort due to check if {{Join}} in 
> {{StorageServiceMB}} is true. Via jconsole it's possible as there is no check.
> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/tools/nodetool/Join.java
> {{nodetool status}} also shows that the node is up and in normal node, on the 
> rest of the cluster node status is  {{DN}}. 
> {{nodetool gossipinfo}} states that the non joined node is in gossip state 
> {{hibernate}}.
> Came across this issue while evaluated the problem of zombies to integrate 
> into automation processes and the documentation states
> {quote}To avoid this problem, run {{nodetool repair}} on any restored node 
> before rejoining it to its cluster. 
> {quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14692) join_ring=false populates wrong value into StorageServiceMB and prevents join by nodetool

2018-09-11 Thread Roland Johann (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14692?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610524#comment-16610524
 ] 

Roland Johann commented on CASSANDRA-14692:
---

Set this option at {{jvm.options}} and it shows up as part of process via {{ps 
aux}}.

Attached {{cassandra_log}} also shows that the property has been interpreted 
correctly:

{quote}INFO  [main] 2018-09-05 15:13:21,224 StorageService.java:694 - Not 
joining ring as requested. Use JMX (StorageService->joinRing()) to initiate 
ring joining{quote}

 The relevant code is in this line: 
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L1192
{code:java}
public boolean isJoined()
{
return tokenMetadata.isMember(FBUtilities.getBroadcastAddressAndPort()) 
&& !isSurveyMode;
}
{code}
So cassandra internally is in state joined when connected at gossip without 
taking gossip state into account. There is a member {{joined}} which will be 
set when joning the ring, which can be used to expose the state: 
https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L895

> join_ring=false populates wrong value into StorageServiceMB and prevents join 
> by nodetool
> -
>
> Key: CASSANDRA-14692
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14692
> Project: Cassandra
>  Issue Type: Bug
>  Components: Lifecycle
>Reporter: Roland Johann
>Priority: Major
> Fix For: 3.11.3
>
> Attachments: Bildschirmfoto 2018-09-05 um 17.29.54.png, 
> cassandra1_log, cassandra1_nodetool_gossipinfo, cassandra1_nodetool_status, 
> cassandra2_nodetool_gossipinfo, cassandra2_nodetool_status
>
>
> Restarting a cassandra cluster member with option 
> {{-Dcassandra.join_ring=false}} populates wrong value to its 
> {{StorageServiceMB}} field {{Joined}} which causes the actual trigger to join 
> via {{nodetool join}} to abort due to check if {{Join}} in 
> {{StorageServiceMB}} is true. Via jconsole it's possible as there is no check.
> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/tools/nodetool/Join.java
> {{nodetool status}} also shows that the node is up and in normal node, on the 
> rest of the cluster node status is  {{DN}}. 
> {{nodetool gossipinfo}} states that the non joined node is in gossip state 
> {{hibernate}}.
> Came across this issue while evaluated the problem of zombies to integrate 
> into automation processes and the documentation states
> {quote}To avoid this problem, run {{nodetool repair}} on any restored node 
> before rejoining it to its cluster. 
> {quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14692) join_ring=false populates wrong value into StorageServiceMB and prevents join by nodetool

2018-09-11 Thread Stefan Podkowinski (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14692?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610506#comment-16610506
 ] 

Stefan Podkowinski commented on CASSANDRA-14692:


Can you please double check that the {{-Dcassandra.join_ring=false}} option is 
really set correctly and will show up as part of the java process parameters? 

> join_ring=false populates wrong value into StorageServiceMB and prevents join 
> by nodetool
> -
>
> Key: CASSANDRA-14692
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14692
> Project: Cassandra
>  Issue Type: Bug
>  Components: Lifecycle
>Reporter: Roland Johann
>Priority: Major
> Fix For: 3.11.3
>
> Attachments: Bildschirmfoto 2018-09-05 um 17.29.54.png, 
> cassandra1_log, cassandra1_nodetool_gossipinfo, cassandra1_nodetool_status, 
> cassandra2_nodetool_gossipinfo, cassandra2_nodetool_status
>
>
> Restarting a cassandra cluster member with option 
> {{-Dcassandra.join_ring=false}} populates wrong value to its 
> {{StorageServiceMB}} field {{Joined}} which causes the actual trigger to join 
> via {{nodetool join}} to abort due to check if {{Join}} in 
> {{StorageServiceMB}} is true. Via jconsole it's possible as there is no check.
> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/tools/nodetool/Join.java
> {{nodetool status}} also shows that the node is up and in normal node, on the 
> rest of the cluster node status is  {{DN}}. 
> {{nodetool gossipinfo}} states that the non joined node is in gossip state 
> {{hibernate}}.
> Came across this issue while evaluated the problem of zombies to integrate 
> into automation processes and the documentation states
> {quote}To avoid this problem, run {{nodetool repair}} on any restored node 
> before rejoining it to its cluster. 
> {quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



cassandra git commit: Ninja cleanup: remove obsoleted python drivers

2018-09-11 Thread spod
Repository: cassandra
Updated Branches:
  refs/heads/trunk f424e03a4 -> 2886cac38


Ninja cleanup: remove obsoleted python drivers


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2886cac3
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2886cac3
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2886cac3

Branch: refs/heads/trunk
Commit: 2886cac382891c6ca981aa57ec9f8b2ba672b752
Parents: f424e03
Author: Stefan Podkowinski 
Authored: Tue Sep 11 13:59:43 2018 +0200
Committer: Stefan Podkowinski 
Committed: Tue Sep 11 14:00:17 2018 +0200

--
 ...driver-internal-only-3.12.0.post0-00f6f77e.zip | Bin 265193 -> 0 bytes
 1 file changed, 0 insertions(+), 0 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2886cac3/lib/cassandra-driver-internal-only-3.12.0.post0-00f6f77e.zip
--
diff --git a/lib/cassandra-driver-internal-only-3.12.0.post0-00f6f77e.zip 
b/lib/cassandra-driver-internal-only-3.12.0.post0-00f6f77e.zip
deleted file mode 100644
index e44da16..000
Binary files a/lib/cassandra-driver-internal-only-3.12.0.post0-00f6f77e.zip and 
/dev/null differ


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



[jira] [Commented] (CASSANDRA-14691) Cassandra 2.1 backport - The JVM should exit if jmx fails to bind

2018-09-11 Thread Thomas Steinmaurer (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610483#comment-16610483
 ] 

Thomas Steinmaurer commented on CASSANDRA-14691:


Well, sure, but how does this ticket about corruption e.g. compares to 
CASSANDRA-14284 included in 2.1.21 (corruption vs. crash)? Thought there might 
be e.g. 2.1.22 anyhow ... Anyway. I will now stop bothering. :-)

> Cassandra 2.1 backport - The JVM should exit if jmx fails to bind
> -
>
> Key: CASSANDRA-14691
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14691
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Reporter: Thomas Steinmaurer
>Priority: Major
>  Labels: lhf
> Fix For: 2.1.x
>
>
> If you are already running a cassandra instance, but for some reason try to 
> start another one, this happens:
> {noformat}
> INFO  20:57:09 JNA mlockall successful
> WARN  20:57:09 JMX is not enabled to receive remote connections. Please see 
> cassandra-env.sh for more info.
> ERROR 20:57:10 Error starting local jmx server:
> java.rmi.server.ExportException: Port already in use: 7199; nested exception 
> is:
> java.net.BindException: Address already in use
> at sun.rmi.transport.tcp.TCPTransport.listen(TCPTransport.java:340) 
> ~[na:1.7.0_76]
> at 
> sun.rmi.transport.tcp.TCPTransport.exportObject(TCPTransport.java:248) 
> ~[na:1.7.0_76]
> at 
> sun.rmi.transport.tcp.TCPEndpoint.exportObject(TCPEndpoint.java:411) 
> ~[na:1.7.0_76]
> at sun.rmi.transport.LiveRef.exportObject(LiveRef.java:147) 
> ~[na:1.7.0_76]
> at 
> sun.rmi.server.UnicastServerRef.exportObject(UnicastServerRef.java:207) 
> ~[na:1.7.0_76]
> at sun.rmi.registry.RegistryImpl.setup(RegistryImpl.java:122) 
> ~[na:1.7.0_76]
> at sun.rmi.registry.RegistryImpl.(RegistryImpl.java:98) 
> ~[na:1.7.0_76]
> at 
> java.rmi.registry.LocateRegistry.createRegistry(LocateRegistry.java:239) 
> ~[na:1.7.0_76]
> at 
> org.apache.cassandra.service.CassandraDaemon.maybeInitJmx(CassandraDaemon.java:100)
>  [main/:na]
> at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:222) 
> [main/:na]
> at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:564)
>  [main/:na]
> at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:653) 
> [main/:na]
> Caused by: java.net.BindException: Address already in use
> at java.net.PlainSocketImpl.socketBind(Native Method) ~[na:1.7.0_76]
> at 
> java.net.AbstractPlainSocketImpl.bind(AbstractPlainSocketImpl.java:376) 
> ~[na:1.7.0_76]
> at java.net.ServerSocket.bind(ServerSocket.java:376) ~[na:1.7.0_76]
> at java.net.ServerSocket.(ServerSocket.java:237) ~[na:1.7.0_76]
> at 
> javax.net.DefaultServerSocketFactory.createServerSocket(ServerSocketFactory.java:231)
>  ~[na:1.7.0_76]
> at 
> org.apache.cassandra.utils.RMIServerSocketFactoryImpl.createServerSocket(RMIServerSocketFactoryImpl.java:13)
>  ~[main/:na]
> at 
> sun.rmi.transport.tcp.TCPEndpoint.newServerSocket(TCPEndpoint.java:666) 
> ~[na:1.7.0_76]
> at sun.rmi.transport.tcp.TCPTransport.listen(TCPTransport.java:329) 
> ~[na:1.7.0_76]
> ... 11 common frames omitted
> {noformat}
> However the startup continues, and ends up replaying commitlogs, which is 
> probably not a good thing.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14691) Cassandra 2.1 backport - The JVM should exit if jmx fails to bind

2018-09-11 Thread Stefan Podkowinski (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610452#comment-16610452
 ] 

Stefan Podkowinski commented on CASSANDRA-14691:


For me, critical also implies that we'd have to cut a new 2.1 release for such 
a patch and urge users to update to the new version, and I don't see how that 
would be justified in this case.

> Cassandra 2.1 backport - The JVM should exit if jmx fails to bind
> -
>
> Key: CASSANDRA-14691
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14691
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Reporter: Thomas Steinmaurer
>Priority: Major
>  Labels: lhf
> Fix For: 2.1.x
>
>
> If you are already running a cassandra instance, but for some reason try to 
> start another one, this happens:
> {noformat}
> INFO  20:57:09 JNA mlockall successful
> WARN  20:57:09 JMX is not enabled to receive remote connections. Please see 
> cassandra-env.sh for more info.
> ERROR 20:57:10 Error starting local jmx server:
> java.rmi.server.ExportException: Port already in use: 7199; nested exception 
> is:
> java.net.BindException: Address already in use
> at sun.rmi.transport.tcp.TCPTransport.listen(TCPTransport.java:340) 
> ~[na:1.7.0_76]
> at 
> sun.rmi.transport.tcp.TCPTransport.exportObject(TCPTransport.java:248) 
> ~[na:1.7.0_76]
> at 
> sun.rmi.transport.tcp.TCPEndpoint.exportObject(TCPEndpoint.java:411) 
> ~[na:1.7.0_76]
> at sun.rmi.transport.LiveRef.exportObject(LiveRef.java:147) 
> ~[na:1.7.0_76]
> at 
> sun.rmi.server.UnicastServerRef.exportObject(UnicastServerRef.java:207) 
> ~[na:1.7.0_76]
> at sun.rmi.registry.RegistryImpl.setup(RegistryImpl.java:122) 
> ~[na:1.7.0_76]
> at sun.rmi.registry.RegistryImpl.(RegistryImpl.java:98) 
> ~[na:1.7.0_76]
> at 
> java.rmi.registry.LocateRegistry.createRegistry(LocateRegistry.java:239) 
> ~[na:1.7.0_76]
> at 
> org.apache.cassandra.service.CassandraDaemon.maybeInitJmx(CassandraDaemon.java:100)
>  [main/:na]
> at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:222) 
> [main/:na]
> at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:564)
>  [main/:na]
> at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:653) 
> [main/:na]
> Caused by: java.net.BindException: Address already in use
> at java.net.PlainSocketImpl.socketBind(Native Method) ~[na:1.7.0_76]
> at 
> java.net.AbstractPlainSocketImpl.bind(AbstractPlainSocketImpl.java:376) 
> ~[na:1.7.0_76]
> at java.net.ServerSocket.bind(ServerSocket.java:376) ~[na:1.7.0_76]
> at java.net.ServerSocket.(ServerSocket.java:237) ~[na:1.7.0_76]
> at 
> javax.net.DefaultServerSocketFactory.createServerSocket(ServerSocketFactory.java:231)
>  ~[na:1.7.0_76]
> at 
> org.apache.cassandra.utils.RMIServerSocketFactoryImpl.createServerSocket(RMIServerSocketFactoryImpl.java:13)
>  ~[main/:na]
> at 
> sun.rmi.transport.tcp.TCPEndpoint.newServerSocket(TCPEndpoint.java:666) 
> ~[na:1.7.0_76]
> at sun.rmi.transport.tcp.TCPTransport.listen(TCPTransport.java:329) 
> ~[na:1.7.0_76]
> ... 11 common frames omitted
> {noformat}
> However the startup continues, and ends up replaying commitlogs, which is 
> probably not a good thing.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14691) Cassandra 2.1 backport - The JVM should exit if jmx fails to bind

2018-09-11 Thread Thomas Steinmaurer (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610432#comment-16610432
 ] 

Thomas Steinmaurer commented on CASSANDRA-14691:


[~spo...@gmail.com], thanks for the feedback. So, potential corruption caused 
by this does not qualify as critical?

> Cassandra 2.1 backport - The JVM should exit if jmx fails to bind
> -
>
> Key: CASSANDRA-14691
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14691
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Reporter: Thomas Steinmaurer
>Priority: Major
>  Labels: lhf
> Fix For: 2.1.x
>
>
> If you are already running a cassandra instance, but for some reason try to 
> start another one, this happens:
> {noformat}
> INFO  20:57:09 JNA mlockall successful
> WARN  20:57:09 JMX is not enabled to receive remote connections. Please see 
> cassandra-env.sh for more info.
> ERROR 20:57:10 Error starting local jmx server:
> java.rmi.server.ExportException: Port already in use: 7199; nested exception 
> is:
> java.net.BindException: Address already in use
> at sun.rmi.transport.tcp.TCPTransport.listen(TCPTransport.java:340) 
> ~[na:1.7.0_76]
> at 
> sun.rmi.transport.tcp.TCPTransport.exportObject(TCPTransport.java:248) 
> ~[na:1.7.0_76]
> at 
> sun.rmi.transport.tcp.TCPEndpoint.exportObject(TCPEndpoint.java:411) 
> ~[na:1.7.0_76]
> at sun.rmi.transport.LiveRef.exportObject(LiveRef.java:147) 
> ~[na:1.7.0_76]
> at 
> sun.rmi.server.UnicastServerRef.exportObject(UnicastServerRef.java:207) 
> ~[na:1.7.0_76]
> at sun.rmi.registry.RegistryImpl.setup(RegistryImpl.java:122) 
> ~[na:1.7.0_76]
> at sun.rmi.registry.RegistryImpl.(RegistryImpl.java:98) 
> ~[na:1.7.0_76]
> at 
> java.rmi.registry.LocateRegistry.createRegistry(LocateRegistry.java:239) 
> ~[na:1.7.0_76]
> at 
> org.apache.cassandra.service.CassandraDaemon.maybeInitJmx(CassandraDaemon.java:100)
>  [main/:na]
> at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:222) 
> [main/:na]
> at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:564)
>  [main/:na]
> at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:653) 
> [main/:na]
> Caused by: java.net.BindException: Address already in use
> at java.net.PlainSocketImpl.socketBind(Native Method) ~[na:1.7.0_76]
> at 
> java.net.AbstractPlainSocketImpl.bind(AbstractPlainSocketImpl.java:376) 
> ~[na:1.7.0_76]
> at java.net.ServerSocket.bind(ServerSocket.java:376) ~[na:1.7.0_76]
> at java.net.ServerSocket.(ServerSocket.java:237) ~[na:1.7.0_76]
> at 
> javax.net.DefaultServerSocketFactory.createServerSocket(ServerSocketFactory.java:231)
>  ~[na:1.7.0_76]
> at 
> org.apache.cassandra.utils.RMIServerSocketFactoryImpl.createServerSocket(RMIServerSocketFactoryImpl.java:13)
>  ~[main/:na]
> at 
> sun.rmi.transport.tcp.TCPEndpoint.newServerSocket(TCPEndpoint.java:666) 
> ~[na:1.7.0_76]
> at sun.rmi.transport.tcp.TCPTransport.listen(TCPTransport.java:329) 
> ~[na:1.7.0_76]
> ... 11 common frames omitted
> {noformat}
> However the startup continues, and ends up replaying commitlogs, which is 
> probably not a good thing.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (CASSANDRA-14701) Cleanup (and other) compaction type(s) not counted in compaction remaining time

2018-09-11 Thread Stefan Podkowinski (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14701?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Stefan Podkowinski updated CASSANDRA-14701:
---
Priority: Major  (was: Critical)

> Cleanup (and other) compaction type(s) not counted in compaction remaining 
> time
> ---
>
> Key: CASSANDRA-14701
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14701
> Project: Cassandra
>  Issue Type: Bug
>  Components: Observability
>Reporter: Thomas Steinmaurer
>Priority: Major
>
> Opened a ticket, as discussed in user list.
> Looks like compaction remaining time only includes compactions of type 
> COMPACTION and other compaction types like cleanup etc. aren't part of the 
> estimation calculation.
> E.g. from one of our environments:
> {noformat}
> nodetool compactionstats -H
> pending tasks: 1
>compaction type   keyspace   table   completed totalunit   
> progress
>CleanupXXX YYY   908.16 GB   1.13 TB   bytes   
>   78.63%
> Active compaction remaining time :   0h00m00s
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14701) Cleanup (and other) compaction type(s) not counted in compaction remaining time

2018-09-11 Thread Stefan Podkowinski (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14701?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610411#comment-16610411
 ] 

Stefan Podkowinski commented on CASSANDRA-14701:


Link to [cassandra-user 
thread|https://lists.apache.org/thread.html/6f355d1c2258478dbd7ea3e7960a03ed480daa4cab91d31e71de1000@%3Cuser.cassandra.apache.org%3E].

> Cleanup (and other) compaction type(s) not counted in compaction remaining 
> time
> ---
>
> Key: CASSANDRA-14701
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14701
> Project: Cassandra
>  Issue Type: Bug
>  Components: Observability
>Reporter: Thomas Steinmaurer
>Priority: Critical
>
> Opened a ticket, as discussed in user list.
> Looks like compaction remaining time only includes compactions of type 
> COMPACTION and other compaction types like cleanup etc. aren't part of the 
> estimation calculation.
> E.g. from one of our environments:
> {noformat}
> nodetool compactionstats -H
> pending tasks: 1
>compaction type   keyspace   table   completed totalunit   
> progress
>CleanupXXX YYY   908.16 GB   1.13 TB   bytes   
>   78.63%
> Active compaction remaining time :   0h00m00s
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (CASSANDRA-14709) Global configuration parameter to reject increment repair and allow full repair only

2018-09-11 Thread Stefan Podkowinski (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14709?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Stefan Podkowinski updated CASSANDRA-14709:
---
Issue Type: Wish  (was: Bug)

> Global configuration parameter to reject increment repair and allow full 
> repair only
> 
>
> Key: CASSANDRA-14709
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14709
> Project: Cassandra
>  Issue Type: Wish
>Reporter: Thomas Steinmaurer
>Priority: Major
> Fix For: 2.2.x, 3.0.x, 3.11.x, 4.0.x
>
>
> We are running Cassandra in AWS and On-Premise at customer sites, currently 
> 2.1 in production with 3.0/3.11 in pre-production stages including loadtest.
> In a migration path from 2.1 to 3.11.x, I’m afraid that at some point in time 
> we end up in incremental repairs being enabled / ran a first time 
> unintentionally, cause:
> a) A lot of online resources / examples do not use the _-full_ command-line 
> option available since 2.2 (?)
> b) Our internal (support) tickets of course also state nodetool repair 
> command without the -full option, as these examples are for 2.1
> Especially for On-Premise customers (with less control than with our AWS 
> deployments), this asks a bit for getting out-of-control once we have 3.11 
> out and nodetool repair being run without the -full command-line option.
> With troubles incremental repair are introducing and incremental being the 
> default since 2.2 (?), what do you think about a JVM system property, 
> cassandra.yaml setting or whatever … to basically let the cluster 
> administrator chose if incremental repairs are allowed or not? I know, such a 
> flag still can be flipped then (by the customer), but as a first safety stage 
> possibly sufficient enough.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Resolved] (CASSANDRA-14691) Cassandra 2.1 backport - The JVM should exit if jmx fails to bind

2018-09-11 Thread Stefan Podkowinski (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14691?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Stefan Podkowinski resolved CASSANDRA-14691.

Resolution: Won't Fix

Not a critical bug, which is the acceptance criteria for 2.1, sorry.

> Cassandra 2.1 backport - The JVM should exit if jmx fails to bind
> -
>
> Key: CASSANDRA-14691
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14691
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Reporter: Thomas Steinmaurer
>Priority: Major
>  Labels: lhf
> Fix For: 2.1.x
>
>
> If you are already running a cassandra instance, but for some reason try to 
> start another one, this happens:
> {noformat}
> INFO  20:57:09 JNA mlockall successful
> WARN  20:57:09 JMX is not enabled to receive remote connections. Please see 
> cassandra-env.sh for more info.
> ERROR 20:57:10 Error starting local jmx server:
> java.rmi.server.ExportException: Port already in use: 7199; nested exception 
> is:
> java.net.BindException: Address already in use
> at sun.rmi.transport.tcp.TCPTransport.listen(TCPTransport.java:340) 
> ~[na:1.7.0_76]
> at 
> sun.rmi.transport.tcp.TCPTransport.exportObject(TCPTransport.java:248) 
> ~[na:1.7.0_76]
> at 
> sun.rmi.transport.tcp.TCPEndpoint.exportObject(TCPEndpoint.java:411) 
> ~[na:1.7.0_76]
> at sun.rmi.transport.LiveRef.exportObject(LiveRef.java:147) 
> ~[na:1.7.0_76]
> at 
> sun.rmi.server.UnicastServerRef.exportObject(UnicastServerRef.java:207) 
> ~[na:1.7.0_76]
> at sun.rmi.registry.RegistryImpl.setup(RegistryImpl.java:122) 
> ~[na:1.7.0_76]
> at sun.rmi.registry.RegistryImpl.(RegistryImpl.java:98) 
> ~[na:1.7.0_76]
> at 
> java.rmi.registry.LocateRegistry.createRegistry(LocateRegistry.java:239) 
> ~[na:1.7.0_76]
> at 
> org.apache.cassandra.service.CassandraDaemon.maybeInitJmx(CassandraDaemon.java:100)
>  [main/:na]
> at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:222) 
> [main/:na]
> at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:564)
>  [main/:na]
> at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:653) 
> [main/:na]
> Caused by: java.net.BindException: Address already in use
> at java.net.PlainSocketImpl.socketBind(Native Method) ~[na:1.7.0_76]
> at 
> java.net.AbstractPlainSocketImpl.bind(AbstractPlainSocketImpl.java:376) 
> ~[na:1.7.0_76]
> at java.net.ServerSocket.bind(ServerSocket.java:376) ~[na:1.7.0_76]
> at java.net.ServerSocket.(ServerSocket.java:237) ~[na:1.7.0_76]
> at 
> javax.net.DefaultServerSocketFactory.createServerSocket(ServerSocketFactory.java:231)
>  ~[na:1.7.0_76]
> at 
> org.apache.cassandra.utils.RMIServerSocketFactoryImpl.createServerSocket(RMIServerSocketFactoryImpl.java:13)
>  ~[main/:na]
> at 
> sun.rmi.transport.tcp.TCPEndpoint.newServerSocket(TCPEndpoint.java:666) 
> ~[na:1.7.0_76]
> at sun.rmi.transport.tcp.TCPTransport.listen(TCPTransport.java:329) 
> ~[na:1.7.0_76]
> ... 11 common frames omitted
> {noformat}
> However the startup continues, and ends up replaying commitlogs, which is 
> probably not a good thing.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14703) Document Apache Cassandra Backup Operations

2018-09-11 Thread Stefan Podkowinski (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14703?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610359#comment-16610359
 ] 

Stefan Podkowinski commented on CASSANDRA-14703:


Already looks promising; please don't forget to confirm "Submit Patch", once 
you're done!

> Document Apache Cassandra Backup Operations
> ---
>
> Key: CASSANDRA-14703
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14703
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Documentation and Website
>Reporter: pedro vidigal
>Priority: Major
> Attachments: backups.rst
>
>
> The documentation for the Backup Operations is missing in on the 
> documentation site 
> ([https://github.com/apache/cassandra/blob/trunk/doc/source/operating/backups.rst)]
>  
> Branch with the changes: 
> https://github.com/vidigalp/cassandra/tree/documentation/backup-strategies



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14702) Cassandra Write failed even when the required nodes to Ack(consistency) are up.

2018-09-11 Thread Rohit Singh (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14702?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610344#comment-16610344
 ] 

Rohit Singh commented on CASSANDRA-14702:
-

We have Batchstatement which inserts in DB and while batch operation was going 
on there was update being performed on nodes one after the other. During this 
cassandra rejected the batchstatement with writetimeoutexception mentioned in 
description of bug.

> Cassandra Write failed even when the required nodes to Ack(consistency) are 
> up.
> ---
>
> Key: CASSANDRA-14702
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14702
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Rohit Singh
>Priority: Blocker
>
> Hi,
> We have following configuration in our project for cassandra. 
> Total nodes in Cluster-5
> Replication Factor- 3
> Consistency- LOCAL_QUORUM
> We get the writetimeout exception from cassandra even when 2 nodes are up and 
> why does stack trace says that 3 replica were required when consistency is 2?
> Below is the exception we got:-
> com.datastax.driver.core.exceptions.WriteTimeoutException: Cassandra timeout 
> during write query at consistency LOCAL_QUORUM (3 replica were required but 
> only 2 acknowledged the write)
>  at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:59)
>  at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:37)
>  at com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:289)
>  at com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:269)
>  at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:88)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Commented] (CASSANDRA-14702) Cassandra Write failed even when the required nodes to Ack(consistency) are up.

2018-09-11 Thread Stefan Podkowinski (JIRA)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-14702?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610335#comment-16610335
 ] 

Stefan Podkowinski commented on CASSANDRA-14702:


What are the steps to reproduce the described issue?

> Cassandra Write failed even when the required nodes to Ack(consistency) are 
> up.
> ---
>
> Key: CASSANDRA-14702
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14702
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Rohit Singh
>Priority: Blocker
>
> Hi,
> We have following configuration in our project for cassandra. 
> Total nodes in Cluster-5
> Replication Factor- 3
> Consistency- LOCAL_QUORUM
> We get the writetimeout exception from cassandra even when 2 nodes are up and 
> why does stack trace says that 3 replica were required when consistency is 2?
> Below is the exception we got:-
> com.datastax.driver.core.exceptions.WriteTimeoutException: Cassandra timeout 
> during write query at consistency LOCAL_QUORUM (3 replica were required but 
> only 2 acknowledged the write)
>  at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:59)
>  at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:37)
>  at com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:289)
>  at com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:269)
>  at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:88)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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



[jira] [Updated] (CASSANDRA-14702) Cassandra Write failed even when the required nodes to Ack(consistency) are up.

2018-09-11 Thread Rohit Singh (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14702?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Rohit Singh updated CASSANDRA-14702:

Priority: Blocker  (was: Major)

> Cassandra Write failed even when the required nodes to Ack(consistency) are 
> up.
> ---
>
> Key: CASSANDRA-14702
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14702
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Rohit Singh
>Priority: Blocker
>
> Hi,
> We have following configuration in our project for cassandra. 
> Total nodes in Cluster-5
> Replication Factor- 3
> Consistency- LOCAL_QUORUM
> We get the writetimeout exception from cassandra even when 2 nodes are up and 
> why does stack trace says that 3 replica were required when consistency is 2?
> Below is the exception we got:-
> com.datastax.driver.core.exceptions.WriteTimeoutException: Cassandra timeout 
> during write query at consistency LOCAL_QUORUM (3 replica were required but 
> only 2 acknowledged the write)
>  at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:59)
>  at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:37)
>  at com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:289)
>  at com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:269)
>  at 
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:88)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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