You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flume.apache.org by Roshan Naik <ro...@hortonworks.com> on 2013/06/24 12:24:50 UTC

Review Request: Revised design for Spillable Mem Channel

-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/
-----------------------------------------------------------

Review request for Flume.


Description
-------

Revised design for Spillable Mem Channel.
We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.

Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)

Essence of this design:
- SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
- SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
- Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
- Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.

 


This addresses bug FLUME-1227.
    https://issues.apache.org/jira/browse/FLUME-1227


Diffs
-----

  flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
  flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/pom.xml 6485d47 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
  flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
  flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
  flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
  flume-ng-dist/pom.xml f20ff1e 
  flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
  flume-ng-node/pom.xml 5cdd9e2 
  flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java e63c601 
  pom.xml 7f6f824 

Diff: https://reviews.apache.org/r/12060/diff/


Testing
-------

Wrote a set of Unit tests. A few are failing and need to be finished up.


Thanks,

Roshan Naik


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Roshan Naik <ro...@hortonworks.com>.

> On Aug. 30, 2013, 8:41 p.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-spillable-memory-channel/pom.xml, lines 65-70
> > <https://reviews.apache.org/r/12060/diff/3/?file=318118#file318118line65>
> >
> >     This should no longer be needed, with the File Channel hadoop dependency gone, right?

yes.


> On Aug. 30, 2013, 8:41 p.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java, line 72
> > <https://reviews.apache.org/r/12060/diff/3/?file=318119#file318119line72>
> >
> >     What does this signify?

should be defaultMemoryCapacity actually.


> On Aug. 30, 2013, 8:41 p.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java, lines 137-138
> > <https://reviews.apache.org/r/12060/diff/3/?file=318119#file318119line137>
> >
> >     I really think we need a better way of deciding if the events are in primary or secondary. We should make it easy to understand. This works, but I'd like it to be clearer (use an Enum or boolean to specify?) Agreed that this is more memory-efficient, at least it should be documented - so we can maintain it properly.

I assume you mean replace counts with bool/enum. Indeed using a stack of enums/bools means (one for every event) becomes very inefficient.

I assume you are suggesting that I add javadocs for the DrainOrderQueue class for better maintainability ? i would be happy to. please confirm.


> On Aug. 30, 2013, 8:41 p.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java, line 138
> > <https://reviews.apache.org/r/12060/diff/3/?file=318119#file318119line138>
> >
> >     Looks like none of the methods in this class are thread-safe and all methods are called from a synchronized(queueLock) block. I think it is better to actually make these methods thread-safe,and minimize the synchronization done using queueLock.

The drain order queue needs to be updated in a single atomic transaction along with an update to the primary or overflow. So the external coarser granularity lock is unavoidable.  Also there is a need to combine a few method invocations on DrainOrderQueue into single atomic transaction. (read/modify/write type operation) 
So an internally synchronized DrainOrderQueue would introduce superfluous locking very frequently (multiple times per event).


> On Aug. 30, 2013, 8:41 p.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java, lines 158-162
> > <https://reviews.apache.org/r/12060/diff/3/?file=318119#file318119line158>
> >
> >     Nit: Braces missing. Usually even if it is one line, we use braces (or put the statement in the same line as the if).

ok.. i have updated the code to fix all such oneliner if/else  instances that i could find.


> On Aug. 30, 2013, 8:41 p.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java, line 421
> > <https://reviews.apache.org/r/12060/diff/3/?file=318119#file318119line421>
> >
> >     What does this mean? Can you make the error message better here?

Is this clearer?    "Unable to insert event into memory queue in spite of spare capacity, this is very unexpected" 


> On Aug. 30, 2013, 8:41 p.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java, lines 600-604
> > <https://reviews.apache.org/r/12060/diff/3/?file=318119#file318119line600>
> >
> >     lets have a different config to disable overflow, than overloading the same param.

Ok. We will then need one config for disabling primary & one for disabling overflow.

Setting memoryCapacity or overflowCapacity to zero would still mean 'do not store anything' in them and consequently retain the same effect. So i didnt introduce additional configs. Let me know what you think.


- Roshan


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/#review25798
-----------------------------------------------------------


On July 3, 2013, 11:56 p.m., Roshan Naik wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/12060/
> -----------------------------------------------------------
> 
> (Updated July 3, 2013, 11:56 p.m.)
> 
> 
> Review request for Flume.
> 
> 
> Bugs: FLUME-1227
>     https://issues.apache.org/jira/browse/FLUME-1227
> 
> 
> Repository: flume-git
> 
> 
> Description
> -------
> 
> Revised design for Spillable Mem Channel.
> We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.
> 
> Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)
> 
> Essence of this design:
> - SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
> - SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
> - Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
> - Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.
> 
> 
> SC Configuration:
> - Accepts all the File Channel settings
> - Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
>  
> 
> Sample config ...
> 
> a1.channels = c1
> a1.sinks = logger
> a1.sources = src
> 
> a1.sources.src.type = exec
> a1.sources.src.command = seq 1 100000
> a1.sources.src.batchSize = 10
> a1.sources.src.channels = c1
> 
> a1.sinks.logger.type = logger
> a1.sinks.logger.channel = c1
> 
> a1.channels.c1.type = spillablememory
> a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
> a1.channels.c1.dataDirs = /tmp/flume/data
> a1.channels.c1.memoryCapacity = 10
> a1.channels.c1.keep-alive = 2
> 
> 
> Diffs
> -----
> 
>   flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
>   flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/pom.xml 5832ab4 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
>   flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
>   flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
>   flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
>   flume-ng-dist/pom.xml 83332a9 
>   flume-ng-doc/sphinx/FlumeUserGuide.rst 63cad21 
>   flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
>   flume-ng-node/pom.xml f1b0c65 
>   pom.xml 15e6d9b 
> 
> Diff: https://reviews.apache.org/r/12060/diff/
> 
> 
> Testing
> -------
> 
> Wrote a set of Unit tests. A few are failing and need to be finished up.
> 
> 
> File Attachments
> ----------------
> 
> Revised design doc
>   https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf
> 
> 
> Thanks,
> 
> Roshan Naik
> 
>


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Hari Shreedharan <hs...@cloudera.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/#review25798
-----------------------------------------------------------


Hi Roshan,

This is starting to look pretty good. I did a quick-ish review, and have some comments. 

Also I am not entirely convinced we need the ChannelFullException - why would ChannelException itself not suffice here?

Also, I think you need to rebase on the newer file channel updates, since the patch seems to need hadoop-common which File Channel no longer needs.


flume-ng-channels/flume-spillable-memory-channel/pom.xml
<https://reviews.apache.org/r/12060/#comment50322>

    This should no longer be needed, with the File Channel hadoop dependency gone, right?



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment50323>

    This should not be public right? We don't want people using the interface directly, but only through configs no? 



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment50324>

    What does this signify?  



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment50336>

    Why is this public? Can this be made final?



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment50328>

    I really think we need a better way of deciding if the events are in primary or secondary. We should make it easy to understand. This works, but I'd like it to be clearer (use an Enum or boolean to specify?) Agreed that this is more memory-efficient, at least it should be documented - so we can maintain it properly.



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment50332>

    Looks like none of the methods in this class are thread-safe and all methods are called from a synchronized(queueLock) block. I think it is better to actually make these methods thread-safe,and minimize the synchronization done using queueLock.



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment50330>

    Nit: Braces missing. Usually even if it is one line, we use braces (or put the statement in the same line as the if).



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment50331>

    Nit:space between - and eventCount



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment50335>

    What does this mean? Can you make the error message better here?



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment50334>

    lets have a different config to disable overflow, than overloading the same param. 


- Hari Shreedharan


On July 3, 2013, 11:56 p.m., Roshan Naik wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/12060/
> -----------------------------------------------------------
> 
> (Updated July 3, 2013, 11:56 p.m.)
> 
> 
> Review request for Flume.
> 
> 
> Bugs: FLUME-1227
>     https://issues.apache.org/jira/browse/FLUME-1227
> 
> 
> Repository: flume-git
> 
> 
> Description
> -------
> 
> Revised design for Spillable Mem Channel.
> We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.
> 
> Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)
> 
> Essence of this design:
> - SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
> - SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
> - Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
> - Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.
> 
> 
> SC Configuration:
> - Accepts all the File Channel settings
> - Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
>  
> 
> Sample config ...
> 
> a1.channels = c1
> a1.sinks = logger
> a1.sources = src
> 
> a1.sources.src.type = exec
> a1.sources.src.command = seq 1 100000
> a1.sources.src.batchSize = 10
> a1.sources.src.channels = c1
> 
> a1.sinks.logger.type = logger
> a1.sinks.logger.channel = c1
> 
> a1.channels.c1.type = spillablememory
> a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
> a1.channels.c1.dataDirs = /tmp/flume/data
> a1.channels.c1.memoryCapacity = 10
> a1.channels.c1.keep-alive = 2
> 
> 
> Diffs
> -----
> 
>   flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
>   flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/pom.xml 5832ab4 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
>   flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
>   flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
>   flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
>   flume-ng-dist/pom.xml 83332a9 
>   flume-ng-doc/sphinx/FlumeUserGuide.rst 63cad21 
>   flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
>   flume-ng-node/pom.xml f1b0c65 
>   pom.xml 15e6d9b 
> 
> Diff: https://reviews.apache.org/r/12060/diff/
> 
> 
> Testing
> -------
> 
> Wrote a set of Unit tests. A few are failing and need to be finished up.
> 
> 
> File Attachments
> ----------------
> 
> Revised design doc
>   https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf
> 
> 
> Thanks,
> 
> Roshan Naik
> 
>


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Roshan Naik <ro...@hortonworks.com>.

> On Oct. 16, 2013, 12:06 a.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java, lines 370-374
> > <https://reviews.apache.org/r/12060/diff/4/?file=364810#file364810line370>
> >
> >     This is exposing a File Channel internal data structure to a new component. Spillable Channel should use super.getTransaction() method to get the new transaction

Yes. Will use super.createTransaction() instead.


> On Oct. 16, 2013, 12:06 a.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java, line 302
> > <https://reviews.apache.org/r/12060/diff/4/?file=364812#file364812line302>
> >
> >     This essentially blocks more than 1 transaction at a time right? That will be a serious performance issue.

I can see your point. However, my performance measurements do not seem to indicate perf issues with multiple sources & sinks (and memory queue disabled) when compared to FC. If you have some ideas around specific scenarios that could expose this perf issue, I could probe it further. The micro benchmark code for SpillChannel is built into the test: TestSpillableChannel.testParallelMultipleSourcesAndSinks in the patch.


> On Oct. 16, 2013, 12:06 a.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java, lines 339-341
> > <https://reviews.apache.org/r/12060/diff/4/?file=364812#file364812line339>
> >
> >     Why only when !overflow?

