Review Board 1.7.22


First patch for Spillable Channel.. capturing feedback.

Review Request #9544 - Created Feb. 21, 2013 and discarded

Roshan Naik
flume-1.4
https://issues.apache.org/jira/browse/FLUME-1227
Reviewers
Flume
flume-git
Initial patch for capturing feedback on SpillableChannel.

Open Issues:  
a) Config & Overflow Specification : 
It currently allows configuration (as shown below)  for selecting any other channel as a overflow. The current config subsystem doesn't support this mechanism well. To make this happen, i have modified the config subsystem. A postconfig step is introduced to enable a channel to obtain a reference to another configured channel by name. 
 One suggested alternative has been to hard code the file channel as the overflow channel at compile time. Downside would be that if a new SSD optimized channel is introduced in the future, the users of Spillable channel would not be able to take advantage of it.

b) Max Transaction Size: This implementation requires a config param  maxTransactionBatchSize, which is the size of the largest batchSize setting used by all the sinks and sources attached to this channel. This helps a good deal in optimizing the implementation of put() and take() internally. Consequently, unlike the transactionSize used in other channels, this value should be a much smaller and closer to the actual batchSize settings in the sources/sinks. Hari had mentioned some concerns in this regard which I think had something to do with not all sources being able to know their batchSize upfront. I not totally clear on why it is the case. Would appreciate more insight. 

Notes:

a) Basic algorithm: This implementation avoids copying events between the in-memory queue and the overflow for better performance. at the time of put(), a decision is made to either put the entire transaction in memory or into overflow based on maxTransactionSize  and slots available in memory queue.  Another queue (called 'drain order queue') is used to memorize the order in which the events were inserted. For example these values in the  drain order queue ... +1000,-200,+400,-100 ... indicate the first 1000 elements are in memory, next 200 (the -ve number) are in overflow, next 400 are in memory and so on. This is used at the time of take() to drain elements in the right order directly from memory/overflow and avoid copying back and forth between memory & overflow. Any put/take Transaction will restricted to either the memory or overflow. Allowing it to operate on both will violate correctness of rollback/commit by requiring nested transactions.

b) Performance: Currently my measurements are showing decent performance improvements over the MemoryChannel when there is no overflow occurring (between 7% and 70% depending on number of sources/sinks attached).  In the case there is overflow, dont have much measurements to provide right now. this will also depend on what we do with point a)



# Name the components on this agent
agent1.sources = source1
agent1.sinks = sink1
agent1.channels = spillChannel1  fChannel

agent1.sources.source1.type = SEQ
agent1.sources.source1.batchSize = 10

agent1.sources.source1.channels = spillChannel1

agent1.sinks.sink1.type = null
agent1.sinks.sink1.channel = spillChannel1
agent1.sinks.sink1.batchSize = 10

agent1.channels.spillChannel1.type = org.apache.flume.channel.SpillableMemoryChannel
agent1.channels.spillChannel1.maxTransactionBatchSize = 10
agent1.channels.spillChannel1.overflowChannel = fChannel
agent1.channels.spillChannel1.memoryCapacity = 100   # memory only      
agent1.channels.spillChannel1.totalCapacity = 10000    # memory + overflow
agent1.channels.spillChannel1.keep-alive = 3


agent1.channels.fChannel.type = file
agent1.channels.fChannel.checkpointDir = /tmp/fchannel/checkpoint
agent1.channels.fChannel.dataDirs = /tmp/fchannel/data
agent1.channels.fChannel.keep-alive = 0
agent1.channels.fChannel.capacity = 10000
Tests have been included in this patch.
pom.xml
Revision 36f989d New Change
[20] 890 lines
[+20]
891
        <version>1.4.0-SNAPSHOT</version>
891
        <version>1.4.0-SNAPSHOT</version>
892
      </dependency>
892
      </dependency>
893

    
   
893

   
894
     <dependency>
894
      <dependency>
895
       <groupId>org.apache.flume.flume-ng-channels</groupId>
895
        <groupId>org.apache.flume.flume-ng-channels</groupId>

    
   
896
        <artifactId>flume-spillable-memory-channel</artifactId>

    
   
897
        <version>1.4.0-SNAPSHOT</version>

    
   
898
      </dependency>

    
   
899

   

    
   
900
     <dependency>

    
   
901
       <groupId>org.apache.flume.flume-ng-channels</groupId>
896
       <artifactId>flume-recoverable-memory-channel</artifactId>
902
       <artifactId>flume-recoverable-memory-channel</artifactId>
897
       <version>1.4.0-SNAPSHOT</version>
903
       <version>1.4.0-SNAPSHOT</version>
898
     </dependency>
904
     </dependency>
899

    
   
905

   
900
      <dependency>
906
      <dependency>
[+20] [20] 155 lines
flume-ng-channels/pom.xml
Revision 5c6fa76 New Change
 
flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java
Revision d98209b New Change
 
flume-ng-channels/flume-spillable-memory-channel/pom.xml
New File
 
flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java
New File
 
flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java
New File
 
flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java
Revision 1e1a46f New Change
 
flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java
Revision 475341d New Change
 
flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java
New File
 
flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java
Revision 1370e66 New Change
 
flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java
Revision 688323d New Change
 
flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java
Revision 6204bc5 New Change
 
flume-ng-node/pom.xml
Revision 035ae06 New Change
 
flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java
Revision e63c601 New Change
 
  1. pom.xml: Loading...
  2. flume-ng-channels/pom.xml: Loading...
  3. flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java: Loading...
  4. flume-ng-channels/flume-spillable-memory-channel/pom.xml: Loading...
  5. flume-ng-channels/flume-spillable-memory-channel/src/main/java/org/apache/flume/channel/SpillableMemoryChannel.java: Loading...
  6. flume-ng-channels/flume-spillable-memory-channel/src/test/java/org/apache/flume/channel/TestSpillableMemoryChannel.java: Loading...
  7. flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelConfiguration.java: Loading...
  8. flume-ng-configuration/src/main/java/org/apache/flume/conf/channel/ChannelType.java: Loading...
  9. flume-ng-core/src/main/java/org/apache/flume/ChannelFullException.java: Loading...
  10. flume-ng-core/src/main/java/org/apache/flume/channel/AbstractChannel.java: Loading...
  11. flume-ng-core/src/main/java/org/apache/flume/channel/MemoryChannel.java: Loading...
  12. flume-ng-embedded-agent/src/main/java/org/apache/flume/agent/embedded/EmbeddedAgentConfiguration.java: Loading...
  13. flume-ng-node/pom.xml: Loading...
  14. flume-ng-node/src/main/java/org/apache/flume/node/AbstractConfigurationProvider.java: Loading...