[ML] Improve file structure finder timestamp format determination (#41948)

This change contains a major refactoring of the timestamp
format determination code used by the ML find file structure
endpoint.

Previously timestamp format determination was done separately
for each piece of text supplied to the timestamp format finder.
This had the drawback that it was not possible to distinguish
dd/MM and MM/dd in the case where both numbers were 12 or less.
In order to do this sensibly it is best to look across all the
available timestamps and see if one of the numbers is greater
than 12 in any of them.  This necessitates making the timestamp
format finder an instantiable class that can accumulate evidence
over time.

Another problem with the previous approach was that it was only
possible to override the timestamp format to one of a limited
set of timestamp formats.  There was no way out if a file to be
analysed had a timestamp that was sane yet not in the supported
set.  This is now changed to allow any timestamp format that can
be parsed by a combination of these Java date/time formats:
yy, yyyy, M, MM, MMM, MMMM, d, dd, EEE, EEEE, H, HH, h, mm, ss,
a, XX, XXX, zzz
Additionally S letter groups (fractional seconds) are supported
providing they occur after ss and separated from the ss by a dot,
comma or colon.  Spacing and punctuation is also permitted with
the exception of the question mark, newline and carriage return
characters, together with literal text enclosed in single quotes.

The full list of changes/improvements in this refactor is:

- Make TimestampFormatFinder an instantiable class
- Overrides must be specified in Java date/time format - Joda
  format is no longer accepted
- Joda timestamp formats in outputs are now derived from the
  determined or overridden Java timestamp formats, not stored
  separately
- Functionality for determining the "best" timestamp format in
  a set of lines has been moved from TextLogFileStructureFinder
  to TimestampFormatFinder, taking advantage of the fact that
  TimestampFormatFinder is now an instantiable class with state
- The functionality to quickly rule out some possible Grok
  patterns when looking for timestamp formats has been changed
  from using simple regular expressions to the much faster
  approach of using the Shift-And method of sub-string search,
  but using an "alphabet" consisting of just 1 (representing any
  digit) and 0 (representing non-digits)
- Timestamp format overrides are now much more flexible
- Timestamp format overrides that do not correspond to a built-in
  Grok pattern are mapped to a %{CUSTOM_TIMESTAMP} Grok pattern
  whose definition is included within the date processor in the
  ingest pipeline
- Grok patterns that correspond to multiple Java date/time
  patterns are now handled better - the Grok pattern is accepted
  as matching broadly, and the required set of Java date/time
  patterns is built up considering all observed samples
- As a result of the more flexible acceptance of Grok patterns,
  when looking for the "best" timestamp in a set of lines
  timestamps are considered different if they are preceded by
  a different sequence of punctuation characters (to prevent
  timestamps far into some lines being considered similar to
  timestamps near the beginning of other lines)
- Out-of-the-box Grok patterns that are considered now include
  %{DATE} and %{DATESTAMP}, which have indeterminate day/month
  ordering
- The order of day/month in formats with indeterminate day/month
  order is determined by considering all observed samples (plus
  the server locale if the observed samples still do not suggest
  an ordering)

Relates #38086
Closes #35137
Closes #35132
This commit is contained in:
David Roberts 2019-05-23 21:06:47 +01:00
parent f864f6a740
commit f472186b9f
13 changed files with 2893 additions and 1165 deletions

View File

@ -147,57 +147,46 @@ is not compulsory to have a timestamp in the file.
-- --
`timestamp_format`:: `timestamp_format`::
(string) The time format of the timestamp field in the file. + (string) The Java time format of the timestamp field in the file. +
+ +
-- --
NOTE: Currently there is a limitation that this format must be one that the NOTE: Only a subset of Java time format letter groups are supported:
structure finder might choose by itself. The reason for this restriction is that
to consistently set all the fields in the response the structure finder needs a
corresponding Grok pattern name and simple regular expression for each timestamp
format. Therefore, there is little value in specifying this parameter for
structured file formats. If you know which field contains your primary timestamp,
it is as good and less error-prone to just specify `timestamp_field`.
The valuable use case for this parameter is when the format is semi-structured * `a`
* `d`
* `dd`
* `EEE`
* `EEEE`
* `H`
* `HH`
* `h`
* `M`
* `MM`
* `MMM`
* `MMMM`
* `mm`
* `ss`
* `XX`
* `XXX`
* `yy`
* `yyyy`
* `zzz`
Additionally `S` letter groups (fractional seconds) of length one to nine are
supported providing they occur after `ss` and separated from the `ss` by a `.`,
`,` or `:`. Spacing and punctuation is also permitted with the exception of `?`,
newline and carriage return, together with literal text enclosed in single
quotes. For example, `MM/dd HH.mm.ss,SSSSSS 'in' yyyy` is a valid override
format.
One valuable use case for this parameter is when the format is semi-structured
text, there are multiple timestamp formats in the file, and you know which text, there are multiple timestamp formats in the file, and you know which
format corresponds to the primary timestamp, but you do not want to specify the format corresponds to the primary timestamp, but you do not want to specify the
full `grok_pattern`. full `grok_pattern`. Another is when the timestamp format is one that the
structure finder does not consider by default.
If this parameter is not specified, the structure finder chooses the best format from If this parameter is not specified, the structure finder chooses the best
the formats it knows, which are these Java time formats: format from a built-in set.
* `dd/MMM/yyyy:HH:mm:ss XX`
* `EEE MMM dd HH:mm zzz yyyy`
* `EEE MMM dd HH:mm:ss yyyy`
* `EEE MMM dd HH:mm:ss zzz yyyy`
* `EEE MMM dd yyyy HH:mm zzz`
* `EEE MMM dd yyyy HH:mm:ss zzz`
* `EEE, dd MMM yyyy HH:mm XX`
* `EEE, dd MMM yyyy HH:mm XXX`
* `EEE, dd MMM yyyy HH:mm:ss XX`
* `EEE, dd MMM yyyy HH:mm:ss XXX`
* `ISO8601`
* `MMM d HH:mm:ss`
* `MMM d HH:mm:ss,SSS`
* `MMM d yyyy HH:mm:ss`
* `MMM dd HH:mm:ss`
* `MMM dd HH:mm:ss,SSS`
* `MMM dd yyyy HH:mm:ss`
* `MMM dd, yyyy h:mm:ss a`
* `TAI64N`
* `UNIX`
* `UNIX_MS`
* `yyyy-MM-dd HH:mm:ss`
* `yyyy-MM-dd HH:mm:ss,SSS`
* `yyyy-MM-dd HH:mm:ss,SSS XX`
* `yyyy-MM-dd HH:mm:ss,SSSXX`
* `yyyy-MM-dd HH:mm:ss,SSSXXX`
* `yyyy-MM-dd HH:mm:ssXX`
* `yyyy-MM-dd HH:mm:ssXXX`
* `yyyy-MM-dd'T'HH:mm:ss,SSS`
* `yyyy-MM-dd'T'HH:mm:ss,SSSXX`
* `yyyy-MM-dd'T'HH:mm:ss,SSSXXX`
* `yyyyMMddHHmmss`
-- --
@ -263,8 +252,18 @@ If the request does not encounter errors, you receive the following result:
"charset" : "UTF-8", <4> "charset" : "UTF-8", <4>
"has_byte_order_marker" : false, <5> "has_byte_order_marker" : false, <5>
"format" : "ndjson", <6> "format" : "ndjson", <6>
"need_client_timezone" : false, <7> "timestamp_field" : "release_date", <7>
"mappings" : { <8> "joda_timestamp_formats" : [ <8>
"ISO8601"
],
"java_timestamp_formats" : [ <9>
"ISO8601"
],
"need_client_timezone" : true, <10>
"mappings" : { <11>
"@timestamp" : {
"type" : "date"
},
"author" : { "author" : {
"type" : "keyword" "type" : "keyword"
}, },
@ -275,10 +274,25 @@ If the request does not encounter errors, you receive the following result:
"type" : "long" "type" : "long"
}, },
"release_date" : { "release_date" : {
"type" : "keyword" "type" : "date",
"format" : "iso8601"
} }
}, },
"field_stats" : { <9> "ingest_pipeline" : {
"description" : "Ingest pipeline created by file structure finder",
"processors" : [
{
"date" : {
"field" : "release_date",
"timezone" : "{{ beat.timezone }}",
"formats" : [
"ISO8601"
]
}
}
]
},
"field_stats" : { <12>
"author" : { "author" : {
"count" : 24, "count" : 24,
"cardinality" : 20, "cardinality" : 20,
@ -484,17 +498,22 @@ If the request does not encounter errors, you receive the following result:
<5> For UTF character encodings, `has_byte_order_marker` indicates whether the <5> For UTF character encodings, `has_byte_order_marker` indicates whether the
file begins with a byte order marker. file begins with a byte order marker.
<6> `format` is one of `ndjson`, `xml`, `delimited` or `semi_structured_text`. <6> `format` is one of `ndjson`, `xml`, `delimited` or `semi_structured_text`.
<7> If a timestamp format is detected that does not include a timezone, <7> The `timestamp_field` names the field considered most likely to be the
`need_client_timezone` will be `true`. The server that parses the file must primary timestamp of each document.
therefore be told the correct timezone by the client. <8> `joda_timestamp_formats` are used to tell Logstash how to parse timestamps.
<8> `mappings` contains some suitable mappings for an index into which the data <9> `java_timestamp_formats` are the Java time formats recognized in the time
could be ingested. In this case, the `release_date` field has been given a fields. Elasticsearch mappings and Ingest pipeline use this format.
`keyword` type as it is not considered specific enough to convert to the <10> If a timestamp format is detected that does not include a timezone,
`date` type. `need_client_timezone` will be `true`. The server that parses the file must
<9> `field_stats` contains the most common values of each field, plus basic therefore be told the correct timezone by the client.
numeric statistics for the numeric `page_count` field. This information <11> `mappings` contains some suitable mappings for an index into which the data
may provide clues that the data needs to be cleaned or transformed prior could be ingested. In this case, the `release_date` field has been given a
to use by other {ml} functionality. `keyword` type as it is not considered specific enough to convert to the
`date` type.
<12> `field_stats` contains the most common values of each field, plus basic
numeric statistics for the numeric `page_count` field. This information
may provide clues that the data needs to be cleaned or transformed prior
to use by other {ml} functionality.
The next example shows how it's possible to find the structure of some New York The next example shows how it's possible to find the structure of some New York
City yellow cab trip data. The first `curl` command downloads the data, the City yellow cab trip data. The first `curl` command downloads the data, the
@ -526,7 +545,7 @@ If the request does not encounter errors, you receive the following result:
"charset" : "UTF-8", "charset" : "UTF-8",
"has_byte_order_marker" : false, "has_byte_order_marker" : false,
"format" : "delimited", <2> "format" : "delimited", <2>
"multiline_start_pattern" : "^.*?,\"?\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", "multiline_start_pattern" : "^.*?,\"?\\d{4}-\\d{2}-\\d{2}[T ]\\d{2}:\\d{2}",
"exclude_lines_pattern" : "^\"?VendorID\"?,\"?tpep_pickup_datetime\"?,\"?tpep_dropoff_datetime\"?,\"?passenger_count\"?,\"?trip_distance\"?,\"?RatecodeID\"?,\"?store_and_fwd_flag\"?,\"?PULocationID\"?,\"?DOLocationID\"?,\"?payment_type\"?,\"?fare_amount\"?,\"?extra\"?,\"?mta_tax\"?,\"?tip_amount\"?,\"?tolls_amount\"?,\"?improvement_surcharge\"?,\"?total_amount\"?", "exclude_lines_pattern" : "^\"?VendorID\"?,\"?tpep_pickup_datetime\"?,\"?tpep_dropoff_datetime\"?,\"?passenger_count\"?,\"?trip_distance\"?,\"?RatecodeID\"?,\"?store_and_fwd_flag\"?,\"?PULocationID\"?,\"?DOLocationID\"?,\"?payment_type\"?,\"?fare_amount\"?,\"?extra\"?,\"?mta_tax\"?,\"?tip_amount\"?,\"?tolls_amount\"?,\"?improvement_surcharge\"?,\"?total_amount\"?",
"column_names" : [ <3> "column_names" : [ <3>
"VendorID", "VendorID",
@ -1361,14 +1380,14 @@ this:
"charset" : "UTF-8", "charset" : "UTF-8",
"has_byte_order_marker" : false, "has_byte_order_marker" : false,
"format" : "semi_structured_text", <1> "format" : "semi_structured_text", <1>
"multiline_start_pattern" : "^\\[\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}", <2> "multiline_start_pattern" : "^\\[\\b\\d{4}-\\d{2}-\\d{2}[T ]\\d{2}:\\d{2}", <2>
"grok_pattern" : "\\[%{TIMESTAMP_ISO8601:timestamp}\\]\\[%{LOGLEVEL:loglevel}.*", <3> "grok_pattern" : "\\[%{TIMESTAMP_ISO8601:timestamp}\\]\\[%{LOGLEVEL:loglevel}.*", <3>
"timestamp_field" : "timestamp", "timestamp_field" : "timestamp",
"joda_timestamp_formats" : [ "joda_timestamp_formats" : [
"ISO8601" "ISO8601"
], ],
"java_timestamp_formats" : [ "java_timestamp_formats" : [
"yyyy-MM-dd'T'HH:mm:ss,SSS" "ISO8601"
], ],
"need_client_timezone" : true, "need_client_timezone" : true,
"mappings" : { "mappings" : {
@ -1398,7 +1417,7 @@ this:
"field" : "timestamp", "field" : "timestamp",
"timezone" : "{{ beat.timezone }}", "timezone" : "{{ beat.timezone }}",
"formats" : [ "formats" : [
"yyyy-MM-dd'T'HH:mm:ss,SSS" "ISO8601"
] ]
} }
}, },
@ -1515,14 +1534,14 @@ this:
"charset" : "UTF-8", "charset" : "UTF-8",
"has_byte_order_marker" : false, "has_byte_order_marker" : false,
"format" : "semi_structured_text", "format" : "semi_structured_text",
"multiline_start_pattern" : "^\\[\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}", "multiline_start_pattern" : "^\\[\\b\\d{4}-\\d{2}-\\d{2}[T ]\\d{2}:\\d{2}",
"grok_pattern" : "\\[%{TIMESTAMP_ISO8601:timestamp}\\]\\[%{LOGLEVEL:loglevel} *\\]\\[%{JAVACLASS:class} *\\] \\[%{HOSTNAME:node}\\] %{JAVALOGMESSAGE:message}", <1> "grok_pattern" : "\\[%{TIMESTAMP_ISO8601:timestamp}\\]\\[%{LOGLEVEL:loglevel} *\\]\\[%{JAVACLASS:class} *\\] \\[%{HOSTNAME:node}\\] %{JAVALOGMESSAGE:message}", <1>
"timestamp_field" : "timestamp", "timestamp_field" : "timestamp",
"joda_timestamp_formats" : [ "joda_timestamp_formats" : [
"ISO8601" "ISO8601"
], ],
"java_timestamp_formats" : [ "java_timestamp_formats" : [
"yyyy-MM-dd'T'HH:mm:ss,SSS" "ISO8601"
], ],
"need_client_timezone" : true, "need_client_timezone" : true,
"mappings" : { "mappings" : {
@ -1558,7 +1577,7 @@ this:
"field" : "timestamp", "field" : "timestamp",
"timezone" : "{{ beat.timezone }}", "timezone" : "{{ beat.timezone }}",
"formats" : [ "formats" : [
"yyyy-MM-dd'T'HH:mm:ss,SSS" "ISO8601"
] ]
} }
}, },

View File

@ -8,7 +8,6 @@ package org.elasticsearch.xpack.ml.filestructurefinder;
import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats; import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats;
import org.elasticsearch.xpack.core.ml.filestructurefinder.FileStructure; import org.elasticsearch.xpack.core.ml.filestructurefinder.FileStructure;
import org.elasticsearch.xpack.ml.filestructurefinder.TimestampFormatFinder.TimestampMatch;
import org.supercsv.exception.SuperCsvException; import org.supercsv.exception.SuperCsvException;
import org.supercsv.io.CsvListReader; import org.supercsv.io.CsvListReader;
import org.supercsv.prefs.CsvPreference; import org.supercsv.prefs.CsvPreference;
@ -27,7 +26,6 @@ import java.util.Locale;
import java.util.Map; import java.util.Map;
import java.util.Random; import java.util.Random;
import java.util.SortedMap; import java.util.SortedMap;
import java.util.regex.Pattern;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.IntStream; import java.util.stream.IntStream;
@ -62,7 +60,7 @@ public class DelimitedFileStructureFinder implements FileStructureFinder {
throw new IllegalArgumentException("[" + overriddenColumnNames.size() + "] column names were specified [" + throw new IllegalArgumentException("[" + overriddenColumnNames.size() + "] column names were specified [" +
String.join(",", overriddenColumnNames) + "] but there are [" + header.length + "] columns in the sample"); String.join(",", overriddenColumnNames) + "] but there are [" + header.length + "] columns in the sample");
} }
columnNames = overriddenColumnNames.toArray(new String[overriddenColumnNames.size()]); columnNames = overriddenColumnNames.toArray(new String[0]);
} else { } else {
// The column names are the header names but with blanks named column1, column2, etc. // The column names are the header names but with blanks named column1, column2, etc.
columnNames = new String[header.length]; columnNames = new String[header.length];
@ -85,11 +83,14 @@ public class DelimitedFileStructureFinder implements FileStructureFinder {
trimFields ? row.stream().map(field -> (field == null) ? null : field.trim()).collect(Collectors.toList()) : row); trimFields ? row.stream().map(field -> (field == null) ? null : field.trim()).collect(Collectors.toList()) : row);
sampleRecords.add(sampleRecord); sampleRecords.add(sampleRecord);
sampleMessages.add( sampleMessages.add(
sampleLines.subList(prevMessageEndLineNumber + 1, lineNumbers.get(index)).stream().collect(Collectors.joining("\n"))); String.join("\n", sampleLines.subList(prevMessageEndLineNumber + 1, lineNumbers.get(index))));
prevMessageEndLineNumber = lineNumber; prevMessageEndLineNumber = lineNumber;
} }
String preamble = Pattern.compile("\n").splitAsStream(sample).limit(lineNumbers.get(1)).collect(Collectors.joining("\n", "", "\n")); String preamble = String.join("\n", sampleLines.subList(0, lineNumbers.get(1))) + "\n";
// null to allow GC before timestamp search
sampleLines = null;
char delimiter = (char) csvPreference.getDelimiterChar(); char delimiter = (char) csvPreference.getDelimiterChar();
FileStructure.Builder structureBuilder = new FileStructure.Builder(FileStructure.Format.DELIMITED) FileStructure.Builder structureBuilder = new FileStructure.Builder(FileStructure.Format.DELIMITED)
@ -107,7 +108,7 @@ public class DelimitedFileStructureFinder implements FileStructureFinder {
structureBuilder.setShouldTrimFields(true); structureBuilder.setShouldTrimFields(true);
} }
Tuple<String, TimestampMatch> timeField = FileStructureUtils.guessTimestampField(explanation, sampleRecords, overrides, Tuple<String, TimestampFormatFinder> timeField = FileStructureUtils.guessTimestampField(explanation, sampleRecords, overrides,
timeoutChecker); timeoutChecker);
if (timeField != null) { if (timeField != null) {
String timeLineRegex = null; String timeLineRegex = null;
@ -119,7 +120,7 @@ public class DelimitedFileStructureFinder implements FileStructureFinder {
for (String column : Arrays.asList(columnNames).subList(0, columnNames.length - 1)) { for (String column : Arrays.asList(columnNames).subList(0, columnNames.length - 1)) {
if (timeField.v1().equals(column)) { if (timeField.v1().equals(column)) {
builder.append("\"?"); builder.append("\"?");
String simpleTimePattern = timeField.v2().simplePattern.pattern(); String simpleTimePattern = timeField.v2().getSimplePattern().pattern();
builder.append(simpleTimePattern.startsWith("\\b") ? simpleTimePattern.substring(2) : simpleTimePattern); builder.append(simpleTimePattern.startsWith("\\b") ? simpleTimePattern.substring(2) : simpleTimePattern);
timeLineRegex = builder.toString(); timeLineRegex = builder.toString();
break; break;
@ -145,11 +146,11 @@ public class DelimitedFileStructureFinder implements FileStructureFinder {
boolean needClientTimeZone = timeField.v2().hasTimezoneDependentParsing(); boolean needClientTimeZone = timeField.v2().hasTimezoneDependentParsing();
structureBuilder.setTimestampField(timeField.v1()) structureBuilder.setTimestampField(timeField.v1())
.setJodaTimestampFormats(timeField.v2().jodaTimestampFormats) .setJodaTimestampFormats(timeField.v2().getJodaTimestampFormats())
.setJavaTimestampFormats(timeField.v2().javaTimestampFormats) .setJavaTimestampFormats(timeField.v2().getJavaTimestampFormats())
.setNeedClientTimezone(needClientTimeZone) .setNeedClientTimezone(needClientTimeZone)
.setIngestPipeline(FileStructureUtils.makeIngestPipelineDefinition(null, timeField.v1(), .setIngestPipeline(FileStructureUtils.makeIngestPipelineDefinition(null, Collections.emptyMap(), timeField.v1(),
timeField.v2().javaTimestampFormats, needClientTimeZone)) timeField.v2().getJavaTimestampFormats(), needClientTimeZone))
.setMultilineStartPattern(timeLineRegex); .setMultilineStartPattern(timeLineRegex);
} }

View File

@ -9,13 +9,11 @@ import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.grok.Grok; import org.elasticsearch.grok.Grok;
import org.elasticsearch.ingest.Pipeline; import org.elasticsearch.ingest.Pipeline;
import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats; import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats;
import org.elasticsearch.xpack.ml.filestructurefinder.TimestampFormatFinder.TimestampMatch;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.Iterator;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -60,27 +58,32 @@ public final class FileStructureUtils {
* values determined by structure analysis. An exception will be thrown if the file structure * values determined by structure analysis. An exception will be thrown if the file structure
* is incompatible with an overridden value. * is incompatible with an overridden value.
* @param timeoutChecker Will abort the operation if its timeout is exceeded. * @param timeoutChecker Will abort the operation if its timeout is exceeded.
* @return A tuple of (field name, timestamp format) if one can be found, or <code>null</code> if * @return A tuple of (field name, timestamp format finder) if one can be found, or <code>null</code> if
* there is no consistent timestamp. * there is no consistent timestamp.
*/ */
static Tuple<String, TimestampMatch> guessTimestampField(List<String> explanation, List<Map<String, ?>> sampleRecords, static Tuple<String, TimestampFormatFinder> guessTimestampField(List<String> explanation, List<Map<String, ?>> sampleRecords,
FileStructureOverrides overrides, TimeoutChecker timeoutChecker) { FileStructureOverrides overrides, TimeoutChecker timeoutChecker) {
if (sampleRecords.isEmpty()) { if (sampleRecords.isEmpty()) {
return null; return null;
} }
StringBuilder exceptionMsg = null;
// Accept the first match from the first sample that is compatible with all the other samples // Accept the first match from the first sample that is compatible with all the other samples
for (Tuple<String, TimestampMatch> candidate : findCandidates(explanation, sampleRecords, overrides, timeoutChecker)) { for (Tuple<String, TimestampFormatFinder> candidate : findCandidates(explanation, sampleRecords, overrides, timeoutChecker)) {
String fieldName = candidate.v1();
TimestampFormatFinder timestampFormatFinder = candidate.v2();
boolean allGood = true; boolean allGood = true;
for (Map<String, ?> sampleRecord : sampleRecords.subList(1, sampleRecords.size())) { for (Map<String, ?> sampleRecord : sampleRecords.subList(1, sampleRecords.size())) {
Object fieldValue = sampleRecord.get(candidate.v1()); Object fieldValue = sampleRecord.get(fieldName);
if (fieldValue == null) { if (fieldValue == null) {
if (overrides.getTimestampField() != null) { if (overrides.getTimestampField() != null) {
throw new IllegalArgumentException("Specified timestamp field [" + overrides.getTimestampField() + throw new IllegalArgumentException("Specified timestamp field [" + overrides.getTimestampField() +
"] is not present in record [" + sampleRecord + "]"); "] is not present in record [" + sampleRecord + "]");
} }
explanation.add("First sample match [" + candidate.v1() + "] ruled out because record [" + sampleRecord + explanation.add("First sample match [" + fieldName + "] ruled out because record [" + sampleRecord +
"] doesn't have field"); "] doesn't have field");
allGood = false; allGood = false;
break; break;
@ -88,15 +91,20 @@ public final class FileStructureUtils {
timeoutChecker.check("timestamp field determination"); timeoutChecker.check("timestamp field determination");
TimestampMatch match = TimestampFormatFinder.findFirstFullMatch(fieldValue.toString(), overrides.getTimestampFormat(), try {
timeoutChecker); timestampFormatFinder.addSample(fieldValue.toString());
if (match == null || match.candidateIndex != candidate.v2().candidateIndex) { } catch (IllegalArgumentException e) {
if (overrides.getTimestampFormat() != null) { if (overrides.getTimestampFormat() != null) {
throw new IllegalArgumentException("Specified timestamp format [" + overrides.getTimestampFormat() + if (exceptionMsg == null) {
"] does not match for record [" + sampleRecord + "]"); exceptionMsg = new StringBuilder("Specified timestamp format [" + overrides.getTimestampFormat() +
"] does not match");
} else {
exceptionMsg.append(", nor");
}
exceptionMsg.append(" for record [").append(sampleRecord).append("] in field [").append(fieldName).append("]");
} }
explanation.add("First sample match [" + candidate.v1() + "] ruled out because record [" + sampleRecord + explanation.add("First sample match " + timestampFormatFinder.getRawJavaTimestampFormats()
"] matches differently: [" + match + "]"); + " ruled out because record [" + sampleRecord + "] does not match");
allGood = false; allGood = false;
break; break;
} }
@ -104,16 +112,21 @@ public final class FileStructureUtils {
if (allGood) { if (allGood) {
explanation.add(((overrides.getTimestampField() == null) ? "Guessing timestamp" : "Timestamp") + explanation.add(((overrides.getTimestampField() == null) ? "Guessing timestamp" : "Timestamp") +
" field is [" + candidate.v1() + "] with format [" + candidate.v2() + "]"); " field is [" + fieldName + "] with format " + timestampFormatFinder.getJavaTimestampFormats());
return candidate; return candidate;
} }
} }
if (exceptionMsg != null) {
throw new IllegalArgumentException(exceptionMsg.toString());
}
return null; return null;
} }
private static List<Tuple<String, TimestampMatch>> findCandidates(List<String> explanation, List<Map<String, ?>> sampleRecords, private static List<Tuple<String, TimestampFormatFinder>> findCandidates(List<String> explanation, List<Map<String, ?>> sampleRecords,
FileStructureOverrides overrides, TimeoutChecker timeoutChecker) { FileStructureOverrides overrides,
TimeoutChecker timeoutChecker) {
assert sampleRecords.isEmpty() == false; assert sampleRecords.isEmpty() == false;
Map<String, ?> firstRecord = sampleRecords.get(0); Map<String, ?> firstRecord = sampleRecords.get(0);
@ -124,7 +137,7 @@ public final class FileStructureUtils {
"] is not present in record [" + firstRecord + "]"); "] is not present in record [" + firstRecord + "]");
} }
List<Tuple<String, TimestampMatch>> candidates = new ArrayList<>(); List<Tuple<String, TimestampFormatFinder>> candidates = new ArrayList<>();
// Get candidate timestamps from the possible field(s) of the first sample record // Get candidate timestamps from the possible field(s) of the first sample record
for (Map.Entry<String, ?> field : firstRecord.entrySet()) { for (Map.Entry<String, ?> field : firstRecord.entrySet()) {
@ -132,12 +145,17 @@ public final class FileStructureUtils {
if (onlyConsiderField == null || onlyConsiderField.equals(fieldName)) { if (onlyConsiderField == null || onlyConsiderField.equals(fieldName)) {
Object value = field.getValue(); Object value = field.getValue();
if (value != null) { if (value != null) {
TimestampMatch match = TimestampFormatFinder.findFirstFullMatch(value.toString(), overrides.getTimestampFormat(), // Construct the TimestampFormatFinder outside the no-op catch because an exception
timeoutChecker); // from the constructor indicates a problem with the overridden format
if (match != null) { TimestampFormatFinder timestampFormatFinder =
Tuple<String, TimestampMatch> candidate = new Tuple<>(fieldName, match); new TimestampFormatFinder(explanation, overrides.getTimestampFormat(), true, true, true, timeoutChecker);
candidates.add(candidate); try {
explanation.add("First sample timestamp match [" + candidate + "]"); timestampFormatFinder.addSample(value.toString());
candidates.add(new Tuple<>(fieldName, timestampFormatFinder));
explanation.add("First sample timestamp match " + timestampFormatFinder.getRawJavaTimestampFormats()
+ " for field [" + fieldName + "]");
} catch (IllegalArgumentException e) {
// No possible timestamp format found in this particular field - not a problem
} }
} }
} }
@ -231,6 +249,27 @@ public final class FileStructureUtils {
} }
} }
/**
* Finds the appropriate date mapping for a collection of field values. Throws
* {@link IllegalArgumentException} if no consistent date mapping can be found.
* @param explanation List of reasons for choosing the overall file structure. This list
* may be non-empty when the method is called, and this method may
* append to it.
* @param fieldValues Values of the field for which mappings are to be guessed. The guessed
* mapping will be compatible with all the provided values. Must not be
* empty.
* @param timeoutChecker Will abort the operation if its timeout is exceeded.
* @return The sub-section of the index mappings most appropriate for the field.
*/
static Map<String, String> findTimestampMapping(List<String> explanation, Collection<String> fieldValues,
TimeoutChecker timeoutChecker) {
assert fieldValues.isEmpty() == false;
TimestampFormatFinder timestampFormatFinder = new TimestampFormatFinder(explanation, true, true, true, timeoutChecker);
fieldValues.forEach(timestampFormatFinder::addSample);
return timestampFormatFinder.getEsDateMappingTypeWithFormat();
}
/** /**
* Given some sample values for a field, guess the most appropriate index mapping for the * Given some sample values for a field, guess the most appropriate index mapping for the
* field. * field.
@ -247,26 +286,17 @@ public final class FileStructureUtils {
*/ */
static Map<String, String> guessScalarMapping(List<String> explanation, String fieldName, Collection<String> fieldValues, static Map<String, String> guessScalarMapping(List<String> explanation, String fieldName, Collection<String> fieldValues,
TimeoutChecker timeoutChecker) { TimeoutChecker timeoutChecker) {
assert fieldValues.isEmpty() == false; assert fieldValues.isEmpty() == false;
if (fieldValues.stream().allMatch(value -> "true".equals(value) || "false".equals(value))) { if (fieldValues.stream().allMatch(value -> "true".equals(value) || "false".equals(value))) {
return Collections.singletonMap(MAPPING_TYPE_SETTING, "boolean"); return Collections.singletonMap(MAPPING_TYPE_SETTING, "boolean");
} }
// This checks if a date mapping would be appropriate, and, if so, finds the correct format try {
Iterator<String> iter = fieldValues.iterator(); return findTimestampMapping(explanation, fieldValues, timeoutChecker);
TimestampMatch timestampMatch = TimestampFormatFinder.findFirstFullMatch(iter.next(), timeoutChecker); } catch (IllegalArgumentException e) {
while (timestampMatch != null && iter.hasNext()) { // To be mapped as type "date" all the values must match the same timestamp format - if
// To be mapped as type date all the values must match the same timestamp format - it is // they don't we'll end up here, and move on to try other possible mappings
// not acceptable for all values to be dates, but with different formats
if (timestampMatch.equals(TimestampFormatFinder.findFirstFullMatch(iter.next(), timestampMatch.candidateIndex,
timeoutChecker)) == false) {
timestampMatch = null;
}
}
if (timestampMatch != null) {
return timestampMatch.getEsDateMappingTypeWithFormat();
} }
if (fieldValues.stream().allMatch(NUMBER_GROK::match)) { if (fieldValues.stream().allMatch(NUMBER_GROK::match)) {
@ -321,6 +351,7 @@ public final class FileStructureUtils {
* Create an ingest pipeline definition appropriate for the file structure. * Create an ingest pipeline definition appropriate for the file structure.
* @param grokPattern The Grok pattern used for parsing semi-structured text formats. <code>null</code> for * @param grokPattern The Grok pattern used for parsing semi-structured text formats. <code>null</code> for
* fully structured formats. * fully structured formats.
* @param customGrokPatternDefinitions The definitions for any custom patterns that {@code grokPattern} uses.
* @param timestampField The input field containing the timestamp to be parsed into <code>@timestamp</code>. * @param timestampField The input field containing the timestamp to be parsed into <code>@timestamp</code>.
* <code>null</code> if there is no timestamp. * <code>null</code> if there is no timestamp.
* @param timestampFormats Timestamp formats to be used for parsing {@code timestampField}. * @param timestampFormats Timestamp formats to be used for parsing {@code timestampField}.
@ -328,7 +359,8 @@ public final class FileStructureUtils {
* @param needClientTimezone Is the timezone of the client supplying data to ingest required to uniquely parse the timestamp? * @param needClientTimezone Is the timezone of the client supplying data to ingest required to uniquely parse the timestamp?
* @return The ingest pipeline definition, or <code>null</code> if none is required. * @return The ingest pipeline definition, or <code>null</code> if none is required.
*/ */
public static Map<String, Object> makeIngestPipelineDefinition(String grokPattern, String timestampField, List<String> timestampFormats, public static Map<String, Object> makeIngestPipelineDefinition(String grokPattern, Map<String, String> customGrokPatternDefinitions,
String timestampField, List<String> timestampFormats,
boolean needClientTimezone) { boolean needClientTimezone) {
if (grokPattern == null && timestampField == null) { if (grokPattern == null && timestampField == null) {
@ -344,7 +376,12 @@ public final class FileStructureUtils {
Map<String, Object> grokProcessorSettings = new LinkedHashMap<>(); Map<String, Object> grokProcessorSettings = new LinkedHashMap<>();
grokProcessorSettings.put("field", "message"); grokProcessorSettings.put("field", "message");
grokProcessorSettings.put("patterns", Collections.singletonList(grokPattern)); grokProcessorSettings.put("patterns", Collections.singletonList(grokPattern));
if (customGrokPatternDefinitions.isEmpty() == false) {
grokProcessorSettings.put("pattern_definitions", customGrokPatternDefinitions);
}
processors.add(Collections.singletonMap("grok", grokProcessorSettings)); processors.add(Collections.singletonMap("grok", grokProcessorSettings));
} else {
assert customGrokPatternDefinitions.isEmpty();
} }
if (timestampField != null) { if (timestampField != null) {

View File

@ -8,7 +8,6 @@ package org.elasticsearch.xpack.ml.filestructurefinder;
import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.grok.Grok; import org.elasticsearch.grok.Grok;
import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats; import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats;
import org.elasticsearch.xpack.ml.filestructurefinder.TimestampFormatFinder.TimestampMatch;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
@ -18,6 +17,7 @@ import java.util.HashMap;
import java.util.LinkedHashSet; import java.util.LinkedHashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.regex.Matcher; import java.util.regex.Matcher;
import java.util.regex.Pattern; import java.util.regex.Pattern;
@ -76,10 +76,12 @@ public final class GrokPatternCreator {
new ValueOnlyGrokPatternCandidate("DATESTAMP_RFC2822", "date", "extra_timestamp"), new ValueOnlyGrokPatternCandidate("DATESTAMP_RFC2822", "date", "extra_timestamp"),
new ValueOnlyGrokPatternCandidate("DATESTAMP_OTHER", "date", "extra_timestamp"), new ValueOnlyGrokPatternCandidate("DATESTAMP_OTHER", "date", "extra_timestamp"),
new ValueOnlyGrokPatternCandidate("DATESTAMP_EVENTLOG", "date", "extra_timestamp"), new ValueOnlyGrokPatternCandidate("DATESTAMP_EVENTLOG", "date", "extra_timestamp"),
new ValueOnlyGrokPatternCandidate("HTTPDERROR_DATE", "date", "extra_timestamp"),
new ValueOnlyGrokPatternCandidate("SYSLOGTIMESTAMP", "date", "extra_timestamp"), new ValueOnlyGrokPatternCandidate("SYSLOGTIMESTAMP", "date", "extra_timestamp"),
new ValueOnlyGrokPatternCandidate("HTTPDATE", "date", "extra_timestamp"), new ValueOnlyGrokPatternCandidate("HTTPDATE", "date", "extra_timestamp"),
new ValueOnlyGrokPatternCandidate("CATALINA_DATESTAMP", "date", "extra_timestamp"), new ValueOnlyGrokPatternCandidate("CATALINA_DATESTAMP", "date", "extra_timestamp"),
new ValueOnlyGrokPatternCandidate("CISCOTIMESTAMP", "date", "extra_timestamp"), new ValueOnlyGrokPatternCandidate("CISCOTIMESTAMP", "date", "extra_timestamp"),
new ValueOnlyGrokPatternCandidate("DATESTAMP", "date", "extra_timestamp"),
new ValueOnlyGrokPatternCandidate("LOGLEVEL", "keyword", "loglevel"), new ValueOnlyGrokPatternCandidate("LOGLEVEL", "keyword", "loglevel"),
new ValueOnlyGrokPatternCandidate("URI", "keyword", "uri"), new ValueOnlyGrokPatternCandidate("URI", "keyword", "uri"),
new ValueOnlyGrokPatternCandidate("UUID", "keyword", "uuid"), new ValueOnlyGrokPatternCandidate("UUID", "keyword", "uuid"),
@ -90,7 +92,8 @@ public final class GrokPatternCreator {
// TODO: would be nice to have IPORHOST here, but HOSTNAME matches almost all words // TODO: would be nice to have IPORHOST here, but HOSTNAME matches almost all words
new ValueOnlyGrokPatternCandidate("IP", "ip", "ipaddress"), new ValueOnlyGrokPatternCandidate("IP", "ip", "ipaddress"),
new ValueOnlyGrokPatternCandidate("DATE", "date", "date"), new ValueOnlyGrokPatternCandidate("DATE", "date", "date"),
new ValueOnlyGrokPatternCandidate("TIME", "date", "time"), // A time with no date cannot be stored in a field of type "date", hence "keyword"
new ValueOnlyGrokPatternCandidate("TIME", "keyword", "time"),
// This already includes pre/post break conditions // This already includes pre/post break conditions
new ValueOnlyGrokPatternCandidate("QUOTEDSTRING", "keyword", "field", "", ""), new ValueOnlyGrokPatternCandidate("QUOTEDSTRING", "keyword", "field", "", ""),
// Disallow +, - and . before numbers, as well as "word" characters, otherwise we'll pick // Disallow +, - and . before numbers, as well as "word" characters, otherwise we'll pick
@ -121,6 +124,7 @@ public final class GrokPatternCreator {
*/ */
private final Map<String, Object> mappings; private final Map<String, Object> mappings;
private final Map<String, FieldStats> fieldStats; private final Map<String, FieldStats> fieldStats;
private final Map<String, String> grokPatternDefinitions;
private final Map<String, Integer> fieldNameCountStore = new HashMap<>(); private final Map<String, Integer> fieldNameCountStore = new HashMap<>();
private final StringBuilder overallGrokPatternBuilder = new StringBuilder(); private final StringBuilder overallGrokPatternBuilder = new StringBuilder();
private final TimeoutChecker timeoutChecker; private final TimeoutChecker timeoutChecker;
@ -131,16 +135,24 @@ public final class GrokPatternCreator {
* can be appended by the methods of this class. * can be appended by the methods of this class.
* @param sampleMessages Sample messages that any Grok pattern found must match. * @param sampleMessages Sample messages that any Grok pattern found must match.
* @param mappings Will be updated with mappings appropriate for the returned pattern, if non-<code>null</code>. * @param mappings Will be updated with mappings appropriate for the returned pattern, if non-<code>null</code>.
* @param timeoutChecker Will abort the operation if its timeout is exceeded.
* @param fieldStats Will be updated with field stats for the fields in the returned pattern, if non-<code>null</code>. * @param fieldStats Will be updated with field stats for the fields in the returned pattern, if non-<code>null</code>.
* @param customGrokPatternDefinitions Custom Grok pattern definitions to add to the built-in ones.
* @param timeoutChecker Will abort the operation if its timeout is exceeded.
*/ */
public GrokPatternCreator(List<String> explanation, Collection<String> sampleMessages, Map<String, Object> mappings, public GrokPatternCreator(List<String> explanation, Collection<String> sampleMessages, Map<String, Object> mappings,
Map<String, FieldStats> fieldStats, TimeoutChecker timeoutChecker) { Map<String, FieldStats> fieldStats, Map<String, String> customGrokPatternDefinitions,
this.explanation = explanation; TimeoutChecker timeoutChecker) {
this.explanation = Objects.requireNonNull(explanation);
this.sampleMessages = Collections.unmodifiableCollection(sampleMessages); this.sampleMessages = Collections.unmodifiableCollection(sampleMessages);
this.mappings = mappings; this.mappings = mappings;
this.fieldStats = fieldStats; this.fieldStats = fieldStats;
this.timeoutChecker = timeoutChecker; if (customGrokPatternDefinitions.isEmpty()) {
grokPatternDefinitions = Grok.getBuiltinPatterns();
} else {
grokPatternDefinitions = new HashMap<>(Grok.getBuiltinPatterns());
grokPatternDefinitions.putAll(customGrokPatternDefinitions);
}
this.timeoutChecker = Objects.requireNonNull(timeoutChecker);
} }
/** /**
@ -171,7 +183,8 @@ public final class GrokPatternCreator {
*/ */
public void validateFullLineGrokPattern(String grokPattern, String timestampField) { public void validateFullLineGrokPattern(String grokPattern, String timestampField) {
FullMatchGrokPatternCandidate candidate = FullMatchGrokPatternCandidate.fromGrokPattern(grokPattern, timestampField); FullMatchGrokPatternCandidate candidate = FullMatchGrokPatternCandidate.fromGrokPattern(grokPattern, timestampField,
grokPatternDefinitions);
if (candidate.matchesAll(sampleMessages, timeoutChecker)) { if (candidate.matchesAll(sampleMessages, timeoutChecker)) {
candidate.processMatch(explanation, sampleMessages, mappings, fieldStats, timeoutChecker); candidate.processMatch(explanation, sampleMessages, mappings, fieldStats, timeoutChecker);
} else { } else {
@ -189,7 +202,7 @@ public final class GrokPatternCreator {
overallGrokPatternBuilder.setLength(0); overallGrokPatternBuilder.setLength(0);
GrokPatternCandidate seedCandidate = new NoMappingGrokPatternCandidate(seedPatternName, seedFieldName); GrokPatternCandidate seedCandidate = new NoMappingGrokPatternCandidate(seedPatternName, seedFieldName, grokPatternDefinitions);
processCandidateAndSplit(seedCandidate, true, sampleMessages, false, 0, false, 0); processCandidateAndSplit(seedCandidate, true, sampleMessages, false, 0, false, 0);
@ -215,8 +228,8 @@ public final class GrokPatternCreator {
Collection<String> prefaces = new ArrayList<>(); Collection<String> prefaces = new ArrayList<>();
Collection<String> epilogues = new ArrayList<>(); Collection<String> epilogues = new ArrayList<>();
String patternBuilderContent = String patternBuilderContent = chosenPattern.processCaptures(explanation, fieldNameCountStore, snippets, prefaces, epilogues,
chosenPattern.processCaptures(fieldNameCountStore, snippets, prefaces, epilogues, mappings, fieldStats, timeoutChecker); mappings, fieldStats, timeoutChecker);
appendBestGrokMatchForStrings(false, prefaces, ignoreKeyValueCandidateLeft, ignoreValueOnlyCandidatesLeft); appendBestGrokMatchForStrings(false, prefaces, ignoreKeyValueCandidateLeft, ignoreValueOnlyCandidatesLeft);
overallGrokPatternBuilder.append(patternBuilderContent); overallGrokPatternBuilder.append(patternBuilderContent);
appendBestGrokMatchForStrings(isLast, epilogues, ignoreKeyValueCandidateRight, ignoreValueOnlyCandidatesRight); appendBestGrokMatchForStrings(isLast, epilogues, ignoreKeyValueCandidateRight, ignoreValueOnlyCandidatesRight);
@ -234,7 +247,7 @@ public final class GrokPatternCreator {
GrokPatternCandidate bestCandidate = null; GrokPatternCandidate bestCandidate = null;
if (snippets.isEmpty() == false) { if (snippets.isEmpty() == false) {
GrokPatternCandidate kvCandidate = new KeyValueGrokPatternCandidate(explanation); GrokPatternCandidate kvCandidate = new KeyValueGrokPatternCandidate();
if (ignoreKeyValueCandidate == false && kvCandidate.matchesAll(snippets)) { if (ignoreKeyValueCandidate == false && kvCandidate.matchesAll(snippets)) {
bestCandidate = kvCandidate; bestCandidate = kvCandidate;
} else { } else {
@ -409,9 +422,9 @@ public final class GrokPatternCreator {
* calculate field stats. * calculate field stats.
* @return The string that needs to be incorporated into the overall Grok pattern for the line. * @return The string that needs to be incorporated into the overall Grok pattern for the line.
*/ */
String processCaptures(Map<String, Integer> fieldNameCountStore, Collection<String> snippets, Collection<String> prefaces, String processCaptures(List<String> explanation, Map<String, Integer> fieldNameCountStore, Collection<String> snippets,
Collection<String> epilogues, Map<String, Object> mappings, Map<String, FieldStats> fieldStats, Collection<String> prefaces, Collection<String> epilogues, Map<String, Object> mappings,
TimeoutChecker timeoutChecker); Map<String, FieldStats> fieldStats, TimeoutChecker timeoutChecker);
} }
/** /**
@ -434,10 +447,22 @@ public final class GrokPatternCreator {
* for the pre and/or post breaks. * for the pre and/or post breaks.
* *
* @param grokPatternName Name of the Grok pattern to try to match - must match one defined in Logstash. * @param grokPatternName Name of the Grok pattern to try to match - must match one defined in Logstash.
* @param mappingType Data type for field in Elasticsearch mappings.
* @param fieldName Name of the field to extract from the match. * @param fieldName Name of the field to extract from the match.
*/ */
ValueOnlyGrokPatternCandidate(String grokPatternName, String mappingType, String fieldName) { ValueOnlyGrokPatternCandidate(String grokPatternName, String mappingType, String fieldName) {
this(grokPatternName, mappingType, fieldName, "\\b", "\\b"); this(grokPatternName, mappingType, fieldName, "\\b", "\\b", Grok.getBuiltinPatterns());
}
/**
* @param grokPatternName Name of the Grok pattern to try to match - must match one defined in Logstash.
* @param mappingType Data type for field in Elasticsearch mappings.
* @param fieldName Name of the field to extract from the match.
* @param grokPatternDefinitions Definitions of Grok patterns to be used.
*/
ValueOnlyGrokPatternCandidate(String grokPatternName, String mappingType, String fieldName,
Map<String, String> grokPatternDefinitions) {
this(grokPatternName, mappingType, fieldName, "\\b", "\\b", grokPatternDefinitions);
} }
/** /**
@ -448,11 +473,24 @@ public final class GrokPatternCreator {
* @param postBreak Only consider the match if it's broken from the following text by this. * @param postBreak Only consider the match if it's broken from the following text by this.
*/ */
ValueOnlyGrokPatternCandidate(String grokPatternName, String mappingType, String fieldName, String preBreak, String postBreak) { ValueOnlyGrokPatternCandidate(String grokPatternName, String mappingType, String fieldName, String preBreak, String postBreak) {
this(grokPatternName, mappingType, fieldName, preBreak, postBreak, Grok.getBuiltinPatterns());
}
/**
* @param grokPatternName Name of the Grok pattern to try to match - must match one defined in Logstash.
* @param mappingType Data type for field in Elasticsearch mappings.
* @param fieldName Name of the field to extract from the match.
* @param preBreak Only consider the match if it's broken from the previous text by this.
* @param postBreak Only consider the match if it's broken from the following text by this.
* @param grokPatternDefinitions Definitions of Grok patterns to be used.
*/
ValueOnlyGrokPatternCandidate(String grokPatternName, String mappingType, String fieldName, String preBreak, String postBreak,
Map<String, String> grokPatternDefinitions) {
this.grokPatternName = grokPatternName; this.grokPatternName = grokPatternName;
this.mappingType = mappingType; this.mappingType = mappingType;
this.fieldName = fieldName; this.fieldName = fieldName;
// The (?m) here has the Ruby meaning, which is equivalent to (?s) in Java // The (?m) here has the Ruby meaning, which is equivalent to (?s) in Java
grok = new Grok(Grok.getBuiltinPatterns(), "(?m)%{DATA:" + PREFACE + "}" + preBreak + grok = new Grok(grokPatternDefinitions, "(?m)%{DATA:" + PREFACE + "}" + preBreak +
"%{" + grokPatternName + ":" + VALUE + "}" + postBreak + "%{GREEDYDATA:" + EPILOGUE + "}", TimeoutChecker.watchdog); "%{" + grokPatternName + ":" + VALUE + "}" + postBreak + "%{GREEDYDATA:" + EPILOGUE + "}", TimeoutChecker.watchdog);
} }
@ -467,9 +505,9 @@ public final class GrokPatternCreator {
* bit that matches. * bit that matches.
*/ */
@Override @Override
public String processCaptures(Map<String, Integer> fieldNameCountStore, Collection<String> snippets, Collection<String> prefaces, public String processCaptures(List<String> explanation, Map<String, Integer> fieldNameCountStore, Collection<String> snippets,
Collection<String> epilogues, Map<String, Object> mappings, Map<String, FieldStats> fieldStats, Collection<String> prefaces, Collection<String> epilogues, Map<String, Object> mappings,
TimeoutChecker timeoutChecker) { Map<String, FieldStats> fieldStats, TimeoutChecker timeoutChecker) {
Collection<String> values = new ArrayList<>(); Collection<String> values = new ArrayList<>();
for (String snippet : snippets) { for (String snippet : snippets) {
Map<String, Object> captures = timeoutChecker.grokCaptures(grok, snippet, "full message Grok pattern field extraction"); Map<String, Object> captures = timeoutChecker.grokCaptures(grok, snippet, "full message Grok pattern field extraction");
@ -485,10 +523,13 @@ public final class GrokPatternCreator {
if (mappings != null) { if (mappings != null) {
Map<String, String> fullMappingType = Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, mappingType); Map<String, String> fullMappingType = Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, mappingType);
if ("date".equals(mappingType)) { if ("date".equals(mappingType)) {
assert values.isEmpty() == false; try {
TimestampMatch timestampMatch = TimestampFormatFinder.findFirstFullMatch(values.iterator().next(), timeoutChecker); fullMappingType = FileStructureUtils.findTimestampMapping(explanation, values, timeoutChecker);
if (timestampMatch != null) { } catch (IllegalArgumentException e) {
fullMappingType = timestampMatch.getEsDateMappingTypeWithFormat(); // This feels like it shouldn't happen, but there may be some obscure edge case
// where it does, and in production it will cause less frustration to just return
// a mapping type of "date" with no format than to fail the whole analysis
assert e == null : e.getMessage();
} }
timeoutChecker.check("mapping determination"); timeoutChecker.check("mapping determination");
} }
@ -509,13 +550,9 @@ public final class GrokPatternCreator {
*/ */
static class KeyValueGrokPatternCandidate implements GrokPatternCandidate { static class KeyValueGrokPatternCandidate implements GrokPatternCandidate {
private static final Pattern kvFinder = Pattern.compile("\\b(\\w+)=[\\w.-]+"); private static final Pattern KV_FINDER = Pattern.compile("\\b(\\w+)=[\\w.-]+");
private final List<String> explanation;
private String fieldName;
KeyValueGrokPatternCandidate(List<String> explanation) { private String fieldName;
this.explanation = explanation;
}
@Override @Override
public boolean matchesAll(Collection<String> snippets) { public boolean matchesAll(Collection<String> snippets) {
@ -523,7 +560,7 @@ public final class GrokPatternCreator {
boolean isFirst = true; boolean isFirst = true;
for (String snippet : snippets) { for (String snippet : snippets) {
if (isFirst) { if (isFirst) {
Matcher matcher = kvFinder.matcher(snippet); Matcher matcher = KV_FINDER.matcher(snippet);
while (matcher.find()) { while (matcher.find()) {
candidateNames.add(matcher.group(1)); candidateNames.add(matcher.group(1));
} }
@ -540,9 +577,9 @@ public final class GrokPatternCreator {
} }
@Override @Override
public String processCaptures(Map<String, Integer> fieldNameCountStore, Collection<String> snippets, Collection<String> prefaces, public String processCaptures(List<String> explanation, Map<String, Integer> fieldNameCountStore, Collection<String> snippets,
Collection<String> epilogues, Map<String, Object> mappings, Map<String, FieldStats> fieldStats, Collection<String> prefaces, Collection<String> epilogues, Map<String, Object> mappings,
TimeoutChecker timeoutChecker) { Map<String, FieldStats> fieldStats, TimeoutChecker timeoutChecker) {
if (fieldName == null) { if (fieldName == null) {
throw new IllegalStateException("Cannot process KV matches until a field name has been determined"); throw new IllegalStateException("Cannot process KV matches until a field name has been determined");
} }
@ -578,15 +615,15 @@ public final class GrokPatternCreator {
*/ */
static class NoMappingGrokPatternCandidate extends ValueOnlyGrokPatternCandidate { static class NoMappingGrokPatternCandidate extends ValueOnlyGrokPatternCandidate {
NoMappingGrokPatternCandidate(String grokPatternName, String fieldName) { NoMappingGrokPatternCandidate(String grokPatternName, String fieldName, Map<String, String> grokPatternDefinitions) {
super(grokPatternName, null, fieldName); super(grokPatternName, null, fieldName, grokPatternDefinitions);
} }
@Override @Override
public String processCaptures(Map<String, Integer> fieldNameCountStore, Collection<String> snippets, Collection<String> prefaces, public String processCaptures(List<String> explanation, Map<String, Integer> fieldNameCountStore, Collection<String> snippets,
Collection<String> epilogues, Map<String, Object> mappings, Map<String, FieldStats> fieldStats, Collection<String> prefaces, Collection<String> epilogues, Map<String, Object> mappings,
TimeoutChecker timeoutChecker) { Map<String, FieldStats> fieldStats, TimeoutChecker timeoutChecker) {
return super.processCaptures(fieldNameCountStore, snippets, prefaces, epilogues, null, fieldStats, timeoutChecker); return super.processCaptures(explanation, fieldNameCountStore, snippets, prefaces, epilogues, null, fieldStats, timeoutChecker);
} }
} }
@ -600,17 +637,27 @@ public final class GrokPatternCreator {
private final Grok grok; private final Grok grok;
static FullMatchGrokPatternCandidate fromGrokPatternName(String grokPatternName, String timeField) { static FullMatchGrokPatternCandidate fromGrokPatternName(String grokPatternName, String timeField) {
return new FullMatchGrokPatternCandidate("%{" + grokPatternName + "}", timeField); return new FullMatchGrokPatternCandidate("%{" + grokPatternName + "}", timeField, Grok.getBuiltinPatterns());
}
static FullMatchGrokPatternCandidate fromGrokPatternName(String grokPatternName, String timeField,
Map<String, String> grokPatternDefinitions) {
return new FullMatchGrokPatternCandidate("%{" + grokPatternName + "}", timeField, grokPatternDefinitions);
} }
static FullMatchGrokPatternCandidate fromGrokPattern(String grokPattern, String timeField) { static FullMatchGrokPatternCandidate fromGrokPattern(String grokPattern, String timeField) {
return new FullMatchGrokPatternCandidate(grokPattern, timeField); return new FullMatchGrokPatternCandidate(grokPattern, timeField, Grok.getBuiltinPatterns());
} }
private FullMatchGrokPatternCandidate(String grokPattern, String timeField) { static FullMatchGrokPatternCandidate fromGrokPattern(String grokPattern, String timeField,
Map<String, String> grokPatternDefinitions) {
return new FullMatchGrokPatternCandidate(grokPattern, timeField, grokPatternDefinitions);
}
private FullMatchGrokPatternCandidate(String grokPattern, String timeField, Map<String, String> grokPatternDefinitions) {
this.grokPattern = grokPattern; this.grokPattern = grokPattern;
this.timeField = timeField; this.timeField = timeField;
grok = new Grok(Grok.getBuiltinPatterns(), grokPattern, TimeoutChecker.watchdog); grok = new Grok(grokPatternDefinitions, grokPattern, TimeoutChecker.watchdog);
} }
public String getTimeField() { public String getTimeField() {

View File

@ -11,7 +11,6 @@ import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats; import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats;
import org.elasticsearch.xpack.core.ml.filestructurefinder.FileStructure; import org.elasticsearch.xpack.core.ml.filestructurefinder.FileStructure;
import org.elasticsearch.xpack.ml.filestructurefinder.TimestampFormatFinder.TimestampMatch;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -53,17 +52,17 @@ public class NdJsonFileStructureFinder implements FileStructureFinder {
.setNumLinesAnalyzed(sampleMessages.size()) .setNumLinesAnalyzed(sampleMessages.size())
.setNumMessagesAnalyzed(sampleRecords.size()); .setNumMessagesAnalyzed(sampleRecords.size());
Tuple<String, TimestampMatch> timeField = Tuple<String, TimestampFormatFinder> timeField =
FileStructureUtils.guessTimestampField(explanation, sampleRecords, overrides, timeoutChecker); FileStructureUtils.guessTimestampField(explanation, sampleRecords, overrides, timeoutChecker);
if (timeField != null) { if (timeField != null) {
boolean needClientTimeZone = timeField.v2().hasTimezoneDependentParsing(); boolean needClientTimeZone = timeField.v2().hasTimezoneDependentParsing();
structureBuilder.setTimestampField(timeField.v1()) structureBuilder.setTimestampField(timeField.v1())
.setJodaTimestampFormats(timeField.v2().jodaTimestampFormats) .setJodaTimestampFormats(timeField.v2().getJodaTimestampFormats())
.setJavaTimestampFormats(timeField.v2().javaTimestampFormats) .setJavaTimestampFormats(timeField.v2().getJavaTimestampFormats())
.setNeedClientTimezone(needClientTimeZone) .setNeedClientTimezone(needClientTimeZone)
.setIngestPipeline(FileStructureUtils.makeIngestPipelineDefinition(null, timeField.v1(), .setIngestPipeline(FileStructureUtils.makeIngestPipelineDefinition(null, Collections.emptyMap(), timeField.v1(),
timeField.v2().javaTimestampFormats, needClientTimeZone)); timeField.v2().getJavaTimestampFormats(), needClientTimeZone));
} }
Tuple<SortedMap<String, Object>, SortedMap<String, FieldStats>> mappingsAndFieldStats = Tuple<SortedMap<String, Object>, SortedMap<String, FieldStats>> mappingsAndFieldStats =

View File

@ -8,16 +8,12 @@ package org.elasticsearch.xpack.ml.filestructurefinder;
import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats; import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats;
import org.elasticsearch.xpack.core.ml.filestructurefinder.FileStructure; import org.elasticsearch.xpack.core.ml.filestructurefinder.FileStructure;
import org.elasticsearch.xpack.ml.filestructurefinder.TimestampFormatFinder.TimestampMatch;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set;
import java.util.SortedMap; import java.util.SortedMap;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.regex.Pattern; import java.util.regex.Pattern;
@ -30,25 +26,33 @@ public class TextLogFileStructureFinder implements FileStructureFinder {
static TextLogFileStructureFinder makeTextLogFileStructureFinder(List<String> explanation, String sample, String charsetName, static TextLogFileStructureFinder makeTextLogFileStructureFinder(List<String> explanation, String sample, String charsetName,
Boolean hasByteOrderMarker, FileStructureOverrides overrides, Boolean hasByteOrderMarker, FileStructureOverrides overrides,
TimeoutChecker timeoutChecker) { TimeoutChecker timeoutChecker) {
String[] sampleLines = sample.split("\n"); String[] sampleLines = sample.split("\n");
Tuple<TimestampMatch, Set<String>> bestTimestamp = mostLikelyTimestamp(sampleLines, overrides, timeoutChecker); TimestampFormatFinder timestampFormatFinder = populateTimestampFormatFinder(explanation, sampleLines, overrides, timeoutChecker);
if (bestTimestamp == null) { switch (timestampFormatFinder.getNumMatchedFormats()) {
// Is it appropriate to treat a file that is neither structured nor has case 0:
// a regular pattern of timestamps as a log file? Probably not... // Is it appropriate to treat a file that is neither structured nor has
throw new IllegalArgumentException("Could not find " + // a regular pattern of timestamps as a log file? Probably not...
((overrides.getTimestampFormat() == null) ? "a timestamp" : "the specified timestamp format") + " in the sample provided"); throw new IllegalArgumentException("Could not find " + ((overrides.getTimestampFormat() == null)
? "a timestamp"
: "the specified timestamp format") + " in the sample provided");
case 1:
// Simple case
break;
default:
timestampFormatFinder.selectBestMatch();
break;
} }
explanation.add(((overrides.getTimestampFormat() == null) ? "Most likely timestamp" : "Timestamp") + " format is [" + explanation.add(((overrides.getTimestampFormat() == null) ? "Most likely timestamp" : "Timestamp") + " format is " +
bestTimestamp.v1() + "]"); timestampFormatFinder.getJavaTimestampFormats());
List<String> sampleMessages = new ArrayList<>(); List<String> sampleMessages = new ArrayList<>();
StringBuilder preamble = new StringBuilder(); StringBuilder preamble = new StringBuilder();
int linesConsumed = 0; int linesConsumed = 0;
StringBuilder message = null; StringBuilder message = null;
int linesInMessage = 0; int linesInMessage = 0;
String multiLineRegex = createMultiLineMessageStartRegex(bestTimestamp.v2(), bestTimestamp.v1().simplePattern.pattern()); String multiLineRegex = createMultiLineMessageStartRegex(timestampFormatFinder.getPrefaces(),
timestampFormatFinder.getSimplePattern().pattern());
Pattern multiLinePattern = Pattern.compile(multiLineRegex); Pattern multiLinePattern = Pattern.compile(multiLineRegex);
for (String sampleLine : sampleLines) { for (String sampleLine : sampleLines) {
if (multiLinePattern.matcher(sampleLine).find()) { if (multiLinePattern.matcher(sampleLine).find()) {
@ -82,6 +86,9 @@ public class TextLogFileStructureFinder implements FileStructureFinder {
+ "problem is probably that the primary timestamp format has been incorrectly detected, so try overriding it."); + "problem is probably that the primary timestamp format has been incorrectly detected, so try overriding it.");
} }
// null to allow GC before Grok pattern search
sampleLines = null;
FileStructure.Builder structureBuilder = new FileStructure.Builder(FileStructure.Format.SEMI_STRUCTURED_TEXT) FileStructure.Builder structureBuilder = new FileStructure.Builder(FileStructure.Format.SEMI_STRUCTURED_TEXT)
.setCharset(charsetName) .setCharset(charsetName)
.setHasByteOrderMarker(hasByteOrderMarker) .setHasByteOrderMarker(hasByteOrderMarker)
@ -97,7 +104,9 @@ public class TextLogFileStructureFinder implements FileStructureFinder {
SortedMap<String, FieldStats> fieldStats = new TreeMap<>(); SortedMap<String, FieldStats> fieldStats = new TreeMap<>();
fieldStats.put("message", FileStructureUtils.calculateFieldStats(sampleMessages, timeoutChecker)); fieldStats.put("message", FileStructureUtils.calculateFieldStats(sampleMessages, timeoutChecker));
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, fieldStats, timeoutChecker); Map<String, String> customGrokPatternDefinitions = timestampFormatFinder.getCustomGrokPatternDefinitions();
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, fieldStats,
customGrokPatternDefinitions, timeoutChecker);
// We can't parse directly into @timestamp using Grok, so parse to some other time field, which the date filter will then remove // We can't parse directly into @timestamp using Grok, so parse to some other time field, which the date filter will then remove
String interimTimestampField = overrides.getTimestampField(); String interimTimestampField = overrides.getTimestampField();
String grokPattern = overrides.getGrokPattern(); String grokPattern = overrides.getGrokPattern();
@ -116,20 +125,22 @@ public class TextLogFileStructureFinder implements FileStructureFinder {
if (interimTimestampField == null) { if (interimTimestampField == null) {
interimTimestampField = "timestamp"; interimTimestampField = "timestamp";
} }
grokPattern = grokPatternCreator.createGrokPatternFromExamples(bestTimestamp.v1().grokPatternName, interimTimestampField); grokPattern =
grokPatternCreator.createGrokPatternFromExamples(timestampFormatFinder.getGrokPatternName(), interimTimestampField);
} }
} }
boolean needClientTimeZone = bestTimestamp.v1().hasTimezoneDependentParsing(); boolean needClientTimeZone = timestampFormatFinder.hasTimezoneDependentParsing();
FileStructure structure = structureBuilder FileStructure structure = structureBuilder
.setTimestampField(interimTimestampField) .setTimestampField(interimTimestampField)
.setJodaTimestampFormats(bestTimestamp.v1().jodaTimestampFormats) .setJodaTimestampFormats(timestampFormatFinder.getJodaTimestampFormats())
.setJavaTimestampFormats(bestTimestamp.v1().javaTimestampFormats) .setJavaTimestampFormats(timestampFormatFinder.getJavaTimestampFormats())
.setNeedClientTimezone(needClientTimeZone) .setNeedClientTimezone(needClientTimeZone)
.setGrokPattern(grokPattern) .setGrokPattern(grokPattern)
.setIngestPipeline(FileStructureUtils.makeIngestPipelineDefinition(grokPattern, interimTimestampField, .setIngestPipeline(FileStructureUtils.makeIngestPipelineDefinition(grokPattern,
bestTimestamp.v1().javaTimestampFormats, needClientTimeZone)) customGrokPatternDefinitions, interimTimestampField,
timestampFormatFinder.getJavaTimestampFormats(), needClientTimeZone))
.setMappings(mappings) .setMappings(mappings)
.setFieldStats(fieldStats) .setFieldStats(fieldStats)
.setExplanation(explanation) .setExplanation(explanation)
@ -153,79 +164,23 @@ public class TextLogFileStructureFinder implements FileStructureFinder {
return structure; return structure;
} }
static Tuple<TimestampMatch, Set<String>> mostLikelyTimestamp(String[] sampleLines, FileStructureOverrides overrides, static TimestampFormatFinder populateTimestampFormatFinder(List<String> explanation, String[] sampleLines,
TimeoutChecker timeoutChecker) { FileStructureOverrides overrides, TimeoutChecker timeoutChecker) {
TimestampFormatFinder timestampFormatFinder =
new TimestampFormatFinder(explanation, overrides.getTimestampFormat(), false, false, false, timeoutChecker);
Map<TimestampMatch, Tuple<Double, Set<String>>> timestampMatches = new LinkedHashMap<>();
int remainingLines = sampleLines.length;
double differenceBetweenTwoHighestWeights = 0.0;
for (String sampleLine : sampleLines) { for (String sampleLine : sampleLines) {
TimestampMatch match = TimestampFormatFinder.findFirstMatch(sampleLine, overrides.getTimestampFormat(), timeoutChecker); timestampFormatFinder.addSample(sampleLine);
if (match != null) {
TimestampMatch pureMatch = new TimestampMatch(match.candidateIndex, "", match.jodaTimestampFormats,
match.javaTimestampFormats, match.simplePattern, match.grokPatternName, "");
timestampMatches.compute(pureMatch, (k, v) -> {
if (v == null) {
return new Tuple<>(weightForMatch(match.preface), new HashSet<>(Collections.singletonList(match.preface)));
} else {
v.v2().add(match.preface);
return new Tuple<>(v.v1() + weightForMatch(match.preface), v.v2());
}
});
differenceBetweenTwoHighestWeights = findDifferenceBetweenTwoHighestWeights(timestampMatches.values());
}
timeoutChecker.check("timestamp format determination");
// The highest possible weight is 1, so if the difference between the two highest weights
// is less than the number of lines remaining then the leader cannot possibly be overtaken
if (differenceBetweenTwoHighestWeights > --remainingLines) {
break;
}
} }
double highestWeight = 0.0; return timestampFormatFinder;
Tuple<TimestampMatch, Set<String>> highestWeightMatch = null;
for (Map.Entry<TimestampMatch, Tuple<Double, Set<String>>> entry : timestampMatches.entrySet()) {
double weight = entry.getValue().v1();
if (weight > highestWeight) {
highestWeight = weight;
highestWeightMatch = new Tuple<>(entry.getKey(), entry.getValue().v2());
}
}
return highestWeightMatch;
} }
/** static String createMultiLineMessageStartRegex(Collection<String> prefaces, String simpleDateRegex) {
* Used to weight a timestamp match according to how far along the line it is found.
* Timestamps at the very beginning of the line are given a weight of 1. The weight
* progressively decreases the more text there is preceding the timestamp match, but
* is always greater than 0.
* @return A weight in the range (0, 1].
*/
private static double weightForMatch(String preface) {
return Math.pow(1.0 + preface.length() / 15.0, -1.1);
}
private static double findDifferenceBetweenTwoHighestWeights(Collection<Tuple<Double, Set<String>>> timestampMatches) {
double highestWeight = 0.0;
double secondHighestWeight = 0.0;
for (Tuple<Double, Set<String>> timestampMatch : timestampMatches) {
double weight = timestampMatch.v1();
if (weight > highestWeight) {
secondHighestWeight = highestWeight;
highestWeight = weight;
} else if (weight > secondHighestWeight) {
secondHighestWeight = weight;
}
}
return highestWeight - secondHighestWeight;
}
static String createMultiLineMessageStartRegex(Collection<String> prefaces, String timestampRegex) {
StringBuilder builder = new StringBuilder("^"); StringBuilder builder = new StringBuilder("^");
GrokPatternCreator.addIntermediateRegex(builder, prefaces); GrokPatternCreator.addIntermediateRegex(builder, prefaces);
builder.append(timestampRegex); builder.append(simpleDateRegex);
if (builder.substring(0, 3).equals("^\\b")) { if (builder.substring(0, 3).equals("^\\b")) {
builder.delete(1, 3); builder.delete(1, 3);
} }

View File

@ -8,7 +8,6 @@ package org.elasticsearch.xpack.ml.filestructurefinder;
import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats; import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats;
import org.elasticsearch.xpack.core.ml.filestructurefinder.FileStructure; import org.elasticsearch.xpack.core.ml.filestructurefinder.FileStructure;
import org.elasticsearch.xpack.ml.filestructurefinder.TimestampFormatFinder.TimestampMatch;
import org.w3c.dom.Document; import org.w3c.dom.Document;
import org.w3c.dom.NamedNodeMap; import org.w3c.dom.NamedNodeMap;
import org.w3c.dom.Node; import org.w3c.dom.Node;
@ -79,6 +78,9 @@ public class XmlFileStructureFinder implements FileStructureFinder {
++linesConsumed; ++linesConsumed;
} }
// null to allow GC before timestamp search
sampleDocEnds = null;
// If we get here the XML parser should have confirmed this // If we get here the XML parser should have confirmed this
assert messagePrefix.charAt(0) == '<'; assert messagePrefix.charAt(0) == '<';
String topLevelTag = messagePrefix.substring(1); String topLevelTag = messagePrefix.substring(1);
@ -91,17 +93,17 @@ public class XmlFileStructureFinder implements FileStructureFinder {
.setNumMessagesAnalyzed(sampleRecords.size()) .setNumMessagesAnalyzed(sampleRecords.size())
.setMultilineStartPattern("^\\s*<" + topLevelTag); .setMultilineStartPattern("^\\s*<" + topLevelTag);
Tuple<String, TimestampMatch> timeField = Tuple<String, TimestampFormatFinder> timeField =
FileStructureUtils.guessTimestampField(explanation, sampleRecords, overrides, timeoutChecker); FileStructureUtils.guessTimestampField(explanation, sampleRecords, overrides, timeoutChecker);
if (timeField != null) { if (timeField != null) {
boolean needClientTimeZone = timeField.v2().hasTimezoneDependentParsing(); boolean needClientTimeZone = timeField.v2().hasTimezoneDependentParsing();
structureBuilder.setTimestampField(timeField.v1()) structureBuilder.setTimestampField(timeField.v1())
.setJodaTimestampFormats(timeField.v2().jodaTimestampFormats) .setJodaTimestampFormats(timeField.v2().getJodaTimestampFormats())
.setJavaTimestampFormats(timeField.v2().javaTimestampFormats) .setJavaTimestampFormats(timeField.v2().getJavaTimestampFormats())
.setNeedClientTimezone(needClientTimeZone) .setNeedClientTimezone(needClientTimeZone)
.setIngestPipeline(FileStructureUtils.makeIngestPipelineDefinition(null, topLevelTag + "." + timeField.v1(), .setIngestPipeline(FileStructureUtils.makeIngestPipelineDefinition(null, Collections.emptyMap(),
timeField.v2().javaTimestampFormats, needClientTimeZone)); topLevelTag + "." + timeField.v1(), timeField.v2().getJavaTimestampFormats(), needClientTimeZone));
} }
Tuple<SortedMap<String, Object>, SortedMap<String, FieldStats>> mappingsAndFieldStats = Tuple<SortedMap<String, Object>, SortedMap<String, FieldStats>> mappingsAndFieldStats =

View File

@ -42,7 +42,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker()); assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker());
} }
assertEquals("^\"?time\"?,\"?message\"?", structure.getExcludeLinesPattern()); assertEquals("^\"?time\"?,\"?message\"?", structure.getExcludeLinesPattern());
assertEquals("^\"?\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern()); assertEquals("^\"?\\d{4}-\\d{2}-\\d{2}[T ]\\d{2}:\\d{2}", structure.getMultilineStartPattern());
assertEquals(Character.valueOf(','), structure.getDelimiter()); assertEquals(Character.valueOf(','), structure.getDelimiter());
assertEquals(Character.valueOf('"'), structure.getQuote()); assertEquals(Character.valueOf('"'), structure.getQuote());
assertTrue(structure.getHasHeaderRow()); assertTrue(structure.getHasHeaderRow());
@ -77,7 +77,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker()); assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker());
} }
assertEquals("^\"?time\"?,\"?message\"?", structure.getExcludeLinesPattern()); assertEquals("^\"?time\"?,\"?message\"?", structure.getExcludeLinesPattern());
assertEquals("^\"?\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern()); assertEquals("^\"?\\d{4}-\\d{2}-\\d{2}[T ]\\d{2}:\\d{2}", structure.getMultilineStartPattern());
assertEquals(Character.valueOf(','), structure.getDelimiter()); assertEquals(Character.valueOf(','), structure.getDelimiter());
assertEquals(Character.valueOf('"'), structure.getQuote()); assertEquals(Character.valueOf('"'), structure.getQuote());
assertTrue(structure.getHasHeaderRow()); assertTrue(structure.getHasHeaderRow());
@ -147,7 +147,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker()); assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker());
} }
assertEquals("^\"?message\"?,\"?time\"?,\"?count\"?", structure.getExcludeLinesPattern()); assertEquals("^\"?message\"?,\"?time\"?,\"?count\"?", structure.getExcludeLinesPattern());
assertEquals("^.*?,\"?\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern()); assertEquals("^.*?,\"?\\d{4}-\\d{2}-\\d{2}[T ]\\d{2}:\\d{2}", structure.getMultilineStartPattern());
assertEquals(Character.valueOf(','), structure.getDelimiter()); assertEquals(Character.valueOf(','), structure.getDelimiter());
assertEquals(Character.valueOf('"'), structure.getQuote()); assertEquals(Character.valueOf('"'), structure.getQuote());
assertTrue(structure.getHasHeaderRow()); assertTrue(structure.getHasHeaderRow());
@ -185,7 +185,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
"\"?RatecodeID\"?,\"?store_and_fwd_flag\"?,\"?PULocationID\"?,\"?DOLocationID\"?,\"?payment_type\"?,\"?fare_amount\"?," + "\"?RatecodeID\"?,\"?store_and_fwd_flag\"?,\"?PULocationID\"?,\"?DOLocationID\"?,\"?payment_type\"?,\"?fare_amount\"?," +
"\"?extra\"?,\"?mta_tax\"?,\"?tip_amount\"?,\"?tolls_amount\"?,\"?improvement_surcharge\"?,\"?total_amount\"?,\"?\"?,\"?\"?", "\"?extra\"?,\"?mta_tax\"?,\"?tip_amount\"?,\"?tolls_amount\"?,\"?improvement_surcharge\"?,\"?total_amount\"?,\"?\"?,\"?\"?",
structure.getExcludeLinesPattern()); structure.getExcludeLinesPattern());
assertEquals("^.*?,\"?\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern()); assertEquals("^.*?,\"?\\d{4}-\\d{2}-\\d{2}[T ]\\d{2}:\\d{2}", structure.getMultilineStartPattern());
assertEquals(Character.valueOf(','), structure.getDelimiter()); assertEquals(Character.valueOf(','), structure.getDelimiter());
assertEquals(Character.valueOf('"'), structure.getQuote()); assertEquals(Character.valueOf('"'), structure.getQuote());
assertTrue(structure.getHasHeaderRow()); assertTrue(structure.getHasHeaderRow());
@ -230,7 +230,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
"\"?RatecodeID\"?,\"?store_and_fwd_flag\"?,\"?PULocationID\"?,\"?DOLocationID\"?,\"?payment_type\"?,\"?fare_amount\"?," + "\"?RatecodeID\"?,\"?store_and_fwd_flag\"?,\"?PULocationID\"?,\"?DOLocationID\"?,\"?payment_type\"?,\"?fare_amount\"?," +
"\"?extra\"?,\"?mta_tax\"?,\"?tip_amount\"?,\"?tolls_amount\"?,\"?improvement_surcharge\"?,\"?total_amount\"?,\"?\"?,\"?\"?", "\"?extra\"?,\"?mta_tax\"?,\"?tip_amount\"?,\"?tolls_amount\"?,\"?improvement_surcharge\"?,\"?total_amount\"?,\"?\"?,\"?\"?",
structure.getExcludeLinesPattern()); structure.getExcludeLinesPattern());
assertEquals("^.*?,.*?,\"?\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern()); assertEquals("^.*?,.*?,\"?\\d{4}-\\d{2}-\\d{2}[T ]\\d{2}:\\d{2}", structure.getMultilineStartPattern());
assertEquals(Character.valueOf(','), structure.getDelimiter()); assertEquals(Character.valueOf(','), structure.getDelimiter());
assertEquals(Character.valueOf('"'), structure.getQuote()); assertEquals(Character.valueOf('"'), structure.getQuote());
assertTrue(structure.getHasHeaderRow()); assertTrue(structure.getHasHeaderRow());
@ -270,7 +270,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
"\"?RatecodeID\"?,\"?store_and_fwd_flag\"?,\"?PULocationID\"?,\"?DOLocationID\"?,\"?payment_type\"?,\"?fare_amount\"?," + "\"?RatecodeID\"?,\"?store_and_fwd_flag\"?,\"?PULocationID\"?,\"?DOLocationID\"?,\"?payment_type\"?,\"?fare_amount\"?," +
"\"?extra\"?,\"?mta_tax\"?,\"?tip_amount\"?,\"?tolls_amount\"?,\"?improvement_surcharge\"?,\"?total_amount\"?", "\"?extra\"?,\"?mta_tax\"?,\"?tip_amount\"?,\"?tolls_amount\"?,\"?improvement_surcharge\"?,\"?total_amount\"?",
structure.getExcludeLinesPattern()); structure.getExcludeLinesPattern());
assertEquals("^.*?,\"?\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern()); assertEquals("^.*?,\"?\\d{4}-\\d{2}-\\d{2}[T ]\\d{2}:\\d{2}", structure.getMultilineStartPattern());
assertEquals(Character.valueOf(','), structure.getDelimiter()); assertEquals(Character.valueOf(','), structure.getDelimiter());
assertEquals(Character.valueOf('"'), structure.getQuote()); assertEquals(Character.valueOf('"'), structure.getQuote());
assertTrue(structure.getHasHeaderRow()); assertTrue(structure.getHasHeaderRow());
@ -317,7 +317,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
"\"?RatecodeID\"?,\"?store_and_fwd_flag\"?,\"?PULocationID\"?,\"?DOLocationID\"?,\"?payment_type\"?,\"?fare_amount\"?," + "\"?RatecodeID\"?,\"?store_and_fwd_flag\"?,\"?PULocationID\"?,\"?DOLocationID\"?,\"?payment_type\"?,\"?fare_amount\"?," +
"\"?extra\"?,\"?mta_tax\"?,\"?tip_amount\"?,\"?tolls_amount\"?,\"?improvement_surcharge\"?,\"?total_amount\"?", "\"?extra\"?,\"?mta_tax\"?,\"?tip_amount\"?,\"?tolls_amount\"?,\"?improvement_surcharge\"?,\"?total_amount\"?",
structure.getExcludeLinesPattern()); structure.getExcludeLinesPattern());
assertEquals("^.*?,\"?\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern()); assertEquals("^.*?,\"?\\d{4}-\\d{2}-\\d{2}[T ]\\d{2}:\\d{2}", structure.getMultilineStartPattern());
assertEquals(Character.valueOf(','), structure.getDelimiter()); assertEquals(Character.valueOf(','), structure.getDelimiter());
assertEquals(Character.valueOf('"'), structure.getQuote()); assertEquals(Character.valueOf('"'), structure.getQuote());
assertTrue(structure.getHasHeaderRow()); assertTrue(structure.getHasHeaderRow());

View File

@ -7,7 +7,6 @@ package org.elasticsearch.xpack.ml.filestructurefinder;
import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats; import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats;
import org.elasticsearch.xpack.ml.filestructurefinder.TimestampFormatFinder.TimestampMatch;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
@ -35,12 +34,12 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
public void testGuessTimestampGivenSingleSampleSingleField() { public void testGuessTimestampGivenSingleSampleSingleField() {
Map<String, String> sample = Collections.singletonMap("field1", "2018-05-24T17:28:31,735"); Map<String, String> sample = Collections.singletonMap("field1", "2018-05-24T17:28:31,735");
Tuple<String, TimestampMatch> match = FileStructureUtils.guessTimestampField(explanation, Collections.singletonList(sample), Tuple<String, TimestampFormatFinder> match = FileStructureUtils.guessTimestampField(explanation, Collections.singletonList(sample),
EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER); EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER);
assertNotNull(match); assertNotNull(match);
assertEquals("field1", match.v1()); assertEquals("field1", match.v1());
assertThat(match.v2().javaTimestampFormats, contains("yyyy-MM-dd'T'HH:mm:ss,SSS")); assertThat(match.v2().getJavaTimestampFormats(), contains("ISO8601"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName); assertEquals("TIMESTAMP_ISO8601", match.v2().getGrokPatternName());
} }
public void testGuessTimestampGivenSingleSampleSingleFieldAndConsistentTimeFieldOverride() { public void testGuessTimestampGivenSingleSampleSingleFieldAndConsistentTimeFieldOverride() {
@ -48,12 +47,12 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
FileStructureOverrides overrides = FileStructureOverrides.builder().setTimestampField("field1").build(); FileStructureOverrides overrides = FileStructureOverrides.builder().setTimestampField("field1").build();
Map<String, String> sample = Collections.singletonMap("field1", "2018-05-24T17:28:31,735"); Map<String, String> sample = Collections.singletonMap("field1", "2018-05-24T17:28:31,735");
Tuple<String, TimestampMatch> match = FileStructureUtils.guessTimestampField(explanation, Collections.singletonList(sample), Tuple<String, TimestampFormatFinder> match = FileStructureUtils.guessTimestampField(explanation, Collections.singletonList(sample),
overrides, NOOP_TIMEOUT_CHECKER); overrides, NOOP_TIMEOUT_CHECKER);
assertNotNull(match); assertNotNull(match);
assertEquals("field1", match.v1()); assertEquals("field1", match.v1());
assertThat(match.v2().javaTimestampFormats, contains("yyyy-MM-dd'T'HH:mm:ss,SSS")); assertThat(match.v2().getJavaTimestampFormats(), contains("ISO8601"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName); assertEquals("TIMESTAMP_ISO8601", match.v2().getGrokPatternName());
} }
public void testGuessTimestampGivenSingleSampleSingleFieldAndImpossibleTimeFieldOverride() { public void testGuessTimestampGivenSingleSampleSingleFieldAndImpossibleTimeFieldOverride() {
@ -73,12 +72,12 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
FileStructureOverrides overrides = FileStructureOverrides.builder().setTimestampFormat("ISO8601").build(); FileStructureOverrides overrides = FileStructureOverrides.builder().setTimestampFormat("ISO8601").build();
Map<String, String> sample = Collections.singletonMap("field1", "2018-05-24T17:28:31,735"); Map<String, String> sample = Collections.singletonMap("field1", "2018-05-24T17:28:31,735");
Tuple<String, TimestampMatch> match = FileStructureUtils.guessTimestampField(explanation, Collections.singletonList(sample), Tuple<String, TimestampFormatFinder> match = FileStructureUtils.guessTimestampField(explanation, Collections.singletonList(sample),
overrides, NOOP_TIMEOUT_CHECKER); overrides, NOOP_TIMEOUT_CHECKER);
assertNotNull(match); assertNotNull(match);
assertEquals("field1", match.v1()); assertEquals("field1", match.v1());
assertThat(match.v2().javaTimestampFormats, contains("yyyy-MM-dd'T'HH:mm:ss,SSS")); assertThat(match.v2().getJavaTimestampFormats(), contains("ISO8601"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName); assertEquals("TIMESTAMP_ISO8601", match.v2().getGrokPatternName());
} }
public void testGuessTimestampGivenSingleSampleSingleFieldAndImpossibleTimeFormatOverride() { public void testGuessTimestampGivenSingleSampleSingleFieldAndImpossibleTimeFormatOverride() {
@ -97,18 +96,18 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
public void testGuessTimestampGivenSamplesWithSameSingleTimeField() { public void testGuessTimestampGivenSamplesWithSameSingleTimeField() {
Map<String, String> sample1 = Collections.singletonMap("field1", "2018-05-24T17:28:31,735"); Map<String, String> sample1 = Collections.singletonMap("field1", "2018-05-24T17:28:31,735");
Map<String, String> sample2 = Collections.singletonMap("field1", "2018-05-24T17:33:39,406"); Map<String, String> sample2 = Collections.singletonMap("field1", "2018-05-24T17:33:39,406");
Tuple<String, TimestampMatch> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), Tuple<String, TimestampFormatFinder> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2),
EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER); EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER);
assertNotNull(match); assertNotNull(match);
assertEquals("field1", match.v1()); assertEquals("field1", match.v1());
assertThat(match.v2().javaTimestampFormats, contains("yyyy-MM-dd'T'HH:mm:ss,SSS")); assertThat(match.v2().getJavaTimestampFormats(), contains("ISO8601"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName); assertEquals("TIMESTAMP_ISO8601", match.v2().getGrokPatternName());
} }
public void testGuessTimestampGivenSamplesWithOneSingleTimeFieldDifferentFormat() { public void testGuessTimestampGivenSamplesWithOneSingleTimeFieldDifferentFormat() {
Map<String, String> sample1 = Collections.singletonMap("field1", "2018-05-24T17:28:31,735"); Map<String, String> sample1 = Collections.singletonMap("field1", "2018-05-24T17:28:31,735");
Map<String, String> sample2 = Collections.singletonMap("field1", "2018-05-24 17:33:39,406"); Map<String, String> sample2 = Collections.singletonMap("field1", "Thu May 24 17:33:39 2018");
Tuple<String, TimestampMatch> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), Tuple<String, TimestampFormatFinder> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2),
EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER); EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER);
assertNull(match); assertNull(match);
} }
@ -116,7 +115,7 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
public void testGuessTimestampGivenSamplesWithDifferentSingleTimeField() { public void testGuessTimestampGivenSamplesWithDifferentSingleTimeField() {
Map<String, String> sample1 = Collections.singletonMap("field1", "2018-05-24T17:28:31,735"); Map<String, String> sample1 = Collections.singletonMap("field1", "2018-05-24T17:28:31,735");
Map<String, String> sample2 = Collections.singletonMap("another_field", "2018-05-24T17:33:39,406"); Map<String, String> sample2 = Collections.singletonMap("another_field", "2018-05-24T17:33:39,406");
Tuple<String, TimestampMatch> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), Tuple<String, TimestampFormatFinder> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2),
EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER); EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER);
assertNull(match); assertNull(match);
} }
@ -126,12 +125,12 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
sample.put("foo", "not a time"); sample.put("foo", "not a time");
sample.put("time", "2018-05-24 17:28:31,735"); sample.put("time", "2018-05-24 17:28:31,735");
sample.put("bar", 42); sample.put("bar", 42);
Tuple<String, TimestampMatch> match = FileStructureUtils.guessTimestampField(explanation, Collections.singletonList(sample), Tuple<String, TimestampFormatFinder> match = FileStructureUtils.guessTimestampField(explanation, Collections.singletonList(sample),
EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER); EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER);
assertNotNull(match); assertNotNull(match);
assertEquals("time", match.v1()); assertEquals("time", match.v1());
assertThat(match.v2().javaTimestampFormats, contains("yyyy-MM-dd HH:mm:ss,SSS")); assertThat(match.v2().getJavaTimestampFormats(), contains("yyyy-MM-dd HH:mm:ss,SSS"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName); assertEquals("TIMESTAMP_ISO8601", match.v2().getGrokPatternName());
} }
public void testGuessTimestampGivenSamplesWithManyFieldsSameSingleTimeFormat() { public void testGuessTimestampGivenSamplesWithManyFieldsSameSingleTimeFormat() {
@ -143,12 +142,12 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
sample2.put("foo", "whatever"); sample2.put("foo", "whatever");
sample2.put("time", "2018-05-29 11:53:02,837"); sample2.put("time", "2018-05-29 11:53:02,837");
sample2.put("bar", 17); sample2.put("bar", 17);
Tuple<String, TimestampMatch> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), Tuple<String, TimestampFormatFinder> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2),
EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER); EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER);
assertNotNull(match); assertNotNull(match);
assertEquals("time", match.v1()); assertEquals("time", match.v1());
assertThat(match.v2().javaTimestampFormats, contains("yyyy-MM-dd HH:mm:ss,SSS")); assertThat(match.v2().getJavaTimestampFormats(), contains("yyyy-MM-dd HH:mm:ss,SSS"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName); assertEquals("TIMESTAMP_ISO8601", match.v2().getGrokPatternName());
} }
public void testGuessTimestampGivenSamplesWithManyFieldsSameTimeFieldDifferentTimeFormat() { public void testGuessTimestampGivenSamplesWithManyFieldsSameTimeFieldDifferentTimeFormat() {
@ -160,7 +159,7 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
sample2.put("foo", "whatever"); sample2.put("foo", "whatever");
sample2.put("time", "May 29 2018 11:53:02"); sample2.put("time", "May 29 2018 11:53:02");
sample2.put("bar", 17); sample2.put("bar", 17);
Tuple<String, TimestampMatch> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), Tuple<String, TimestampFormatFinder> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2),
EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER); EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER);
assertNull(match); assertNull(match);
} }
@ -174,12 +173,12 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
sample2.put("red_herring", "whatever"); sample2.put("red_herring", "whatever");
sample2.put("time", "2018-05-29 11:53:02,837"); sample2.put("time", "2018-05-29 11:53:02,837");
sample2.put("bar", 17); sample2.put("bar", 17);
Tuple<String, TimestampMatch> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), Tuple<String, TimestampFormatFinder> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2),
EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER); EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER);
assertNotNull(match); assertNotNull(match);
assertEquals("time", match.v1()); assertEquals("time", match.v1());
assertThat(match.v2().javaTimestampFormats, contains("yyyy-MM-dd HH:mm:ss,SSS")); assertThat(match.v2().getJavaTimestampFormats(), contains("yyyy-MM-dd HH:mm:ss,SSS"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName); assertEquals("TIMESTAMP_ISO8601", match.v2().getGrokPatternName());
} }
public void testGuessTimestampGivenSamplesWithManyFieldsSameSingleTimeFormatDistractionAfter() { public void testGuessTimestampGivenSamplesWithManyFieldsSameSingleTimeFormatDistractionAfter() {
@ -191,12 +190,12 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
sample2.put("foo", "whatever"); sample2.put("foo", "whatever");
sample2.put("time", "May 29 2018 11:53:02"); sample2.put("time", "May 29 2018 11:53:02");
sample2.put("red_herring", "17"); sample2.put("red_herring", "17");
Tuple<String, TimestampMatch> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), Tuple<String, TimestampFormatFinder> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2),
EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER); EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER);
assertNotNull(match); assertNotNull(match);
assertEquals("time", match.v1()); assertEquals("time", match.v1());
assertThat(match.v2().javaTimestampFormats, contains("MMM dd yyyy HH:mm:ss", "MMM d yyyy HH:mm:ss")); assertThat(match.v2().getJavaTimestampFormats(), contains("MMM dd yyyy HH:mm:ss", "MMM d yyyy HH:mm:ss"));
assertEquals("CISCOTIMESTAMP", match.v2().grokPatternName); assertEquals("CISCOTIMESTAMP", match.v2().getGrokPatternName());
} }
public void testGuessTimestampGivenSamplesWithManyFieldsInconsistentTimeFields() { public void testGuessTimestampGivenSamplesWithManyFieldsInconsistentTimeFields() {
@ -208,7 +207,7 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
sample2.put("foo", "whatever"); sample2.put("foo", "whatever");
sample2.put("time2", "May 29 2018 11:53:02"); sample2.put("time2", "May 29 2018 11:53:02");
sample2.put("bar", 42); sample2.put("bar", 42);
Tuple<String, TimestampMatch> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), Tuple<String, TimestampFormatFinder> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2),
EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER); EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER);
assertNull(match); assertNull(match);
} }
@ -224,12 +223,12 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
sample2.put("time2", "May 10 2018 11:53:02"); sample2.put("time2", "May 10 2018 11:53:02");
sample2.put("time3", "Thu, May 10 2018 11:53:02"); sample2.put("time3", "Thu, May 10 2018 11:53:02");
sample2.put("bar", 42); sample2.put("bar", 42);
Tuple<String, TimestampMatch> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), Tuple<String, TimestampFormatFinder> match = FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2),
EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER); EMPTY_OVERRIDES, NOOP_TIMEOUT_CHECKER);
assertNotNull(match); assertNotNull(match);
assertEquals("time2", match.v1()); assertEquals("time2", match.v1());
assertThat(match.v2().javaTimestampFormats, contains("MMM dd yyyy HH:mm:ss", "MMM d yyyy HH:mm:ss")); assertThat(match.v2().getJavaTimestampFormats(), contains("MMM dd yyyy HH:mm:ss", "MMM d yyyy HH:mm:ss"));
assertEquals("CISCOTIMESTAMP", match.v2().grokPatternName); assertEquals("CISCOTIMESTAMP", match.v2().getGrokPatternName());
} }
public void testGuessMappingGivenNothing() { public void testGuessMappingGivenNothing() {
@ -273,7 +272,9 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
} }
public void testGuessMappingGivenDate() { public void testGuessMappingGivenDate() {
Map<String, String> expected = Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "date"); Map<String, String> expected = new HashMap<>();
expected.put(FileStructureUtils.MAPPING_TYPE_SETTING, "date");
expected.put(FileStructureUtils.MAPPING_FORMAT_SETTING, "iso8601");
assertEquals(expected, guessMapping(explanation, "foo", Arrays.asList("2018-06-11T13:26:47Z", "2018-06-11T13:27:12Z"))); assertEquals(expected, guessMapping(explanation, "foo", Arrays.asList("2018-06-11T13:26:47Z", "2018-06-11T13:27:12Z")));
} }
@ -347,18 +348,19 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
public void testMakeIngestPipelineDefinitionGivenStructuredWithoutTimestamp() { public void testMakeIngestPipelineDefinitionGivenStructuredWithoutTimestamp() {
assertNull(FileStructureUtils.makeIngestPipelineDefinition(null, null, null, false)); assertNull(FileStructureUtils.makeIngestPipelineDefinition(null, Collections.emptyMap(), null, null, false));
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public void testMakeIngestPipelineDefinitionGivenStructuredWithTimestamp() { public void testMakeIngestPipelineDefinitionGivenStructuredWithTimestamp() {
String timestampField = randomAlphaOfLength(10); String timestampField = randomAlphaOfLength(10);
List<String> timestampFormats = randomFrom(TimestampFormatFinder.ORDERED_CANDIDATE_FORMATS).javaTimestampFormats; List<String> timestampFormats = randomFrom(Collections.singletonList("ISO8601"),
Arrays.asList("EEE MMM dd HH:mm:ss yyyy", "EEE MMM d HH:mm:ss yyyy"));
boolean needClientTimezone = randomBoolean(); boolean needClientTimezone = randomBoolean();
Map<String, Object> pipeline = Map<String, Object> pipeline = FileStructureUtils.makeIngestPipelineDefinition(null, Collections.emptyMap(), timestampField,
FileStructureUtils.makeIngestPipelineDefinition(null, timestampField, timestampFormats, needClientTimezone); timestampFormats, needClientTimezone);
assertNotNull(pipeline); assertNotNull(pipeline);
assertEquals("Ingest pipeline created by file structure finder", pipeline.remove("description")); assertEquals("Ingest pipeline created by file structure finder", pipeline.remove("description"));
@ -382,11 +384,12 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
String grokPattern = randomAlphaOfLength(100); String grokPattern = randomAlphaOfLength(100);
String timestampField = randomAlphaOfLength(10); String timestampField = randomAlphaOfLength(10);
List<String> timestampFormats = randomFrom(TimestampFormatFinder.ORDERED_CANDIDATE_FORMATS).javaTimestampFormats; List<String> timestampFormats = randomFrom(Collections.singletonList("ISO8601"),
Arrays.asList("EEE MMM dd HH:mm:ss yyyy", "EEE MMM d HH:mm:ss yyyy"));
boolean needClientTimezone = randomBoolean(); boolean needClientTimezone = randomBoolean();
Map<String, Object> pipeline = Map<String, Object> pipeline = FileStructureUtils.makeIngestPipelineDefinition(grokPattern, Collections.emptyMap(), timestampField,
FileStructureUtils.makeIngestPipelineDefinition(grokPattern, timestampField, timestampFormats, needClientTimezone); timestampFormats, needClientTimezone);
assertNotNull(pipeline); assertNotNull(pipeline);
assertEquals("Ingest pipeline created by file structure finder", pipeline.remove("description")); assertEquals("Ingest pipeline created by file structure finder", pipeline.remove("description"));

View File

@ -43,7 +43,7 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
Collection<String> prefaces = new ArrayList<>(); Collection<String> prefaces = new ArrayList<>();
Collection<String> epilogues = new ArrayList<>(); Collection<String> epilogues = new ArrayList<>();
candidate.processCaptures(fieldNameCountStore, matchingStrings, prefaces, epilogues, null, null, NOOP_TIMEOUT_CHECKER); candidate.processCaptures(explanation, fieldNameCountStore, matchingStrings, prefaces, epilogues, null, null, NOOP_TIMEOUT_CHECKER);
assertThat(prefaces, containsInAnyOrder("[", "[", "junk [", "[")); assertThat(prefaces, containsInAnyOrder("[", "[", "junk [", "["));
assertThat(epilogues, containsInAnyOrder("] DEBUG ", "] ERROR ", "] INFO ", "] DEBUG ")); assertThat(epilogues, containsInAnyOrder("] DEBUG ", "] ERROR ", "] INFO ", "] DEBUG "));
@ -60,7 +60,7 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
Collection<String> prefaces = new ArrayList<>(); Collection<String> prefaces = new ArrayList<>();
Collection<String> epilogues = new ArrayList<>(); Collection<String> epilogues = new ArrayList<>();
candidate.processCaptures(fieldNameCountStore, matchingStrings, prefaces, epilogues, null, null, NOOP_TIMEOUT_CHECKER); candidate.processCaptures(explanation, fieldNameCountStore, matchingStrings, prefaces, epilogues, null, null, NOOP_TIMEOUT_CHECKER);
assertThat(prefaces, containsInAnyOrder("before ", "abc ", "")); assertThat(prefaces, containsInAnyOrder("before ", "abc ", ""));
assertThat(epilogues, containsInAnyOrder(" after", " xyz", "")); assertThat(epilogues, containsInAnyOrder(" after", " xyz", ""));
@ -73,7 +73,8 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
"junk [2018-01-22T07:33:23] INFO ", "junk [2018-01-22T07:33:23] INFO ",
"[2018-01-21T03:33:23] DEBUG "); "[2018-01-21T03:33:23] DEBUG ");
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, NOOP_TIMEOUT_CHECKER); GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER);
grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0); grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0);
assertEquals(".*?\\[%{TIMESTAMP_ISO8601:extra_timestamp}\\] %{LOGLEVEL:loglevel} ", assertEquals(".*?\\[%{TIMESTAMP_ISO8601:extra_timestamp}\\] %{LOGLEVEL:loglevel} ",
@ -87,7 +88,8 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
" (4)", " (4)",
" (-5) "); " (-5) ");
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, NOOP_TIMEOUT_CHECKER); GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER);
grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0); grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0);
assertEquals(".*?\\(%{INT:field}\\).*?", grokPatternCreator.getOverallGrokPatternBuilder().toString()); assertEquals(".*?\\(%{INT:field}\\).*?", grokPatternCreator.getOverallGrokPatternBuilder().toString());
@ -99,7 +101,8 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
"prior to-3", "prior to-3",
"-4"); "-4");
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, NOOP_TIMEOUT_CHECKER); GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER);
grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0); grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0);
// It seems sensible that we don't detect these suffices as either base 10 or base 16 numbers // It seems sensible that we don't detect these suffices as either base 10 or base 16 numbers
@ -113,7 +116,8 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
" -123", " -123",
"1f is hex"); "1f is hex");
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, NOOP_TIMEOUT_CHECKER); GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER);
grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0); grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0);
assertEquals(".*?%{BASE16NUM:field}.*?", grokPatternCreator.getOverallGrokPatternBuilder().toString()); assertEquals(".*?%{BASE16NUM:field}.*?", grokPatternCreator.getOverallGrokPatternBuilder().toString());
@ -124,7 +128,8 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
Collection<String> snippets = Arrays.asList("<host1.1.p2ps:", Collection<String> snippets = Arrays.asList("<host1.1.p2ps:",
"<host2.1.p2ps:"); "<host2.1.p2ps:");
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, NOOP_TIMEOUT_CHECKER); GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER);
grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0); grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0);
// We don't want the .1. in the middle to get detected as a hex number // We don't want the .1. in the middle to get detected as a hex number
@ -137,7 +142,8 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
"abc bob@acme.com xyz", "abc bob@acme.com xyz",
"carol@acme.com"); "carol@acme.com");
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, NOOP_TIMEOUT_CHECKER); GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER);
grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0); grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0);
assertEquals(".*?%{EMAILADDRESS:email}.*?", grokPatternCreator.getOverallGrokPatternBuilder().toString()); assertEquals(".*?%{EMAILADDRESS:email}.*?", grokPatternCreator.getOverallGrokPatternBuilder().toString());
@ -149,7 +155,8 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
"https://www.elastic.co/guide/en/x-pack/current/ml-configuring-categories.html#ml-configuring-categories is a section", "https://www.elastic.co/guide/en/x-pack/current/ml-configuring-categories.html#ml-configuring-categories is a section",
"download today from https://www.elastic.co/downloads"); "download today from https://www.elastic.co/downloads");
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, NOOP_TIMEOUT_CHECKER); GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER);
grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0); grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0);
assertEquals(".*?%{URI:uri}.*?", grokPatternCreator.getOverallGrokPatternBuilder().toString()); assertEquals(".*?%{URI:uri}.*?", grokPatternCreator.getOverallGrokPatternBuilder().toString());
@ -161,7 +168,8 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
"on Windows C:\\Users\\dave", "on Windows C:\\Users\\dave",
"on Linux /home/dave"); "on Linux /home/dave");
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, NOOP_TIMEOUT_CHECKER); GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER);
grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0); grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0);
assertEquals(".*? .*? %{PATH:path}", grokPatternCreator.getOverallGrokPatternBuilder().toString()); assertEquals(".*? .*? %{PATH:path}", grokPatternCreator.getOverallGrokPatternBuilder().toString());
@ -174,7 +182,8 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
"foo=3 bar=c", "foo=3 bar=c",
" foo=1 bar=a "); " foo=1 bar=a ");
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, NOOP_TIMEOUT_CHECKER); GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER);
grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0); grokPatternCreator.appendBestGrokMatchForStrings(false, snippets, false, 0);
assertEquals(".*?\\bfoo=%{USER:foo} .*?\\bbar=%{USER:bar}.*?", grokPatternCreator.getOverallGrokPatternBuilder().toString()); assertEquals(".*?\\bfoo=%{USER:foo} .*?\\bbar=%{USER:bar}.*?", grokPatternCreator.getOverallGrokPatternBuilder().toString());
@ -189,7 +198,7 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
"Sep 8 11:55:42 linux named[22529]: error (unexpected RCODE REFUSED) resolving 'b.akamaiedge.net/A/IN': 95.110.64.205#53"); "Sep 8 11:55:42 linux named[22529]: error (unexpected RCODE REFUSED) resolving 'b.akamaiedge.net/A/IN': 95.110.64.205#53");
Map<String, Object> mappings = new HashMap<>(); Map<String, Object> mappings = new HashMap<>();
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER); NOOP_TIMEOUT_CHECKER);
assertEquals("%{SYSLOGTIMESTAMP:timestamp} .*? .*?\\[%{INT:field}\\]: %{LOGLEVEL:loglevel} \\(.*? .*? .*?\\) .*? " + assertEquals("%{SYSLOGTIMESTAMP:timestamp} .*? .*?\\[%{INT:field}\\]: %{LOGLEVEL:loglevel} \\(.*? .*? .*?\\) .*? " +
@ -216,7 +225,7 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
"Invalid chunk ignored."); "Invalid chunk ignored.");
Map<String, Object> mappings = new HashMap<>(); Map<String, Object> mappings = new HashMap<>();
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER); NOOP_TIMEOUT_CHECKER);
assertEquals("%{CATALINA_DATESTAMP:timestamp} .*? .*?\\n%{LOGLEVEL:loglevel}: .*", assertEquals("%{CATALINA_DATESTAMP:timestamp} .*? .*?\\n%{LOGLEVEL:loglevel}: .*",
@ -239,7 +248,7 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
"Info\tsshd\tsubsystem request for sftp"); "Info\tsshd\tsubsystem request for sftp");
Map<String, Object> mappings = new HashMap<>(); Map<String, Object> mappings = new HashMap<>();
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER); NOOP_TIMEOUT_CHECKER);
assertEquals("%{INT:field}\\t%{TIMESTAMP_ISO8601:timestamp}\\t%{TIMESTAMP_ISO8601:extra_timestamp}\\t%{INT:field2}\\t.*?\\t" + assertEquals("%{INT:field}\\t%{TIMESTAMP_ISO8601:timestamp}\\t%{TIMESTAMP_ISO8601:extra_timestamp}\\t%{INT:field2}\\t.*?\\t" +
@ -247,7 +256,101 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
grokPatternCreator.createGrokPatternFromExamples("TIMESTAMP_ISO8601", "timestamp")); grokPatternCreator.createGrokPatternFromExamples("TIMESTAMP_ISO8601", "timestamp"));
assertEquals(5, mappings.size()); assertEquals(5, mappings.size());
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("field")); assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("field"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "date"), mappings.get("extra_timestamp")); Map<String, String> expectedDateMapping = new HashMap<>();
expectedDateMapping.put(FileStructureUtils.MAPPING_TYPE_SETTING, "date");
expectedDateMapping.put(FileStructureUtils.MAPPING_FORMAT_SETTING, "iso8601");
assertEquals(expectedDateMapping, mappings.get("extra_timestamp"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("field2"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "ip"), mappings.get("ipaddress"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("loglevel"));
}
public void testCreateGrokPatternFromExamplesGivenMultiTimestampLogsAndIndeterminateFormat() {
// Two timestamps: one ISO8601, one indeterminate day/month
Collection<String> sampleMessages = Arrays.asList(
"559550912540598297\t2016-04-20T14:06:53\t20/04/2016 21:06:53,123456\t38545844\tserv02nw07\t192.168.114.28\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp",
"559550912548986880\t2016-04-20T14:06:53\t20/04/2016 21:06:53,123456\t9049724\tserv02nw03\t10.120.48.147\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp",
"559550912548986887\t2016-04-20T14:06:53\t20/04/2016 21:06:53,123456\t884343\tserv02tw03\t192.168.121.189\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp",
"559550912603512850\t2016-04-20T14:06:53\t20/04/2016 21:06:53,123456\t8907014\tserv02nw01\t192.168.118.208\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp");
Map<String, Object> mappings = new HashMap<>();
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER);
assertEquals("%{INT:field}\\t%{TIMESTAMP_ISO8601:timestamp}\\t%{DATESTAMP:extra_timestamp}\\t%{INT:field2}\\t.*?\\t" +
"%{IP:ipaddress}\\t.*?\\t%{LOGLEVEL:loglevel}\\t.*",
grokPatternCreator.createGrokPatternFromExamples("TIMESTAMP_ISO8601", "timestamp"));
assertEquals(5, mappings.size());
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("field"));
Map<String, String> expectedDateMapping = new HashMap<>();
expectedDateMapping.put(FileStructureUtils.MAPPING_TYPE_SETTING, "date");
expectedDateMapping.put(FileStructureUtils.MAPPING_FORMAT_SETTING, "dd/MM/yyyy HH:mm:ss,SSSSSS");
assertEquals(expectedDateMapping, mappings.get("extra_timestamp"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("field2"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "ip"), mappings.get("ipaddress"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("loglevel"));
}
public void testCreateGrokPatternFromExamplesGivenMultiTimestampLogsAndCustomDefinition() {
// Two timestamps: one custom, one built-in
Collection<String> sampleMessages = Arrays.asList(
"559550912540598297\t4/20/2016 2:06PM\t2016-04-20T21:06:53Z\t38545844\tserv02nw07\t192.168.114.28\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp",
"559550912548986880\t4/20/2016 2:06PM\t2016-04-20T21:06:53Z\t9049724\tserv02nw03\t10.120.48.147\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp",
"559550912548986887\t4/20/2016 2:06PM\t2016-04-20T21:06:53Z\t884343\tserv02tw03\t192.168.121.189\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp",
"559550912603512850\t4/20/2016 2:06PM\t2016-04-20T21:06:53Z\t8907014\tserv02nw01\t192.168.118.208\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp");
Map<String, Object> mappings = new HashMap<>();
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null,
Collections.singletonMap("CUSTOM_TIMESTAMP", "%{MONTHNUM}/%{MONTHDAY}/%{YEAR} %{HOUR}:%{MINUTE}(?:AM|PM)"),
NOOP_TIMEOUT_CHECKER);
assertEquals("%{INT:field}\\t%{CUSTOM_TIMESTAMP:timestamp}\\t%{TIMESTAMP_ISO8601:extra_timestamp}\\t%{INT:field2}\\t.*?\\t" +
"%{IP:ipaddress}\\t.*?\\t%{LOGLEVEL:loglevel}\\t.*",
grokPatternCreator.createGrokPatternFromExamples("CUSTOM_TIMESTAMP", "timestamp"));
assertEquals(5, mappings.size());
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("field"));
Map<String, String> expectedDateMapping = new HashMap<>();
expectedDateMapping.put(FileStructureUtils.MAPPING_TYPE_SETTING, "date");
expectedDateMapping.put(FileStructureUtils.MAPPING_FORMAT_SETTING, "iso8601");
assertEquals(expectedDateMapping, mappings.get("extra_timestamp"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("field2"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "ip"), mappings.get("ipaddress"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("loglevel"));
}
public void testCreateGrokPatternFromExamplesGivenTimestampAndTimeWithoutDate() {
// Two timestamps: one with date, one without
Collection<String> sampleMessages = Arrays.asList(
"559550912540598297\t2016-04-20T14:06:53\t21:06:53.123456\t38545844\tserv02nw07\t192.168.114.28\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp",
"559550912548986880\t2016-04-20T14:06:53\t21:06:53.123456\t9049724\tserv02nw03\t10.120.48.147\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp",
"559550912548986887\t2016-04-20T14:06:53\t21:06:53.123456\t884343\tserv02tw03\t192.168.121.189\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp",
"559550912603512850\t2016-04-20T14:06:53\t21:06:53.123456\t8907014\tserv02nw01\t192.168.118.208\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp");
Map<String, Object> mappings = new HashMap<>();
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER);
assertEquals("%{INT:field}\\t%{TIMESTAMP_ISO8601:timestamp}\\t%{TIME:time}\\t%{INT:field2}\\t.*?\\t" +
"%{IP:ipaddress}\\t.*?\\t%{LOGLEVEL:loglevel}\\t.*",
grokPatternCreator.createGrokPatternFromExamples("TIMESTAMP_ISO8601", "timestamp"));
assertEquals(5, mappings.size());
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("field"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("time"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("field2")); assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("field2"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "ip"), mappings.get("ipaddress")); assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "ip"), mappings.get("ipaddress"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("loglevel")); assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("loglevel"));
@ -273,7 +376,7 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
"AppleWebKit/537.36 (KHTML, like Gecko) Chrome/32.0.1700.77 Safari/537.36\""); "AppleWebKit/537.36 (KHTML, like Gecko) Chrome/32.0.1700.77 Safari/537.36\"");
Map<String, Object> mappings = new HashMap<>(); Map<String, Object> mappings = new HashMap<>();
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER); NOOP_TIMEOUT_CHECKER);
assertEquals(new Tuple<>("timestamp", "%{COMBINEDAPACHELOG}"), assertEquals(new Tuple<>("timestamp", "%{COMBINEDAPACHELOG}"),
@ -304,7 +407,8 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
",\"rule1\",\"Accept\",\"\",\"\",\"\",\"0000000000000000\"" ",\"rule1\",\"Accept\",\"\",\"\",\"\",\"0000000000000000\""
); );
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, NOOP_TIMEOUT_CHECKER); GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER);
Collection<String> adjustedSnippets = grokPatternCreator.adjustForPunctuation(snippets); Collection<String> adjustedSnippets = grokPatternCreator.adjustForPunctuation(snippets);
assertEquals("\",", grokPatternCreator.getOverallGrokPatternBuilder().toString()); assertEquals("\",", grokPatternCreator.getOverallGrokPatternBuilder().toString());
@ -321,7 +425,8 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
"was added by 'User1'(id:2) to servergroup 'GAME'(id:9)" "was added by 'User1'(id:2) to servergroup 'GAME'(id:9)"
); );
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, NOOP_TIMEOUT_CHECKER); GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, snippets, null, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER);
Collection<String> adjustedSnippets = grokPatternCreator.adjustForPunctuation(snippets); Collection<String> adjustedSnippets = grokPatternCreator.adjustForPunctuation(snippets);
assertEquals("", grokPatternCreator.getOverallGrokPatternBuilder().toString()); assertEquals("", grokPatternCreator.getOverallGrokPatternBuilder().toString());
@ -347,17 +452,60 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
"Info\tsshd\tsubsystem request for sftp"); "Info\tsshd\tsubsystem request for sftp");
Map<String, Object> mappings = new HashMap<>(); Map<String, Object> mappings = new HashMap<>();
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER); NOOP_TIMEOUT_CHECKER);
grokPatternCreator.validateFullLineGrokPattern(grokPattern, timestampField); grokPatternCreator.validateFullLineGrokPattern(grokPattern, timestampField);
assertEquals(9, mappings.size()); assertEquals(9, mappings.size());
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("serial_no")); assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("serial_no"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "date"), mappings.get("local_timestamp")); Map<String, String> expectedDateMapping = new HashMap<>();
expectedDateMapping.put(FileStructureUtils.MAPPING_TYPE_SETTING, "date");
expectedDateMapping.put(FileStructureUtils.MAPPING_FORMAT_SETTING, "iso8601");
assertEquals(expectedDateMapping, mappings.get("local_timestamp"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("user_id")); assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("user_id"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("host")); assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("host"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "ip"), mappings.get("client_ip")); assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "ip"), mappings.get("client_ip"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("method")); assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("method"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("severity"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("program"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("message"));
}
public void testValidateFullLineGrokPatternGivenValidAndCustomDefinition() {
String timestampField = "local_timestamp";
String grokPattern = "%{INT:serial_no}\\t%{CUSTOM_TIMESTAMP:local_timestamp}\\t%{TIMESTAMP_ISO8601:utc_timestamp}\\t" +
"%{INT:user_id}\\t%{HOSTNAME:host}\\t%{IP:client_ip}\\t%{WORD:method}\\t%{LOGLEVEL:severity}\\t%{PROG:program}\\t" +
"%{GREEDYDATA:message}";
// Two timestamps: one local, one UTC
Collection<String> sampleMessages = Arrays.asList(
"559550912540598297\t4/20/2016 2:06PM\t2016-04-20T21:06:53Z\t38545844\tserv02nw07\t192.168.114.28\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp",
"559550912548986880\t4/20/2016 2:06PM\t2016-04-20T21:06:53Z\t9049724\tserv02nw03\t10.120.48.147\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp",
"559550912548986887\t4/20/2016 2:06PM\t2016-04-20T21:06:53Z\t884343\tserv02tw03\t192.168.121.189\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp",
"559550912603512850\t4/20/2016 2:06PM\t2016-04-20T21:06:53Z\t8907014\tserv02nw01\t192.168.118.208\tAuthpriv\t" +
"Info\tsshd\tsubsystem request for sftp");
Map<String, Object> mappings = new HashMap<>();
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null,
Collections.singletonMap("CUSTOM_TIMESTAMP", "%{MONTHNUM}/%{MONTHDAY}/%{YEAR} %{HOUR}:%{MINUTE}(?:AM|PM)"),
NOOP_TIMEOUT_CHECKER);
grokPatternCreator.validateFullLineGrokPattern(grokPattern, timestampField);
assertEquals(9, mappings.size());
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("serial_no"));
Map<String, String> expectedDateMapping = new HashMap<>();
expectedDateMapping.put(FileStructureUtils.MAPPING_TYPE_SETTING, "date");
expectedDateMapping.put(FileStructureUtils.MAPPING_FORMAT_SETTING, "iso8601");
assertEquals(expectedDateMapping, mappings.get("utc_timestamp"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "long"), mappings.get("user_id"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("host"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "ip"), mappings.get("client_ip"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("method"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("severity"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("program")); assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("program"));
assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("message")); assertEquals(Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword"), mappings.get("message"));
} }
@ -376,7 +524,7 @@ public class GrokPatternCreatorTests extends FileStructureTestCase {
"Sep 8 11:55:42 linux named[22529]: error (unexpected RCODE REFUSED) resolving 'b.akamaiedge.net/A/IN': 95.110.64.205#53"); "Sep 8 11:55:42 linux named[22529]: error (unexpected RCODE REFUSED) resolving 'b.akamaiedge.net/A/IN': 95.110.64.205#53");
Map<String, Object> mappings = new HashMap<>(); Map<String, Object> mappings = new HashMap<>();
GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, GrokPatternCreator grokPatternCreator = new GrokPatternCreator(explanation, sampleMessages, mappings, null, Collections.emptyMap(),
NOOP_TIMEOUT_CHECKER); NOOP_TIMEOUT_CHECKER);
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, IllegalArgumentException e = expectThrows(IllegalArgumentException.class,

View File

@ -5,11 +5,9 @@
*/ */
package org.elasticsearch.xpack.ml.filestructurefinder; package org.elasticsearch.xpack.ml.filestructurefinder;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats; import org.elasticsearch.xpack.core.ml.filestructurefinder.FieldStats;
import org.elasticsearch.xpack.core.ml.filestructurefinder.FileStructure; import org.elasticsearch.xpack.core.ml.filestructurefinder.FileStructure;
import org.elasticsearch.xpack.ml.filestructurefinder.TimestampFormatFinder.TimestampMatch;
import java.util.Collections; import java.util.Collections;
import java.util.Set; import java.util.Set;
@ -20,90 +18,6 @@ import static org.hamcrest.Matchers.not;
public class TextLogFileStructureFinderTests extends FileStructureTestCase { public class TextLogFileStructureFinderTests extends FileStructureTestCase {
private static final String EXCEPTION_TRACE_SAMPLE =
"[2018-02-28T14:49:40,517][DEBUG][o.e.a.b.TransportShardBulkAction] [an_index][2] failed to execute bulk item " +
"(index) BulkShardRequest [[an_index][2]] containing [33] requests\n" +
"java.lang.IllegalArgumentException: Document contains at least one immense term in field=\"message.keyword\" (whose UTF8 " +
"encoding is longer than the max length 32766), all of which were skipped. Please correct the analyzer to not produce " +
"such terms. The prefix of the first immense term is: '[60, 83, 79, 65, 80, 45, 69, 78, 86, 58, 69, 110, 118, 101, 108, " +
"111, 112, 101, 32, 120, 109, 108, 110, 115, 58, 83, 79, 65, 80, 45]...', original message: bytes can be at most 32766 " +
"in length; got 49023\n" +
"\tat org.apache.lucene.index.DefaultIndexingChain$PerField.invert(DefaultIndexingChain.java:796) " +
"~[lucene-core-7.2.1.jar:7.2.1 b2b6438b37073bee1fca40374e85bf91aa457c0b - ubuntu - 2018-01-10 00:48:43]\n" +
"\tat org.apache.lucene.index.DefaultIndexingChain.processField(DefaultIndexingChain.java:430) " +
"~[lucene-core-7.2.1.jar:7.2.1 b2b6438b37073bee1fca40374e85bf91aa457c0b - ubuntu - 2018-01-10 00:48:43]\n" +
"\tat org.apache.lucene.index.DefaultIndexingChain.processDocument(DefaultIndexingChain.java:392) " +
"~[lucene-core-7.2.1.jar:7.2.1 b2b6438b37073bee1fca40374e85bf91aa457c0b - ubuntu - 2018-01-10 00:48:43]\n" +
"\tat org.apache.lucene.index.DocumentsWriterPerThread.updateDocument(DocumentsWriterPerThread.java:240) " +
"~[lucene-core-7.2.1.jar:7.2.1 b2b6438b37073bee1fca40374e85bf91aa457c0b - ubuntu - 2018-01-10 00:48:43]\n" +
"\tat org.apache.lucene.index.DocumentsWriter.updateDocument(DocumentsWriter.java:496) " +
"~[lucene-core-7.2.1.jar:7.2.1 b2b6438b37073bee1fca40374e85bf91aa457c0b - ubuntu - 2018-01-10 00:48:43]\n" +
"\tat org.apache.lucene.index.IndexWriter.updateDocument(IndexWriter.java:1729) " +
"~[lucene-core-7.2.1.jar:7.2.1 b2b6438b37073bee1fca40374e85bf91aa457c0b - ubuntu - 2018-01-10 00:48:43]\n" +
"\tat org.apache.lucene.index.IndexWriter.addDocument(IndexWriter.java:1464) " +
"~[lucene-core-7.2.1.jar:7.2.1 b2b6438b37073bee1fca40374e85bf91aa457c0b - ubuntu - 2018-01-10 00:48:43]\n" +
"\tat org.elasticsearch.index.engine.InternalEngine.index(InternalEngine.java:1070) ~[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.index.engine.InternalEngine.indexIntoLucene(InternalEngine.java:1012) " +
"~[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.index.engine.InternalEngine.index(InternalEngine.java:878) ~[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.index.shard.IndexShard.index(IndexShard.java:738) ~[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.index.shard.IndexShard.applyIndexOperation(IndexShard.java:707) ~[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.index.shard.IndexShard.applyIndexOperationOnPrimary(IndexShard.java:673) " +
"~[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.bulk.TransportShardBulkAction.executeIndexRequestOnPrimary(TransportShardBulkAction.java:548) " +
"~[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.bulk.TransportShardBulkAction.executeIndexRequest(TransportShardBulkAction.java:140) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.bulk.TransportShardBulkAction.executeBulkItemRequest(TransportShardBulkAction.java:236) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.bulk.TransportShardBulkAction.performOnPrimary(TransportShardBulkAction.java:123) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.bulk.TransportShardBulkAction.shardOperationOnPrimary(TransportShardBulkAction.java:110) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.bulk.TransportShardBulkAction.shardOperationOnPrimary(TransportShardBulkAction.java:72) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.support.replication.TransportReplicationAction$PrimaryShardReference.perform" +
"(TransportReplicationAction.java:1034) [elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.support.replication.TransportReplicationAction$PrimaryShardReference.perform" +
"(TransportReplicationAction.java:1012) [elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.support.replication.ReplicationOperation.execute(ReplicationOperation.java:103) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.support.replication.TransportReplicationAction$AsyncPrimaryAction.onResponse" +
"(TransportReplicationAction.java:359) [elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.support.replication.TransportReplicationAction$AsyncPrimaryAction.onResponse" +
"(TransportReplicationAction.java:299) [elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.support.replication.TransportReplicationAction$1.onResponse" +
"(TransportReplicationAction.java:975) [elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.support.replication.TransportReplicationAction$1.onResponse" +
"(TransportReplicationAction.java:972) [elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.index.shard.IndexShardOperationPermits.acquire(IndexShardOperationPermits.java:238) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.index.shard.IndexShard.acquirePrimaryOperationPermit(IndexShard.java:2220) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.support.replication.TransportReplicationAction.acquirePrimaryShardReference" +
"(TransportReplicationAction.java:984) [elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.support.replication.TransportReplicationAction.access$500(TransportReplicationAction.java:98) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.support.replication.TransportReplicationAction$AsyncPrimaryAction.doRun" +
"(TransportReplicationAction.java:320) [elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.common.util.concurrent.AbstractRunnable.run(AbstractRunnable.java:37) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.support.replication.TransportReplicationAction$PrimaryOperationTransportHandler" +
".messageReceived(TransportReplicationAction.java:295) [elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.action.support.replication.TransportReplicationAction$PrimaryOperationTransportHandler" +
".messageReceived(TransportReplicationAction.java:282) [elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.transport.RequestHandlerRegistry.processMessageReceived(RequestHandlerRegistry.java:66) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.transport.TransportService$7.doRun(TransportService.java:656) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingAbstractRunnable.doRun(ThreadContext.java:635) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat org.elasticsearch.common.util.concurrent.AbstractRunnable.run(AbstractRunnable.java:37) " +
"[elasticsearch-6.2.1.jar:6.2.1]\n" +
"\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_144]\n" +
"\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_144]\n" +
"\tat java.lang.Thread.run(Thread.java:748) [?:1.8.0_144]\n";
private FileStructureFinderFactory factory = new TextLogFileStructureFinderFactory(); private FileStructureFinderFactory factory = new TextLogFileStructureFinderFactory();
public void testCreateConfigsGivenElasticsearchLog() throws Exception { public void testCreateConfigsGivenElasticsearchLog() throws Exception {
@ -124,7 +38,7 @@ public class TextLogFileStructureFinderTests extends FileStructureTestCase {
assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker()); assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker());
} }
assertNull(structure.getExcludeLinesPattern()); assertNull(structure.getExcludeLinesPattern());
assertEquals("^\\[\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}", structure.getMultilineStartPattern()); assertEquals("^\\[\\b\\d{4}-\\d{2}-\\d{2}[T ]\\d{2}:\\d{2}", structure.getMultilineStartPattern());
assertNull(structure.getDelimiter()); assertNull(structure.getDelimiter());
assertNull(structure.getQuote()); assertNull(structure.getQuote());
assertNull(structure.getHasHeaderRow()); assertNull(structure.getHasHeaderRow());
@ -139,6 +53,47 @@ public class TextLogFileStructureFinderTests extends FileStructureTestCase {
} }
} }
public void testCreateConfigsGivenElasticsearchLogAndTimestampFormatOverride() throws Exception {
String sample = "12/31/2018 1:40PM INFO foo\n" +
"1/31/2019 11:40AM DEBUG bar\n" +
"2/1/2019 11:00PM INFO foo\n" +
"2/2/2019 1:23AM DEBUG bar\n";
FileStructureOverrides overrides = FileStructureOverrides.builder().setTimestampFormat("M/d/yyyy h:mma").build();
assertTrue(factory.canCreateFromSample(explanation, sample));
String charset = randomFrom(POSSIBLE_CHARSETS);
Boolean hasByteOrderMarker = randomHasByteOrderMarker(charset);
FileStructureFinder structureFinder = factory.createFromSample(explanation, sample, charset, hasByteOrderMarker, overrides,
NOOP_TIMEOUT_CHECKER);
FileStructure structure = structureFinder.getStructure();
assertEquals(FileStructure.Format.SEMI_STRUCTURED_TEXT, structure.getFormat());
assertEquals(charset, structure.getCharset());
if (hasByteOrderMarker == null) {
assertNull(structure.getHasByteOrderMarker());
} else {
assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker());
}
assertNull(structure.getExcludeLinesPattern());
assertEquals("^\\d{1,2}/\\d{1,2}/\\d{4} \\d{1,2}:\\d{2}[AP]M\\b", structure.getMultilineStartPattern());
assertNull(structure.getDelimiter());
assertNull(structure.getQuote());
assertNull(structure.getHasHeaderRow());
assertNull(structure.getShouldTrimFields());
assertEquals("%{CUSTOM_TIMESTAMP:timestamp} %{LOGLEVEL:loglevel} .*", structure.getGrokPattern());
assertEquals("timestamp", structure.getTimestampField());
assertEquals(Collections.singletonList("M/d/YYYY h:mma"), structure.getJodaTimestampFormats());
FieldStats messageFieldStats = structure.getFieldStats().get("message");
assertNotNull(messageFieldStats);
for (String statMessage : messageFieldStats.getTopHits().stream().map(m -> (String) m.get("value")).collect(Collectors.toList())) {
assertThat(structureFinder.getSampleMessages(), hasItem(statMessage));
}
}
public void testCreateConfigsGivenElasticsearchLogAndTimestampFieldOverride() throws Exception { public void testCreateConfigsGivenElasticsearchLogAndTimestampFieldOverride() throws Exception {
FileStructureOverrides overrides = FileStructureOverrides.builder().setTimestampField("my_time").build(); FileStructureOverrides overrides = FileStructureOverrides.builder().setTimestampField("my_time").build();
@ -160,7 +115,7 @@ public class TextLogFileStructureFinderTests extends FileStructureTestCase {
assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker()); assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker());
} }
assertNull(structure.getExcludeLinesPattern()); assertNull(structure.getExcludeLinesPattern());
assertEquals("^\\[\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}", structure.getMultilineStartPattern()); assertEquals("^\\[\\b\\d{4}-\\d{2}-\\d{2}[T ]\\d{2}:\\d{2}", structure.getMultilineStartPattern());
assertNull(structure.getDelimiter()); assertNull(structure.getDelimiter());
assertNull(structure.getQuote()); assertNull(structure.getQuote());
assertNull(structure.getHasHeaderRow()); assertNull(structure.getHasHeaderRow());
@ -197,7 +152,7 @@ public class TextLogFileStructureFinderTests extends FileStructureTestCase {
assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker()); assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker());
} }
assertNull(structure.getExcludeLinesPattern()); assertNull(structure.getExcludeLinesPattern());
assertEquals("^\\[\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}", structure.getMultilineStartPattern()); assertEquals("^\\[\\b\\d{4}-\\d{2}-\\d{2}[T ]\\d{2}:\\d{2}", structure.getMultilineStartPattern());
assertNull(structure.getDelimiter()); assertNull(structure.getDelimiter());
assertNull(structure.getQuote()); assertNull(structure.getQuote());
assertNull(structure.getHasHeaderRow()); assertNull(structure.getHasHeaderRow());
@ -303,98 +258,4 @@ public class TextLogFileStructureFinderTests extends FileStructureTestCase {
TextLogFileStructureFinder.createMultiLineMessageStartRegex(prefaces, simpleDateRegex)); TextLogFileStructureFinder.createMultiLineMessageStartRegex(prefaces, simpleDateRegex));
} }
} }
public void testMostLikelyTimestampGivenAllSame() {
String sample = "[2018-06-27T11:59:22,125][INFO ][o.e.n.Node ] [node-0] initializing ...\n" +
"[2018-06-27T11:59:22,201][INFO ][o.e.e.NodeEnvironment ] [node-0] using [1] data paths, mounts [[/ (/dev/disk1)]], " +
"net usable_space [216.1gb], net total_space [464.7gb], types [hfs]\n" +
"[2018-06-27T11:59:22,202][INFO ][o.e.e.NodeEnvironment ] [node-0] heap size [494.9mb], " +
"compressed ordinary object pointers [true]\n" +
"[2018-06-27T11:59:22,204][INFO ][o.e.n.Node ] [node-0] node name [node-0], node ID [Ha1gD8nNSDqjd6PIyu3DJA]\n" +
"[2018-06-27T11:59:22,204][INFO ][o.e.n.Node ] [node-0] version[6.4.0-SNAPSHOT], pid[2785], " +
"build[default/zip/3c60efa/2018-06-26T14:55:15.206676Z], OS[Mac OS X/10.12.6/x86_64], " +
"JVM[\"Oracle Corporation\"/Java HotSpot(TM) 64-Bit Server VM/10/10+46]\n" +
"[2018-06-27T11:59:22,205][INFO ][o.e.n.Node ] [node-0] JVM arguments [-Xms1g, -Xmx1g, " +
"-XX:+UseConcMarkSweepGC, -XX:CMSInitiatingOccupancyFraction=75, -XX:+UseCMSInitiatingOccupancyOnly, " +
"-XX:+AlwaysPreTouch, -Xss1m, -Djava.awt.headless=true, -Dfile.encoding=UTF-8, -Djna.nosys=true, " +
"-XX:-OmitStackTraceInFastThrow, -Dio.netty.noUnsafe=true, -Dio.netty.noKeySetOptimization=true, " +
"-Dio.netty.recycler.maxCapacityPerThread=0, -Dlog4j.shutdownHookEnabled=false, -Dlog4j2.disable.jmx=true, " +
"-Djava.io.tmpdir=/var/folders/k5/5sqcdlps5sg3cvlp783gcz740000h0/T/elasticsearch.nFUyeMH1, " +
"-XX:+HeapDumpOnOutOfMemoryError, -XX:HeapDumpPath=data, -XX:ErrorFile=logs/hs_err_pid%p.log, " +
"-Xlog:gc*,gc+age=trace,safepoint:file=logs/gc.log:utctime,pid,tags:filecount=32,filesize=64m, " +
"-Djava.locale.providers=COMPAT, -Dio.netty.allocator.type=unpooled, -ea, -esa, -Xms512m, -Xmx512m, " +
"-Des.path.home=/Users/dave/elasticsearch/distribution/build/cluster/run node0/elasticsearch-6.4.0-SNAPSHOT, " +
"-Des.path.conf=/Users/dave/elasticsearch/distribution/build/cluster/run node0/elasticsearch-6.4.0-SNAPSHOT/config, " +
"-Des.distribution.flavor=default, -Des.distribution.type=zip]\n" +
"[2018-06-27T11:59:22,205][WARN ][o.e.n.Node ] [node-0] version [6.4.0-SNAPSHOT] is a pre-release version of " +
"Elasticsearch and is not suitable for production\n" +
"[2018-06-27T11:59:23,585][INFO ][o.e.p.PluginsService ] [node-0] loaded module [aggs-matrix-stats]\n" +
"[2018-06-27T11:59:23,586][INFO ][o.e.p.PluginsService ] [node-0] loaded module [analysis-common]\n" +
"[2018-06-27T11:59:23,586][INFO ][o.e.p.PluginsService ] [node-0] loaded module [ingest-common]\n" +
"[2018-06-27T11:59:23,586][INFO ][o.e.p.PluginsService ] [node-0] loaded module [lang-expression]\n" +
"[2018-06-27T11:59:23,586][INFO ][o.e.p.PluginsService ] [node-0] loaded module [lang-mustache]\n" +
"[2018-06-27T11:59:23,586][INFO ][o.e.p.PluginsService ] [node-0] loaded module [lang-painless]\n" +
"[2018-06-27T11:59:23,586][INFO ][o.e.p.PluginsService ] [node-0] loaded module [mapper-extras]\n" +
"[2018-06-27T11:59:23,586][INFO ][o.e.p.PluginsService ] [node-0] loaded module [parent-join]\n" +
"[2018-06-27T11:59:23,586][INFO ][o.e.p.PluginsService ] [node-0] loaded module [percolator]\n" +
"[2018-06-27T11:59:23,586][INFO ][o.e.p.PluginsService ] [node-0] loaded module [rank-eval]\n" +
"[2018-06-27T11:59:23,586][INFO ][o.e.p.PluginsService ] [node-0] loaded module [reindex]\n" +
"[2018-06-27T11:59:23,586][INFO ][o.e.p.PluginsService ] [node-0] loaded module [repository-url]\n" +
"[2018-06-27T11:59:23,587][INFO ][o.e.p.PluginsService ] [node-0] loaded module [transport-netty4]\n" +
"[2018-06-27T11:59:23,587][INFO ][o.e.p.PluginsService ] [node-0] loaded module [x-pack-core]\n" +
"[2018-06-27T11:59:23,587][INFO ][o.e.p.PluginsService ] [node-0] loaded module [x-pack-deprecation]\n" +
"[2018-06-27T11:59:23,587][INFO ][o.e.p.PluginsService ] [node-0] loaded module [x-pack-graph]\n" +
"[2018-06-27T11:59:23,587][INFO ][o.e.p.PluginsService ] [node-0] loaded module [x-pack-logstash]\n" +
"[2018-06-27T11:59:23,587][INFO ][o.e.p.PluginsService ] [node-0] loaded module [x-pack-ml]\n" +
"[2018-06-27T11:59:23,587][INFO ][o.e.p.PluginsService ] [node-0] loaded module [x-pack-monitoring]\n" +
"[2018-06-27T11:59:23,587][INFO ][o.e.p.PluginsService ] [node-0] loaded module [x-pack-rollup]\n" +
"[2018-06-27T11:59:23,587][INFO ][o.e.p.PluginsService ] [node-0] loaded module [x-pack-security]\n" +
"[2018-06-27T11:59:23,587][INFO ][o.e.p.PluginsService ] [node-0] loaded module [x-pack-sql]\n" +
"[2018-06-27T11:59:23,588][INFO ][o.e.p.PluginsService ] [node-0] loaded module [x-pack-upgrade]\n" +
"[2018-06-27T11:59:23,588][INFO ][o.e.p.PluginsService ] [node-0] loaded module [x-pack-watcher]\n" +
"[2018-06-27T11:59:23,588][INFO ][o.e.p.PluginsService ] [node-0] no plugins loaded\n";
Tuple<TimestampMatch, Set<String>> mostLikelyMatch =
TextLogFileStructureFinder.mostLikelyTimestamp(sample.split("\n"), FileStructureOverrides.EMPTY_OVERRIDES,
NOOP_TIMEOUT_CHECKER);
assertNotNull(mostLikelyMatch);
assertEquals(new TimestampMatch(9, "", "ISO8601", "yyyy-MM-dd'T'HH:mm:ss,SSS",
"\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}", "TIMESTAMP_ISO8601", ""), mostLikelyMatch.v1());
}
public void testMostLikelyTimestampGivenExceptionTrace() {
Tuple<TimestampMatch, Set<String>> mostLikelyMatch =
TextLogFileStructureFinder.mostLikelyTimestamp(EXCEPTION_TRACE_SAMPLE.split("\n"), FileStructureOverrides.EMPTY_OVERRIDES,
NOOP_TIMEOUT_CHECKER);
assertNotNull(mostLikelyMatch);
// Even though many lines have a timestamp near the end (in the Lucene version information),
// these are so far along the lines that the weight of the timestamp near the beginning of the
// first line should take precedence
assertEquals(new TimestampMatch(9, "", "ISO8601", "yyyy-MM-dd'T'HH:mm:ss,SSS",
"\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}", "TIMESTAMP_ISO8601", ""), mostLikelyMatch.v1());
}
public void testMostLikelyTimestampGivenExceptionTraceAndTimestampFormatOverride() {
FileStructureOverrides overrides = FileStructureOverrides.builder().setTimestampFormat("yyyy-MM-dd HH:mm:ss").build();
Tuple<TimestampMatch, Set<String>> mostLikelyMatch =
TextLogFileStructureFinder.mostLikelyTimestamp(EXCEPTION_TRACE_SAMPLE.split("\n"), overrides, NOOP_TIMEOUT_CHECKER);
assertNotNull(mostLikelyMatch);
// The override should force the seemingly inferior choice of timestamp
assertEquals(new TimestampMatch(6, "", "YYYY-MM-dd HH:mm:ss", "yyyy-MM-dd HH:mm:ss", "\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}",
"TIMESTAMP_ISO8601", ""), mostLikelyMatch.v1());
}
public void testMostLikelyTimestampGivenExceptionTraceAndImpossibleTimestampFormatOverride() {
FileStructureOverrides overrides = FileStructureOverrides.builder().setTimestampFormat("MMM dd HH:mm:ss").build();
Tuple<TimestampMatch, Set<String>> mostLikelyMatch =
TextLogFileStructureFinder.mostLikelyTimestamp(EXCEPTION_TRACE_SAMPLE.split("\n"), overrides, NOOP_TIMEOUT_CHECKER);
assertNull(mostLikelyMatch);
}
} }