You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flume.apache.org by qiao wen <31...@qq.com> on 2016/07/26 09:02:46 UTC

Re: Review Request 50438: FLUME-2961:Make TaildirSource work with multiline

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

(Updated \u4e03\u6708 26, 2016, 9:02 a.m.)


Review request for Flume.


Summary (updated)
-----------------

FLUME-2961:Make TaildirSource work with multiline


Repository: flume-git


Description
-------

TaidirSource defaults to LINE, this has issue when multiline log events like stack traces and have request/responses. Following part is Java traceback logs. We expect to have log line start regex Key to aggregate all the log lines till the next regex key is found.
2016-07-16 14:59:43,956 ERROR lifecycleSupervisor-1-7 LifecycleSupervisor.run - Unable to start EventDrivenSourceRunner: { source:cn.yottabyte.flume.source.http.HTTPSource{name:sourceHttp,state:IDLE} } - Exception follows.
java.lang.IllegalStateException: Running HTTP Server found in source: sourceHttp before I started one. Will not attempt to start.
    at com.google.common.base.Preconditions.checkState(Preconditions.java:145)
    at cn.yottabyte.flume.source.http.HTTPSource.startHttpSourceServer(HTTPSource.java:170)
    at cn.yottabyte.flume.source.http.HTTPSource.start(HTTPSource.java:166)
    at org.apache.flume.source.EventDrivenSourceRunner.start(EventDrivenSourceRunner.java:44)
    at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:251)
    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
    at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
    at java.lang.Thread.run(Thread.java:745)


Diffs
-----

  flume-ng-doc/sphinx/FlumeUserGuide.rst 1334500 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java 8838320 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java 42474c4 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java a107a01 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java f2347f3 
  flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java 097ee0b 

Diff: https://reviews.apache.org/r/50438/diff/


Testing
-------

All tests in TestTaildirSource passed.


Thanks,

qiao wen


Re: Review Request 50438: FLUME-2961:Make TaildirSource work with multiline

Posted by qiao wen <31...@qq.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/50438/
-----------------------------------------------------------

(Updated 十月 26, 2017, 3 p.m.)


Review request for Flume.


Changes
-------

Fix the read position problem when the file is closed and the bufferEvent is not empty


Repository: flume-git


Description
-------

TaidirSource defaults to LINE, this has issue when multiline log events like stack traces and have request/responses. Following part is Java traceback logs. We expect to have log line start regex Key to aggregate all the log lines till the next regex key is found.
2016-07-16 14:59:43,956 ERROR lifecycleSupervisor-1-7 LifecycleSupervisor.run - Unable to start EventDrivenSourceRunner: { source:cn.yottabyte.flume.source.http.HTTPSource{name:sourceHttp,state:IDLE} } - Exception follows.
java.lang.IllegalStateException: Running HTTP Server found in source: sourceHttp before I started one. Will not attempt to start.
    at com.google.common.base.Preconditions.checkState(Preconditions.java:145)
    at cn.yottabyte.flume.source.http.HTTPSource.startHttpSourceServer(HTTPSource.java:170)
    at cn.yottabyte.flume.source.http.HTTPSource.start(HTTPSource.java:166)
    at org.apache.flume.source.EventDrivenSourceRunner.start(EventDrivenSourceRunner.java:44)
    at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:251)
    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
    at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
    at java.lang.Thread.run(Thread.java:745)


Diffs (updated)
-----

  flume-ng-doc/sphinx/FlumeUserGuide.rst 909fe4ab 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java 633d3c19 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java 42474c4f 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java a107a017 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java f2347f38 
  flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java 097ee0ba 


Diff: https://reviews.apache.org/r/50438/diff/5/

Changes: https://reviews.apache.org/r/50438/diff/4-5/


Testing
-------

All tests in TestTaildirSource passed.


Thanks,

qiao wen


Re: Review Request 50438: FLUME-2961:Make TaildirSource work with multiline

Posted by qiao wen <31...@qq.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/50438/
-----------------------------------------------------------

(Updated \u56db\u6708 13, 2017, 2:20 a.m.)


Review request for Flume.


Changes
-------

Resolve nullpointer exception and numberformat exception.