I assume you are enquiring about the comment. 
Since in this case its operating only on takeList which is thread private. So no race condition here.


> On Oct. 16, 2013, 12:06 a.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java, line 87
> > <https://reviews.apache.org/r/12060/diff/4/?file=364810#file364810line87>
> >
> >     Why is this being made protected? This is a File Channel internal component - we could change the implementation of the channel and get rid of the Log class altogether, so we should not expose this.

was not required anymore. fixed.


> On Oct. 16, 2013, 12:06 a.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java, line 433
> > <https://reviews.apache.org/r/12060/diff/4/?file=364810#file364810line433>
> >
> >     This should be private/package-private(like it is now). We should not expose this to a new component.

not required anymore. fixed.


> On Oct. 16, 2013, 12:06 a.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java, lines 233-234
> > <https://reviews.apache.org/r/12060/diff/4/?file=364812#file364812line233>
> >
> >     These initial numbers seem a bit low. Let's bump this to 100


> On Oct. 16, 2013, 12:06 a.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java, lines 475-479
> > <https://reviews.apache.org/r/12060/diff/4/?file=364812#file364812line475>
> >
> >     Evne though gc will come and clear up all this, it might just be a good idea to clear up just for clarity and to help gc


> On Oct. 16, 2013, 12:06 a.m., Hari Shreedharan wrote:
> > flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java, lines 348-350
> > <https://reviews.apache.org/r/12060/diff/4/?file=364812#file364812line348>
> >
> >     As mentioned above, this should not depend on FileBackedTxn

Will use super.createTransaction() instead.


- Roshan


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/#review27047
-----------------------------------------------------------


On Oct. 24, 2013, 10:16 p.m., Roshan Naik wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/12060/
> -----------------------------------------------------------
> 
> (Updated Oct. 24, 2013, 10:16 p.m.)
> 
> 
> Review request for Flume.
> 
> 
> Bugs: FLUME-1227
>     https://issues.apache.org/jira/browse/FLUME-1227
> 
> 
> Repository: flume-git
> 
> 
> Description
> -------
> 
> Revised design for Spillable Mem Channel.
> We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.
> 
> Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)
> 
> Essence of this design:
> - SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
> - SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
> - Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
> - Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.
> 
> 
> SC Configuration:
> - Accepts all the File Channel settings
> - Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
>  
> 
> Sample config ...
> 
> a1.channels = c1
> a1.sinks = logger
> a1.sources = src
> 
> a1.sources.src.type = exec
> a1.sources.src.command = seq 1 100000
> a1.sources.src.batchSize = 10
> a1.sources.src.channels = c1
> 
> a1.sinks.logger.type = logger
> a1.sinks.logger.channel = c1
> 
> a1.channels.c1.type = spillablememory
> a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
> a1.channels.c1.dataDirs = /tmp/flume/data
> a1.channels.c1.memoryCapacity = 10
> a1.channels.c1.keep-alive = 2
> 
> 
> Diffs
> -----
> 
>   flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
>   flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/pom.xml 5832ab4 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
>   flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
>   flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
>   flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
>   flume-ng-dist/pom.xml 2d0ee47 
>   flume-ng-doc/sphinx/FlumeUserGuide.rst 98859ce 
>   flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
>   flume-ng-node/pom.xml f1b0c65 
>   pom.xml 267925f 
> 
> Diff: https://reviews.apache.org/r/12060/diff/
> 
> 
> Testing
> -------
> 
> Wrote a set of Unit tests. A few are failing and need to be finished up.
> 
> 
> File Attachments
> ----------------
> 
> Revised design doc
>   https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf
> 
> 
> Thanks,
> 
> Roshan Naik
> 
>


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Hari Shreedharan <hs...@cloudera.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/#review27047
-----------------------------------------------------------


Hey Roshan,

I did a review of the patch - generally looks good. I have some comments though. Apart from the comments below, there are a lot of style issues that need to be taken care of. Also, the drain order queue needs to be documented better.

In addition, I think the SpillableTxn class has a lot of conditionals which probably can be simplified. There is a bigger problem, which is the whole put/take methods being enclosed in synchronized blocks - locking the entire channel up. That would be ok, if the operation done was just polling of the queue, but this seems a little heavyweight to block the entire channel off. Can that be simplified/improved?


flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java
<https://reviews.apache.org/r/12060/#comment52662>

    Why is this being made protected? This is a File Channel internal component - we could change the implementation of the channel and get rid of the Log class altogether, so we should not expose this.



flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java
<https://reviews.apache.org/r/12060/#comment52660>

    This is exposing a File Channel internal data structure to a new component. Spillable Channel should use super.getTransaction() method to get the new transaction



flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java
<https://reviews.apache.org/r/12060/#comment52661>

    This should be private/package-private(like it is now). We should not expose this to a new component.



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment52663>

    These initial numbers seem a bit low. Let's bump this to 100



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment52665>

    This essentially blocks more than 1 transaction at a time right? That will be a serious performance issue.



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment52666>

    Why only when !overflow?



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment52667>

    As mentioned above, this should not depend on FileBackedTxn 



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment52669>

    Evne though gc will come and clear up all this, it might just be a good idea to clear up just for clarity and to help gc


- Hari Shreedharan


