[jira] [Created] (CASSANDRA-4768) Add separate max_hint_window_in_ms option for remote data centers
Alexey Zotov created CASSANDRA-4768: --- Summary: Add separate max_hint_window_in_ms option for remote data centers Key: CASSANDRA-4768 URL: https://issues.apache.org/jira/browse/CASSANDRA-4768 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.2 Reporter: Alexey Zotov Assignee: Alexey Zotov Priority: Minor It would be nice to have possibility to configure hint window size for remote dc separately. It will allow to prevent accumulating of big amount of data for remote dc and long hints delivery as the result of it. I suggest to add max_hint_window_for_remote_dc_in_ms option. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (CASSANDRA-4768) Add separate max_hint_window_in_ms option for remote data centers
[ https://issues.apache.org/jira/browse/CASSANDRA-4768?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alexey Zotov updated CASSANDRA-4768: Attachment: cassandra-1.2-4768-remote_hint_window.txt cassandra-1.1-4768-remote_hint_window.txt Add separate max_hint_window_in_ms option for remote data centers - Key: CASSANDRA-4768 URL: https://issues.apache.org/jira/browse/CASSANDRA-4768 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.2 Reporter: Alexey Zotov Assignee: Alexey Zotov Priority: Minor Labels: configuration, hintedhandoff Attachments: cassandra-1.1-4768-remote_hint_window.txt, cassandra-1.2-4768-remote_hint_window.txt It would be nice to have possibility to configure hint window size for remote dc separately. It will allow to prevent accumulating of big amount of data for remote dc and long hints delivery as the result of it. I suggest to add max_hint_window_for_remote_dc_in_ms option. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (CASSANDRA-4768) Add separate max_hint_window_in_ms option for remote data centers
[ https://issues.apache.org/jira/browse/CASSANDRA-4768?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alexey Zotov updated CASSANDRA-4768: Fix Version/s: 1.2.0 1.1.6 Add separate max_hint_window_in_ms option for remote data centers - Key: CASSANDRA-4768 URL: https://issues.apache.org/jira/browse/CASSANDRA-4768 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.2 Reporter: Alexey Zotov Assignee: Alexey Zotov Priority: Minor Labels: configuration, hintedhandoff Fix For: 1.1.6, 1.2.0 Attachments: cassandra-1.1-4768-remote_hint_window.txt, cassandra-1.2-4768-remote_hint_window.txt It would be nice to have possibility to configure hint window size for remote dc separately. It will allow to prevent accumulating of big amount of data for remote dc and long hints delivery as the result of it. I suggest to add max_hint_window_for_remote_dc_in_ms option. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Reopened] (CASSANDRA-4710) High key hashing overhead for index scans when using RandomPartitioner
[ https://issues.apache.org/jira/browse/CASSANDRA-4710?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuki Morishita reopened CASSANDRA-4710: --- When I was trying to reproduce CASSANDRA-4733, I stumbled upon following error. {code} ERROR [ValidationExecutor:2] 2012-10-04 15:24:43,440 CassandraDaemon.java (line 132) Exception in thread Thread[ValidationExecutor:2,1,main] java.lang.AssertionError: 113427529603963934725865253558964126270 is not contained in (56713727820156410577229101238628035242,113427455640312821154458202477256070484] at org.apache.cassandra.service.AntiEntropyService$Validator.add(AntiEntropyService.java:345) at org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:727) at org.apache.cassandra.db.compaction.CompactionManager.access$600(CompactionManager.java:66) at org.apache.cassandra.db.compaction.CompactionManager$8.call(CompactionManager.java:451) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) {code} It turned out that the cause was SSTR#getPositionsForRanges returning unrelated section of file due to bug in SSTR#getPosition. getPosition was returning null when it should return position. getPosition starts search for key from nearest sampled index up to index interval count. The following check inside getPosition: {code} while (!input.isEOF() i DatabaseDescriptor.getIndexInterval()) {code} stops search for indexed position when it searches all indexes between index sampling intervals and method returns null. But with the check above, when searching for key that is greater than the last key inside index interval but is less than next sampled index, the method returns null instead of the position. I think the fix for this is changing to =. High key hashing overhead for index scans when using RandomPartitioner -- Key: CASSANDRA-4710 URL: https://issues.apache.org/jira/browse/CASSANDRA-4710 Project: Cassandra Issue Type: Improvement Reporter: Daniel Norberg Assignee: Daniel Norberg Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 0001-SSTableReader-compare-raw-key-when-scanning-index.patch For a workload where the dataset is completely in ram, the md5 hashing of the keys during index scans becomes a bottleneck for reads when using RandomPartitioner, according to profiling. Instead performing a raw key equals check in SSTableReader.getPosition() for EQ operations improves throughput by some 30% for my workload (moving the bottleneck elsewhere). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4684) Binary protocol: inform clients of schema changes
[ https://issues.apache.org/jira/browse/CASSANDRA-4684?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470072#comment-13470072 ] Sylvain Lebresne commented on CASSANDRA-4684: - All I'm saying is that there is a lot of case where a client needs to access the schema. And for some tools, like an eclipse plugin for instance, they would need to access the schema all the time (to offer say completion or validation). Now you don't need this to access the schema, you query the system table. However, if you do need to access the schema often, how do you do that? Well, either you query the database every damn time, and your plugin/tool/code will be super slow. Or, more likely, you cache the schema client side and implement some regular polling to refresh that cache. Which works, mostly, but has the defaults of polling: should you poll often or not? If you poll too often it's inefficient, if you poll not often enough you'll provide a bad user experience. Don't get me wrong, I'm not pretending that this is the worst problem database face today, but that is not far fetched either and all this ticket does is to provide a better solution to that problem. So why wouldn't we give people a better solution if we can? And we can very easily. It's not like this small patch touch any sensible part of the code, or somehow makes parts of the code unreadable (at least to me it seems like a very minor addition). I also note that the patch don't even impose anything on clients implementors since events are optional (and optional by type of events). As for libpq, as I said, this patch is an optimization so the fact that libpq don't support it is not a proof that it's useless either. And as a side note, and while I'm not expert in libpq, it has an aynchronous notification mechanism and I wouldn't be surprised that along with some simple trigger you can very easily have schema change notifications (does the eclipse plugin uses that if that does work? I don't know and frankly I don't care). Binary protocol: inform clients of schema changes - Key: CASSANDRA-4684 URL: https://issues.apache.org/jira/browse/CASSANDRA-4684 Project: Cassandra Issue Type: Improvement Affects Versions: 1.2.0 beta 1 Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 0001-Return-schema-change-infos.txt, 0002-Add-migration-events.txt It would be nice to inform clients when a schema change occurs as this would allow said client to maintain the current state of the schema, which might be useful/desirable. To allow that, we can: # return that a query has changed the schema (instead of simply a 'void' return), in the same spirit than CASSANDRA-3707. # add events notification on schema change. Just to be clear, the goal is only to inform that a change has occured, the client would still have to query the system table to know the exact content of the change, but at least it'll know when to do such query. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4733) Last written key = current key exception when streaming
[ https://issues.apache.org/jira/browse/CASSANDRA-4733?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470075#comment-13470075 ] Yuki Morishita commented on CASSANDRA-4733: --- I still cannot reproduce above error, but I believe this was caused by the change in CASSANDRA-4710. SSTableReader#getPositionsForRanges is used to determine the sections to transfer inside sstable, but the method returns incorrect sections for some cases. In fact, system.log file that I got from Brandon showed that the node was trying to stream sections way bigger than actual sstable file size. Last written key = current key exception when streaming Key: CASSANDRA-4733 URL: https://issues.apache.org/jira/browse/CASSANDRA-4733 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.2.0 beta 1 Reporter: Brandon Williams Assignee: Yuki Morishita {noformat} ERROR 16:52:56,260 Exception in thread Thread[Streaming to /10.179.111.137:1,5,main] java.lang.RuntimeException: java.io.IOException: Connection reset by peer at com.google.common.base.Throwables.propagate(Throwables.java:160) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) Caused by: java.io.IOException: Connection reset by peer at sun.nio.ch.FileDispatcher.write0(Native Method) at sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:29) at sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:72) at sun.nio.ch.IOUtil.write(IOUtil.java:43) at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:334) at java.nio.channels.Channels.writeFullyImpl(Channels.java:59) at java.nio.channels.Channels.writeFully(Channels.java:81) at java.nio.channels.Channels.access$000(Channels.java:47) at java.nio.channels.Channels$1.write(Channels.java:155) at com.ning.compress.lzf.ChunkEncoder.encodeAndWriteChunk(ChunkEncoder.java:133) at com.ning.compress.lzf.LZFOutputStream.writeCompressedBlock(LZFOutputStream.java:203) at com.ning.compress.lzf.LZFOutputStream.write(LZFOutputStream.java:97) at org.apache.cassandra.streaming.FileStreamTask.write(FileStreamTask.java:218) at org.apache.cassandra.streaming.FileStreamTask.stream(FileStreamTask.java:164) at org.apache.cassandra.streaming.FileStreamTask.runMayThrow(FileStreamTask.java:91) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ... 3 more ERROR 16:53:03,951 Exception in thread Thread[Thread-11,5,main] java.lang.RuntimeException: Last written key DecoratedKey(113424593524874987650593774422007331058, 3036303936343535) = current key DecoratedKey(59229538317742990547810678738983628664, 3036313133373139) writing into /var/lib/cassandra/data/Keyspace1-Standard1-tmp-ia-95-Data.db at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:132) at org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:208) at org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:164) at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:107) at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:220) at org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:165) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:65) {noformat} I didn't do anything fancy here, just inserted about 6M keys at rf=2, then ran repair and got this. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (CASSANDRA-4769) Prevent parallel hint delivery to the node
Alexey Zotov created CASSANDRA-4769: --- Summary: Prevent parallel hint delivery to the node Key: CASSANDRA-4769 URL: https://issues.apache.org/jira/browse/CASSANDRA-4769 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.2 Reporter: Alexey Zotov It's actual only in case of the using a big enough cluster. After node's failure other nodes try to send hints to the restored node. So theoretically it can affect performance of restored node. I suggest to create some mechanism for synchronization of hints delivery processes to restored node. Could you please explain how it can be implemented. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
git commit: Fix support of collections in prepared statements
Updated Branches: refs/heads/trunk d54a93f2d - 8b00f3a25 Fix support of collections in prepared statements patch by slebresne; reviewed by jbellis for CASSANDRA-4739 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8b00f3a2 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8b00f3a2 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8b00f3a2 Branch: refs/heads/trunk Commit: 8b00f3a258fcc04f0350d4f46760eacacbfed3df Parents: d54a93f Author: Sylvain Lebresne sylv...@datastax.com Authored: Fri Oct 5 09:29:17 2012 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Fri Oct 5 09:29:17 2012 +0200 -- CHANGES.txt|1 + doc/native_protocol.spec | 25 - .../cassandra/cql3/operations/ColumnOperation.java | 20 .../cassandra/cql3/operations/ListOperation.java | 27 ++ .../cassandra/cql3/operations/MapOperation.java| 25 + .../cassandra/cql3/operations/SetOperation.java| 28 ++ .../cassandra/cql3/statements/UpdateStatement.java | 35 +- .../org/apache/cassandra/db/marshal/ListType.java | 28 +++--- .../org/apache/cassandra/db/marshal/MapType.java | 39 ++- .../org/apache/cassandra/db/marshal/SetType.java | 28 +++--- 10 files changed, 210 insertions(+), 46 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b00f3a2/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 8d25639..868183e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -19,6 +19,7 @@ * Add support for multiple column family outputs in CFOF (CASSANDRA-4208) * Support repairing only the local DC nodes (CASSANDRA-4747) * Use rpc_address for binary protocol and change default port (CASSANRA-4751) + * Fix use of collections in prepared statements (CASSANDRA-4739) 1.2-beta1 http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b00f3a2/doc/native_protocol.spec -- diff --git a/doc/native_protocol.spec b/doc/native_protocol.spec index 908..2e03a02 100644 --- a/doc/native_protocol.spec +++ b/doc/native_protocol.spec @@ -33,7 +33,8 @@ Table of Contents 4.2.5.4. Prepared 4.2.6. EVENT 5. Compression - 6. Error codes + 6. Collection types + 7. Error codes 1. Overview @@ -286,7 +287,7 @@ Table of Contents Indicates an error processing a request. The body of the message will be an error code ([int]) followed by a [string] error message. Then, depending on the exception, more content may follow. The error codes are defined in - Section 6, along with their additional content if any. + Section 7, along with their additional content if any. 4.2.2. READY @@ -452,7 +453,25 @@ Table of Contents flag (see Section 2.2) is set. -6. Error codes +6. Collection types + + This section describe the serialization format for the collection types: + list, map and set. This serialization format is both useful to decode values + returned in RESULT messages but also to encode values for EXECUTE ones. + + The serialization formats are: + List: a [short] n indicating the size of the list, followed by n elements. + Each element is [short bytes] representing the serialized element + value. + Map: a [short] n indicating the size of the map, followed by n entries. + Each entry is composed of two [short bytes] representing the key and + the value of the entry map. + Set: a [short] n indicating the size of the set, followed by n elements. + Each element is [short bytes] representing the serialized element + value. + + +7. Error codes The supported error codes are described below: 0xServer error: something unexpected happened. This indicates a http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b00f3a2/src/java/org/apache/cassandra/cql3/operations/ColumnOperation.java -- diff --git a/src/java/org/apache/cassandra/cql3/operations/ColumnOperation.java b/src/java/org/apache/cassandra/cql3/operations/ColumnOperation.java index e7086c1..0f4c1fc 100644 --- a/src/java/org/apache/cassandra/cql3/operations/ColumnOperation.java +++ b/src/java/org/apache/cassandra/cql3/operations/ColumnOperation.java @@ -27,6 +27,9 @@ import org.apache.cassandra.db.IColumn; import org.apache.cassandra.db.filter.QueryPath; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.CollectionType; +import org.apache.cassandra.db.marshal.ListType; +import
git commit: Store more informations in peers table
Updated Branches: refs/heads/trunk 8b00f3a25 - d5ec013ce Store more informations in peers table patch by slebresne; reviewed by jbellis for CASSANDRA-4351 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d5ec013c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d5ec013c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d5ec013c Branch: refs/heads/trunk Commit: d5ec013cee4f3d923d9618694716a265ab04fe1b Parents: 8b00f3a Author: Sylvain Lebresne sylv...@datastax.com Authored: Fri Oct 5 09:34:51 2012 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Fri Oct 5 09:34:51 2012 +0200 -- CHANGES.txt|1 + .../org/apache/cassandra/config/CFMetaData.java| 12 +- .../apache/cassandra/cql3/UntypedResultSet.java|6 + src/java/org/apache/cassandra/db/SystemTable.java | 134 +++ .../apache/cassandra/service/StorageService.java | 19 ++ 5 files changed, 95 insertions(+), 77 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/d5ec013c/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 868183e..342135f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -20,6 +20,7 @@ * Support repairing only the local DC nodes (CASSANDRA-4747) * Use rpc_address for binary protocol and change default port (CASSANRA-4751) * Fix use of collections in prepared statements (CASSANDRA-4739) + * Store more information into peers table (CASSANDRA-4351) 1.2-beta1 http://git-wip-us.apache.org/repos/asf/cassandra/blob/d5ec013c/src/java/org/apache/cassandra/config/CFMetaData.java -- diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java index 6abeb33..ef25d2a 100644 --- a/src/java/org/apache/cassandra/config/CFMetaData.java +++ b/src/java/org/apache/cassandra/config/CFMetaData.java @@ -158,13 +158,19 @@ public final class CFMetaData + AND COMMENT='hints awaiting delivery'); public static final CFMetaData PeersCf = compile(12, CREATE TABLE + SystemTable.PEERS_CF + ( - + token_bytes blob PRIMARY KEY, - + peer inet + + peer inet PRIMARY KEY, + + ring_id uuid, + + tokens setblob, + + schema_version uuid, + + release_version text, + + rpc_address inet, + + data_center text, + + rack text + ) WITH COMMENT='known peers in the cluster'); public static final CFMetaData LocalCf = compile(13, CREATE TABLE + SystemTable.LOCAL_CF + ( + key text PRIMARY KEY, - + token_bytes blob, + + tokens setblob, + cluster_name text, + gossip_generation int, + bootstrapped text, http://git-wip-us.apache.org/repos/asf/cassandra/blob/d5ec013c/src/java/org/apache/cassandra/cql3/UntypedResultSet.java -- diff --git a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java index 203e4c1..ca3acf5 100644 --- a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java +++ b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java @@ -25,6 +25,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import com.google.common.collect.AbstractIterator; @@ -130,6 +131,11 @@ public class UntypedResultSet implements IterableUntypedResultSet.Row return DateType.instance.compose(data.get(column)); } +public T SetT getSet(String column, AbstractTypeT type) +{ +return SetType.getInstance(type).compose(data.get(column)); +} +
buildbot failure in ASF Buildbot on cassandra-trunk
The Buildbot has detected a new failure on builder cassandra-trunk while building cassandra. Full details are available at: http://ci.apache.org/builders/cassandra-trunk/builds/1901 Buildbot URL: http://ci.apache.org/ Buildslave for this Build: portunus_ubuntu Build Reason: scheduler Build Source Stamp: [branch trunk] d5ec013cee4f3d923d9618694716a265ab04fe1b Blamelist: Sylvain Lebresne sylv...@datastax.com BUILD FAILED: failed shell sincerely, -The Buildbot
[jira] [Commented] (CASSANDRA-4351) Consider storing more informations on peers in system tables
[ https://issues.apache.org/jira/browse/CASSANDRA-4351?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470087#comment-13470087 ] Sylvain Lebresne commented on CASSANDRA-4351: - Committed 0001 but holding this open a little long to see if we decide something on the 2nd part. bq. what if instead we change LongToken to/fromString to hex-encode with a constant width, the way CASSANDRA-4550 wanted? Of course then we'd need to switch it to unsigned comparison That's not a bad idea (since we have no backward compatibility problem) so why not (and switching to unsigned comparison is probably not a big deal (though we do have to be careful about the fact that the minimum token shouldn't be a valid token, so tokens value will have to be in [1, 2^64-1])). That being said, I'm not sure about the instead in the sentence above. Was that to be understood as in addition to 0002? Consider storing more informations on peers in system tables - Key: CASSANDRA-4351 URL: https://issues.apache.org/jira/browse/CASSANDRA-4351 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Sylvain Lebresne Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 0001-4351.txt, 0002-Save-tokens-as-strings.txt Currently, the only thing we keep in system tables about other peers is their token and IP addresses. We should probably also record the new ring_id, but since CASSANDRA-4018 makes system table easily queriable, may it could be worth adding some more information (basically most of what we gossip could be a candidate (schema UUID, status, C* version, ...)) as a simple way to expose the ring state to users (even if it's just a view of the ring state from one specific node I believe it's still nice). Of course that means storing information that may not be absolutely needed by the server, but I'm not sure there is much harm to that. Note that doing this cleanly may require changing the schema of current system tables but as long as we do that in the 1.2 timeframe it's ok (since the concerned system table 'local' and 'peers' are news anyway). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4763) SSTableLoader shouldn't get keyspace from path
[ https://issues.apache.org/jira/browse/CASSANDRA-4763?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470088#comment-13470088 ] Sylvain Lebresne commented on CASSANDRA-4763: - bq. we require sstable files to be named according to the convention cassandra is using? We do, and so there is no need for the user to provide the keyspace name as argument to the loader. That being said, we can lift the limitation that sstables currently must be in a directory named after the keyspace (I think we all agree here, just wanted to clarify). SSTableLoader shouldn't get keyspace from path -- Key: CASSANDRA-4763 URL: https://issues.apache.org/jira/browse/CASSANDRA-4763 Project: Cassandra Issue Type: Bug Components: Tools Affects Versions: 1.2.0 beta 1 Reporter: Nick Bailey Priority: Minor SSTableLoader currently gets the keyspace it is going to load to from the path of the directoy of sstables it is loading. This isn't really documented (or I didn't see it), but also isn't really a good way of doing it in general. {noformat} this.keyspace = directory.getParentFile().getName(); {noformat} We should probably just let users pass the name in. If you are loading a snapshot the file names will have the keyspace which is slightly better but people manually creating their own sstables might not format them the same. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
git commit: Fix comparison against IndexInterval in SSTR.getPosition()
Updated Branches: refs/heads/trunk d5ec013ce - 2a91a4818 Fix comparison against IndexInterval in SSTR.getPosition() Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2a91a481 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2a91a481 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2a91a481 Branch: refs/heads/trunk Commit: 2a91a48181b269684d491d961a0c513bf81baf25 Parents: d5ec013 Author: Sylvain Lebresne sylv...@datastax.com Authored: Fri Oct 5 10:24:37 2012 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Fri Oct 5 10:24:37 2012 +0200 -- .../apache/cassandra/io/sstable/SSTableReader.java |9 ++--- 1 files changed, 6 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2a91a481/src/java/org/apache/cassandra/io/sstable/SSTableReader.java -- diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java index b89ee24..a67c1ab 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java @@ -758,15 +758,18 @@ public class SSTableReader extends SSTable // scan the on-disk index, starting at the nearest sampled position. // The check against IndexInterval is to be exit the loop in the EQ case when the key looked for is not present -// (bloom filter false positive). +// (bloom filter false positive). But note that for non-EQ cases, we might need to check the first key of the +// next index position because the searched key can be greater the last key of the index interval checked if it +// is lesser than the first key of next interval (and in that case we must return the position of the first key +// of the next interval). int i = 0; IteratorFileDataInput segments = ifile.iterator(sampledPosition, INDEX_FILE_BUFFER_BYTES); -while (segments.hasNext() i DatabaseDescriptor.getIndexInterval()) +while (segments.hasNext() i = DatabaseDescriptor.getIndexInterval()) { FileDataInput in = segments.next(); try { -while (!in.isEOF() i DatabaseDescriptor.getIndexInterval()) +while (!in.isEOF() i = DatabaseDescriptor.getIndexInterval()) { i++;
[jira] [Commented] (CASSANDRA-4710) High key hashing overhead for index scans when using RandomPartitioner
[ https://issues.apache.org/jira/browse/CASSANDRA-4710?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470100#comment-13470100 ] Sylvain Lebresne commented on CASSANDRA-4710: - I agree, good catch. I went ahead a committed the fix (with some comment) in commit 2a91a48. Thanks Yuki. High key hashing overhead for index scans when using RandomPartitioner -- Key: CASSANDRA-4710 URL: https://issues.apache.org/jira/browse/CASSANDRA-4710 Project: Cassandra Issue Type: Improvement Reporter: Daniel Norberg Assignee: Daniel Norberg Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 0001-SSTableReader-compare-raw-key-when-scanning-index.patch For a workload where the dataset is completely in ram, the md5 hashing of the keys during index scans becomes a bottleneck for reads when using RandomPartitioner, according to profiling. Instead performing a raw key equals check in SSTableReader.getPosition() for EQ operations improves throughput by some 30% for my workload (moving the bottleneck elsewhere). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Resolved] (CASSANDRA-4710) High key hashing overhead for index scans when using RandomPartitioner
[ https://issues.apache.org/jira/browse/CASSANDRA-4710?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne resolved CASSANDRA-4710. - Resolution: Fixed High key hashing overhead for index scans when using RandomPartitioner -- Key: CASSANDRA-4710 URL: https://issues.apache.org/jira/browse/CASSANDRA-4710 Project: Cassandra Issue Type: Improvement Reporter: Daniel Norberg Assignee: Daniel Norberg Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 0001-SSTableReader-compare-raw-key-when-scanning-index.patch For a workload where the dataset is completely in ram, the md5 hashing of the keys during index scans becomes a bottleneck for reads when using RandomPartitioner, according to profiling. Instead performing a raw key equals check in SSTableReader.getPosition() for EQ operations improves throughput by some 30% for my workload (moving the bottleneck elsewhere). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4710) High key hashing overhead for index scans when using RandomPartitioner
[ https://issues.apache.org/jira/browse/CASSANDRA-4710?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470105#comment-13470105 ] Daniel Norberg commented on CASSANDRA-4710: --- Good catch. High key hashing overhead for index scans when using RandomPartitioner -- Key: CASSANDRA-4710 URL: https://issues.apache.org/jira/browse/CASSANDRA-4710 Project: Cassandra Issue Type: Improvement Reporter: Daniel Norberg Assignee: Daniel Norberg Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 0001-SSTableReader-compare-raw-key-when-scanning-index.patch For a workload where the dataset is completely in ram, the md5 hashing of the keys during index scans becomes a bottleneck for reads when using RandomPartitioner, according to profiling. Instead performing a raw key equals check in SSTableReader.getPosition() for EQ operations improves throughput by some 30% for my workload (moving the bottleneck elsewhere). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
git commit: Fix SystemTableTest
Updated Branches: refs/heads/trunk 2a91a4818 - b7716c76b Fix SystemTableTest Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b7716c76 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b7716c76 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b7716c76 Branch: refs/heads/trunk Commit: b7716c76b338ad8eea0a19822ec9ba99de1699b6 Parents: 2a91a48 Author: Sylvain Lebresne sylv...@datastax.com Authored: Fri Oct 5 14:54:12 2012 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Fri Oct 5 14:54:12 2012 +0200 -- src/java/org/apache/cassandra/db/SystemTable.java |2 +- 1 files changed, 1 insertions(+), 1 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/b7716c76/src/java/org/apache/cassandra/db/SystemTable.java -- diff --git a/src/java/org/apache/cassandra/db/SystemTable.java b/src/java/org/apache/cassandra/db/SystemTable.java index e2ff161..ff7d81a 100644 --- a/src/java/org/apache/cassandra/db/SystemTable.java +++ b/src/java/org/apache/cassandra/db/SystemTable.java @@ -238,7 +238,7 @@ public class SystemTable continue; String req = UPDATE system.%s SET tokens = tokens - %s WHERE peer = '%s'; -processInternal(String.format(req, PEERS_CF, serializeTokens(toRemove), entry.getKey())); +processInternal(String.format(req, PEERS_CF, serializeTokens(toRemove), entry.getKey().getHostAddress())); } forceBlockingFlush(PEERS_CF); }
git commit: (cql3) protect against null prepared variables (and avoid flooding the log on InvalidException errors)
Updated Branches: refs/heads/trunk b7716c76b - e232407ad (cql3) protect against null prepared variables (and avoid flooding the log on InvalidException errors) Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e232407a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e232407a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e232407a Branch: refs/heads/trunk Commit: e232407adf534e54950c2766347d4a0321c039ad Parents: b7716c7 Author: Sylvain Lebresne sylv...@datastax.com Authored: Fri Oct 5 16:31:26 2012 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Fri Oct 5 16:31:26 2012 +0200 -- src/java/org/apache/cassandra/cql3/Term.java |3 +++ .../cassandra/transport/messages/ErrorMessage.java |2 +- .../cassandra/transport/messages/QueryMessage.java | 10 ++ 3 files changed, 6 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e232407a/src/java/org/apache/cassandra/cql3/Term.java -- diff --git a/src/java/org/apache/cassandra/cql3/Term.java b/src/java/org/apache/cassandra/cql3/Term.java index 3f46c35..dc180ce 100644 --- a/src/java/org/apache/cassandra/cql3/Term.java +++ b/src/java/org/apache/cassandra/cql3/Term.java @@ -127,6 +127,9 @@ public class Term throw new AssertionError(a marker Term was encountered with no index value); ByteBuffer value = variables.get(bindIndex); +// We don't yet support null values in prepared statements +if (value == null) +throw new InvalidRequestException(Invalid null value for prepared variable + bindIndex); validator.validate(value); return value; } http://git-wip-us.apache.org/repos/asf/cassandra/blob/e232407a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java -- diff --git a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java index 8ed8e94..b7b0bee 100644 --- a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java +++ b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java @@ -181,7 +181,7 @@ public class ErrorMessage extends Message.Response return new ErrorMessage((TransportException)e); // Unexpected exception -logger.debug(Unexpected exception during request, e); +logger.error(Unexpected exception during request, e); return new ErrorMessage(new ServerError(e)); } http://git-wip-us.apache.org/repos/asf/cassandra/blob/e232407a/src/java/org/apache/cassandra/transport/messages/QueryMessage.java -- diff --git a/src/java/org/apache/cassandra/transport/messages/QueryMessage.java b/src/java/org/apache/cassandra/transport/messages/QueryMessage.java index c291e1d..0880ee0 100644 --- a/src/java/org/apache/cassandra/transport/messages/QueryMessage.java +++ b/src/java/org/apache/cassandra/transport/messages/QueryMessage.java @@ -20,9 +20,7 @@ package org.apache.cassandra.transport.messages; import org.jboss.netty.buffer.ChannelBuffer; import org.apache.cassandra.cql3.QueryProcessor; -import org.apache.cassandra.exceptions.InvalidRequestException; -import org.apache.cassandra.exceptions.RequestTimeoutException; -import org.apache.cassandra.exceptions.UnavailableException; +import org.apache.cassandra.exceptions.*; import org.apache.cassandra.transport.*; /** @@ -65,12 +63,8 @@ public class QueryMessage extends Message.Request } catch (Exception e) { -if (!((e instanceof UnavailableException) - || (e instanceof InvalidRequestException) - || (e instanceof RequestTimeoutException))) -{ +if (!((e instanceof RequestValidationException) || (e instanceof RequestExecutionException))) logger.error(Unexpected error during query, e); -} return ErrorMessage.fromException(e); } }
[jira] [Commented] (CASSANDRA-4768) Add separate max_hint_window_in_ms option for remote data centers
[ https://issues.apache.org/jira/browse/CASSANDRA-4768?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470367#comment-13470367 ] Jonathan Ellis commented on CASSANDRA-4768: --- I'm not sure if this is the right approach. Cross-DC repair is painful too, after all, and that's basically your alternative. I think parallelizing HH as discussed in your other ticket is a better solution. Add separate max_hint_window_in_ms option for remote data centers - Key: CASSANDRA-4768 URL: https://issues.apache.org/jira/browse/CASSANDRA-4768 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.2 Reporter: Alexey Zotov Assignee: Alexey Zotov Priority: Minor Labels: configuration, hintedhandoff Fix For: 1.1.6, 1.2.0 Attachments: cassandra-1.1-4768-remote_hint_window.txt, cassandra-1.2-4768-remote_hint_window.txt It would be nice to have possibility to configure hint window size for remote dc separately. It will allow to prevent accumulating of big amount of data for remote dc and long hints delivery as the result of it. I suggest to add max_hint_window_for_remote_dc_in_ms option. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4684) Binary protocol: inform clients of schema changes
[ https://issues.apache.org/jira/browse/CASSANDRA-4684?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470374#comment-13470374 ] Jonathan Ellis commented on CASSANDRA-4684: --- I still think this is premature complexification. As mentioned, our only real prior art for schema caching is cli because it needs to for ASSUME; otherwise query-schema-every-time is completely fast enough. Of course any such tool will have to implement query-full-schema anyway to get initial state, so that's the simplest possible solution. My gut is that this is probably adequate so let's put push in the YAGNI column until demonstrated otherwise. Binary protocol: inform clients of schema changes - Key: CASSANDRA-4684 URL: https://issues.apache.org/jira/browse/CASSANDRA-4684 Project: Cassandra Issue Type: Improvement Affects Versions: 1.2.0 beta 1 Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 0001-Return-schema-change-infos.txt, 0002-Add-migration-events.txt It would be nice to inform clients when a schema change occurs as this would allow said client to maintain the current state of the schema, which might be useful/desirable. To allow that, we can: # return that a query has changed the schema (instead of simply a 'void' return), in the same spirit than CASSANDRA-3707. # add events notification on schema change. Just to be clear, the goal is only to inform that a change has occured, the client would still have to query the system table to know the exact content of the change, but at least it'll know when to do such query. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4684) Binary protocol: inform clients of schema changes
[ https://issues.apache.org/jira/browse/CASSANDRA-4684?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470399#comment-13470399 ] Sylvain Lebresne commented on CASSANDRA-4684: - But it's very very little complexification. I mean, it's about zero added complexity client side unless they want to care about optimizing that use case (and they are willing to, why limit them?). And server side, it adds a few lines of code, sure, but I hardly see how it complexify things. bq. so let's put push in the YAGNI column until demonstrated otherwise My problem with that is that this is that since there is an inferior workaround (polling or taking the hit of query the schema each time), it's unclear we'll hear from client that would benefit from that. Or we may only hear about that when lots of people have implemented that less then inferior workaround. I agree that we shouldn't add feature just because we can, but I really think that's not a case of that. Having to access the schema regularly client-side feels hardly like a crazy thing people would need. Why not optimize it if it cost us nothing (and again, I do think it cost us about nothing). Binary protocol: inform clients of schema changes - Key: CASSANDRA-4684 URL: https://issues.apache.org/jira/browse/CASSANDRA-4684 Project: Cassandra Issue Type: Improvement Affects Versions: 1.2.0 beta 1 Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 0001-Return-schema-change-infos.txt, 0002-Add-migration-events.txt It would be nice to inform clients when a schema change occurs as this would allow said client to maintain the current state of the schema, which might be useful/desirable. To allow that, we can: # return that a query has changed the schema (instead of simply a 'void' return), in the same spirit than CASSANDRA-3707. # add events notification on schema change. Just to be clear, the goal is only to inform that a change has occured, the client would still have to query the system table to know the exact content of the change, but at least it'll know when to do such query. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (CASSANDRA-4770) (CQL3) data type not in lowercase are not handled correctly.
Sylvain Lebresne created CASSANDRA-4770: --- Summary: (CQL3) data type not in lowercase are not handled correctly. Key: CASSANDRA-4770 URL: https://issues.apache.org/jira/browse/CASSANDRA-4770 Project: Cassandra Issue Type: Bug Affects Versions: 1.1.0 Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Priority: Minor Fix For: 1.1.6 Seems that we accept {{int}} but we don't accept {{INT}} (that is, the parser accepts it, but we fail later to recognize it). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (CASSANDRA-4770) (CQL3) data type not in lowercase are not handled correctly.
[ https://issues.apache.org/jira/browse/CASSANDRA-4770?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-4770: Attachment: 4770.txt Trivial fix attached. (CQL3) data type not in lowercase are not handled correctly. Key: CASSANDRA-4770 URL: https://issues.apache.org/jira/browse/CASSANDRA-4770 Project: Cassandra Issue Type: Bug Affects Versions: 1.1.0 Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Priority: Minor Labels: cql3 Fix For: 1.1.6 Attachments: 4770.txt Seems that we accept {{int}} but we don't accept {{INT}} (that is, the parser accepts it, but we fail later to recognize it). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[Cassandra Wiki] Update of Partitioners by JonathanEllis
Dear Wiki user, You have subscribed to a wiki page or wiki category on Cassandra Wiki for change notification. The Partitioners page has been changed by JonathanEllis: http://wiki.apache.org/cassandra/Partitioners New page: The Partitioner, along with the ReplicationStrategy, is responsible for deciding what data is placed on which nodes in your cluster. A partition is determined by a ''partition key;'' in the simplest form, a partition is a single row, but a partition may also be made up of multiple rows. Rows within a partition are ordered, which is often useful at query time. Rows may be retrieved from a partition in ascending or descending order. This is covered in more detail here: http://www.datastax.com/dev/blog/schema-in-cassandra-1-1. A specific example of doing time series data is here: http://www.rustyrazorblade.com/2012/10/cassandra-cql3-and-time-series-data-with-timeuuid/ Cassandra supports the following partitioners, in order of preference. TLDR: always use !Murmur3Partitioner in new clusters. == !Murmur3Partitioner == This is the default in Cassandra 1.2. It's functionally the same as RandomPartitioner, but Murmur3 is a much faster hash than MD5. We don't care about MD5's cryptographic properties; all we need is a good distribution over the hash space, which Murmur3 delivers. == !RandomPartitioner == Default partitioner in Cassandra 1.1 and earlier. Hashes with MD5. == !ByteOrderedPartitioner == An order-preserving partitioner that operates on partition key bytes lexicographically. == !OrderPreservingPartitioner == Assumes keys are UTF8 strings. Not recommended both because of this limitation and because globally ordering all your partitions generates hot spots: some partitions close together will get more activity than others, and the node hosting those will be overloaded relative to others. You can try to mitigate with active load balancing but this works poorly in practice; by the time you can adjust token assignments so that less hot partitions are on the overloaded node, your workload often changes enough that the hot spot is now elsewhere. Remember that preserving global order means you can't just pick and choose hot partitions to relocate, you have to relocate contiguous ranges.
[Cassandra Wiki] Update of Partitioners by JonathanEllis
Dear Wiki user, You have subscribed to a wiki page or wiki category on Cassandra Wiki for change notification. The Partitioners page has been changed by JonathanEllis: http://wiki.apache.org/cassandra/Partitioners?action=diffrev1=1rev2=2 Cassandra supports the following partitioners, in order of preference. TLDR: always use !Murmur3Partitioner in new clusters. - == !Murmur3Partitioner == + == Murmur3Partitioner == This is the default in Cassandra 1.2. It's functionally the same as RandomPartitioner, but Murmur3 is a much faster hash than MD5. We don't care about MD5's cryptographic properties; all we need is a good distribution over the hash space, which Murmur3 delivers. - == !RandomPartitioner == + == RandomPartitioner == Default partitioner in Cassandra 1.1 and earlier. Hashes with MD5. - == !ByteOrderedPartitioner == + == ByteOrderedPartitioner == An order-preserving partitioner that operates on partition key bytes lexicographically. - == !OrderPreservingPartitioner == + == OrderPreservingPartitioner == Assumes keys are UTF8 strings. Not recommended both because of this limitation and because globally ordering all your partitions generates hot spots: some partitions close together will get more activity than others, and the node hosting those will be overloaded relative to others. You can try to mitigate with active load balancing but this works poorly in practice; by the time you can adjust token assignments so that less hot partitions are on the overloaded node, your workload often changes enough that the hot spot is now elsewhere. Remember that preserving global order means you can't just pick and choose hot partitions to relocate, you have to relocate contiguous ranges.
[jira] [Commented] (CASSANDRA-4766) ReverseCompaction
[ https://issues.apache.org/jira/browse/CASSANDRA-4766?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470430#comment-13470430 ] T Jake Luciani commented on CASSANDRA-4766: --- You could do this with a new MaxSizeCompactionStrategy. It wouldn't need to be exact or anything (imagine a row that blows the threshold) ReverseCompaction - Key: CASSANDRA-4766 URL: https://issues.apache.org/jira/browse/CASSANDRA-4766 Project: Cassandra Issue Type: New Feature Reporter: Edward Capriolo Priority: Minor Sometimes you run into a situation where your sized tiered SSTables get to be a funky size. Maybe a repair starts this or you were forced into a compaction. In any case we should be able to anti-compact a table, since anti-compact is a bad word lets call this reverse compact. This could be done online or offline. Closely related is SSTables cound have a max size. Leveled is not right for everyone. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[Cassandra Wiki] Update of Partitioners by JonathanEllis
Dear Wiki user, You have subscribed to a wiki page or wiki category on Cassandra Wiki for change notification. The Partitioners page has been changed by JonathanEllis: http://wiki.apache.org/cassandra/Partitioners?action=diffrev1=2rev2=3 == RandomPartitioner == - Default partitioner in Cassandra 1.1 and earlier. Hashes with MD5. + Default partitioner in Cassandra 1.1 and earlier. Hashes with MD5. Fine to use on legacy clusters but Murmur3 is faster with no downsides. == ByteOrderedPartitioner == - An order-preserving partitioner that operates on partition key bytes lexicographically. + An order-preserving partitioner that operates on partition key bytes lexicographically. Not recommended (see below). == OrderPreservingPartitioner ==
[jira] [Commented] (CASSANDRA-4732) clean up unnecessary copies on KeyCache path
[ https://issues.apache.org/jira/browse/CASSANDRA-4732?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470436#comment-13470436 ] Yuki Morishita commented on CASSANDRA-4732: --- +1 clean up unnecessary copies on KeyCache path Key: CASSANDRA-4732 URL: https://issues.apache.org/jira/browse/CASSANDRA-4732 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.0 Reporter: Jonathan Ellis Assignee: Jonathan Ellis Priority: Trivial Fix For: 1.2.0 beta 2 Attachments: 4732-v2.txt, keycache-cleanup.txt -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[Cassandra Wiki] Update of FAQ by JonathanEllis
Dear Wiki user, You have subscribed to a wiki page or wiki category on Cassandra Wiki for change notification. The FAQ page has been changed by JonathanEllis: http://wiki.apache.org/cassandra/FAQ?action=diffrev1=150rev2=151 Comment: remove some obsolete questions and add Partitioners link * [[#ports|What ports does Cassandra use?]] * [[#slows_down_after_lotso_inserts|Why does Cassandra slow down after doing a lot of inserts?]] * [[#existing_data_when_adding_new_nodes|What happens to existing data in my cluster when I add new nodes?]] - * [[#modify_cf_config|Can I add/remove/rename Column Families on a working cluster?]] * [[#node_clients_connect_to|Does it matter which node a Thrift or higher-level client connects to?]] * [[#what_kind_of_hardware_should_i_use|What kind of hardware should I run Cassandra on?]] * [[#architecture|What are SSTables and Memtables?]] * [[#working_with_timeuuid_in_java|Why is it so hard to work with TimeUUIDType in Java?]] * [[#i_deleted_what_gives|I delete data from Cassandra, but disk usage stays the same. What gives?]] - * [[#reads_slower_writes|Why are reads slower than writes?]] * [[#cloned|Why does nodeprobe ring only show one entry, even though my nodes logged that they see each other joining the ring?]] * [[#range_ghosts|Why do deleted keys show up during range scans?]] * [[#change_replication|Can I change the ReplicationFactor on a live cluster?]] @@ -20, +18 @@ * [[#iter_world|How can I iterate over all the rows in a ColumnFamily?]] * [[#no_keyspaces|Why were none of the keyspaces described in storage-conf.xml loaded?]] * [[#gui|Is there a GUI admin tool for Cassandra?]] - * [[#a_long_is_exactly_8_bytes|Insert operation throws InvalidRequestException with message A long is exactly 8 bytes]] * [[#clustername_mismatch|Cassandra says ClusterName mismatch: oldClusterName != newClusterName and refuses to start]] * [[#batch_mutate_atomic|Are batch_mutate operations atomic?]] * [[#hadoop_support|Is Hadoop (i.e. Map/Reduce, Pig, Hive) supported?]] @@ -33, +30 @@ * [[#bulkloading|How do I bulk load data into Cassandra?]] * [[#range_rp|Why aren't range slices/sequential scans giving me the expected results?]] * [[#unsubscribe|How do I unsubscribe from the email list?]] - * [[#cleaning_compacted_tables|I compacted, so why did space used not decrease?]] * [[#mmap|Why does top report that Cassandra is using a lot more memory than the Java heap max?]] * [[#jna|I'm getting java.io.IOException: Cannot run program ln when trying to snapshot or update a keyspace]] * [[#replicaplacement|How does Cassandra decide which nodes have what data?]] * [[#cachehitrateunits|I have a row or key cache hit rate of 0.XX123456789. Is that XX% or 0.XX% ?]] - * [[#bigcommitlog|Commit Log gets very big. Cassandra does not delete old commit logs. Why?]] * [[#seed|What are seeds?]] * [[#seed_spof|Does single seed mean single point of failure?]] * [[#jconsole_array_arg|Why can't I call jmx method X on jconsole? (ex. getNaturalEndpoints)]] @@ -48, +43 @@ * [[#dropped_messages|Why do I see ... messages dropped.. in the logs?]] * [[#cli_keys|Why does the 0.8 cli not assume keys are strings anymore?]] * [[#memlock|Cassandra dies with java.lang.OutOfMemoryError: Map failed]] + * [[#opp|Why should I avoid order-preserving partitioners]] Anchor(cant_listen_on_ip_any) @@ -84, +80 @@ Unless you know precisely what you're doing and are aware of how the Cassandra internals work you should never introduce a new empty node to your cluster and have autoboostrap disabled. In version 0.7 under write load it will cause writes to be sent to the new node before the schema arrives from another member of the cluster. This would also indicate to clients that the new node is responsible for servicing reads for data that it definitely doesn't have. In Cassandra 0.4 and below, it is recommended that you manually specify a value for InitialToken in the config file of a new node. - - Anchor(modify_cf_config) - - == Can I add/remove/rename Column Families on a working cluster? == - Yes, but it's important that you do it correctly. For Cassandra 0.7 and newer use cassandra-cli. - - For Cassandra versions before 0.7: - - 1. Empty the commitlog with nodetool drain. - 1. Shutdown Cassandra and verify that there is no remaining data in the commitlog. - 1. Delete the sstable files (-Data.db, -Index.db, and -Filter.db) for any CFs removed, and rename the files for any CFs that were renamed. - 1. Make necessary changes to your storage-conf.xml. - 1. Start Cassandra back up and your edits should take effect. - - ''see also: [[https://issues.apache.org/jira/browse/CASSANDRA-44|CASSANDRA-44]]'' Anchor(node_clients_connect_to) @@ -222, +203 @@ == I delete data from Cassandra, but disk usage stays the same. What gives? == Data you write to Cassandra gets persisted to
[jira] [Resolved] (CASSANDRA-4769) Prevent parallel hint delivery to the node
[ https://issues.apache.org/jira/browse/CASSANDRA-4769?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis resolved CASSANDRA-4769. --- Resolution: Not A Problem Set your hint throttle level appropriately instead of trying to synchronize. Prevent parallel hint delivery to the node --- Key: CASSANDRA-4769 URL: https://issues.apache.org/jira/browse/CASSANDRA-4769 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.2 Reporter: Alexey Zotov It's actual only in case of the using a big enough cluster. After node's failure other nodes try to send hints to the restored node. So theoretically it can affect performance of restored node. I suggest to create some mechanism for synchronization of hints delivery processes to restored node. Could you please explain how it can be implemented. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4351) Consider storing more informations on peers in system tables
[ https://issues.apache.org/jira/browse/CASSANDRA-4351?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470442#comment-13470442 ] Jonathan Ellis commented on CASSANDRA-4351: --- No, I did mean instead, since then the human-readable token as seen in nodetool is the same as what you'd get from cqlsh. Consider storing more informations on peers in system tables - Key: CASSANDRA-4351 URL: https://issues.apache.org/jira/browse/CASSANDRA-4351 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Sylvain Lebresne Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 0001-4351.txt, 0002-Save-tokens-as-strings.txt Currently, the only thing we keep in system tables about other peers is their token and IP addresses. We should probably also record the new ring_id, but since CASSANDRA-4018 makes system table easily queriable, may it could be worth adding some more information (basically most of what we gossip could be a candidate (schema UUID, status, C* version, ...)) as a simple way to expose the ring state to users (even if it's just a view of the ring state from one specific node I believe it's still nice). Of course that means storing information that may not be absolutely needed by the server, but I'm not sure there is much harm to that. Note that doing this cleanly may require changing the schema of current system tables but as long as we do that in the 1.2 timeframe it's ok (since the concerned system table 'local' and 'peers' are news anyway). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4770) (CQL3) data type not in lowercase are not handled correctly.
[ https://issues.apache.org/jira/browse/CASSANDRA-4770?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470443#comment-13470443 ] Jonathan Ellis commented on CASSANDRA-4770: --- +1 (CQL3) data type not in lowercase are not handled correctly. Key: CASSANDRA-4770 URL: https://issues.apache.org/jira/browse/CASSANDRA-4770 Project: Cassandra Issue Type: Bug Affects Versions: 1.1.0 Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Priority: Minor Labels: cql3 Fix For: 1.1.6 Attachments: 4770.txt Seems that we accept {{int}} but we don't accept {{INT}} (that is, the parser accepts it, but we fail later to recognize it). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
git commit: clean up unnecessary copy on KeyCache lookup patch by jbellis; reviewed by yukim for CASSANDRA-4732
Updated Branches: refs/heads/trunk e232407ad - e3f642b8c clean up unnecessary copy on KeyCache lookup patch by jbellis; reviewed by yukim for CASSANDRA-4732 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e3f642b8 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e3f642b8 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e3f642b8 Branch: refs/heads/trunk Commit: e3f642b8c50db3af3f398d40dbad26f4c417eb16 Parents: e232407 Author: Jonathan Ellis jbel...@apache.org Authored: Fri Oct 5 11:47:18 2012 -0500 Committer: Jonathan Ellis jbel...@apache.org Committed: Fri Oct 5 11:47:54 2012 -0500 -- .../org/apache/cassandra/cache/KeyCacheKey.java|3 ++ .../apache/cassandra/io/sstable/SSTableReader.java | 19 +++ 2 files changed, 12 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3f642b8/src/java/org/apache/cassandra/cache/KeyCacheKey.java -- diff --git a/src/java/org/apache/cassandra/cache/KeyCacheKey.java b/src/java/org/apache/cassandra/cache/KeyCacheKey.java index ef0db40..509599b 100644 --- a/src/java/org/apache/cassandra/cache/KeyCacheKey.java +++ b/src/java/org/apache/cassandra/cache/KeyCacheKey.java @@ -28,6 +28,9 @@ import org.apache.cassandra.utils.Pair; public class KeyCacheKey implements CacheKey { public final Descriptor desc; + +// keeping an array instead of a ByteBuffer lowers the overhead of the key cache working set, +// without extra copies on lookup since client-provided key ByteBuffers will be array-backed already public final byte[] key; public KeyCacheKey(Descriptor desc, ByteBuffer key) http://git-wip-us.apache.org/repos/asf/cassandra/blob/e3f642b8/src/java/org/apache/cassandra/io/sstable/SSTableReader.java -- diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java index a67c1ab..aee576e 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java @@ -84,7 +84,7 @@ public class SSTableReader extends SSTable private IndexSummary indexSummary; private Filter bf; -private InstrumentingCacheKeyCacheKey, RowIndexEntry keyCache; +private final InstrumentingCacheKeyCacheKey, RowIndexEntry keyCache = CacheService.instance.keyCache; private final BloomFilterTracker bloomFilterTracker = new BloomFilterTracker(); @@ -311,7 +311,6 @@ public class SSTableReader extends SSTable public void setTrackedBy(DataTracker tracker) { -keyCache = CacheService.instance.keyCache; deletingTask.setTracker(tracker); } @@ -679,14 +678,14 @@ public class SSTableReader extends SSTable { CFMetaData.Caching caching = metadata.getCaching(); -if (keyCache == null -|| caching == CFMetaData.Caching.NONE -|| caching == CFMetaData.Caching.ROWS_ONLY -|| keyCache.getCapacity() == 0) +if (caching == CFMetaData.Caching.NONE +|| caching == CFMetaData.Caching.ROWS_ONLY +|| keyCache.getCapacity() == 0) +{ return; +} -// avoid keeping a permanent reference to the original key buffer -KeyCacheKey cacheKey = new KeyCacheKey(descriptor, ByteBufferUtil.clone(key.key)); +KeyCacheKey cacheKey = new KeyCacheKey(descriptor, key.key); logger.trace(Adding cache entry for {} - {}, cacheKey, info); keyCache.put(cacheKey, info); } @@ -698,7 +697,7 @@ public class SSTableReader extends SSTable private RowIndexEntry getCachedPosition(KeyCacheKey unifiedKey, boolean updateStats) { -if (keyCache != null keyCache.getCapacity() 0) +if (keyCache.getCapacity() 0) return updateStats ? keyCache.get(unifiedKey) : keyCache.getInternal(unifiedKey); return null; } @@ -798,7 +797,7 @@ public class SSTableReader extends SSTable { // read data position from index entry RowIndexEntry indexEntry = RowIndexEntry.serializer.deserialize(in, descriptor.version); -if (exactMatch keyCache != null keyCache.getCapacity() 0 updateCacheAndStats) +if (exactMatch updateCacheAndStats) { assert key instanceof DecoratedKey; // key can be == to the index key only if it's a true row key DecoratedKey decoratedKey = (DecoratedKey)key;
[jira] [Commented] (CASSANDRA-4351) Consider storing more informations on peers in system tables
[ https://issues.apache.org/jira/browse/CASSANDRA-4351?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470450#comment-13470450 ] Sylvain Lebresne commented on CASSANDRA-4351: - Oh I see. Was mislead by the fact that cqlsh don't print blob in hex currently. But if we fix that then yes, that'd be the same, at least for Murmur3Partitioner. Though I do note that Murmur3Partitioner is only an option for brand new clusters (i.e. almost no-one will have Murmur3Partitioner at first). But again, I don't care too much. I suppose that in the long run, with vnodes, the pretty printing of tokens won't be very useful anymore. Consider storing more informations on peers in system tables - Key: CASSANDRA-4351 URL: https://issues.apache.org/jira/browse/CASSANDRA-4351 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Sylvain Lebresne Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 0001-4351.txt, 0002-Save-tokens-as-strings.txt Currently, the only thing we keep in system tables about other peers is their token and IP addresses. We should probably also record the new ring_id, but since CASSANDRA-4018 makes system table easily queriable, may it could be worth adding some more information (basically most of what we gossip could be a candidate (schema UUID, status, C* version, ...)) as a simple way to expose the ring state to users (even if it's just a view of the ring state from one specific node I believe it's still nice). Of course that means storing information that may not be absolutely needed by the server, but I'm not sure there is much harm to that. Note that doing this cleanly may require changing the schema of current system tables but as long as we do that in the 1.2 timeframe it's ok (since the concerned system table 'local' and 'peers' are news anyway). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
git commit: Correctly accept upper-case data type (CQL3)
Updated Branches: refs/heads/cassandra-1.1 0d44159e6 - 6eafeb2b0 Correctly accept upper-case data type (CQL3) patch by slebresne; reviewed by jbellis for CASSANDRA-4770 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6eafeb2b Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6eafeb2b Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6eafeb2b Branch: refs/heads/cassandra-1.1 Commit: 6eafeb2b0147a923d66405a7f376242acab790d5 Parents: 0d44159 Author: Sylvain Lebresne sylv...@datastax.com Authored: Fri Oct 5 17:59:47 2012 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Fri Oct 5 18:52:07 2012 +0200 -- CHANGES.txt|1 + src/java/org/apache/cassandra/cql3/CFPropDefs.java |2 +- 2 files changed, 2 insertions(+), 1 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eafeb2b/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 218c320..c680f03 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -18,6 +18,7 @@ * (CQL3) Don't return ReversedType in result set metadata (CASSANDRA-4717) * Pluggable Thrift transport factories for CLI (CASSANDRA-4609) * Backport adding AlterKeyspace statement (CASSANDRA-4611) + * (CQL3) Correcty accept upper-case data types (CASSANDRA-4770) Merged from 1.0: * Switch from NBHM to CHM in MessagingService's callback map, which prevents OOM in long-running instances (CASSANDRA-4708) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eafeb2b/src/java/org/apache/cassandra/cql3/CFPropDefs.java -- diff --git a/src/java/org/apache/cassandra/cql3/CFPropDefs.java b/src/java/org/apache/cassandra/cql3/CFPropDefs.java index ab209dd..ec8bd5a 100644 --- a/src/java/org/apache/cassandra/cql3/CFPropDefs.java +++ b/src/java/org/apache/cassandra/cql3/CFPropDefs.java @@ -105,7 +105,7 @@ public class CFPropDefs extends PropertyDefinitions { try { -String className = comparators.get(type); +String className = comparators.get(type.toLowerCase()); if (className == null) className = type; return TypeParser.parse(className);
[2/2] git commit: Correctly accept upper-case data type (CQL3)
Correctly accept upper-case data type (CQL3) patch by slebresne; reviewed by jbellis for CASSANDRA-4770 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6eafeb2b Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6eafeb2b Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6eafeb2b Branch: refs/heads/trunk Commit: 6eafeb2b0147a923d66405a7f376242acab790d5 Parents: 0d44159 Author: Sylvain Lebresne sylv...@datastax.com Authored: Fri Oct 5 17:59:47 2012 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Fri Oct 5 18:52:07 2012 +0200 -- CHANGES.txt|1 + src/java/org/apache/cassandra/cql3/CFPropDefs.java |2 +- 2 files changed, 2 insertions(+), 1 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eafeb2b/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 218c320..c680f03 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -18,6 +18,7 @@ * (CQL3) Don't return ReversedType in result set metadata (CASSANDRA-4717) * Pluggable Thrift transport factories for CLI (CASSANDRA-4609) * Backport adding AlterKeyspace statement (CASSANDRA-4611) + * (CQL3) Correcty accept upper-case data types (CASSANDRA-4770) Merged from 1.0: * Switch from NBHM to CHM in MessagingService's callback map, which prevents OOM in long-running instances (CASSANDRA-4708) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eafeb2b/src/java/org/apache/cassandra/cql3/CFPropDefs.java -- diff --git a/src/java/org/apache/cassandra/cql3/CFPropDefs.java b/src/java/org/apache/cassandra/cql3/CFPropDefs.java index ab209dd..ec8bd5a 100644 --- a/src/java/org/apache/cassandra/cql3/CFPropDefs.java +++ b/src/java/org/apache/cassandra/cql3/CFPropDefs.java @@ -105,7 +105,7 @@ public class CFPropDefs extends PropertyDefinitions { try { -String className = comparators.get(type); +String className = comparators.get(type.toLowerCase()); if (className == null) className = type; return TypeParser.parse(className);
[1/2] git commit: Merge branch 'cassandra-1.1' into trunk
Updated Branches: refs/heads/trunk e3f642b8c - 1b56ad122 Merge branch 'cassandra-1.1' into trunk Conflicts: CHANGES.txt src/java/org/apache/cassandra/cql3/CFPropDefs.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/1b56ad12 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/1b56ad12 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/1b56ad12 Branch: refs/heads/trunk Commit: 1b56ad122ee966757865e8de589d1d93af3f3242 Parents: e3f642b 6eafeb2 Author: Sylvain Lebresne sylv...@datastax.com Authored: Fri Oct 5 18:55:48 2012 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Fri Oct 5 18:55:48 2012 +0200 -- CHANGES.txt |2 ++ 1 files changed, 2 insertions(+), 0 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b56ad12/CHANGES.txt --
[jira] [Commented] (CASSANDRA-4351) Consider storing more informations on peers in system tables
[ https://issues.apache.org/jira/browse/CASSANDRA-4351?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470456#comment-13470456 ] Jonathan Ellis commented on CASSANDRA-4351: --- You're right, 0002 is useful even if we change m3p, +1 Consider storing more informations on peers in system tables - Key: CASSANDRA-4351 URL: https://issues.apache.org/jira/browse/CASSANDRA-4351 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Sylvain Lebresne Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 0001-4351.txt, 0002-Save-tokens-as-strings.txt Currently, the only thing we keep in system tables about other peers is their token and IP addresses. We should probably also record the new ring_id, but since CASSANDRA-4018 makes system table easily queriable, may it could be worth adding some more information (basically most of what we gossip could be a candidate (schema UUID, status, C* version, ...)) as a simple way to expose the ring state to users (even if it's just a view of the ring state from one specific node I believe it's still nice). Of course that means storing information that may not be absolutely needed by the server, but I'm not sure there is much harm to that. Note that doing this cleanly may require changing the schema of current system tables but as long as we do that in the 1.2 timeframe it's ok (since the concerned system table 'local' and 'peers' are news anyway). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4705) Speculative execution for CL_ONE
[ https://issues.apache.org/jira/browse/CASSANDRA-4705?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470461#comment-13470461 ] Chris Burroughs commented on CASSANDRA-4705: Looks like metrics-core exposes 75, 95, 97, 99 and 99.9 Reporters have a limited set (ie you can't generate new values that will pop up in jmx on the fly), but in code you should be able to get at any percentile you want: https://github.com/codahale/metrics/blob/2.x-maintenance/metrics-core/src/main/java/com/yammer/metrics/stats/Snapshot.java#L54 Speculative execution for CL_ONE Key: CASSANDRA-4705 URL: https://issues.apache.org/jira/browse/CASSANDRA-4705 Project: Cassandra Issue Type: Improvement Affects Versions: 1.2.0 Reporter: Vijay Assignee: Vijay Priority: Minor Attachments: 0001-CASSANDRA-4705.patch When read_repair is not 1.0, we send the request to one node for some of the requests. When a node goes down or when a node is too busy the client has to wait for the timeout before it can retry. It would be nice to watch for latency and execute an additional request to a different node, if the response is not received within average/99% of the response times recorded in the past. CASSANDRA-2540 might be able to solve the variance when read_repair is set to 1.0 1) May be we need to use metrics-core to record various Percentiles 2) Modify ReadCallback.get to execute additional request speculatively. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
git commit: Save tokens as string in local and peers system table
Updated Branches: refs/heads/trunk 1b56ad122 - c2a8f1288 Save tokens as string in local and peers system table patch by slebresne; reviewed by jbellis for CASSANDRA-4351 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c2a8f128 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c2a8f128 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c2a8f128 Branch: refs/heads/trunk Commit: c2a8f1288f8985cdec374c88a063ac84cec60181 Parents: 1b56ad1 Author: Sylvain Lebresne sylv...@datastax.com Authored: Fri Oct 5 19:03:26 2012 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Fri Oct 5 19:03:26 2012 +0200 -- .../org/apache/cassandra/config/CFMetaData.java|4 ++-- src/java/org/apache/cassandra/db/SystemTable.java | 15 --- 2 files changed, 10 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2a8f128/src/java/org/apache/cassandra/config/CFMetaData.java -- diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java index ef25d2a..176d63a 100644 --- a/src/java/org/apache/cassandra/config/CFMetaData.java +++ b/src/java/org/apache/cassandra/config/CFMetaData.java @@ -160,7 +160,7 @@ public final class CFMetaData public static final CFMetaData PeersCf = compile(12, CREATE TABLE + SystemTable.PEERS_CF + ( + peer inet PRIMARY KEY, + ring_id uuid, - + tokens setblob, + + tokens setvarchar, + schema_version uuid, + release_version text, + rpc_address inet, @@ -170,7 +170,7 @@ public final class CFMetaData public static final CFMetaData LocalCf = compile(13, CREATE TABLE + SystemTable.LOCAL_CF + ( + key text PRIMARY KEY, - + tokens setblob, + + tokens setvarchar, + cluster_name text, + gossip_generation int, + bootstrapped text, http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2a8f128/src/java/org/apache/cassandra/db/SystemTable.java -- diff --git a/src/java/org/apache/cassandra/db/SystemTable.java b/src/java/org/apache/cassandra/db/SystemTable.java index ff7d81a..2883e0b 100644 --- a/src/java/org/apache/cassandra/db/SystemTable.java +++ b/src/java/org/apache/cassandra/db/SystemTable.java @@ -41,6 +41,7 @@ import org.apache.cassandra.db.filter.QueryFilter; import org.apache.cassandra.db.filter.QueryPath; import org.apache.cassandra.db.marshal.AsciiType; import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; @@ -208,7 +209,7 @@ public class SystemTable IteratorToken iter = tokens.iterator(); while (iter.hasNext()) { - sb.append(').append(ByteBufferUtil.bytesToHex(factory.toByteArray(iter.next(.append('); +sb.append(').append(factory.toString(iter.next())).append('); if (iter.hasNext()) sb.append(,); } @@ -216,12 +217,12 @@ public class SystemTable return sb.toString(); } -private static CollectionToken deserializeTokens(CollectionByteBuffer tokensBytes) +private static CollectionToken deserializeTokens(CollectionString tokensStrings) { Token.TokenFactory factory = StorageService.getPartitioner().getTokenFactory(); -ListToken tokens = new ArrayListToken(tokensBytes.size()); -for (ByteBuffer tk : tokensBytes) -tokens.add(factory.fromByteArray(tk)); +ListToken tokens = new ArrayListToken(tokensStrings.size()); +for (String tk : tokensStrings) +tokens.add(factory.fromString(tk)); return tokens; } @@ -296,7 +297,7 @@ public class SystemTable { InetAddress peer = row.getInetAddress(peer); if (row.has(tokens)) -tokenMap.putAll(peer,
[jira] [Resolved] (CASSANDRA-4351) Consider storing more informations on peers in system tables
[ https://issues.apache.org/jira/browse/CASSANDRA-4351?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne resolved CASSANDRA-4351. - Resolution: Fixed Reviewer: jbellis Assignee: Sylvain Lebresne Alright, 0002 committed. I don't know if we want to bother with the change to M3P then, but we can always do that in another ticket in any case, so closing that one now. Consider storing more informations on peers in system tables - Key: CASSANDRA-4351 URL: https://issues.apache.org/jira/browse/CASSANDRA-4351 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 0001-4351.txt, 0002-Save-tokens-as-strings.txt Currently, the only thing we keep in system tables about other peers is their token and IP addresses. We should probably also record the new ring_id, but since CASSANDRA-4018 makes system table easily queriable, may it could be worth adding some more information (basically most of what we gossip could be a candidate (schema UUID, status, C* version, ...)) as a simple way to expose the ring state to users (even if it's just a view of the ring state from one specific node I believe it's still nice). Of course that means storing information that may not be absolutely needed by the server, but I'm not sure there is much harm to that. Note that doing this cleanly may require changing the schema of current system tables but as long as we do that in the 1.2 timeframe it's ok (since the concerned system table 'local' and 'peers' are news anyway). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4705) Speculative execution for CL_ONE
[ https://issues.apache.org/jira/browse/CASSANDRA-4705?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470470#comment-13470470 ] Jonathan Ellis commented on CASSANDRA-4705: --- Thanks Chris! Speculative execution for CL_ONE Key: CASSANDRA-4705 URL: https://issues.apache.org/jira/browse/CASSANDRA-4705 Project: Cassandra Issue Type: Improvement Affects Versions: 1.2.0 Reporter: Vijay Assignee: Vijay Priority: Minor Attachments: 0001-CASSANDRA-4705.patch When read_repair is not 1.0, we send the request to one node for some of the requests. When a node goes down or when a node is too busy the client has to wait for the timeout before it can retry. It would be nice to watch for latency and execute an additional request to a different node, if the response is not received within average/99% of the response times recorded in the past. CASSANDRA-2540 might be able to solve the variance when read_repair is set to 1.0 1) May be we need to use metrics-core to record various Percentiles 2) Modify ReadCallback.get to execute additional request speculatively. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4684) Binary protocol: inform clients of schema changes
[ https://issues.apache.org/jira/browse/CASSANDRA-4684?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470487#comment-13470487 ] Jonathan Ellis commented on CASSANDRA-4684: --- All right, +1 Binary protocol: inform clients of schema changes - Key: CASSANDRA-4684 URL: https://issues.apache.org/jira/browse/CASSANDRA-4684 Project: Cassandra Issue Type: Improvement Affects Versions: 1.2.0 beta 1 Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 0001-Return-schema-change-infos.txt, 0002-Add-migration-events.txt It would be nice to inform clients when a schema change occurs as this would allow said client to maintain the current state of the schema, which might be useful/desirable. To allow that, we can: # return that a query has changed the schema (instead of simply a 'void' return), in the same spirit than CASSANDRA-3707. # add events notification on schema change. Just to be clear, the goal is only to inform that a change has occured, the client would still have to query the system table to know the exact content of the change, but at least it'll know when to do such query. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
git commit: Adds binary protocol events for schema changes
Updated Branches: refs/heads/trunk c2a8f1288 - 11f7d7253 Adds binary protocol events for schema changes patch by slebresne; reviewed by jbellis for CASSANDRA-4684 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/11f7d725 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/11f7d725 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/11f7d725 Branch: refs/heads/trunk Commit: 11f7d72536c3acf2ecdb255ddbcee545f29d6742 Parents: c2a8f12 Author: Sylvain Lebresne sylv...@datastax.com Authored: Fri Oct 5 19:46:10 2012 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Fri Oct 5 19:46:10 2012 +0200 -- CHANGES.txt|1 + doc/native_protocol.spec | 28 +- .../cql3/statements/AlterKeyspaceStatement.java| 12 +++ .../cql3/statements/AlterTableStatement.java |5 + .../statements/CreateColumnFamilyStatement.java|6 + .../cql3/statements/CreateIndexStatement.java |7 ++ .../cql3/statements/CreateKeyspaceStatement.java | 12 +++ .../cql3/statements/DropColumnFamilyStatement.java |6 + .../cql3/statements/DropIndexStatement.java|7 ++ .../cql3/statements/DropKeyspaceStatement.java | 12 +++ .../cql3/statements/SchemaAlteringStatement.java |5 +- src/java/org/apache/cassandra/db/DefsTable.java| 16 +++ .../cassandra/service/IMigrationListener.java | 30 ++ .../apache/cassandra/service/MigrationManager.java | 53 ++ .../apache/cassandra/service/StorageService.java |2 +- src/java/org/apache/cassandra/transport/Event.java | 48 +- .../org/apache/cassandra/transport/Server.java | 38 +++- .../transport/messages/ResultMessage.java | 74 ++- 18 files changed, 355 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/11f7d725/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 7cfc311..cab1425 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -113,6 +113,7 @@ * Pluggable Thrift transport factories for CLI (CASSANDRA-4609) * Backport adding AlterKeyspace statement (CASSANDRA-4611) * (CQL3) Correcty accept upper-case data types (CASSANDRA-4770) + * Add binary protocol events for schema changes (CASSANDRA-4684) Merged from 1.0: * Switch from NBHM to CHM in MessagingService's callback map, which prevents OOM in long-running instances (CASSANDRA-4708) http://git-wip-us.apache.org/repos/asf/cassandra/blob/11f7d725/doc/native_protocol.spec -- diff --git a/doc/native_protocol.spec b/doc/native_protocol.spec index 2e03a02..71a7c71 100644 --- a/doc/native_protocol.spec +++ b/doc/native_protocol.spec @@ -31,6 +31,7 @@ Table of Contents 4.2.5.2. Rows 4.2.5.3. Set_keyspace 4.2.5.4. Prepared +4.2.5.5. Schema_change 4.2.6. EVENT 5. Compression 6. Collection types @@ -328,7 +329,8 @@ Table of Contents 0x0001Void: for results carrying no information. 0x0002Rows: for results to select queries, returning a set of rows. 0x0003Set_keyspace: the result to a `use` query. -0x0004Prepared: result to a PREPARE message +0x0004Prepared: result to a PREPARE message. +0x0005Schema_change: the result to a schema altering query. The body for each kind (after the [int] kind) is defined below. @@ -416,6 +418,24 @@ Table of Contents - id is [short bytes] representing the prepared query ID. - metadata is defined exactly as for a Rows RESULT (See section 4.2.5.2). +4.2.5.5. Schema_change + + The result to a schema altering query (creation/update/drop of a + keyspace/table/index). The body (after the kind [int]) is composed of 3 + [string]: +changekeyspacetable + where: +- change describe the type of change that has occured. It can be one of + CREATED, UPDATED or DROPPED. +- keyspace is the name of the affected keyspace or the keyspace of the + affected table. +- table is the name of the affected table. table will be empty (i.e. + the empty string ) if the change was affecting a keyspace and not a + table. + + Note that queries to create and drop an index are considered as change + updating the table the index is on. + 4.2.6. EVENT @@ -434,6 +454,12 @@ Table of Contents consists of a [string] and an [inet], corresponding respectively to the type of status change (UP or DOWN) followed by the address of the concerned node. +- SCHEMA_CHANGE: events related to schema change. The body of the message + (after the event type) consists of
[jira] [Commented] (CASSANDRA-4687) Exception: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk)
[ https://issues.apache.org/jira/browse/CASSANDRA-4687?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470512#comment-13470512 ] T Jake Luciani commented on CASSANDRA-4687: --- I am seeing something similar here. {code} ERROR [ReadStage:70] 2012-10-05 13:26:40,215 AbstractCassandraDaemon.java (line 135) Exception in thread Thread[ReadStage:70,5,main] java.io.IOError: java.io.EOFException: unable to seek to position 165713136 in /opt/data/cassandra/data/prod/market_data_state/prod-market_data_state-he-5-Data.db (74622511 bytes) in read-only mode at org.apache.cassandra.io.util.CompressedSegmentedFile.getSegment(CompressedSegmentedFile.java:69) at org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:898) at org.apache.cassandra.db.columniterator.SSTableSliceIterator.init(SSTableSliceIterator.java:50) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1345) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1207) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1142) at org.apache.cassandra.db.Table.getRow(Table.java:378) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69) at org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:51) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) Caused by: java.io.EOFException: unable to seek to position 165713136 in /opt/data/cassandra/data/prod/market_data_state/prod-market_data_state-he-5-Data.db (74622511 bytes) in read-only mode at org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:253) at org.apache.cassandra.io.util.CompressedSegmentedFile.getSegment(CompressedSegmentedFile.java:64) ... 16 more {code} It seems to be hopping SSTables in compaction for the same row I'm trying to access so could be serialization issue. Exception: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk) --- Key: CASSANDRA-4687 URL: https://issues.apache.org/jira/browse/CASSANDRA-4687 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.5 Environment: CentOS 6.3 64-bit, Oracle JRE 1.6.0.33 64-bit, single node cluster Reporter: Leonid Shalupov Assignee: Pavel Yaskevich Priority: Critical Fix For: 1.1.6 Attachments: 4687-debugging.txt Under heavy write load sometimes cassandra fails with assertion error. git bisect leads to commit 295aedb278e7a495213241b66bc46d763fd4ce66. works fine if global key/row caches disabled in code. {quote} java.lang.AssertionError: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk) in /var/lib/cassandra/data/...-he-1-Data.db at org.apache.cassandra.db.columniterator.SSTableSliceIterator.init(SSTableSliceIterator.java:60) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1345) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1207) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1142) at org.apache.cassandra.db.Table.getRow(Table.java:378) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69) at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:819) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1253) at
git commit: Fix incorrect call of serializedSize
Updated Branches: refs/heads/trunk 11f7d7253 - 074f4befa Fix incorrect call of serializedSize Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/074f4bef Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/074f4bef Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/074f4bef Branch: refs/heads/trunk Commit: 074f4befaa0e5b7a7a3bf8dbaabdc3388ff61f8e Parents: 11f7d72 Author: Yuki Morishita yu...@apache.org Authored: Fri Oct 5 13:20:19 2012 -0500 Committer: Yuki Morishita yu...@apache.org Committed: Fri Oct 5 13:20:19 2012 -0500 -- .../org/apache/cassandra/db/RangeTombstone.java|3 +-- 1 files changed, 1 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/074f4bef/src/java/org/apache/cassandra/db/RangeTombstone.java -- diff --git a/src/java/org/apache/cassandra/db/RangeTombstone.java b/src/java/org/apache/cassandra/db/RangeTombstone.java index b8ff027..1b85c59 100644 --- a/src/java/org/apache/cassandra/db/RangeTombstone.java +++ b/src/java/org/apache/cassandra/db/RangeTombstone.java @@ -170,10 +170,9 @@ public class RangeTombstone extends IntervalByteBuffer, DeletionTime implement toWrite.add(tombstone); } -TypeSizes typeSizes = TypeSizes.NATIVE; for (RangeTombstone tombstone : toWrite) { -size += tombstone.serializedSize(typeSizes); +size += tombstone.serializedSizeForSSTable(); atomCount++; if (out != null) atomSerializer.serializeForSSTable(tombstone, out);
[jira] [Commented] (CASSANDRA-4687) Exception: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk)
[ https://issues.apache.org/jira/browse/CASSANDRA-4687?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470517#comment-13470517 ] T Jake Luciani commented on CASSANDRA-4687: --- On Pavel's advice I disabled the key cache and it resolved the issue for me. Exception: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk) --- Key: CASSANDRA-4687 URL: https://issues.apache.org/jira/browse/CASSANDRA-4687 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.5 Environment: CentOS 6.3 64-bit, Oracle JRE 1.6.0.33 64-bit, single node cluster Reporter: Leonid Shalupov Assignee: Pavel Yaskevich Priority: Critical Fix For: 1.1.6 Attachments: 4687-debugging.txt Under heavy write load sometimes cassandra fails with assertion error. git bisect leads to commit 295aedb278e7a495213241b66bc46d763fd4ce66. works fine if global key/row caches disabled in code. {quote} java.lang.AssertionError: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk) in /var/lib/cassandra/data/...-he-1-Data.db at org.apache.cassandra.db.columniterator.SSTableSliceIterator.init(SSTableSliceIterator.java:60) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1345) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1207) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1142) at org.apache.cassandra.db.Table.getRow(Table.java:378) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69) at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:819) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1253) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) {quote} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
buildbot success in ASF Buildbot on cassandra-trunk
The Buildbot has detected a restored build on builder cassandra-trunk while building cassandra. Full details are available at: http://ci.apache.org/builders/cassandra-trunk/builds/1909 Buildbot URL: http://ci.apache.org/ Buildslave for this Build: portunus_ubuntu Build Reason: scheduler Build Source Stamp: [branch trunk] 074f4befaa0e5b7a7a3bf8dbaabdc3388ff61f8e Blamelist: Yuki Morishita yu...@apache.org Build succeeded! sincerely, -The Buildbot
[jira] [Commented] (CASSANDRA-4687) Exception: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk)
[ https://issues.apache.org/jira/browse/CASSANDRA-4687?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470531#comment-13470531 ] T Jake Luciani commented on CASSANDRA-4687: --- I've been loading data so it's 99% writes. I found on the ML that someone else hit this and was bulk loading data... http://mail-archives.apache.org/mod_mbox/cassandra-user/201208.mbox/%3c95ecc1abc58041c8a94503285e016...@ntoklo.com%3E I am not using the bulk loader though... Exception: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk) --- Key: CASSANDRA-4687 URL: https://issues.apache.org/jira/browse/CASSANDRA-4687 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.5 Environment: CentOS 6.3 64-bit, Oracle JRE 1.6.0.33 64-bit, single node cluster Reporter: Leonid Shalupov Assignee: Pavel Yaskevich Priority: Critical Fix For: 1.1.6 Attachments: 4687-debugging.txt Under heavy write load sometimes cassandra fails with assertion error. git bisect leads to commit 295aedb278e7a495213241b66bc46d763fd4ce66. works fine if global key/row caches disabled in code. {quote} java.lang.AssertionError: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk) in /var/lib/cassandra/data/...-he-1-Data.db at org.apache.cassandra.db.columniterator.SSTableSliceIterator.init(SSTableSliceIterator.java:60) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1345) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1207) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1142) at org.apache.cassandra.db.Table.getRow(Table.java:378) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69) at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:819) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1253) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) {quote} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4769) Prevent parallel hint delivery to the node
[ https://issues.apache.org/jira/browse/CASSANDRA-4769?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470583#comment-13470583 ] Alexey Zotov commented on CASSANDRA-4769: - Yes, it's the simplest way. But what's about the following use case: There are N remote data centers. Each data center sends hints to restored node. So that node will receive throttle_in_kb*N*Nrf of data per second. (Nrf - replication factor in Nth data center) You could reduce throttle threshold, but in case of local node very small threshold is not a very good idea. I think that mechanism shoud be more configurable. Prevent parallel hint delivery to the node --- Key: CASSANDRA-4769 URL: https://issues.apache.org/jira/browse/CASSANDRA-4769 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.2 Reporter: Alexey Zotov It's actual only in case of the using a big enough cluster. After node's failure other nodes try to send hints to the restored node. So theoretically it can affect performance of restored node. I suggest to create some mechanism for synchronization of hints delivery processes to restored node. Could you please explain how it can be implemented. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (CASSANDRA-4771) Setting TTL to Integer.MAX causes columns to not be persisted.
Todd Nine created CASSANDRA-4771: Summary: Setting TTL to Integer.MAX causes columns to not be persisted. Key: CASSANDRA-4771 URL: https://issues.apache.org/jira/browse/CASSANDRA-4771 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.0.12 Reporter: Todd Nine Priority: Blocker When inserting columns via batch mutation, we have an edge case where columns will be set to Integer.MAX. When setting the column expiration time to Integer.MAX, the columns do not appear to be persisted. Fails: Integer.MAX Integer.MAX/2 Works: Integer.MAX/3 -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4687) Exception: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk)
[ https://issues.apache.org/jira/browse/CASSANDRA-4687?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470596#comment-13470596 ] T Jake Luciani commented on CASSANDRA-4687: --- One other thought. This only happened on one of the nodes and I think that was the one node that had restarted Cassandra. So could possibly be the saved cache. This was the one row I was searching previously before it broke. Exception: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk) --- Key: CASSANDRA-4687 URL: https://issues.apache.org/jira/browse/CASSANDRA-4687 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.5 Environment: CentOS 6.3 64-bit, Oracle JRE 1.6.0.33 64-bit, single node cluster Reporter: Leonid Shalupov Assignee: Pavel Yaskevich Priority: Critical Fix For: 1.1.6 Attachments: 4687-debugging.txt Under heavy write load sometimes cassandra fails with assertion error. git bisect leads to commit 295aedb278e7a495213241b66bc46d763fd4ce66. works fine if global key/row caches disabled in code. {quote} java.lang.AssertionError: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk) in /var/lib/cassandra/data/...-he-1-Data.db at org.apache.cassandra.db.columniterator.SSTableSliceIterator.init(SSTableSliceIterator.java:60) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1345) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1207) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1142) at org.apache.cassandra.db.Table.getRow(Table.java:378) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69) at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:819) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1253) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) {quote} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4769) Prevent parallel hint delivery to the node
[ https://issues.apache.org/jira/browse/CASSANDRA-4769?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470603#comment-13470603 ] Jonathan Ellis commented on CASSANDRA-4769: --- Hints are per node, not per datacenter. Prevent parallel hint delivery to the node --- Key: CASSANDRA-4769 URL: https://issues.apache.org/jira/browse/CASSANDRA-4769 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.2 Reporter: Alexey Zotov It's actual only in case of the using a big enough cluster. After node's failure other nodes try to send hints to the restored node. So theoretically it can affect performance of restored node. I suggest to create some mechanism for synchronization of hints delivery processes to restored node. Could you please explain how it can be implemented. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4768) Add separate max_hint_window_in_ms option for remote data centers
[ https://issues.apache.org/jira/browse/CASSANDRA-4768?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470628#comment-13470628 ] Alexey Zotov commented on CASSANDRA-4768: - I'll try to explain my motivation. There is the following use case: We have a few DCs. One of DCs has been lost connection to others. So we should store hints for all ranges of data (for all data set) for that DC. Write's throughput is very high. Amount of stored hints for the lost remote DC grows very quickly. We can reduce max_hint_window_size, but we want to keep repair mechanism for local nodes. Add separate max_hint_window_in_ms option for remote data centers - Key: CASSANDRA-4768 URL: https://issues.apache.org/jira/browse/CASSANDRA-4768 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.2 Reporter: Alexey Zotov Assignee: Alexey Zotov Priority: Minor Labels: configuration, hintedhandoff Fix For: 1.1.6, 1.2.0 Attachments: cassandra-1.1-4768-remote_hint_window.txt, cassandra-1.2-4768-remote_hint_window.txt It would be nice to have possibility to configure hint window size for remote dc separately. It will allow to prevent accumulating of big amount of data for remote dc and long hints delivery as the result of it. I suggest to add max_hint_window_for_remote_dc_in_ms option. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4157) Allow KS + CF names up to 48 characters
[ https://issues.apache.org/jira/browse/CASSANDRA-4157?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470654#comment-13470654 ] Chris Chiappone commented on CASSANDRA-4157: This actually causes a bunch of problems for customers that have used longer the 48 character keyspace and cf names. How will this affect the upgrade path for those customers? Allow KS + CF names up to 48 characters --- Key: CASSANDRA-4157 URL: https://issues.apache.org/jira/browse/CASSANDRA-4157 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.0 Reporter: Jonathan Ellis Assignee: Jonathan Ellis Priority: Minor Fix For: 1.1.0 Attachments: 4157.txt CASSANDRA-2749 imposed a 32-character limit on KS and CF names. We can be a little more lenient than that and still be safe for path names (see CASSANDRA-4110). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-2749) fine-grained control over data directories
[ https://issues.apache.org/jira/browse/CASSANDRA-2749?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470655#comment-13470655 ] Chris Chiappone commented on CASSANDRA-2749: This actually causes a bunch of problems for customers that have used longer the 48 character keyspace and cf names. How will this affect the upgrade path for those customers? fine-grained control over data directories -- Key: CASSANDRA-2749 URL: https://issues.apache.org/jira/browse/CASSANDRA-2749 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Assignee: Sylvain Lebresne Priority: Minor Fix For: 1.1.0 Attachments: 0001-2749.patch, 0001-Make-it-possible-to-put-column-families-in-subdirect.patch, 0001-non-backwards-compatible-patch-for-2749-putting-cfs-.patch.gz, 0002-fix-unit-tests.patch, 0003-Fixes.patch, 2749_backwards_compatible_v1.patch, 2749_backwards_compatible_v2.patch, 2749_backwards_compatible_v3.patch, 2749_backwards_compatible_v4.patch, 2749_backwards_compatible_v4_rebase1.patch, 2749_not_backwards.tar.gz, 2749_proper.tar.gz, 2749.tar.gz Currently Cassandra supports multiple data directories but no way to control what sstables are placed where. Particularly for systems with mixed SSDs and rotational disks, it would be nice to pin frequently accessed columnfamilies to the SSDs. Postgresql does this with tablespaces (http://www.postgresql.org/docs/9.0/static/manage-ag-tablespaces.html) but we should probably avoid using that name because of confusing similarity to keyspaces. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (CASSANDRA-4772) HintedHandoff fails to deliver hints after first repaired node
Alexey Zotov created CASSANDRA-4772: --- Summary: HintedHandoff fails to deliver hints after first repaired node Key: CASSANDRA-4772 URL: https://issues.apache.org/jira/browse/CASSANDRA-4772 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.4 Reporter: Alexey Zotov Assignee: Alexey Zotov Priority: Blocker Fix For: 1.2.0 beta 2 If some node has hints for a few nodes it will deliver hints only for the first one of them. After all hints delivery for the first node compaction process is started. After compaction all data from hints cf is removed. target fix for 1.2 version: {code} diff --git a/src/java/org/apache/cassandra/db/HintedHandOffManager.java b/src/java/org/apache/cassandra/db/HintedHandOffManager.java index e5ff163..c02997e 100644 --- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java +++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java @@ -189,7 +189,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean ArrayListDescriptor descriptors = new ArrayListDescriptor(); for (SSTable sstable : hintStore.getSSTables()) descriptors.add(sstable.descriptor); -return CompactionManager.instance.submitUserDefined(hintStore, descriptors, Integer.MAX_VALUE); +return CompactionManager.instance.submitUserDefined(hintStore, descriptors, (int) System.currentTimeMillis() / 1000); } private static boolean pagingFinished(ColumnFamily hintColumnFamily, ByteBuffer startColumn) {code} Can I expect to see that fix in 1.1.6 version? -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4157) Allow KS + CF names up to 48 characters
[ https://issues.apache.org/jira/browse/CASSANDRA-4157?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470670#comment-13470670 ] Jonathan Ellis commented on CASSANDRA-4157: --- They will need to fix that before upgrading. Do note that this restriction only applies to Windows deployments. Allow KS + CF names up to 48 characters --- Key: CASSANDRA-4157 URL: https://issues.apache.org/jira/browse/CASSANDRA-4157 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.0 Reporter: Jonathan Ellis Assignee: Jonathan Ellis Priority: Minor Fix For: 1.1.0 Attachments: 4157.txt CASSANDRA-2749 imposed a 32-character limit on KS and CF names. We can be a little more lenient than that and still be safe for path names (see CASSANDRA-4110). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4157) Allow KS + CF names up to 48 characters
[ https://issues.apache.org/jira/browse/CASSANDRA-4157?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470683#comment-13470683 ] Chris Chiappone commented on CASSANDRA-4157: We are attempting to upgrade and are using the DataStax latest Linux AMI. This restriction is present in Cassandra 1.1.5 on linux. Allow KS + CF names up to 48 characters --- Key: CASSANDRA-4157 URL: https://issues.apache.org/jira/browse/CASSANDRA-4157 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.0 Reporter: Jonathan Ellis Assignee: Jonathan Ellis Priority: Minor Fix For: 1.1.0 Attachments: 4157.txt CASSANDRA-2749 imposed a 32-character limit on KS and CF names. We can be a little more lenient than that and still be safe for path names (see CASSANDRA-4110). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4723) Improve write timeout exceptions
[ https://issues.apache.org/jira/browse/CASSANDRA-4723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470688#comment-13470688 ] Jonathan Ellis commented on CASSANDRA-4723: --- +1 Improve write timeout exceptions - Key: CASSANDRA-4723 URL: https://issues.apache.org/jira/browse/CASSANDRA-4723 Project: Cassandra Issue Type: Improvement Affects Versions: 1.2.0 beta 1 Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Priority: Minor Fix For: 1.2.0 beta 2 Attachments: 4723-alternative.txt, 4723.txt, 4723-v2.txt Through the binary protocol (and to a lesser extend in thrift), we now expose more information with a timeout, so that clients can take the right decision as far as retrying the operation is concerned. Concerning write timeouts, there is two places where I think we should improve that a bit: * regarding batchlog writes: what clients are interested in is to know if the option was atomic basically. If it was, there is no good reason to retry the write, otherwise, you should (or at least you know there might be inconsistencies if you don't). * we should return a separate exception for counter writes as in that case no retry should ever be done. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
git commit: Add ability to use custom TServerFactory implementations patch by Jason Brown; reviewed by Pavel Yaskevich for CASSANDRA-4608
Updated Branches: refs/heads/cassandra-1.1 6eafeb2b0 - 8264eb21c Add ability to use custom TServerFactory implementations patch by Jason Brown; reviewed by Pavel Yaskevich for CASSANDRA-4608 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8264eb21 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8264eb21 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8264eb21 Branch: refs/heads/cassandra-1.1 Commit: 8264eb21ccb20423ff7bdae0fbef6d88fe2b2529 Parents: 6eafeb2 Author: Pavel Yaskevich xe...@apache.org Authored: Fri Oct 5 14:11:13 2012 -0700 Committer: Pavel Yaskevich xe...@apache.org Committed: Fri Oct 5 14:12:35 2012 -0700 -- CHANGES.txt|1 + conf/cassandra.yaml|5 +- .../cassandra/config/DatabaseDescriptor.java |2 - .../apache/cassandra/thrift/CassandraDaemon.java | 120 ++- .../apache/cassandra/thrift/CustomTHsHaServer.java | 39 + .../cassandra/thrift/CustomTNonBlockingServer.java | 31 .../cassandra/thrift/CustomTThreadPoolServer.java | 32 .../cassandra/thrift/TServerCustomFactory.java | 75 + .../apache/cassandra/thrift/TServerFactory.java| 43 + 9 files changed, 241 insertions(+), 107 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/8264eb21/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index c680f03..4d2fd27 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -19,6 +19,7 @@ * Pluggable Thrift transport factories for CLI (CASSANDRA-4609) * Backport adding AlterKeyspace statement (CASSANDRA-4611) * (CQL3) Correcty accept upper-case data types (CASSANDRA-4770) + * Add ability to use custom TServerFactory implementations (CASSANDRA-4608) Merged from 1.0: * Switch from NBHM to CHM in MessagingService's callback map, which prevents OOM in long-running instances (CASSANDRA-4708) http://git-wip-us.apache.org/repos/asf/cassandra/blob/8264eb21/conf/cassandra.yaml -- diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index c4732db..5e0be98 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -287,7 +287,7 @@ rpc_port: 9160 # enable or disable keepalive on rpc connections rpc_keepalive: true -# Cassandra provides three options for the RPC Server: +# Cassandra provides three out-of-the-box options for the RPC Server: # # sync - One connection per thread in the rpc pool (see below). # For a very large number of clients, memory will be your limiting @@ -305,6 +305,9 @@ rpc_keepalive: true # # The default is sync because on Windows hsha is about 30% slower. On Linux, # sync/hsha performance is about the same, with hsha of course using less memory. +# +# Alternatively, can provide your own RPC server by providing the fully-qualified class name +# of an o.a.c.t.TServerFactory that can create an instance of it. rpc_server_type: sync # Uncomment rpc_min|max|thread to set request pool size. http://git-wip-us.apache.org/repos/asf/cassandra/blob/8264eb21/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 20fa981..7ed6170 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -381,8 +381,6 @@ public class DatabaseDescriptor if (conf.stream_throughput_outbound_megabits_per_sec == null) conf.stream_throughput_outbound_megabits_per_sec = 400; -if (!CassandraDaemon.rpc_server_types.contains(conf.rpc_server_type.toLowerCase())) -throw new ConfigurationException(Unknown rpc_server_type: + conf.rpc_server_type); if (conf.rpc_min_threads == null) conf.rpc_min_threads = conf.rpc_server_type.toLowerCase().equals(hsha) ? Runtime.getRuntime().availableProcessors() * 4 http://git-wip-us.apache.org/repos/asf/cassandra/blob/8264eb21/src/java/org/apache/cassandra/thrift/CassandraDaemon.java -- diff --git a/src/java/org/apache/cassandra/thrift/CassandraDaemon.java b/src/java/org/apache/cassandra/thrift/CassandraDaemon.java index 7153c08..2decb8e 100644 --- a/src/java/org/apache/cassandra/thrift/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/thrift/CassandraDaemon.java @@ -20,28 +20,14 @@ package
[1/2] git commit: merge from 1.1
Updated Branches: refs/heads/trunk 074f4befa - 6b83663ca merge from 1.1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6b83663c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6b83663c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6b83663c Branch: refs/heads/trunk Commit: 6b83663ca494123d87477f69205633497e4bef1d Parents: 074f4be 8264eb2 Author: Pavel Yaskevich xe...@apache.org Authored: Fri Oct 5 14:59:13 2012 -0700 Committer: Pavel Yaskevich xe...@apache.org Committed: Fri Oct 5 14:59:13 2012 -0700 -- CHANGES.txt|1 + conf/cassandra.yaml|5 +- .../cassandra/config/DatabaseDescriptor.java |2 - .../apache/cassandra/thrift/CustomTHsHaServer.java | 39 .../cassandra/thrift/CustomTNonBlockingServer.java | 31 +++ .../cassandra/thrift/CustomTThreadPoolServer.java | 63 ++ .../cassandra/thrift/TServerCustomFactory.java | 75 +++ .../apache/cassandra/thrift/TServerFactory.java| 43 .../org/apache/cassandra/thrift/ThriftServer.java | 150 ++- 9 files changed, 274 insertions(+), 135 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6b83663c/CHANGES.txt -- diff --cc CHANGES.txt index cab1425,4d2fd27..4fffbf7 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -113,7 -19,7 +113,8 @@@ * Pluggable Thrift transport factories for CLI (CASSANDRA-4609) * Backport adding AlterKeyspace statement (CASSANDRA-4611) * (CQL3) Correcty accept upper-case data types (CASSANDRA-4770) + * Add binary protocol events for schema changes (CASSANDRA-4684) + * Add ability to use custom TServerFactory implementations (CASSANDRA-4608) Merged from 1.0: * Switch from NBHM to CHM in MessagingService's callback map, which prevents OOM in long-running instances (CASSANDRA-4708) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6b83663c/conf/cassandra.yaml -- diff --cc conf/cassandra.yaml index f98f9f0,5e0be98..84c0ae5 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@@ -333,31 -287,39 +333,34 @@@ rpc_port: 916 # enable or disable keepalive on rpc connections rpc_keepalive: true - # Cassandra provides three options for the RPC Server: + # Cassandra provides three out-of-the-box options for the RPC Server: # -# sync - One connection per thread in the rpc pool (see below). -# For a very large number of clients, memory will be your limiting -# factor; on a 64 bit JVM, 128KB is the minimum stack size per thread. -# Connection pooling is very, very strongly recommended. -# -# async - Nonblocking server implementation with one thread to serve -# rpc connections. This is not recommended for high throughput use -# cases. Async has been tested to be about 50% slower than sync -# or hsha and is deprecated: it will be removed in the next major release. +# sync - One thread per thrift connection. For a very large number of clients, memory +# will be your limiting factor. On a 64 bit JVM, 128KB is the minimum stack size +# per thread, and that will correspond to your use of virtual memory (but physical memory +# may be limited depending on use of stack space). # -# hsha - Stands for half synchronous, half asynchronous. The rpc thread pool -# (see below) is used to manage requests, but the threads are multiplexed -# across the different clients. +# hsha - Stands for half synchronous, half asynchronous. All thrift clients are handled +# asynchronously using a small number of threads that does not vary with the amount +# of thrift clients (and thus scales well to many clients). The rpc requests are still +# synchronous (one thread per active request). # # The default is sync because on Windows hsha is about 30% slower. On Linux, # sync/hsha performance is about the same, with hsha of course using less memory. + # + # Alternatively, can provide your own RPC server by providing the fully-qualified class name + # of an o.a.c.t.TServerFactory that can create an instance of it. rpc_server_type: sync -# Uncomment rpc_min|max|thread to set request pool size. -# You would primarily set max for the sync server to safeguard against -# misbehaved clients; if you do hit the max, Cassandra will block until one -# disconnects before accepting more. The defaults for sync are min of 16 and max -# unlimited. -# -# For the Hsha server, the min and max both default to quadruple the number of -# CPU cores. +# Uncomment
[2/2] git commit: Add ability to use custom TServerFactory implementations patch by Jason Brown; reviewed by Pavel Yaskevich for CASSANDRA-4608
Add ability to use custom TServerFactory implementations patch by Jason Brown; reviewed by Pavel Yaskevich for CASSANDRA-4608 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8264eb21 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8264eb21 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8264eb21 Branch: refs/heads/trunk Commit: 8264eb21ccb20423ff7bdae0fbef6d88fe2b2529 Parents: 6eafeb2 Author: Pavel Yaskevich xe...@apache.org Authored: Fri Oct 5 14:11:13 2012 -0700 Committer: Pavel Yaskevich xe...@apache.org Committed: Fri Oct 5 14:12:35 2012 -0700 -- CHANGES.txt|1 + conf/cassandra.yaml|5 +- .../cassandra/config/DatabaseDescriptor.java |2 - .../apache/cassandra/thrift/CassandraDaemon.java | 120 ++- .../apache/cassandra/thrift/CustomTHsHaServer.java | 39 + .../cassandra/thrift/CustomTNonBlockingServer.java | 31 .../cassandra/thrift/CustomTThreadPoolServer.java | 32 .../cassandra/thrift/TServerCustomFactory.java | 75 + .../apache/cassandra/thrift/TServerFactory.java| 43 + 9 files changed, 241 insertions(+), 107 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/8264eb21/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index c680f03..4d2fd27 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -19,6 +19,7 @@ * Pluggable Thrift transport factories for CLI (CASSANDRA-4609) * Backport adding AlterKeyspace statement (CASSANDRA-4611) * (CQL3) Correcty accept upper-case data types (CASSANDRA-4770) + * Add ability to use custom TServerFactory implementations (CASSANDRA-4608) Merged from 1.0: * Switch from NBHM to CHM in MessagingService's callback map, which prevents OOM in long-running instances (CASSANDRA-4708) http://git-wip-us.apache.org/repos/asf/cassandra/blob/8264eb21/conf/cassandra.yaml -- diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index c4732db..5e0be98 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -287,7 +287,7 @@ rpc_port: 9160 # enable or disable keepalive on rpc connections rpc_keepalive: true -# Cassandra provides three options for the RPC Server: +# Cassandra provides three out-of-the-box options for the RPC Server: # # sync - One connection per thread in the rpc pool (see below). # For a very large number of clients, memory will be your limiting @@ -305,6 +305,9 @@ rpc_keepalive: true # # The default is sync because on Windows hsha is about 30% slower. On Linux, # sync/hsha performance is about the same, with hsha of course using less memory. +# +# Alternatively, can provide your own RPC server by providing the fully-qualified class name +# of an o.a.c.t.TServerFactory that can create an instance of it. rpc_server_type: sync # Uncomment rpc_min|max|thread to set request pool size. http://git-wip-us.apache.org/repos/asf/cassandra/blob/8264eb21/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 20fa981..7ed6170 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -381,8 +381,6 @@ public class DatabaseDescriptor if (conf.stream_throughput_outbound_megabits_per_sec == null) conf.stream_throughput_outbound_megabits_per_sec = 400; -if (!CassandraDaemon.rpc_server_types.contains(conf.rpc_server_type.toLowerCase())) -throw new ConfigurationException(Unknown rpc_server_type: + conf.rpc_server_type); if (conf.rpc_min_threads == null) conf.rpc_min_threads = conf.rpc_server_type.toLowerCase().equals(hsha) ? Runtime.getRuntime().availableProcessors() * 4 http://git-wip-us.apache.org/repos/asf/cassandra/blob/8264eb21/src/java/org/apache/cassandra/thrift/CassandraDaemon.java -- diff --git a/src/java/org/apache/cassandra/thrift/CassandraDaemon.java b/src/java/org/apache/cassandra/thrift/CassandraDaemon.java index 7153c08..2decb8e 100644 --- a/src/java/org/apache/cassandra/thrift/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/thrift/CassandraDaemon.java @@ -20,28 +20,14 @@ package org.apache.cassandra.thrift; import java.net.InetAddress; import java.net.InetSocketAddress;
[jira] [Updated] (CASSANDRA-4667) optimize memtable deletions for batchlog
[ https://issues.apache.org/jira/browse/CASSANDRA-4667?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-4667: -- Attachment: 4667-handle-empty-flushes.txt fix attached. optimize memtable deletions for batchlog Key: CASSANDRA-4667 URL: https://issues.apache.org/jira/browse/CASSANDRA-4667 Project: Cassandra Issue Type: Sub-task Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 1.2.0 beta 2 Attachments: 4667-handle-empty-flushes.txt, CASSANDRA-4667-v1.1.patch, CASSANDRA-4667-v2.patch, CASSANDRA-4667-v3.txt, CASSANDRA-4667-v4.txt Batchlog writes with the same key are never retried. This means that if a batchlog row is in the memtable, it can't be in any of the sstables, ever. In such cases we don't need to write a tombstone to disk. We can purge the row completely from the memtable and only write a tombstone if the row had been flushed already (if it's not in the memtable then it must be in one of the sstables). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4687) Exception: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk)
[ https://issues.apache.org/jira/browse/CASSANDRA-4687?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470724#comment-13470724 ] Pavel Yaskevich commented on CASSANDRA-4687: [~tpatterson] Can you try to test saved cache scenario? e.g. while inserting try reading (to simulate write-heavy environment), make sure that cache was saved (probably setting small save interval would be the best), restart the node, try reading data. Exception: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk) --- Key: CASSANDRA-4687 URL: https://issues.apache.org/jira/browse/CASSANDRA-4687 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.5 Environment: CentOS 6.3 64-bit, Oracle JRE 1.6.0.33 64-bit, single node cluster Reporter: Leonid Shalupov Assignee: Pavel Yaskevich Priority: Critical Fix For: 1.1.6 Attachments: 4687-debugging.txt Under heavy write load sometimes cassandra fails with assertion error. git bisect leads to commit 295aedb278e7a495213241b66bc46d763fd4ce66. works fine if global key/row caches disabled in code. {quote} java.lang.AssertionError: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk) in /var/lib/cassandra/data/...-he-1-Data.db at org.apache.cassandra.db.columniterator.SSTableSliceIterator.init(SSTableSliceIterator.java:60) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1345) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1207) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1142) at org.apache.cassandra.db.Table.getRow(Table.java:378) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69) at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:819) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1253) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) {quote} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4667) optimize memtable deletions for batchlog
[ https://issues.apache.org/jira/browse/CASSANDRA-4667?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470751#comment-13470751 ] Aleksey Yeschenko commented on CASSANDRA-4667: -- The patch works. +1 optimize memtable deletions for batchlog Key: CASSANDRA-4667 URL: https://issues.apache.org/jira/browse/CASSANDRA-4667 Project: Cassandra Issue Type: Sub-task Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 1.2.0 beta 2 Attachments: 4667-handle-empty-flushes.txt, CASSANDRA-4667-v1.1.patch, CASSANDRA-4667-v2.patch, CASSANDRA-4667-v3.txt, CASSANDRA-4667-v4.txt Batchlog writes with the same key are never retried. This means that if a batchlog row is in the memtable, it can't be in any of the sstables, ever. In such cases we don't need to write a tombstone to disk. We can purge the row completely from the memtable and only write a tombstone if the row had been flushed already (if it's not in the memtable then it must be in one of the sstables). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Comment Edited] (CASSANDRA-4667) optimize memtable deletions for batchlog
[ https://issues.apache.org/jira/browse/CASSANDRA-4667?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470751#comment-13470751 ] Aleksey Yeschenko edited comment on CASSANDRA-4667 at 10/5/12 10:47 PM: The fix works. +1 was (Author: iamaleksey): The patch works. +1 optimize memtable deletions for batchlog Key: CASSANDRA-4667 URL: https://issues.apache.org/jira/browse/CASSANDRA-4667 Project: Cassandra Issue Type: Sub-task Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 1.2.0 beta 2 Attachments: 4667-handle-empty-flushes.txt, CASSANDRA-4667-v1.1.patch, CASSANDRA-4667-v2.patch, CASSANDRA-4667-v3.txt, CASSANDRA-4667-v4.txt Batchlog writes with the same key are never retried. This means that if a batchlog row is in the memtable, it can't be in any of the sstables, ever. In such cases we don't need to write a tombstone to disk. We can purge the row completely from the memtable and only write a tombstone if the row had been flushed already (if it's not in the memtable then it must be in one of the sstables). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4050) cassandra unnecessarily holds file locks on snapshot files
[ https://issues.apache.org/jira/browse/CASSANDRA-4050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470759#comment-13470759 ] Alexander Kashirin commented on CASSANDRA-4050: --- I have the same problem with Cassandra 1.1.5 on Windows 7 and Windows Server 2008 R2: The nodetool stably creates snapshots that can't be deleted by the clearsnapshot command later (- IOException: Failed to delete ...). The bare facts are below. For some (not for all) snapshot files (= hard links): 1) WinExplorer-Delete says The process cannot access the file because it is being used by another process. 2) Command interpreter writes Access denied on del command. 3) Troubleshooting tools (- Process Explorer and Unlocker) do not find open file handles. Moreover, Unlocker is able to delete these files without problems. 4) The files become deletable by the rest tools just after Cassandra-server has been stopped. 5) After restart, the same files become locked again. 6) Everything repeats after computer has been restarted. cassandra unnecessarily holds file locks on snapshot files -- Key: CASSANDRA-4050 URL: https://issues.apache.org/jira/browse/CASSANDRA-4050 Project: Cassandra Issue Type: Bug Affects Versions: 1.0.8 Environment: Windows 7 Reporter: Jim Newsham Priority: Minor I'm using Cassandra 1.0.8, on Windows 7. When I take a snapshot of the database, I find that I am unable to delete the snapshot directory (i.e., dir named {datadir}\{keyspacename}\snapshots\{snapshottag}) while Cassandra is running: The action can't be completed because the folder or a file in it is open in another program. Close the folder or file and try again [in Windows Explorer]. If I terminate Cassandra, then I can delete the directory with no problem. I expect to be able to move or delete the snapshotted files while Cassandra is running, as this should not affect the runtime operation of Cassandra. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (CASSANDRA-4657) cql version race condition with rpc_server_type: sync
[ https://issues.apache.org/jira/browse/CASSANDRA-4657?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-4657: -- Reviewer: dbrosius Priority: Minor (was: Major) Affects Version/s: (was: 1.1.5) (was: 1.1.2) (was: 1.1.1) 1.1.0 Fix Version/s: 1.1.6 Assignee: Jonathan Ellis cql version race condition with rpc_server_type: sync - Key: CASSANDRA-4657 URL: https://issues.apache.org/jira/browse/CASSANDRA-4657 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.0 Environment: Ubuntu 12.04 Reporter: Emmanuel Courreges Assignee: Jonathan Ellis Priority: Minor Labels: features Fix For: 1.1.6 Attachments: 4657.patch If clients connect to a cassandra cluster configured with rpc_server_type: sync with heterogeneous cql versions (2 and 3), the cql version used for execution on the server changes seemingly randomly. It's due to the fact that CustomTThreadPoolServer.java does not set the remoteSocket anytime, or does not clear the cql version in the ThreadLocal clientState object. When CassandraServer.java calls state() it gets the ThreadLocal object clientState, which has its cqlversion already changed by a previous socket that was using the same thread. The easiest fix is probably to do a SocketSessionManagementService.instance.set when accepting a new client and SocketSessionManagementService.instance.remove when the client is closed, but if you really want to use the ThreadLocal clientState and not alloc/destroy a ClientState everytime, then you should clear this clientState on accept of a new client. The problem can be reproduced with cqlsh -3 on one side and a client that does not set the cql version, expecting to get version 2 by default, but actually gettingv v2/v3 depending on which thread it connects to. The problem does not happen with other rpc_server_types, nor with clients that set their cql version at connection. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4657) cql version race condition with rpc_server_type: sync
[ https://issues.apache.org/jira/browse/CASSANDRA-4657?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470899#comment-13470899 ] Jonathan Ellis commented on CASSANDRA-4657: --- Thanks Emmanuel! Your patch will work, but I'd like to do a deeper cleanup. I've pushed this to https://github.com/jbellis/cassandra/branches/4657. In order these commits # fix cql version reset under the existing regime of CassandraServer.clientState threadlocal # centralizes all session management into ThriftSessionManager # removes the threadlocal approach in favor of standardizing on the socket-based approach everywhere cql version race condition with rpc_server_type: sync - Key: CASSANDRA-4657 URL: https://issues.apache.org/jira/browse/CASSANDRA-4657 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.0 Environment: Ubuntu 12.04 Reporter: Emmanuel Courreges Labels: features Fix For: 1.1.6 Attachments: 4657.patch If clients connect to a cassandra cluster configured with rpc_server_type: sync with heterogeneous cql versions (2 and 3), the cql version used for execution on the server changes seemingly randomly. It's due to the fact that CustomTThreadPoolServer.java does not set the remoteSocket anytime, or does not clear the cql version in the ThreadLocal clientState object. When CassandraServer.java calls state() it gets the ThreadLocal object clientState, which has its cqlversion already changed by a previous socket that was using the same thread. The easiest fix is probably to do a SocketSessionManagementService.instance.set when accepting a new client and SocketSessionManagementService.instance.remove when the client is closed, but if you really want to use the ThreadLocal clientState and not alloc/destroy a ClientState everytime, then you should clear this clientState on accept of a new client. The problem can be reproduced with cqlsh -3 on one side and a client that does not set the cql version, expecting to get version 2 by default, but actually gettingv v2/v3 depending on which thread it connects to. The problem does not happen with other rpc_server_types, nor with clients that set their cql version at connection. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (CASSANDRA-4050) Unable to remove snapshot files on Windows while original sstables are live
[ https://issues.apache.org/jira/browse/CASSANDRA-4050?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-4050: -- Affects Version/s: (was: 1.0.8) Summary: Unable to remove snapshot files on Windows while original sstables are live (was: cassandra unnecessarily holds file locks on snapshot files) Unable to remove snapshot files on Windows while original sstables are live --- Key: CASSANDRA-4050 URL: https://issues.apache.org/jira/browse/CASSANDRA-4050 Project: Cassandra Issue Type: Bug Environment: Windows 7 Reporter: Jim Newsham Priority: Minor I'm using Cassandra 1.0.8, on Windows 7. When I take a snapshot of the database, I find that I am unable to delete the snapshot directory (i.e., dir named {datadir}\{keyspacename}\snapshots\{snapshottag}) while Cassandra is running: The action can't be completed because the folder or a file in it is open in another program. Close the folder or file and try again [in Windows Explorer]. If I terminate Cassandra, then I can delete the directory with no problem. I expect to be able to move or delete the snapshotted files while Cassandra is running, as this should not affect the runtime operation of Cassandra. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4050) cassandra unnecessarily holds file locks on snapshot files
[ https://issues.apache.org/jira/browse/CASSANDRA-4050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470901#comment-13470901 ] Jonathan Ellis commented on CASSANDRA-4050: --- Right, so NTFS is locking the underlying data since the original sstable is still open in Cassandra, so the behavior described by Jim applies. We'd have to add a workaround like the one given on superuser.com -- move the links to a garbage location to clean up on restart. This is pretty low priority for me but I'd be glad to point someone interested in the right direction. cassandra unnecessarily holds file locks on snapshot files -- Key: CASSANDRA-4050 URL: https://issues.apache.org/jira/browse/CASSANDRA-4050 Project: Cassandra Issue Type: Bug Environment: Windows 7 Reporter: Jim Newsham Priority: Minor I'm using Cassandra 1.0.8, on Windows 7. When I take a snapshot of the database, I find that I am unable to delete the snapshot directory (i.e., dir named {datadir}\{keyspacename}\snapshots\{snapshottag}) while Cassandra is running: The action can't be completed because the folder or a file in it is open in another program. Close the folder or file and try again [in Windows Explorer]. If I terminate Cassandra, then I can delete the directory with no problem. I expect to be able to move or delete the snapshotted files while Cassandra is running, as this should not affect the runtime operation of Cassandra. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
git commit: handle empty flushes of batchlog memtable patch by jbellis; tested by Aleksey Yeschenko for CASSANDRA-4667
Updated Branches: refs/heads/trunk 6b83663ca - 801d7d3f5 handle empty flushes of batchlog memtable patch by jbellis; tested by Aleksey Yeschenko for CASSANDRA-4667 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/801d7d3f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/801d7d3f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/801d7d3f Branch: refs/heads/trunk Commit: 801d7d3f5c5ce31e1335d2633e82111faa512716 Parents: 6b83663 Author: Jonathan Ellis jbel...@apache.org Authored: Fri Oct 5 23:08:02 2012 -0500 Committer: Jonathan Ellis jbel...@apache.org Committed: Fri Oct 5 23:08:56 2012 -0500 -- .../org/apache/cassandra/db/ColumnFamilyStore.java |3 +- src/java/org/apache/cassandra/db/DataTracker.java | 21 ++ src/java/org/apache/cassandra/db/Memtable.java | 16 +-- 3 files changed, 30 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/801d7d3f/src/java/org/apache/cassandra/db/ColumnFamilyStore.java -- diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index b539228..8f1b21c 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -972,7 +972,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean void replaceFlushed(Memtable memtable, SSTableReader sstable) { data.replaceFlushed(memtable, sstable); -CompactionManager.instance.submitBackground(this); +if (sstable != null) +CompactionManager.instance.submitBackground(this); } public boolean isValid() http://git-wip-us.apache.org/repos/asf/cassandra/blob/801d7d3f/src/java/org/apache/cassandra/db/DataTracker.java -- diff --git a/src/java/org/apache/cassandra/db/DataTracker.java b/src/java/org/apache/cassandra/db/DataTracker.java index 1e7b1bf..fd11f6c 100644 --- a/src/java/org/apache/cassandra/db/DataTracker.java +++ b/src/java/org/apache/cassandra/db/DataTracker.java @@ -123,13 +123,17 @@ public class DataTracker public void replaceFlushed(Memtable memtable, SSTableReader sstable) { +// sstable may be null if we flushed batchlog and nothing needed to be retained + if (!cfstore.isValid()) { View currentView, newView; do { currentView = view.get(); -newView = currentView.replaceFlushed(memtable, sstable).replace(Arrays.asList(sstable), Collections.SSTableReaderemptyList()); +newView = currentView.replaceFlushed(memtable, sstable); +if (sstable != null) +newView = newView.replace(Arrays.asList(sstable), Collections.SSTableReaderemptyList()); } while (!view.compareAndSet(currentView, newView)); return; @@ -143,10 +147,12 @@ public class DataTracker } while (!view.compareAndSet(currentView, newView)); -addNewSSTablesSize(Arrays.asList(sstable)); - -notifyAdded(sstable); -incrementallyBackup(sstable); +if (sstable != null) +{ +addNewSSTablesSize(Arrays.asList(sstable)); +notifyAdded(sstable); +incrementallyBackup(sstable); +} } public void incrementallyBackup(final SSTableReader sstable) @@ -504,7 +510,9 @@ public class DataTracker public View replaceFlushed(Memtable flushedMemtable, SSTableReader newSSTable) { SetMemtable newPending = ImmutableSet.copyOf(Sets.difference(memtablesPendingFlush, Collections.singleton(flushedMemtable))); -ListSSTableReader newSSTables = newSSTables(newSSTable); +ListSSTableReader newSSTables = newSSTable == null +? Collections.SSTableReaderemptyList() +: newSSTables(newSSTable); SSTableIntervalTree intervalTree = buildIntervalTree(newSSTables); return new View(memtable, newPending, Collections.unmodifiableList(newSSTables), compacting, intervalTree); } @@ -530,6 +538,7 @@ public class DataTracker private ListSSTableReader newSSTables(SSTableReader newSSTable) { +assert newSSTable != null; // not performance-sensitive, don't obsess over doing a selection merge here return newSSTables(Collections.SSTableReaderemptyList(), Collections.singletonList(newSSTable)); }
[jira] [Resolved] (CASSANDRA-4667) optimize memtable deletions for batchlog
[ https://issues.apache.org/jira/browse/CASSANDRA-4667?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis resolved CASSANDRA-4667. --- Resolution: Fixed committed optimize memtable deletions for batchlog Key: CASSANDRA-4667 URL: https://issues.apache.org/jira/browse/CASSANDRA-4667 Project: Cassandra Issue Type: Sub-task Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 1.2.0 beta 2 Attachments: 4667-handle-empty-flushes.txt, CASSANDRA-4667-v1.1.patch, CASSANDRA-4667-v2.patch, CASSANDRA-4667-v3.txt, CASSANDRA-4667-v4.txt Batchlog writes with the same key are never retried. This means that if a batchlog row is in the memtable, it can't be in any of the sstables, ever. In such cases we don't need to write a tombstone to disk. We can purge the row completely from the memtable and only write a tombstone if the row had been flushed already (if it's not in the memtable then it must be in one of the sstables). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4157) Allow KS + CF names up to 48 characters
[ https://issues.apache.org/jira/browse/CASSANDRA-4157?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470905#comment-13470905 ] Jonathan Ellis commented on CASSANDRA-4157: --- My mistake. The distinction is that: - New KS/CF are restricted to 48 characters everywhere - Cassandra will detect too-long paths on Windows and fail startup if existing schema definitions are too long. Otherwise, existing data is allowed to retain the current names. So, recreating schema from scratch will fail everywhere, but upgrading existing schema should always be fine on Linux. Allow KS + CF names up to 48 characters --- Key: CASSANDRA-4157 URL: https://issues.apache.org/jira/browse/CASSANDRA-4157 Project: Cassandra Issue Type: Improvement Components: Core Affects Versions: 1.1.0 Reporter: Jonathan Ellis Assignee: Jonathan Ellis Priority: Minor Fix For: 1.1.0 Attachments: 4157.txt CASSANDRA-2749 imposed a 32-character limit on KS and CF names. We can be a little more lenient than that and still be safe for path names (see CASSANDRA-4110). -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (CASSANDRA-4773) CQL shell not reflecting latest data when timestamp is passed as part of insert statements
Ananth Gundabattula created CASSANDRA-4773: -- Summary: CQL shell not reflecting latest data when timestamp is passed as part of insert statements Key: CASSANDRA-4773 URL: https://issues.apache.org/jira/browse/CASSANDRA-4773 Project: Cassandra Issue Type: Bug Affects Versions: 1.1.4 Environment: CentOS Reporter: Ananth Gundabattula Priority: Critical While using the CQLSH, I try inserting a row using timestamp and TTL along with consistency. The insert works fine for the first time. When I try to insert using the same key but different value and then issue a select , the value of the returned row is always the first value of the inserted row and not the value that was inserted later. Here are the details : 1. Replication Factor = 1 2. Consistency Level = ONE 3. TTL = 14 days 4. Timestamp = A value which reflects 10 days before the current day 5. Cassandra 1.1.4 6. CQL shell version 2 It may be noted that if the timstamp is not used while issuing the insert statement, the second insert on the same key works fine. Here are the details: 1. INSERT INTO Table1 (col1,col2,col3,col4,col5,col6) VALUES('abcde', 0, 87, 1345603159,222,'65DE') USING TIMESTAMP 1349476620 AND TTL 1209600; This works fine 2. Issuing a select works fine for the above row. 3. INSERT INTO Table1 (col1,col2,col3,col4,col5,col6) VALUES('abcde', 0, 87, 1345603159,222,'a2134') USING TIMESTAMP 1349476620 AND TTL 1209600; There is no error on this statement 4. Issuing a select returns the data inserted as given in step 1. 5. Giving a timestamp later than the above timestamp also does not change things 6. However, if I issue an insert without specifying the timestamp , the select statement gives the latest data always. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (CASSANDRA-4657) cql version race condition with rpc_server_type: sync
[ https://issues.apache.org/jira/browse/CASSANDRA-4657?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13470921#comment-13470921 ] Dave Brosius commented on CASSANDRA-4657: - +1 patch lgtm. cql version race condition with rpc_server_type: sync - Key: CASSANDRA-4657 URL: https://issues.apache.org/jira/browse/CASSANDRA-4657 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.0 Environment: Ubuntu 12.04 Reporter: Emmanuel Courreges Assignee: Jonathan Ellis Priority: Minor Labels: features Fix For: 1.1.6 Attachments: 4657.patch If clients connect to a cassandra cluster configured with rpc_server_type: sync with heterogeneous cql versions (2 and 3), the cql version used for execution on the server changes seemingly randomly. It's due to the fact that CustomTThreadPoolServer.java does not set the remoteSocket anytime, or does not clear the cql version in the ThreadLocal clientState object. When CassandraServer.java calls state() it gets the ThreadLocal object clientState, which has its cqlversion already changed by a previous socket that was using the same thread. The easiest fix is probably to do a SocketSessionManagementService.instance.set when accepting a new client and SocketSessionManagementService.instance.remove when the client is closed, but if you really want to use the ThreadLocal clientState and not alloc/destroy a ClientState everytime, then you should clear this clientState on accept of a new client. The problem can be reproduced with cqlsh -3 on one side and a client that does not set the cql version, expecting to get version 2 by default, but actually gettingv v2/v3 depending on which thread it connects to. The problem does not happen with other rpc_server_types, nor with clients that set their cql version at connection. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira