NIFI-2565: add Grok parser

This closes #1108.

Signed-off-by: Andre F de Miranda <trixpan@users.noreply.github.com>
This commit is contained in:
Selim Namsi 2016-10-05 18:54:37 +02:00 committed by Andre F de Miranda
parent 17cb2e284d
commit 2ef7c15b5d
9 changed files with 534 additions and 0 deletions

View File

@ -1059,6 +1059,11 @@ The following binary components are provided under the Apache Software License v
ParCEFone
Copyright 2016 Fluenda
(ASLv2) Grok
The following NOTICE information applies:
Grok
Copyright 2014 Anthony Corbacho, and contributors.
(ASLv2) The Netty Project
The following NOTICE information applies:

View File

@ -169,6 +169,11 @@ The following binary components are provided under the Apache Software License v
ParCEFone
Copyright 2016 Fluenda
(ASLv2) Grok
The following NOTICE information applies:
Grok
Copyright 2014 Anthony Corbacho, and contributors.
************************
Common Development and Distribution License 1.1
************************

View File

@ -260,6 +260,17 @@ language governing permissions and limitations under the License. -->
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>io.thekraken</groupId>
<artifactId>grok</artifactId>
<version>0.1.5</version>
<exclusions>
<exclusion>
<groupId>com.google.code</groupId>
<artifactId>gson</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>
<build>
@ -390,6 +401,9 @@ language governing permissions and limitations under the License. -->
<exclude>src/test/resources/TestEncryptContent/salted_128_raw.enc</exclude>
<exclude>src/test/resources/TestEncryptContent/unsalted_raw.enc</exclude>
<exclude>src/test/resources/TestEncryptContent/unsalted_128_raw.enc</exclude>
<exclude>src/test/resources/TestExtractGrok/apache.log</exclude>
<exclude>src/test/resources/TestExtractGrok/simple_text.log</exclude>
<exclude>src/test/resources/TestExtractGrok/patterns</exclude>
<!-- This file is copied from https://github.com/jeremyh/jBCrypt because the binary is compiled for Java 8 and we must support Java 7 -->
<exclude>src/main/java/org/apache/nifi/processors/standard/util/crypto/bcrypt/BCrypt.java</exclude>
</excludes>

View File

@ -0,0 +1,299 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.standard;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.thekraken.grok.api.Grok;
import io.thekraken.grok.api.Match;
import io.thekraken.grok.api.exception.GrokException;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.io.StreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.stream.io.BufferedOutputStream;
import org.apache.nifi.stream.io.StreamUtils;
import org.apache.nifi.util.StopWatch;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.Charset;
import java.util.List;
import java.util.Map;
import java.util.HashMap;
import java.util.Set;
import java.util.HashSet;
import java.util.ArrayList;
import java.util.Collections;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
@Tags({"grok", "log", "text", "parse", "delimit", "extract"})
@CapabilityDescription("Evaluates one or more Grok Expressions against the content of a FlowFile, " +
"adding the results as attributes or replacing the content of the FlowFile with a JSON " +
"notation of the matched content")
@WritesAttributes({
@WritesAttribute(attribute = "grok.XXX", description = "When operating in flowfile-attribute mode, each of the Grok identifier that is matched in the flowfile " +
"will be added as an attribute, prefixed with \"grok.\" For example," +
"if the grok identifier \"timestamp\" is matched, then the value will be added to an attribute named \"grok.timestamp\"")})
public class ExtractGrok extends AbstractProcessor {
public static final String FLOWFILE_ATTRIBUTE = "flowfile-attribute";
public static final String FLOWFILE_CONTENT = "flowfile-content";
private static final String APPLICATION_JSON = "application/json";
public static final PropertyDescriptor GROK_EXPRESSION = new PropertyDescriptor
.Builder().name("Grok Expression")
.description("Grok expression")
.required(true)
.addValidator(validateGrokExpression())
.build();
public static final PropertyDescriptor GROK_PATTERN_FILE = new PropertyDescriptor
.Builder().name("Grok Pattern file")
.description("Grok Pattern file definition")
.required(true)
.addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
.build();
public static final PropertyDescriptor DESTINATION = new PropertyDescriptor.Builder()
.name("Destination")
.description("Control if Grok output value is written as a new flowfile attributes, in this case " +
"each of the Grok identifier that is matched in the flowfile will be added as an attribute, " +
"prefixed with \"grok.\" or written in the flowfile content. Writing to flowfile content " +
"will overwrite any existing flowfile content.")
.required(true)
.allowableValues(FLOWFILE_ATTRIBUTE, FLOWFILE_CONTENT)
.defaultValue(FLOWFILE_ATTRIBUTE)
.build();
public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor
.Builder().name("Character Set")
.description("The Character Set in which the file is encoded")
.required(true)
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
.defaultValue("UTF-8")
.build();
public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor
.Builder().name("Maximum Buffer Size")
.description("Specifies the maximum amount of data to buffer (per file) in order to apply the Grok expressions. Files larger than the specified maximum will not be fully evaluated.")
.required(true)
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
.addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE))
.defaultValue("1 MB")
.build();
public static final Relationship REL_MATCH = new Relationship.Builder()
.name("matched")
.description("FlowFiles are routed to this relationship when the Grok Expression is successfully evaluated and the FlowFile is modified as a result")
.build();
public static final Relationship REL_NO_MATCH = new Relationship.Builder()
.name("unmatched")
.description("FlowFiles are routed to this relationship when no provided Grok Expression matches the content of the FlowFile")
.build();
private final static List<PropertyDescriptor> descriptors;
private final static Set<Relationship> relationships;
private final static Grok grok = Grok.EMPTY;
private final BlockingQueue<byte[]> bufferQueue = new LinkedBlockingQueue<>();
static {
final Set<Relationship> _relationships = new HashSet<>();
_relationships.add(REL_MATCH);
_relationships.add(REL_NO_MATCH);
relationships = Collections.unmodifiableSet(_relationships);
final List<PropertyDescriptor> _descriptors = new ArrayList<>();
_descriptors.add(GROK_EXPRESSION);
_descriptors.add(GROK_PATTERN_FILE);
_descriptors.add(DESTINATION);
_descriptors.add(CHARACTER_SET);
_descriptors.add(MAX_BUFFER_SIZE);
descriptors = Collections.unmodifiableList(_descriptors);
}
@Override
public Set<Relationship> getRelationships() {
return relationships;
}
@Override
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return descriptors;
}
@OnStopped
public void onStopped() {
bufferQueue.clear();
}
@OnScheduled
public void onScheduled(final ProcessContext context) throws GrokException {
for (int i = 0; i < context.getMaxConcurrentTasks(); i++) {
final int maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
final byte[] buffer = new byte[maxBufferSize];
bufferQueue.add(buffer);
}
grok.addPatternFromFile(context.getProperty(GROK_PATTERN_FILE).getValue());
grok.compile(context.getProperty(GROK_EXPRESSION).getValue());
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
FlowFile flowFile = session.get();
if (flowFile == null) {
return;
}
final StopWatch stopWatch = new StopWatch(true);
final Charset charset = Charset.forName(context.getProperty(CHARACTER_SET).getValue());
final String contentString;
byte[] buffer = bufferQueue.poll();
if (buffer == null) {
final int maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
buffer = new byte[maxBufferSize];
}
try {
final byte[] byteBuffer = buffer;
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(InputStream in) throws IOException {
StreamUtils.fillBuffer(in, byteBuffer, false);
}
});
final long len = Math.min(byteBuffer.length, flowFile.getSize());
contentString = new String(byteBuffer, 0, (int) len, charset);
} finally {
bufferQueue.offer(buffer);
}
final Match gm = grok.match(contentString);
gm.captures();
if (gm.toMap().isEmpty()) {
session.transfer(flowFile, REL_NO_MATCH);
getLogger().info("Did not match any Grok Expressions for FlowFile {}", new Object[]{flowFile});
return;
}
final ObjectMapper objectMapper = new ObjectMapper();
switch (context.getProperty(DESTINATION).getValue()) {
case FLOWFILE_ATTRIBUTE:
Map<String, String> grokResults = new HashMap<>();
for (Map.Entry<String, Object> entry : gm.toMap().entrySet()) {
if (null != entry.getValue()) {
grokResults.put("grok." + entry.getKey(), entry.getValue().toString());
}
}
flowFile = session.putAllAttributes(flowFile, grokResults);
session.getProvenanceReporter().modifyAttributes(flowFile);
session.transfer(flowFile, REL_MATCH);
getLogger().info("Matched {} Grok Expressions and added attributes to FlowFile {}", new Object[]{grokResults.size(), flowFile});
break;
case FLOWFILE_CONTENT:
FlowFile conFlowfile = session.write(flowFile, new StreamCallback() {
@Override
public void process(InputStream in, OutputStream out) throws IOException {
try (OutputStream outputStream = new BufferedOutputStream(out)) {
outputStream.write(objectMapper.writeValueAsBytes(gm.toMap()));
}
}
});
conFlowfile = session.putAttribute(conFlowfile, CoreAttributes.MIME_TYPE.key(), APPLICATION_JSON);
session.getProvenanceReporter().modifyContent(conFlowfile, "Replaced content with parsed Grok fields and values", stopWatch.getElapsed(TimeUnit.MILLISECONDS));
session.transfer(conFlowfile, REL_MATCH);
break;
}
}
public static final Validator validateGrokExpression() {
return new Validator() {
@Override
public ValidationResult validate(String subject, String input, ValidationContext context) {
Grok grok = new Grok();
try {
grok.compile(input);
} catch (GrokException e) {
return new ValidationResult.Builder()
.subject(subject)
.input(input)
.valid(false)
.explanation("Not a valid Grok Expression")
.build();
} catch (java.util.regex.PatternSyntaxException e) {
return new ValidationResult.Builder()
.subject(subject)
.input(input)
.valid(false)
.explanation("Not a valid Grok Expression")
.build();
}
return new ValidationResult.Builder().subject(subject).input(input).valid(true).build();
}
};
}
}

View File

@ -46,6 +46,7 @@ org.apache.nifi.processors.standard.JoltTransformJSON
org.apache.nifi.processors.standard.GenerateTableFetch
org.apache.nifi.processors.standard.GetJMSQueue
org.apache.nifi.processors.standard.GetJMSTopic
org.apache.nifi.processors.standard.ExtractGrok
org.apache.nifi.processors.standard.ListDatabaseTables
org.apache.nifi.processors.standard.ListFile
org.apache.nifi.processors.standard.ListenHTTP

View File

@ -0,0 +1,101 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.nifi.processors.standard;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.nio.file.Path;
import java.nio.file.Paths;
public class TestExtractGrok {
private TestRunner testRunner;
final static Path GROK_LOG_INPUT = Paths.get("src/test/resources/TestExtractGrok/apache.log");
final static Path GROK_TEXT_INPUT = Paths.get("src/test/resources/TestExtractGrok/simple_text.log");
@Before
public void init() {
testRunner = TestRunners.newTestRunner(ExtractGrok.class);
}
@Test
public void testExtractGrokWithMatchedContent() throws IOException {
testRunner.setProperty(ExtractGrok.GROK_EXPRESSION, "%{COMMONAPACHELOG}");
testRunner.setProperty(ExtractGrok.GROK_PATTERN_FILE, "src/test/resources/TestExtractGrok/patterns");
testRunner.enqueue(GROK_LOG_INPUT);
testRunner.run();
testRunner.assertAllFlowFilesTransferred(ExtractGrok.REL_MATCH);
final MockFlowFile matched = testRunner.getFlowFilesForRelationship(ExtractGrok.REL_MATCH).get(0);
matched.assertAttributeEquals("grok.verb","GET");
matched.assertAttributeEquals("grok.response","401");
matched.assertAttributeEquals("grok.bytes","12846");
matched.assertAttributeEquals("grok.clientip","64.242.88.10");
matched.assertAttributeEquals("grok.auth","-");
matched.assertAttributeEquals("grok.timestamp","07/Mar/2004:16:05:49 -0800");
matched.assertAttributeEquals("grok.request","/twiki/bin/edit/Main/Double_bounce_sender?topicparent=Main.ConfigurationVariables");
matched.assertAttributeEquals("grok.httpversion","1.1");
}
@Test
public void testExtractGrokWithUnMatchedContent() throws IOException {
testRunner.setProperty(ExtractGrok.GROK_EXPRESSION, "%{ADDRESS}");
testRunner.setProperty(ExtractGrok.GROK_PATTERN_FILE, "src/test/resources/TestExtractGrok/patterns");
testRunner.enqueue(GROK_TEXT_INPUT);
testRunner.run();
testRunner.assertAllFlowFilesTransferred(ExtractGrok.REL_NO_MATCH);
final MockFlowFile notMatched = testRunner.getFlowFilesForRelationship(ExtractGrok.REL_NO_MATCH).get(0);
notMatched.assertContentEquals(GROK_TEXT_INPUT);
}
@Test
public void testExtractGrokWithNotFoundPatternFile() throws IOException {
testRunner.setProperty(ExtractGrok.GROK_EXPRESSION, "%{COMMONAPACHELOG}");
testRunner.setProperty(ExtractGrok.GROK_PATTERN_FILE, "src/test/resources/TestExtractGrok/toto_file");
testRunner.enqueue(GROK_LOG_INPUT);
testRunner.assertNotValid();
}
@Test
public void testExtractGrokWithBadGrokExpression() throws IOException {
testRunner.setProperty(ExtractGrok.GROK_EXPRESSION, "%{TOTO");
testRunner.setProperty(ExtractGrok.GROK_PATTERN_FILE, "src/test/resources/TestExtractGrok/patterns");
testRunner.enqueue(GROK_LOG_INPUT);
testRunner.assertNotValid();
}
}

View File

@ -0,0 +1 @@
64.242.88.10 - - [07/Mar/2004:16:05:49 -0800] "GET /twiki/bin/edit/Main/Double_bounce_sender?topicparent=Main.ConfigurationVariables HTTP/1.1" 401 12846

View File

@ -0,0 +1,107 @@
USERNAME [a-zA-Z0-9._-]+
USER %{USERNAME:UNWANTED}
INT (?:[+-]?(?:[0-9]+))
BASE10NUM (?<![0-9.+-])(?>[+-]?(?:(?:[0-9]+(?:\.[0-9]+)?)|(?:\.[0-9]+)))
NUMBER (?:%{BASE10NUM:UNWANTED})
BASE16NUM (?<![0-9A-Fa-f])(?:[+-]?(?:0x)?(?:[0-9A-Fa-f]+))
BASE16FLOAT \b(?<![0-9A-Fa-f.])(?:[+-]?(?:0x)?(?:(?:[0-9A-Fa-f]+(?:\.[0-9A-Fa-f]*)?)|(?:\.[0-9A-Fa-f]+)))\b
POSINT \b(?:[1-9][0-9]*)\b
NONNEGINT \b(?:[0-9]+)\b
WORD \b\w+\b
NOTSPACE \S+
SPACE \s*
DATA .*?
GREEDYDATA .*
#QUOTEDSTRING (?:(?<!\\)(?:"(?:\\.|[^\\"])*"|(?:'(?:\\.|[^\\'])*')|(?:`(?:\\.|[^\\`])*`)))
QUOTEDSTRING (?>(?<!\\)(?>"(?>\\.|[^\\"]+)+"|""|(?>'(?>\\.|[^\\']+)+')|''|(?>`(?>\\.|[^\\`]+)+`)|``))
UUID [A-Fa-f0-9]{8}-(?:[A-Fa-f0-9]{4}-){3}[A-Fa-f0-9]{12}
# Networking
MAC (?:%{CISCOMAC:UNWANTED}|%{WINDOWSMAC:UNWANTED}|%{COMMONMAC:UNWANTED})
CISCOMAC (?:(?:[A-Fa-f0-9]{4}\.){2}[A-Fa-f0-9]{4})
WINDOWSMAC (?:(?:[A-Fa-f0-9]{2}-){5}[A-Fa-f0-9]{2})
COMMONMAC (?:(?:[A-Fa-f0-9]{2}:){5}[A-Fa-f0-9]{2})
IPV6 ((([0-9A-Fa-f]{1,4}:){7}([0-9A-Fa-f]{1,4}|:))|(([0-9A-Fa-f]{1,4}:){6}(:[0-9A-Fa-f]{1,4}|((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){5}(((:[0-9A-Fa-f]{1,4}){1,2})|:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){4}(((:[0-9A-Fa-f]{1,4}){1,3})|((:[0-9A-Fa-f]{1,4})?:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){3}(((:[0-9A-Fa-f]{1,4}){1,4})|((:[0-9A-Fa-f]{1,4}){0,2}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){2}(((:[0-9A-Fa-f]{1,4}){1,5})|((:[0-9A-Fa-f]{1,4}){0,3}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){1}(((:[0-9A-Fa-f]{1,4}){1,6})|((:[0-9A-Fa-f]{1,4}){0,4}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(:(((:[0-9A-Fa-f]{1,4}){1,7})|((:[0-9A-Fa-f]{1,4}){0,5}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:)))(%.+)?
IPV4 (?<![0-9])(?:(?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2})[.](?:25[0-5]|2[0-4][0-9]|[0-1]?[0-9]{1,2}))(?![0-9])
IP (?:%{IPV6:UNWANTED}|%{IPV4:UNWANTED})
HOSTNAME \b(?:[0-9A-Za-z][0-9A-Za-z-]{0,62})(?:\.(?:[0-9A-Za-z][0-9A-Za-z-]{0,62}))*(\.?|\b)
HOST %{HOSTNAME:UNWANTED}
IPORHOST (?:%{HOSTNAME:UNWANTED}|%{IP:UNWANTED})
HOSTPORT (?:%{IPORHOST}:%{POSINT:PORT})
# paths
PATH (?:%{UNIXPATH}|%{WINPATH})
UNIXPATH (?>/(?>[\w_%!$@:.,~-]+|\\.)*)+
#UNIXPATH (?<![\w\/])(?:/[^\/\s?*]*)+
TTY (?:/dev/(pts|tty([pq])?)(\w+)?/?(?:[0-9]+))
WINPATH (?>[A-Za-z]+:|\\)(?:\\[^\\?*]*)+
URIPROTO [A-Za-z]+(\+[A-Za-z+]+)?
URIHOST %{IPORHOST}(?::%{POSINT:port})?
# uripath comes loosely from RFC1738, but mostly from what Firefox
# doesn't turn into %XX
URIPATH (?:/[A-Za-z0-9$.+!*'(){},~:;=@#%_\-]*)+
#URIPARAM \?(?:[A-Za-z0-9]+(?:=(?:[^&]*))?(?:&(?:[A-Za-z0-9]+(?:=(?:[^&]*))?)?)*)?
URIPARAM \?[A-Za-z0-9$.+!*'|(){},~@#%&/=:;_?\-\[\]]*
URIPATHPARAM %{URIPATH}(?:%{URIPARAM})?
URI %{URIPROTO}://(?:%{USER}(?::[^@]*)?@)?(?:%{URIHOST})?(?:%{URIPATHPARAM})?
# Months: January, Feb, 3, 03, 12, December
MONTH \b(?:Jan(?:uary)?|Feb(?:ruary)?|Mar(?:ch)?|Apr(?:il)?|May|Jun(?:e)?|Jul(?:y)?|Aug(?:ust)?|Sep(?:tember)?|Oct(?:ober)?|Nov(?:ember)?|Dec(?:ember)?)\b
MONTHNUM (?:0?[1-9]|1[0-2])
MONTHNUM2 (?:0[1-9]|1[0-2])
MONTHDAY (?:(?:0[1-9])|(?:[12][0-9])|(?:3[01])|[1-9])
# Days: Monday, Tue, Thu, etc...
DAY (?:Mon(?:day)?|Tue(?:sday)?|Wed(?:nesday)?|Thu(?:rsday)?|Fri(?:day)?|Sat(?:urday)?|Sun(?:day)?)
# Years?
YEAR (?>\d\d){1,2}
# Time: HH:MM:SS
#TIME \d{2}:\d{2}(?::\d{2}(?:\.\d+)?)?
# I'm still on the fence about using grok to perform the time match,
# since it's probably slower.
# TIME %{POSINT<24}:%{POSINT<60}(?::%{POSINT<60}(?:\.%{POSINT})?)?
HOUR (?:2[0123]|[01]?[0-9])
MINUTE (?:[0-5][0-9])
# '60' is a leap second in most time standards and thus is valid.
SECOND (?:(?:[0-5]?[0-9]|60)(?:[:.,][0-9]+)?)
TIME (?!<[0-9])%{HOUR}:%{MINUTE}(?::%{SECOND})(?![0-9])
# datestamp is YYYY/MM/DD-HH:MM:SS.UUUU (or something like it)
DATE_US %{MONTHNUM}[/-]%{MONTHDAY}[/-]%{YEAR}
DATE_EU %{MONTHDAY}[./-]%{MONTHNUM}[./-]%{YEAR}
ISO8601_TIMEZONE (?:Z|[+-]%{HOUR}(?::?%{MINUTE}))
ISO8601_SECOND (?:%{SECOND}|60)
TIMESTAMP_ISO8601 %{YEAR}-%{MONTHNUM}-%{MONTHDAY}[T ]%{HOUR}:?%{MINUTE}(?::?%{SECOND})?%{ISO8601_TIMEZONE}?
DATE %{DATE_US}|%{DATE_EU}
DATESTAMP %{DATE}[- ]%{TIME}
TZ (?:[PMCE][SD]T|UTC)
DATESTAMP_RFC822 %{DAY} %{MONTH} %{MONTHDAY} %{YEAR} %{TIME} %{TZ}
DATESTAMP_RFC2822 %{DAY}, %{MONTHDAY} %{MONTH} %{YEAR} %{TIME} %{ISO8601_TIMEZONE}
DATESTAMP_OTHER %{DAY} %{MONTH} %{MONTHDAY} %{TIME} %{TZ} %{YEAR}
DATESTAMP_EVENTLOG %{YEAR}%{MONTHNUM2}%{MONTHDAY}%{HOUR}%{MINUTE}%{SECOND}
# Syslog Dates: Month Day HH:MM:SS
SYSLOGTIMESTAMP %{MONTH} +%{MONTHDAY} %{TIME}
PROG (?:[\w._/%-]+)
SYSLOGPROG %{PROG:program}(?:\[%{POSINT:pid}\])?
SYSLOGHOST %{IPORHOST}
SYSLOGFACILITY <%{NONNEGINT:facility}.%{NONNEGINT:priority}>
HTTPDATE %{MONTHDAY}/%{MONTH}/%{YEAR}:%{TIME} %{INT}
# Shortcuts
QS %{QUOTEDSTRING:UNWANTED}
# Log formats
SYSLOGBASE %{SYSLOGTIMESTAMP:timestamp} (?:%{SYSLOGFACILITY} )?%{SYSLOGHOST:logsource} %{SYSLOGPROG}:
MESSAGESLOG %{SYSLOGBASE} %{DATA}
COMMONAPACHELOG %{IPORHOST:clientip} %{USER:ident} %{USER:auth} \[%{HTTPDATE:timestamp}\] "(?:%{WORD:verb} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion})?|%{DATA:rawrequest})" %{NUMBER:response} (?:%{NUMBER:bytes}|-)
COMBINEDAPACHELOG %{COMMONAPACHELOG} %{QS:referrer} %{QS:agent}
COMMONAPACHELOG_DATATYPED %{IPORHOST:clientip} %{USER:ident;boolean} %{USER:auth} \[%{HTTPDATE:timestamp;date;dd/MMM/yyyy:HH:mm:ss Z}\] "(?:%{WORD:verb;string} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion;float})?|%{DATA:rawrequest})" %{NUMBER:response;int} (?:%{NUMBER:bytes;long}|-)
# Log Levels
LOGLEVEL ([A|a]lert|ALERT|[T|t]race|TRACE|[D|d]ebug|DEBUG|[N|n]otice|NOTICE|[I|i]nfo|INFO|[W|w]arn?(?:ing)?|WARN?(?:ING)?|[E|e]rr?(?:or)?|ERR?(?:OR)?|[C|c]rit?(?:ical)?|CRIT?(?:ICAL)?|[F|f]atal|FATAL|[S|s]evere|SEVERE|EMERG(?:ENCY)?|[Ee]merg(?:ency)?)