Repository: flume-git


Description
-------

TaidirSource defaults to LINE, this has issue when multiline log events like stack traces and have request/responses. Following part is Java traceback logs. We expect to have log line start regex Key to aggregate all the log lines till the next regex key is found.
2016-07-16 14:59:43,956 ERROR lifecycleSupervisor-1-7 LifecycleSupervisor.run - Unable to start EventDrivenSourceRunner: { source:cn.yottabyte.flume.source.http.HTTPSource{name:sourceHttp,state:IDLE} } - Exception follows.
java.lang.IllegalStateException: Running HTTP Server found in source: sourceHttp before I started one. Will not attempt to start.
    at com.google.common.base.Preconditions.checkState(Preconditions.java:145)
    at cn.yottabyte.flume.source.http.HTTPSource.startHttpSourceServer(HTTPSource.java:170)
    at cn.yottabyte.flume.source.http.HTTPSource.start(HTTPSource.java:166)
    at org.apache.flume.source.EventDrivenSourceRunner.start(EventDrivenSourceRunner.java:44)
    at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:251)
    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
    at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
    at java.lang.Thread.run(Thread.java:745)


Diffs (updated)
-----

  flume-ng-doc/sphinx/FlumeUserGuide.rst a5d64f0 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java 8838320 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java 42474c4 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java a107a01 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java f2347f3 
  flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java 097ee0b 


Diff: https://reviews.apache.org/r/50438/diff/4/

Changes: https://reviews.apache.org/r/50438/diff/3-4/


Testing
-------

All tests in TestTaildirSource passed.


Thanks,

qiao wen


Re: Review Request 50438: FLUME-2961:Make TaildirSource work with multiline

Posted by qiao wen <31...@qq.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/50438/
-----------------------------------------------------------

(Updated \u4e09\u6708 6, 2017, 1:15 p.m.)


Review request for Flume.


Changes
-------

Change function name isNeedFlushBufferEvent() to needFlushTimeoutEvent() in TailFile. Modify userguide about multilinePatternBelong. Change local var pattern in TailFile to instance var.


Repository: flume-git


Description
-------

TaidirSource defaults to LINE, this has issue when multiline log events like stack traces and have request/responses. Following part is Java traceback logs. We expect to have log line start regex Key to aggregate all the log lines till the next regex key is found.
2016-07-16 14:59:43,956 ERROR lifecycleSupervisor-1-7 LifecycleSupervisor.run - Unable to start EventDrivenSourceRunner: { source:cn.yottabyte.flume.source.http.HTTPSource{name:sourceHttp,state:IDLE} } - Exception follows.
java.lang.IllegalStateException: Running HTTP Server found in source: sourceHttp before I started one. Will not attempt to start.
    at com.google.common.base.Preconditions.checkState(Preconditions.java:145)
    at cn.yottabyte.flume.source.http.HTTPSource.startHttpSourceServer(HTTPSource.java:170)
    at cn.yottabyte.flume.source.http.HTTPSource.start(HTTPSource.java:166)
    at org.apache.flume.source.EventDrivenSourceRunner.start(EventDrivenSourceRunner.java:44)
    at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:251)
    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
    at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
    at java.lang.Thread.run(Thread.java:745)


Diffs (updated)
-----

  flume-ng-doc/sphinx/FlumeUserGuide.rst d863068 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java 8838320 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java 42474c4 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java a107a01 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java f2347f3 
  flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java 097ee0b 


Diff: https://reviews.apache.org/r/50438/diff/3/

Changes: https://reviews.apache.org/r/50438/diff/2-3/


Testing
-------

All tests in TestTaildirSource passed.


Thanks,

qiao wen


Re: Review Request 50438: FLUME-2961:Make TaildirSource work with multiline

Posted by qiao wen <31...@qq.com>.

> On \u4e8c\u6708 28, 2017, 2:24 p.m., Bal�zs Don�t Bessenyei wrote:
> > flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java, line 196
> > <https://reviews.apache.org/r/50438/diff/2/?file=1638339#file1638339line196>
> >
> >     Can the pattern be cached? (Eg. in configure)

OK, I will fix it.


> On \u4e8c\u6708 28, 2017, 2:24 p.m., Bal�zs Don�t Bessenyei wrote:
> > flume-ng-doc/sphinx/FlumeUserGuide.rst, line 1191
> > <https://reviews.apache.org/r/50438/diff/2/?file=1638337#file1638337line1191>
> >
> >     Can you please clarify this setting?

It's explained in the function readMultilineEventPre() and readMultilineEventNext(). And you can see the effect in TestTaildirSource.

Previous:   If not matched, this line is not part of previous event when the buffer event is not null.
               Then create a new event with buffer event's message and put the current line into the
               cleared buffer event.
Next:        If not matched, this line is not part of next event. Then merge the current line into the
               buffer event and create a new event with the merged message.

private Event readMultilineEventPre(LineResult line, boolean match, Pattern pattern)
          throws IOException {
    Event event = null;
    Matcher m = pattern.matcher(new String(line.line));
    boolean find = m.find();
    match = (find && match) || (!find && !match);
    byte[] lineBytes = toOriginBytes(line);
    if (match) {
      /** If matched, merge it to the buffer event. */
      mergeEvent(line);
    } else {
      /**
       * If not matched, this line is not part of previous event when the buffer event is not null.
       * Then create a new event with buffer event's message and put the current line into the
       * cleared buffer event.
       */
      if (bufferEvent != null) {
        event = EventBuilder.withBody(bufferEvent.getBody());
      }
      bufferEvent = null;
      bufferEvent = EventBuilder.withBody(lineBytes);
      if (line.lineSepInclude) {
        bufferEvent.getHeaders().put("lineCount", "1");
      }
      long now = System.currentTimeMillis();
      bufferEvent.getHeaders().put(TimestampInterceptor.Constants.TIMESTAMP, Long.toString(now));
    }
    return event;
  }

  private Event readMultilineEventNext(LineResult line, boolean match, Pattern pattern)
          throws IOException {
    Event event = null;
    Matcher m = pattern.matcher(new String(line.line));
    boolean find = m.find();
    match = (find && match) || (!find && !match);
    if (match) {
      /** If matched, merge it to the buffer event. */
      mergeEvent(line);
    } else {
      /**
       * If not matched, this line is not part of next event. Then merge the current line into the
       * buffer event and create a new event with the merged message.
       */
      mergeEvent(line);
      event = EventBuilder.withBody(bufferEvent.getBody());
      bufferEvent = null;
    }
    return event;
  }


> On \u4e8c\u6708 28, 2017, 2:24 p.m., Bal�zs Don�t Bessenyei wrote:
> > flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java, line 115
> > <https://reviews.apache.org/r/50438/diff/2/?file=1638339#file1638339line115>
> >
> >     Isn't maxBytes and maxLines missing here?

No, maxBytes and maxLines logic is after readline() in readEvents(). Precisely the function should be called needFlushTimeoutEvent(). I will fix it ASAP.

