> 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
> 
>

Reply via email to