[jira] [Commented] (CASSANDRA-13475) First version of pluggable storage engine API.
[ https://issues.apache.org/jira/browse/CASSANDRA-13475?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226281#comment-16226281 ] Blake Eggleston commented on CASSANDRA-13475: - Sure, but everywhere it’s used, Cassandra is interacting with the storage layer. You’re going to have to make changes everywhere it’s used anyway. I don’t think it’s going to be practical to start with an api and refactor Cassandra to conform to it. In other words, a storage api (albeit a leaky one) already exists. Adding another api on top of the existing one is just going to complicate things. I think any effort to make storage pluggable needs to start with the loose api that already exists, and start working out ways to refine it and hide implementation details behind a more generic interface as needed, piece by piece. That’s going to be 99.99% of the work involved here. > First version of pluggable storage engine API. > -- > > Key: CASSANDRA-13475 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13475 > Project: Cassandra > Issue Type: Sub-task >Reporter: Dikang Gu >Assignee: Dikang Gu > > In order to support pluggable storage engine, we need to define a unified > interface/API, which can allow us to plug in different storage engines for > different requirements. > In very high level, the storage engine interface should include APIs to: > 1. Apply update into the engine. > 2. Query data from the engine. > 3. Stream data in/out to/from the engine. > 4. Table operations, like create/drop/truncate a table, etc. > 5. Various stats about the engine. > I create this ticket to start the discussions about the interface. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-13475) First version of pluggable storage engine API.
[ https://issues.apache.org/jira/browse/CASSANDRA-13475?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226190#comment-16226190 ] Dikang Gu commented on CASSANDRA-13475: --- [~bdeggleston], thanks for bring it up, yeah, I thought about that, my concern is that ColumnFamilyStore is widely used in Cassandra source code, if I change it to be an interface like IColumnFamilyStore, then I will end up need to change every usage of ColumnFamilyStore. So instead of inheritance, I choose to use composition, where I keep ColumnFamilyStore class, and delegate the storage engine related calls to the new storage engine API. I think it will have much less impact to current call stack, and it will be easier for me to refactor the code base step by step. Does it make sense? > First version of pluggable storage engine API. > -- > > Key: CASSANDRA-13475 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13475 > Project: Cassandra > Issue Type: Sub-task >Reporter: Dikang Gu >Assignee: Dikang Gu > > In order to support pluggable storage engine, we need to define a unified > interface/API, which can allow us to plug in different storage engines for > different requirements. > In very high level, the storage engine interface should include APIs to: > 1. Apply update into the engine. > 2. Query data from the engine. > 3. Stream data in/out to/from the engine. > 4. Table operations, like create/drop/truncate a table, etc. > 5. Various stats about the engine. > I create this ticket to start the discussions about the interface. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-13475) First version of pluggable storage engine API.
[ https://issues.apache.org/jira/browse/CASSANDRA-13475?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226027#comment-16226027 ] Blake Eggleston commented on CASSANDRA-13475: - What I mean is that CFS basically exposes a storage api to the rest of Cassandra, and has a lot of the per-table storage implementation details and state. So instead of adding another interface, why not make CFS one of the primary extension points for the storage api? So you’d extract an IColumnFamilyStore interface, the current ColumnFamilyStore class would be CassandraColumnFamilyStore or whatever, and rocksdb would have a RocksDBColumnFamilyStore. This will be less invasive, highlight the areas where the storage layer implementation details leak into other parts of the project, and prevent cassandra storage layer specific stuff (like compaction strategies) from getting created/started/accessed. > First version of pluggable storage engine API. > -- > > Key: CASSANDRA-13475 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13475 > Project: Cassandra > Issue Type: Sub-task >Reporter: Dikang Gu >Assignee: Dikang Gu > > In order to support pluggable storage engine, we need to define a unified > interface/API, which can allow us to plug in different storage engines for > different requirements. > In very high level, the storage engine interface should include APIs to: > 1. Apply update into the engine. > 2. Query data from the engine. > 3. Stream data in/out to/from the engine. > 4. Table operations, like create/drop/truncate a table, etc. > 5. Various stats about the engine. > I create this ticket to start the discussions about the interface. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-13874) nodetool setcachecapacity behaves oddly when cache size = 0
[ https://issues.apache.org/jira/browse/CASSANDRA-13874?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jon Haddad updated CASSANDRA-13874: --- Reviewer: Jon Haddad > nodetool setcachecapacity behaves oddly when cache size = 0 > --- > > Key: CASSANDRA-13874 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13874 > Project: Cassandra > Issue Type: Bug >Reporter: Jon Haddad > Labels: lhf, user-experience > Attachments: 13874-trunk.txt > > > If a node has row cache disabled, trying to turn it on via setcachecapacity > doesn't issue an error, and doesn't turn it on, it just silently doesn't work. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-13872) document speculative_retry on DDL page
[ https://issues.apache.org/jira/browse/CASSANDRA-13872?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jon Haddad updated CASSANDRA-13872: --- Resolution: Fixed Fix Version/s: (was: 3.11.x) (was: 4.x) 4.0 Status: Resolved (was: Patch Available) Thanks for the patch! I made a quick edit to fix the case, as 99PERCENTILE is current case sensitive. Committed as bc5ad7bfa9 to trunk. > document speculative_retry on DDL page > -- > > Key: CASSANDRA-13872 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13872 > Project: Cassandra > Issue Type: Improvement > Components: Documentation and Website >Reporter: Jon Haddad >Assignee: Jordan Vaughan > Labels: docuentation, lhf > Fix For: 4.0 > > > There's no mention of speculative_retry or how it works on > https://cassandra.apache.org/doc/latest/cql/ddl.html -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
cassandra git commit: Document speculative_retry on the Cassandra DDL page
Repository: cassandra Updated Branches: refs/heads/trunk 569c6abcf -> bc5ad7bfa Document speculative_retry on the Cassandra DDL page Patch by Jordan Vaughan; Reviewed by Jon Haddad for CASSANDRA-13872 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/bc5ad7bf Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/bc5ad7bf Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/bc5ad7bf Branch: refs/heads/trunk Commit: bc5ad7bfa949cd566919e8ba28b327514d98c132 Parents: 569c6ab Author: Jordan VaughanAuthored: Sat Oct 28 15:57:15 2017 -0700 Committer: Jon Haddad Committed: Mon Oct 30 16:44:02 2017 -0700 -- doc/source/cql/ddl.rst | 31 +++ 1 file changed, 31 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/bc5ad7bf/doc/source/cql/ddl.rst -- diff --git a/doc/source/cql/ddl.rst b/doc/source/cql/ddl.rst index 6361983..a09265b 100644 --- a/doc/source/cql/ddl.rst +++ b/doc/source/cql/ddl.rst @@ -462,6 +462,9 @@ A table supports the following options: || | | belonging to the same data center than the read | || | | coordinator for the purpose of read repairs. | ++--+-+---+ +| ``speculative_retry`` | *simple* | 99PERCENTILE| :ref:`Speculative retry options | +|| | | `. | +++--+-+---+ | ``gc_grace_seconds`` | *simple* | 864000 | Time to wait before garbage collecting tombstones | || | | (deletion markers). | ++--+-+---+ @@ -482,6 +485,34 @@ A table supports the following options: | ``memtable_flush_period_in_ms``| *simple* | 0 | Time (in ms) before Cassandra flushes memtables to disk. | ++--+-+---+ +.. _speculative-retry-options: + +Speculative retry options +# + +By default, Cassandra read coordinators only query as many replicas as necessary to satisfy +consistency levels: one for consistency level ``ONE``, a quorum for ``QUORUM``, and so on. +``speculative_retry`` determines when coordinators may query additional replicas, which is useful +when replicas are slow or unresponsive. The following are legal values: + += = + FormatExample Description += = + ``XPERCENTILE`` 90.5PERCENTILE Coordinators record average per-table response times for all replicas. +If a replica takes longer than ``X`` percent of this table's average +response time, the coordinator queries an additional replica. +``X`` must be between 0 and 100. + ``Yms`` 25ms If a replica takes more than ``Y`` milliseconds to respond, +the coordinator queries an additional replica. + ``ALWAYS`` Coordinators always query all replicas. + ``NONE`` Coordinators never query additional replicas. += = + +This setting does not affect reads with consistency level ``ALL`` because they already query all replicas. + +Note that frequently reading from additional replicas can hurt cluster performance. +When in doubt, keep the default ``99PERCENTILE``. + .. _cql-compaction-options: Compaction options - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Comment Edited] (CASSANDRA-13475) First version of pluggable storage engine API.
[ https://issues.apache.org/jira/browse/CASSANDRA-13475?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16225983#comment-16225983 ] Dikang Gu edited comment on CASSANDRA-13475 at 10/30/17 11:34 PM: -- [~bdeggleston] My major motivation is that we probably don't really need the CFS reference inside the StorageEngine, at least in our RocksDB based implementation. In most case, I just need an identifier to the column family I'm current working on, could be table name or table uuid (maybe uuid is better than table name), not necessary to be the CFS. That's why I address [~spo...@gmail.com] 's comments on the CFS. Can you explain more about `main hooks for a pluggable storage layer should be a CFS implementation`? Current in my proposal, write requests will be implemented under the `applyMutate` API and read requests will be handled by implementing the Partition interface. For Keyspace and CFS, most of the coordinating logic should be remain unchanged. was (Author: dikanggu): [~bdeggleston] My major motivation is that we probably don't really need the CFS reference inside the StorageEngine, at least in our RocksDB based implementation. In most case, I just need an identify to the column family I'm current working on, could be table name or table uuid (maybe uuid is better than table name), not necessary to be the CFS. That's why I address [~spo...@gmail.com] 's comments on the CFS. Can you explain more about `main hooks for a pluggable storage layer should be a CFS implementation`? Current in my proposal, write requests will be implemented under the `applyMutate` API and read requests will be handled by implementing the Partition interface. For Keyspace and CFS, most of the coordinating logic should be remain unchanged. > First version of pluggable storage engine API. > -- > > Key: CASSANDRA-13475 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13475 > Project: Cassandra > Issue Type: Sub-task >Reporter: Dikang Gu >Assignee: Dikang Gu > > In order to support pluggable storage engine, we need to define a unified > interface/API, which can allow us to plug in different storage engines for > different requirements. > In very high level, the storage engine interface should include APIs to: > 1. Apply update into the engine. > 2. Query data from the engine. > 3. Stream data in/out to/from the engine. > 4. Table operations, like create/drop/truncate a table, etc. > 5. Various stats about the engine. > I create this ticket to start the discussions about the interface. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Comment Edited] (CASSANDRA-13475) First version of pluggable storage engine API.
[ https://issues.apache.org/jira/browse/CASSANDRA-13475?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16225983#comment-16225983 ] Dikang Gu edited comment on CASSANDRA-13475 at 10/30/17 11:34 PM: -- [~bdeggleston] My major motivation is that we probably don't really need the CFS reference inside the StorageEngine, at least in our RocksDB based implementation. In most case, I just need an identify to the column family I'm current working on, could be table name or table uuid (maybe uuid is better than table name), not necessary to be the CFS. That's why I address [~spo...@gmail.com] 's comments on the CFS. Can you explain more about `main hooks for a pluggable storage layer should be a CFS implementation`? Current in my proposal, write requests will be implemented under the `applyMutate` API and read requests will be handled by implementing the Partition interface. For Keyspace and CFS, most of the coordinating logic should be remain unchanged. was (Author: dikanggu): My major motivation is that we probably don't really need the CFS reference inside the StorageEngine, at least in our RocksDB based implementation. In most case, I just need an identify to the column family I'm current working on, could be table name or table uuid (maybe uuid is better than table name), not necessary to be the CFS. That's why I address [~spo...@gmail.com] 's comments on the CFS. Can you explain more about `main hooks for a pluggable storage layer should be a CFS implementation`? Current in my proposal, write requests will be implemented under the `applyMutate` API and read requests will be handled by implementing the Partition interface. For Keyspace and CFS, most of the coordinating logic should be remain unchanged. > First version of pluggable storage engine API. > -- > > Key: CASSANDRA-13475 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13475 > Project: Cassandra > Issue Type: Sub-task >Reporter: Dikang Gu >Assignee: Dikang Gu > > In order to support pluggable storage engine, we need to define a unified > interface/API, which can allow us to plug in different storage engines for > different requirements. > In very high level, the storage engine interface should include APIs to: > 1. Apply update into the engine. > 2. Query data from the engine. > 3. Stream data in/out to/from the engine. > 4. Table operations, like create/drop/truncate a table, etc. > 5. Various stats about the engine. > I create this ticket to start the discussions about the interface. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-13475) First version of pluggable storage engine API.
[ https://issues.apache.org/jira/browse/CASSANDRA-13475?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16225983#comment-16225983 ] Dikang Gu commented on CASSANDRA-13475: --- My major motivation is that we probably don't really need the CFS reference inside the StorageEngine, at least in our RocksDB based implementation. In most case, I just need an identify to the column family I'm current working on, could be table name or table uuid (maybe uuid is better than table name), not necessary to be the CFS. That's why I address [~spo...@gmail.com] 's comments on the CFS. Can you explain more about `main hooks for a pluggable storage layer should be a CFS implementation`? Current in my proposal, write requests will be implemented under the `applyMutate` API and read requests will be handled by implementing the Partition interface. For Keyspace and CFS, most of the coordinating logic should be remain unchanged. > First version of pluggable storage engine API. > -- > > Key: CASSANDRA-13475 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13475 > Project: Cassandra > Issue Type: Sub-task >Reporter: Dikang Gu >Assignee: Dikang Gu > > In order to support pluggable storage engine, we need to define a unified > interface/API, which can allow us to plug in different storage engines for > different requirements. > In very high level, the storage engine interface should include APIs to: > 1. Apply update into the engine. > 2. Query data from the engine. > 3. Stream data in/out to/from the engine. > 4. Table operations, like create/drop/truncate a table, etc. > 5. Various stats about the engine. > I create this ticket to start the discussions about the interface. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-13874) nodetool setcachecapacity behaves oddly when cache size = 0
[ https://issues.apache.org/jira/browse/CASSANDRA-13874?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michał Szczygieł updated CASSANDRA-13874: - Status: Patch Available (was: Open) > nodetool setcachecapacity behaves oddly when cache size = 0 > --- > > Key: CASSANDRA-13874 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13874 > Project: Cassandra > Issue Type: Bug >Reporter: Jon Haddad > Labels: lhf, user-experience > Attachments: 13874-trunk.txt > > > If a node has row cache disabled, trying to turn it on via setcachecapacity > doesn't issue an error, and doesn't turn it on, it just silently doesn't work. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-13874) nodetool setcachecapacity behaves oddly when cache size = 0
[ https://issues.apache.org/jira/browse/CASSANDRA-13874?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michał Szczygieł updated CASSANDRA-13874: - Attachment: 13874-trunk.txt > nodetool setcachecapacity behaves oddly when cache size = 0 > --- > > Key: CASSANDRA-13874 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13874 > Project: Cassandra > Issue Type: Bug >Reporter: Jon Haddad > Labels: lhf, user-experience > Attachments: 13874-trunk.txt > > > If a node has row cache disabled, trying to turn it on via setcachecapacity > doesn't issue an error, and doesn't turn it on, it just silently doesn't work. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-13475) First version of pluggable storage engine API.
[ https://issues.apache.org/jira/browse/CASSANDRA-13475?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16225906#comment-16225906 ] Blake Eggleston commented on CASSANDRA-13475: - Why get rid of (or minimize the importance of) the ColumnFamilyStore class? CFS is a pretty central component of the current storage engine. I think it would be a lot more natural to focus on extracting an interface from the current api, and not introduce another interface. In fact, I’d say the main hooks for a pluggable storage layer should be a CFS implementation (and Keyspace#apply) > First version of pluggable storage engine API. > -- > > Key: CASSANDRA-13475 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13475 > Project: Cassandra > Issue Type: Sub-task >Reporter: Dikang Gu >Assignee: Dikang Gu > > In order to support pluggable storage engine, we need to define a unified > interface/API, which can allow us to plug in different storage engines for > different requirements. > In very high level, the storage engine interface should include APIs to: > 1. Apply update into the engine. > 2. Query data from the engine. > 3. Stream data in/out to/from the engine. > 4. Table operations, like create/drop/truncate a table, etc. > 5. Various stats about the engine. > I create this ticket to start the discussions about the interface. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Assigned] (CASSANDRA-13982) Refactoring to specialised functional interfaces
[ https://issues.apache.org/jira/browse/CASSANDRA-13982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Jirsa reassigned CASSANDRA-13982: -- Assignee: Ameya Ketkar External issue URL: https://github.com/apache/cassandra/pull/168 > Refactoring to specialised functional interfaces > > > Key: CASSANDRA-13982 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13982 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Ameya Ketkar >Assignee: Ameya Ketkar >Priority: Minor > Labels: static-analysis > Fix For: 4.x > > Original Estimate: 24h > Remaining Estimate: 24h > > Usage of specialised functional interfaces provided by JDK, will reduce the > autoboxing overhead hence. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-13982) Refactoring to specialised functional interfaces
[ https://issues.apache.org/jira/browse/CASSANDRA-13982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Jirsa updated CASSANDRA-13982: --- Labels: static-analysis (was: ) > Refactoring to specialised functional interfaces > > > Key: CASSANDRA-13982 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13982 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Ameya Ketkar >Priority: Minor > Labels: static-analysis > Fix For: 4.x > > Original Estimate: 24h > Remaining Estimate: 24h > > Usage of specialised functional interfaces provided by JDK, will reduce the > autoboxing overhead hence. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-13982) Refactoring to specialised functional interfaces
[ https://issues.apache.org/jira/browse/CASSANDRA-13982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Jirsa updated CASSANDRA-13982: --- Component/s: Core > Refactoring to specialised functional interfaces > > > Key: CASSANDRA-13982 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13982 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Ameya Ketkar >Priority: Minor > Labels: static-analysis > Fix For: 4.x > > Original Estimate: 24h > Remaining Estimate: 24h > > Usage of specialised functional interfaces provided by JDK, will reduce the > autoboxing overhead hence. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-13982) Refactoring to specialised functional interfaces
[ https://issues.apache.org/jira/browse/CASSANDRA-13982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Jirsa updated CASSANDRA-13982: --- Fix Version/s: 4.x > Refactoring to specialised functional interfaces > > > Key: CASSANDRA-13982 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13982 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Ameya Ketkar >Priority: Minor > Labels: static-analysis > Fix For: 4.x > > Original Estimate: 24h > Remaining Estimate: 24h > > Usage of specialised functional interfaces provided by JDK, will reduce the > autoboxing overhead hence. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Created] (CASSANDRA-13982) Refactoring to specialised functional interfaces
Ameya Ketkar created CASSANDRA-13982: Summary: Refactoring to specialised functional interfaces Key: CASSANDRA-13982 URL: https://issues.apache.org/jira/browse/CASSANDRA-13982 Project: Cassandra Issue Type: Improvement Reporter: Ameya Ketkar Priority: Minor Usage of specialised functional interfaces provided by JDK, will reduce the autoboxing overhead hence. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Assigned] (CASSANDRA-13849) GossipStage blocks because of race in ActiveRepairService
[ https://issues.apache.org/jira/browse/CASSANDRA-13849?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Jirsa reassigned CASSANDRA-13849: -- Assignee: Sergey Lapukhov > GossipStage blocks because of race in ActiveRepairService > - > > Key: CASSANDRA-13849 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13849 > Project: Cassandra > Issue Type: Bug >Reporter: Tom van der Woerdt >Assignee: Sergey Lapukhov > Labels: patch > Fix For: 3.0.x, 3.11.x > > Attachments: CAS-13849.patch, CAS-13849_2.patch > > > Bad luck caused a kernel panic in a cluster, and that took another node with > it because GossipStage stopped responding. > I think it's pretty obvious what's happening, here are the relevant excerpts > from the stack traces : > {noformat} > "Thread-24004" #393781 daemon prio=5 os_prio=0 tid=0x7efca9647400 > nid=0xe75c waiting on condition [0x7efaa47fe000] >java.lang.Thread.State: TIMED_WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x00052b63a7e8> (a > java.util.concurrent.CountDownLatch$Sync) > at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328) > at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277) > at > org.apache.cassandra.service.ActiveRepairService.prepareForRepair(ActiveRepairService.java:332) > - locked <0x0002e6bc99f0> (a > org.apache.cassandra.service.ActiveRepairService) > at > org.apache.cassandra.repair.RepairRunnable.runMayThrow(RepairRunnable.java:211) > at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > > at > java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at > org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$3/1498438472.run(Unknown > Source) > at java.lang.Thread.run(Thread.java:748) > "GossipTasks:1" #367 daemon prio=5 os_prio=0 tid=0x7efc5e971000 > nid=0x700b waiting for monitor entry [0x7dfb839fe000] >java.lang.Thread.State: BLOCKED (on object monitor) > at > org.apache.cassandra.service.ActiveRepairService.removeParentRepairSession(ActiveRepairService.java:421) > - waiting to lock <0x0002e6bc99f0> (a > org.apache.cassandra.service.ActiveRepairService) > at > org.apache.cassandra.service.ActiveRepairService.convict(ActiveRepairService.java:776) > at > org.apache.cassandra.gms.FailureDetector.interpret(FailureDetector.java:306) > at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:775) > > at > org.apache.cassandra.gms.Gossiper.access$800(Gossiper.java:67) > at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:187) > at > org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at > org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$3/1498438472.run(Unknown > Source) > at java.lang.Thread.run(Thread.java:748) > "GossipStage:1" #320 daemon prio=5 os_prio=0 tid=0x7efc5b9f2c00 > nid=0x6fcd waiting for monitor entry [0x7e260186a000] >java.lang.Thread.State: BLOCKED (on object monitor) > at > org.apache.cassandra.service.ActiveRepairService.removeParentRepairSession(ActiveRepairService.java:421) > - waiting to lock <0x0002e6bc99f0> (a >
[jira] [Commented] (CASSANDRA-13849) GossipStage blocks because of race in ActiveRepairService
[ https://issues.apache.org/jira/browse/CASSANDRA-13849?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16225432#comment-16225432 ] Blake Eggleston commented on CASSANDRA-13849: - I'd agree that the timeout probably doesn't need to be that long. In the meantime, you should probably re-evaluate whether this method actually needs to be synchronized. This method is unsynchronized in trunk, and I think it may be because it was a holdover from more synchronous repair code of the past. Just looking at it briefly in 3.0, the only thing that *might* need synchonization is the call to registerParentRepairSession, everything else if just method local message sending / receiving stuff. > GossipStage blocks because of race in ActiveRepairService > - > > Key: CASSANDRA-13849 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13849 > Project: Cassandra > Issue Type: Bug >Reporter: Tom van der Woerdt > Labels: patch > Fix For: 3.0.x, 3.11.x > > Attachments: CAS-13849.patch, CAS-13849_2.patch > > > Bad luck caused a kernel panic in a cluster, and that took another node with > it because GossipStage stopped responding. > I think it's pretty obvious what's happening, here are the relevant excerpts > from the stack traces : > {noformat} > "Thread-24004" #393781 daemon prio=5 os_prio=0 tid=0x7efca9647400 > nid=0xe75c waiting on condition [0x7efaa47fe000] >java.lang.Thread.State: TIMED_WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x00052b63a7e8> (a > java.util.concurrent.CountDownLatch$Sync) > at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328) > at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277) > at > org.apache.cassandra.service.ActiveRepairService.prepareForRepair(ActiveRepairService.java:332) > - locked <0x0002e6bc99f0> (a > org.apache.cassandra.service.ActiveRepairService) > at > org.apache.cassandra.repair.RepairRunnable.runMayThrow(RepairRunnable.java:211) > at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > > at > java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at > org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$3/1498438472.run(Unknown > Source) > at java.lang.Thread.run(Thread.java:748) > "GossipTasks:1" #367 daemon prio=5 os_prio=0 tid=0x7efc5e971000 > nid=0x700b waiting for monitor entry [0x7dfb839fe000] >java.lang.Thread.State: BLOCKED (on object monitor) > at > org.apache.cassandra.service.ActiveRepairService.removeParentRepairSession(ActiveRepairService.java:421) > - waiting to lock <0x0002e6bc99f0> (a > org.apache.cassandra.service.ActiveRepairService) > at > org.apache.cassandra.service.ActiveRepairService.convict(ActiveRepairService.java:776) > at > org.apache.cassandra.gms.FailureDetector.interpret(FailureDetector.java:306) > at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:775) > > at > org.apache.cassandra.gms.Gossiper.access$800(Gossiper.java:67) > at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:187) > at > org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at >
[jira] [Updated] (CASSANDRA-13849) GossipStage blocks because of race in ActiveRepairService
[ https://issues.apache.org/jira/browse/CASSANDRA-13849?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Jirsa updated CASSANDRA-13849: --- Fix Version/s: 3.11.x 3.0.x > GossipStage blocks because of race in ActiveRepairService > - > > Key: CASSANDRA-13849 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13849 > Project: Cassandra > Issue Type: Bug >Reporter: Tom van der Woerdt > Labels: patch > Fix For: 3.0.x, 3.11.x > > Attachments: CAS-13849.patch, CAS-13849_2.patch > > > Bad luck caused a kernel panic in a cluster, and that took another node with > it because GossipStage stopped responding. > I think it's pretty obvious what's happening, here are the relevant excerpts > from the stack traces : > {noformat} > "Thread-24004" #393781 daemon prio=5 os_prio=0 tid=0x7efca9647400 > nid=0xe75c waiting on condition [0x7efaa47fe000] >java.lang.Thread.State: TIMED_WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x00052b63a7e8> (a > java.util.concurrent.CountDownLatch$Sync) > at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328) > at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277) > at > org.apache.cassandra.service.ActiveRepairService.prepareForRepair(ActiveRepairService.java:332) > - locked <0x0002e6bc99f0> (a > org.apache.cassandra.service.ActiveRepairService) > at > org.apache.cassandra.repair.RepairRunnable.runMayThrow(RepairRunnable.java:211) > at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > > at > java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at > org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$3/1498438472.run(Unknown > Source) > at java.lang.Thread.run(Thread.java:748) > "GossipTasks:1" #367 daemon prio=5 os_prio=0 tid=0x7efc5e971000 > nid=0x700b waiting for monitor entry [0x7dfb839fe000] >java.lang.Thread.State: BLOCKED (on object monitor) > at > org.apache.cassandra.service.ActiveRepairService.removeParentRepairSession(ActiveRepairService.java:421) > - waiting to lock <0x0002e6bc99f0> (a > org.apache.cassandra.service.ActiveRepairService) > at > org.apache.cassandra.service.ActiveRepairService.convict(ActiveRepairService.java:776) > at > org.apache.cassandra.gms.FailureDetector.interpret(FailureDetector.java:306) > at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:775) > > at > org.apache.cassandra.gms.Gossiper.access$800(Gossiper.java:67) > at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:187) > at > org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at > org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$3/1498438472.run(Unknown > Source) > at java.lang.Thread.run(Thread.java:748) > "GossipStage:1" #320 daemon prio=5 os_prio=0 tid=0x7efc5b9f2c00 > nid=0x6fcd waiting for monitor entry [0x7e260186a000] >java.lang.Thread.State: BLOCKED (on object monitor) > at > org.apache.cassandra.service.ActiveRepairService.removeParentRepairSession(ActiveRepairService.java:421) > - waiting to lock <0x0002e6bc99f0> (a > org.apache.cassandra.service.ActiveRepairService)
[jira] [Updated] (CASSANDRA-13981) Enable Cassandra for Persistent Memory
[ https://issues.apache.org/jira/browse/CASSANDRA-13981?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Jirsa updated CASSANDRA-13981: --- Component/s: Core > Enable Cassandra for Persistent Memory > --- > > Key: CASSANDRA-13981 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13981 > Project: Cassandra > Issue Type: New Feature > Components: Core >Reporter: Preetika Tyagi >Assignee: Preetika Tyagi > Fix For: 4.x > > > Currently, Cassandra relies on disks for data storage and hence it needs data > serialization, compaction, bloom filters and partition summary/index for > speedy access of the data. However, with persistent memory, data can be > stored directly in the form of Java objects and collections, which can > greatly simplify the retrieval mechanism of the data. What we are proposing > is to make use of faster and scalable B+ tree-based data collections built > for persistent memory in Java (PCJ: https://github.com/pmem/pcj) and enable a > complete in-memory version of Cassandra, while still keeping the data > persistent. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-13981) Enable Cassandra for Persistent Memory
[ https://issues.apache.org/jira/browse/CASSANDRA-13981?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Jirsa updated CASSANDRA-13981: --- Fix Version/s: 4.x > Enable Cassandra for Persistent Memory > --- > > Key: CASSANDRA-13981 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13981 > Project: Cassandra > Issue Type: New Feature > Components: Core >Reporter: Preetika Tyagi >Assignee: Preetika Tyagi > Fix For: 4.x > > > Currently, Cassandra relies on disks for data storage and hence it needs data > serialization, compaction, bloom filters and partition summary/index for > speedy access of the data. However, with persistent memory, data can be > stored directly in the form of Java objects and collections, which can > greatly simplify the retrieval mechanism of the data. What we are proposing > is to make use of faster and scalable B+ tree-based data collections built > for persistent memory in Java (PCJ: https://github.com/pmem/pcj) and enable a > complete in-memory version of Cassandra, while still keeping the data > persistent. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-13849) GossipStage blocks because of race in ActiveRepairService
[ https://issues.apache.org/jira/browse/CASSANDRA-13849?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16225366#comment-16225366 ] Sergey Lapukhov commented on CASSANDRA-13849: - Stefan, thank you for your response. I have tried your idea in our test setting - it fixes the problem for us. I have uploaded this as a patch ([^CAS-13849_2.patch]), could you please take a look? > GossipStage blocks because of race in ActiveRepairService > - > > Key: CASSANDRA-13849 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13849 > Project: Cassandra > Issue Type: Bug >Reporter: Tom van der Woerdt > Labels: patch > Attachments: CAS-13849.patch, CAS-13849_2.patch > > > Bad luck caused a kernel panic in a cluster, and that took another node with > it because GossipStage stopped responding. > I think it's pretty obvious what's happening, here are the relevant excerpts > from the stack traces : > {noformat} > "Thread-24004" #393781 daemon prio=5 os_prio=0 tid=0x7efca9647400 > nid=0xe75c waiting on condition [0x7efaa47fe000] >java.lang.Thread.State: TIMED_WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x00052b63a7e8> (a > java.util.concurrent.CountDownLatch$Sync) > at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328) > at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277) > at > org.apache.cassandra.service.ActiveRepairService.prepareForRepair(ActiveRepairService.java:332) > - locked <0x0002e6bc99f0> (a > org.apache.cassandra.service.ActiveRepairService) > at > org.apache.cassandra.repair.RepairRunnable.runMayThrow(RepairRunnable.java:211) > at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > > at > java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at > org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$3/1498438472.run(Unknown > Source) > at java.lang.Thread.run(Thread.java:748) > "GossipTasks:1" #367 daemon prio=5 os_prio=0 tid=0x7efc5e971000 > nid=0x700b waiting for monitor entry [0x7dfb839fe000] >java.lang.Thread.State: BLOCKED (on object monitor) > at > org.apache.cassandra.service.ActiveRepairService.removeParentRepairSession(ActiveRepairService.java:421) > - waiting to lock <0x0002e6bc99f0> (a > org.apache.cassandra.service.ActiveRepairService) > at > org.apache.cassandra.service.ActiveRepairService.convict(ActiveRepairService.java:776) > at > org.apache.cassandra.gms.FailureDetector.interpret(FailureDetector.java:306) > at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:775) > > at > org.apache.cassandra.gms.Gossiper.access$800(Gossiper.java:67) > at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:187) > at > org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at > org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$3/1498438472.run(Unknown > Source) > at java.lang.Thread.run(Thread.java:748) > "GossipStage:1" #320 daemon prio=5 os_prio=0 tid=0x7efc5b9f2c00 > nid=0x6fcd waiting for monitor entry [0x7e260186a000] >java.lang.Thread.State: BLOCKED (on object monitor) > at >
[jira] [Updated] (CASSANDRA-13849) GossipStage blocks because of race in ActiveRepairService
[ https://issues.apache.org/jira/browse/CASSANDRA-13849?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sergey Lapukhov updated CASSANDRA-13849: Attachment: CAS-13849_2.patch Patch reducing 1 hour timeout for the PrepareMessage callback to 1 minute. > GossipStage blocks because of race in ActiveRepairService > - > > Key: CASSANDRA-13849 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13849 > Project: Cassandra > Issue Type: Bug >Reporter: Tom van der Woerdt > Labels: patch > Attachments: CAS-13849.patch, CAS-13849_2.patch > > > Bad luck caused a kernel panic in a cluster, and that took another node with > it because GossipStage stopped responding. > I think it's pretty obvious what's happening, here are the relevant excerpts > from the stack traces : > {noformat} > "Thread-24004" #393781 daemon prio=5 os_prio=0 tid=0x7efca9647400 > nid=0xe75c waiting on condition [0x7efaa47fe000] >java.lang.Thread.State: TIMED_WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x00052b63a7e8> (a > java.util.concurrent.CountDownLatch$Sync) > at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328) > at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277) > at > org.apache.cassandra.service.ActiveRepairService.prepareForRepair(ActiveRepairService.java:332) > - locked <0x0002e6bc99f0> (a > org.apache.cassandra.service.ActiveRepairService) > at > org.apache.cassandra.repair.RepairRunnable.runMayThrow(RepairRunnable.java:211) > at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > > at > java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at > org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$3/1498438472.run(Unknown > Source) > at java.lang.Thread.run(Thread.java:748) > "GossipTasks:1" #367 daemon prio=5 os_prio=0 tid=0x7efc5e971000 > nid=0x700b waiting for monitor entry [0x7dfb839fe000] >java.lang.Thread.State: BLOCKED (on object monitor) > at > org.apache.cassandra.service.ActiveRepairService.removeParentRepairSession(ActiveRepairService.java:421) > - waiting to lock <0x0002e6bc99f0> (a > org.apache.cassandra.service.ActiveRepairService) > at > org.apache.cassandra.service.ActiveRepairService.convict(ActiveRepairService.java:776) > at > org.apache.cassandra.gms.FailureDetector.interpret(FailureDetector.java:306) > at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:775) > > at > org.apache.cassandra.gms.Gossiper.access$800(Gossiper.java:67) > at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:187) > at > org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at > org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$3/1498438472.run(Unknown > Source) > at java.lang.Thread.run(Thread.java:748) > "GossipStage:1" #320 daemon prio=5 os_prio=0 tid=0x7efc5b9f2c00 > nid=0x6fcd waiting for monitor entry [0x7e260186a000] >java.lang.Thread.State: BLOCKED (on object monitor) > at > org.apache.cassandra.service.ActiveRepairService.removeParentRepairSession(ActiveRepairService.java:421) > - waiting to lock <0x0002e6bc99f0> (a >
[jira] [Resolved] (CASSANDRA-13980) Compaction deadlock
[ https://issues.apache.org/jira/browse/CASSANDRA-13980?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson resolved CASSANDRA-13980. - Resolution: Duplicate Fix Version/s: (was: 3.11.x) Probably the same as CASSANDRA-13948 > Compaction deadlock > --- > > Key: CASSANDRA-13980 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13980 > Project: Cassandra > Issue Type: Bug > Components: Compaction > Environment: Cassandra 3.11.2 > 48 nodes cluster using LCS, JBOD > Big nodes with many SSTables >Reporter: Loic Lambiel >Priority: Critical > Attachments: threaddump.log > > > While upgrading the cluster from 2.1.16 from 3.11.2, after a few hours most > of the upgraded nodes started to go in a compaction infinite loop and showing > many events like the one below (always for the same SSTable): > {code:java} > INFO [CompactionExecutor:4] 2017-10-29 00:28:31,480 LeveledManifest.java:474 > - Adding high-level (L5) > BigTableReader(path='/var/lib/cassandra/data/datadisk4/blobstore/block-1d63273065b911e49cd7ef0972cffde6/blobstore-block-ka-201694-Data.db') > to candidates > {code} > Since the log get spammed at a huge rate, I'm unable to get any previous > events. > Tried restarts and sstablescrub -m without success. The only workaround that > seems to work (so far) was sstablelevelreset. > I've attached the dump. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org