[jira] [Commented] (CASSANDRA-13123) Draining a node might fail to delete all inactive commitlogs
[ https://issues.apache.org/jira/browse/CASSANDRA-13123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186704#comment-16186704 ] Jeff Jirsa commented on CASSANDRA-13123: Hi folks, Pretty sure this commit breaks {{CommitLogSegmentManagerTest}} - have seen a pretty sharp rise in failures, and reverting this commit seems to solve them. {code} [junit] INFO 23:34:52 Initializing CommitLogTest.Standard1 [junit] INFO 23:34:52 Initializing CommitLogTest.Standard2 [junit] - --- [junit] Testcase: testShutdownWithPendingTasks(org.apache.cassandra.db.commitlog.CommitLogSegmentManagerTest): FAILED [junit] null [junit] junit.framework.AssertionFailedError [junit] at org.apache.cassandra.db.Keyspace.open(Keyspace.java:105) [junit] at org.apache.cassandra.db.commitlog.CommitLogSegmentManagerTest.testShutdownWithPendingTasks(CommitLogSegmentManagerTest.java:147) [junit] at org.jboss.byteman.contrib.bmunit.BMUnitRunner$10.evaluate(BMUnitRunner.java:371) [junit] at org.jboss.byteman.contrib.bmunit.BMUnitRunner$6.evaluate(BMUnitRunner.java:241) [junit] at org.jboss.byteman.contrib.bmunit.BMUnitRunner$1.evaluate(BMUnitRunner.java:75) [junit] [junit] [junit] Testcase: testCompressedCommitLogBackpressure(org.apache.cassandra.db.commitlog.CommitLogSegmentManagerTest): FAILED [junit] expected:<3> but was:<1> [junit] junit.framework.AssertionFailedError: expected:<3> but was:<1> [junit] at org.apache.cassandra.Util.spinAssertEquals(Util.java:535) [junit] at org.apache.cassandra.db.commitlog.CommitLogSegmentManagerTest.testCompressedCommitLogBackpressure(CommitLogSegmentManagerTest.java:112) [junit] at org.jboss.byteman.contrib.bmunit.BMUnitRunner$9.evaluate(BMUnitRunner.java:342) [junit] at org.jboss.byteman.contrib.bmunit.BMUnitRunner$6.evaluate(BMUnitRunner.java:241) [junit] at org.jboss.byteman.contrib.bmunit.BMUnitRunner$1.evaluate(BMUnitRunner.java:75) [junit] [junit] [junit] Test org.apache.cassandra.db.commitlog.CommitLogSegmentManagerTest FAILED [delete] Deleting directory /Users/jjirsa/Desktop/Dev/cassandra/build/test/cassandra/commitlog:0 [delete] Deleting directory /Users/jjirsa/Desktop/Dev/cassandra/build/test/cassandra/data:0 [delete] Deleting directory /Users/jjirsa/Desktop/Dev/cassandra/build/test/cassandra/saved_caches:0 [junitreport] Processing /Users/jjirsa/Desktop/Dev/cassandra/build/test/TESTS-TestSuites.xml to /var/folders/nq/4w83hn7s3h13dc5wxmvcdn9wgn/T/null1048031913 [junitreport] Loading stylesheet jar:file:/usr/local/ant/lib/ant-junit.jar!/org/apache/tools/ant/taskdefs/optional/junit/xsl/junit-frames.xsl [junitreport] Transform time: 256ms [junitreport] Deleting: /var/folders/nq/4w83hn7s3h13dc5wxmvcdn9wgn/T/null1048031913 {code} > Draining a node might fail to delete all inactive commitlogs > > > Key: CASSANDRA-13123 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13123 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths >Reporter: Jan Urbański >Assignee: Jan Urbański > Fix For: 3.0.15, 3.11.1, 4.0 > > Attachments: 13123-2.2.8.txt, 13123-3.0.10.txt, 13123-3.9.txt, > 13123-trunk.txt > > > After issuing a drain command, it's possible that not all of the inactive > commitlogs are removed. > The drain command shuts down the CommitLog instance, which in turn shuts down > the CommitLogSegmentManager. This has the effect of discarding any pending > management tasks it might have, like the removal of inactive commitlogs. > This in turn leads to an excessive amount of commitlogs being left behind > after a drain and a lengthy recovery after a restart. With a fleet of dozens > of nodes, each of them leaving several GB of commitlogs after a drain and > taking up to two minutes to recover them on restart, the additional time > required to restart the entire fleet becomes noticeable. > This problem is not present in 3.x or trunk because of the CLSM rewrite done > in CASSANDRA-8844. -- 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] [Reopened] (CASSANDRA-13123) Draining a node might fail to delete all inactive commitlogs
[ https://issues.apache.org/jira/browse/CASSANDRA-13123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Jirsa reopened CASSANDRA-13123: Reproduced In: 3.0.10, 2.2.8 (was: 2.2.8, 3.0.10) > Draining a node might fail to delete all inactive commitlogs > > > Key: CASSANDRA-13123 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13123 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths >Reporter: Jan Urbański >Assignee: Jan Urbański > Fix For: 3.0.15, 3.11.1, 4.0 > > Attachments: 13123-2.2.8.txt, 13123-3.0.10.txt, 13123-3.9.txt, > 13123-trunk.txt > > > After issuing a drain command, it's possible that not all of the inactive > commitlogs are removed. > The drain command shuts down the CommitLog instance, which in turn shuts down > the CommitLogSegmentManager. This has the effect of discarding any pending > management tasks it might have, like the removal of inactive commitlogs. > This in turn leads to an excessive amount of commitlogs being left behind > after a drain and a lengthy recovery after a restart. With a fleet of dozens > of nodes, each of them leaving several GB of commitlogs after a drain and > taking up to two minutes to recover them on restart, the additional time > required to restart the entire fleet becomes noticeable. > This problem is not present in 3.x or trunk because of the CLSM rewrite done > in CASSANDRA-8844. -- 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-13910) Consider deprecating (then removing) read_repair_chance/dclocal_read_repair_chance
[ https://issues.apache.org/jira/browse/CASSANDRA-13910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186643#comment-16186643 ] Jeremiah Jordan commented on CASSANDRA-13910: - bq. Asked differently: why would someone set read_repair_chance to a high value? If someone had a multi-dc setup, wrote with LOCAL_ cls, and then read from that DC, they'd trigger the async repair into the other DC, working around the dc local consistency without paying the latency cost on the app side. Sort of a "I can force hint delivery on read without waiting for it" technique. I don't know how common such a use case would be, but I would believe that someone, somewhere is relying on it. I definitely have done exactly this in the past when hints were WAY less reliable. With the current hints implementation that is pretty reliable, I do not see the need for doing something like this anymore. > Consider deprecating (then removing) > read_repair_chance/dclocal_read_repair_chance > -- > > Key: CASSANDRA-13910 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13910 > Project: Cassandra > Issue Type: Improvement >Reporter: Sylvain Lebresne >Priority: Minor > Labels: CommunityFeedbackRequested > > First, let me clarify so this is not misunderstood that I'm not *at all* > suggesting to remove the read-repair mechanism of detecting and repairing > inconsistencies between read responses: that mechanism is imo fine and > useful. But the {{read_repair_chance}} and {{dclocal_read_repair_chance}} > have never been about _enabling_ that mechanism, they are about querying all > replicas (even when this is not required by the consistency level) for the > sole purpose of maybe read-repairing some of the replica that wouldn't have > been queried otherwise. Which btw, bring me to reason 1 for considering their > removal: their naming/behavior is super confusing. Over the years, I've seen > countless users (and not only newbies) misunderstanding what those options > do, and as a consequence misunderstand when read-repair itself was happening. > But my 2nd reason for suggesting this is that I suspect > {{read_repair_chance}}/{{dclocal_read_repair_chance}} are, especially > nowadays, more harmful than anything else when enabled. When those option > kick in, what you trade-off is additional resources consumption (all nodes > have to execute the read) for a _fairly remote chance_ of having some > inconsistencies repaired on _some_ replica _a bit faster_ than they would > otherwise be. To justify that last part, let's recall that: > # most inconsistencies are actually fixed by hints in practice; and in the > case where a node stay dead for a long time so that hints ends up timing-out, > you really should repair the node when it comes back (if not simply > re-bootstrapping it). Read-repair probably don't fix _that_ much stuff in > the first place. > # again, read-repair do happen without those options kicking in. If you do > reads at {{QUORUM}}, inconsistencies will eventually get read-repaired all > the same. Just a tiny bit less quickly. > # I suspect almost everyone use a low "chance" for those options at best > (because the extra resources consumption is real), so at the end of the day, > it's up to chance how much faster this fixes inconsistencies. > Overall, I'm having a hard time imagining real cases where that trade-off > really make sense. Don't get me wrong, those options had their places a long > time ago when hints weren't working all that well, but I think they bring > more confusion than benefits now. > And I think it's sane to reconsider stuffs every once in a while, and to > clean up anything that may not make all that much sense anymore, which I > think is the case here. > Tl;dr, I feel the benefits brought by those options are very slim at best and > well overshadowed by the confusion they bring, and not worth maintaining the > code that supports them (which, to be fair, isn't huge, but getting rid of > {{ReadCallback.AsyncRepairRunner}} wouldn't hurt for instance). > Lastly, if the consensus here ends up being that they can have their use in > weird case and that we fill supporting those cases is worth confusing > everyone else and maintaining that code, I would still suggest disabling them > totally by default. -- 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-13918) Header only commit logs should be filtered before recovery
[ https://issues.apache.org/jira/browse/CASSANDRA-13918?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Blake Eggleston updated CASSANDRA-13918: Resolution: Fixed Status: Resolved (was: Ready to Commit) committed as {{95839aae2fde28fa29b16741de6bd52c0697843f}} > Header only commit logs should be filtered before recovery > -- > > Key: CASSANDRA-13918 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13918 > Project: Cassandra > Issue Type: Bug >Reporter: Blake Eggleston >Assignee: Blake Eggleston > Fix For: 3.0.15, 4.0, 3.11.x > > > Commit log recovery will tolerate commit log truncation in the most recent > log file found on disk, but will abort startup if problems are detected in > others. > Since we allocate commit log segments before they're used though, it's > possible to get into a state where the last commit log file actually written > to is not the same file that was most recently allocated, preventing startup > for what should otherwise be allowable incomplete final segments. > Excluding header only files on recovery should prevent this from happening. -- 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
[2/6] cassandra git commit: Filter header only commit logs before recovery
Filter header only commit logs before recovery Patch by Blake Eggleston; Reviewed by Sam Tunnicliffe for CASSANDRA-13918 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/95839aae Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/95839aae Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/95839aae Branch: refs/heads/cassandra-3.11 Commit: 95839aae2fde28fa29b16741de6bd52c0697843f Parents: ab0adf9 Author: Blake EgglestonAuthored: Thu Sep 28 15:01:35 2017 -0700 Committer: Blake Eggleston Committed: Fri Sep 29 15:05:30 2017 -0700 -- CHANGES.txt | 1 + .../db/commitlog/CommitLogReplayer.java | 57 +--- .../db/commitlog/CommitLogSegment.java | 2 +- .../db/commitlog/CompressedSegment.java | 2 +- .../db/commitlog/MemoryMappedSegment.java | 2 +- .../cassandra/db/commitlog/CommitLogTest.java | 53 ++ 6 files changed, 108 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/95839aae/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index a1f49cd..7ff61d3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.15 + * Filter header only commit logs before recovery (CASSANDRA-13918) * AssertionError prepending to a list (CASSANDRA-13149) * Fix support for SuperColumn tables (CASSANDRA-12373) * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) http://git-wip-us.apache.org/repos/asf/cassandra/blob/95839aae/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java -- diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java index b3b26dd..4fd263c 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java @@ -25,18 +25,15 @@ import java.io.FileOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.*; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; import java.util.zip.CRC32; import com.google.common.base.Predicate; -import com.google.common.base.Throwables; import com.google.common.collect.HashMultimap; import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; import com.google.common.collect.Ordering; -import com.google.common.util.concurrent.Uninterruptibles; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -138,11 +135,59 @@ public class CommitLogReplayer return new CommitLogReplayer(commitLog, globalPosition, cfPersisted, replayFilter); } +private static boolean shouldSkip(File file) throws IOException, ConfigurationException +{ +CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.getName()); +if (desc.version < CommitLogDescriptor.VERSION_21) +{ +return false; +} +try(ChannelProxy channel = new ChannelProxy(file); +RandomAccessReader reader = RandomAccessReader.open(channel)) +{ +CommitLogDescriptor.readHeader(reader); +int end = reader.readInt(); +long filecrc = reader.readInt() & 0xL; +return end == 0 && filecrc == 0; +} +} + +private static List filterCommitLogFiles(File[] toFilter) +{ +List filtered = new ArrayList<>(toFilter.length); +for (File file: toFilter) +{ +try +{ +if (shouldSkip(file)) +{ +logger.info("Skipping playback of empty log: {}", file.getName()); +} +else +{ +filtered.add(file); +} +} +catch (Exception e) +{ +// let recover deal with it +filtered.add(file); +} +} + +return filtered; +} + public void recover(File[] clogs) throws IOException { -int i; -for (i = 0; i < clogs.length; ++i) -recover(clogs[i], i + 1 == clogs.length); +List filteredLogs = filterCommitLogFiles(clogs); + +int i = 0; +for (File clog: filteredLogs) +{ +i++; +recover(clog, i == filteredLogs.size()); +} } /**
[4/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11
Merge branch 'cassandra-3.0' into cassandra-3.11 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/be211749 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/be211749 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/be211749 Branch: refs/heads/trunk Commit: be2117492f3d9ace24bbf18e57e94b2a08965763 Parents: 2a24acf 95839aa Author: Blake EgglestonAuthored: Fri Sep 29 15:31:37 2017 -0700 Committer: Blake Eggleston Committed: Fri Sep 29 15:32:32 2017 -0700 -- CHANGES.txt | 2 +- .../cassandra/db/commitlog/CommitLogReader.java | 53 +++- .../db/commitlog/CommitLogReplayer.java | 3 -- .../db/commitlog/CommitLogSegment.java | 2 +- .../db/commitlog/CompressedSegment.java | 2 +- .../db/commitlog/EncryptedSegment.java | 4 +- .../db/commitlog/MemoryMappedSegment.java | 2 +- .../cassandra/db/commitlog/CommitLogTest.java | 53 8 files changed, 110 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/be211749/CHANGES.txt -- diff --cc CHANGES.txt index 6c3a1d0,7ff61d3..a782333 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,20 -1,9 +1,20 @@@ -3.0.15 +3.11.1 - === + * Fix the computation of cdc_total_space_in_mb for exabyte filesystems (CASSANDRA-13808) + * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) + * AbstractTokenTreeBuilder#serializedSize returns wrong value when there is a single leaf and overflow collisions (CASSANDRA-13869) + * Add a compaction option to TWCS to ignore sstables overlapping checks (CASSANDRA-13418) + * BTree.Builder memory leak (CASSANDRA-13754) + * Revert CASSANDRA-10368 of supporting non-pk column filtering due to correctness (CASSANDRA-13798) + * Fix cassandra-stress hang issues when an error during cluster connection happens (CASSANDRA-12938) + * Better bootstrap failure message when blocked by (potential) range movement (CASSANDRA-13744) + * "ignore" option is ignored in sstableloader (CASSANDRA-13721) + * Deadlock in AbstractCommitLogSegmentManager (CASSANDRA-13652) + * Duplicate the buffer before passing it to analyser in SASI operation (CASSANDRA-13512) + * Properly evict pstmts from prepared statements cache (CASSANDRA-13641) +Merged from 3.0: + * Filter header only commit logs before recovery (CASSANDRA-13918) * AssertionError prepending to a list (CASSANDRA-13149) * Fix support for SuperColumn tables (CASSANDRA-12373) - * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) - * Fix missing original update in TriggerExecutor (CASSANDRA-13894) * Remove non-rpc-ready nodes from counter leader candidates (CASSANDRA-13043) * Improve short read protection performance (CASSANDRA-13794) * Fix sstable reader to support range-tombstone-marker for multi-slices (CASSANDRA-13787) http://git-wip-us.apache.org/repos/asf/cassandra/blob/be211749/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java -- diff --cc src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java index 8c04329,000..4d74557 mode 100644,00..100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java @@@ -1,515 -1,0 +1,564 @@@ +/* + * 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.db.commitlog; + +import java.io.*; +import java.util.*; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.zip.CRC32; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import
[5/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11
Merge branch 'cassandra-3.0' into cassandra-3.11 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/be211749 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/be211749 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/be211749 Branch: refs/heads/cassandra-3.11 Commit: be2117492f3d9ace24bbf18e57e94b2a08965763 Parents: 2a24acf 95839aa Author: Blake EgglestonAuthored: Fri Sep 29 15:31:37 2017 -0700 Committer: Blake Eggleston Committed: Fri Sep 29 15:32:32 2017 -0700 -- CHANGES.txt | 2 +- .../cassandra/db/commitlog/CommitLogReader.java | 53 +++- .../db/commitlog/CommitLogReplayer.java | 3 -- .../db/commitlog/CommitLogSegment.java | 2 +- .../db/commitlog/CompressedSegment.java | 2 +- .../db/commitlog/EncryptedSegment.java | 4 +- .../db/commitlog/MemoryMappedSegment.java | 2 +- .../cassandra/db/commitlog/CommitLogTest.java | 53 8 files changed, 110 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/be211749/CHANGES.txt -- diff --cc CHANGES.txt index 6c3a1d0,7ff61d3..a782333 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,20 -1,9 +1,20 @@@ -3.0.15 +3.11.1 - === + * Fix the computation of cdc_total_space_in_mb for exabyte filesystems (CASSANDRA-13808) + * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) + * AbstractTokenTreeBuilder#serializedSize returns wrong value when there is a single leaf and overflow collisions (CASSANDRA-13869) + * Add a compaction option to TWCS to ignore sstables overlapping checks (CASSANDRA-13418) + * BTree.Builder memory leak (CASSANDRA-13754) + * Revert CASSANDRA-10368 of supporting non-pk column filtering due to correctness (CASSANDRA-13798) + * Fix cassandra-stress hang issues when an error during cluster connection happens (CASSANDRA-12938) + * Better bootstrap failure message when blocked by (potential) range movement (CASSANDRA-13744) + * "ignore" option is ignored in sstableloader (CASSANDRA-13721) + * Deadlock in AbstractCommitLogSegmentManager (CASSANDRA-13652) + * Duplicate the buffer before passing it to analyser in SASI operation (CASSANDRA-13512) + * Properly evict pstmts from prepared statements cache (CASSANDRA-13641) +Merged from 3.0: + * Filter header only commit logs before recovery (CASSANDRA-13918) * AssertionError prepending to a list (CASSANDRA-13149) * Fix support for SuperColumn tables (CASSANDRA-12373) - * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) - * Fix missing original update in TriggerExecutor (CASSANDRA-13894) * Remove non-rpc-ready nodes from counter leader candidates (CASSANDRA-13043) * Improve short read protection performance (CASSANDRA-13794) * Fix sstable reader to support range-tombstone-marker for multi-slices (CASSANDRA-13787) http://git-wip-us.apache.org/repos/asf/cassandra/blob/be211749/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java -- diff --cc src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java index 8c04329,000..4d74557 mode 100644,00..100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java @@@ -1,515 -1,0 +1,564 @@@ +/* + * 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.db.commitlog; + +import java.io.*; +import java.util.*; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.zip.CRC32; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; +import
[3/6] cassandra git commit: Filter header only commit logs before recovery
Filter header only commit logs before recovery Patch by Blake Eggleston; Reviewed by Sam Tunnicliffe for CASSANDRA-13918 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/95839aae Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/95839aae Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/95839aae Branch: refs/heads/trunk Commit: 95839aae2fde28fa29b16741de6bd52c0697843f Parents: ab0adf9 Author: Blake EgglestonAuthored: Thu Sep 28 15:01:35 2017 -0700 Committer: Blake Eggleston Committed: Fri Sep 29 15:05:30 2017 -0700 -- CHANGES.txt | 1 + .../db/commitlog/CommitLogReplayer.java | 57 +--- .../db/commitlog/CommitLogSegment.java | 2 +- .../db/commitlog/CompressedSegment.java | 2 +- .../db/commitlog/MemoryMappedSegment.java | 2 +- .../cassandra/db/commitlog/CommitLogTest.java | 53 ++ 6 files changed, 108 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/95839aae/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index a1f49cd..7ff61d3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.15 + * Filter header only commit logs before recovery (CASSANDRA-13918) * AssertionError prepending to a list (CASSANDRA-13149) * Fix support for SuperColumn tables (CASSANDRA-12373) * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) http://git-wip-us.apache.org/repos/asf/cassandra/blob/95839aae/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java -- diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java index b3b26dd..4fd263c 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java @@ -25,18 +25,15 @@ import java.io.FileOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.*; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; import java.util.zip.CRC32; import com.google.common.base.Predicate; -import com.google.common.base.Throwables; import com.google.common.collect.HashMultimap; import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; import com.google.common.collect.Ordering; -import com.google.common.util.concurrent.Uninterruptibles; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -138,11 +135,59 @@ public class CommitLogReplayer return new CommitLogReplayer(commitLog, globalPosition, cfPersisted, replayFilter); } +private static boolean shouldSkip(File file) throws IOException, ConfigurationException +{ +CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.getName()); +if (desc.version < CommitLogDescriptor.VERSION_21) +{ +return false; +} +try(ChannelProxy channel = new ChannelProxy(file); +RandomAccessReader reader = RandomAccessReader.open(channel)) +{ +CommitLogDescriptor.readHeader(reader); +int end = reader.readInt(); +long filecrc = reader.readInt() & 0xL; +return end == 0 && filecrc == 0; +} +} + +private static List filterCommitLogFiles(File[] toFilter) +{ +List filtered = new ArrayList<>(toFilter.length); +for (File file: toFilter) +{ +try +{ +if (shouldSkip(file)) +{ +logger.info("Skipping playback of empty log: {}", file.getName()); +} +else +{ +filtered.add(file); +} +} +catch (Exception e) +{ +// let recover deal with it +filtered.add(file); +} +} + +return filtered; +} + public void recover(File[] clogs) throws IOException { -int i; -for (i = 0; i < clogs.length; ++i) -recover(clogs[i], i + 1 == clogs.length); +List filteredLogs = filterCommitLogFiles(clogs); + +int i = 0; +for (File clog: filteredLogs) +{ +i++; +recover(clog, i == filteredLogs.size()); +} } /**
[6/6] cassandra git commit: Merge branch 'cassandra-3.11' into trunk
Merge branch 'cassandra-3.11' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/77abf868 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/77abf868 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/77abf868 Branch: refs/heads/trunk Commit: 77abf868a4f60f6978c8d3e334c1a2275c4c37a3 Parents: ebefc96 be21174 Author: Blake EgglestonAuthored: Fri Sep 29 15:33:44 2017 -0700 Committer: Blake Eggleston Committed: Fri Sep 29 15:34:44 2017 -0700 -- CHANGES.txt | 1 + .../cassandra/db/commitlog/CommitLogReader.java | 48 +- .../db/commitlog/CommitLogReplayer.java | 7 ++- .../db/commitlog/CommitLogSegment.java | 2 +- .../db/commitlog/CompressedSegment.java | 2 +- .../db/commitlog/EncryptedSegment.java | 4 +- .../db/commitlog/MemoryMappedSegment.java | 2 +- .../cassandra/db/commitlog/CommitLogTest.java | 52 8 files changed, 109 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/77abf868/CHANGES.txt -- diff --cc CHANGES.txt index 99b5a59,a782333..1495c5d --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -255,6 -115,6 +255,7 @@@ Merged from 2.1 * Fix cqlsh automatic protocol downgrade regression (CASSANDRA-13307) * Tracing payload not passed from QueryMessage to tracing session (CASSANDRA-12835) Merged from 3.0: ++ * Filter header only commit logs before recovery (CASSANDRA-13918) * Ensure int overflow doesn't occur when calculating large partition warning size (CASSANDRA-13172) * Ensure consistent view of partition columns between coordinator and replica in ColumnFilter (CASSANDRA-13004) * Failed unregistering mbean during drop keyspace (CASSANDRA-13346) http://git-wip-us.apache.org/repos/asf/cassandra/blob/77abf868/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java -- diff --cc src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java index 864325b,4d74557..75ef8e9 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java @@@ -35,7 -34,8 +35,8 @@@ import org.apache.cassandra.db.commitlo import org.apache.cassandra.db.commitlog.CommitLogReadHandler.CommitLogReadException; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.rows.SerializationHelper; +import org.apache.cassandra.exceptions.UnknownTableException; + import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.io.util.ChannelProxy; import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.FileDataInput; import org.apache.cassandra.io.util.RandomAccessReader; @@@ -78,6 -77,48 +79,44 @@@ public class CommitLogReade readAllFiles(handler, files, CommitLogPosition.NONE); } + private static boolean shouldSkip(File file) throws IOException, ConfigurationException + { + CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.getName()); -if (desc.version < CommitLogDescriptor.VERSION_21) -{ -return false; -} + try(RandomAccessReader reader = RandomAccessReader.open(file)) + { + CommitLogDescriptor.readHeader(reader, DatabaseDescriptor.getEncryptionContext()); + int end = reader.readInt(); + long filecrc = reader.readInt() & 0xL; + return end == 0 && filecrc == 0; + } + } + -private static List filterCommitLogFiles(File[] toFilter) ++static List filterCommitLogFiles(File[] toFilter) + { + List filtered = new ArrayList<>(toFilter.length); + for (File file: toFilter) + { + try + { + if (shouldSkip(file)) + { + logger.info("Skipping playback of empty log: {}", file.getName()); + } + else + { + filtered.add(file); + } + } + catch (Exception e) + { + // let recover deal with it + filtered.add(file); + } + } + + return filtered; + } + /** * Reads all passed in files with minPosition, no start, and no mutation limit. */ http://git-wip-us.apache.org/repos/asf/cassandra/blob/77abf868/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
[1/6] cassandra git commit: Filter header only commit logs before recovery
Repository: cassandra Updated Branches: refs/heads/cassandra-3.0 ab0adf9f9 -> 95839aae2 refs/heads/cassandra-3.11 2a24acfa9 -> be2117492 refs/heads/trunk ebefc96a8 -> 77abf868a Filter header only commit logs before recovery Patch by Blake Eggleston; Reviewed by Sam Tunnicliffe for CASSANDRA-13918 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/95839aae Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/95839aae Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/95839aae Branch: refs/heads/cassandra-3.0 Commit: 95839aae2fde28fa29b16741de6bd52c0697843f Parents: ab0adf9 Author: Blake EgglestonAuthored: Thu Sep 28 15:01:35 2017 -0700 Committer: Blake Eggleston Committed: Fri Sep 29 15:05:30 2017 -0700 -- CHANGES.txt | 1 + .../db/commitlog/CommitLogReplayer.java | 57 +--- .../db/commitlog/CommitLogSegment.java | 2 +- .../db/commitlog/CompressedSegment.java | 2 +- .../db/commitlog/MemoryMappedSegment.java | 2 +- .../cassandra/db/commitlog/CommitLogTest.java | 53 ++ 6 files changed, 108 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/95839aae/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index a1f49cd..7ff61d3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.15 + * Filter header only commit logs before recovery (CASSANDRA-13918) * AssertionError prepending to a list (CASSANDRA-13149) * Fix support for SuperColumn tables (CASSANDRA-12373) * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) http://git-wip-us.apache.org/repos/asf/cassandra/blob/95839aae/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java -- diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java index b3b26dd..4fd263c 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java @@ -25,18 +25,15 @@ import java.io.FileOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.*; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; import java.util.zip.CRC32; import com.google.common.base.Predicate; -import com.google.common.base.Throwables; import com.google.common.collect.HashMultimap; import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; import com.google.common.collect.Ordering; -import com.google.common.util.concurrent.Uninterruptibles; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -138,11 +135,59 @@ public class CommitLogReplayer return new CommitLogReplayer(commitLog, globalPosition, cfPersisted, replayFilter); } +private static boolean shouldSkip(File file) throws IOException, ConfigurationException +{ +CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.getName()); +if (desc.version < CommitLogDescriptor.VERSION_21) +{ +return false; +} +try(ChannelProxy channel = new ChannelProxy(file); +RandomAccessReader reader = RandomAccessReader.open(channel)) +{ +CommitLogDescriptor.readHeader(reader); +int end = reader.readInt(); +long filecrc = reader.readInt() & 0xL; +return end == 0 && filecrc == 0; +} +} + +private static List filterCommitLogFiles(File[] toFilter) +{ +List filtered = new ArrayList<>(toFilter.length); +for (File file: toFilter) +{ +try +{ +if (shouldSkip(file)) +{ +logger.info("Skipping playback of empty log: {}", file.getName()); +} +else +{ +filtered.add(file); +} +} +catch (Exception e) +{ +// let recover deal with it +filtered.add(file); +} +} + +return filtered; +} + public void recover(File[] clogs) throws IOException { -int i; -for (i = 0; i < clogs.length; ++i) -recover(clogs[i], i + 1 == clogs.length); +List filteredLogs = filterCommitLogFiles(clogs); + +int i = 0; +for (File clog: filteredLogs) +
[jira] [Commented] (CASSANDRA-13906) Properly close StreamCompressionInputStream to release any ByteBuf
[ https://issues.apache.org/jira/browse/CASSANDRA-13906?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186600#comment-16186600 ] Jason Brown commented on CASSANDRA-13906: - bq. Isn't that kind of abusing the idiom of reference counting by not counting? That is true to a degree, but I'm never sure if any code would, in some broken way, become executed twice and fail on the refCnt decrement and mask some other problem. I could set the buffer to null after a simple call to {{release()}}. wdyt? > Properly close StreamCompressionInputStream to release any ByteBuf > -- > > Key: CASSANDRA-13906 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13906 > Project: Cassandra > Issue Type: Bug >Reporter: Jason Brown >Assignee: Jason Brown > > When running dtests for trunk (4.x) that perform some streaming, sometimes a > {{ByteBuf}} is not released properly, and we get this error in the logs > (causing the dtest to fail): > {code} > ERROR [MessagingService-NettyOutbound-Thread-4-2] 2017-09-26 13:42:37,940 > Slf4JLogger.java:176 - LEAK: ByteBuf.release() was not called before it's > garbage-collected. Enable advanced leak reporting to find out where the leak > occurred. To enable advanced leak reporting, specify the JVM option > '-Dio.netty.leakDetection.level=advanced' or call > ResourceLeakDetector.setLevel() See > http://netty.io/wiki/reference-counted-objects.html for more information. > {code} -- 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-13915) Create a Docker container to build the docs
[ https://issues.apache.org/jira/browse/CASSANDRA-13915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186557#comment-16186557 ] Jason Brown edited comment on CASSANDRA-13915 at 9/29/17 10:14 PM: --- Ok, so I applied this patch to trunk. I had docker installed, but I needed to install docker-compose. I'm using ubuntu 17.04 and just the docker/docker-compase packages it provides via apt-get: docker version = "Docker version 1.12.6, build 78d1802", docker-compose version = "docker-compose version 1.8.0, build unknown". I get the following error when using the instructions in the patch: {code} ERROR: Version in "./docker-compose.yml" is unsupported. You might be seeing this error because you're using the wrong Compose file version. Either specify a version of "2" (or "2.0") and place your service definitions under the `services` key, or omit the `version` key and place your service definitions at the root of the file to use version 1. For more on the Compose file format versions, see https://docs.docker.com/compose/compose-file/ {code} Note: I know nothing about docker, so this might be a noob problem. UPDATE: this was a total noob problem. Removed the apt packages and used the instructions on the docker website to install was (Author: jasobrown): Ok, so I applied this patch to trunk. I had docker installed, but I needed to install docker-compose. I'm using ubuntu 17.04 and just the docker/docker-compase packages it provides via apt-get: docker version = "Docker version 1.12.6, build 78d1802", docker-compose version = "docker-compose version 1.8.0, build unknown". I get the following error when using the instructions in the patch: {code} ERROR: Version in "./docker-compose.yml" is unsupported. You might be seeing this error because you're using the wrong Compose file version. Either specify a version of "2" (or "2.0") and place your service definitions under the `services` key, or omit the `version` key and place your service definitions at the root of the file to use version 1. For more on the Compose file format versions, see https://docs.docker.com/compose/compose-file/ {code} Note: I know nothing about docker, so this might be a noob problem. > Create a Docker container to build the docs > --- > > Key: CASSANDRA-13915 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13915 > Project: Cassandra > Issue Type: Improvement >Reporter: Joaquin Casares >Assignee: Joaquin Casares > > As requested by [~rustyrazorblade], I will be adding a Docker container to > build the docs without any prereqs (other than Docker). -- 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-13915) Create a Docker container to build the docs
[ https://issues.apache.org/jira/browse/CASSANDRA-13915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186581#comment-16186581 ] Joaquin Casares commented on CASSANDRA-13915: - Thanks for taking a look [~jasobrown]! I just pushed an update to my branch so that it supports Docker Compose 2.0+ yamls. Basically, the version of Docker Compose in 17.04 seems to be a bit old, but I'm not using any 2.1-specific features, so I just downgraded the version on my {{docker-compose.yml}} from {{2.1}} to {{2.0}}. Applying the new commit should work well for you. Thanks again! > Create a Docker container to build the docs > --- > > Key: CASSANDRA-13915 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13915 > Project: Cassandra > Issue Type: Improvement >Reporter: Joaquin Casares >Assignee: Joaquin Casares > > As requested by [~rustyrazorblade], I will be adding a Docker container to > build the docs without any prereqs (other than Docker). -- 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-13915) Create a Docker container to build the docs
[ https://issues.apache.org/jira/browse/CASSANDRA-13915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186557#comment-16186557 ] Jason Brown edited comment on CASSANDRA-13915 at 9/29/17 10:00 PM: --- Ok, so I applied this patch to trunk. I had docker installed, but I needed to install docker-compose. I'm using ubuntu 17.04 and just the docker/docker-compase packages it provides via apt-get: docker version = "Docker version 1.12.6, build 78d1802", docker-compose version = "docker-compose version 1.8.0, build unknown". I get the following error when using the instructions in the patch: {code} ERROR: Version in "./docker-compose.yml" is unsupported. You might be seeing this error because you're using the wrong Compose file version. Either specify a version of "2" (or "2.0") and place your service definitions under the `services` key, or omit the `version` key and place your service definitions at the root of the file to use version 1. For more on the Compose file format versions, see https://docs.docker.com/compose/compose-file/ {code} Note: I know nothing about docker, so this might be a noob problem. was (Author: jasobrown): Ok, so I applied this patch to trunk. I had docker installed, but I needed to install docker-compose. I'm using ubuntu 17.04 and just the docker/docker-compase packages it provides via apt-get: docker version = "Docker version 1.12.6, build 78d1802", docker-compose version = "docker-compose version 1.8.0, build unknown". I get the following error when using the instructions in the patch: {notformat} ERROR: Version in "./docker-compose.yml" is unsupported. You might be seeing this error because you're using the wrong Compose file version. Either specify a version of "2" (or "2.0") and place your service definitions under the `services` key, or omit the `version` key and place your service definitions at the root of the file to use version 1. For more on the Compose file format versions, see https://docs.docker.com/compose/compose-file/ {noformat} Note: I know nothing about docker, so this might be a noob problem. > Create a Docker container to build the docs > --- > > Key: CASSANDRA-13915 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13915 > Project: Cassandra > Issue Type: Improvement >Reporter: Joaquin Casares >Assignee: Joaquin Casares > > As requested by [~rustyrazorblade], I will be adding a Docker container to > build the docs without any prereqs (other than Docker). -- 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-13915) Create a Docker container to build the docs
[ https://issues.apache.org/jira/browse/CASSANDRA-13915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186557#comment-16186557 ] Jason Brown commented on CASSANDRA-13915: - Ok, so I applied this patch to trunk. I had docker installed, but I needed to install docker-compose. I'm using ubuntu 17.04 and just the docker/docker-compase packages it provides via apt-get: docker version = "Docker version 1.12.6, build 78d1802", docker-compose version = "docker-compose version 1.8.0, build unknown". I get the following error when using the instructions in the patch: {notformat} ERROR: Version in "./docker-compose.yml" is unsupported. You might be seeing this error because you're using the wrong Compose file version. Either specify a version of "2" (or "2.0") and place your service definitions under the `services` key, or omit the `version` key and place your service definitions at the root of the file to use version 1. For more on the Compose file format versions, see https://docs.docker.com/compose/compose-file/ {noformat} Note: I know nothing about docker, so this might be a noob problem. > Create a Docker container to build the docs > --- > > Key: CASSANDRA-13915 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13915 > Project: Cassandra > Issue Type: Improvement >Reporter: Joaquin Casares >Assignee: Joaquin Casares > > As requested by [~rustyrazorblade], I will be adding a Docker container to > build the docs without any prereqs (other than Docker). -- 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-13918) Header only commit logs should be filtered before recovery
[ https://issues.apache.org/jira/browse/CASSANDRA-13918?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186459#comment-16186459 ] Blake Eggleston commented on CASSANDRA-13918: - utests look good on all branches, dtests look good on 3.0 & 3.11. trunk dtests aborted, but looks like it was due to an existing issue. > Header only commit logs should be filtered before recovery > -- > > Key: CASSANDRA-13918 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13918 > Project: Cassandra > Issue Type: Bug >Reporter: Blake Eggleston >Assignee: Blake Eggleston > Fix For: 3.0.15, 4.0, 3.11.x > > > Commit log recovery will tolerate commit log truncation in the most recent > log file found on disk, but will abort startup if problems are detected in > others. > Since we allocate commit log segments before they're used though, it's > possible to get into a state where the last commit log file actually written > to is not the same file that was most recently allocated, preventing startup > for what should otherwise be allowable incomplete final segments. > Excluding header only files on recovery should prevent this from happening. -- 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-13915) Create a Docker container to build the docs
[ https://issues.apache.org/jira/browse/CASSANDRA-13915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186448#comment-16186448 ] Jon Haddad commented on CASSANDRA-13915: The more eyes the better, especially given the lack of attention docs have had historically. I'll still also review it Monday. Once this is done I'll work with [~mshuler] to do the back half, pushing the updating site docs up and documenting that process. > Create a Docker container to build the docs > --- > > Key: CASSANDRA-13915 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13915 > Project: Cassandra > Issue Type: Improvement >Reporter: Joaquin Casares >Assignee: Joaquin Casares > > As requested by [~rustyrazorblade], I will be adding a Docker container to > build the docs without any prereqs (other than Docker). -- 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-13915) Create a Docker container to build the docs
[ https://issues.apache.org/jira/browse/CASSANDRA-13915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186442#comment-16186442 ] Jason Brown commented on CASSANDRA-13915: - [~rustyrazorblade] [~mshuler] I love this. Thanks for the details! I can review the ticket, as well, since I've started poking into it anyway. > Create a Docker container to build the docs > --- > > Key: CASSANDRA-13915 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13915 > Project: Cassandra > Issue Type: Improvement >Reporter: Joaquin Casares >Assignee: Joaquin Casares > > As requested by [~rustyrazorblade], I will be adding a Docker container to > build the docs without any prereqs (other than Docker). -- 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
[1/2] cassandra-dtest git commit: Fix ISE thrown by UPI.Serializer.hasNext() for some SELECT queries
Repository: cassandra-dtest Updated Branches: refs/heads/master f39b468b3 -> b76a06672 Fix ISE thrown by UPI.Serializer.hasNext() for some SELECT queries patch by Aleksey Yeschenko; reviewed by Sam Tunnicliffe for CASSANDRA-13911 Project: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/commit/51ad68ec Tree: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/tree/51ad68ec Diff: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/diff/51ad68ec Branch: refs/heads/master Commit: 51ad68ec45c7a40de1c51b31651632f2e87ceaa4 Parents: f39b468 Author: Aleksey YeschenkoAuthored: Wed Sep 27 13:09:05 2017 +0100 Committer: Aleksey Yeschenko Committed: Fri Sep 29 22:13:08 2017 +0100 -- consistency_test.py | 49 1 file changed, 49 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/51ad68ec/consistency_test.py -- diff --git a/consistency_test.py b/consistency_test.py index 407873c..b6ecff6 100644 --- a/consistency_test.py +++ b/consistency_test.py @@ -774,6 +774,55 @@ class TestAccuracy(TestHelper): class TestConsistency(Tester): @since('3.0') +def test_13911(self): +""" +@jira_ticket CASSANDRA-13911 +""" +cluster = self.cluster + +# disable hinted handoff and set batch commit log so this doesn't interfere with the test +cluster.set_configuration_options(values={'hinted_handoff_enabled': False}) +cluster.set_batch_commitlog(enabled=True) + +cluster.populate(2).start(wait_other_notice=True) +node1, node2 = cluster.nodelist() + +session = self.patient_cql_connection(node1) + +query = "CREATE KEYSPACE test WITH replication = {'class': 'NetworkTopologyStrategy', 'datacenter1': 2};" +session.execute(query) + +query = 'CREATE TABLE test.test (pk int, ck int, PRIMARY KEY (pk, ck));' +session.execute(query) + +# with node2 down, insert row 0 on node1 +# +# node1, partition 0 | 0 +# node2, partition 0 | + +node2.stop(wait_other_notice=True) +session.execute('INSERT INTO test.test (pk, ck) VALUES (0, 0);') +node2.start(wait_other_notice=True, wait_for_binary_proto=True) + +# with node1 down, delete row 1 and 2 on node2 +# +# node1, partition 0 | 0 +# node2, partition 0 | x x + +session = self.patient_cql_connection(node2) + +node1.stop(wait_other_notice=True) +session.execute('DELETE FROM test.test WHERE pk = 0 AND ck IN (1, 2);') +node1.start(wait_other_notice=True, wait_for_binary_proto=True) + +# with both nodes up, do a CL.ALL query with per partition limit of 1; +# prior to CASSANDRA-13911 this would trigger an IllegalStateException +assert_all(session, + 'SELECT DISTINCT pk FROM test.test;', + [[0]], + cl=ConsistencyLevel.ALL) + +@since('3.0') def test_13880(self): """ @jira_ticket CASSANDRA-13880 - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[2/2] cassandra-dtest git commit: Implement short read protection on partition boundaries
Implement short read protection on partition boundaries patch by Aleksey Yeschenko; reviewed by Sam Tunnicliffe for CASSANDRA-13595 Project: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/commit/b76a0667 Tree: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/tree/b76a0667 Diff: http://git-wip-us.apache.org/repos/asf/cassandra-dtest/diff/b76a0667 Branch: refs/heads/master Commit: b76a06672ca418a2a7e90278886252deccdc9edd Parents: 51ad68e Author: Aleksey YeschenkoAuthored: Mon Sep 25 16:50:26 2017 +0100 Committer: Aleksey Yeschenko Committed: Fri Sep 29 22:13:11 2017 +0100 -- consistency_test.py | 70 1 file changed, 70 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra-dtest/blob/b76a0667/consistency_test.py -- diff --git a/consistency_test.py b/consistency_test.py index b6ecff6..1a624c3 100644 --- a/consistency_test.py +++ b/consistency_test.py @@ -16,6 +16,7 @@ from dtest import DISABLE_VNODES, MultiError, Tester, debug, create_ks, create_c from tools.data import (create_c1c2_table, insert_c1c2, insert_columns, query_c1c2, rows_to_list) from tools.decorators import since +from tools.jmxutils import JolokiaAgent, make_mbean, remove_perf_disable_shared_mem ExpectedConsistency = namedtuple('ExpectedConsistency', ('num_write_nodes', 'num_read_nodes', 'is_strong')) @@ -911,6 +912,75 @@ class TestConsistency(Tester): assert_length_equal(result, 5) @since('3.0') +def test_13595(self): +""" +@jira_ticket CASSANDRA-13595 +""" +cluster = self.cluster + +# disable hinted handoff and set batch commit log so this doesn't interfere with the test +cluster.set_configuration_options(values={'hinted_handoff_enabled': False}) +cluster.set_batch_commitlog(enabled=True) + +cluster.populate(2) +node1, node2 = cluster.nodelist() +remove_perf_disable_shared_mem(node1) # necessary for jmx +cluster.start(wait_other_notice=True) + +session = self.patient_cql_connection(node1) + +query = "CREATE KEYSPACE IF NOT EXISTS test WITH replication = {'class': 'NetworkTopologyStrategy', 'datacenter1': 2};" +session.execute(query) + +query = 'CREATE TABLE IF NOT EXISTS test.test (id int PRIMARY KEY);' +session.execute(query) + +# populate the table with 10 partitions, +# then delete a bunch of them on different nodes +# until we get the following pattern: + +#token | k | 1 | 2 | +# -7509452495886106294 | 5 | n | y | +# -4069959284402364209 | 1 | y | n | +# -3799847372828181882 | 8 | n | y | +# -3485513579396041028 | 0 | y | n | +# -3248873570005575792 | 2 | n | y | +# -2729420104000364805 | 4 | y | n | +# 1634052884888577606 | 7 | n | y | +# 2705480034054113608 | 6 | y | n | +# 3728482343045213994 | 9 | n | y | +# 9010454139840013625 | 3 | y | y | + +stmt = session.prepare('INSERT INTO test.test (id) VALUES (?);') +for id in range(0, 10): +session.execute(stmt, [id], ConsistencyLevel.ALL) + +# delete every other partition on node1 while node2 is down +node2.stop(wait_other_notice=True) +session.execute('DELETE FROM test.test WHERE id IN (5, 8, 2, 7, 9);') +node2.start(wait_other_notice=True, wait_for_binary_proto=True) + +session = self.patient_cql_connection(node2) + +# delete every other alternate partition on node2 while node1 is down +node1.stop(wait_other_notice=True) +session.execute('DELETE FROM test.test WHERE id IN (1, 0, 4, 6);') +node1.start(wait_other_notice=True, wait_for_binary_proto=True) + +session = self.patient_exclusive_cql_connection(node1) + +# until #13595 the query would incorrectly return [1] +assert_all(session, + 'SELECT id FROM test.test LIMIT 1;', + [[3]], + cl = ConsistencyLevel.ALL) + +srp = make_mbean('metrics', type='Table', name='ShortReadProtectionRequests', keyspace='test', scope='test') +with JolokiaAgent(node1) as jmx: +# 4 srp requests for node1 and 5 for node2, total of 9 +self.assertEqual(9, jmx.read_attribute(srp, 'Count')) + +@since('3.0') def test_12872(self): """ @jira_ticket CASSANDRA-12872 - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional
[jira] [Comment Edited] (CASSANDRA-13915) Create a Docker container to build the docs
[ https://issues.apache.org/jira/browse/CASSANDRA-13915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186405#comment-16186405 ] Michael Shuler edited comment on CASSANDRA-13915 at 9/29/17 8:50 PM: - I found that even building docs on the same version of jekyll, etc. on Mac or Linux produced different resulting html files. We thought a docker instance with all the right bits installed and a run script should produce the same results for anyone. (*Edit*: I may actually be thinking about the site pages here..) was (Author: mshuler): I found that even building docs on the same version of jekyll, etc. on Mac or Linux produced different resulting html files. We thought a docker instance with all the right bits installed and a run script should produce the same results for anyone. > Create a Docker container to build the docs > --- > > Key: CASSANDRA-13915 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13915 > Project: Cassandra > Issue Type: Improvement >Reporter: Joaquin Casares >Assignee: Joaquin Casares > > As requested by [~rustyrazorblade], I will be adding a Docker container to > build the docs without any prereqs (other than Docker). -- 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-13915) Create a Docker container to build the docs
[ https://issues.apache.org/jira/browse/CASSANDRA-13915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186405#comment-16186405 ] Michael Shuler commented on CASSANDRA-13915: I found that even building docs on the same version of jekyll, etc. on Mac or Linux produced different resulting html files. We thought a docker instance with all the right bits installed and a run script should produce the same results for anyone. > Create a Docker container to build the docs > --- > > Key: CASSANDRA-13915 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13915 > Project: Cassandra > Issue Type: Improvement >Reporter: Joaquin Casares >Assignee: Joaquin Casares > > As requested by [~rustyrazorblade], I will be adding a Docker container to > build the docs without any prereqs (other than Docker). -- 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-13915) Create a Docker container to build the docs
[ https://issues.apache.org/jira/browse/CASSANDRA-13915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186395#comment-16186395 ] Jon Haddad commented on CASSANDRA-13915: [~jasobrown] Currently the dependencies of the docs are difficult to install correctly. I had worked with [~bdeggleston] on CASSANDRA-13253 a ways back and simply getting the docs to build _at all_ in a reliable and repeatable required several rounds of attempts and turned into an unnecessary time suck. Blake and I had a difficult time getting it working despite a combined 15+ years working with Python and maybe 10 years working with Sphinx. I'm looking to simplify the process and make it easier for folks to build and contribute to the documentation. Contributing to the docs, in my opinion, should have a very low barrier to entry. Getting the dependencies right isn't a great use of time, and Docker lets us standardize it. The docker file will also be helpful in standardizing the way we publish the docs to the site, which, at the moment, is very difficult and rarely done. The desire to do this partially came out of a discussion I had with [~mshuler] at NGCC where I expressed interest in making the online docs easier to update. > Create a Docker container to build the docs > --- > > Key: CASSANDRA-13915 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13915 > Project: Cassandra > Issue Type: Improvement >Reporter: Joaquin Casares >Assignee: Joaquin Casares > > As requested by [~rustyrazorblade], I will be adding a Docker container to > build the docs without any prereqs (other than Docker). -- 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-12966) Gossip thread slows down when using batch commit log
[ https://issues.apache.org/jira/browse/CASSANDRA-12966?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186341#comment-16186341 ] Jason Brown commented on CASSANDRA-12966: - [~jjirsa] fixed > Gossip thread slows down when using batch commit log > > > Key: CASSANDRA-12966 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12966 > Project: Cassandra > Issue Type: Improvement >Reporter: Jason Brown >Assignee: Jason Brown >Priority: Minor > Fix For: 3.0.15, 3.11.1, 4.0 > > > When using batch commit log mode, the Gossip thread slows down when peers > after a node bounces. This is because we perform a bunch of updates to the > peers table via {{SystemKeyspace.updatePeerInfo}}, which is a synchronized > method. How quickly each one of those individual updates takes depends on how > busy the system is at the time wrt write traffic. If the system is largely > quiescent, each update will be relatively quick (just waiting for the fsync). > If the system is getting a lot of writes, and depending on the > commitlog_sync_batch_window_in_ms, each of the Gossip thread's updates can > get stuck in the backlog, which causes the Gossip thread to stop processing. > We have observed in large clusters that a rolling restart causes triggers and > exacerbates this behavior. -- 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-12966) Gossip thread slows down when using batch commit log
[ https://issues.apache.org/jira/browse/CASSANDRA-12966?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Brown updated CASSANDRA-12966: Fix Version/s: 4.0 3.11.1 3.0.15 > Gossip thread slows down when using batch commit log > > > Key: CASSANDRA-12966 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12966 > Project: Cassandra > Issue Type: Improvement >Reporter: Jason Brown >Assignee: Jason Brown >Priority: Minor > Fix For: 3.0.15, 3.11.1, 4.0 > > > When using batch commit log mode, the Gossip thread slows down when peers > after a node bounces. This is because we perform a bunch of updates to the > peers table via {{SystemKeyspace.updatePeerInfo}}, which is a synchronized > method. How quickly each one of those individual updates takes depends on how > busy the system is at the time wrt write traffic. If the system is largely > quiescent, each update will be relatively quick (just waiting for the fsync). > If the system is getting a lot of writes, and depending on the > commitlog_sync_batch_window_in_ms, each of the Gossip thread's updates can > get stuck in the backlog, which causes the Gossip thread to stop processing. > We have observed in large clusters that a rolling restart causes triggers and > exacerbates this behavior. -- 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-13529) cassandra-stress light-weight transaction support
[ https://issues.apache.org/jira/browse/CASSANDRA-13529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186315#comment-16186315 ] Jaydeepkumar Chovatia commented on CASSANDRA-13529: --- Hi [~jasobrown] Could you please review this whenever you get a chance? Thanks Jaydeep > cassandra-stress light-weight transaction support > - > > Key: CASSANDRA-13529 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13529 > Project: Cassandra > Issue Type: New Feature > Components: Stress >Reporter: Jaydeepkumar Chovatia >Assignee: Jaydeepkumar Chovatia >Priority: Minor > Fix For: 4.x > > Attachments: 13529.txt, lwttest.yaml > > > It would be nice to have a light-weight transaction support in > cassandra-stress. > Although currently in cassandra-stress we can achieve light-weight > transaction partially by using static conditions like "IF col1 != null" or > "IF not EXIST". > If would be ideal to have full fledged light-weight transaction support like > "IF col1 = ? and col2 = ?". One way to implement is to read values from > Cassandra and use that in the condition so it will execute all the paxos > phases in Cassandra. > Please find git link for the patch: > https://github.com/apache/cassandra/compare/trunk...jaydeepkumar1984:13529-trunk?expand=1 > ||trunk| > |[branch|https://github.com/jaydeepkumar1984/cassandra/tree/13529-trunk]| > |[utests|https://circleci.com/gh/jaydeepkumar1984/cassandra/8]| -- 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-13906) Properly close StreamCompressionInputStream to release any ByteBuf
[ https://issues.apache.org/jira/browse/CASSANDRA-13906?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186310#comment-16186310 ] Ariel Weisberg commented on CASSANDRA-13906: Is releasing references to a buffer using the {{refCnt()}} a good idea? Isn't that kind of abusing the idiom of reference counting by not counting? Otherwise looks good. > Properly close StreamCompressionInputStream to release any ByteBuf > -- > > Key: CASSANDRA-13906 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13906 > Project: Cassandra > Issue Type: Bug >Reporter: Jason Brown >Assignee: Jason Brown > > When running dtests for trunk (4.x) that perform some streaming, sometimes a > {{ByteBuf}} is not released properly, and we get this error in the logs > (causing the dtest to fail): > {code} > ERROR [MessagingService-NettyOutbound-Thread-4-2] 2017-09-26 13:42:37,940 > Slf4JLogger.java:176 - LEAK: ByteBuf.release() was not called before it's > garbage-collected. Enable advanced leak reporting to find out where the leak > occurred. To enable advanced leak reporting, specify the JVM option > '-Dio.netty.leakDetection.level=advanced' or call > ResourceLeakDetector.setLevel() See > http://netty.io/wiki/reference-counted-objects.html for more information. > {code} -- 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-13595) Implement short read protection on partition boundaries
[ https://issues.apache.org/jira/browse/CASSANDRA-13595?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186272#comment-16186272 ] Aleksey Yeschenko edited comment on CASSANDRA-13595 at 9/29/17 7:48 PM: Thanks. Squashed the last two commits together, will commit once CI is happy. was (Author: iamaleksey): Thanks. Squashed the last to commits together, will commit once CI is happy. > Implement short read protection on partition boundaries > --- > > Key: CASSANDRA-13595 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13595 > Project: Cassandra > Issue Type: Bug > Components: Coordination >Reporter: Andrés de la Peña >Assignee: Aleksey Yeschenko > Labels: Correctness > Fix For: 3.0.15, 3.11.1 > > > It seems that short read protection doesn't work when the short read is done > at the end of a partition in a range query. The final assertion of this dtest > fails: > {code} > def short_read_partitions_delete_test(self): > cluster = self.cluster > cluster.set_configuration_options(values={'hinted_handoff_enabled': > False}) > cluster.set_batch_commitlog(enabled=True) > cluster.populate(2).start(wait_other_notice=True) > node1, node2 = self.cluster.nodelist() > session = self.patient_cql_connection(node1) > create_ks(session, 'ks', 2) > session.execute("CREATE TABLE t (k int, c int, PRIMARY KEY(k, c)) > WITH read_repair_chance = 0.0") > # we write 1 and 2 in a partition: all nodes get it. > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (1, 1)", > consistency_level=ConsistencyLevel.ALL)) > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (2, 1)", > consistency_level=ConsistencyLevel.ALL)) > # we delete partition 1: only node 1 gets it. > node2.flush() > node2.stop(wait_other_notice=True) > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 1")) > node2.start(wait_other_notice=True) > # we delete partition 2: only node 2 gets it. > node1.flush() > node1.stop(wait_other_notice=True) > session = self.patient_cql_connection(node2, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 2")) > node1.start(wait_other_notice=True) > # read from both nodes > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ALL) > assert_none(session, "SELECT * FROM t LIMIT 1") > {code} > However, the dtest passes if we remove the {{LIMIT 1}}. > Short read protection [uses a > {{SinglePartitionReadCommand}}|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/DataResolver.java#L484], > maybe it should use a {{PartitionRangeReadCommand}} instead? -- 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-12966) Gossip thread slows down when using batch commit log
[ https://issues.apache.org/jira/browse/CASSANDRA-12966?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186295#comment-16186295 ] Jeff Jirsa commented on CASSANDRA-12966: Missing fixversions here, [~jasobrown] > Gossip thread slows down when using batch commit log > > > Key: CASSANDRA-12966 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12966 > Project: Cassandra > Issue Type: Improvement >Reporter: Jason Brown >Assignee: Jason Brown >Priority: Minor > > When using batch commit log mode, the Gossip thread slows down when peers > after a node bounces. This is because we perform a bunch of updates to the > peers table via {{SystemKeyspace.updatePeerInfo}}, which is a synchronized > method. How quickly each one of those individual updates takes depends on how > busy the system is at the time wrt write traffic. If the system is largely > quiescent, each update will be relatively quick (just waiting for the fsync). > If the system is getting a lot of writes, and depending on the > commitlog_sync_batch_window_in_ms, each of the Gossip thread's updates can > get stuck in the backlog, which causes the Gossip thread to stop processing. > We have observed in large clusters that a rolling restart causes triggers and > exacerbates this behavior. -- 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-13913) Be more optimistic when fetching more rows for SRP
[ https://issues.apache.org/jira/browse/CASSANDRA-13913?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-13913: -- Fix Version/s: (was: 3.11.1) (was: 3.0.15) 3.11.x 3.0.x > Be more optimistic when fetching more rows for SRP > -- > > Key: CASSANDRA-13913 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13913 > Project: Cassandra > Issue Type: Improvement >Reporter: Aleksey Yeschenko >Assignee: Aleksey Yeschenko > Fix For: 3.0.x, 3.11.x > > > CASSANDRA-13794 fixed the issue with fetching at most 1 extra row for SRP by > instead fetching the per partition limit (if set) or global limit (if not). > As noticed, however, it might be a bit too pessimistic, and double the load > even when are only missing an extra row. > Before 3.0.15 with that change is released, I'd like to make a small change, > so nobody gets surprised by sudden load increases. We'd start with {{2 * > (limit - counted)}} and if that's not enough, exponentially increase # of > extra rows fetched until we hit {{limit}}. -- 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-13595) Implement short read protection on partition boundaries
[ https://issues.apache.org/jira/browse/CASSANDRA-13595?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186272#comment-16186272 ] Aleksey Yeschenko commented on CASSANDRA-13595: --- Thanks. Squashed the last to commits together, will commit once CI is happy. > Implement short read protection on partition boundaries > --- > > Key: CASSANDRA-13595 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13595 > Project: Cassandra > Issue Type: Bug > Components: Coordination >Reporter: Andrés de la Peña >Assignee: Aleksey Yeschenko > Labels: Correctness > Fix For: 3.0.15, 3.11.1 > > > It seems that short read protection doesn't work when the short read is done > at the end of a partition in a range query. The final assertion of this dtest > fails: > {code} > def short_read_partitions_delete_test(self): > cluster = self.cluster > cluster.set_configuration_options(values={'hinted_handoff_enabled': > False}) > cluster.set_batch_commitlog(enabled=True) > cluster.populate(2).start(wait_other_notice=True) > node1, node2 = self.cluster.nodelist() > session = self.patient_cql_connection(node1) > create_ks(session, 'ks', 2) > session.execute("CREATE TABLE t (k int, c int, PRIMARY KEY(k, c)) > WITH read_repair_chance = 0.0") > # we write 1 and 2 in a partition: all nodes get it. > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (1, 1)", > consistency_level=ConsistencyLevel.ALL)) > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (2, 1)", > consistency_level=ConsistencyLevel.ALL)) > # we delete partition 1: only node 1 gets it. > node2.flush() > node2.stop(wait_other_notice=True) > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 1")) > node2.start(wait_other_notice=True) > # we delete partition 2: only node 2 gets it. > node1.flush() > node1.stop(wait_other_notice=True) > session = self.patient_cql_connection(node2, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 2")) > node1.start(wait_other_notice=True) > # read from both nodes > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ALL) > assert_none(session, "SELECT * FROM t LIMIT 1") > {code} > However, the dtest passes if we remove the {{LIMIT 1}}. > Short read protection [uses a > {{SinglePartitionReadCommand}}|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/DataResolver.java#L484], > maybe it should use a {{PartitionRangeReadCommand}} instead? -- 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-13921) Unable to start Cassandra
Ramesh S Raj created CASSANDRA-13921: Summary: Unable to start Cassandra Key: CASSANDRA-13921 URL: https://issues.apache.org/jira/browse/CASSANDRA-13921 Project: Cassandra Issue Type: Test Components: Testing Environment: CentOS Linux release 7.2.1511 (Core) I am unable to start the cassandra and getting the below error: Reporter: Ramesh S Raj [root@btpvm0603 sbin]# ./cassandra -R [root@btpvm0603 sbin]# [root@btpvm0603 sbin]# CompilerOracle: dontinline org/apache/cassandra/db/Columns$Serializer.deserializeLargeSubset (Lorg/apache/cassandra/io/util/DataInputPlus;Lorg/apache/cassandra/db/Columns;I)Lorg/apache/cassandra/db/Columns; CompilerOracle: dontinline org/apache/cassandra/db/Columns$Serializer.serializeLargeSubset (Ljava/util/Collection;ILorg/apache/cassandra/db/Columns;ILorg/apache/cassandra/io/util/DataOutputPlus;)V CompilerOracle: dontinline org/apache/cassandra/db/Columns$Serializer.serializeLargeSubsetSize (Ljava/util/Collection;ILorg/apache/cassandra/db/Columns;I)I CompilerOracle: dontinline org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.advanceAllocatingFrom (Lorg/apache/cassandra/db/commitlog/CommitLogSegment;)V CompilerOracle: dontinline org/apache/cassandra/db/transform/BaseIterator.tryGetMoreContents ()Z CompilerOracle: dontinline org/apache/cassandra/db/transform/StoppingTransformation.stop ()V CompilerOracle: dontinline org/apache/cassandra/db/transform/StoppingTransformation.stopInPartition ()V CompilerOracle: dontinline org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.doFlush (I)V CompilerOracle: dontinline org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.writeExcessSlow ()V CompilerOracle: dontinline org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.writeSlow (JI)V CompilerOracle: dontinline org/apache/cassandra/io/util/RebufferingInputStream.readPrimitiveSlowly (I)J CompilerOracle: inline org/apache/cassandra/db/rows/UnfilteredSerializer.serializeRowBody (Lorg/apache/cassandra/db/rows/Row;ILorg/apache/cassandra/db/SerializationHeader;Lorg/apache/cassandra/io/util/DataOutputPlus;)V CompilerOracle: inline org/apache/cassandra/io/util/Memory.checkBounds (JJ)V CompilerOracle: inline org/apache/cassandra/io/util/SafeMemory.checkBounds (JJ)V CompilerOracle: inline org/apache/cassandra/utils/AsymmetricOrdering.selectBoundary (Lorg/apache/cassandra/utils/AsymmetricOrdering/Op;II)I CompilerOracle: inline org/apache/cassandra/utils/AsymmetricOrdering.strictnessOfLessThan (Lorg/apache/cassandra/utils/AsymmetricOrdering/Op;)I CompilerOracle: inline org/apache/cassandra/utils/BloomFilter.indexes (Lorg/apache/cassandra/utils/IFilter/FilterKey;)[J CompilerOracle: inline org/apache/cassandra/utils/BloomFilter.setIndexes (JJIJ[J)V CompilerOracle: inline org/apache/cassandra/utils/ByteBufferUtil.compare (Ljava/nio/ByteBuffer;[B)I CompilerOracle: inline org/apache/cassandra/utils/ByteBufferUtil.compare ([BLjava/nio/ByteBuffer;)I CompilerOracle: inline org/apache/cassandra/utils/ByteBufferUtil.compareUnsigned (Ljava/nio/ByteBuffer;Ljava/nio/ByteBuffer;)I CompilerOracle: inline org/apache/cassandra/utils/FastByteOperations$UnsafeOperations.compareTo (Ljava/lang/Object;JILjava/lang/Object;JI)I CompilerOracle: inline org/apache/cassandra/utils/FastByteOperations$UnsafeOperations.compareTo (Ljava/lang/Object;JILjava/nio/ByteBuffer;)I CompilerOracle: inline org/apache/cassandra/utils/FastByteOperations$UnsafeOperations.compareTo (Ljava/nio/ByteBuffer;Ljava/nio/ByteBuffer;)I CompilerOracle: inline org/apache/cassandra/utils/vint/VIntCoding.encodeVInt (JI)[B [root@btpvm0603 sbin]# Error: A JNI error has occurred, please check your installation and try again Exception in thread "main" java.lang.NoClassDefFoundError: org/yaml/snakeyaml/introspector/PropertyUtils at java.lang.Class.getDeclaredMethods0(Native Method) at java.lang.Class.privateGetDeclaredMethods(Class.java:2701) at java.lang.Class.privateGetMethodRecursive(Class.java:3048) at java.lang.Class.getMethod0(Class.java:3018) at java.lang.Class.getMethod(Class.java:1784) at sun.launcher.LauncherHelper.validateMainClass(LauncherHelper.java:544) at sun.launcher.LauncherHelper.checkAndLoadMain(LauncherHelper.java:526) Caused by: java.lang.ClassNotFoundException: org.yaml.snakeyaml.introspector.PropertyUtils at java.net.URLClassLoader.findClass(URLClassLoader.java:381) at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:331) at java.lang.ClassLoader.loadClass(ClassLoader.java:357) ... 7 more [root@btpvm0603 sbin]# [root@btpvm0603 sbin]# -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To
[jira] [Commented] (CASSANDRA-13915) Create a Docker container to build the docs
[ https://issues.apache.org/jira/browse/CASSANDRA-13915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186203#comment-16186203 ] Jason Brown commented on CASSANDRA-13915: - [~rustyrazorblade] & [~j.casares] stoopidly naive question: why do we need Docker to build our docs? > Create a Docker container to build the docs > --- > > Key: CASSANDRA-13915 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13915 > Project: Cassandra > Issue Type: Improvement >Reporter: Joaquin Casares >Assignee: Joaquin Casares > > As requested by [~rustyrazorblade], I will be adding a Docker container to > build the docs without any prereqs (other than Docker). -- 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-13915) Create a Docker container to build the docs
[ https://issues.apache.org/jira/browse/CASSANDRA-13915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186191#comment-16186191 ] Joaquin Casares commented on CASSANDRA-13915: - [~rustyrazorblade] could you review https://github.com/apache/cassandra/pull/154? Thanks! > Create a Docker container to build the docs > --- > > Key: CASSANDRA-13915 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13915 > Project: Cassandra > Issue Type: Improvement >Reporter: Joaquin Casares >Assignee: Joaquin Casares > > As requested by [~rustyrazorblade], I will be adding a Docker container to > build the docs without any prereqs (other than Docker). -- 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-13915) Create a Docker container to build the docs
[ https://issues.apache.org/jira/browse/CASSANDRA-13915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186190#comment-16186190 ] ASF GitHub Bot commented on CASSANDRA-13915: GitHub user joaquincasares opened a pull request: https://github.com/apache/cassandra/pull/154 CASSANDRA-13915 https://issues.apache.org/jira/browse/CASSANDRA-13915 You can merge this pull request into a Git repository by running: $ git pull https://github.com/joaquincasares/cassandra CASSANDRA-13915 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/cassandra/pull/154.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #154 commit eb558abda811dbbf42f6881382f883c752702d7b Author: Joaquin CasaresDate: 2017-09-29T18:17:45Z Fix documentation build warnings commit 9c35e3aa3f3dc2c736eae6f327676828d2c475b6 Author: Joaquin Casares Date: 2017-09-29T18:19:49Z Include removing doc/build as part of 'ant realclean' commit 2bcef57e744ee209caf74c6bac067d7599381304 Author: Joaquin Casares Date: 2017-09-29T18:23:21Z Add a Docker Compose setup to build the docs > Create a Docker container to build the docs > --- > > Key: CASSANDRA-13915 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13915 > Project: Cassandra > Issue Type: Improvement >Reporter: Joaquin Casares >Assignee: Joaquin Casares > > As requested by [~rustyrazorblade], I will be adding a Docker container to > build the docs without any prereqs (other than Docker). -- 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-13595) Implement short read protection on partition boundaries
[ https://issues.apache.org/jira/browse/CASSANDRA-13595?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186172#comment-16186172 ] Sam Tunnicliffe commented on CASSANDRA-13595: - Thanks, that explanation is helpful. Looks good to me, +1 assuming CI looks rosy. > Implement short read protection on partition boundaries > --- > > Key: CASSANDRA-13595 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13595 > Project: Cassandra > Issue Type: Bug > Components: Coordination >Reporter: Andrés de la Peña >Assignee: Aleksey Yeschenko > Labels: Correctness > Fix For: 3.0.15, 3.11.1 > > > It seems that short read protection doesn't work when the short read is done > at the end of a partition in a range query. The final assertion of this dtest > fails: > {code} > def short_read_partitions_delete_test(self): > cluster = self.cluster > cluster.set_configuration_options(values={'hinted_handoff_enabled': > False}) > cluster.set_batch_commitlog(enabled=True) > cluster.populate(2).start(wait_other_notice=True) > node1, node2 = self.cluster.nodelist() > session = self.patient_cql_connection(node1) > create_ks(session, 'ks', 2) > session.execute("CREATE TABLE t (k int, c int, PRIMARY KEY(k, c)) > WITH read_repair_chance = 0.0") > # we write 1 and 2 in a partition: all nodes get it. > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (1, 1)", > consistency_level=ConsistencyLevel.ALL)) > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (2, 1)", > consistency_level=ConsistencyLevel.ALL)) > # we delete partition 1: only node 1 gets it. > node2.flush() > node2.stop(wait_other_notice=True) > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 1")) > node2.start(wait_other_notice=True) > # we delete partition 2: only node 2 gets it. > node1.flush() > node1.stop(wait_other_notice=True) > session = self.patient_cql_connection(node2, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 2")) > node1.start(wait_other_notice=True) > # read from both nodes > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ALL) > assert_none(session, "SELECT * FROM t LIMIT 1") > {code} > However, the dtest passes if we remove the {{LIMIT 1}}. > Short read protection [uses a > {{SinglePartitionReadCommand}}|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/DataResolver.java#L484], > maybe it should use a {{PartitionRangeReadCommand}} instead? -- 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-13918) Header only commit logs should be filtered before recovery
[ https://issues.apache.org/jira/browse/CASSANDRA-13918?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186170#comment-16186170 ] Blake Eggleston commented on CASSANDRA-13918: - Thanks, will fix on commit > Header only commit logs should be filtered before recovery > -- > > Key: CASSANDRA-13918 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13918 > Project: Cassandra > Issue Type: Bug >Reporter: Blake Eggleston >Assignee: Blake Eggleston > Fix For: 3.0.15, 4.0, 3.11.x > > > Commit log recovery will tolerate commit log truncation in the most recent > log file found on disk, but will abort startup if problems are detected in > others. > Since we allocate commit log segments before they're used though, it's > possible to get into a state where the last commit log file actually written > to is not the same file that was most recently allocated, preventing startup > for what should otherwise be allowable incomplete final segments. > Excluding header only files on recovery should prevent this from happening. -- 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-13918) Header only commit logs should be filtered before recovery
[ https://issues.apache.org/jira/browse/CASSANDRA-13918?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186167#comment-16186167 ] Sam Tunnicliffe commented on CASSANDRA-13918: - LGTM. Minuscule nit: there's an unused import in {{CommitLogReader}} on the 3.11 branch. > Header only commit logs should be filtered before recovery > -- > > Key: CASSANDRA-13918 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13918 > Project: Cassandra > Issue Type: Bug >Reporter: Blake Eggleston >Assignee: Blake Eggleston > Fix For: 3.0.15, 4.0, 3.11.x > > > Commit log recovery will tolerate commit log truncation in the most recent > log file found on disk, but will abort startup if problems are detected in > others. > Since we allocate commit log segments before they're used though, it's > possible to get into a state where the last commit log file actually written > to is not the same file that was most recently allocated, preventing startup > for what should otherwise be allowable incomplete final segments. > Excluding header only files on recovery should prevent this from happening. -- 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-13918) Header only commit logs should be filtered before recovery
[ https://issues.apache.org/jira/browse/CASSANDRA-13918?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sam Tunnicliffe updated CASSANDRA-13918: Status: Ready to Commit (was: Patch Available) > Header only commit logs should be filtered before recovery > -- > > Key: CASSANDRA-13918 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13918 > Project: Cassandra > Issue Type: Bug >Reporter: Blake Eggleston >Assignee: Blake Eggleston > Fix For: 3.0.15, 4.0, 3.11.x > > > Commit log recovery will tolerate commit log truncation in the most recent > log file found on disk, but will abort startup if problems are detected in > others. > Since we allocate commit log segments before they're used though, it's > possible to get into a state where the last commit log file actually written > to is not the same file that was most recently allocated, preventing startup > for what should otherwise be allowable incomplete final segments. > Excluding header only files on recovery should prevent this from happening. -- 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-13906) Properly close StreamCompressionInputStream to release any ByteBuf
[ https://issues.apache.org/jira/browse/CASSANDRA-13906?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186149#comment-16186149 ] Jason Brown commented on CASSANDRA-13906: - OK, I pushed up a fresh branch using try-with-resources for both {{StreamReader}} and {{CompressedStreamReader}}. utests and dtests have been been kicked off, as well. > Properly close StreamCompressionInputStream to release any ByteBuf > -- > > Key: CASSANDRA-13906 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13906 > Project: Cassandra > Issue Type: Bug >Reporter: Jason Brown >Assignee: Jason Brown > > When running dtests for trunk (4.x) that perform some streaming, sometimes a > {{ByteBuf}} is not released properly, and we get this error in the logs > (causing the dtest to fail): > {code} > ERROR [MessagingService-NettyOutbound-Thread-4-2] 2017-09-26 13:42:37,940 > Slf4JLogger.java:176 - LEAK: ByteBuf.release() was not called before it's > garbage-collected. Enable advanced leak reporting to find out where the leak > occurred. To enable advanced leak reporting, specify the JVM option > '-Dio.netty.leakDetection.level=advanced' or call > ResourceLeakDetector.setLevel() See > http://netty.io/wiki/reference-counted-objects.html for more information. > {code} -- 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-13595) Implement short read protection on partition boundaries
[ https://issues.apache.org/jira/browse/CASSANDRA-13595?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186125#comment-16186125 ] Aleksey Yeschenko commented on CASSANDRA-13595: --- [~beobal] Sorry, it was indeed a little unclear. {{forShortReadRetry()}} was modified to retain the per partition limit, so asking for more rows then the limit no longer worked, and made the code a bit confusing. It also just happens that that arbitrary minimum was bothering me anyway. We should probably not go beyond the limit set by the user - unless we know for sure that it is safe. But until we are smarter about this, and start taking row sizes into account, I feel uneasy about enforcing arbitrary minimums. Pushed another commit on top that addresses two issues: 1. An NPE when trying to perform SRP on an empty iterator ({{lastPartitionKey}} would be null) 2. Lack of a reliable stop condition causing looping With the change all relevant dtests are passing locally. > Implement short read protection on partition boundaries > --- > > Key: CASSANDRA-13595 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13595 > Project: Cassandra > Issue Type: Bug > Components: Coordination >Reporter: Andrés de la Peña >Assignee: Aleksey Yeschenko > Labels: Correctness > Fix For: 3.0.15, 3.11.1 > > > It seems that short read protection doesn't work when the short read is done > at the end of a partition in a range query. The final assertion of this dtest > fails: > {code} > def short_read_partitions_delete_test(self): > cluster = self.cluster > cluster.set_configuration_options(values={'hinted_handoff_enabled': > False}) > cluster.set_batch_commitlog(enabled=True) > cluster.populate(2).start(wait_other_notice=True) > node1, node2 = self.cluster.nodelist() > session = self.patient_cql_connection(node1) > create_ks(session, 'ks', 2) > session.execute("CREATE TABLE t (k int, c int, PRIMARY KEY(k, c)) > WITH read_repair_chance = 0.0") > # we write 1 and 2 in a partition: all nodes get it. > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (1, 1)", > consistency_level=ConsistencyLevel.ALL)) > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (2, 1)", > consistency_level=ConsistencyLevel.ALL)) > # we delete partition 1: only node 1 gets it. > node2.flush() > node2.stop(wait_other_notice=True) > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 1")) > node2.start(wait_other_notice=True) > # we delete partition 2: only node 2 gets it. > node1.flush() > node1.stop(wait_other_notice=True) > session = self.patient_cql_connection(node2, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 2")) > node1.start(wait_other_notice=True) > # read from both nodes > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ALL) > assert_none(session, "SELECT * FROM t LIMIT 1") > {code} > However, the dtest passes if we remove the {{LIMIT 1}}. > Short read protection [uses a > {{SinglePartitionReadCommand}}|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/DataResolver.java#L484], > maybe it should use a {{PartitionRangeReadCommand}} instead? -- 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-13595) Implement short read protection on partition boundaries
[ https://issues.apache.org/jira/browse/CASSANDRA-13595?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-13595: -- Status: Patch Available (was: In Progress) > Implement short read protection on partition boundaries > --- > > Key: CASSANDRA-13595 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13595 > Project: Cassandra > Issue Type: Bug > Components: Coordination >Reporter: Andrés de la Peña >Assignee: Aleksey Yeschenko > Labels: Correctness > Fix For: 3.0.15, 3.11.1 > > > It seems that short read protection doesn't work when the short read is done > at the end of a partition in a range query. The final assertion of this dtest > fails: > {code} > def short_read_partitions_delete_test(self): > cluster = self.cluster > cluster.set_configuration_options(values={'hinted_handoff_enabled': > False}) > cluster.set_batch_commitlog(enabled=True) > cluster.populate(2).start(wait_other_notice=True) > node1, node2 = self.cluster.nodelist() > session = self.patient_cql_connection(node1) > create_ks(session, 'ks', 2) > session.execute("CREATE TABLE t (k int, c int, PRIMARY KEY(k, c)) > WITH read_repair_chance = 0.0") > # we write 1 and 2 in a partition: all nodes get it. > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (1, 1)", > consistency_level=ConsistencyLevel.ALL)) > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (2, 1)", > consistency_level=ConsistencyLevel.ALL)) > # we delete partition 1: only node 1 gets it. > node2.flush() > node2.stop(wait_other_notice=True) > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 1")) > node2.start(wait_other_notice=True) > # we delete partition 2: only node 2 gets it. > node1.flush() > node1.stop(wait_other_notice=True) > session = self.patient_cql_connection(node2, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 2")) > node1.start(wait_other_notice=True) > # read from both nodes > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ALL) > assert_none(session, "SELECT * FROM t LIMIT 1") > {code} > However, the dtest passes if we remove the {{LIMIT 1}}. > Short read protection [uses a > {{SinglePartitionReadCommand}}|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/DataResolver.java#L484], > maybe it should use a {{PartitionRangeReadCommand}} instead? -- 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-11511) disable Anticompaction after repair
[ https://issues.apache.org/jira/browse/CASSANDRA-11511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186092#comment-16186092 ] Pankaj Yadav commented on CASSANDRA-11511: -- We are also observing the same thing, more than half of the time of repair is eaten by anticompaction/cleanup activities. Since we only want to perform full repair (using nodetool -pr -seq -full), please provide us a way to disable anticompaction after repair. > disable Anticompaction after repair > --- > > Key: CASSANDRA-11511 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11511 > Project: Cassandra > Issue Type: Improvement >Reporter: Jack Money >Priority: Minor > > I use write consistency level ALL. > Sometimes when my data grows to fast i cant normal add new node, > [CASSANDRA-10797 issue > occurs|https://issues.apache.org/jira/browse/CASSANDRA-10797]. > After this i set auto_bootstrap to false, and run nodetool repair - full on > all tables one by one. > 70% percent time of repair is taken by "Anticompaction after repair". > But i don't need it, because i do not need to run repair (consistency level > ALL.). > To avoid anticompaction i run repair with token range [Avoid anticompaction > when doing subrange > repair|https://issues.apache.org/jira/browse/CASSANDRA-10422] but i think it > should be options to avoid anticompaction, i.e. nodetool repair -full > *-disableanticompaction* -- 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-13906) Properly close StreamCompressionInputStream to release any ByteBuf
[ https://issues.apache.org/jira/browse/CASSANDRA-13906?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186087#comment-16186087 ] Ariel Weisberg edited comment on CASSANDRA-13906 at 9/29/17 5:05 PM: - If you implement Closable it's going to cause warnings any place that doesn't use try with resources. Given the other usages I think what you have done is probably fine although you can still switch to try with resources just for {{StreamCompressionInputStream}}. was (Author: aweisberg): If you implement Closable it's going to cause warnings any place that doesn't use try with resources. Given the other usages I think what you have done is probably fine although you can still switch to try with resources. > Properly close StreamCompressionInputStream to release any ByteBuf > -- > > Key: CASSANDRA-13906 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13906 > Project: Cassandra > Issue Type: Bug >Reporter: Jason Brown >Assignee: Jason Brown > > When running dtests for trunk (4.x) that perform some streaming, sometimes a > {{ByteBuf}} is not released properly, and we get this error in the logs > (causing the dtest to fail): > {code} > ERROR [MessagingService-NettyOutbound-Thread-4-2] 2017-09-26 13:42:37,940 > Slf4JLogger.java:176 - LEAK: ByteBuf.release() was not called before it's > garbage-collected. Enable advanced leak reporting to find out where the leak > occurred. To enable advanced leak reporting, specify the JVM option > '-Dio.netty.leakDetection.level=advanced' or call > ResourceLeakDetector.setLevel() See > http://netty.io/wiki/reference-counted-objects.html for more information. > {code} -- 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-13906) Properly close StreamCompressionInputStream to release any ByteBuf
[ https://issues.apache.org/jira/browse/CASSANDRA-13906?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186087#comment-16186087 ] Ariel Weisberg commented on CASSANDRA-13906: If you implement Closable it's going to cause warnings any place that doesn't use try with resources. Given the other usages I think what you have done is probably fine although you can still switch to try with resources. > Properly close StreamCompressionInputStream to release any ByteBuf > -- > > Key: CASSANDRA-13906 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13906 > Project: Cassandra > Issue Type: Bug >Reporter: Jason Brown >Assignee: Jason Brown > > When running dtests for trunk (4.x) that perform some streaming, sometimes a > {{ByteBuf}} is not released properly, and we get this error in the logs > (causing the dtest to fail): > {code} > ERROR [MessagingService-NettyOutbound-Thread-4-2] 2017-09-26 13:42:37,940 > Slf4JLogger.java:176 - LEAK: ByteBuf.release() was not called before it's > garbage-collected. Enable advanced leak reporting to find out where the leak > occurred. To enable advanced leak reporting, specify the JVM option > '-Dio.netty.leakDetection.level=advanced' or call > ResourceLeakDetector.setLevel() See > http://netty.io/wiki/reference-counted-objects.html for more information. > {code} -- 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-13906) Properly close StreamCompressionInputStream to release any ByteBuf
[ https://issues.apache.org/jira/browse/CASSANDRA-13906?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186066#comment-16186066 ] Jason Brown commented on CASSANDRA-13906: - bq. Wouldn't the most idiomatic way to do this be to have {{TrackedDataInputPlus}} implement {{Closable}} and use try with resources I thought about that when I was fixing this, but {{TrackedDataInputPlus}} wraps a {{DataInput}}, which does not declare a {{#close()}} method. In a {{TrackedDataInputPlus#close()}} I could check if the wrapped instance implements {{Closeable}} and invoke it if it does. wdyt? There are a couple of other uses of {{TrackedDataInputPlus}} ({{HintMessage}}, {{IndexedEntry}} called on load saved cache path), but they should not be affected by {{TrackedDataInputPlus}} implementing {{Closeable}} as the are not allocated via try-with-resources. Note: if we choose to make this change, which is reasonable, I can also cleanup {{CompressedStreamReader}} to also allocate {{TrackedDataInputPlus}} in a try-with-resources - it has the same concerns as {{StreamReader}} that you raised. Branch coming shortly ... > Properly close StreamCompressionInputStream to release any ByteBuf > -- > > Key: CASSANDRA-13906 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13906 > Project: Cassandra > Issue Type: Bug >Reporter: Jason Brown >Assignee: Jason Brown > > When running dtests for trunk (4.x) that perform some streaming, sometimes a > {{ByteBuf}} is not released properly, and we get this error in the logs > (causing the dtest to fail): > {code} > ERROR [MessagingService-NettyOutbound-Thread-4-2] 2017-09-26 13:42:37,940 > Slf4JLogger.java:176 - LEAK: ByteBuf.release() was not called before it's > garbage-collected. Enable advanced leak reporting to find out where the leak > occurred. To enable advanced leak reporting, specify the JVM option > '-Dio.netty.leakDetection.level=advanced' or call > ResourceLeakDetector.setLevel() See > http://netty.io/wiki/reference-counted-objects.html for more information. > {code} -- 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-13920) Adding regular column to COMPACT STORAGE with w/o clustering keys table causes an exception
[ https://issues.apache.org/jira/browse/CASSANDRA-13920?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Petrov updated CASSANDRA-13920: Description: When trying to {{ALTER}} a {{COMPACT TABLE}} without clustering keys (i.e. non-dense one), you'll get an exception. Adding regular columns to non-dense compact tables should be forbidden (adding static ones already fails with {{Static columns are not allowed in COMPACT STORAGE tables}}), just as adding columns to dense compact tables which throws {{Cannot add new column to a COMPACT STORAGE table}} (or the error message should be adjusted). dtest to reproduce: {code} from cql_tests import CQLTester class StorageProxyCQLTester(CQLTester): def test_sparse_compact(self): session = self.prepare(nodes=2, rf=2) session.execute("CREATE TABLE sparse_compact_table (k int PRIMARY KEY, v1 int, v2 int) WITH COMPACT STORAGE;") session.execute("ALTER TABLE sparse_compact_table ADD wat int",) {code} Exception: {code} java.lang.AssertionError: null at org.apache.cassandra.db.CompactTables.getCompactValueColumn(CompactTables.java:67) ~[main/:na] at org.apache.cassandra.config.CFMetaData.rebuild(CFMetaData.java:337) ~[main/:na] at org.apache.cassandra.config.CFMetaData.validate(CFMetaData.java:935) ~[main/:na] at org.apache.cassandra.service.MigrationManager.announceColumnFamilyUpdate(MigrationManager.java:421) ~[main/:na] at org.apache.cassandra.cql3.statements.AlterTableStatement.announceMigration(AlterTableStatement.java:288) ~[main/:na] at org.apache.cassandra.cql3.statements.SchemaAlteringStatement.execute(SchemaAlteringStatement.java:93) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:237) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:222) ~[main/:na] at org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:115) ~[main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:513) [main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:407) [main/:na] at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) [netty-all-4.0.44.Final.jar:4.0.44.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:357) [netty-all-4.0.44.Final.jar:4.0.44.Final] at io.netty.channel.AbstractChannelHandlerContext.access$600(AbstractChannelHandlerContext.java:35) [netty-all-4.0.44.Final.jar:4.0.44.Final] at io.netty.channel.AbstractChannelHandlerContext$7.run(AbstractChannelHandlerContext.java:348) [netty-all-4.0.44.Final.jar:4.0.44.Final] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_121] at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) [main/:na] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [main/:na] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_121] {code} was: When trying to {{ALTER}} a {{COMPACT TABLE}} without clustering keys (i.e. non-dense one), you'll get an exception. Adding regular columns to non-dense compact tables should be forbidden (adding static ones works just fine), just as altering dense columns which throws {{Cannot add new column to a COMPACT STORAGE table}} (or the error message should be adjusted). dtest to reproduce: {code} from cql_tests import CQLTester class StorageProxyCQLTester(CQLTester): def test_sparse_compact(self): session = self.prepare(nodes=2, rf=2) session.execute("CREATE TABLE sparse_compact_table (k int PRIMARY KEY, v1 int, v2 int) WITH COMPACT STORAGE;") session.execute("ALTER TABLE sparse_compact_table ADD wat int",) {code} Exception: {code} java.lang.AssertionError: null at org.apache.cassandra.db.CompactTables.getCompactValueColumn(CompactTables.java:67) ~[main/:na] at org.apache.cassandra.config.CFMetaData.rebuild(CFMetaData.java:337) ~[main/:na] at org.apache.cassandra.config.CFMetaData.validate(CFMetaData.java:935) ~[main/:na] at org.apache.cassandra.service.MigrationManager.announceColumnFamilyUpdate(MigrationManager.java:421) ~[main/:na] at org.apache.cassandra.cql3.statements.AlterTableStatement.announceMigration(AlterTableStatement.java:288) ~[main/:na] at org.apache.cassandra.cql3.statements.SchemaAlteringStatement.execute(SchemaAlteringStatement.java:93) ~[main/:na] at
[jira] [Updated] (CASSANDRA-13920) Adding regular column to COMPACT STORAGE with w/o clustering keys table causes an exception
[ https://issues.apache.org/jira/browse/CASSANDRA-13920?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Petrov updated CASSANDRA-13920: Description: When trying to {{ALTER}} a {{COMPACT TABLE}} without clustering keys (i.e. non-dense one), you'll get an exception. Adding regular columns to non-dense compact tables should be forbidden (adding static ones works just fine), just as altering dense columns which throws {{Cannot add new column to a COMPACT STORAGE table}} (or the error message should be adjusted). dtest to reproduce: {code} from cql_tests import CQLTester class StorageProxyCQLTester(CQLTester): def test_sparse_compact(self): session = self.prepare(nodes=2, rf=2) session.execute("CREATE TABLE sparse_compact_table (k int PRIMARY KEY, v1 int, v2 int) WITH COMPACT STORAGE;") session.execute("ALTER TABLE sparse_compact_table ADD wat int",) {code} Exception: {code} java.lang.AssertionError: null at org.apache.cassandra.db.CompactTables.getCompactValueColumn(CompactTables.java:67) ~[main/:na] at org.apache.cassandra.config.CFMetaData.rebuild(CFMetaData.java:337) ~[main/:na] at org.apache.cassandra.config.CFMetaData.validate(CFMetaData.java:935) ~[main/:na] at org.apache.cassandra.service.MigrationManager.announceColumnFamilyUpdate(MigrationManager.java:421) ~[main/:na] at org.apache.cassandra.cql3.statements.AlterTableStatement.announceMigration(AlterTableStatement.java:288) ~[main/:na] at org.apache.cassandra.cql3.statements.SchemaAlteringStatement.execute(SchemaAlteringStatement.java:93) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:237) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:222) ~[main/:na] at org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:115) ~[main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:513) [main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:407) [main/:na] at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) [netty-all-4.0.44.Final.jar:4.0.44.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:357) [netty-all-4.0.44.Final.jar:4.0.44.Final] at io.netty.channel.AbstractChannelHandlerContext.access$600(AbstractChannelHandlerContext.java:35) [netty-all-4.0.44.Final.jar:4.0.44.Final] at io.netty.channel.AbstractChannelHandlerContext$7.run(AbstractChannelHandlerContext.java:348) [netty-all-4.0.44.Final.jar:4.0.44.Final] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_121] at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) [main/:na] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [main/:na] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_121] {code} was: When trying to {{ALTER}} a {{COMPACT TABLE}} without clustering keys (i.e. non-dense one), you'll get an exception. Alter'ing non-dense compact tables should be forbidden, just as altering dense columns which throws {{Cannot add new column to a COMPACT STORAGE table}} (or the error message should be adjusted). dtest to reproduce: {code} from cql_tests import CQLTester class StorageProxyCQLTester(CQLTester): def test_sparse_compact(self): session = self.prepare(nodes=2, rf=2) session.execute("CREATE TABLE sparse_compact_table (k int PRIMARY KEY, v1 int, v2 int) WITH COMPACT STORAGE;") session.execute("ALTER TABLE sparse_compact_table ADD wat int",) {code} Exception: {code} java.lang.AssertionError: null at org.apache.cassandra.db.CompactTables.getCompactValueColumn(CompactTables.java:67) ~[main/:na] at org.apache.cassandra.config.CFMetaData.rebuild(CFMetaData.java:337) ~[main/:na] at org.apache.cassandra.config.CFMetaData.validate(CFMetaData.java:935) ~[main/:na] at org.apache.cassandra.service.MigrationManager.announceColumnFamilyUpdate(MigrationManager.java:421) ~[main/:na] at org.apache.cassandra.cql3.statements.AlterTableStatement.announceMigration(AlterTableStatement.java:288) ~[main/:na] at org.apache.cassandra.cql3.statements.SchemaAlteringStatement.execute(SchemaAlteringStatement.java:93) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:237) ~[main/:na] at
[jira] [Updated] (CASSANDRA-13920) Adding regular column to COMPACT STORAGE w/o clustering keys table causes an exception
[ https://issues.apache.org/jira/browse/CASSANDRA-13920?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Petrov updated CASSANDRA-13920: Summary: Adding regular column to COMPACT STORAGE w/o clustering keys table causes an exception (was: Altering COMPACT STORAGE w/o clustering keys table causes an exception) > Adding regular column to COMPACT STORAGE w/o clustering keys table causes an > exception > -- > > Key: CASSANDRA-13920 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13920 > Project: Cassandra > Issue Type: Bug >Reporter: Alex Petrov >Priority: Minor > > When trying to {{ALTER}} a {{COMPACT TABLE}} without clustering keys (i.e. > non-dense one), you'll get an exception. > Alter'ing non-dense compact tables should be forbidden, just as altering > dense columns which throws {{Cannot add new column to a COMPACT STORAGE > table}} (or the error message should be adjusted). > dtest to reproduce: > {code} > from cql_tests import CQLTester > class StorageProxyCQLTester(CQLTester): > def test_sparse_compact(self): > session = self.prepare(nodes=2, rf=2) > session.execute("CREATE TABLE sparse_compact_table (k int PRIMARY > KEY, v1 int, v2 int) WITH COMPACT STORAGE;") > session.execute("ALTER TABLE sparse_compact_table ADD wat int",) > {code} > Exception: > {code} > java.lang.AssertionError: null > at > org.apache.cassandra.db.CompactTables.getCompactValueColumn(CompactTables.java:67) > ~[main/:na] > at > org.apache.cassandra.config.CFMetaData.rebuild(CFMetaData.java:337) > ~[main/:na] > at > org.apache.cassandra.config.CFMetaData.validate(CFMetaData.java:935) > ~[main/:na] > at > org.apache.cassandra.service.MigrationManager.announceColumnFamilyUpdate(MigrationManager.java:421) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.AlterTableStatement.announceMigration(AlterTableStatement.java:288) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.SchemaAlteringStatement.execute(SchemaAlteringStatement.java:93) > ~[main/:na] > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206) > ~[main/:na] > at > org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:237) > ~[main/:na] > at > org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:222) > ~[main/:na] > at > org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:115) > ~[main/:na] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:513) > [main/:na] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:407) > [main/:na] > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:357) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > io.netty.channel.AbstractChannelHandlerContext.access$600(AbstractChannelHandlerContext.java:35) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > io.netty.channel.AbstractChannelHandlerContext$7.run(AbstractChannelHandlerContext.java:348) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > [na:1.8.0_121] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) > [main/:na] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) > [main/:na] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_121] > {code} -- 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-13920) Adding regular column to COMPACT STORAGE with w/o clustering keys table causes an exception
[ https://issues.apache.org/jira/browse/CASSANDRA-13920?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Petrov updated CASSANDRA-13920: Summary: Adding regular column to COMPACT STORAGE with w/o clustering keys table causes an exception (was: Adding regular column to COMPACT STORAGE w/o clustering keys table causes an exception) > Adding regular column to COMPACT STORAGE with w/o clustering keys table > causes an exception > --- > > Key: CASSANDRA-13920 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13920 > Project: Cassandra > Issue Type: Bug >Reporter: Alex Petrov >Priority: Minor > > When trying to {{ALTER}} a {{COMPACT TABLE}} without clustering keys (i.e. > non-dense one), you'll get an exception. > Alter'ing non-dense compact tables should be forbidden, just as altering > dense columns which throws {{Cannot add new column to a COMPACT STORAGE > table}} (or the error message should be adjusted). > dtest to reproduce: > {code} > from cql_tests import CQLTester > class StorageProxyCQLTester(CQLTester): > def test_sparse_compact(self): > session = self.prepare(nodes=2, rf=2) > session.execute("CREATE TABLE sparse_compact_table (k int PRIMARY > KEY, v1 int, v2 int) WITH COMPACT STORAGE;") > session.execute("ALTER TABLE sparse_compact_table ADD wat int",) > {code} > Exception: > {code} > java.lang.AssertionError: null > at > org.apache.cassandra.db.CompactTables.getCompactValueColumn(CompactTables.java:67) > ~[main/:na] > at > org.apache.cassandra.config.CFMetaData.rebuild(CFMetaData.java:337) > ~[main/:na] > at > org.apache.cassandra.config.CFMetaData.validate(CFMetaData.java:935) > ~[main/:na] > at > org.apache.cassandra.service.MigrationManager.announceColumnFamilyUpdate(MigrationManager.java:421) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.AlterTableStatement.announceMigration(AlterTableStatement.java:288) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.SchemaAlteringStatement.execute(SchemaAlteringStatement.java:93) > ~[main/:na] > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206) > ~[main/:na] > at > org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:237) > ~[main/:na] > at > org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:222) > ~[main/:na] > at > org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:115) > ~[main/:na] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:513) > [main/:na] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:407) > [main/:na] > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:357) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > io.netty.channel.AbstractChannelHandlerContext.access$600(AbstractChannelHandlerContext.java:35) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > io.netty.channel.AbstractChannelHandlerContext$7.run(AbstractChannelHandlerContext.java:348) > [netty-all-4.0.44.Final.jar:4.0.44.Final] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > [na:1.8.0_121] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) > [main/:na] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) > [main/:na] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_121] > {code} -- 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-13905) Correctly close netty channels when a stream session ends
[ https://issues.apache.org/jira/browse/CASSANDRA-13905?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Brown updated CASSANDRA-13905: Resolution: Fixed Fix Version/s: (was: 4.x) 4.0 Status: Resolved (was: Ready to Commit) bq. Collectors are the don't be lazy step in streams Yeah that's where I messed up originally :). committed as sha {[ebefc96a8fe63aca5f324984f7f3147f10218643}}. > Correctly close netty channels when a stream session ends > - > > Key: CASSANDRA-13905 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13905 > Project: Cassandra > Issue Type: Bug > Components: Streaming and Messaging >Reporter: Jason Brown >Assignee: Jason Brown > Fix For: 4.0 > > > Netty channels in stream sessions were not being closed correctly. TL;DR I > was using a lambda that was not executing as it is lazily evaluated. This was > causing a {{RejectedExecutionException}} at the end of some streaming-related > dtests -- 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: Correctly close netty channels when a stream session ends
Repository: cassandra Updated Branches: refs/heads/trunk e296ff063 -> ebefc96a8 Correctly close netty channels when a stream session ends patch by jasobrown; reviewed by Ariel Weisberg for CASSANDRA-13905 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ebefc96a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ebefc96a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ebefc96a Branch: refs/heads/trunk Commit: ebefc96a8fe63aca5f324984f7f3147f10218643 Parents: e296ff0 Author: Jason BrownAuthored: Mon Sep 25 15:39:17 2017 -0700 Committer: Jason Brown Committed: Fri Sep 29 09:33:08 2017 -0700 -- CHANGES.txt | 1 + .../async/NettyStreamingMessageSender.java | 24 .../org/apache/cassandra/utils/FBUtilities.java | 17 +- 3 files changed, 31 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ebefc96a/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index ea73fcd..99b5a59 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Correctly close netty channels when a stream session ends (CASSANDRA-13905) * Update lz4 to 1.4.0 (CASSANDRA-13741) * Optimize Paxos prepare and propose stage for local requests (CASSANDRA-13862) * Throttle base partitions during MV repair streaming to prevent OOM (CASSANDRA-13299) http://git-wip-us.apache.org/repos/asf/cassandra/blob/ebefc96a/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java -- diff --git a/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java b/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java index f872005..0b38760 100644 --- a/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java +++ b/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java @@ -21,7 +21,9 @@ package org.apache.cassandra.streaming.async; import java.io.IOError; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Collection; +import java.util.List; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.LinkedBlockingQueue; @@ -106,10 +108,9 @@ public class NettyStreamingMessageSender implements StreamingMessageSender private final ThreadPoolExecutor fileTransferExecutor; /** - * A {@link ThreadLocal} used by the threads in {@link #fileTransferExecutor} to stash references to constructed - * and connected {@link Channel}s. + * A mapping of each {@link #fileTransferExecutor} thread to a channel that can be written to (on that thread). */ -private final ConcurrentMap threadLocalChannel = new ConcurrentHashMap<>(); +private final ConcurrentMap threadToChannelMap = new ConcurrentHashMap<>(); /** * A netty channel attribute used to indicate if a channel is currently transferring a file. This is primarily used @@ -373,12 +374,12 @@ public class NettyStreamingMessageSender implements StreamingMessageSender Thread currentThread = Thread.currentThread(); try { -Channel channel = threadLocalChannel.get(currentThread); +Channel channel = threadToChannelMap.get(currentThread); if (channel != null) return channel; channel = createChannel(); -threadLocalChannel.put(currentThread, channel); +threadToChannelMap.put(currentThread, channel); return channel; } catch (Exception e) @@ -393,10 +394,10 @@ public class NettyStreamingMessageSender implements StreamingMessageSender void injectChannel(Channel channel) { Thread currentThread = Thread.currentThread(); -if (threadLocalChannel.get(currentThread) != null) +if (threadToChannelMap.get(currentThread) != null) throw new IllegalStateException("previous channel already set"); -threadLocalChannel.put(currentThread, channel); +threadToChannelMap.put(currentThread, channel); } /** @@ -404,7 +405,7 @@ public class NettyStreamingMessageSender implements StreamingMessageSender */ void unsetChannel() { -threadLocalChannel.remove(Thread.currentThread()); +threadToChannelMap.remove(Thread.currentThread()); }
[jira] [Commented] (CASSANDRA-13906) Properly close StreamCompressionInputStream to release any ByteBuf
[ https://issues.apache.org/jira/browse/CASSANDRA-13906?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186051#comment-16186051 ] Ariel Weisberg commented on CASSANDRA-13906: So to bikeshed this all to hell. Wouldn't the most idiomatic way to do this be to have TrackedDataInputPlus implement Closable and use try with resources? In fact try with resources will let you declare all the Closeable things in the same try block and clean up if one of the things you are using to wrap throws in it's constructor preventing the wrapped resource from leaking. > Properly close StreamCompressionInputStream to release any ByteBuf > -- > > Key: CASSANDRA-13906 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13906 > Project: Cassandra > Issue Type: Bug >Reporter: Jason Brown >Assignee: Jason Brown > > When running dtests for trunk (4.x) that perform some streaming, sometimes a > {{ByteBuf}} is not released properly, and we get this error in the logs > (causing the dtest to fail): > {code} > ERROR [MessagingService-NettyOutbound-Thread-4-2] 2017-09-26 13:42:37,940 > Slf4JLogger.java:176 - LEAK: ByteBuf.release() was not called before it's > garbage-collected. Enable advanced leak reporting to find out where the leak > occurred. To enable advanced leak reporting, specify the JVM option > '-Dio.netty.leakDetection.level=advanced' or call > ResourceLeakDetector.setLevel() See > http://netty.io/wiki/reference-counted-objects.html for more information. > {code} -- 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-13920) Altering COMPACT STORAGE w/o clustering keys table causes an exception
Alex Petrov created CASSANDRA-13920: --- Summary: Altering COMPACT STORAGE w/o clustering keys table causes an exception Key: CASSANDRA-13920 URL: https://issues.apache.org/jira/browse/CASSANDRA-13920 Project: Cassandra Issue Type: Bug Reporter: Alex Petrov Priority: Minor When trying to {{ALTER}} a {{COMPACT TABLE}} without clustering keys (i.e. non-dense one), you'll get an exception. Alter'ing non-dense compact tables should be forbidden, just as altering dense columns which throws {{Cannot add new column to a COMPACT STORAGE table}} (or the error message should be adjusted). dtest to reproduce: {code} from cql_tests import CQLTester class StorageProxyCQLTester(CQLTester): def test_sparse_compact(self): session = self.prepare(nodes=2, rf=2) session.execute("CREATE TABLE sparse_compact_table (k int PRIMARY KEY, v1 int, v2 int) WITH COMPACT STORAGE;") session.execute("ALTER TABLE sparse_compact_table ADD wat int",) {code} Exception: {code} java.lang.AssertionError: null at org.apache.cassandra.db.CompactTables.getCompactValueColumn(CompactTables.java:67) ~[main/:na] at org.apache.cassandra.config.CFMetaData.rebuild(CFMetaData.java:337) ~[main/:na] at org.apache.cassandra.config.CFMetaData.validate(CFMetaData.java:935) ~[main/:na] at org.apache.cassandra.service.MigrationManager.announceColumnFamilyUpdate(MigrationManager.java:421) ~[main/:na] at org.apache.cassandra.cql3.statements.AlterTableStatement.announceMigration(AlterTableStatement.java:288) ~[main/:na] at org.apache.cassandra.cql3.statements.SchemaAlteringStatement.execute(SchemaAlteringStatement.java:93) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:237) ~[main/:na] at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:222) ~[main/:na] at org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:115) ~[main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:513) [main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:407) [main/:na] at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) [netty-all-4.0.44.Final.jar:4.0.44.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:357) [netty-all-4.0.44.Final.jar:4.0.44.Final] at io.netty.channel.AbstractChannelHandlerContext.access$600(AbstractChannelHandlerContext.java:35) [netty-all-4.0.44.Final.jar:4.0.44.Final] at io.netty.channel.AbstractChannelHandlerContext$7.run(AbstractChannelHandlerContext.java:348) [netty-all-4.0.44.Final.jar:4.0.44.Final] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_121] at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) [main/:na] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [main/:na] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_121] {code} -- 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-13905) Correctly close netty channels when a stream session ends
[ https://issues.apache.org/jira/browse/CASSANDRA-13905?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg updated CASSANDRA-13905: --- Status: Ready to Commit (was: Patch Available) > Correctly close netty channels when a stream session ends > - > > Key: CASSANDRA-13905 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13905 > Project: Cassandra > Issue Type: Bug > Components: Streaming and Messaging >Reporter: Jason Brown >Assignee: Jason Brown > Fix For: 4.x > > > Netty channels in stream sessions were not being closed correctly. TL;DR I > was using a lambda that was not executing as it is lazily evaluated. This was > causing a {{RejectedExecutionException}} at the end of some streaming-related > dtests -- 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-13905) Correctly close netty channels when a stream session ends
[ https://issues.apache.org/jira/browse/CASSANDRA-13905?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16186026#comment-16186026 ] Ariel Weisberg commented on CASSANDRA-13905: +1 to the fix. If you wanted to stick with the Streamisms you could do Collectors.toList() after the map and then wait on futures for that. Collectors are the don't be lazy step in streams. Although if you aren't being lazy I'm not sure the syntax for streams is really all that much clearer. > Correctly close netty channels when a stream session ends > - > > Key: CASSANDRA-13905 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13905 > Project: Cassandra > Issue Type: Bug > Components: Streaming and Messaging >Reporter: Jason Brown >Assignee: Jason Brown > Fix For: 4.x > > > Netty channels in stream sessions were not being closed correctly. TL;DR I > was using a lambda that was not executing as it is lazily evaluated. This was > causing a {{RejectedExecutionException}} at the end of some streaming-related > dtests -- 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-13906) Properly close StreamCompressionInputStream to release any ByteBuf
[ https://issues.apache.org/jira/browse/CASSANDRA-13906?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg updated CASSANDRA-13906: --- Reviewer: Ariel Weisberg > Properly close StreamCompressionInputStream to release any ByteBuf > -- > > Key: CASSANDRA-13906 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13906 > Project: Cassandra > Issue Type: Bug >Reporter: Jason Brown >Assignee: Jason Brown > > When running dtests for trunk (4.x) that perform some streaming, sometimes a > {{ByteBuf}} is not released properly, and we get this error in the logs > (causing the dtest to fail): > {code} > ERROR [MessagingService-NettyOutbound-Thread-4-2] 2017-09-26 13:42:37,940 > Slf4JLogger.java:176 - LEAK: ByteBuf.release() was not called before it's > garbage-collected. Enable advanced leak reporting to find out where the leak > occurred. To enable advanced leak reporting, specify the JVM option > '-Dio.netty.leakDetection.level=advanced' or call > ResourceLeakDetector.setLevel() See > http://netty.io/wiki/reference-counted-objects.html for more information. > {code} -- 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-13905) Correctly close netty channels when a stream session ends
[ https://issues.apache.org/jira/browse/CASSANDRA-13905?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ariel Weisberg updated CASSANDRA-13905: --- Reviewer: Ariel Weisberg > Correctly close netty channels when a stream session ends > - > > Key: CASSANDRA-13905 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13905 > Project: Cassandra > Issue Type: Bug > Components: Streaming and Messaging >Reporter: Jason Brown >Assignee: Jason Brown > Fix For: 4.x > > > Netty channels in stream sessions were not being closed correctly. TL;DR I > was using a lambda that was not executing as it is lazily evaluated. This was > causing a {{RejectedExecutionException}} at the end of some streaming-related > dtests -- 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-13910) Consider deprecating (then removing) read_repair_chance/dclocal_read_repair_chance
[ https://issues.apache.org/jira/browse/CASSANDRA-13910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185992#comment-16185992 ] Aleksey Yeschenko commented on CASSANDRA-13910: --- bq. If someone had a multi-dc setup, wrote with LOCAL_ cls, and then read from that DC, they'd trigger the async repair into the other DC, working around the dc local consistency without paying the latency cost on the app side. Sort of a "I can force hint delivery on read without waiting for it" technique. I don't know how common such a use case would be, but I would believe that someone, somewhere is relying on it. Right. But so long as they have hints enabled, then even with LOCAL_* cls they'll get the same effect, but more reliably, on write. No waiting for it. > Consider deprecating (then removing) > read_repair_chance/dclocal_read_repair_chance > -- > > Key: CASSANDRA-13910 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13910 > Project: Cassandra > Issue Type: Improvement >Reporter: Sylvain Lebresne >Priority: Minor > Labels: CommunityFeedbackRequested > > First, let me clarify so this is not misunderstood that I'm not *at all* > suggesting to remove the read-repair mechanism of detecting and repairing > inconsistencies between read responses: that mechanism is imo fine and > useful. But the {{read_repair_chance}} and {{dclocal_read_repair_chance}} > have never been about _enabling_ that mechanism, they are about querying all > replicas (even when this is not required by the consistency level) for the > sole purpose of maybe read-repairing some of the replica that wouldn't have > been queried otherwise. Which btw, bring me to reason 1 for considering their > removal: their naming/behavior is super confusing. Over the years, I've seen > countless users (and not only newbies) misunderstanding what those options > do, and as a consequence misunderstand when read-repair itself was happening. > But my 2nd reason for suggesting this is that I suspect > {{read_repair_chance}}/{{dclocal_read_repair_chance}} are, especially > nowadays, more harmful than anything else when enabled. When those option > kick in, what you trade-off is additional resources consumption (all nodes > have to execute the read) for a _fairly remote chance_ of having some > inconsistencies repaired on _some_ replica _a bit faster_ than they would > otherwise be. To justify that last part, let's recall that: > # most inconsistencies are actually fixed by hints in practice; and in the > case where a node stay dead for a long time so that hints ends up timing-out, > you really should repair the node when it comes back (if not simply > re-bootstrapping it). Read-repair probably don't fix _that_ much stuff in > the first place. > # again, read-repair do happen without those options kicking in. If you do > reads at {{QUORUM}}, inconsistencies will eventually get read-repaired all > the same. Just a tiny bit less quickly. > # I suspect almost everyone use a low "chance" for those options at best > (because the extra resources consumption is real), so at the end of the day, > it's up to chance how much faster this fixes inconsistencies. > Overall, I'm having a hard time imagining real cases where that trade-off > really make sense. Don't get me wrong, those options had their places a long > time ago when hints weren't working all that well, but I think they bring > more confusion than benefits now. > And I think it's sane to reconsider stuffs every once in a while, and to > clean up anything that may not make all that much sense anymore, which I > think is the case here. > Tl;dr, I feel the benefits brought by those options are very slim at best and > well overshadowed by the confusion they bring, and not worth maintaining the > code that supports them (which, to be fair, isn't huge, but getting rid of > {{ReadCallback.AsyncRepairRunner}} wouldn't hurt for instance). > Lastly, if the consensus here ends up being that they can have their use in > weird case and that we fill supporting those cases is worth confusing > everyone else and maintaining that code, I would still suggest disabling them > totally by default. -- 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-13910) Consider deprecating (then removing) read_repair_chance/dclocal_read_repair_chance
[ https://issues.apache.org/jira/browse/CASSANDRA-13910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185983#comment-16185983 ] Jeff Jirsa commented on CASSANDRA-13910: Been thinking about this a bit more. {quote} I'm having a hard time imagining real cases where that trade-off really make sense. {quote} I think the real cases where the trade off make sense are where your earlier assumption is wrong : {quote} I suspect almost everyone use a low "chance" for those options at best (because the extra resources consumption is real), so at the end of the day, it's up to chance how much faster this fixes inconsistencies. {quote} Asked differently: why would someone set {{read_repair_chance}} to a high value? If someone had a multi-dc setup, wrote with {{LOCAL_}} cls, and then read from that DC, they'd trigger the async repair into the other DC, working around the dc local consistency without paying the latency cost on the app side. Sort of a "I can force hint delivery on read without waiting for it" technique. I don't know how common such a use case would be, but I would believe that someone, somewhere is relying on it. > Consider deprecating (then removing) > read_repair_chance/dclocal_read_repair_chance > -- > > Key: CASSANDRA-13910 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13910 > Project: Cassandra > Issue Type: Improvement >Reporter: Sylvain Lebresne >Priority: Minor > Labels: CommunityFeedbackRequested > > First, let me clarify so this is not misunderstood that I'm not *at all* > suggesting to remove the read-repair mechanism of detecting and repairing > inconsistencies between read responses: that mechanism is imo fine and > useful. But the {{read_repair_chance}} and {{dclocal_read_repair_chance}} > have never been about _enabling_ that mechanism, they are about querying all > replicas (even when this is not required by the consistency level) for the > sole purpose of maybe read-repairing some of the replica that wouldn't have > been queried otherwise. Which btw, bring me to reason 1 for considering their > removal: their naming/behavior is super confusing. Over the years, I've seen > countless users (and not only newbies) misunderstanding what those options > do, and as a consequence misunderstand when read-repair itself was happening. > But my 2nd reason for suggesting this is that I suspect > {{read_repair_chance}}/{{dclocal_read_repair_chance}} are, especially > nowadays, more harmful than anything else when enabled. When those option > kick in, what you trade-off is additional resources consumption (all nodes > have to execute the read) for a _fairly remote chance_ of having some > inconsistencies repaired on _some_ replica _a bit faster_ than they would > otherwise be. To justify that last part, let's recall that: > # most inconsistencies are actually fixed by hints in practice; and in the > case where a node stay dead for a long time so that hints ends up timing-out, > you really should repair the node when it comes back (if not simply > re-bootstrapping it). Read-repair probably don't fix _that_ much stuff in > the first place. > # again, read-repair do happen without those options kicking in. If you do > reads at {{QUORUM}}, inconsistencies will eventually get read-repaired all > the same. Just a tiny bit less quickly. > # I suspect almost everyone use a low "chance" for those options at best > (because the extra resources consumption is real), so at the end of the day, > it's up to chance how much faster this fixes inconsistencies. > Overall, I'm having a hard time imagining real cases where that trade-off > really make sense. Don't get me wrong, those options had their places a long > time ago when hints weren't working all that well, but I think they bring > more confusion than benefits now. > And I think it's sane to reconsider stuffs every once in a while, and to > clean up anything that may not make all that much sense anymore, which I > think is the case here. > Tl;dr, I feel the benefits brought by those options are very slim at best and > well overshadowed by the confusion they bring, and not worth maintaining the > code that supports them (which, to be fair, isn't huge, but getting rid of > {{ReadCallback.AsyncRepairRunner}} wouldn't hurt for instance). > Lastly, if the consensus here ends up being that they can have their use in > weird case and that we fill supporting those cases is worth confusing > everyone else and maintaining that code, I would still suggest disabling them > totally by default. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail:
[jira] [Updated] (CASSANDRA-13595) Implement short read protection on partition boundaries
[ https://issues.apache.org/jira/browse/CASSANDRA-13595?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-13595: -- Status: Open (was: Patch Available) > Implement short read protection on partition boundaries > --- > > Key: CASSANDRA-13595 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13595 > Project: Cassandra > Issue Type: Bug > Components: Coordination >Reporter: Andrés de la Peña >Assignee: Aleksey Yeschenko > Labels: Correctness > Fix For: 3.0.15, 3.11.1 > > > It seems that short read protection doesn't work when the short read is done > at the end of a partition in a range query. The final assertion of this dtest > fails: > {code} > def short_read_partitions_delete_test(self): > cluster = self.cluster > cluster.set_configuration_options(values={'hinted_handoff_enabled': > False}) > cluster.set_batch_commitlog(enabled=True) > cluster.populate(2).start(wait_other_notice=True) > node1, node2 = self.cluster.nodelist() > session = self.patient_cql_connection(node1) > create_ks(session, 'ks', 2) > session.execute("CREATE TABLE t (k int, c int, PRIMARY KEY(k, c)) > WITH read_repair_chance = 0.0") > # we write 1 and 2 in a partition: all nodes get it. > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (1, 1)", > consistency_level=ConsistencyLevel.ALL)) > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (2, 1)", > consistency_level=ConsistencyLevel.ALL)) > # we delete partition 1: only node 1 gets it. > node2.flush() > node2.stop(wait_other_notice=True) > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 1")) > node2.start(wait_other_notice=True) > # we delete partition 2: only node 2 gets it. > node1.flush() > node1.stop(wait_other_notice=True) > session = self.patient_cql_connection(node2, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 2")) > node1.start(wait_other_notice=True) > # read from both nodes > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ALL) > assert_none(session, "SELECT * FROM t LIMIT 1") > {code} > However, the dtest passes if we remove the {{LIMIT 1}}. > Short read protection [uses a > {{SinglePartitionReadCommand}}|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/DataResolver.java#L484], > maybe it should use a {{PartitionRangeReadCommand}} instead? -- 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-13321) Add a checksum component for the sstable metadata (-Statistics.db) file
[ https://issues.apache.org/jira/browse/CASSANDRA-13321?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185878#comment-16185878 ] Marcus Eriksson edited comment on CASSANDRA-13321 at 9/29/17 2:51 PM: -- I have rebased and fixed a few issues with the simpler solution (https://github.com/krummas/cassandra/commits/marcuse/simplerchecksum) - I think we should go with that version for now as the swapping of sstable metadata seems to be working (and this doesn't change that part at all) https://circleci.com/gh/krummas/cassandra/126 https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/351/ -Note that the checksumming is a bit awkward since I didn't want to add a length before each metadata component, instead everything is hashed as it is being read. With sstable metadata, we could probably do that and verify the actual bytes instead - this is something you could consider during review- edit: checksumming the bytes instead was much nicer, so doing that, link to tree is updated in this comment was (Author: krummas): I have rebased and fixed a few issues with the simpler solution (https://github.com/krummas/cassandra/commits/marcuse/appendedchecksum) - I think we should go with that version for now as the swapping of sstable metadata seems to be working (and this doesn't change that part at all) https://circleci.com/gh/krummas/cassandra/125 https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/350/ Note that the checksumming is a bit awkward since I didn't want to add a length before each metadata component, instead everything is hashed as it is being read. With sstable metadata, we could probably do that and verify the actual bytes instead - this is something you could consider during review > Add a checksum component for the sstable metadata (-Statistics.db) file > --- > > Key: CASSANDRA-13321 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13321 > Project: Cassandra > Issue Type: Improvement >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson > Fix For: 4.x > > > Since we keep important information in the sstable metadata file now, we > should add a checksum component for it. One danger being if a bit gets > flipped in repairedAt we could consider the sstable repaired when it is not. -- 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-13595) Implement short read protection on partition boundaries
[ https://issues.apache.org/jira/browse/CASSANDRA-13595?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185928#comment-16185928 ] Sam Tunnicliffe commented on CASSANDRA-13595: - The patch looks good to me, there's just one thing I'm not entirely clear about. In CASSANDRA-13794 we added an (arbitrary) lower bound of 8 to limit of the single partition read in SRRP. This removes that, but it I'm not quite sure why. > Implement short read protection on partition boundaries > --- > > Key: CASSANDRA-13595 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13595 > Project: Cassandra > Issue Type: Bug > Components: Coordination >Reporter: Andrés de la Peña >Assignee: Aleksey Yeschenko > Labels: Correctness > Fix For: 3.0.15, 3.11.1 > > > It seems that short read protection doesn't work when the short read is done > at the end of a partition in a range query. The final assertion of this dtest > fails: > {code} > def short_read_partitions_delete_test(self): > cluster = self.cluster > cluster.set_configuration_options(values={'hinted_handoff_enabled': > False}) > cluster.set_batch_commitlog(enabled=True) > cluster.populate(2).start(wait_other_notice=True) > node1, node2 = self.cluster.nodelist() > session = self.patient_cql_connection(node1) > create_ks(session, 'ks', 2) > session.execute("CREATE TABLE t (k int, c int, PRIMARY KEY(k, c)) > WITH read_repair_chance = 0.0") > # we write 1 and 2 in a partition: all nodes get it. > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (1, 1)", > consistency_level=ConsistencyLevel.ALL)) > session.execute(SimpleStatement("INSERT INTO t (k, c) VALUES (2, 1)", > consistency_level=ConsistencyLevel.ALL)) > # we delete partition 1: only node 1 gets it. > node2.flush() > node2.stop(wait_other_notice=True) > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 1")) > node2.start(wait_other_notice=True) > # we delete partition 2: only node 2 gets it. > node1.flush() > node1.stop(wait_other_notice=True) > session = self.patient_cql_connection(node2, 'ks', > consistency_level=ConsistencyLevel.ONE) > session.execute(SimpleStatement("DELETE FROM t WHERE k = 2")) > node1.start(wait_other_notice=True) > # read from both nodes > session = self.patient_cql_connection(node1, 'ks', > consistency_level=ConsistencyLevel.ALL) > assert_none(session, "SELECT * FROM t LIMIT 1") > {code} > However, the dtest passes if we remove the {{LIMIT 1}}. > Short read protection [uses a > {{SinglePartitionReadCommand}}|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/DataResolver.java#L484], > maybe it should use a {{PartitionRangeReadCommand}} instead? -- 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-13916) Remove OpenJDK log warning
[ https://issues.apache.org/jira/browse/CASSANDRA-13916?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185919#comment-16185919 ] Eric Evans commented on CASSANDRA-13916: +1 This warning just creates confusion, AFAICT > Remove OpenJDK log warning > -- > > Key: CASSANDRA-13916 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13916 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Anthony Grasso >Priority: Minor > > The following warning message will appear in the logs when using OpenJDK > {noformat} > WARN [main] ... OpenJDK is not recommended. Please upgrade to the newest > Oracle Java release > {noformat} > The above warning dates back to when OpenJDK 6 was released and there were > some issues in early releases of this version. The OpenJDK implementation is > used as a reference for the OracleJDK which means the implementations are > very close. In addition, most users have moved off Java 6 so we can probably > remove this warning message. -- 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-13321) Add a checksum component for the sstable metadata (-Statistics.db) file
[ https://issues.apache.org/jira/browse/CASSANDRA-13321?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-13321: Status: Patch Available (was: Open) I have rebased and fixed a few issues with the simpler solution (https://github.com/krummas/cassandra/commits/marcuse/appendedchecksum) - I think we should go with that version for now as the swapping of sstable metadata seems to be working (and this doesn't change that part at all) https://circleci.com/gh/krummas/cassandra/125 https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/350/ Note that the checksumming is a bit awkward since I didn't want to add a length before each metadata component, instead everything is hashed as it is being read. With sstable metadata, we could probably do that and verify the actual bytes instead - this is something you could consider during review > Add a checksum component for the sstable metadata (-Statistics.db) file > --- > > Key: CASSANDRA-13321 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13321 > Project: Cassandra > Issue Type: Improvement >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson > Fix For: 4.x > > > Since we keep important information in the sstable metadata file now, we > should add a checksum component for it. One danger being if a bit gets > flipped in repairedAt we could consider the sstable repaired when it is not. -- 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-13918) Header only commit logs should be filtered before recovery
[ https://issues.apache.org/jira/browse/CASSANDRA-13918?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sam Tunnicliffe updated CASSANDRA-13918: Reviewer: Sam Tunnicliffe > Header only commit logs should be filtered before recovery > -- > > Key: CASSANDRA-13918 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13918 > Project: Cassandra > Issue Type: Bug >Reporter: Blake Eggleston >Assignee: Blake Eggleston > Fix For: 3.0.15, 4.0, 3.11.x > > > Commit log recovery will tolerate commit log truncation in the most recent > log file found on disk, but will abort startup if problems are detected in > others. > Since we allocate commit log segments before they're used though, it's > possible to get into a state where the last commit log file actually written > to is not the same file that was most recently allocated, preventing startup > for what should otherwise be allowable incomplete final segments. > Excluding header only files on recovery should prevent this from happening. -- 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-13911) IllegalStateException thrown by UPI.Serializer.hasNext() for some SELECT queries
[ https://issues.apache.org/jira/browse/CASSANDRA-13911?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185799#comment-16185799 ] Sam Tunnicliffe commented on CASSANDRA-13911: - +1 > IllegalStateException thrown by UPI.Serializer.hasNext() for some SELECT > queries > > > Key: CASSANDRA-13911 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13911 > Project: Cassandra > Issue Type: Bug > Components: Coordination >Reporter: Aleksey Yeschenko >Assignee: Aleksey Yeschenko > Fix For: 3.0.15, 3.11.1 > > > Certain combinations of rows, in presence of per partition limit (set > explicitly in 3.6+ or implicitly to 1 via DISTINCT) cause > {{UnfilteredPartitionIterators.Serializer.hasNext()}} to throw > {{IllegalStateException}} . > Relevant code snippet: > {code} > // We can't answer this until the previously returned iterator has been fully > consumed, > // so complain if that's not the case. > if (next != null && next.hasNext()) > throw new IllegalStateException("Cannot call hasNext() until the previous > iterator has been fully consumed"); > {code} > Since {{UnfilteredPartitionIterators.Serializer}} and > {{UnfilteredRowIteratorSerializer.serializer}} deserialize partitions/rows > lazily, it is required for correct operation of the partition iterator to > have the previous partition fully consumed, so that deserializing the next > one can start from the correct position in the byte buffer. However, that > condition won’t always be satisfied, as there are legitimate combinations of > rows that do not consume every row in every partition. > For example, look at [this > dtest|https://github.com/iamaleksey/cassandra-dtest/commits/13911]. > In case we end up with a following pattern of rows: > {code} > node1, partition 0 | 0 > node2, partition 0 | x x > {code} > , where {{x}} and {{x}} a row tombstones for rows 1 and 2, it’s sufficient > for {{MergeIterator}} to only look at row 0 in partition from node1 and at > row tombstone 1 from node2 to satisfy the per partition limit of 1. The > stopping merge result counter will stop iteration right there, leaving row > tombstone 2 from node2 unvisited and not deseiralized. Switching to the next > partition will in turn trigger the {{IllegalStateException}} because we > aren’t done yet. > The stopping counter is behaving correctly, so is the {{MergeIterator}}. I’ll > note that simply removing that condition is not enough to fix the problem > properly - it’d just cause us to deseiralize garbage, trying to deserialize a > new partition from a position in the bytebuffer that precedes remaining rows > in the previous partition. -- 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-13911) IllegalStateException thrown by UPI.Serializer.hasNext() for some SELECT queries
[ https://issues.apache.org/jira/browse/CASSANDRA-13911?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sam Tunnicliffe updated CASSANDRA-13911: Status: Ready to Commit (was: Patch Available) > IllegalStateException thrown by UPI.Serializer.hasNext() for some SELECT > queries > > > Key: CASSANDRA-13911 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13911 > Project: Cassandra > Issue Type: Bug > Components: Coordination >Reporter: Aleksey Yeschenko >Assignee: Aleksey Yeschenko > Fix For: 3.0.15, 3.11.1 > > > Certain combinations of rows, in presence of per partition limit (set > explicitly in 3.6+ or implicitly to 1 via DISTINCT) cause > {{UnfilteredPartitionIterators.Serializer.hasNext()}} to throw > {{IllegalStateException}} . > Relevant code snippet: > {code} > // We can't answer this until the previously returned iterator has been fully > consumed, > // so complain if that's not the case. > if (next != null && next.hasNext()) > throw new IllegalStateException("Cannot call hasNext() until the previous > iterator has been fully consumed"); > {code} > Since {{UnfilteredPartitionIterators.Serializer}} and > {{UnfilteredRowIteratorSerializer.serializer}} deserialize partitions/rows > lazily, it is required for correct operation of the partition iterator to > have the previous partition fully consumed, so that deserializing the next > one can start from the correct position in the byte buffer. However, that > condition won’t always be satisfied, as there are legitimate combinations of > rows that do not consume every row in every partition. > For example, look at [this > dtest|https://github.com/iamaleksey/cassandra-dtest/commits/13911]. > In case we end up with a following pattern of rows: > {code} > node1, partition 0 | 0 > node2, partition 0 | x x > {code} > , where {{x}} and {{x}} a row tombstones for rows 1 and 2, it’s sufficient > for {{MergeIterator}} to only look at row 0 in partition from node1 and at > row tombstone 1 from node2 to satisfy the per partition limit of 1. The > stopping merge result counter will stop iteration right there, leaving row > tombstone 2 from node2 unvisited and not deseiralized. Switching to the next > partition will in turn trigger the {{IllegalStateException}} because we > aren’t done yet. > The stopping counter is behaving correctly, so is the {{MergeIterator}}. I’ll > note that simply removing that condition is not enough to fix the problem > properly - it’d just cause us to deseiralize garbage, trying to deserialize a > new partition from a position in the bytebuffer that precedes remaining rows > in the previous partition. -- 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-13789) Reduce memory copies and object creations when acting on ByteBufs
[ https://issues.apache.org/jira/browse/CASSANDRA-13789?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185791#comment-16185791 ] Jason Brown commented on CASSANDRA-13789: - reverted as sha {{e296ff0632a41d5bbd29b407a83d886c10787f51}}. > Reduce memory copies and object creations when acting on ByteBufs > -- > > Key: CASSANDRA-13789 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13789 > Project: Cassandra > Issue Type: Improvement >Reporter: Norman Maurer >Assignee: Norman Maurer > Fix For: 4.0 > > Attachments: > 0001-CBUtil.sizeOfLongString-encodes-String-to-byte-to-ca.patch, > 0001-Reduce-memory-copies-and-object-creations-when-actin.patch > > > There are multiple "low-hanging-fruits" when it comes to reduce memory copies > and object allocations when acting on ByteBufs. -- 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: revert CBUtil.writeValue as pre discussion on CASSANDRA-13789
Repository: cassandra Updated Branches: refs/heads/trunk 55772088e -> e296ff063 revert CBUtil.writeValue as pre discussion on CASSANDRA-13789 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e296ff06 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e296ff06 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e296ff06 Branch: refs/heads/trunk Commit: e296ff0632a41d5bbd29b407a83d886c10787f51 Parents: 5577208 Author: Jason BrownAuthored: Fri Sep 29 06:04:41 2017 -0700 Committer: Jason Brown Committed: Fri Sep 29 06:04:41 2017 -0700 -- CHANGES.txt | 2 +- src/java/org/apache/cassandra/transport/CBUtil.java | 7 +-- 2 files changed, 2 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e296ff06/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 6e91a48..ea73fcd 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,5 @@ 4.0 - * Update lz4 to 1.4.0 (13741) + * Update lz4 to 1.4.0 (CASSANDRA-13741) * Optimize Paxos prepare and propose stage for local requests (CASSANDRA-13862) * Throttle base partitions during MV repair streaming to prevent OOM (CASSANDRA-13299) * Use compaction threshold for STCS in L0 (CASSANDRA-13861) http://git-wip-us.apache.org/repos/asf/cassandra/blob/e296ff06/src/java/org/apache/cassandra/transport/CBUtil.java -- diff --git a/src/java/org/apache/cassandra/transport/CBUtil.java b/src/java/org/apache/cassandra/transport/CBUtil.java index 52217a9..80b80b4 100644 --- a/src/java/org/apache/cassandra/transport/CBUtil.java +++ b/src/java/org/apache/cassandra/transport/CBUtil.java @@ -444,12 +444,7 @@ public abstract class CBUtil cb.writeInt(remaining); if (remaining > 0) -{ -// write and reset original position so we not need to create a duplicate. -int position = bytes.position(); -cb.writeBytes(bytes); -bytes.position(position); -} +cb.writeBytes(bytes.duplicate()); } public static int sizeOfValue(byte[] bytes) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-13789) Reduce memory copies and object creations when acting on ByteBufs
[ https://issues.apache.org/jira/browse/CASSANDRA-13789?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185785#comment-16185785 ] Jason Brown commented on CASSANDRA-13789: - I like [~slebresne]'s suggestion, and we can pursue that on the netty project itself, and upgrade if it happens. For the time being, though, I will revert the {{CBUtil#writeValue()}} problem that [~Stefania] uncovered. > Reduce memory copies and object creations when acting on ByteBufs > -- > > Key: CASSANDRA-13789 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13789 > Project: Cassandra > Issue Type: Improvement >Reporter: Norman Maurer >Assignee: Norman Maurer > Fix For: 4.0 > > Attachments: > 0001-CBUtil.sizeOfLongString-encodes-String-to-byte-to-ca.patch, > 0001-Reduce-memory-copies-and-object-creations-when-actin.patch > > > There are multiple "low-hanging-fruits" when it comes to reduce memory copies > and object allocations when acting on ByteBufs. -- 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-13741) Replace Cassandra's lz4-1.3.0.jar with lz4-java-1.4.0.jar
[ https://issues.apache.org/jira/browse/CASSANDRA-13741?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Brown updated CASSANDRA-13741: Resolution: Fixed Fix Version/s: (was: 4.x) 4.0 Status: Resolved (was: Ready to Commit) +1. Committed as sha {{55772088e0f6e227c45df935bd8283492a5f73f8}}. Thanks, everybody. > Replace Cassandra's lz4-1.3.0.jar with lz4-java-1.4.0.jar > - > > Key: CASSANDRA-13741 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13741 > Project: Cassandra > Issue Type: Improvement > Components: Libraries >Reporter: Amitkumar Ghatwal >Assignee: Michael Kjellman > Fix For: 4.0 > > > Hi All, > The latest lz4-java library has been released > (https://github.com/lz4/lz4-java/releases) and uploaded to maven central . > Please replace in mainline the current version ( 1.3.0) with the latest one ( > 1.4.0) from here - http://repo1.maven.org/maven2/org/lz4/lz4-java/1.4.0/ > Adding : [~ReiOdaira]. > Regards, > Amit -- 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-13741) Replace Cassandra's lz4-1.3.0.jar with lz4-java-1.4.0.jar
[ https://issues.apache.org/jira/browse/CASSANDRA-13741?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Brown updated CASSANDRA-13741: Status: Ready to Commit (was: Patch Available) > Replace Cassandra's lz4-1.3.0.jar with lz4-java-1.4.0.jar > - > > Key: CASSANDRA-13741 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13741 > Project: Cassandra > Issue Type: Improvement > Components: Libraries >Reporter: Amitkumar Ghatwal >Assignee: Michael Kjellman > Fix For: 4.x > > > Hi All, > The latest lz4-java library has been released > (https://github.com/lz4/lz4-java/releases) and uploaded to maven central . > Please replace in mainline the current version ( 1.3.0) with the latest one ( > 1.4.0) from here - http://repo1.maven.org/maven2/org/lz4/lz4-java/1.4.0/ > Adding : [~ReiOdaira]. > Regards, > Amit -- 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: Upgrade lz4 to 1.4.0
Repository: cassandra Updated Branches: refs/heads/trunk 6b4c8a973 -> 55772088e Upgrade lz4 to 1.4.0 patch by Michael Kjellman; reviewed by jasobrown for CASSANDRA-13741 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/55772088 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/55772088 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/55772088 Branch: refs/heads/trunk Commit: 55772088e0f6e227c45df935bd8283492a5f73f8 Parents: 6b4c8a9 Author: Michael KjellmanAuthored: Tue Sep 26 22:31:11 2017 -0700 Committer: Jason Brown Committed: Fri Sep 29 05:47:32 2017 -0700 -- CHANGES.txt| 1 + build.xml | 4 ++-- lib/lz4-1.3.0.jar | Bin 277311 -> 0 bytes lib/lz4-java-1.4.0.jar | Bin 0 -> 370119 bytes 4 files changed, 3 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/55772088/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index f2f23bd..6e91a48 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Update lz4 to 1.4.0 (13741) * Optimize Paxos prepare and propose stage for local requests (CASSANDRA-13862) * Throttle base partitions during MV repair streaming to prevent OOM (CASSANDRA-13299) * Use compaction threshold for STCS in L0 (CASSANDRA-13861) http://git-wip-us.apache.org/repos/asf/cassandra/blob/55772088/build.xml -- diff --git a/build.xml b/build.xml index 63518c5..4e7f9b1 100644 --- a/build.xml +++ b/build.xml @@ -358,7 +358,7 @@ - + @@ -565,7 +565,7 @@ version="${version}"/> - + http://git-wip-us.apache.org/repos/asf/cassandra/blob/55772088/lib/lz4-1.3.0.jar -- diff --git a/lib/lz4-1.3.0.jar b/lib/lz4-1.3.0.jar deleted file mode 100644 index 0fb0109..000 Binary files a/lib/lz4-1.3.0.jar and /dev/null differ http://git-wip-us.apache.org/repos/asf/cassandra/blob/55772088/lib/lz4-java-1.4.0.jar -- diff --git a/lib/lz4-java-1.4.0.jar b/lib/lz4-java-1.4.0.jar new file mode 100644 index 000..301908b Binary files /dev/null and b/lib/lz4-java-1.4.0.jar differ - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-13910) Consider deprecating (then removing) read_repair_chance/dclocal_read_repair_chance
[ https://issues.apache.org/jira/browse/CASSANDRA-13910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185759#comment-16185759 ] Stefan Podkowinski commented on CASSANDRA-13910: I'd also suggest to get some more user feedback on this, before dumping a working, but potentially confusing feature. > Consider deprecating (then removing) > read_repair_chance/dclocal_read_repair_chance > -- > > Key: CASSANDRA-13910 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13910 > Project: Cassandra > Issue Type: Improvement >Reporter: Sylvain Lebresne >Priority: Minor > Labels: CommunityFeedbackRequested > > First, let me clarify so this is not misunderstood that I'm not *at all* > suggesting to remove the read-repair mechanism of detecting and repairing > inconsistencies between read responses: that mechanism is imo fine and > useful. But the {{read_repair_chance}} and {{dclocal_read_repair_chance}} > have never been about _enabling_ that mechanism, they are about querying all > replicas (even when this is not required by the consistency level) for the > sole purpose of maybe read-repairing some of the replica that wouldn't have > been queried otherwise. Which btw, bring me to reason 1 for considering their > removal: their naming/behavior is super confusing. Over the years, I've seen > countless users (and not only newbies) misunderstanding what those options > do, and as a consequence misunderstand when read-repair itself was happening. > But my 2nd reason for suggesting this is that I suspect > {{read_repair_chance}}/{{dclocal_read_repair_chance}} are, especially > nowadays, more harmful than anything else when enabled. When those option > kick in, what you trade-off is additional resources consumption (all nodes > have to execute the read) for a _fairly remote chance_ of having some > inconsistencies repaired on _some_ replica _a bit faster_ than they would > otherwise be. To justify that last part, let's recall that: > # most inconsistencies are actually fixed by hints in practice; and in the > case where a node stay dead for a long time so that hints ends up timing-out, > you really should repair the node when it comes back (if not simply > re-bootstrapping it). Read-repair probably don't fix _that_ much stuff in > the first place. > # again, read-repair do happen without those options kicking in. If you do > reads at {{QUORUM}}, inconsistencies will eventually get read-repaired all > the same. Just a tiny bit less quickly. > # I suspect almost everyone use a low "chance" for those options at best > (because the extra resources consumption is real), so at the end of the day, > it's up to chance how much faster this fixes inconsistencies. > Overall, I'm having a hard time imagining real cases where that trade-off > really make sense. Don't get me wrong, those options had their places a long > time ago when hints weren't working all that well, but I think they bring > more confusion than benefits now. > And I think it's sane to reconsider stuffs every once in a while, and to > clean up anything that may not make all that much sense anymore, which I > think is the case here. > Tl;dr, I feel the benefits brought by those options are very slim at best and > well overshadowed by the confusion they bring, and not worth maintaining the > code that supports them (which, to be fair, isn't huge, but getting rid of > {{ReadCallback.AsyncRepairRunner}} wouldn't hurt for instance). > Lastly, if the consensus here ends up being that they can have their use in > weird case and that we fill supporting those cases is worth confusing > everyone else and maintaining that code, I would still suggest disabling them > totally by default. -- 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] [Resolved] (CASSANDRA-13149) AssertionError prepending to a list
[ https://issues.apache.org/jira/browse/CASSANDRA-13149?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Brown resolved CASSANDRA-13149. - Resolution: Fixed Fix Version/s: 4.0 3.11.1 3.0.15 Committed as sha {{ab0adf9f9bc72074a02025bdecd9479f790d6463}}. Thanks! > AssertionError prepending to a list > --- > > Key: CASSANDRA-13149 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13149 > Project: Cassandra > Issue Type: Bug > Components: CQL > Environment: 3.0.8 >Reporter: Steven Warren >Assignee: Jason Brown > Fix For: 3.0.15, 3.11.1, 4.0 > > > Prepending to a list produces the following AssertionError randomly. Changing > the update to append (and sort in the client) works around the issue. > {code} > java.lang.AssertionError: null > at > org.apache.cassandra.cql3.Lists$PrecisionTime.getNext(Lists.java:275) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at org.apache.cassandra.cql3.Lists$Prepender.execute(Lists.java:430) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.statements.UpdateStatement.addUpdateForKey(UpdateStatement.java:94) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.statements.ModificationStatement.addUpdates(ModificationStatement.java:682) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.statements.ModificationStatement.getMutations(ModificationStatement.java:613) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.statements.ModificationStatement.executeWithoutCondition(ModificationStatement.java:420) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.statements.ModificationStatement.execute(ModificationStatement.java:408) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:487) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:464) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:130) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:507) > [apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:401) > [apache-cassandra-3.0.8.jar:3.0.8] > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > [netty-all-4.0.23.Final.jar:4.0.23.Final] > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) > [netty-all-4.0.23.Final.jar:4.0.23.Final] > at > io.netty.channel.AbstractChannelHandlerContext.access$700(AbstractChannelHandlerContext.java:32) > [netty-all-4.0.23.Final.jar:4.0.23.Final] > at > io.netty.channel.AbstractChannelHandlerContext$8.run(AbstractChannelHandlerContext.java:324) > [netty-all-4.0.23.Final.jar:4.0.23.Final] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > [na:1.8.0_101] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) > [apache-cassandra-3.0.8.jar:3.0.8] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) > [apache-cassandra-3.0.8.jar:3.0.8] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101] > {code} -- 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] [Resolved] (CASSANDRA-13919) API changes review for Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-13919?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stefan Podkowinski resolved CASSANDRA-13919. Resolution: Invalid Please only file actionable tickets with our Jira and otherwise reach out to us on our [mailing lists|http://cassandra.apache.org/community/]. Thank you! > API changes review for Cassandra > > > Key: CASSANDRA-13919 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13919 > Project: Cassandra > Issue Type: Test >Reporter: Andrey Ponomarenko > Attachments: cassandra-1.png, cassandra-2.png > > > The review of API changes for the Cassandra library since 3.0.0 version: > https://abi-laboratory.pro/java/tracker/timeline/cassandra/ > The report is updated three times a week. Hope it will be helpful for users > and maintainers of the library. > The report is generated by the https://github.com/lvc/japi-compliance-checker > tool. > Thank you. > !cassandra-2.png|API symbols timeline! > !cassandra-1.png|API changes review! -- 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
[4/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11
Merge branch 'cassandra-3.0' into cassandra-3.11 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2a24acfa Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2a24acfa Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2a24acfa Branch: refs/heads/trunk Commit: 2a24acfa9926408216f5fde325abe0a8fd6b608a Parents: 0bc45aa ab0adf9 Author: Jason BrownAuthored: Fri Sep 29 05:21:32 2017 -0700 Committer: Jason Brown Committed: Fri Sep 29 05:24:11 2017 -0700 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/cql3/Lists.java | 76 +++-- .../org/apache/cassandra/cql3/ListsTest.java| 166 +++ 3 files changed, 226 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a24acfa/CHANGES.txt -- diff --cc CHANGES.txt index 80a8295,a1f49cd..6c3a1d0 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,19 -1,8 +1,20 @@@ -3.0.15 +3.11.1 +=== + * Fix the computation of cdc_total_space_in_mb for exabyte filesystems (CASSANDRA-13808) + * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) + * AbstractTokenTreeBuilder#serializedSize returns wrong value when there is a single leaf and overflow collisions (CASSANDRA-13869) + * Add a compaction option to TWCS to ignore sstables overlapping checks (CASSANDRA-13418) + * BTree.Builder memory leak (CASSANDRA-13754) + * Revert CASSANDRA-10368 of supporting non-pk column filtering due to correctness (CASSANDRA-13798) + * Fix cassandra-stress hang issues when an error during cluster connection happens (CASSANDRA-12938) + * Better bootstrap failure message when blocked by (potential) range movement (CASSANDRA-13744) + * "ignore" option is ignored in sstableloader (CASSANDRA-13721) + * Deadlock in AbstractCommitLogSegmentManager (CASSANDRA-13652) + * Duplicate the buffer before passing it to analyser in SASI operation (CASSANDRA-13512) + * Properly evict pstmts from prepared statements cache (CASSANDRA-13641) +Merged from 3.0: + * AssertionError prepending to a list (CASSANDRA-13149) * Fix support for SuperColumn tables (CASSANDRA-12373) - * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) - * Fix missing original update in TriggerExecutor (CASSANDRA-13894) * Remove non-rpc-ready nodes from counter leader candidates (CASSANDRA-13043) * Improve short read protection performance (CASSANDRA-13794) * Fix sstable reader to support range-tombstone-marker for multi-slices (CASSANDRA-13787) http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a24acfa/src/java/org/apache/cassandra/cql3/Lists.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a24acfa/test/unit/org/apache/cassandra/cql3/ListsTest.java -- diff --cc test/unit/org/apache/cassandra/cql3/ListsTest.java index 000,9ca0010..07623a2 mode 00,100644..100644 --- a/test/unit/org/apache/cassandra/cql3/ListsTest.java +++ b/test/unit/org/apache/cassandra/cql3/ListsTest.java @@@ -1,0 -1,166 +1,166 @@@ + /* + * 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.cql3; + + import java.nio.ByteBuffer; + import java.util.ArrayList; + import java.util.Collections; + import java.util.List; + import java.util.UUID; + + import com.google.common.collect.Iterators; + import org.junit.Assert; + import org.junit.Test; + + import org.apache.cassandra.config.CFMetaData; + import org.apache.cassandra.config.ColumnDefinition; + import org.apache.cassandra.cql3.Lists.PrecisionTime; + import org.apache.cassandra.db.Clustering; + import org.apache.cassandra.db.DecoratedKey; + import org.apache.cassandra.db.rows.Cell; + import org.apache.cassandra.db.rows.Row; + import
[6/6] cassandra git commit: Merge branch 'cassandra-3.11' into trunk
Merge branch 'cassandra-3.11' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6b4c8a97 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6b4c8a97 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6b4c8a97 Branch: refs/heads/trunk Commit: 6b4c8a9733f6eef276d602b440ae2f1906c1f264 Parents: 289f6b1 2a24acf Author: Jason BrownAuthored: Fri Sep 29 05:24:27 2017 -0700 Committer: Jason Brown Committed: Fri Sep 29 05:26:51 2017 -0700 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/cql3/Lists.java | 75 +++-- .../org/apache/cassandra/cql3/ListsTest.java| 166 +++ 3 files changed, 225 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6b4c8a97/CHANGES.txt -- diff --cc CHANGES.txt index a9108aa,6c3a1d0..f2f23bd --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -153,7 -13,8 +153,8 @@@ * Duplicate the buffer before passing it to analyser in SASI operation (CASSANDRA-13512) * Properly evict pstmts from prepared statements cache (CASSANDRA-13641) Merged from 3.0: + * AssertionError prepending to a list (CASSANDRA-13149) - * Fix support for SuperColumn tables (CASSANDRA-12373) + * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) * Remove non-rpc-ready nodes from counter leader candidates (CASSANDRA-13043) * Improve short read protection performance (CASSANDRA-13794) * Fix sstable reader to support range-tombstone-marker for multi-slices (CASSANDRA-13787) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6b4c8a97/src/java/org/apache/cassandra/cql3/Lists.java -- diff --cc src/java/org/apache/cassandra/cql3/Lists.java index 48fe54f,6a2a5a5..4a68df9 --- a/src/java/org/apache/cassandra/cql3/Lists.java +++ b/src/java/org/apache/cassandra/cql3/Lists.java @@@ -22,13 -22,12 +22,14 @@@ import static org.apache.cassandra.cql3 import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; +import java.util.Objects; +import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import org.apache.cassandra.schema.ColumnMetadata; + import com.google.common.annotations.VisibleForTesting; - -import org.apache.cassandra.config.ColumnDefinition; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.db.*; import org.apache.cassandra.db.rows.*; http://git-wip-us.apache.org/repos/asf/cassandra/blob/6b4c8a97/test/unit/org/apache/cassandra/cql3/ListsTest.java -- diff --cc test/unit/org/apache/cassandra/cql3/ListsTest.java index 000,07623a2..63c496c mode 00,100644..100644 --- a/test/unit/org/apache/cassandra/cql3/ListsTest.java +++ b/test/unit/org/apache/cassandra/cql3/ListsTest.java @@@ -1,0 -1,166 +1,166 @@@ + /* + * 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.cql3; + + import java.nio.ByteBuffer; + import java.util.ArrayList; + import java.util.Collections; + import java.util.List; + import java.util.UUID; + + import com.google.common.collect.Iterators; + import org.junit.Assert; + import org.junit.Test; + -import org.apache.cassandra.config.CFMetaData; -import org.apache.cassandra.config.ColumnDefinition; + import org.apache.cassandra.cql3.Lists.PrecisionTime; + import org.apache.cassandra.db.Clustering; + import org.apache.cassandra.db.DecoratedKey; + import org.apache.cassandra.db.rows.Cell; + import org.apache.cassandra.db.rows.Row; + import org.apache.cassandra.dht.Murmur3Partitioner; ++import org.apache.cassandra.schema.ColumnMetadata; ++import org.apache.cassandra.schema.TableMetadata; + import
[2/6] cassandra git commit: AssertionError prepending to a list
AssertionError prepending to a list patch by jasobrown, reviewed by Sam Tunnicliffe for CASSANDRA-13149 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ab0adf9f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ab0adf9f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ab0adf9f Branch: refs/heads/cassandra-3.11 Commit: ab0adf9f9bc72074a02025bdecd9479f790d6463 Parents: ce8c9b5 Author: Jason BrownAuthored: Tue Sep 12 17:05:13 2017 -0700 Committer: Jason Brown Committed: Fri Sep 29 05:18:54 2017 -0700 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/cql3/Lists.java | 76 +++-- .../org/apache/cassandra/cql3/ListsTest.java| 166 +++ 3 files changed, 226 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ab0adf9f/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 7745e8c..a1f49cd 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.15 + * AssertionError prepending to a list (CASSANDRA-13149) * Fix support for SuperColumn tables (CASSANDRA-12373) * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) * Fix missing original update in TriggerExecutor (CASSANDRA-13894) http://git-wip-us.apache.org/repos/asf/cassandra/blob/ab0adf9f/src/java/org/apache/cassandra/cql3/Lists.java -- diff --git a/src/java/org/apache/cassandra/cql3/Lists.java b/src/java/org/apache/cassandra/cql3/Lists.java index 559cf3f..065f74a 100644 --- a/src/java/org/apache/cassandra/cql3/Lists.java +++ b/src/java/org/apache/cassandra/cql3/Lists.java @@ -25,6 +25,8 @@ import java.util.List; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import com.google.common.annotations.VisibleForTesting; + import org.apache.cassandra.config.ColumnDefinition; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.db.*; @@ -243,18 +245,17 @@ public abstract class Lists } } -/* +/** * For prepend, we need to be able to generate unique but decreasing time - * UUID, which is a bit challenging. To do that, given a time in milliseconds, - * we adds a number representing the 100-nanoseconds precision and make sure - * that within the same millisecond, that number is always decreasing. We - * do rely on the fact that the user will only provide decreasing - * milliseconds timestamp for that purpose. + * UUIDs, which is a bit challenging. To do that, given a time in milliseconds, + * we add a number representing the 100-nanoseconds precision and make sure + * that within the same millisecond, that number is always decreasing. */ -private static class PrecisionTime +static class PrecisionTime { // Our reference time (1 jan 2010, 00:00:00) in milliseconds. private static final long REFERENCE_TIME = 126230400L; +static final int MAX_NANOS = ; private static final AtomicReference last = new AtomicReference<>(new PrecisionTime(Long.MAX_VALUE, 0)); public final long millis; @@ -266,21 +267,52 @@ public abstract class Lists this.nanos = nanos; } -static PrecisionTime getNext(long millis) +static PrecisionTime getNext(long millis, int count) { +if (count == 0) +return last.get(); + while (true) { PrecisionTime current = last.get(); -assert millis <= current.millis; -PrecisionTime next = millis < current.millis -? new PrecisionTime(millis, ) -: new PrecisionTime(millis, Math.max(0, current.nanos - 1)); +final PrecisionTime next; +if (millis < current.millis) +{ +next = new PrecisionTime(millis, MAX_NANOS - count); +} +else +{ +// in addition to being at the same millisecond, we handle the unexpected case of the millis parameter +// being in the past. That could happen if the System.currentTimeMillis() not operating montonically +// or if one thread is just a really big loser in the compareAndSet game of life. +long millisToUse = millis <= current.millis ? millis : current.millis; + +// if we will go below zero on the nanos, decrement the millis by one +final int nanosToUse;
[5/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11
Merge branch 'cassandra-3.0' into cassandra-3.11 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2a24acfa Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2a24acfa Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2a24acfa Branch: refs/heads/cassandra-3.11 Commit: 2a24acfa9926408216f5fde325abe0a8fd6b608a Parents: 0bc45aa ab0adf9 Author: Jason BrownAuthored: Fri Sep 29 05:21:32 2017 -0700 Committer: Jason Brown Committed: Fri Sep 29 05:24:11 2017 -0700 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/cql3/Lists.java | 76 +++-- .../org/apache/cassandra/cql3/ListsTest.java| 166 +++ 3 files changed, 226 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a24acfa/CHANGES.txt -- diff --cc CHANGES.txt index 80a8295,a1f49cd..6c3a1d0 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,19 -1,8 +1,20 @@@ -3.0.15 +3.11.1 +=== + * Fix the computation of cdc_total_space_in_mb for exabyte filesystems (CASSANDRA-13808) + * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) + * AbstractTokenTreeBuilder#serializedSize returns wrong value when there is a single leaf and overflow collisions (CASSANDRA-13869) + * Add a compaction option to TWCS to ignore sstables overlapping checks (CASSANDRA-13418) + * BTree.Builder memory leak (CASSANDRA-13754) + * Revert CASSANDRA-10368 of supporting non-pk column filtering due to correctness (CASSANDRA-13798) + * Fix cassandra-stress hang issues when an error during cluster connection happens (CASSANDRA-12938) + * Better bootstrap failure message when blocked by (potential) range movement (CASSANDRA-13744) + * "ignore" option is ignored in sstableloader (CASSANDRA-13721) + * Deadlock in AbstractCommitLogSegmentManager (CASSANDRA-13652) + * Duplicate the buffer before passing it to analyser in SASI operation (CASSANDRA-13512) + * Properly evict pstmts from prepared statements cache (CASSANDRA-13641) +Merged from 3.0: + * AssertionError prepending to a list (CASSANDRA-13149) * Fix support for SuperColumn tables (CASSANDRA-12373) - * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) - * Fix missing original update in TriggerExecutor (CASSANDRA-13894) * Remove non-rpc-ready nodes from counter leader candidates (CASSANDRA-13043) * Improve short read protection performance (CASSANDRA-13794) * Fix sstable reader to support range-tombstone-marker for multi-slices (CASSANDRA-13787) http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a24acfa/src/java/org/apache/cassandra/cql3/Lists.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a24acfa/test/unit/org/apache/cassandra/cql3/ListsTest.java -- diff --cc test/unit/org/apache/cassandra/cql3/ListsTest.java index 000,9ca0010..07623a2 mode 00,100644..100644 --- a/test/unit/org/apache/cassandra/cql3/ListsTest.java +++ b/test/unit/org/apache/cassandra/cql3/ListsTest.java @@@ -1,0 -1,166 +1,166 @@@ + /* + * 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.cql3; + + import java.nio.ByteBuffer; + import java.util.ArrayList; + import java.util.Collections; + import java.util.List; + import java.util.UUID; + + import com.google.common.collect.Iterators; + import org.junit.Assert; + import org.junit.Test; + + import org.apache.cassandra.config.CFMetaData; + import org.apache.cassandra.config.ColumnDefinition; + import org.apache.cassandra.cql3.Lists.PrecisionTime; + import org.apache.cassandra.db.Clustering; + import org.apache.cassandra.db.DecoratedKey; + import org.apache.cassandra.db.rows.Cell; + import org.apache.cassandra.db.rows.Row; + import
[1/6] cassandra git commit: AssertionError prepending to a list
Repository: cassandra Updated Branches: refs/heads/cassandra-3.0 ce8c9b559 -> ab0adf9f9 refs/heads/cassandra-3.11 0bc45aa46 -> 2a24acfa9 refs/heads/trunk 289f6b157 -> 6b4c8a973 AssertionError prepending to a list patch by jasobrown, reviewed by Sam Tunnicliffe for CASSANDRA-13149 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ab0adf9f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ab0adf9f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ab0adf9f Branch: refs/heads/cassandra-3.0 Commit: ab0adf9f9bc72074a02025bdecd9479f790d6463 Parents: ce8c9b5 Author: Jason BrownAuthored: Tue Sep 12 17:05:13 2017 -0700 Committer: Jason Brown Committed: Fri Sep 29 05:18:54 2017 -0700 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/cql3/Lists.java | 76 +++-- .../org/apache/cassandra/cql3/ListsTest.java| 166 +++ 3 files changed, 226 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ab0adf9f/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 7745e8c..a1f49cd 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.15 + * AssertionError prepending to a list (CASSANDRA-13149) * Fix support for SuperColumn tables (CASSANDRA-12373) * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) * Fix missing original update in TriggerExecutor (CASSANDRA-13894) http://git-wip-us.apache.org/repos/asf/cassandra/blob/ab0adf9f/src/java/org/apache/cassandra/cql3/Lists.java -- diff --git a/src/java/org/apache/cassandra/cql3/Lists.java b/src/java/org/apache/cassandra/cql3/Lists.java index 559cf3f..065f74a 100644 --- a/src/java/org/apache/cassandra/cql3/Lists.java +++ b/src/java/org/apache/cassandra/cql3/Lists.java @@ -25,6 +25,8 @@ import java.util.List; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import com.google.common.annotations.VisibleForTesting; + import org.apache.cassandra.config.ColumnDefinition; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.db.*; @@ -243,18 +245,17 @@ public abstract class Lists } } -/* +/** * For prepend, we need to be able to generate unique but decreasing time - * UUID, which is a bit challenging. To do that, given a time in milliseconds, - * we adds a number representing the 100-nanoseconds precision and make sure - * that within the same millisecond, that number is always decreasing. We - * do rely on the fact that the user will only provide decreasing - * milliseconds timestamp for that purpose. + * UUIDs, which is a bit challenging. To do that, given a time in milliseconds, + * we add a number representing the 100-nanoseconds precision and make sure + * that within the same millisecond, that number is always decreasing. */ -private static class PrecisionTime +static class PrecisionTime { // Our reference time (1 jan 2010, 00:00:00) in milliseconds. private static final long REFERENCE_TIME = 126230400L; +static final int MAX_NANOS = ; private static final AtomicReference last = new AtomicReference<>(new PrecisionTime(Long.MAX_VALUE, 0)); public final long millis; @@ -266,21 +267,52 @@ public abstract class Lists this.nanos = nanos; } -static PrecisionTime getNext(long millis) +static PrecisionTime getNext(long millis, int count) { +if (count == 0) +return last.get(); + while (true) { PrecisionTime current = last.get(); -assert millis <= current.millis; -PrecisionTime next = millis < current.millis -? new PrecisionTime(millis, ) -: new PrecisionTime(millis, Math.max(0, current.nanos - 1)); +final PrecisionTime next; +if (millis < current.millis) +{ +next = new PrecisionTime(millis, MAX_NANOS - count); +} +else +{ +// in addition to being at the same millisecond, we handle the unexpected case of the millis parameter +// being in the past. That could happen if the System.currentTimeMillis() not operating montonically +// or if one thread is just a really big loser in the compareAndSet game of life. +long millisToUse = millis
[3/6] cassandra git commit: AssertionError prepending to a list
AssertionError prepending to a list patch by jasobrown, reviewed by Sam Tunnicliffe for CASSANDRA-13149 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ab0adf9f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ab0adf9f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ab0adf9f Branch: refs/heads/trunk Commit: ab0adf9f9bc72074a02025bdecd9479f790d6463 Parents: ce8c9b5 Author: Jason BrownAuthored: Tue Sep 12 17:05:13 2017 -0700 Committer: Jason Brown Committed: Fri Sep 29 05:18:54 2017 -0700 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/cql3/Lists.java | 76 +++-- .../org/apache/cassandra/cql3/ListsTest.java| 166 +++ 3 files changed, 226 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ab0adf9f/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 7745e8c..a1f49cd 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.15 + * AssertionError prepending to a list (CASSANDRA-13149) * Fix support for SuperColumn tables (CASSANDRA-12373) * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) * Fix missing original update in TriggerExecutor (CASSANDRA-13894) http://git-wip-us.apache.org/repos/asf/cassandra/blob/ab0adf9f/src/java/org/apache/cassandra/cql3/Lists.java -- diff --git a/src/java/org/apache/cassandra/cql3/Lists.java b/src/java/org/apache/cassandra/cql3/Lists.java index 559cf3f..065f74a 100644 --- a/src/java/org/apache/cassandra/cql3/Lists.java +++ b/src/java/org/apache/cassandra/cql3/Lists.java @@ -25,6 +25,8 @@ import java.util.List; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import com.google.common.annotations.VisibleForTesting; + import org.apache.cassandra.config.ColumnDefinition; import org.apache.cassandra.cql3.functions.Function; import org.apache.cassandra.db.*; @@ -243,18 +245,17 @@ public abstract class Lists } } -/* +/** * For prepend, we need to be able to generate unique but decreasing time - * UUID, which is a bit challenging. To do that, given a time in milliseconds, - * we adds a number representing the 100-nanoseconds precision and make sure - * that within the same millisecond, that number is always decreasing. We - * do rely on the fact that the user will only provide decreasing - * milliseconds timestamp for that purpose. + * UUIDs, which is a bit challenging. To do that, given a time in milliseconds, + * we add a number representing the 100-nanoseconds precision and make sure + * that within the same millisecond, that number is always decreasing. */ -private static class PrecisionTime +static class PrecisionTime { // Our reference time (1 jan 2010, 00:00:00) in milliseconds. private static final long REFERENCE_TIME = 126230400L; +static final int MAX_NANOS = ; private static final AtomicReference last = new AtomicReference<>(new PrecisionTime(Long.MAX_VALUE, 0)); public final long millis; @@ -266,21 +267,52 @@ public abstract class Lists this.nanos = nanos; } -static PrecisionTime getNext(long millis) +static PrecisionTime getNext(long millis, int count) { +if (count == 0) +return last.get(); + while (true) { PrecisionTime current = last.get(); -assert millis <= current.millis; -PrecisionTime next = millis < current.millis -? new PrecisionTime(millis, ) -: new PrecisionTime(millis, Math.max(0, current.nanos - 1)); +final PrecisionTime next; +if (millis < current.millis) +{ +next = new PrecisionTime(millis, MAX_NANOS - count); +} +else +{ +// in addition to being at the same millisecond, we handle the unexpected case of the millis parameter +// being in the past. That could happen if the System.currentTimeMillis() not operating montonically +// or if one thread is just a really big loser in the compareAndSet game of life. +long millisToUse = millis <= current.millis ? millis : current.millis; + +// if we will go below zero on the nanos, decrement the millis by one +final int nanosToUse; +
[jira] [Updated] (CASSANDRA-13911) IllegalStateException thrown by UPI.Serializer.hasNext() for some SELECT queries
[ https://issues.apache.org/jira/browse/CASSANDRA-13911?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-13911: -- Status: Patch Available (was: In Progress) > IllegalStateException thrown by UPI.Serializer.hasNext() for some SELECT > queries > > > Key: CASSANDRA-13911 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13911 > Project: Cassandra > Issue Type: Bug > Components: Coordination >Reporter: Aleksey Yeschenko >Assignee: Aleksey Yeschenko > Fix For: 3.0.15, 3.11.1 > > > Certain combinations of rows, in presence of per partition limit (set > explicitly in 3.6+ or implicitly to 1 via DISTINCT) cause > {{UnfilteredPartitionIterators.Serializer.hasNext()}} to throw > {{IllegalStateException}} . > Relevant code snippet: > {code} > // We can't answer this until the previously returned iterator has been fully > consumed, > // so complain if that's not the case. > if (next != null && next.hasNext()) > throw new IllegalStateException("Cannot call hasNext() until the previous > iterator has been fully consumed"); > {code} > Since {{UnfilteredPartitionIterators.Serializer}} and > {{UnfilteredRowIteratorSerializer.serializer}} deserialize partitions/rows > lazily, it is required for correct operation of the partition iterator to > have the previous partition fully consumed, so that deserializing the next > one can start from the correct position in the byte buffer. However, that > condition won’t always be satisfied, as there are legitimate combinations of > rows that do not consume every row in every partition. > For example, look at [this > dtest|https://github.com/iamaleksey/cassandra-dtest/commits/13911]. > In case we end up with a following pattern of rows: > {code} > node1, partition 0 | 0 > node2, partition 0 | x x > {code} > , where {{x}} and {{x}} a row tombstones for rows 1 and 2, it’s sufficient > for {{MergeIterator}} to only look at row 0 in partition from node1 and at > row tombstone 1 from node2 to satisfy the per partition limit of 1. The > stopping merge result counter will stop iteration right there, leaving row > tombstone 2 from node2 unvisited and not deseiralized. Switching to the next > partition will in turn trigger the {{IllegalStateException}} because we > aren’t done yet. > The stopping counter is behaving correctly, so is the {{MergeIterator}}. I’ll > note that simply removing that condition is not enough to fix the problem > properly - it’d just cause us to deseiralize garbage, trying to deserialize a > new partition from a position in the bytebuffer that precedes remaining rows > in the previous partition. -- 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-13911) IllegalStateException thrown by UPI.Serializer.hasNext() for some SELECT queries
[ https://issues.apache.org/jira/browse/CASSANDRA-13911?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185724#comment-16185724 ] Aleksey Yeschenko commented on CASSANDRA-13911: --- Branches with fixes here: [3.0|https://github.com/iamaleksey/cassandra/commits/13911-3.0], [3.11|https://github.com/iamaleksey/cassandra/commits/13911-3.11], [4.0|https://github.com/iamaleksey/cassandra/commits/13911-4.0]. 3.11 needs an extra dtest to show that the change in {{DataResolver}} is necessary (using {{PER PARTITION LIMIT}} > 1), working on it. Tests are running. > IllegalStateException thrown by UPI.Serializer.hasNext() for some SELECT > queries > > > Key: CASSANDRA-13911 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13911 > Project: Cassandra > Issue Type: Bug > Components: Coordination >Reporter: Aleksey Yeschenko >Assignee: Aleksey Yeschenko > Fix For: 3.0.15, 3.11.1 > > > Certain combinations of rows, in presence of per partition limit (set > explicitly in 3.6+ or implicitly to 1 via DISTINCT) cause > {{UnfilteredPartitionIterators.Serializer.hasNext()}} to throw > {{IllegalStateException}} . > Relevant code snippet: > {code} > // We can't answer this until the previously returned iterator has been fully > consumed, > // so complain if that's not the case. > if (next != null && next.hasNext()) > throw new IllegalStateException("Cannot call hasNext() until the previous > iterator has been fully consumed"); > {code} > Since {{UnfilteredPartitionIterators.Serializer}} and > {{UnfilteredRowIteratorSerializer.serializer}} deserialize partitions/rows > lazily, it is required for correct operation of the partition iterator to > have the previous partition fully consumed, so that deserializing the next > one can start from the correct position in the byte buffer. However, that > condition won’t always be satisfied, as there are legitimate combinations of > rows that do not consume every row in every partition. > For example, look at [this > dtest|https://github.com/iamaleksey/cassandra-dtest/commits/13911]. > In case we end up with a following pattern of rows: > {code} > node1, partition 0 | 0 > node2, partition 0 | x x > {code} > , where {{x}} and {{x}} a row tombstones for rows 1 and 2, it’s sufficient > for {{MergeIterator}} to only look at row 0 in partition from node1 and at > row tombstone 1 from node2 to satisfy the per partition limit of 1. The > stopping merge result counter will stop iteration right there, leaving row > tombstone 2 from node2 unvisited and not deseiralized. Switching to the next > partition will in turn trigger the {{IllegalStateException}} because we > aren’t done yet. > The stopping counter is behaving correctly, so is the {{MergeIterator}}. I’ll > note that simply removing that condition is not enough to fix the problem > properly - it’d just cause us to deseiralize garbage, trying to deserialize a > new partition from a position in the bytebuffer that precedes remaining rows > in the previous partition. -- 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-9608) Support Java 9
[ https://issues.apache.org/jira/browse/CASSANDRA-9608?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185566#comment-16185566 ] Mark Paluch commented on CASSANDRA-9608: Any update on the {{DirectBuffer}} cleaner issue caused in {{FileUtils.:72}}? > Support Java 9 > -- > > Key: CASSANDRA-9608 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9608 > Project: Cassandra > Issue Type: Task >Reporter: Robert Stupp >Priority: Minor > > This ticket is intended to group all issues found to support Java 9 in the > future. > From what I've found out so far: > * Maven dependency {{com.sun:tools:jar:0}} via cobertura cannot be resolved. > It can be easily solved using this patch: > {code} > - artifactId="cobertura"/> > + artifactId="cobertura"> > + > + > {code} > * Another issue is that {{sun.misc.Unsafe}} no longer contains the methods > {{monitorEnter}} + {{monitorExit}}. These methods are used by > {{o.a.c.utils.concurrent.Locks}} which is only used by > {{o.a.c.db.AtomicBTreeColumns}}. > I don't mind to start working on this yet since Java 9 is in a too early > development phase. -- 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-7899) SSL does not work in cassandra-cli
[ https://issues.apache.org/jira/browse/CASSANDRA-7899?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185549#comment-16185549 ] jahar commented on CASSANDRA-7899: -- Hi what is correct command to run cli with SSL. I am using cassandra 2.1.16 and running following command. /etc/bin/cassandra-cli -u -pw -p 10760 -h --jmxusername --jmxpassword -ts /var/opt/keypath/abc.jks -tspw --debug It gives me following error: "Login failure. Did you specify 'keyspace', 'username' and 'password'?" When looking at source code, it simply returns the error message there is no stacktrace. > SSL does not work in cassandra-cli > -- > > Key: CASSANDRA-7899 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7899 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: Linux 2.6.32-431.20.3.el6.x86_64 #1 SMP Thu Jun 19 > 21:14:45 UTC 2014 x86_64 x86_64 x86_64 GNU/Linux > java version "1.7.0_17" > Java(TM) SE Runtime Environment (build 1.7.0_17-b02) > Java HotSpot(TM) 64-Bit Server VM (build 23.7-b01, mixed mode) > [cqlsh 4.1.1 | Cassandra 2.0.10 | CQL spec 3.1.1 | Thrift protocol 19.39.0] >Reporter: Zdenek Ott >Assignee: Jason Brown > Fix For: 2.0.11, 2.1.1 > > Attachments: 7899-v1.txt, 7899-v2.txt > > > When specify transport factory parameter '-tf > org.apache.cassandra.cli.transport.SSLTransportFactory' it throws exception, > see below, because SSLTransportFactory extends TTransportFactory not > ITransportFactory. > Exception in thread "main" java.lang.IllegalArgumentException: Cannot create > a transport factory 'org.apache.cassandra.cli.transport.SSLTransportFactory'. > at > org.apache.cassandra.cli.CliOptions.validateAndSetTransportFactory(CliOptions.java:288) > at > org.apache.cassandra.cli.CliOptions.processArgs(CliOptions.java:223) > at org.apache.cassandra.cli.CliMain.main(CliMain.java:230) > Caused by: java.lang.IllegalArgumentException: transport factory > 'org.apache.cassandra.cli.transport.SSLTransportFactory' not derived from > ITransportFactory > at > org.apache.cassandra.cli.CliOptions.validateAndSetTransportFactory(CliOptions.java:282) > ... 2 more -- 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-13741) Replace Cassandra's lz4-1.3.0.jar with lz4-java-1.4.0.jar
[ https://issues.apache.org/jira/browse/CASSANDRA-13741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185483#comment-16185483 ] Amitkumar Ghatwal commented on CASSANDRA-13741: --- Thanks - [~mkjellman] , let us know if you could review this - [~jasobrown] [~jjirsa] ? > Replace Cassandra's lz4-1.3.0.jar with lz4-java-1.4.0.jar > - > > Key: CASSANDRA-13741 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13741 > Project: Cassandra > Issue Type: Improvement > Components: Libraries >Reporter: Amitkumar Ghatwal >Assignee: Michael Kjellman > Fix For: 4.x > > > Hi All, > The latest lz4-java library has been released > (https://github.com/lz4/lz4-java/releases) and uploaded to maven central . > Please replace in mainline the current version ( 1.3.0) with the latest one ( > 1.4.0) from here - http://repo1.maven.org/maven2/org/lz4/lz4-java/1.4.0/ > Adding : [~ReiOdaira]. > Regards, > Amit -- 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-13808) Integer overflows with Amazon Elastic File System (EFS)
[ https://issues.apache.org/jira/browse/CASSANDRA-13808?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benjamin Lerer updated CASSANDRA-13808: --- Fix Version/s: 4.0 > Integer overflows with Amazon Elastic File System (EFS) > --- > > Key: CASSANDRA-13808 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13808 > Project: Cassandra > Issue Type: Bug > Components: Configuration, Core >Reporter: Vitalii Ishchenko >Assignee: Benjamin Lerer > Fix For: 3.11.1, 4.0 > > > Integer overflow issue was fixed for cassandra 2.2, but in 3.8 new property > was introduced in config that also derives from disk size > {{cdc_total_space_in_mb}}, see CASSANDRA-8844 > It should be updated too > https://github.com/apache/cassandra/blob/6b7d73a49695c0ceb78bc7a003ace606a806c13a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java#L484 -- 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-13808) Integer overflows with Amazon Elastic File System (EFS)
[ https://issues.apache.org/jira/browse/CASSANDRA-13808?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benjamin Lerer updated CASSANDRA-13808: --- Component/s: Configuration > Integer overflows with Amazon Elastic File System (EFS) > --- > > Key: CASSANDRA-13808 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13808 > Project: Cassandra > Issue Type: Bug > Components: Configuration, Core >Reporter: Vitalii Ishchenko >Assignee: Benjamin Lerer > Fix For: 3.11.1, 4.0 > > > Integer overflow issue was fixed for cassandra 2.2, but in 3.8 new property > was introduced in config that also derives from disk size > {{cdc_total_space_in_mb}}, see CASSANDRA-8844 > It should be updated too > https://github.com/apache/cassandra/blob/6b7d73a49695c0ceb78bc7a003ace606a806c13a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java#L484 -- 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-13808) Integer overflows with Amazon Elastic File System (EFS)
[ https://issues.apache.org/jira/browse/CASSANDRA-13808?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benjamin Lerer updated CASSANDRA-13808: --- Resolution: Fixed Status: Resolved (was: Ready to Commit) Committed into cassandra-3.11 at 0bc45aa46766625698e6e4c47085dfe94766c7df and merged into trunk. > Integer overflows with Amazon Elastic File System (EFS) > --- > > Key: CASSANDRA-13808 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13808 > Project: Cassandra > Issue Type: Bug > Components: Configuration, Core >Reporter: Vitalii Ishchenko >Assignee: Benjamin Lerer > Fix For: 3.11.1 > > > Integer overflow issue was fixed for cassandra 2.2, but in 3.8 new property > was introduced in config that also derives from disk size > {{cdc_total_space_in_mb}}, see CASSANDRA-8844 > It should be updated too > https://github.com/apache/cassandra/blob/6b7d73a49695c0ceb78bc7a003ace606a806c13a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java#L484 -- 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-13808) Integer overflows with Amazon Elastic File System (EFS)
[ https://issues.apache.org/jira/browse/CASSANDRA-13808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185475#comment-16185475 ] Benjamin Lerer commented on CASSANDRA-13808: Thanks for the review. > Integer overflows with Amazon Elastic File System (EFS) > --- > > Key: CASSANDRA-13808 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13808 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Vitalii Ishchenko >Assignee: Benjamin Lerer > Fix For: 3.11.1 > > > Integer overflow issue was fixed for cassandra 2.2, but in 3.8 new property > was introduced in config that also derives from disk size > {{cdc_total_space_in_mb}}, see CASSANDRA-8844 > It should be updated too > https://github.com/apache/cassandra/blob/6b7d73a49695c0ceb78bc7a003ace606a806c13a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java#L484 -- 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-13149) AssertionError prepending to a list
[ https://issues.apache.org/jira/browse/CASSANDRA-13149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185468#comment-16185468 ] Sam Tunnicliffe commented on CASSANDRA-13149: - LGTM. Doesn't look like there are any new dtest regressions, but CircleCI barfed on 2 of the runs and there are a couple of failures on the 3.0 branch (which I'm sure are unrelated), so IDK what you want to do there. +1 from me though > AssertionError prepending to a list > --- > > Key: CASSANDRA-13149 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13149 > Project: Cassandra > Issue Type: Bug > Components: CQL > Environment: 3.0.8 >Reporter: Steven Warren >Assignee: Jason Brown > > Prepending to a list produces the following AssertionError randomly. Changing > the update to append (and sort in the client) works around the issue. > {code} > java.lang.AssertionError: null > at > org.apache.cassandra.cql3.Lists$PrecisionTime.getNext(Lists.java:275) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at org.apache.cassandra.cql3.Lists$Prepender.execute(Lists.java:430) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.statements.UpdateStatement.addUpdateForKey(UpdateStatement.java:94) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.statements.ModificationStatement.addUpdates(ModificationStatement.java:682) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.statements.ModificationStatement.getMutations(ModificationStatement.java:613) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.statements.ModificationStatement.executeWithoutCondition(ModificationStatement.java:420) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.statements.ModificationStatement.execute(ModificationStatement.java:408) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:487) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:464) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:130) > ~[apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:507) > [apache-cassandra-3.0.8.jar:3.0.8] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:401) > [apache-cassandra-3.0.8.jar:3.0.8] > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > [netty-all-4.0.23.Final.jar:4.0.23.Final] > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) > [netty-all-4.0.23.Final.jar:4.0.23.Final] > at > io.netty.channel.AbstractChannelHandlerContext.access$700(AbstractChannelHandlerContext.java:32) > [netty-all-4.0.23.Final.jar:4.0.23.Final] > at > io.netty.channel.AbstractChannelHandlerContext$8.run(AbstractChannelHandlerContext.java:324) > [netty-all-4.0.23.Final.jar:4.0.23.Final] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > [na:1.8.0_101] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) > [apache-cassandra-3.0.8.jar:3.0.8] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) > [apache-cassandra-3.0.8.jar:3.0.8] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101] > {code} -- 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
[3/3] cassandra git commit: Merge branch cassandra-3.11 into trunk
Merge branch cassandra-3.11 into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/289f6b15 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/289f6b15 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/289f6b15 Branch: refs/heads/trunk Commit: 289f6b157ab8d044b5103e059724f71413bdb702 Parents: 5ea2c6b 0bc45aa Author: Benjamin LererAuthored: Fri Sep 29 09:34:41 2017 +0200 Committer: Benjamin Lerer Committed: Fri Sep 29 09:34:41 2017 +0200 -- CHANGES.txt | 3 ++- src/java/org/apache/cassandra/config/DatabaseDescriptor.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/289f6b15/CHANGES.txt -- diff --cc CHANGES.txt index 6d8d67e,80a8295..a9108aa --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,150 -1,11 +1,151 @@@ +4.0 + * Optimize Paxos prepare and propose stage for local requests (CASSANDRA-13862) + * Throttle base partitions during MV repair streaming to prevent OOM (CASSANDRA-13299) + * Use compaction threshold for STCS in L0 (CASSANDRA-13861) + * Fix problem with min_compress_ratio: 1 and disallow ratio < 1 (CASSANDRA-13703) + * Add extra information to SASI timeout exception (CASSANDRA-13677) + * Add incremental repair support for --hosts, --force, and subrange repair (CASSANDRA-13818) + * Rework CompactionStrategyManager.getScanners synchronization (CASSANDRA-13786) + * Add additional unit tests for batch behavior, TTLs, Timestamps (CASSANDRA-13846) + * Add keyspace and table name in schema validation exception (CASSANDRA-13845) + * Emit metrics whenever we hit tombstone failures and warn thresholds (CASSANDRA-13771) + * Make netty EventLoopGroups daemon threads (CASSANDRA-13837) + * Race condition when closing stream sessions (CASSANDRA-13852) + * NettyFactoryTest is failing in trunk on macOS (CASSANDRA-13831) + * Allow changing log levels via nodetool for related classes (CASSANDRA-12696) + * Add stress profile yaml with LWT (CASSANDRA-7960) + * Reduce memory copies and object creations when acting on ByteBufs (CASSANDRA-13789) + * Simplify mx4j configuration (Cassandra-13578) + * Fix trigger example on 4.0 (CASSANDRA-13796) + * Force minumum timeout value (CASSANDRA-9375) + * Use netty for streaming (CASSANDRA-12229) + * Use netty for internode messaging (CASSANDRA-8457) + * Add bytes repaired/unrepaired to nodetool tablestats (CASSANDRA-13774) + * Don't delete incremental repair sessions if they still have sstables (CASSANDRA-13758) + * Fix pending repair manager index out of bounds check (CASSANDRA-13769) + * Don't use RangeFetchMapCalculator when RF=1 (CASSANDRA-13576) + * Don't optimise trivial ranges in RangeFetchMapCalculator (CASSANDRA-13664) + * Use an ExecutorService for repair commands instead of new Thread(..).start() (CASSANDRA-13594) + * Fix race / ref leak in anticompaction (CASSANDRA-13688) + * Expose tasks queue length via JMX (CASSANDRA-12758) + * Fix race / ref leak in PendingRepairManager (CASSANDRA-13751) + * Enable ppc64le runtime as unsupported architecture (CASSANDRA-13615) + * Improve sstablemetadata output (CASSANDRA-11483) + * Support for migrating legacy users to roles has been dropped (CASSANDRA-13371) + * Introduce error metrics for repair (CASSANDRA-13387) + * Refactoring to primitive functional interfaces in AuthCache (CASSANDRA-13732) + * Update metrics to 3.1.5 (CASSANDRA-13648) + * batch_size_warn_threshold_in_kb can now be set at runtime (CASSANDRA-13699) + * Avoid always rebuilding secondary indexes at startup (CASSANDRA-13725) + * Upgrade JMH from 1.13 to 1.19 (CASSANDRA-13727) + * Upgrade SLF4J from 1.7.7 to 1.7.25 (CASSANDRA-12996) + * Default for start_native_transport now true if not set in config (CASSANDRA-13656) + * Don't add localhost to the graph when calculating where to stream from (CASSANDRA-13583) + * Make CDC availability more deterministic via hard-linking (CASSANDRA-12148) + * Allow skipping equality-restricted clustering columns in ORDER BY clause (CASSANDRA-10271) + * Use common nowInSec for validation compactions (CASSANDRA-13671) + * Improve handling of IR prepare failures (CASSANDRA-13672) + * Send IR coordinator messages synchronously (CASSANDRA-13673) + * Flush system.repair table before IR finalize promise (CASSANDRA-13660) + * Fix column filter creation for wildcard queries (CASSANDRA-13650) + * Add 'nodetool getbatchlogreplaythrottle' and 'nodetool setbatchlogreplaythrottle' (CASSANDRA-13614) + * fix race condition in PendingRepairManager (CASSANDRA-13659) + * Allow noop incremental repair state transitions
[1/3] cassandra git commit: Fix the computation of cdc_total_space_in_mb for exabyte filesystems
Repository: cassandra Updated Branches: refs/heads/cassandra-3.11 4734ce7d9 -> 0bc45aa46 refs/heads/trunk 5ea2c6b8e -> 289f6b157 Fix the computation of cdc_total_space_in_mb for exabyte filesystems patch by Benjamin Lerer; reviewed by Alex Petrov for CASSANDRA-13808 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0bc45aa4 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0bc45aa4 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0bc45aa4 Branch: refs/heads/cassandra-3.11 Commit: 0bc45aa46766625698e6e4c47085dfe94766c7df Parents: 4734ce7 Author: Benjamin LererAuthored: Fri Sep 29 09:30:02 2017 +0200 Committer: Benjamin Lerer Committed: Fri Sep 29 09:30:02 2017 +0200 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/config/DatabaseDescriptor.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bc45aa4/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 06d62b8..80a8295 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,6 @@ 3.11.1 === + * Fix the computation of cdc_total_space_in_mb for exabyte filesystems (CASSANDRA-13808) * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) * AbstractTokenTreeBuilder#serializedSize returns wrong value when there is a single leaf and overflow collisions (CASSANDRA-13869) * Add a compaction option to TWCS to ignore sstables overlapping checks (CASSANDRA-13418) http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bc45aa4/src/java/org/apache/cassandra/config/DatabaseDescriptor.java -- diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 97c868e..4ce2728 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -484,7 +484,7 @@ public class DatabaseDescriptor try { // use 1/8th of available space. See discussion on #10013 and #10199 on the CL, taking half that for CDC -minSize = Ints.checkedCast((guessFileStore(conf.cdc_raw_directory).getTotalSpace() / 1048576) / 8); +minSize = Ints.saturatedCast((guessFileStore(conf.cdc_raw_directory).getTotalSpace() / 1048576) / 8); } catch (IOException e) { - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[2/3] cassandra git commit: Fix the computation of cdc_total_space_in_mb for exabyte filesystems
Fix the computation of cdc_total_space_in_mb for exabyte filesystems patch by Benjamin Lerer; reviewed by Alex Petrov for CASSANDRA-13808 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0bc45aa4 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0bc45aa4 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0bc45aa4 Branch: refs/heads/trunk Commit: 0bc45aa46766625698e6e4c47085dfe94766c7df Parents: 4734ce7 Author: Benjamin LererAuthored: Fri Sep 29 09:30:02 2017 +0200 Committer: Benjamin Lerer Committed: Fri Sep 29 09:30:02 2017 +0200 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/config/DatabaseDescriptor.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bc45aa4/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 06d62b8..80a8295 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,6 @@ 3.11.1 === + * Fix the computation of cdc_total_space_in_mb for exabyte filesystems (CASSANDRA-13808) * Handle limit correctly on tables with strict liveness (CASSANDRA-13883) * AbstractTokenTreeBuilder#serializedSize returns wrong value when there is a single leaf and overflow collisions (CASSANDRA-13869) * Add a compaction option to TWCS to ignore sstables overlapping checks (CASSANDRA-13418) http://git-wip-us.apache.org/repos/asf/cassandra/blob/0bc45aa4/src/java/org/apache/cassandra/config/DatabaseDescriptor.java -- diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 97c868e..4ce2728 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -484,7 +484,7 @@ public class DatabaseDescriptor try { // use 1/8th of available space. See discussion on #10013 and #10199 on the CL, taking half that for CDC -minSize = Ints.checkedCast((guessFileStore(conf.cdc_raw_directory).getTotalSpace() / 1048576) / 8); +minSize = Ints.saturatedCast((guessFileStore(conf.cdc_raw_directory).getTotalSpace() / 1048576) / 8); } catch (IOException e) { - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-13909) Improve TRUNCATE performance with many sstables
[ https://issues.apache.org/jira/browse/CASSANDRA-13909?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16185446#comment-16185446 ] Marcus Eriksson commented on CASSANDRA-13909: - Didn't merge cleanly to 3.11/trunk, so running dtests on those branches before merging https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/345/ https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/346/ > Improve TRUNCATE performance with many sstables > --- > > Key: CASSANDRA-13909 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13909 > Project: Cassandra > Issue Type: Bug >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson > Fix For: 3.0.x, 3.11.x, 4.x > > > Truncate is very slow in 3.0, mostly due to {{LogRecord.make}} listing all > files in the directory for every sstable. -- 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-13919) API changes review for Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-13919?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrey Ponomarenko updated CASSANDRA-13919: --- Description: The review of API changes for the Cassandra library since 3.0.0 version: https://abi-laboratory.pro/java/tracker/timeline/cassandra/ The report is updated three times a week. Hope it will be helpful for users and maintainers of the library. The report is generated by the https://github.com/lvc/japi-compliance-checker tool. Thank you. !cassandra-2.png|API symbols timeline! !cassandra-1.png|API changes review! was: The review of API changes for the Cassandra library since 3.0.0 version: https://abi-laboratory.pro/java/tracker/timeline/cassandra/ The report is updated three times a week. Hope it will be helpful for users and maintainers of the library. The report is generated by the https://github.com/lvc/japi-compliance-checker tool. Thank you. !cassandra-report-2.png|API symbols timeline! !cassandra-report-1.png|API changes review! > API changes review for Cassandra > > > Key: CASSANDRA-13919 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13919 > Project: Cassandra > Issue Type: Test >Reporter: Andrey Ponomarenko > Attachments: cassandra-1.png, cassandra-2.png > > > The review of API changes for the Cassandra library since 3.0.0 version: > https://abi-laboratory.pro/java/tracker/timeline/cassandra/ > The report is updated three times a week. Hope it will be helpful for users > and maintainers of the library. > The report is generated by the https://github.com/lvc/japi-compliance-checker > tool. > Thank you. > !cassandra-2.png|API symbols timeline! > !cassandra-1.png|API changes review! -- 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-13919) API changes review for Cassandra
Andrey Ponomarenko created CASSANDRA-13919: -- Summary: API changes review for Cassandra Key: CASSANDRA-13919 URL: https://issues.apache.org/jira/browse/CASSANDRA-13919 Project: Cassandra Issue Type: Test Reporter: Andrey Ponomarenko Attachments: cassandra-1.png, cassandra-2.png The review of API changes for the Cassandra library since 3.0.0 version: https://abi-laboratory.pro/java/tracker/timeline/cassandra/ The report is updated three times a week. Hope it will be helpful for users and maintainers of the library. The report is generated by the https://github.com/lvc/japi-compliance-checker tool. Thank you. !cassandra-report-2.png|API symbols timeline! !cassandra-report-1.png|API changes review! -- 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