Re: Have a CDC commitLog process option in Cassandra

2017-02-14 Thread Josh McKenzie
The furthest we got to a reference daemon was Carl's work on 11575. Hasn't
been prioritized since then since the interface refactors were the lion's
share of the necessary complexity to allow people to build off the feature.

As for any follow-up work,
https://issues.apache.org/jira/browse/CASSANDRA-12148 is the only other
thing on the list ToDo to improve the feature. As with 11575, I haven't had
time to get back to that in awhile although it's quite close to merge-able;
just need to finish up a rather nasty dtest for it that spins up 3 separate
clusters and bounces data along and rebase the patch.

On Mon, Feb 13, 2017 at 7:01 PM, Jay Zhuang 
wrote:

> Thanks Josh for your response. I agree that it may cause the JVM unstable
> if that feature is enabled. (we may still want to try that out internally
> and do some perf/stress tests, will share more information if we have the
> results. If you have any suggestions on testing, please let me know.)
>
> Another question, in your design doc: https://docs.google.com/docume
> nt/d/1ZxCWYkeZTquxsvf5hdPc0fiUnUHna8POvgt6TIzML4Y/edit
> You mentioned: "We will be providing a cdc compactor using the
> CommitLogReader interface shortly w/a config file specifying CF’s to
> preserve that will take cdc_raw data and compact it to cdc_processed"
> Is this feature implemented? Feels like it's doing similar work as the
> "daemon". If the customer want to implement such "daemon" inside cassandra,
> they could just extend that class, or have a callback interface in
> cdc_compactor, right? Would you please share more thoughts on that?
>
> If there's any CDC improvement(bug) in your mind, would you please create
> JIRA? maybe we could contribute to. Also we're back porting CDC feature to
> 3.0 internally, any suggestions would be appreciated.
>
> Thanks,
> Jay
>
>
> On 2/10/17 5:39 AM, Josh McKenzie wrote:
>
>> The primary reason I avoided integrating a daemon into the Cassandra
>> process was the increase in heap pressure and further muddying of the
>> profile of heap usage. We've already seen that mixing read/write,
>> compaction, streaming, and repair in the same JVM causes a nasty mix of
>> allocation patterns that are pretty much impossible to optimize for, so
>> furthering that problem wasn't on my ToDo list.
>>
>> Having a tool in-tree? Sure. But I'd strongly recommend against having it
>> be in-process.
>>
>> On Thu, Feb 9, 2017 at 7:19 PM, Jay Zhuang 
>> wrote:
>>
>> No. It's going to have Cassandra to manage the CDC logs, instead of having
>>> another daemon process to handle that.
>>>
>>> Here is CDC design JIRA: CASSANDRA-8844. The pain point is to develop and
>>> manage the daemon. If they're integrated, it's going to be easier to
>>> manage
>>> and monitor that.
>>>
>>> Thanks,
>>> Jay
>>>
>>>
>>> On 2/9/17 3:57 PM, Dikang Gu wrote:
>>>
>>> Is it for testing purpose?

 On Thu, Feb 9, 2017 at 3:54 PM, Jay Zhuang 
 wrote:

 Hi,

>
> To process the CDC commitLogs, it requires a separate Daemon process,
> Carl
> has a Daemon example here: CASSANDRA-11575.
>
> Does it make sense to integrate it into Cassandra? So the user doesn't
> have to manage another JVM on the same box. Then provide an ITrigger
> like
> interface (https://github.com/apache/cassandra/blob/trunk/src/java/org
> /apache/cassandra/triggers/ITrigger.java#L49) to process the data.
>
> Or maybe provide an interface option to handle the CDC commitLog in
> SegmentManager(https://github.com/apache/cassandra/blob/trun
> k/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmen
> tManagerCDC.java#L68).
>
> Any comments? If it make sense, I could create a JIRA for that.
>
> Thanks,
> Jay
>
>
>



>>


Re: Have a CDC commitLog process option in Cassandra

2017-02-10 Thread Josh McKenzie
The primary reason I avoided integrating a daemon into the Cassandra
process was the increase in heap pressure and further muddying of the
profile of heap usage. We've already seen that mixing read/write,
compaction, streaming, and repair in the same JVM causes a nasty mix of
allocation patterns that are pretty much impossible to optimize for, so
furthering that problem wasn't on my ToDo list.

Having a tool in-tree? Sure. But I'd strongly recommend against having it
be in-process.

On Thu, Feb 9, 2017 at 7:19 PM, Jay Zhuang 
wrote:

> No. It's going to have Cassandra to manage the CDC logs, instead of having
> another daemon process to handle that.
>
> Here is CDC design JIRA: CASSANDRA-8844. The pain point is to develop and
> manage the daemon. If they're integrated, it's going to be easier to manage
> and monitor that.
>
> Thanks,
> Jay
>
>
> On 2/9/17 3:57 PM, Dikang Gu wrote:
>
>> Is it for testing purpose?
>>
>> On Thu, Feb 9, 2017 at 3:54 PM, Jay Zhuang 
>> wrote:
>>
>> Hi,
>>>
>>> To process the CDC commitLogs, it requires a separate Daemon process,
>>> Carl
>>> has a Daemon example here: CASSANDRA-11575.
>>>
>>> Does it make sense to integrate it into Cassandra? So the user doesn't
>>> have to manage another JVM on the same box. Then provide an ITrigger like
>>> interface (https://github.com/apache/cassandra/blob/trunk/src/java/org
>>> /apache/cassandra/triggers/ITrigger.java#L49) to process the data.
>>>
>>> Or maybe provide an interface option to handle the CDC commitLog in
>>> SegmentManager(https://github.com/apache/cassandra/blob/trun
>>> k/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmen
>>> tManagerCDC.java#L68).
>>>
>>> Any comments? If it make sense, I could create a JIRA for that.
>>>
>>> Thanks,
>>> Jay
>>>
>>>
>>
>>
>>


Re: Have a CDC commitLog process option in Cassandra

2017-02-09 Thread Jay Zhuang
No. It's going to have Cassandra to manage the CDC logs, instead of 
having another daemon process to handle that.


Here is CDC design JIRA: CASSANDRA-8844. The pain point is to develop 
and manage the daemon. If they're integrated, it's going to be easier to 
manage and monitor that.


Thanks,
Jay

On 2/9/17 3:57 PM, Dikang Gu wrote:

Is it for testing purpose?

On Thu, Feb 9, 2017 at 3:54 PM, Jay Zhuang 
wrote:


Hi,

To process the CDC commitLogs, it requires a separate Daemon process, Carl
has a Daemon example here: CASSANDRA-11575.

Does it make sense to integrate it into Cassandra? So the user doesn't
have to manage another JVM on the same box. Then provide an ITrigger like
interface (https://github.com/apache/cassandra/blob/trunk/src/java/org
/apache/cassandra/triggers/ITrigger.java#L49) to process the data.

Or maybe provide an interface option to handle the CDC commitLog in
SegmentManager(https://github.com/apache/cassandra/blob/trun
k/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmen
tManagerCDC.java#L68).

Any comments? If it make sense, I could create a JIRA for that.

Thanks,
Jay







Re: Have a CDC commitLog process option in Cassandra

2017-02-09 Thread Dikang Gu
Is it for testing purpose?

On Thu, Feb 9, 2017 at 3:54 PM, Jay Zhuang 
wrote:

> Hi,
>
> To process the CDC commitLogs, it requires a separate Daemon process, Carl
> has a Daemon example here: CASSANDRA-11575.
>
> Does it make sense to integrate it into Cassandra? So the user doesn't
> have to manage another JVM on the same box. Then provide an ITrigger like
> interface (https://github.com/apache/cassandra/blob/trunk/src/java/org
> /apache/cassandra/triggers/ITrigger.java#L49) to process the data.
>
> Or maybe provide an interface option to handle the CDC commitLog in
> SegmentManager(https://github.com/apache/cassandra/blob/trun
> k/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmen
> tManagerCDC.java#L68).
>
> Any comments? If it make sense, I could create a JIRA for that.
>
> Thanks,
> Jay
>



-- 
Dikang


Have a CDC commitLog process option in Cassandra

2017-02-09 Thread Jay Zhuang

Hi,

To process the CDC commitLogs, it requires a separate Daemon process, 
Carl has a Daemon example here: CASSANDRA-11575.


Does it make sense to integrate it into Cassandra? So the user doesn't 
have to manage another JVM on the same box. Then provide an ITrigger 
like interface 
(https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/triggers/ITrigger.java#L49) 
to process the data.


Or maybe provide an interface option to handle the CDC commitLog in 
SegmentManager(https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java#L68).


Any comments? If it make sense, I could create a JIRA for that.

Thanks,
Jay