Updated Branches: refs/heads/flume-1.4 9dade66ae -> 64b73dfa3
FLUME-1277. Error parsing Syslog rfc 3164 messages with null values. (Brock Noland via Mike Percy) Project: http://git-wip-us.apache.org/repos/asf/flume/repo Commit: http://git-wip-us.apache.org/repos/asf/flume/commit/64b73dfa Tree: http://git-wip-us.apache.org/repos/asf/flume/tree/64b73dfa Diff: http://git-wip-us.apache.org/repos/asf/flume/diff/64b73dfa Branch: refs/heads/flume-1.4 Commit: 64b73dfa381cdc0dfc744e7283471279ef26b2aa Parents: 9dade66 Author: Mike Percy <[email protected]> Authored: Sat Nov 10 12:35:07 2012 -0800 Committer: Mike Percy <[email protected]> Committed: Sat Nov 10 12:37:38 2012 -0800 ---------------------------------------------------------------------- .../java/org/apache/flume/source/SyslogUtils.java | 52 +++++++-------- .../org/apache/flume/source/TestSyslogUtils.java | 22 +++++-- 2 files changed, 42 insertions(+), 32 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flume/blob/64b73dfa/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java ---------------------------------------------------------------------- diff --git a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java index 4cded11..c2a29a1 100644 --- a/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java +++ b/flume-ng-core/src/main/java/org/apache/flume/source/SyslogUtils.java @@ -19,6 +19,12 @@ package org.apache.flume.source; +import org.apache.flume.Event; +import org.apache.flume.event.EventBuilder; +import org.jboss.netty.buffer.ChannelBuffer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.ByteArrayOutputStream; import java.io.IOException; import java.text.ParseException; @@ -27,14 +33,9 @@ import java.util.ArrayList; import java.util.Calendar; import java.util.HashMap; import java.util.Map; -import java.util.Scanner; import java.util.regex.MatchResult; - -import org.apache.flume.Event; -import org.apache.flume.event.EventBuilder; -import org.jboss.netty.buffer.ChannelBuffer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import java.util.regex.Matcher; +import java.util.regex.Pattern; public class SyslogUtils { final public static String SYSLOG_TIMESTAMP_FORMAT_RFC5424_2 = "yyyy-MM-dd'T'HH:mm:ss.SZ"; @@ -44,7 +45,7 @@ public class SyslogUtils { final public static String SYSLOG_TIMESTAMP_FORMAT_RFC3164_1 = "yyyyMMM d HH:mm:ss"; final public static String SYSLOG_MSG_RFC5424_0 = - "[(?:\\d\\s)]?" +// version + "(?:\\d\\s)?" +// version // yyyy-MM-dd'T'HH:mm:ss.SZ or yyyy-MM-dd'T'HH:mm:ss.S+hh:mm or - (null stamp) "(?:(\\d{4}[-]\\d{2}[-]\\d{2}[T]\\d{2}[:]\\d{2}[:]\\d{2}(?:\\.\\d{1,6})?(?:[+-]\\d{2}[:]\\d{2}|Z)?)|-)" + // stamp "\\s" + // separator @@ -79,14 +80,14 @@ public class SyslogUtils { private boolean isIncompleteEvent; private Integer maxSize; - private class SyslogFormater { - public String regexPattern; + private class SyslogFormatter { + public Pattern regexPattern; public ArrayList<String> searchPattern = new ArrayList<String>(); public ArrayList<String> replacePattern = new ArrayList<String>(); public ArrayList<SimpleDateFormat> dateFormat = new ArrayList<SimpleDateFormat>(); public boolean addYear; } - private ArrayList<SyslogFormater> formats = new ArrayList<SyslogFormater>(); + private ArrayList<SyslogFormatter> formats = new ArrayList<SyslogFormatter>(); private String timeStamp = null; private String hostName = null; @@ -115,9 +116,9 @@ public class SyslogUtils { SyslogSourceConfigurationConstants.CONFIG_REGEX)) { return; } - SyslogFormater fmt1 = new SyslogFormater(); - fmt1.regexPattern = formatProp.get( - SyslogSourceConfigurationConstants.CONFIG_REGEX); + SyslogFormatter fmt1 = new SyslogFormatter(); + fmt1.regexPattern = Pattern.compile( formatProp.get( + SyslogSourceConfigurationConstants.CONFIG_REGEX) ); if (formatProp.containsKey( SyslogSourceConfigurationConstants.CONFIG_SEARCH)) { fmt1.searchPattern.add(formatProp.get( @@ -139,8 +140,8 @@ public class SyslogUtils { // setup built-in formats private void initHeaderFormats() { // setup RFC5424 formater - SyslogFormater fmt1 = new SyslogFormater(); - fmt1.regexPattern = SYSLOG_MSG_RFC5424_0; + SyslogFormatter fmt1 = new SyslogFormatter(); + fmt1.regexPattern = Pattern.compile(SYSLOG_MSG_RFC5424_0); // 'Z' in timestamp indicates UTC zone, so replace it it with '+0000' for date formatting fmt1.searchPattern.add("Z"); fmt1.replacePattern.add("+0000"); @@ -154,8 +155,8 @@ public class SyslogUtils { fmt1.addYear = false; // setup RFC3164 formater - SyslogFormater fmt2 = new SyslogFormater(); - fmt2.regexPattern = SYSLOG_MSG_RFC3164_0; + SyslogFormatter fmt2 = new SyslogFormatter(); + fmt2.regexPattern = Pattern.compile(SYSLOG_MSG_RFC3164_0); // the single digit date has two spaces, so trim it fmt2.searchPattern.add(" "); fmt2.replacePattern.add(" "); @@ -230,18 +231,15 @@ public class SyslogUtils { // Apply each known pattern to message private void formatHeaders() { - Scanner scanner = new Scanner(baos.toString()); - MatchResult res = null; - + String eventStr = baos.toString(); for(int p=0; p < formats.size(); p++) { - SyslogFormater fmt = formats.get(p); - try { - scanner.findInLine(fmt.regexPattern); - res = scanner.match(); - } catch (IllegalStateException e) { - // Ignore and move on .. + SyslogFormatter fmt = formats.get(p); + Pattern pattern = fmt.regexPattern; + Matcher matcher = pattern.matcher(eventStr); + if (! matcher.matches()) { continue; } + MatchResult res = matcher.toMatchResult(); for (int grp=1; grp <= res.groupCount(); grp++) { String value = res.group(grp); if (grp == SYSLOG_TIMESTAMP_POS) { http://git-wip-us.apache.org/repos/asf/flume/blob/64b73dfa/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java ---------------------------------------------------------------------- diff --git a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java index acfb29d..7208464 100644 --- a/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java +++ b/flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUtils.java @@ -19,16 +19,17 @@ package org.apache.flume.source; -import java.text.ParseException; -import java.text.SimpleDateFormat; -import java.util.Calendar; -import java.util.Map; import org.apache.flume.Event; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBuffers; import org.junit.Assert; import org.junit.Test; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Calendar; +import java.util.Map; + public class TestSyslogUtils { @Test public void TestHeader0() throws ParseException { @@ -150,9 +151,20 @@ public class TestSyslogUtils { format1, host1, data1); } + @Test + public void TestRfc3164HeaderApacheLogWithNulls() throws ParseException { + String stamp1 = "Apr 1 13:14:04"; + String format1 = "yyyyMMM d HH:mm:ss"; + String host1 = "ubuntu-11.cloudera.com"; + String data1 = "- hyphen_null_breaks_5424_pattern [07/Jun/2012:14:46:44 -0600]"; + String msg1 = "<10>" + stamp1 + " " + host1 + " " + data1 + "\n"; + checkHeader(msg1, String.valueOf(Calendar.getInstance().get(Calendar.YEAR)) + stamp1, + format1, host1, data1); + } + public void checkHeader(String msg1, String stamp1, String format1, String host1, String data1) throws ParseException { SyslogUtils util = new SyslogUtils(false); - ChannelBuffer buff = ChannelBuffers.buffer(100); + ChannelBuffer buff = ChannelBuffers.buffer(200); buff.writeBytes(msg1.getBytes()); Event e = util.extractEvent(buff);