public List<Event> readEvents(int numEvents, boolean backoffWithoutNL,
      boolean addByteOffset) throws IOException {
    List<Event> events = Lists.newLinkedList();
    if (this.multiline) {
      boolean match = this.multilinePatternMatched;
      Pattern pattern = Pattern.compile(this.multilinePattern);
      while (events.size() < numEvents) {
        LineResult line = readLine();
        if (line == null) {
          break;
        }
        Event event = null;
        switch (this.multilinePatternBelong) {
          case "next":
            event = readMultilineEventNext(line, match, pattern);
            break;
          case "previous":
            event = readMultilineEventPre(line, match, pattern);
            break;
          default:
            break;
        }
        if (event != null) {
          events.add(event);
        }
        if (bufferEvent != null) {
          if (bufferEvent.getBody().length >= multilineMaxBytes
              || Integer.parseInt(bufferEvent.getHeaders().get("lineCount")) == multilineMaxLines) {
            flushBufferEvent(events);
          }
        }
      }
      if (isNeedFlushBufferEvent()) {
        flushBufferEvent(events);
      }
    } else {
    ...
    }


- qiao


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


On \u4e8c\u6708 17, 2017, 11:13 a.m., qiao wen wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/50438/
> -----------------------------------------------------------
> 
> (Updated \u4e8c\u6708 17, 2017, 11:13 a.m.)
> 
> 
> Review request for Flume.
> 
> 
> Repository: flume-git
> 
> 
> Description
> -------
> 
> TaidirSource defaults to LINE, this has issue when multiline log events like stack traces and have request/responses. Following part is Java traceback logs. We expect to have log line start regex Key to aggregate all the log lines till the next regex key is found.
> 2016-07-16 14:59:43,956 ERROR lifecycleSupervisor-1-7 LifecycleSupervisor.run - Unable to start EventDrivenSourceRunner: { source:cn.yottabyte.flume.source.http.HTTPSource{name:sourceHttp,state:IDLE} } - Exception follows.
> java.lang.IllegalStateException: Running HTTP Server found in source: sourceHttp before I started one. Will not attempt to start.
>     at com.google.common.base.Preconditions.checkState(Preconditions.java:145)
>     at cn.yottabyte.flume.source.http.HTTPSource.startHttpSourceServer(HTTPSource.java:170)
>     at cn.yottabyte.flume.source.http.HTTPSource.start(HTTPSource.java:166)
>     at org.apache.flume.source.EventDrivenSourceRunner.start(EventDrivenSourceRunner.java:44)
>     at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:251)
>     at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>     at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
>     at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
>     at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>     at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>     at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>     at java.lang.Thread.run(Thread.java:745)
> 
> 
> Diffs
> -----
> 
>   flume-ng-doc/sphinx/FlumeUserGuide.rst afa6625 
>   flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java 8838320 
>   flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java 42474c4 
>   flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java a107a01 
>   flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java f2347f3 
>   flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java 097ee0b 
> 
> Diff: https://reviews.apache.org/r/50438/diff/
> 
> 
> Testing
> -------
> 
> All tests in TestTaildirSource passed.
> 
> 
> Thanks,
> 
> qiao wen
> 
>


Re: Review Request 50438: FLUME-2961:Make TaildirSource work with multiline

Posted by Balázs Donát Bessenyei <be...@apache.org>.

> On Feb. 28, 2017, 2:24 p.m., Bal�zs Don�t Bessenyei wrote:
> > flume-ng-doc/sphinx/FlumeUserGuide.rst
> > Lines 1191 (patched)
> > <https://reviews.apache.org/r/50438/diff/2/?file=1638337#file1638337line1191>
> >
> >     Can you please clarify this setting?
> 
> qiao wen wrote:
>     It's explained in the function readMultilineEventPre() and readMultilineEventNext(). And you can see the effect in TestTaildirSource.
>     
>     Previous:   If not matched, this line is not part of previous event when the buffer event is not null.
>                    Then create a new event with buffer event's message and put the current line into the
>                    cleared buffer event.
>     Next:        If not matched, this line is not part of next event. Then merge the current line into the
>                    buffer event and create a new event with the merged message.
>     
>     private Event readMultilineEventPre(LineResult line, boolean match, Pattern pattern)
>               throws IOException {
>         Event event = null;
>         Matcher m = pattern.matcher(new String(line.line));
>         boolean find = m.find();
>         match = (find && match) || (!find && !match);
>         byte[] lineBytes = toOriginBytes(line);
>         if (match) {
>           /** If matched, merge it to the buffer event. */
>           mergeEvent(line);
>         } else {
>           /**
>            * If not matched, this line is not part of previous event when the buffer event is not null.
>            * Then create a new event with buffer event's message and put the current line into the
>            * cleared buffer event.
>            */
>           if (bufferEvent != null) {
>             event = EventBuilder.withBody(bufferEvent.getBody());
>           }
>           bufferEvent = null;
>           bufferEvent = EventBuilder.withBody(lineBytes);
>           if (line.lineSepInclude) {
>             bufferEvent.getHeaders().put("lineCount", "1");
>           }
>           long now = System.currentTimeMillis();
>           bufferEvent.getHeaders().put(TimestampInterceptor.Constants.TIMESTAMP, Long.toString(now));
>         }
>         return event;
>       }
>     
>       private Event readMultilineEventNext(LineResult line, boolean match, Pattern pattern)
>               throws IOException {
>         Event event = null;
>         Matcher m = pattern.matcher(new String(line.line));
>         boolean find = m.find();
>         match = (find && match) || (!find && !match);
>         if (match) {
>           /** If matched, merge it to the buffer event. */
>           mergeEvent(line);
>         } else {
>           /**
>            * If not matched, this line is not part of next event. Then merge the current line into the
>            * buffer event and create a new event with the merged message.
>            */
>           mergeEvent(line);
>           event = EventBuilder.withBody(bufferEvent.getBody());
>           bufferEvent = null;
>         }
>         return event;
>       }

When I read the code, it's perfectly clear. However, it isn't necessarily so for users. I think including some description in the user guide would be very useful.


- Bal�zs Don�t


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


On Feb. 17, 2017, 11:13 a.m., qiao wen wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/50438/
> -----------------------------------------------------------
> 
> (Updated Feb. 17, 2017, 11:13 a.m.)
> 
> 
> Review request for Flume.
> 
> 
> Repository: flume-git
> 
> 
> Description
> -------
> 
> TaidirSource defaults to LINE, this has issue when multiline log events like stack traces and have request/responses. Following part is Java traceback logs. We expect to have log line start regex Key to aggregate all the log lines till the next regex key is found.
> 2016-07-16 14:59:43,956 ERROR lifecycleSupervisor-1-7 LifecycleSupervisor.run - Unable to start EventDrivenSourceRunner: { source:cn.yottabyte.flume.source.http.HTTPSource{name:sourceHttp,state:IDLE} } - Exception follows.
> java.lang.IllegalStateException: Running HTTP Server found in source: sourceHttp before I started one. Will not attempt to start.
>     at com.google.common.base.Preconditions.checkState(Preconditions.java:145)
>     at cn.yottabyte.flume.source.http.HTTPSource.startHttpSourceServer(HTTPSource.java:170)
>     at cn.yottabyte.flume.source.http.HTTPSource.start(HTTPSource.java:166)
>     at org.apache.flume.source.EventDrivenSourceRunner.start(EventDrivenSourceRunner.java:44)
>     at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:251)
>     at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>     at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
>     at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
>     at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>     at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>     at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>     at java.lang.Thread.run(Thread.java:745)
> 
> 
> Diffs
> -----
> 
>   flume-ng-doc/sphinx/FlumeUserGuide.rst afa6625 
>   flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java 8838320 
>   flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java 42474c4 
>   flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java a107a01 
>   flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java f2347f3 
>   flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java 097ee0b 
> 
> 
> Diff: https://reviews.apache.org/r/50438/diff/2/
> 
> 
> Testing
> -------
> 
> All tests in TestTaildirSource passed.
> 
> 
> Thanks,
> 
> qiao wen
> 
>


Re: Review Request 50438: FLUME-2961:Make TaildirSource work with multiline

Posted by Balázs Donát Bessenyei <be...@apache.org>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/50438/#review167074
-----------------------------------------------------------




flume-ng-doc/sphinx/FlumeUserGuide.rst (line 1191)
<https://reviews.apache.org/r/50438/#comment239204>

    Can you please clarify this setting?



flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java (line 115)
<https://reviews.apache.org/r/50438/#comment239208>

    Isn't maxBytes and maxLines missing here?



flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java (line 196)
<https://reviews.apache.org/r/50438/#comment239201>

    Can the pattern be cached? (Eg. in configure)


- Bal�zs Don�t Bessenyei