On Oct. 15, 2013, 8:44 p.m., Roshan Naik wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/12060/
> -----------------------------------------------------------
> 
> (Updated Oct. 15, 2013, 8:44 p.m.)
> 
> 
> Review request for Flume.
> 
> 
> Bugs: FLUME-1227
>     https://issues.apache.org/jira/browse/FLUME-1227
> 
> 
> Repository: flume-git
> 
> 
> Description
> -------
> 
> Revised design for Spillable Mem Channel.
> We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.
> 
> Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)
> 
> Essence of this design:
> - SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
> - SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
> - Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
> - Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.
> 
> 
> SC Configuration:
> - Accepts all the File Channel settings
> - Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
>  
> 
> Sample config ...
> 
> a1.channels = c1
> a1.sinks = logger
> a1.sources = src
> 
> a1.sources.src.type = exec
> a1.sources.src.command = seq 1 100000
> a1.sources.src.batchSize = 10
> a1.sources.src.channels = c1
> 
> a1.sinks.logger.type = logger
> a1.sinks.logger.channel = c1
> 
> a1.channels.c1.type = spillablememory
> a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
> a1.channels.c1.dataDirs = /tmp/flume/data
> a1.channels.c1.memoryCapacity = 10
> a1.channels.c1.keep-alive = 2
> 
> 
> Diffs
> -----
> 
>   flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
>   flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/pom.xml 5832ab4 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
>   flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
>   flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
>   flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
>   flume-ng-dist/pom.xml 2d0ee47 
>   flume-ng-doc/sphinx/FlumeUserGuide.rst 98859ce 
>   flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
>   flume-ng-node/pom.xml f1b0c65 
>   pom.xml 267925f 
> 
> Diff: https://reviews.apache.org/r/12060/diff/
> 
> 
> Testing
> -------
> 
> Wrote a set of Unit tests. A few are failing and need to be finished up.
> 
> 
> File Attachments
> ----------------
> 
> Revised design doc
>   https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf
> 
> 
> Thanks,
> 
> Roshan Naik
> 
>


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Roshan Naik <ro...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/#review27964
-----------------------------------------------------------



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54413>

    made this change in others places too



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54414>

    made this fix in other locations too



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54429>

    Thanks. good catch.



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54418>

    In this instance, it would nice to make an exception for readability .. since the comment needs to appear before the else



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54428>

    Thanks. Good catch.


- Roshan Naik


On Oct. 24, 2013, 10:18 p.m., Roshan Naik wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/12060/
> -----------------------------------------------------------
> 
> (Updated Oct. 24, 2013, 10:18 p.m.)
> 
> 
> Review request for Flume.
> 
> 
> Bugs: FLUME-1227
>     https://issues.apache.org/jira/browse/FLUME-1227
> 
> 
> Repository: flume-git
> 
> 
> Description
> -------
> 
> Revised design for Spillable Mem Channel.
> We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.
> 
> Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)
> 
> Essence of this design:
> - SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
> - SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
> - Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
> - Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.
> 
> 
> SC Configuration:
> - Accepts all the File Channel settings
> - Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
>  
> 
> Sample config ...
> 
> a1.channels = c1
> a1.sinks = logger
> a1.sources = src
> 
> a1.sources.src.type = exec
> a1.sources.src.command = seq 1 100000
> a1.sources.src.batchSize = 10
> a1.sources.src.channels = c1
> 
> a1.sinks.logger.type = logger
> a1.sinks.logger.channel = c1
> 
> a1.channels.c1.type = spillablememory
> a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
> a1.channels.c1.dataDirs = /tmp/flume/data
> a1.channels.c1.memoryCapacity = 10
> a1.channels.c1.keep-alive = 2
> 
> 
> Diffs
> -----
> 
>   flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
>   flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/pom.xml 5832ab4 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
>   flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
>   flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
>   flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
>   flume-ng-dist/pom.xml 2d0ee47 
>   flume-ng-doc/sphinx/FlumeUserGuide.rst 98859ce 
>   flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
>   flume-ng-node/pom.xml f1b0c65 
>   pom.xml 267925f 
> 
> Diff: https://reviews.apache.org/r/12060/diff/
> 
> 
> Testing
> -------
> 
> Wrote a set of Unit tests. A few are failing and need to be finished up.
> 
> 
> File Attachments
> ----------------
> 
> Revised design doc
>   https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf
> 
> 
> Thanks,
> 
> Roshan Naik
> 
>


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Hari Shreedharan <hs...@cloudera.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/#review27847
-----------------------------------------------------------

Ship it!


Roshan,

This looks good to go, but I am fairly sure this needs considerable testing and improvements before we call it production ready. So let's commit this for now, but mark it as alpha and experimental in the user docs and the javadocs. Let's iterate on this over to make sure it is production ready.


Also can you please make sure the formats are correct. An if looks like:

if (condition) {
} else {
}

similarly for for/while etc. An IDE can probably fix most of it up for you.

Please make these fixes and attach the patch to the jira.


flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54213>

    Nit: Space before and after += makes it more readable.



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54211>

    Extra space after if (should generally be before the paranthesis).



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54214>

    same as above



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54215>

    same as above



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54212>

    same as above



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54216>

    the spaces in the if are off. an if should look like:
    
    if (condition) {
    
    } else {
    
    }



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54217>

    additional space after return



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54218>

    Space after !



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54219>

    additional space within (), missing space before.



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54220>

    space issue



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54221>

    space issue


- Hari Shreedharan


