By default the TokenAwarePolicy does shuffle replicas, and it can be
disabled if you want to only hit the primary replica for the token range
you're querying :
http://docs.datastax.com/en/drivers/java/3.0/com/datastax/driver/core/policies/TokenAwarePolicy.html
On Mon, Mar 27, 2017 at 9:41 AM Avi
Is the driver doing the right thing by directing all reads for a given
token to the same node? If that node fails, then all of those reads
will be directed at other nodes, all oh whom will be cache-cold for the
the failed node's primary token range. Seems like the driver should
distribute
Keep in mind there are side effects to increasing to RF = 4
- Storage requirements for each node will increase. Depending on the
number of nodes in the cluster and the size of the data this could be
significant.
- Whilst the number of available coordinators increases, the number of
Yes, throughput for a given partition key cannot be improved with
horizontal scaling. You can increase RF to theoretically improve
throughput on that key, but actually in this case smart clients might hold
you back, because they're probably token aware, and will try to serve that
read off the
On 24/03/2017 01:00, Eric Stevens wrote:
Assuming an even distribution of data in your cluster, and an even
distribution across those keys by your readers, you would not need to
increase RF with cluster size to increase read performance. If you have
3 nodes with RF=3, and do 3 million reads,
Assuming an even distribution of data in your cluster, and an even
distribution across those keys by your readers, you would not need to
increase RF with cluster size to increase read performance. If you have 3
nodes with RF=3, and do 3 million reads, with good distribution, each node
has served
On 20/03/2017 22:05, Michael Wojcikiewicz wrote:
Not sure if someone has suggested this, but I believe it's not
sufficient to simply add nodes to a cluster to increase read
performance: you also need to alter the ReplicationFactor of the
keyspace to a larger value as you increase your cluster
On 20/03/2017 02:35, S G wrote:
2)
https://docs.datastax.com/en/developer/java-driver/3.1/manual/statements/prepared/
tells me to avoid preparing select queries if I expect a change of
columns in my table down the road.
The problem is also related to select * which is considered bad practice
ng to
> try.
> If not, please do share some more details.
>
>
>
>
> On Sat, Mar 18, 2017 at 2:00 PM, <j.kes...@enercast.de> wrote:
>
> +1 for executeAsync – had a long time to argue that it’s not bad as with
> good old rdbms.
>
>
>
>
>
>
>
> Gesende
On 19/03/2017 02:54, S G wrote:
Forgot to mention that this vmstat picture is for the client-cluster
reading from Cassandra.
Hi SG,
Your numbers are low, 15k req/sec would be ok for a single node, for a
12 nodes cluster, something goes wrong... how do you measure the
throughput?
As
uteAsync – had a long time to argue that it’s not bad as with
>>> good old rdbms.
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> Gesendet von meinem Windows 10 Phone
>>>
>>>
>>>
>>> *Von: *Arvydas Jo
meinem Windows 10 Phone
>>
>>
>>
>> *Von: *Arvydas Jonusonis <arvydas.jonuso...@gmail.com>
>> *Gesendet: *Samstag, 18. März 2017 19:08
>> *An: *user@cassandra.apache.org
>> *Betreff: *Re: How can I scale my read rate?
>>
>>
>>
>> ..
dows 10 Phone
>
>
>
> *Von: *Arvydas Jonusonis <arvydas.jonuso...@gmail.com>
> *Gesendet: *Samstag, 18. März 2017 19:08
> *An: *user@cassandra.apache.org
> *Betreff: *Re: How can I scale my read rate?
>
>
>
> ..then you're not taking advantage of request p
+1 for executeAsync – had a long time to argue that it’s not bad as with good
old rdbms.
Gesendet von meinem Windows 10 Phone
Von: Arvydas Jonusonis
Gesendet: Samstag, 18. März 2017 19:08
An: user@cassandra.apache.org
Betreff: Re: How can I scale my read rate?
..then you're not taking
14 matches
Mail list logo