Re: [DISCUSS] Flume 1.3.0 release
Yes, thats a bigger point. I saw the checkin today, also I agree with FLUME-1629. FLUME-1660 could be moved behind? I'll work through some open reviews today. +1 for 1.3.0 Thanks, Alex On Nov 6, 2012, at 2:45 AM, Mike Percy wrote: > Personally I'd like to get FLUME-1425 in. I'm trying to get it checked in > ASAP. I think we should also pull in FLUME-1629 and FLUME-1660. We're a bit > ahead on patches and behind on reviews at the moment. > > Regards > Mike > > On Mon, Nov 5, 2012 at 10:57 AM, Roshan Naik wrote: > >> Thought of checking back on the availability of the Release Candidate.. any >> ETA ? >> >> -roshan >> -- Alexander Alten-Lorenz http://mapredit.blogspot.com German Hadoop LinkedIn Group: http://goo.gl/N8pCF
Re: Review Request: FLUME-1425: Create a SpoolDirectory Source and Client
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/6377/ --- (Updated Nov. 6, 2012, 6:34 a.m.) Review request for Flume. Changes --- This is a small patch that should fix the unit test issues. I need someone to run on Mac to confirm that this works. Description --- This patch adds a spooling directory based source. The idea is that a user can have a spool directory where files are deposited for ingestion into flume. Once ingested, the files are clearly renamed and the implementation guarantees at-least-once delivery semantics similar to those achieved within flume itself, even across failures and restarts of the JVM running the code. This helps fill the gap for people who want a way to get reliable delivery of events into flume, but don't want to directly write their application against the flume API. They can simply drop log files off in a spooldir and let flume ingest asynchronously (using some shell scripts or other automated process). Unlike the prior iteration, this patch implements a first-class source. It also extends the avro client to support spooling in a similar manner. This addresses bug FlUME-1425. https://issues.apache.org/jira/browse/FlUME-1425 Diffs (updated) - flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java da804d7 flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceType.java abbbf1c flume-ng-core/src/main/java/org/apache/flume/client/avro/AvroCLIClient.java 4a5ecae flume-ng-core/src/main/java/org/apache/flume/client/avro/BufferedLineReader.java PRE-CREATION flume-ng-core/src/main/java/org/apache/flume/client/avro/LineReader.java PRE-CREATION flume-ng-core/src/main/java/org/apache/flume/client/avro/SpoolingFileLineReader.java PRE-CREATION flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java PRE-CREATION flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java PRE-CREATION flume-ng-core/src/test/java/org/apache/flume/client/avro/TestBufferedLineReader.java PRE-CREATION flume-ng-core/src/test/java/org/apache/flume/client/avro/TestSpoolingFileLineReader.java PRE-CREATION flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java PRE-CREATION flume-ng-doc/sphinx/FlumeUserGuide.rst 953a670 Diff: https://reviews.apache.org/r/6377/diff/ Testing --- Extensive unit tests and I also built and played with this using a stub flume agent. If you look at the JIRA I have a configuration file for an agent that will print out Avro events to the command line - that's helpful when testing this. Thanks, Patrick Wendell
[jira] [Updated] (FLUME-1425) Create a SpoolDirectory Source and Client
[ https://issues.apache.org/jira/browse/FLUME-1425?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Patrick Wendell updated FLUME-1425: --- Attachment: FLUME-1425.v9.patch.txt This patch addresses a bug in the way that file timestamps are treated in the unit tests. Due to varying time granularity in filesystems, tests had inconsistent results. This should fix that error. > Create a SpoolDirectory Source and Client > - > > Key: FLUME-1425 > URL: https://issues.apache.org/jira/browse/FLUME-1425 > Project: Flume > Issue Type: Improvement >Reporter: Patrick Wendell >Assignee: Patrick Wendell > Fix For: v1.3.0 > > Attachments: FileProcessingSource.java, > FLUME-1425.avro-conf-file.txt, FLUME-1425.patch.v1.txt, > FLUME-1425.v5.patch.txt, FLUME-1425.v6.patch.txt, FLUME-1425.v6.patch.txt, > FLUME-1425.v7.patch.txt, FLUME-1425.v8.patch.txt, FLUME-1425.v9.patch.txt > > > The proposal is to create a small executable client which reads logs from a > spooling directory and sends them to a flume sink, then performs cleanup on > the directory (either by deleting or moving the logs). It would make the > following assumptions > - Files placed in the directory are uniquely named > - Files placed in the directory are immutable > The problem this is trying to solve is that there is currently no way to do > guaranteed event delivery across flume agent restarts when the data is being > collected through an asynchronous source (and not directly from the client > API). Say, for instance, you are using a exec("tail -F") source. If the agent > restarts due to error or intentionally, tail may pick up at a new location > and you lose the intermediate data. > At the same time, there are users who want at-least-once semantics, and > expect those to apply as soon as the data is written to disk from the initial > logger process (e.g. apache logs), not just once it has reached a flume > agent. This idea would bridge that gap, assuming the user is able to copy > immutable logs to a spooling directory through a cron script or something. > The basic internal logic of such a client would be as follows: > - Scan the directory for files > - Chose a file and read through, while sending events to an agent > - Close the file and delete it (or rename, or otherwise mark completed) > That's about it. We could add sync-points to make recovery more efficient in > the case of failure. > A key question is whether this should be implemented as a standalone client > or as a source. My instinct is actually to do this as a source, but there > could be some benefit to not requiring an entire agent in order to run this, > specifically that it would become platform independent and you could stick it > on Windows machines. Others I have talked to have also sided on a standalone > executable. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (FLUME-1502) Support for running simple configurations embedded in host process
[ https://issues.apache.org/jira/browse/FLUME-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13491214#comment-13491214 ] Ralph Goers commented on FLUME-1502: Yes, what is really required is the call to getChannelProcessor().processEvent(event). It didn't occur to me to see how to locate the ChannelProcessor(s) instead of a Source. As for the FileChannel being slow - our tests showed the non-embedded agent (i.e. Avro) takes just less than .1 seconds per event while the embedded agent takes between .015 and .001 seconds per event (interestingly, it got faster as more events were written). I would expect this is fine for a single container. That said, I have nothing against a simpler implementation of channel with guaranteed delivery. > Support for running simple configurations embedded in host process > -- > > Key: FLUME-1502 > URL: https://issues.apache.org/jira/browse/FLUME-1502 > Project: Flume > Issue Type: Improvement >Affects Versions: v1.2.0 >Reporter: Arvind Prabhakar >Assignee: Brock Noland > Attachments: embeeded-agent-1.pdf > > > Flume should provide a light-weight embeddable node manager that can be > started in process where necessary. This will allow the users to embed > light-weight agents within the host process where necessary. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (FLUME-1227) Introduce some sort of SpillableChannel
[ https://issues.apache.org/jira/browse/FLUME-1227?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13491209#comment-13491209 ] Mike Percy commented on FLUME-1227: --- I don't know of anyone actively working on this... > Introduce some sort of SpillableChannel > --- > > Key: FLUME-1227 > URL: https://issues.apache.org/jira/browse/FLUME-1227 > Project: Flume > Issue Type: New Feature > Components: Channel >Reporter: Jarek Jarcec Cecho >Assignee: Patrick Wendell > > I would like to introduce new channel that would behave similarly as scribe > (https://github.com/facebook/scribe). It would be something between memory > and file channel. Input events would be saved directly to the memory (only) > and would be served from there. In case that the memory would be full, we > would outsource the events to file. > Let me describe the use case behind this request. We have plenty of frontend > servers that are generating events. We want to send all events to just > limited number of machines from where we would send the data to HDFS (some > sort of staging layer). Reason for this second layer is our need to decouple > event aggregation and front end code to separate machines. Using memory > channel is fully sufficient as we can survive lost of some portion of the > events. However in order to sustain maintenance windows or networking issues > we would have to end up with a lot of memory assigned to those "staging" > machines. Referenced "scribe" is dealing with this problem by implementing > following logic - events are saved in memory similarly as our MemoryChannel. > However in case that the memory gets full (because of maintenance, networking > issues, ...) it will spill data to disk where they will be sitting until > everything start working again. > I would like to introduce channel that would implement similar logic. It's > durability guarantees would be same as MemoryChannel - in case that someone > would remove power cord, this channel would lose data. Based on the > discussion in FLUME-1201, I would propose to have the implementation > completely independent on any other channel internal code. > Jarcec -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (FLUME-1502) Support for running simple configurations embedded in host process
[ https://issues.apache.org/jira/browse/FLUME-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13491208#comment-13491208 ] Mike Percy commented on FLUME-1502: --- Heh, Arvind, clearly I should read more slowly since I think what I said exactly matches what you said. Ralph, not sure why you are using a source in this case, other than that's all you could get access to, which is understandable. I believe what you really wanted all along was a channel connected to a sink, right? Also, worth noting that the File channel without multiple puts per transaction is dog slow, due to the fsync() call. So we definitely need to expose some type of batch interface. > Support for running simple configurations embedded in host process > -- > > Key: FLUME-1502 > URL: https://issues.apache.org/jira/browse/FLUME-1502 > Project: Flume > Issue Type: Improvement >Affects Versions: v1.2.0 >Reporter: Arvind Prabhakar >Assignee: Brock Noland > Attachments: embeeded-agent-1.pdf > > > Flume should provide a light-weight embeddable node manager that can be > started in process where necessary. This will allow the users to embed > light-weight agents within the host process where necessary. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (FLUME-1502) Support for running simple configurations embedded in host process
[ https://issues.apache.org/jira/browse/FLUME-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13491201#comment-13491201 ] Ralph Goers commented on FLUME-1502: Arvind, it would be hard for me to disagree with your point of view since it matches exactly with how the embedded agent is currently implemented in Log4j 2. The current incarnation allows any channels or sinks, but all channels are automatically connected to the source (which is really the appender itself). As for simplicity, my intention with Log4j 2 is to have the configuration end up looking like and then maybe something to configure encryption. Although Log4j 2 will support configuration by Flume properties, my guess is that most people would prefer the default configuration. If they don't want the FileChannel it would be fairly simple to add channel="Memory|File" as an attribute. If you look at the embedded Appender you will see that it creates the FlumeEvent (using code common to both the embedded and non-embedded versions) and then does public void send(FlumeEvent event) { sourceCounter.incrementAppendReceivedCount(); sourceCounter.incrementEventReceivedCount(); try { getChannelProcessor().processEvent(event); } catch (ChannelException ex) { logger.warn("Unabled to process event {}" + event, ex); throw ex; } sourceCounter.incrementAppendAcceptedCount(); sourceCounter.incrementEventAcceptedCount(); } in a class that extends AbstractSource. The only real challenge I had was in getting it into the Flume configuration so Flume could create the Source object and then obtaining a reference to the object so the Appender could call it. To do that I had to do SourceRunner runner = node.getConfiguration().getSourceRunners().get(SOURCE_NAME); if (runner == null || runner.getSource() == null) { throw new IllegalStateException("No Source has been created for Appender " + shortName); } source = (Log4jEventSource) runner.getSource(); It would be much better if I could pass the Source class to the configuration processor and get back a Source instance when the embedded agent is started. In this csae I would recommend the source has to implement something like public interface EmbeddedSource { void send(FlumeEvent event); } OTOH, you could just provide EmbeddedSource that implements send() as shown above. > Support for running simple configurations embedded in host process > -- > > Key: FLUME-1502 > URL: https://issues.apache.org/jira/browse/FLUME-1502 > Project: Flume > Issue Type: Improvement >Affects Versions: v1.2.0 >Reporter: Arvind Prabhakar >Assignee: Brock Noland > Attachments: embeeded-agent-1.pdf > > > Flume should provide a light-weight embeddable node manager that can be > started in process where necessary. This will allow the users to embed > light-weight agents within the host process where necessary. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (FLUME-1502) Support for running simple configurations embedded in host process
[ https://issues.apache.org/jira/browse/FLUME-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13491180#comment-13491180 ] Mike Percy commented on FLUME-1502: --- Agreed that we will need File Channel. A common problem is an application using the client SDK library needing to buffer its own events. File Channel would alleviate that in a durable way. Another thing to consider regarding skipping the source altogether and allowing an interface to the Channel such that an application could open a Transaction, put() events on the channel, and commit()/close() the channel. This would make the so-called embedded agent basically a glorified client. But that's the use case I think this is morphing into. In such a case I think we should consider disallowing take() calls, but that's a secondary point. Thoughts? > Support for running simple configurations embedded in host process > -- > > Key: FLUME-1502 > URL: https://issues.apache.org/jira/browse/FLUME-1502 > Project: Flume > Issue Type: Improvement >Affects Versions: v1.2.0 >Reporter: Arvind Prabhakar >Assignee: Brock Noland > Attachments: embeeded-agent-1.pdf > > > Flume should provide a light-weight embeddable node manager that can be > started in process where necessary. This will allow the users to embed > light-weight agents within the host process where necessary. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (FLUME-1682) Improve logging message when encrypted file channel is unable to be initialized due to invalid key/keystore
Jeff Lord created FLUME-1682: Summary: Improve logging message when encrypted file channel is unable to be initialized due to invalid key/keystore Key: FLUME-1682 URL: https://issues.apache.org/jira/browse/FLUME-1682 Project: Flume Issue Type: Improvement Affects Versions: v1.2.0 Reporter: Jeff Lord Currently if you have data in the file channel and stop flume for some reason (perhaps an upgrade) and then delete the keystore and regenerate the keystore. When flume is restarted it will throw an error similar to the following. It would be good if we could detect the reason for this failure to initialize as a change/mismatch in the keystore and report as such enabling self diagnosis and subsequent fix. 2012-10-23 09:21:32,230 ERROR file.Log: Failed to initialize Log on [channel=fileChannel] java.io.IOException: Unable to read next Transaction from log file /flume/file-channel/data10/log-10 at offset 31519759 at org.apache.flume.channel.file.LogFile$SequentialReader.next(LogFile.java:456) at org.apache.flume.channel.file.ReplayHandler.replayLog(ReplayHandler.java:245) at org.apache.flume.channel.file.Log.replay(Log.java:356) at org.apache.flume.channel.file.FileChannel.start(FileChannel.java:258) at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441) at java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:317) at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:150) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$101(ScheduledThreadPoolExecutor.java:98) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.runPeriodic(ScheduledThreadPoolExecutor.java:180) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:204) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) Caused by: com.google.protobuf.InvalidProtocolBufferException: Protocol message tag had invalid wire type. at com.google.protobuf.InvalidProtocolBufferException.invalidWireType(InvalidProtocolBufferException.java:78) at com.google.protobuf.UnknownFieldSet$Builder.mergeFieldFrom(UnknownFieldSet.java:498) at com.google.protobuf.GeneratedMessage$Builder.parseUnknownField(GeneratedMessage.java:438) at org.apache.flume.channel.file.proto.ProtosFactory$TransactionEventHeader$Builder.mergeFrom(ProtosFactory.java:2880) at org.apache.flume.channel.file.proto.ProtosFactory$TransactionEventHeader$Builder.mergeFrom(ProtosFactory.java:2732) at com.google.protobuf.AbstractMessageLite$Builder.mergeFrom(AbstractMessageLite.java:212) at com.google.protobuf.AbstractMessage$Builder.mergeFrom(AbstractMessage.java:746) at com.google.protobuf.AbstractMessage$Builder.mergeFrom(AbstractMessage.java:238) at com.google.protobuf.AbstractMessageLite$Builder.mergeDelimitedFrom(AbstractMessageLite.java:282) at com.google.protobuf.AbstractMessage$Builder.mergeDelimitedFrom(AbstractMessage.java:760) at com.google.protobuf.AbstractMessageLite$Builder.mergeDelimitedFrom(AbstractMessageLite.java:288) at com.google.protobuf.AbstractMessage$Builder.mergeDelimitedFrom(AbstractMessage.java:752) at org.apache.flume.channel.file.proto.ProtosFactory$TransactionEventHeader.parseDelimitedFrom(ProtosFactory.java:2689) at org.apache.flume.channel.file.TransactionEventRecord.fromByteArray(TransactionEventRecord.java:193) at org.apache.flume.channel.file.LogFileV3$SequentialReader.doNext(LogFileV3.java:327) at org.apache.flume.channel.file.LogFile$SequentialReader.next(LogFile.java:452) ... 13 more 2012-10-23 09:21:32,236 ERROR file.FileChannel: Failed to start the file channel [channel=fileChannel] java.io.IOException: Unable to read next Transaction from log file /app/flume/file-channel/data1/log-1 at offset 31519759 at org.apache.flume.channel.file.LogFile$SequentialReader.next(LogFile.java:456) at org.apache.flume.channel.file.ReplayHandler.replayLog(ReplayHandler.java:245) at org.apache.flume.channel.file.Log.replay(Log.java:356) at org.apache.flume.channel.file.FileChannel.start(FileChannel.java:258) at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441) at java.util.concurrent.FutureTask$Sync.innerRunAndReset(
[jira] [Comment Edited] (FLUME-1502) Support for running simple configurations embedded in host process
[ https://issues.apache.org/jira/browse/FLUME-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13491159#comment-13491159 ] Brock Noland edited comment on FLUME-1502 at 11/6/12 3:00 AM: -- Hi guys, I will add FileChannel as one of the two Channel choices of the embedded agent. I am not opposed to embedding a file channel, I just want this first iteration to be as simple as possible until we see how people are using the agent. In the design doc I state that users will have to use the RPCClient to talk to the source despite this being the same JVM. I did that because the RPC client is well tested and as such didn't require us to create an additional embedded only source. Doing this would require more code because the embedded agent have to have a reference to the channel whereas if we use the avro source we re-use the same boostrap logic we have today. As such, I'd prefer to require the use of the RPCClient but I am not tied to this direction. We could of course put in some syntactic sugar so users didn't have to create the RPCClient themselves. The embedded agent could take of that for them and just expose a put() or putBatch() method. was (Author: brocknoland): Hi guys, I will add FileChannel as one of the two Channel choices of the embedded agent. I am not opposed to embedding a file channel, I just want this first iteration to be as simple as possible until we see how people are using the agent. In the design doc I state that users will have to use the RPCClient to talk to the source despite this being the same JVM. I did that because the RPC client is well tested and as such didn't require us to create an additional embedded only source. Doing this would require more code because the embedded agent have to have a reference to the channel whereas if we use the avro source we re-use the same boostrap logic we have today. As such, I'd prefer to require the use of the RPCClient but I am not tied to this direction. > Support for running simple configurations embedded in host process > -- > > Key: FLUME-1502 > URL: https://issues.apache.org/jira/browse/FLUME-1502 > Project: Flume > Issue Type: Improvement >Affects Versions: v1.2.0 >Reporter: Arvind Prabhakar >Assignee: Brock Noland > Attachments: embeeded-agent-1.pdf > > > Flume should provide a light-weight embeddable node manager that can be > started in process where necessary. This will allow the users to embed > light-weight agents within the host process where necessary. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (FLUME-1502) Support for running simple configurations embedded in host process
[ https://issues.apache.org/jira/browse/FLUME-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13491159#comment-13491159 ] Brock Noland commented on FLUME-1502: - Hi guys, I will add FileChannel as one of the two Channel choices of the embedded agent. I am not opposed to embedding a file channel, I just want this first iteration to be as simple as possible until we see how people are using the agent. In the design doc I state that users will have to use the RPCClient to talk to the source despite this being the same JVM. I did that because the RPC client is well tested and as such didn't require us to create an additional embedded only source. Doing this would require more code because the embedded agent have to have a reference to the channel whereas if we use the avro source we re-use the same boostrap logic we have today. As such, I'd prefer to require the use of the RPCClient but I am not tied to this direction. > Support for running simple configurations embedded in host process > -- > > Key: FLUME-1502 > URL: https://issues.apache.org/jira/browse/FLUME-1502 > Project: Flume > Issue Type: Improvement >Affects Versions: v1.2.0 >Reporter: Arvind Prabhakar >Assignee: Brock Noland > Attachments: embeeded-agent-1.pdf > > > Flume should provide a light-weight embeddable node manager that can be > started in process where necessary. This will allow the users to embed > light-weight agents within the host process where necessary. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (FLUME-1227) Introduce some sort of SpillableChannel
[ https://issues.apache.org/jira/browse/FLUME-1227?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13491154#comment-13491154 ] Rahul Ravindran commented on FLUME-1227: Is there a timeline on when this new channel would be out? > Introduce some sort of SpillableChannel > --- > > Key: FLUME-1227 > URL: https://issues.apache.org/jira/browse/FLUME-1227 > Project: Flume > Issue Type: New Feature > Components: Channel >Reporter: Jarek Jarcec Cecho >Assignee: Patrick Wendell > > I would like to introduce new channel that would behave similarly as scribe > (https://github.com/facebook/scribe). It would be something between memory > and file channel. Input events would be saved directly to the memory (only) > and would be served from there. In case that the memory would be full, we > would outsource the events to file. > Let me describe the use case behind this request. We have plenty of frontend > servers that are generating events. We want to send all events to just > limited number of machines from where we would send the data to HDFS (some > sort of staging layer). Reason for this second layer is our need to decouple > event aggregation and front end code to separate machines. Using memory > channel is fully sufficient as we can survive lost of some portion of the > events. However in order to sustain maintenance windows or networking issues > we would have to end up with a lot of memory assigned to those "staging" > machines. Referenced "scribe" is dealing with this problem by implementing > following logic - events are saved in memory similarly as our MemoryChannel. > However in case that the memory gets full (because of maintenance, networking > issues, ...) it will spill data to disk where they will be sitting until > everything start working again. > I would like to introduce channel that would implement similar logic. It's > durability guarantees would be same as MemoryChannel - in case that someone > would remove power cord, this channel would lose data. Based on the > discussion in FLUME-1201, I would propose to have the implementation > completely independent on any other channel internal code. > Jarcec -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (FLUME-1681) Disable empty-file unit test for Spooling File Reader
[ https://issues.apache.org/jira/browse/FLUME-1681?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mike Percy updated FLUME-1681: -- Attachment: FLUME-1681.patch Small patch to @Ignore the failing unit test. I don't believe the feature is implemented yet, and I don't believe that it's a critical unit test. > Disable empty-file unit test for Spooling File Reader > - > > Key: FLUME-1681 > URL: https://issues.apache.org/jira/browse/FLUME-1681 > Project: Flume > Issue Type: Improvement >Reporter: Mike Percy >Assignee: Mike Percy > Fix For: v1.3.0 > > Attachments: FLUME-1681.patch > > > Disable empty-file unit test which is currently failing -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (FLUME-1425) Create a SpoolDirectory Source and Client
[ https://issues.apache.org/jira/browse/FLUME-1425?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13491133#comment-13491133 ] Mike Percy commented on FLUME-1425: --- Rather than @Ignore the test on commit, I posted a small patch to FLUME-1681 to disable the unit test for now. > Create a SpoolDirectory Source and Client > - > > Key: FLUME-1425 > URL: https://issues.apache.org/jira/browse/FLUME-1425 > Project: Flume > Issue Type: Improvement >Reporter: Patrick Wendell >Assignee: Patrick Wendell > Fix For: v1.3.0 > > Attachments: FileProcessingSource.java, > FLUME-1425.avro-conf-file.txt, FLUME-1425.patch.v1.txt, > FLUME-1425.v5.patch.txt, FLUME-1425.v6.patch.txt, FLUME-1425.v6.patch.txt, > FLUME-1425.v7.patch.txt, FLUME-1425.v8.patch.txt > > > The proposal is to create a small executable client which reads logs from a > spooling directory and sends them to a flume sink, then performs cleanup on > the directory (either by deleting or moving the logs). It would make the > following assumptions > - Files placed in the directory are uniquely named > - Files placed in the directory are immutable > The problem this is trying to solve is that there is currently no way to do > guaranteed event delivery across flume agent restarts when the data is being > collected through an asynchronous source (and not directly from the client > API). Say, for instance, you are using a exec("tail -F") source. If the agent > restarts due to error or intentionally, tail may pick up at a new location > and you lose the intermediate data. > At the same time, there are users who want at-least-once semantics, and > expect those to apply as soon as the data is written to disk from the initial > logger process (e.g. apache logs), not just once it has reached a flume > agent. This idea would bridge that gap, assuming the user is able to copy > immutable logs to a spooling directory through a cron script or something. > The basic internal logic of such a client would be as follows: > - Scan the directory for files > - Chose a file and read through, while sending events to an agent > - Close the file and delete it (or rename, or otherwise mark completed) > That's about it. We could add sync-points to make recovery more efficient in > the case of failure. > A key question is whether this should be implemented as a standalone client > or as a source. My instinct is actually to do this as a source, but there > could be some benefit to not requiring an entire agent in order to run this, > specifically that it would become platform independent and you could stick it > on Windows machines. Others I have talked to have also sided on a standalone > executable. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (FLUME-1681) Disable empty-file unit test for Spooling File Reader
Mike Percy created FLUME-1681: - Summary: Disable empty-file unit test for Spooling File Reader Key: FLUME-1681 URL: https://issues.apache.org/jira/browse/FLUME-1681 Project: Flume Issue Type: Improvement Reporter: Mike Percy Fix For: v1.3.0 Disable empty-file unit test which is currently failing -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Assigned] (FLUME-1681) Disable empty-file unit test for Spooling File Reader
[ https://issues.apache.org/jira/browse/FLUME-1681?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mike Percy reassigned FLUME-1681: - Assignee: Mike Percy > Disable empty-file unit test for Spooling File Reader > - > > Key: FLUME-1681 > URL: https://issues.apache.org/jira/browse/FLUME-1681 > Project: Flume > Issue Type: Improvement >Reporter: Mike Percy >Assignee: Mike Percy > Fix For: v1.3.0 > > > Disable empty-file unit test which is currently failing -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (FLUME-1425) Create a SpoolDirectory Source and Client
[ https://issues.apache.org/jira/browse/FLUME-1425?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13491125#comment-13491125 ] Mike Percy commented on FLUME-1425: --- +1 > Create a SpoolDirectory Source and Client > - > > Key: FLUME-1425 > URL: https://issues.apache.org/jira/browse/FLUME-1425 > Project: Flume > Issue Type: Improvement >Reporter: Patrick Wendell >Assignee: Patrick Wendell > Attachments: FileProcessingSource.java, > FLUME-1425.avro-conf-file.txt, FLUME-1425.patch.v1.txt, > FLUME-1425.v5.patch.txt, FLUME-1425.v6.patch.txt, FLUME-1425.v6.patch.txt, > FLUME-1425.v7.patch.txt, FLUME-1425.v8.patch.txt > > > The proposal is to create a small executable client which reads logs from a > spooling directory and sends them to a flume sink, then performs cleanup on > the directory (either by deleting or moving the logs). It would make the > following assumptions > - Files placed in the directory are uniquely named > - Files placed in the directory are immutable > The problem this is trying to solve is that there is currently no way to do > guaranteed event delivery across flume agent restarts when the data is being > collected through an asynchronous source (and not directly from the client > API). Say, for instance, you are using a exec("tail -F") source. If the agent > restarts due to error or intentionally, tail may pick up at a new location > and you lose the intermediate data. > At the same time, there are users who want at-least-once semantics, and > expect those to apply as soon as the data is written to disk from the initial > logger process (e.g. apache logs), not just once it has reached a flume > agent. This idea would bridge that gap, assuming the user is able to copy > immutable logs to a spooling directory through a cron script or something. > The basic internal logic of such a client would be as follows: > - Scan the directory for files > - Chose a file and read through, while sending events to an agent > - Close the file and delete it (or rename, or otherwise mark completed) > That's about it. We could add sync-points to make recovery more efficient in > the case of failure. > A key question is whether this should be implemented as a standalone client > or as a source. My instinct is actually to do this as a source, but there > could be some benefit to not requiring an entire agent in order to run this, > specifically that it would become platform independent and you could stick it > on Windows machines. Others I have talked to have also sided on a standalone > executable. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
Re: Review Request: FLUME-1425: Create a SpoolDirectory Source and Client
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/6377/#review13121 --- Ship it! Hey Patrick, really good work overall. I'm going to commit this as-is. We can follow-up more after the initial commit. - Mike Percy On Oct. 22, 2012, 8:36 p.m., Patrick Wendell wrote: > > --- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/6377/ > --- > > (Updated Oct. 22, 2012, 8:36 p.m.) > > > Review request for Flume. > > > Description > --- > > This patch adds a spooling directory based source. The idea is that a user > can have a spool directory where files are deposited for ingestion into > flume. Once ingested, the files are clearly renamed and the implementation > guarantees at-least-once delivery semantics similar to those achieved within > flume itself, even across failures and restarts of the JVM running the code. > > This helps fill the gap for people who want a way to get reliable delivery of > events into flume, but don't want to directly write their application against > the flume API. They can simply drop log files off in a spooldir and let flume > ingest asynchronously (using some shell scripts or other automated process). > > Unlike the prior iteration, this patch implements a first-class source. It > also extends the avro client to support spooling in a similar manner. > > > This addresses bug FlUME-1425. > https://issues.apache.org/jira/browse/FlUME-1425 > > > Diffs > - > > > flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java > da804d7 > > flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceType.java > abbbf1c > flume-ng-core/src/main/java/org/apache/flume/client/avro/AvroCLIClient.java > 4a5ecae > > flume-ng-core/src/main/java/org/apache/flume/client/avro/BufferedLineReader.java > PRE-CREATION > flume-ng-core/src/main/java/org/apache/flume/client/avro/LineReader.java > PRE-CREATION > > flume-ng-core/src/main/java/org/apache/flume/client/avro/SpoolingFileLineReader.java > PRE-CREATION > > flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java > PRE-CREATION > > flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java > PRE-CREATION > > flume-ng-core/src/test/java/org/apache/flume/client/avro/TestBufferedLineReader.java > PRE-CREATION > > flume-ng-core/src/test/java/org/apache/flume/client/avro/TestSpoolingFileLineReader.java > PRE-CREATION > > flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java > PRE-CREATION > flume-ng-doc/sphinx/FlumeUserGuide.rst 953a670 > > Diff: https://reviews.apache.org/r/6377/diff/ > > > Testing > --- > > Extensive unit tests and I also built and played with this using a stub flume > agent. If you look at the JIRA I have a configuration file for an agent that > will print out Avro events to the command line - that's helpful when testing > this. > > > Thanks, > > Patrick Wendell > >
Re: [DISCUSS] Flume 1.3.0 release
Personally I'd like to get FLUME-1425 in. I'm trying to get it checked in ASAP. I think we should also pull in FLUME-1629 and FLUME-1660. We're a bit ahead on patches and behind on reviews at the moment. Regards Mike On Mon, Nov 5, 2012 at 10:57 AM, Roshan Naik wrote: > Thought of checking back on the availability of the Release Candidate.. any > ETA ? > > -roshan >
Re: Review Request: FLUME-1425: Create a SpoolDirectory Source and Client
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/6377/#review13120 --- Hi Patrick, overall this looks good to me. Only thing is the test failure. I am tempted to just commit this, but I'm trying to understand what's going on with the test. It looks to me that the test is trying to assert that the files not be processed if they are empty. I added some debug messages to the assert and this is what it is printing: testBehaviorWithEmptyFile(org.apache.flume.client.avro.TestSpoolingFileLineReader) Time elapsed: 1.012 sec <<< FAILURE! java.lang.AssertionError: Does not contain file: /var/folders/fg/ym7gqsvs5h30gg13yjpt1hm0gn/T/1352164166517-0/file1, only contains: [/var/folders/fg/ym7gqsvs5h30gg13yjpt1hm0gn/T/1352164166517-0/file1.COMPLETE, /var/folders/fg/ym7gqsvs5h30gg13yjpt1hm0gn/T/1352164166517-0/file2.COMPLETE] at org.junit.Assert.fail(Assert.java:93) at org.junit.Assert.assertTrue(Assert.java:43) at org.apache.flume.client.avro.TestSpoolingFileLineReader.testBehaviorWithEmptyFile(TestSpoolingFileLineReader.java:399) So the test is asserting that it should not have rolled test1 to test1.COMPLETE, but it has (At the end of this test, it has actually run both). I don't see any code in the patch indicating that files that have no data should not be read. So I don't understand how this test could be passing for you. Please try re-running the test using this command from the Flume top-level: mvn clean install -Dtest=TestSpoolingFileLineReader -DfailIfNoTests=false ... I will be very surprised if it passes. I am still reviewing this patch and am currently leaning toward adding an @Ignore to this test for commit, since I'd like to get this feature into 1.3.0. In the mean time, any comments on the desired behavior are appreciated. Regards, Mike - Mike Percy On Oct. 22, 2012, 8:36 p.m., Patrick Wendell wrote: > > --- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/6377/ > --- > > (Updated Oct. 22, 2012, 8:36 p.m.) > > > Review request for Flume. > > > Description > --- > > This patch adds a spooling directory based source. The idea is that a user > can have a spool directory where files are deposited for ingestion into > flume. Once ingested, the files are clearly renamed and the implementation > guarantees at-least-once delivery semantics similar to those achieved within > flume itself, even across failures and restarts of the JVM running the code. > > This helps fill the gap for people who want a way to get reliable delivery of > events into flume, but don't want to directly write their application against > the flume API. They can simply drop log files off in a spooldir and let flume > ingest asynchronously (using some shell scripts or other automated process). > > Unlike the prior iteration, this patch implements a first-class source. It > also extends the avro client to support spooling in a similar manner. > > > This addresses bug FlUME-1425. > https://issues.apache.org/jira/browse/FlUME-1425 > > > Diffs > - > > > flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceConfiguration.java > da804d7 > > flume-ng-configuration/src/main/java/org/apache/flume/conf/source/SourceType.java > abbbf1c > flume-ng-core/src/main/java/org/apache/flume/client/avro/AvroCLIClient.java > 4a5ecae > > flume-ng-core/src/main/java/org/apache/flume/client/avro/BufferedLineReader.java > PRE-CREATION > flume-ng-core/src/main/java/org/apache/flume/client/avro/LineReader.java > PRE-CREATION > > flume-ng-core/src/main/java/org/apache/flume/client/avro/SpoolingFileLineReader.java > PRE-CREATION > > flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySource.java > PRE-CREATION > > flume-ng-core/src/main/java/org/apache/flume/source/SpoolDirectorySourceConfigurationConstants.java > PRE-CREATION > > flume-ng-core/src/test/java/org/apache/flume/client/avro/TestBufferedLineReader.java > PRE-CREATION > > flume-ng-core/src/test/java/org/apache/flume/client/avro/TestSpoolingFileLineReader.java > PRE-CREATION > > flume-ng-core/src/test/java/org/apache/flume/source/TestSpoolDirectorySource.java > PRE-CREATION > flume-ng-doc/sphinx/FlumeUserGuide.rst 953a670 > > Diff: https://reviews.apache.org/r/6377/diff/ > > > Testing > --- > > Extensive unit tests and I also built and played with this using a stub flume > agent. If you look at the JIRA I have a configuration file for an agent that > will print out Avro events to the command line - that's helpful when testing > this. > > > Thanks, > > Patrick Wendell > >
[jira] [Commented] (FLUME-1502) Support for running simple configurations embedded in host process
[ https://issues.apache.org/jira/browse/FLUME-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13491105#comment-13491105 ] Arvind Prabhakar commented on FLUME-1502: - @Brock, thanks for the design document. On the point of File Channel, I do feel that it is important to have that support to ensure that we do not put excessive strain on memory for the host process, and that we do not lose events in the case of host process failure. Another point to consider is whether the source would be any different from a regular source when running in embedded mode. For example, does it make sense to have embedded agent with a network source like Avro working on it? For instance, it may make sense to have no source support, but a direct pass-through for the client API that directly talks with the channel in question. > Support for running simple configurations embedded in host process > -- > > Key: FLUME-1502 > URL: https://issues.apache.org/jira/browse/FLUME-1502 > Project: Flume > Issue Type: Improvement >Affects Versions: v1.2.0 >Reporter: Arvind Prabhakar >Assignee: Brock Noland > Attachments: embeeded-agent-1.pdf > > > Flume should provide a light-weight embeddable node manager that can be > started in process where necessary. This will allow the users to embed > light-weight agents within the host process where necessary. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (FLUME-1661) ExecSource cannot execute (little complicated..) *nix commands
[ https://issues.apache.org/jira/browse/FLUME-1661?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13491068#comment-13491068 ] Roshan Naik commented on FLUME-1661: Nitin. I dont understand what problem the suggested code solves. The command you provide should work fine with the proposed patch. Something similar was already part of the test suite. I added the precise command you provide to the test suite anyway. > ExecSource cannot execute (little complicated..) *nix commands > -- > > Key: FLUME-1661 > URL: https://issues.apache.org/jira/browse/FLUME-1661 > Project: Flume > Issue Type: Improvement > Components: Sinks+Sources >Affects Versions: v1.2.0 >Reporter: Yoonseok Woo >Assignee: Roshan Naik > Fix For: v1.3.0 > > Attachments: FLUME-1661-1.patch, FLUME-1661.patch, > FLUME-1661.patch.v2, FLUME-1661.patch.v3, FLUME-1661.patch.v4, > FlumeProcessRunner.tar.gz > > > * command line parsing > ** conf/flume.conf > {code} > agent.sources.source1.type = exec > agent.sources.source1.command = tail -f > /some/path/logs/exception/error.log.`date +%Y%m%d%H` > {code} > ** result > {code} > tail: /some/path/logs/exception/error.log.`date: No such file or directory > tail: +%Y%m%d%H`: No such file or directory > {code} > ** needs to be improved > {code} > (ExecSouce.java:242) String[] commandArgs = command.split("\\s+") > {code} > * using special character (e.g. *, `, ', ...) > ** conf/flume.conf > {code} > agent.sources.source1.type = exec > agent.sources.source1.command = tail -f /some/path/logs/exception/error.log.* > {code} > ** result > {code} > tail: /some/path/logs/exception/error.log.*: No such file or directory > {code} > ** needs to be improved > {code} > (ExecSouce.java:243) process = new ProcessBuilder(commandArgs).start(); > {code} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (FLUME-1661) ExecSource cannot execute (little complicated..) *nix commands
[ https://issues.apache.org/jira/browse/FLUME-1661?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Roshan Naik updated FLUME-1661: --- Attachment: FLUME-1661.patch.v4 Adding another test case (as noted by Nitin), and excluding resource file from rat checks. > ExecSource cannot execute (little complicated..) *nix commands > -- > > Key: FLUME-1661 > URL: https://issues.apache.org/jira/browse/FLUME-1661 > Project: Flume > Issue Type: Improvement > Components: Sinks+Sources >Affects Versions: v1.2.0 >Reporter: Yoonseok Woo >Assignee: Roshan Naik > Fix For: v1.3.0 > > Attachments: FLUME-1661-1.patch, FLUME-1661.patch, > FLUME-1661.patch.v2, FLUME-1661.patch.v3, FLUME-1661.patch.v4, > FlumeProcessRunner.tar.gz > > > * command line parsing > ** conf/flume.conf > {code} > agent.sources.source1.type = exec > agent.sources.source1.command = tail -f > /some/path/logs/exception/error.log.`date +%Y%m%d%H` > {code} > ** result > {code} > tail: /some/path/logs/exception/error.log.`date: No such file or directory > tail: +%Y%m%d%H`: No such file or directory > {code} > ** needs to be improved > {code} > (ExecSouce.java:242) String[] commandArgs = command.split("\\s+") > {code} > * using special character (e.g. *, `, ', ...) > ** conf/flume.conf > {code} > agent.sources.source1.type = exec > agent.sources.source1.command = tail -f /some/path/logs/exception/error.log.* > {code} > ** result > {code} > tail: /some/path/logs/exception/error.log.*: No such file or directory > {code} > ** needs to be improved > {code} > (ExecSouce.java:243) process = new ProcessBuilder(commandArgs).start(); > {code} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
target/* not excluded in rat checks
Folks, The directory 'target' is not in the rat exclusion list. Is this intentional ? I occasionally see rat check errors originating from files in target directory. Thought of double checking before providing a patch. -roshan
Re: [DISCUSS] Flume 1.3.0 release
Thought of checking back on the availability of the Release Candidate.. any ETA ? -roshan
[jira] [Commented] (FLUME-1502) Support for running simple configurations embedded in host process
[ https://issues.apache.org/jira/browse/FLUME-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13490730#comment-13490730 ] Ralph Goers commented on FLUME-1502: Only partially. In my applications there are many events where the application must be sure that the audit event has reached a point where delivery is guaranteed before it can attempt to perform the action being audited. With the Memory Channel the event will be accepted and the application can continue even though the event may not actually ever be delivered and will be lost if the JVM goes down. My understanding is that currently the only channel that provides sufficient guarantees is the File Channel. > Support for running simple configurations embedded in host process > -- > > Key: FLUME-1502 > URL: https://issues.apache.org/jira/browse/FLUME-1502 > Project: Flume > Issue Type: Improvement >Affects Versions: v1.2.0 >Reporter: Arvind Prabhakar >Assignee: Brock Noland > Attachments: embeeded-agent-1.pdf > > > Flume should provide a light-weight embeddable node manager that can be > started in process where necessary. This will allow the users to embed > light-weight agents within the host process where necessary. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (FLUME-1502) Support for running simple configurations embedded in host process
[ https://issues.apache.org/jira/browse/FLUME-1502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13490717#comment-13490717 ] Brock Noland commented on FLUME-1502: - Ralph, Thank you very much for providing feedback so quickly. How about we change the propose to allow multiple sinks? AvroSinks are fairly lightweight so I'd prefer to allow them over embedding a more complex file channel. Would that alleviate your concerns? Brock > Support for running simple configurations embedded in host process > -- > > Key: FLUME-1502 > URL: https://issues.apache.org/jira/browse/FLUME-1502 > Project: Flume > Issue Type: Improvement >Affects Versions: v1.2.0 >Reporter: Arvind Prabhakar >Assignee: Brock Noland > Attachments: embeeded-agent-1.pdf > > > Flume should provide a light-weight embeddable node manager that can be > started in process where necessary. This will allow the users to embed > light-weight agents within the host process where necessary. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira