Re: Setting bloom_filter_fp_chance < 0.01

2016-05-30 Thread Adarsh Kumar
Thank Tyler for your comments. Created following ticket:
https://issues.apache.org/jira/browse/CASSANDRA-11920


Adarsh

On Thu, May 26, 2016 at 9:37 PM, Tyler Hobbs  wrote:

>
> On Thu, May 26, 2016 at 4:36 AM, Adarsh Kumar 
> wrote:
>
>>
>> 1). Is there any other way to configure no of buckets along with
>> bloom_fileter_fp_chance, to avoid this exception?
>>
>
> No, it's hard coded, although we could theoretically hard code it to
> support a higher number of buckets.
>
>
>> 2). If this validation is hard coaded then why it is even allowed to set
>> such value of bloom_fileter_fp_chance, that can prevent ssTable generation.
>>
>
> You're right, we should be validating this upfront when the probability is
> set.  Can you open a ticket here for that?
> https://issues.apache.org/jira/browse/CASSANDRA
>
>
> --
> Tyler Hobbs
> DataStax 
>


Cassandra data modeling for a social network

2016-05-30 Thread Mohammad Kermani
We are using Cassandra for our social network and we are designing/data
modeling tables we need, it is confusing for us and we don't know how to
design some tables and we have some little problems!

*As we understood for every query we have to have different tables*, and
for example user A is following user C and B.

Now, in Cassandra we have a table that is posts_by_user:

user_id  |  post_id   |  text  |  created_on  |  deleted  |
view_count

likes_count  |  comments_count  |  user_full_name

And we have a table according to the followers of users, we insert the
post's info to the table called user_timeline that when the follower users
are visiting the first web page we get the post from database from
user_timeline table.
And here is user_timeline table:

follower_id  |  post_id  | user_id (who posted)  |  likes_count  |

comments_count   |   location_name   |  user_full_name

*First, Is this data modeling correct for follow base (follower, following
actions) social network?*

And now we want to count likes of a post, as you see we have number of
likes in both tables*(user_timeline, posts_by_user)*, and imagine one user
has 1000 followers then by each like action we have to update all 1000 rows
in user_timeline and 1 row in posts_by_users; And this is not logical!
*Then, my second question is How should it be? I mean how should like
(favorite) table be?*

Thank you
I wish I can get answer


Re: Node Stuck while restarting

2016-05-30 Thread Bhuvan Rawal
We took backup of commitlogs and restarted the node, it started fine. As
the node was down for more than one day we can say for sure that it was
stuck and was not processing.

Wondering how we can tune our settings so as to avoid a similar scenario in
future, possibly not taking a hacky measure.

On Sun, May 29, 2016 at 7:12 PM, Bhuvan Rawal  wrote:

> Hi Mike,
>
> PFA the details you asked for: and some others if that helps:
> we are using jvm params
> -Xms8G
> -Xmx8G
>
> MAX_HEAP_SIZE: & HEAP_NEWSIZE: is not being set and possibly calculated
> by calculate_heap_sizes function. (And we are using default calculations):
> here are the results, pls correct me if im wrong :
> system_memory_in_mb : 64544
> system_cpu_cores : 16
>
> for MAX_HEAP_SIZE:
>
> # set max heap size based on the following
> # max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
> # calculate 1/2 ram and cap to 1024MB
> # calculate 1/4 ram and cap to 8192MB
> # pick the max
>
> By this I can figure out that MAX_HEAP_SIZE is 8GB - (From the first case
> & third case)
>
> max_sensible_yg_per_core_in_mb="100"
> max_sensible_yg_in_mb=`expr $max_sensible_yg_per_core_in_mb "*"
> $system_cpu_cores` -  100* 16 = 1600 MB
> desired_yg_in_mb=`expr $max_heap_size_in_mb / 4 ---That comes out to
> be- 8GB/4 = 2GB
>
> if [ "$desired_yg_in_mb" -gt "$max_sensible_yg_in_mb" ]
> then
> HEAP_NEWSIZE="${max_sensible_yg_in_mb}M"
> else
> HEAP_NEWSIZE="${desired_yg_in_mb}M"
> fi
>
> That should set HEAP_NEWSIZE to 1600MB by first case.
>
>
> memtable_allocation_type: heap_buffers
>
> memtable_cleanup_threshold- we are using default:
> # memtable_cleanup_threshold defaults to 1 / (memtable_flush_writers + 1)
> # memtable_cleanup_threshold: 0.11
>
> memtable_flush_writers - default (2)
> we can increase this as we are using SSD with IOPS of around 300/s
>
> memtable_heap_space_in_mb - default values
> # memtable_heap_space_in_mb: 2048
> # memtable_offheap_space_in_mb: 2048
>
> We are using G1 garbage collector and jdk1.8.0_45
>
> Best Regards,
>
>
> On Sun, May 29, 2016 at 5:07 PM, Mike Yeap  wrote:
>
>> Hi Bhuvan, how big are your current commit logs in the failed node, and
>> what are the sizes MAX_HEAP_SIZE and HEAP_NEWSIZE?
>>
>> Also the values of following properties in cassandra.yaml??
>>
>> memtable_allocation_type
>> memtable_cleanup_threshold
>> memtable_flush_writers
>> memtable_heap_space_in_mb
>> memtable_offheap_space_in_mb
>>
>>
>> Regards,
>> Mike Yeap
>>
>>
>>
>> On Sun, May 29, 2016 at 6:18 PM, Bhuvan Rawal 
>> wrote:
>>
>>> Hi,
>>>
>>> We are running a 6 Node cluster in 2 DC on DSC 3.0.3, with 3 Node each.
>>> One of the node was showing UNREACHABLE on other nodes in nodetool
>>> describecluster  and on that node it was showing all others UNREACHABLE and
>>> as a measure we restarted the node.
>>>
>>> But on doing that it is stuck possibly at with these messages in
>>> system.log:
>>>
>>> DEBUG [SlabPoolCleaner] 2016-05-29 14:07:28,156
>>> ColumnFamilyStore.java:829 - Enqueuing flush of batches: 226784704 (11%)
>>> on-heap, 0 (0%) off-heap
>>> DEBUG [main] 2016-05-29 14:07:28,576 CommitLogReplayer.java:415 -
>>> Replaying /commitlog/data/CommitLog-6-1464508993391.log (CL version 6,
>>> messaging version 10, compression null)
>>> DEBUG [main] 2016-05-29 14:07:28,781 ColumnFamilyStore.java:829 -
>>> Enqueuing flush of batches: 207333510 (10%) on-heap, 0 (0%) off-heap
>>>
>>> MemtablePostFlush / MemtableFlushWriter stages where it is stuck with
>>> pending messages.
>>> This has been the status of them as per *nodetool tpstats *for long.
>>> MemtablePostFlush Active - 1pending - 52
>>>   completed - 16
>>> MemtableFlushWriter   Active - 2pending - 13
>>>   completed - 15
>>>
>>>
>>> We restarted the node by setting log level to TRACE but in vain. What
>>> could be a possible contingency plan in such a scenario?
>>>
>>> Best Regards,
>>> Bhuvan
>>>
>>>
>>
>


Re: Per node limit for Disk Space

2016-05-30 Thread Eric Stevens
Those are rough guidelines, actual effective node size is going to depend
on your read/write workload and the compaction strategy you choose.  The
biggest reason data density per node usually needs to be limited is due to
data grooming overhead introduced by compaction.  Data at rest essentially
becomes I/O debt.  If you're using Leveled compaction, the interest rate on
that debt is higher.

If you're writing aggressively you'll find that you run out of I/O capacity
for smaller data at rest.  If you use compaction strategies that allow for
data to eventually stop compacting (Date Tiered, Time Windowed), you may be
able to have higher data density per node assuming that some of your data
is going into the no-longer-compacting stages.

Beyond that it'll be hard to say what the right size for you is.  Target
the recommended numbers and if you find that you're not running out of I/O
as you approach them you can probably go bigger.  Just remember to leave
~50% disk capacity free to leave room for compaction to happen.

On Fri, May 27, 2016 at 1:52 PM Anshu Vajpayee 
wrote:

> Hi All,
> I have question regarding max disk space limit  on a node.
>
> As per Data stax, We can have 1TB max disk space for rotational disks and
> up to 5 TB for SSDs on a node.
>
> Could you please suggest per your experience what would be limit for space
> on a single node with out causing so much stress on a  node?
>
>
>
>
>
> *​Thanks,​*
>
>


Time series data with only inserts

2016-05-30 Thread Rakesh Kumar
Let us assume that there is a table which gets only inserts and under
normal circumstances no reads on it. If we assume TTL to be 7 days,
what event
will trigger a compaction/purge of old data if the old data is not in
the mem cache and no session needs it.

thanks.


Re: Time series data with only inserts

2016-05-30 Thread Jeff Jirsa

Your compaction strategy gets triggered whenever you flush memtables to disk.

Most compaction strategies, especially those designed for write-only 
time-series workloads, check for fully expired sstables 
(getFullyExpiredSStables()) “often” (DTCS does it every 10 minutes, because 
it’s fairly expensive). That’s THE most efficient way to drop expired data - 
full sstable drops because it’s fully expired. Given that you’re not doing 
reads, it’s likely that getFullyExpiredSStables will have few (or no) blockers, 
and will search for / return fully expired sstables 7 days after they’re 
created, assuming you manage to use a compaction strategy that doesn’t mix old 
data with new data (DTCS is the only ‘official’ one that does this now, though 
TWCS in #9666 may be interesting to you).

Unfortunately, life being what it is, it’s pretty easy to end up in a situation 
where read repairs or other overlaps cause ‘blockers’ which prevent sstables 
from being fully expired. In those situations, using the tombstone compaction 
sub properties can nudge things in the right direction (for example, you can 
tell cassandra to compact a sstable with itself if it’s over 24 hours old and 
contains more than 80% tombstones, where 24 and 80 are both variables you 
control). Check out 
http://docs.datastax.com/en/cql/3.1/cql/cql_reference/compactSubprop.html for 
the tombstone related options.

- Jeff



On 5/30/16, 3:54 PM, "Rakesh Kumar"  wrote:

>Let us assume that there is a table which gets only inserts and under
>normal circumstances no reads on it. If we assume TTL to be 7 days,
>what event
>will trigger a compaction/purge of old data if the old data is not in
>the mem cache and no session needs it.
>
>thanks.

smime.p7s
Description: S/MIME cryptographic signature