Re: FW: Re: Is ReplicationFactor (eventually) guaranteed?
Attached log and conf file to https://issues.apache.org/jira/browse/CASSANDRA-924. Thanks. On Sat, Mar 27, 2010 at 2:43 PM, Stu Hood stu.h...@rackspace.com wrote: Could you try running your experiment again with DEBUG logging enabled, and then attaching the logs to a JIRA? -Original Message- From: Jianing Hu jian...@gmail.com Sent: Saturday, March 27, 2010 12:07pm To: user@cassandra.apache.org Subject: Re: FW: Re: Is ReplicationFactor (eventually) guaranteed? Here's the conf file, with comments removed. Thanks a lot for your help. Storage ClusterNamedev/ClusterName AutoBootstrapfalse/AutoBootstrap Keyspaces Keyspace Name=Keyspace1 KeysCachedFraction0.01/KeysCachedFraction ColumnFamily CompareWith=BytesType Name=Standard1/ ColumnFamily CompareWith=UTF8Type Name=Standard2/ ColumnFamily CompareWith=UTF8Type Name=Standard3/ ColumnFamily CompareWith=TimeUUIDType Name=StandardByUUID1/ ColumnFamily ColumnType=Super CompareWith=UTF8Type CompareSubcolumnsWith=UTF8Type Name=Super1 Comment=A column family with supercolumns, whose column and subcolumn names are UTF8 strings/ /Keyspace /Keyspaces Partitionerorg.apache.cassandra.dht.OrderPreservingPartitioner/Partitioner InitialTokenfoo3/InitialToken EndPointSnitchorg.apache.cassandra.locator.EndPointSnitch/EndPointSnitch ReplicaPlacementStrategyorg.apache.cassandra.locator.RackUnawareStrategy/ReplicaPlacementStrategy ReplicationFactor2/ReplicationFactor CommitLogDirectory/var/lib/cassandra/commitlog/CommitLogDirectory DataFileDirectories DataFileDirectory/var/lib/cassandra/data/DataFileDirectory /DataFileDirectories CalloutLocation/var/lib/cassandra/callouts/CalloutLocation StagingFileDirectory/var/lib/cassandra/staging/StagingFileDirectory Seeds Seedcs1/Seed Seedcs2/Seed Seedcs3/Seed /Seeds RpcTimeoutInMillis5000/RpcTimeoutInMillis CommitLogRotationThresholdInMB128/CommitLogRotationThresholdInMB ListenAddress10.0.1.1/ListenAddress !-- TCP port, for commands and data -- StoragePort7000/StoragePort !-- UDP port, for membership communications (gossip) -- ControlPort7001/ControlPort ThriftAddress10.0.1.1/ThriftAddress ThriftPort9160/ThriftPort ThriftFramedTransportfalse/ThriftFramedTransport SlicedBufferSizeInKB64/SlicedBufferSizeInKB FlushDataBufferSizeInMB32/FlushDataBufferSizeInMB FlushIndexBufferSizeInMB8/FlushIndexBufferSizeInMB ColumnIndexSizeInKB64/ColumnIndexSizeInKB MemtableSizeInMB64/MemtableSizeInMB MemtableObjectCountInMillions0.1/MemtableObjectCountInMillions MemtableFlushAfterMinutes60/MemtableFlushAfterMinutes ConcurrentReads16/ConcurrentReads ConcurrentWrites64/ConcurrentWrites CommitLogSyncperiodic/CommitLogSync CommitLogSyncPeriodInMS1/CommitLogSyncPeriodInMS GCGraceSeconds864000/GCGraceSeconds BinaryMemtableSizeInMB256/BinaryMemtableSizeInMB /Storage On Fri, Mar 26, 2010 at 10:00 PM, Stu Hood stu.h...@rackspace.com wrote: Ack... very sorry. I read the original message too quickly. The fact that neither read-repair nor anti-entropy are working is suspicious though. Do you think you could paste your config somewhere? -Original Message- From: Stu Hood stu.h...@rackspace.com Sent: Friday, March 26, 2010 11:57pm To: user@cassandra.apache.org Subject: Re: Is ReplicationFactor (eventually) guaranteed? replication factor == 1 means that there is only one copy of the data. And you deleted it. Repair depends on the replication factor being greater than 1. -Original Message- From: Jianing Hu jian...@gmail.com Sent: Friday, March 26, 2010 9:33pm To: user@cassandra.apache.org Subject: Re: Is ReplicationFactor (eventually) guaranteed? That's not what I saw in my test. I'm probably making some noob mistakes. Can someone enlighten me? Here's what I did: 1) Bring up a cluster with three servers cs1,2,3, with their initial token set to 'foo3', 'foo6', and 'foo9', respectively. ReplicationFactor is set to 2 on all 3. 2) Insert 9 columns with keys from 'foo1' to 'foo9', and flush. Now I have foo1,2,3,7,8,9 on cs1, foo1,2,3,4,5,6, on cs2, and foo4,5,6,7,8,9 on cs3. So far so good 3) Bring down cs3 and wipe out its data directory 4) Bring up cs3 5) run repair Keyspace1 on cs3, the flush At this point I expect to see cs3 getting its data back. But there's nothing in its data directory. I also tried getting all columns with ConsistencyLevel::ALL to see if that'll do a read pair. But still cs3's data directory is empty. What am I doing wrong? This is 0.5.1 BTW. Thanks, - Jianing On Fri, Mar 26, 2010 at 6:12 PM, Rob Coli rc...@digg.com wrote: On 3/26/10 5:57 PM, Jianing Hu wrote: In a cluster with ReplicationFactor 1, if one server goes down, will new replicas be created on other servers to satisfy the set ReplicationFactor? Yes, via Anti-Entropy.
Multi-indexing data
Hi, I have a question about Cassandra's data model I was hoping you guys could help me with. Most of our queries are performed against a series of tables containing crypto keys and their associated meta data. A key could have any number of identifiable attributes that need to be searchable: iasn, 64bit key id, 32bit key id, expiration, revoker etc… From what I understand I don't believe tagging the same information with multiple keys is supported. The best that I could think of was to add key/value pairs in the form keyid_32_0x73A5DC55:some junk data to the row or maintain a separate set of columns that provide keyid_32_0x73A5DC55:key row name mappings and perform two queries. I'm not a fan of either of these options. Is there some other solution that I may have overlooked? Thanks, -Matt
Re: Range scan performance in 0.6.0 beta2
On Fri, Mar 26, 2010 at 14:47, Jonathan Ellis jbel...@gmail.com wrote: On Fri, Mar 26, 2010 at 7:40 AM, Henrik Schröder skro...@gmail.com wrote: For each indexvalue we insert a row where the key is indexid + : + indexvalue encoded as hex string, and the row contains only one column, where the name is the object key encoded as a bytearray, and the value is empty. It's a unique index then? And you're trying to read things ordered by the index, not just give me keys with that have a column with this value? Yes, because if we have more than one column per row, there's no way of (easily) limiting the result. As it is now we rarely want all object keys associated with a range of indexvalues. However, this means we will have a lot of rows if we do it in Cassandra. /Henrik
Re: Range scan performance in 0.6.0 beta2
On Mon, Mar 29, 2010 at 4:06 AM, Henrik Schröder skro...@gmail.com wrote: On Fri, Mar 26, 2010 at 14:47, Jonathan Ellis jbel...@gmail.com wrote: It's a unique index then? And you're trying to read things ordered by the index, not just give me keys with that have a column with this value? Yes, because if we have more than one column per row, there's no way of (easily) limiting the result. That's exactly what the count parameter of SliceRange is for... ? -Jonathan
Re: Question about node failure...
So, what does anti-entropy repair do then? Sounds like you have to 'decommission' the dead node, then I thought run 'nodeprobe repair' to get the data adjusted back to a replication factor of 3, right? Also, what is the method to decommission a dead node? pass in the IP address of the dead node to nodeprobe on a member of the cluster? I've only used 'decommission' to remove the node I ran it on from the cluster... not a different node. It seems like if you decommission a node it should fix the replication factor for data that was on that node in this case... On Mon, Mar 29, 2010 at 10:32 AM, Jonathan Ellis jbel...@gmail.com wrote: On Mon, Mar 29, 2010 at 12:27 PM, Ned Wolpert ned.wolp...@imemories.com wrote: Folks- Can someone point out what happens during a node failure. Here is the Specific usecase: - Cassandra cluster with 4 nodes, replication factor of 3 - One node fails. - At this point, data that existed on the one failed node has copies on 2 live nodes. - The failed node never comes back First question: At what point does Cassandra re-migrate that data that only exists on 2 nodes to another node to retain the replication factor of 3? When you tell it to decommission the dead one. Second question: Given the above case, if a brand new node is added to the cluster, does anything happen to the data that now only exists on 2 nodes? No, Cassandra doesn't automatically assume that this node is never coming back w/o intervention, by design. (Temporary failures are much more common than permanent ones.) -Jonathan -- Virtually, Ned Wolpert Settle thy studies, Faustus, and begin... --Marlowe
Re: How reliable is cassandra?
On Mon, Mar 29, 2010 at 10:31 AM, Matthew Stump mrevilgn...@gmail.com wrote: Am I crazy to want to switch our server's primary data store from postgres to cassandra? This is a system used by banks and governments to store crypto keys which absolutely can not be lost. What benefits would you get from switching? Joe
Re: Which client API to choose?
On Wed, Mar 24, 2010 at 5:07 PM, Peter Chang pete...@gmail.com wrote: Hector is the way to go if you're using java. I'm using it right now and it's made things worlds easier. The reason why it wasn't bundled was because it's a separate and relatively new project. I think it's under a month old and it was done by a lone developer (Ran). Also, the Cassandra project wants to be able to give you options on which clients to use but (so far) hasn't sanctioned any as THE option which is fine.. it's just respecting the layers of abstraction. There was also another project here announced recently you can find at http://github.com/charliem/OCM which is basically object-cassandra mapping. I suggest you look into this for a more friendly user experience. I haven't used it personally for myself yet but I've personally implemented something similar along these lines to abstract my POJOs to Cassandra. Full disclosure - I am the author of OCM :) OCM is a more user friendly layer built on top of Hector. It designed to make some common client use cases a lot simpler. If its a good fit with what you are looking for then by all means use it. As far as I know its about the highest level interface currently available for Java. At its most basic level you use the generated classes like Java Beans to read and write the objects' fields. Then use the Load and Save methods to sync it with Cassandra. Its got a few more features than that but you get the idea. If you find it too high level then definitely go with Hector. You loose so little over using the Thrift interface directly I can't imagine a reason not to use it. Even if something you need is missing it would be a lot better to add it to Hector than to use Thrift directly. Charlie M
Re: How reliable is cassandra?
On Mar 29, 2010, at 12:40 PM, Eric Hauser wrote: BTW, does anyone from Digg patrol the list? I'm really interested in some additional the implementation of atomic counters with ZooKeeper. I know at least three Diggers patrol the list and one of them is a committer to Cassandra. Last I heard from my former coworkers at Digg was that ZooKeeper can be more overhead than wanted when doing locks in a high write environment. I think Ian quoted something obscene like 100ms+ to get a lock. I've also heard the Python implementation is lacking. That being said, I've heard lots of good things from many people about ZK. --Joe
Re: How reliable is cassandra?
On Mon, Mar 29, 2010 at 2:41 PM, Joe Stump j...@joestump.net wrote: I know at least three Diggers patrol the list and one of them is a committer to Cassandra. Last I heard from my former coworkers at Digg was that ZooKeeper can be more overhead than wanted when doing locks in a high write environment. ZK basically does 2PC for each write, so all the ZK members need to be in the same datacenter. Digg is operating 2 data centers now, so that's not a good fit -- either you have bad latency every time a client in the 2nd DC needs to access ZK or everyone has terrible write performance. ZK should be fine in a single datacenter though. -Jonathan
Write times
Are writes on OrderPreservingPartitioner always slower than RandomPartitioner? Is the replication factor a 'factor' in the write times? Thanks, Carlos This email message and any attachments are for the sole use of the intended recipients and may contain proprietary and/or confidential information which may be privileged or otherwise protected from disclosure. Any unauthorized review, use, disclosure or distribution is prohibited. If you are not an intended recipient, please contact the sender by reply email and destroy the original message and any copies of the message as well as any attachments to the original message.
Re: Write times
OPP should only affect write speed if OPP's tendency to unevenly distribute load causes some nodes to be overworked. In other words, OPP vs. RP on a single node system should have no real effect. -Original Message- From: Carlos Sanchez carlos.sanc...@riskmetrics.com Date: Mon, 29 Mar 2010 18:58:50 To: user@cassandra.apache.orguser@cassandra.apache.org Subject: Write times Are writes on OrderPreservingPartitioner always slower than RandomPartitioner? Is the replication factor a 'factor' in the write times? Thanks, Carlos This email message and any attachments are for the sole use of the intended recipients and may contain proprietary and/or confidential information which may be privileged or otherwise protected from disclosure. Any unauthorized review, use, disclosure or distribution is prohibited. If you are not an intended recipient, please contact the sender by reply email and destroy the original message and any copies of the message as well as any attachments to the original message.
Re: Ring management and load balance
On Fri, Mar 26, 2010 at 4:35 PM, Mike Malone m...@simplegeo.com wrote: With the random partitioner there's no need to suggest a token. The key space is statistically random so you should be able to just split 2^128 into equal sized segments and get fairly equal storage load. Your read / write load could get out of whack if you have hot spots and stuff, I guess. But for a large distributed data set I think that's unlikely. For order preserving partitioners it's harder. We've been thinking about this issue at SimpleGeo and were planning on implementing an algorithm that could determine the median row key statistically without having to inspect every key. Basically, it would pull a random sample of row keys (maybe from the Index file?) and then determine the median of that sample. Thoughts? That's exactly what the bootstrap token calculation does for OPP, after picking the most-loaded node to talk to. You could expose that over JMX, or generalize it to giving say 100 tokens, evenly spaced, so the tool could estimate position to within 1%. -Jonathan