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>
>  <ClusterName>dev</ClusterName>
>  <AutoBootstrap>false</AutoBootstrap>
>  <Keyspaces>
>    <Keyspace Name="Keyspace1">
>      <KeysCachedFraction>0.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>
>  <Partitioner>org.apache.cassandra.dht.OrderPreservingPartitioner</Partitioner>
>  <InitialToken>foo3</InitialToken>
>  <EndPointSnitch>org.apache.cassandra.locator.EndPointSnitch</EndPointSnitch>
> <ReplicaPlacementStrategy>org.apache.cassandra.locator.RackUnawareStrategy</ReplicaPlacementStrategy>
>  <ReplicationFactor>2</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>
>      <Seed>cs1</Seed>
>      <Seed>cs2</Seed>
>      <Seed>cs3</Seed>
>  </Seeds>
>  <RpcTimeoutInMillis>5000</RpcTimeoutInMillis>
>  <CommitLogRotationThresholdInMB>128</CommitLogRotationThresholdInMB>
>  <ListenAddress>10.0.1.1</ListenAddress>
>  <!-- TCP port, for commands and data -->
>  <StoragePort>7000</StoragePort>
>  <!-- UDP port, for membership communications (gossip) -->
>  <ControlPort>7001</ControlPort>
>  <ThriftAddress>10.0.1.1</ThriftAddress>
>  <ThriftPort>9160</ThriftPort>
>  <ThriftFramedTransport>false</ThriftFramedTransport>
>  <SlicedBufferSizeInKB>64</SlicedBufferSizeInKB>
>  <FlushDataBufferSizeInMB>32</FlushDataBufferSizeInMB>
>  <FlushIndexBufferSizeInMB>8</FlushIndexBufferSizeInMB>
>  <ColumnIndexSizeInKB>64</ColumnIndexSizeInKB>
>  <MemtableSizeInMB>64</MemtableSizeInMB>
>  <MemtableObjectCountInMillions>0.1</MemtableObjectCountInMillions>
>  <MemtableFlushAfterMinutes>60</MemtableFlushAfterMinutes>
>  <ConcurrentReads>16</ConcurrentReads>
>  <ConcurrentWrites>64</ConcurrentWrites>
>  <CommitLogSync>periodic</CommitLogSync>
>  <CommitLogSyncPeriodInMS>10000</CommitLogSyncPeriodInMS>
>  <GCGraceSeconds>864000</GCGraceSeconds>
>  <BinaryMemtableSizeInMB>256</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.
>>>
>>> http://wiki.apache.org/cassandra/AntiEntropy
>>> http://wiki.apache.org/cassandra/ArchitectureAntiEntropy
>>>
>>> It's worth noting that "hot" keys are likely to be re-replicated by Read
>>> Repair before Anti Entropy is triggered.
>>>
>>> http://wiki.apache.org/cassandra/ReadRepair
>>>
>>> =Rob
>>>
>>>
>>>
>>>
>>
>>
>>
>>
>
>
>

Reply via email to