On Oct. 24, 2013, 10:18 p.m., Roshan Naik wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/12060/
> -----------------------------------------------------------
> 
> (Updated Oct. 24, 2013, 10:18 p.m.)
> 
> 
> Review request for Flume.
> 
> 
> Bugs: FLUME-1227
>     https://issues.apache.org/jira/browse/FLUME-1227
> 
> 
> Repository: flume-git
> 
> 
> Description
> -------
> 
> Revised design for Spillable Mem Channel.
> We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.
> 
> Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)
> 
> Essence of this design:
> - SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
> - SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
> - Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
> - Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.
> 
> 
> SC Configuration:
> - Accepts all the File Channel settings
> - Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
>  
> 
> Sample config ...
> 
> a1.channels = c1
> a1.sinks = logger
> a1.sources = src
> 
> a1.sources.src.type = exec
> a1.sources.src.command = seq 1 100000
> a1.sources.src.batchSize = 10
> a1.sources.src.channels = c1
> 
> a1.sinks.logger.type = logger
> a1.sinks.logger.channel = c1
> 
> a1.channels.c1.type = spillablememory
> a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
> a1.channels.c1.dataDirs = /tmp/flume/data
> a1.channels.c1.memoryCapacity = 10
> a1.channels.c1.keep-alive = 2
> 
> 
> Diffs
> -----
> 
>   flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
>   flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/pom.xml 5832ab4 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
>   flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
>   flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
>   flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
>   flume-ng-dist/pom.xml 2d0ee47 
>   flume-ng-doc/sphinx/FlumeUserGuide.rst 98859ce 
>   flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
>   flume-ng-node/pom.xml f1b0c65 
>   pom.xml 267925f 
> 
> Diff: https://reviews.apache.org/r/12060/diff/
> 
> 
> Testing
> -------
> 
> Wrote a set of Unit tests. A few are failing and need to be finished up.
> 
> 
> File Attachments
> ----------------
> 
> Revised design doc
>   https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf
> 
> 
> Thanks,
> 
> Roshan Naik
> 
>


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Hari Shreedharan <hs...@cloudera.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/#review27941
-----------------------------------------------------------


Sorry, I was looking at the code once again and found a couple issues which I think should be fixed. 


flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54379>

    If one of these two throws an exception, then the totalStored semaphore has the wrong value. You need to enclose the whole thing in a try-finally and release the totalStored if it was not successful. For the file channel, it is possible that a txn.take() throws, since it gets written to disk.



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54383>

    else should be on this line



flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
<https://reviews.apache.org/r/12060/#comment54391>

    Why is this not being done when the overflow channel is in use? 


- Hari Shreedharan


On Oct. 24, 2013, 10:18 p.m., Roshan Naik wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/12060/
> -----------------------------------------------------------
> 
> (Updated Oct. 24, 2013, 10:18 p.m.)
> 
> 
> Review request for Flume.
> 
> 
> Bugs: FLUME-1227
>     https://issues.apache.org/jira/browse/FLUME-1227
> 
> 
> Repository: flume-git
> 
> 
> Description
> -------
> 
> Revised design for Spillable Mem Channel.
> We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.
> 
> Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)
> 
> Essence of this design:
> - SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
> - SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
> - Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
> - Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.
> 
> 
> SC Configuration:
> - Accepts all the File Channel settings
> - Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
>  
> 
> Sample config ...
> 
> a1.channels = c1
> a1.sinks = logger
> a1.sources = src
> 
> a1.sources.src.type = exec
> a1.sources.src.command = seq 1 100000
> a1.sources.src.batchSize = 10
> a1.sources.src.channels = c1
> 
> a1.sinks.logger.type = logger
> a1.sinks.logger.channel = c1
> 
> a1.channels.c1.type = spillablememory
> a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
> a1.channels.c1.dataDirs = /tmp/flume/data
> a1.channels.c1.memoryCapacity = 10
> a1.channels.c1.keep-alive = 2
> 
> 
> Diffs
> -----
> 
>   flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
>   flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
>   flume-ng-channels/pom.xml 5832ab4 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
>   flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
>   flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
>   flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
>   flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
>   flume-ng-dist/pom.xml 2d0ee47 
>   flume-ng-doc/sphinx/FlumeUserGuide.rst 98859ce 
>   flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
>   flume-ng-node/pom.xml f1b0c65 
>   pom.xml 267925f 
> 
> Diff: https://reviews.apache.org/r/12060/diff/
> 
> 
> Testing
> -------
> 
> Wrote a set of Unit tests. A few are failing and need to be finished up.
> 
> 
> File Attachments
> ----------------
> 
> Revised design doc
>   https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf
> 
> 
> Thanks,
> 
> Roshan Naik
> 
>


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Roshan Naik <ro...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/
-----------------------------------------------------------

(Updated Dec. 18, 2013, 11:43 a.m.)


Review request for Flume.


Changes
-------

updating with patch v8.

This patch fixes the issues with
- the 80char line,
- The counter issue you noted plus other counters also fixed.. additional unit tests added to check counters.
- testTotalSemaphore failure.. (now renamed to testCounters)


Bugs: FLUME-1227
    https://issues.apache.org/jira/browse/FLUME-1227


Repository: flume-git


Description
-------

Revised design for Spillable Mem Channel.
We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.

Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)

Essence of this design:
- SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
- SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
- Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
- Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.


SC Configuration:
- Accepts all the File Channel settings
- Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
 

Sample config ...

a1.channels = c1
a1.sinks = logger
a1.sources = src

a1.sources.src.type = exec
a1.sources.src.command = seq 1 100000
a1.sources.src.batchSize = 10
a1.sources.src.channels = c1

a1.sinks.logger.type = logger
a1.sinks.logger.channel = c1

a1.channels.c1.type = spillablememory
a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
a1.channels.c1.dataDirs = /tmp/flume/data
a1.channels.c1.memoryCapacity = 10
a1.channels.c1.keep-alive = 2


Diffs (updated)
-----

  flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 2cd7f03 
  flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/pom.xml 5832ab4 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
  flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
  flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
  flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java f10a79f 
  flume-ng-dist/pom.xml 2d0ee47 
  flume-ng-doc/sphinx/FlumeUserGuide.rst 3088400 
  flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
  flume-ng-node/pom.xml f1b0c65 
  pom.xml 3c741c3 

Diff: https://reviews.apache.org/r/12060/diff/


Testing
-------

Wrote a set of Unit tests. A few are failing and need to be finished up.


File Attachments
----------------

Revised design doc
  https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf


Thanks,

Roshan Naik


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Roshan Naik <ro...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/
-----------------------------------------------------------

(Updated Nov. 5, 2013, 3:34 a.m.)


Review request for Flume.


Changes
-------

Adding test case to verify totalSemaphore usage is correct. Also updating channelCounter correctly on commit.


Bugs: FLUME-1227
    https://issues.apache.org/jira/browse/FLUME-1227


Repository: flume-git


Description
-------

Revised design for Spillable Mem Channel.
We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.

Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)

Essence of this design:
- SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
- SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
- Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
- Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.


SC Configuration:
- Accepts all the File Channel settings
- Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
 

Sample config ...

a1.channels = c1
a1.sinks = logger
a1.sources = src

a1.sources.src.type = exec
a1.sources.src.command = seq 1 100000
a1.sources.src.batchSize = 10
a1.sources.src.channels = c1

a1.sinks.logger.type = logger
a1.sinks.logger.channel = c1

a1.channels.c1.type = spillablememory
a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
a1.channels.c1.dataDirs = /tmp/flume/data
a1.channels.c1.memoryCapacity = 10
a1.channels.c1.keep-alive = 2


Diffs (updated)
-----

  flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
  flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/pom.xml 5832ab4 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
  flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
  flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
  flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
  flume-ng-dist/pom.xml 2d0ee47 
  flume-ng-doc/sphinx/FlumeUserGuide.rst 3a3038c 
  flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
  flume-ng-node/pom.xml f1b0c65 
  pom.xml 267925f 

Diff: https://reviews.apache.org/r/12060/diff/


Testing
-------

Wrote a set of Unit tests. A few are failing and need to be finished up.


File Attachments
----------------

Revised design doc
  https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf


Thanks,

Roshan Naik


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Roshan Naik <ro...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/
-----------------------------------------------------------

(Updated Oct. 31, 2013, 11:59 p.m.)


Review request for Flume.


Changes
-------

Addressed comments from last review.
- Marks the channel as experimental
- fixed various style issues
- fixes couple of bugs pointed out by hari


Bugs: FLUME-1227
    https://issues.apache.org/jira/browse/FLUME-1227


Repository: flume-git


Description
-------

Revised design for Spillable Mem Channel.
We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.

Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)

Essence of this design:
- SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
- SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
- Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
- Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.


SC Configuration:
- Accepts all the File Channel settings
- Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
 

Sample config ...

a1.channels = c1
a1.sinks = logger
a1.sources = src

a1.sources.src.type = exec
a1.sources.src.command = seq 1 100000
a1.sources.src.batchSize = 10
a1.sources.src.channels = c1

a1.sinks.logger.type = logger
a1.sinks.logger.channel = c1

a1.channels.c1.type = spillablememory
a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
a1.channels.c1.dataDirs = /tmp/flume/data
a1.channels.c1.memoryCapacity = 10
a1.channels.c1.keep-alive = 2


Diffs (updated)
-----

  flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
  flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/pom.xml 5832ab4 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
  flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
  flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
  flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
  flume-ng-dist/pom.xml 2d0ee47 
  flume-ng-doc/sphinx/FlumeUserGuide.rst 3a3038c 
  flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
  flume-ng-node/pom.xml f1b0c65 
  pom.xml 267925f 

Diff: https://reviews.apache.org/r/12060/diff/


Testing
-------

Wrote a set of Unit tests. A few are failing and need to be finished up.


File Attachments
----------------

Revised design doc
  https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf


Thanks,

Roshan Naik


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Roshan Naik <ro...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/
-----------------------------------------------------------

(Updated Oct. 24, 2013, 10:18 p.m.)


Review request for Flume.


Changes
-------

updating new patch which incorporates the feedback.


Bugs: FLUME-1227
    https://issues.apache.org/jira/browse/FLUME-1227


Repository: flume-git


Description
-------

Revised design for Spillable Mem Channel.
We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.

Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)

Essence of this design:
- SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
- SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
- Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
- Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.


SC Configuration:
- Accepts all the File Channel settings
- Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
 

Sample config ...

a1.channels = c1
a1.sinks = logger
a1.sources = src

a1.sources.src.type = exec
a1.sources.src.command = seq 1 100000
a1.sources.src.batchSize = 10
a1.sources.src.channels = c1

a1.sinks.logger.type = logger
a1.sinks.logger.channel = c1

a1.channels.c1.type = spillablememory
a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
a1.channels.c1.dataDirs = /tmp/flume/data
a1.channels.c1.memoryCapacity = 10
a1.channels.c1.keep-alive = 2


Diffs (updated)
-----

  flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
  flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/pom.xml 5832ab4 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
  flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
  flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
  flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
  flume-ng-dist/pom.xml 2d0ee47 
  flume-ng-doc/sphinx/FlumeUserGuide.rst 98859ce 
  flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
  flume-ng-node/pom.xml f1b0c65 
  pom.xml 267925f 

Diff: https://reviews.apache.org/r/12060/diff/


Testing
-------

Wrote a set of Unit tests. A few are failing and need to be finished up.


File Attachments
----------------

Revised design doc
  https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf


Thanks,

Roshan Naik


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Roshan Naik <ro...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/
-----------------------------------------------------------

(Updated Oct. 24, 2013, 10:16 p.m.)


Review request for Flume.


Changes
-------

Addressing review comments.


Bugs: FLUME-1227
    https://issues.apache.org/jira/browse/FLUME-1227


Repository: flume-git


Description
-------

Revised design for Spillable Mem Channel.
We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.

Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)

Essence of this design:
- SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
- SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
- Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
- Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.


SC Configuration:
- Accepts all the File Channel settings
- Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
 

Sample config ...

a1.channels = c1
a1.sinks = logger
a1.sources = src

a1.sources.src.type = exec
a1.sources.src.command = seq 1 100000
a1.sources.src.batchSize = 10
a1.sources.src.channels = c1

a1.sinks.logger.type = logger
a1.sinks.logger.channel = c1

a1.channels.c1.type = spillablememory
a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
a1.channels.c1.dataDirs = /tmp/flume/data
a1.channels.c1.memoryCapacity = 10
a1.channels.c1.keep-alive = 2


Diffs (updated)
-----

  flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
  flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/pom.xml 5832ab4 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
  flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
  flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
  flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
  flume-ng-dist/pom.xml 2d0ee47 
  flume-ng-doc/sphinx/FlumeUserGuide.rst 98859ce 
  flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
  flume-ng-node/pom.xml f1b0c65 
  pom.xml 267925f 

Diff: https://reviews.apache.org/r/12060/diff/


Testing
-------

Wrote a set of Unit tests. A few are failing and need to be finished up.


File Attachments
----------------

Revised design doc
  https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf


Thanks,

Roshan Naik


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Roshan Naik <ro...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/
-----------------------------------------------------------

(Updated Oct. 15, 2013, 8:44 p.m.)


Review request for Flume.


Changes
-------

Updating patch... incorporates review comments. Rebasing patch.


Bugs: FLUME-1227
    https://issues.apache.org/jira/browse/FLUME-1227


Repository: flume-git


Description
-------

Revised design for Spillable Mem Channel.
We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.

Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)

Essence of this design:
- SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
- SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
- Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
- Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.


SC Configuration:
- Accepts all the File Channel settings
- Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
 

Sample config ...

a1.channels = c1
a1.sinks = logger
a1.sources = src

a1.sources.src.type = exec
a1.sources.src.command = seq 1 100000
a1.sources.src.batchSize = 10
a1.sources.src.channels = c1

a1.sinks.logger.type = logger
a1.sinks.logger.channel = c1

a1.channels.c1.type = spillablememory
a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
a1.channels.c1.dataDirs = /tmp/flume/data
a1.channels.c1.memoryCapacity = 10
a1.channels.c1.keep-alive = 2


Diffs (updated)
-----

  flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
  flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/pom.xml 5832ab4 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
  flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
  flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
  flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
  flume-ng-dist/pom.xml 2d0ee47 
  flume-ng-doc/sphinx/FlumeUserGuide.rst 98859ce 
  flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
  flume-ng-node/pom.xml f1b0c65 
  pom.xml 267925f 

Diff: https://reviews.apache.org/r/12060/diff/


Testing
-------

Wrote a set of Unit tests. A few are failing and need to be finished up.


File Attachments
----------------

Revised design doc
  https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf


Thanks,

Roshan Naik


Re: Review Request 12060: Revised design for Spillable Mem Channel

Posted by Roshan Naik <ro...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/
-----------------------------------------------------------

(Updated July 3, 2013, 11:56 p.m.)


Review request for Flume.


Changes
-------

Updating patch :
+ Added support for recovering the overflow data when agent recovers from a crash.
+ Added documentation 


Bugs: FLUME-1227
    https://issues.apache.org/jira/browse/FLUME-1227


Repository: flume-git


Description
-------

Revised design for Spillable Mem Channel.
We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.

Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)

Essence of this design:
- SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
- SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
- Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
- Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.


SC Configuration:
- Accepts all the File Channel settings
- Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
 

Sample config ...

a1.channels = c1
a1.sinks = logger
a1.sources = src

a1.sources.src.type = exec
a1.sources.src.command = seq 1 100000
a1.sources.src.batchSize = 10
a1.sources.src.channels = c1

a1.sinks.logger.type = logger
a1.sinks.logger.channel = c1

a1.channels.c1.type = spillablememory
a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
a1.channels.c1.dataDirs = /tmp/flume/data
a1.channels.c1.memoryCapacity = 10
a1.channels.c1.keep-alive = 2


Diffs (updated)
-----

  flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
  flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/pom.xml 5832ab4 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
  flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
  flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
  flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
  flume-ng-dist/pom.xml 83332a9 
  flume-ng-doc/sphinx/FlumeUserGuide.rst 63cad21 
  flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
  flume-ng-node/pom.xml f1b0c65 
  pom.xml 15e6d9b 

Diff: https://reviews.apache.org/r/12060/diff/


Testing
-------

Wrote a set of Unit tests. A few are failing and need to be finished up.


File Attachments
----------------

Revised design doc
  https://reviews.apache.org/media/uploaded/files/2013/06/24/SpillableMemory_Channel_Design_2_1.pdf


Thanks,

Roshan Naik


Re: Review Request: Revised design for Spillable Mem Channel

Posted by Roshan Naik <ro...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/
-----------------------------------------------------------

(Updated June 24, 2013, 11:41 p.m.)


Review request for Flume.


Changes
-------

Design doc for this patch


Description
-------

Revised design for Spillable Mem Channel.
We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.

Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)

Essence of this design:
- SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
- SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
- Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
- Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.


SC Configuration:
- Accepts all the File Channel settings
- Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
 

Sample config ...

a1.channels = c1
a1.sinks = logger
a1.sources = src

a1.sources.src.type = exec
a1.sources.src.command = seq 1 100000
a1.sources.src.batchSize = 10
a1.sources.src.channels = c1

a1.sinks.logger.type = logger
a1.sinks.logger.channel = c1

a1.channels.c1.type = spillablememory
a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
a1.channels.c1.dataDirs = /tmp/flume/data
a1.channels.c1.memoryCapacity = 10
a1.channels.c1.keep-alive = 2


This addresses bug FLUME-1227.
    https://issues.apache.org/jira/browse/FLUME-1227


Diffs
-----

  flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
  flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/pom.xml 6485d47 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
  flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
  flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
  flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
  flume-ng-dist/pom.xml f20ff1e 
  flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
  flume-ng-node/pom.xml 5cdd9e2 
  flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java e63c601 
  pom.xml 7f6f824 

Diff: https://reviews.apache.org/r/12060/diff/


Testing
-------

Wrote a set of Unit tests. A few are failing and need to be finished up.


Thanks,

Roshan Naik


Re: Review Request: Revised design for Spillable Mem Channel

Posted by Roshan Naik <ro...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/
-----------------------------------------------------------

(Updated June 24, 2013, 10:30 a.m.)


Review request for Flume.


Changes
-------

added a note on the new configuration method for SC


Description (updated)
-------

Revised design for Spillable Mem Channel.
We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.

Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)

Essence of this design:
- SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
- SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
- Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
- Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.


SC Configuration:
- Accepts all the File Channel settings
- Introduces one additional setting: 'memoryCapacity' which indicates the number of items it can hold in memory
 

Sample config ...

a1.channels = c1
a1.sinks = logger
a1.sources = src

a1.sources.src.type = exec
a1.sources.src.command = seq 1 100000
a1.sources.src.batchSize = 10
a1.sources.src.channels = c1

a1.sinks.logger.type = logger
a1.sinks.logger.channel = c1

a1.channels.c1.type = spillablememory
a1.channels.c1.checkpointDir = /tmp/flume/checkpoint
a1.channels.c1.dataDirs = /tmp/flume/data
a1.channels.c1.memoryCapacity = 10
a1.channels.c1.keep-alive = 2


This addresses bug FLUME-1227.
    https://issues.apache.org/jira/browse/FLUME-1227


Diffs
-----

  flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
  flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/pom.xml 6485d47 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
  flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
  flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
  flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
  flume-ng-dist/pom.xml f20ff1e 
  flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
  flume-ng-node/pom.xml 5cdd9e2 
  flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java e63c601 
  pom.xml 7f6f824 

Diff: https://reviews.apache.org/r/12060/diff/


Testing
-------

Wrote a set of Unit tests. A few are failing and need to be finished up.


Thanks,

Roshan Naik


Re: Review Request: Revised design for Spillable Mem Channel

Posted by Roshan Naik <ro...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/12060/
-----------------------------------------------------------

(Updated June 24, 2013, 10:24 a.m.)


Review request for Flume.


Description
-------

Revised design for Spillable Mem Channel.
We no longer have Spillable channel config pointing to another channel (by name) as in the previous design.

Spillable Channel instead derives from FileChannel (as per https://issues.apache.org/jira/browse/FLUME-1227?focusedCommentId=13628201&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13628201)

Essence of this design:
- SC derives from File channel and maintains an in memory queue. If memory queue is full, events are sent to disk overflow (i.e. File channel).
- SC maintains a 'Drain-Order' queue (DOQ) for remembering the order in which the incoming events were interleaved between main memory and 
- Put transaction: All the elements in the putList are committed to mem queue if it has space, else written to disk (ie file channel). Head of DOQ is updated to indicate where the elements put.
- Take transaction : Tail of DOQ is consulted to determine whether the next set of events are to be taken from the memory queue or from disk overflow. DOQ's tail is updated after events are taken out.

 


This addresses bug FLUME-1227.
    https://issues.apache.org/jira/browse/FLUME-1227


Diffs
-----

  flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java 36f150b 
  flume-ng-channels/flume-spillable-memory-channel/pom.xml PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java PRE-CREATION 
  flume-ng-channels/pom.xml 6485d47 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java 26f4dd7 
  flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java 15b8cc3 
  flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java PRE-CREATION 
  flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java 1370e66 
  flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java 688323d 
  flume-ng-dist/pom.xml f20ff1e 
  flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java 6204bc5 
  flume-ng-node/pom.xml 5cdd9e2 
  flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java e63c601 
  pom.xml 7f6f824 

Diff: https://reviews.apache.org/r/12060/diff/


Testing
-------

Wrote a set of Unit tests. A few are failing and need to be finished up.


Thanks,

Roshan Naik