On Feb. 17, 2017, 11:13 a.m., qiao wen wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/50438/
> -----------------------------------------------------------
> 
> (Updated Feb. 17, 2017, 11:13 a.m.)
> 
> 
> Review request for Flume.
> 
> 
> Repository: flume-git
> 
> 
> Description
> -------
> 
> TaidirSource defaults to LINE, this has issue when multiline log events like stack traces and have request/responses. Following part is Java traceback logs. We expect to have log line start regex Key to aggregate all the log lines till the next regex key is found.
> 2016-07-16 14:59:43,956 ERROR lifecycleSupervisor-1-7 LifecycleSupervisor.run - Unable to start EventDrivenSourceRunner: { source:cn.yottabyte.flume.source.http.HTTPSource{name:sourceHttp,state:IDLE} } - Exception follows.
> java.lang.IllegalStateException: Running HTTP Server found in source: sourceHttp before I started one. Will not attempt to start.
>     at com.google.common.base.Preconditions.checkState(Preconditions.java:145)
>     at cn.yottabyte.flume.source.http.HTTPSource.startHttpSourceServer(HTTPSource.java:170)
>     at cn.yottabyte.flume.source.http.HTTPSource.start(HTTPSource.java:166)
>     at org.apache.flume.source.EventDrivenSourceRunner.start(EventDrivenSourceRunner.java:44)
>     at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:251)
>     at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>     at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
>     at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
>     at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>     at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>     at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>     at java.lang.Thread.run(Thread.java:745)
> 
> 
> Diffs
> -----
> 
>   flume-ng-doc/sphinx/FlumeUserGuide.rst afa6625 
>   flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java 8838320 
>   flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java 42474c4 
>   flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java a107a01 
>   flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java f2347f3 
>   flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java 097ee0b 
> 
> Diff: https://reviews.apache.org/r/50438/diff/
> 
> 
> Testing
> -------
> 
> All tests in TestTaildirSource passed.
> 
> 
> Thanks,
> 
> qiao wen
> 
>


Re: Review Request 50438: FLUME-2961:Make TaildirSource work with multiline

Posted by qiao wen <31...@qq.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/50438/
-----------------------------------------------------------

(Updated \u4e8c\u6708 17, 2017, 11:13 a.m.)


Review request for Flume.


Changes
-------

Add some configurations about multiline.
multiline: Whether to support joining of multiline messages into a single flume event.
multilinePattern: Regexp which matches the start or the end of an event consisting of multilines.
multilinePatternBelong: Indicate the pattern belongs to the next or previous event. Value can be {'previous','next'}.
multilineMatched: Whether to match the pattern. If 'false', a message not matching the pattern will be combined with the previous or the next line.
multilineEventTimeoutSeconds: Maximum seconds before an event automatically be flushed. Default value 0 means never time out.
multilineMaxBytes: If the length of multiline event bytes exceeds this value, the event will be flushed. Default value 10MB. It's used in combination multilineMaxLines.
multilineMaxLines :If the lines of multiline event exceeds this value, the event will be flushed. Default value 500. It's used in combination multilineMaxBytes.


Repository: flume-git


Description
-------

TaidirSource defaults to LINE, this has issue when multiline log events like stack traces and have request/responses. Following part is Java traceback logs. We expect to have log line start regex Key to aggregate all the log lines till the next regex key is found.
2016-07-16 14:59:43,956 ERROR lifecycleSupervisor-1-7 LifecycleSupervisor.run - Unable to start EventDrivenSourceRunner: { source:cn.yottabyte.flume.source.http.HTTPSource{name:sourceHttp,state:IDLE} } - Exception follows.
java.lang.IllegalStateException: Running HTTP Server found in source: sourceHttp before I started one. Will not attempt to start.
    at com.google.common.base.Preconditions.checkState(Preconditions.java:145)
    at cn.yottabyte.flume.source.http.HTTPSource.startHttpSourceServer(HTTPSource.java:170)
    at cn.yottabyte.flume.source.http.HTTPSource.start(HTTPSource.java:166)
    at org.apache.flume.source.EventDrivenSourceRunner.start(EventDrivenSourceRunner.java:44)
    at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:251)
    at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
    at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
    at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
    at java.lang.Thread.run(Thread.java:745)


Diffs (updated)
-----

  flume-ng-doc/sphinx/FlumeUserGuide.rst afa6625 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/ReliableTaildirEventReader.java 8838320 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TailFile.java 42474c4 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSource.java a107a01 
  flume-ng-sources/flume-taildir-source/src/main/java/org/apache/flume/source/taildir/TaildirSourceConfigurationConstants.java f2347f3 
  flume-ng-sources/flume-taildir-source/src/test/java/org/apache/flume/source/taildir/TestTaildirSource.java 097ee0b 

Diff: https://reviews.apache.org/r/50438/diff/


Testing
-------

All tests in TestTaildirSource passed.


Thanks,

qiao wen