[ 
https://issues.apache.org/jira/browse/FLUME-1085?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13257391#comment-13257391
 ] 

[email protected] commented on FLUME-1085:
------------------------------------------------------


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


Thanks for the patch Brock. The implementation looks great to me! Some feedback 
follows:

Some convention nits:
* Please make sure the line lengths are under 80 chars.
* Please use all caps for variables that are static final.

One high-level consideration: Since it is possible to have multiple file 
channels within the same agent, we should either make file channel multi-tenant 
capable like the jdbc channel, or instead have the ability to associate 
directory locks for the file channel instance for checkpoint and data 
directories. Doing this will ensure that there is no corruption and the system 
exits cleanly in case of misconfiguration.

If you chose to implement the non-multitenant system, it will be preferable to 
use the channel name within the default paths for checkpoint and data 
directories. This will ensure minimum configuration necessary even when using 
multiple file channels within the same agent.




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

    Please use space instead of tab.



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

    The initialization of the Log instance along with replaying it should be 
done in the start() method. Correspondingly, the stop() method should shutdown 
the log and release all associated resources.



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

    It would be better to add the shutdown hook in the start() method instead. 
Also, in the stop() the shutdown hook should be removed to avoid spurious 
exceptions.
    



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

    wish: it would be better to have this as a thread local to safeguard 
against transaction leaks.  



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

    an IllegalStateException with a message indicating the scope of the 
damage/corruption would be appropriate.



flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java
<https://reviews.apache.org/r/4661/#comment15057>

    considering this class is called Log, it may be better to call this member 
LOGGER or DEBUG etc.



flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java
<https://reviews.apache.org/r/4661/#comment15058>

    Suggest using public accessor.



flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionEventRecord.java
<https://reviews.apache.org/r/4661/#comment15059>

    Please use enums instead.



flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionEventRecord.java
<https://reviews.apache.org/r/4661/#comment15060>

    Better to use IllegalStateException with a descriptive message.



flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionEventRecord.java
<https://reviews.apache.org/r/4661/#comment15061>

    Better to use IllegalStateException with descriptive message.



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

    Please remove tab.



flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java
<https://reviews.apache.org/r/4661/#comment15608>

    Not a request, but something to keep an eye on: this implementation uses 
the modulo operation heavily to calculate the queue positions. The alternative 
way of doing this would be to maintain head, size and optionally tail pointers. 
That will reduce the number of overall modulo operations needed. 
    
    The only reason I point this out is that modulo is CPU intensive for large 
numbers and therefore may become a concern for performance.



flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java
<https://reviews.apache.org/r/4661/#comment15607>

    From cursory analysis it seems that this will likely not be able to handle 
the wrap-around logic correctly. For example, if the capacity is 10, next is 9 
and size/index is 1: the calculated index will be 10, when it should be 0.
    
    One way to address this would be modulo the converted value before return:
    
    return (next + index % elements.capacity()) % elements.capacity();
    



flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java
<https://reviews.apache.org/r/4661/#comment15574>

    To work out of the box, this directory would need to be created if it does 
not exist. That would be similar to the behavior of JDBC channel as well.



flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java
<https://reviews.apache.org/r/4661/#comment15575>

    Same as above - will need to create the directory if it does not exist.


- Arvind


On 2012-04-17 06:36:33, Brock Noland wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/4661/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2012-04-17 06:36:33)
bq.  
bq.  
bq.  Review request for Flume.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  This patch implements a durable file channel. It does by writing all 
transaction events to disk and syncing to disk when a commit occurs. It does 
have a memory component in that pointers to the event on disk are kept in 
memory. This will consume 8 bytes of direct memory (non-heap) per event. Some 
basic calculations are in the FileChannel java docs.
bq.  
bq.  
bq.  This addresses bug FLUME-1085.
bq.      https://issues.apache.org/jira/browse/FLUME-1085
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    flume-ng-channels/flume-file-channel/pom.xml e8155be 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Checkpoint.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Commit.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java
 a279453 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannelConfiguration.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEvent.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventPointer.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FlumeEventQueue.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Log.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogFile.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/LogUtils.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Pair.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Put.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/ReplayHandler.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Rollback.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/Take.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/TransactionEventRecord.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/CountingSinkRunner.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/CountingSourceRunner.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestCheckpoint.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFileChannel.java
 ab66998 
bq.    
flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFlumeEvent.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFlumeEventPointer.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestFlumeEventQueue.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLog.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestLogFile.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestTransactionEventRecord.java
 PRE-CREATION 
bq.    
flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestUtils.java
 PRE-CREATION 
bq.    flume-ng-channels/flume-file-channel/src/test/resources/log4j.properties 
739ecc8 
bq.    
flume-ng-channels/flume-recoverable-memory-channel/src/main/java/org/apache/flume/channel/recoverable/memory/RecoverableMemoryChannel.java
 0622f27 
bq.    
flume-ng-channels/flume-recoverable-memory-channel/src/main/java/org/apache/flume/channel/recoverable/memory/wal/SequenceIDBuffer.java
 fa63b73 
bq.    flume-ng-core/src/main/java/org/apache/flume/channel/ChannelType.java 
d8419e8 
bq.    flume-ng-core/src/main/java/org/apache/flume/sink/NullSink.java c812851 
bq.    flume-ng-core/src/main/java/org/apache/flume/source/StressSource.java 
PRE-CREATION 
bq.    
flume-ng-core/src/main/java/org/apache/flume/tools/DirectMemoryUtils.java 
PRE-CREATION 
bq.    
flume-ng-sdk/src/test/java/org/apache/flume/api/TestFailoverRpcClient.java 
225cd34 
bq.    
flume-ng-sdk/src/test/java/org/apache/flume/api/TestNettyAvroRpcClient.java 
0b8a2c0 
bq.  
bq.  Diff: https://reviews.apache.org/r/4661/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Unit tests and integration tests added to cover obvious cases.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  Brock
bq.  
bq.


                
> Implement a durable FileChannel
> -------------------------------
>
>                 Key: FLUME-1085
>                 URL: https://issues.apache.org/jira/browse/FLUME-1085
>             Project: Flume
>          Issue Type: New Feature
>          Components: Channel
>    Affects Versions: v1.2.0
>            Reporter: Brock Noland
>            Assignee: Brock Noland
>
> FLUME-896 turned into a durable Memory Channel, we need a durable File Channel

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to