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