[ML] Return both Joda and Java formats from structure finder (#33900)

Previously the timestamp_formats field in the response
from the find_file_structure endpoint contained Joda
timestamp formats.  This change makes that clear by
renaming the field to joda_timestamp_formats, and also
adds a java_timestamp_formats field containing the
equivalent Java time format strings.
This commit is contained in:
David Roberts 2018-09-25 12:52:51 +01:00 committed by GitHub
parent 612201aee0
commit dfe5af0411
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 1254 additions and 233 deletions

View File

@ -159,7 +159,7 @@ format corresponds to the primary timestamp, but you do not want to specify the
full `grok_pattern`.
If this parameter is not specified, the structure finder chooses the best format from
the formats it knows, which are:
the formats it knows, which are these Joda formats and their Java time equivalents:
* `dd/MMM/YYYY:HH:mm:ss Z`
* `EEE MMM dd HH:mm zzz YYYY`
@ -178,7 +178,7 @@ the formats it knows, which are:
* `MMM dd HH:mm:ss`
* `MMM dd HH:mm:ss,SSS`
* `MMM dd YYYY HH:mm:ss`
* `MMM dd, YYYY K:mm:ss a`
* `MMM dd, YYYY h:mm:ss a`
* `TAI64N`
* `UNIX`
* `UNIX_MS`
@ -487,3 +487,791 @@ If the request does not encounter errors, you receive the following result:
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
City yellow cab trip data. The first `curl` command downloads the data, the
first 20000 lines of which are then piped into the `find_file_structure`
endpoint. The `lines_to_sample` query parameter of the endpoint is set to 20000
to match what is specified in the `head` command.
[source,js]
----
curl -s "s3.amazonaws.com/nyc-tlc/trip+data/yellow_tripdata_2018-06.csv" | head -20000 | curl -s -H "Content-Type: application/json" -XPOST "localhost:9200/_xpack/ml/find_file_structure?pretty&lines_to_sample=20000" -T -
----
// NOTCONSOLE
// Not converting to console because this shows how curl can be used
--
NOTE: The `Content-Type: application/json` header must be set even though in
this case the data is not JSON. (Alternatively the `Content-Type` can be set
to any other supported by Elasticsearch, but it must be set.)
--
If the request does not encounter errors, you receive the following result:
[source,js]
----
{
"num_lines_analyzed" : 20000,
"num_messages_analyzed" : 19998, <1>
"sample_start" : "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\n\n1,2018-06-01 00:15:40,2018-06-01 00:16:46,1,.00,1,N,145,145,2,3,0.5,0.5,0,0,0.3,4.3\n",
"charset" : "UTF-8",
"has_byte_order_marker" : false,
"format" : "delimited", <2>
"multiline_start_pattern" : "^.*?,\"?\\d{4}-\\d{2}-\\d{2} \\d{2}:\\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\"?",
"column_names" : [ <3>
"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"
],
"has_header_row" : true, <4>
"delimiter" : ",", <5>
"quote" : "\"", <6>
"timestamp_field" : "tpep_pickup_datetime", <7>
"joda_timestamp_formats" : [ <8>
"YYYY-MM-dd HH:mm:ss"
],
"java_timestamp_formats" : [ <9>
"yyyy-MM-dd HH:mm:ss"
],
"need_client_timezone" : true, <10>
"mappings" : {
"@timestamp" : {
"type" : "date"
},
"DOLocationID" : {
"type" : "long"
},
"PULocationID" : {
"type" : "long"
},
"RatecodeID" : {
"type" : "long"
},
"VendorID" : {
"type" : "long"
},
"extra" : {
"type" : "double"
},
"fare_amount" : {
"type" : "double"
},
"improvement_surcharge" : {
"type" : "double"
},
"mta_tax" : {
"type" : "double"
},
"passenger_count" : {
"type" : "long"
},
"payment_type" : {
"type" : "long"
},
"store_and_fwd_flag" : {
"type" : "keyword"
},
"tip_amount" : {
"type" : "double"
},
"tolls_amount" : {
"type" : "double"
},
"total_amount" : {
"type" : "double"
},
"tpep_dropoff_datetime" : {
"type" : "date",
"format" : "YYYY-MM-dd HH:mm:ss"
},
"tpep_pickup_datetime" : {
"type" : "date",
"format" : "YYYY-MM-dd HH:mm:ss"
},
"trip_distance" : {
"type" : "double"
}
},
"field_stats" : {
"DOLocationID" : {
"count" : 19998,
"cardinality" : 240,
"min_value" : 1,
"max_value" : 265,
"mean_value" : 150.26532653265312,
"median_value" : 148,
"top_hits" : [
{
"value" : 79,
"count" : 760
},
{
"value" : 48,
"count" : 683
},
{
"value" : 68,
"count" : 529
},
{
"value" : 170,
"count" : 506
},
{
"value" : 107,
"count" : 468
},
{
"value" : 249,
"count" : 457
},
{
"value" : 230,
"count" : 441
},
{
"value" : 186,
"count" : 432
},
{
"value" : 141,
"count" : 409
},
{
"value" : 263,
"count" : 386
}
]
},
"PULocationID" : {
"count" : 19998,
"cardinality" : 154,
"min_value" : 1,
"max_value" : 265,
"mean_value" : 153.4042404240424,
"median_value" : 148,
"top_hits" : [
{
"value" : 79,
"count" : 1067
},
{
"value" : 230,
"count" : 949
},
{
"value" : 148,
"count" : 940
},
{
"value" : 132,
"count" : 897
},
{
"value" : 48,
"count" : 853
},
{
"value" : 161,
"count" : 820
},
{
"value" : 234,
"count" : 750
},
{
"value" : 249,
"count" : 722
},
{
"value" : 164,
"count" : 663
},
{
"value" : 114,
"count" : 646
}
]
},
"RatecodeID" : {
"count" : 19998,
"cardinality" : 5,
"min_value" : 1,
"max_value" : 5,
"mean_value" : 1.0656565656565653,
"median_value" : 1,
"top_hits" : [
{
"value" : 1,
"count" : 19311
},
{
"value" : 2,
"count" : 468
},
{
"value" : 5,
"count" : 195
},
{
"value" : 4,
"count" : 17
},
{
"value" : 3,
"count" : 7
}
]
},
"VendorID" : {
"count" : 19998,
"cardinality" : 2,
"min_value" : 1,
"max_value" : 2,
"mean_value" : 1.59005900590059,
"median_value" : 2,
"top_hits" : [
{
"value" : 2,
"count" : 11800
},
{
"value" : 1,
"count" : 8198
}
]
},
"extra" : {
"count" : 19998,
"cardinality" : 3,
"min_value" : -0.5,
"max_value" : 0.5,
"mean_value" : 0.4815981598159816,
"median_value" : 0.5,
"top_hits" : [
{
"value" : 0.5,
"count" : 19281
},
{
"value" : 0,
"count" : 698
},
{
"value" : -0.5,
"count" : 19
}
]
},
"fare_amount" : {
"count" : 19998,
"cardinality" : 208,
"min_value" : -100,
"max_value" : 300,
"mean_value" : 13.937719771977209,
"median_value" : 9.5,
"top_hits" : [
{
"value" : 6,
"count" : 1004
},
{
"value" : 6.5,
"count" : 935
},
{
"value" : 5.5,
"count" : 909
},
{
"value" : 7,
"count" : 903
},
{
"value" : 5,
"count" : 889
},
{
"value" : 7.5,
"count" : 854
},
{
"value" : 4.5,
"count" : 802
},
{
"value" : 8.5,
"count" : 790
},
{
"value" : 8,
"count" : 789
},
{
"value" : 9,
"count" : 711
}
]
},
"improvement_surcharge" : {
"count" : 19998,
"cardinality" : 3,
"min_value" : -0.3,
"max_value" : 0.3,
"mean_value" : 0.29915991599159913,
"median_value" : 0.3,
"top_hits" : [
{
"value" : 0.3,
"count" : 19964
},
{
"value" : -0.3,
"count" : 22
},
{
"value" : 0,
"count" : 12
}
]
},
"mta_tax" : {
"count" : 19998,
"cardinality" : 3,
"min_value" : -0.5,
"max_value" : 0.5,
"mean_value" : 0.4962246224622462,
"median_value" : 0.5,
"top_hits" : [
{
"value" : 0.5,
"count" : 19868
},
{
"value" : 0,
"count" : 109
},
{
"value" : -0.5,
"count" : 21
}
]
},
"passenger_count" : {
"count" : 19998,
"cardinality" : 7,
"min_value" : 0,
"max_value" : 6,
"mean_value" : 1.6201620162016201,
"median_value" : 1,
"top_hits" : [
{
"value" : 1,
"count" : 14219
},
{
"value" : 2,
"count" : 2886
},
{
"value" : 5,
"count" : 1047
},
{
"value" : 3,
"count" : 804
},
{
"value" : 6,
"count" : 523
},
{
"value" : 4,
"count" : 406
},
{
"value" : 0,
"count" : 113
}
]
},
"payment_type" : {
"count" : 19998,
"cardinality" : 4,
"min_value" : 1,
"max_value" : 4,
"mean_value" : 1.315631563156316,
"median_value" : 1,
"top_hits" : [
{
"value" : 1,
"count" : 13936
},
{
"value" : 2,
"count" : 5857
},
{
"value" : 3,
"count" : 160
},
{
"value" : 4,
"count" : 45
}
]
},
"store_and_fwd_flag" : {
"count" : 19998,
"cardinality" : 2,
"top_hits" : [
{
"value" : "N",
"count" : 19910
},
{
"value" : "Y",
"count" : 88
}
]
},
"tip_amount" : {
"count" : 19998,
"cardinality" : 717,
"min_value" : 0,
"max_value" : 128,
"mean_value" : 2.010959095909593,
"median_value" : 1.45,
"top_hits" : [
{
"value" : 0,
"count" : 6917
},
{
"value" : 1,
"count" : 1178
},
{
"value" : 2,
"count" : 624
},
{
"value" : 3,
"count" : 248
},
{
"value" : 1.56,
"count" : 206
},
{
"value" : 1.46,
"count" : 205
},
{
"value" : 1.76,
"count" : 196
},
{
"value" : 1.45,
"count" : 195
},
{
"value" : 1.36,
"count" : 191
},
{
"value" : 1.5,
"count" : 187
}
]
},
"tolls_amount" : {
"count" : 19998,
"cardinality" : 26,
"min_value" : 0,
"max_value" : 35,
"mean_value" : 0.2729697969796978,
"median_value" : 0,
"top_hits" : [
{
"value" : 0,
"count" : 19107
},
{
"value" : 5.76,
"count" : 791
},
{
"value" : 10.5,
"count" : 36
},
{
"value" : 2.64,
"count" : 21
},
{
"value" : 11.52,
"count" : 8
},
{
"value" : 5.54,
"count" : 4
},
{
"value" : 8.5,
"count" : 4
},
{
"value" : 17.28,
"count" : 4
},
{
"value" : 2,
"count" : 2
},
{
"value" : 2.16,
"count" : 2
}
]
},
"total_amount" : {
"count" : 19998,
"cardinality" : 1267,
"min_value" : -100.3,
"max_value" : 389.12,
"mean_value" : 17.499898989898995,
"median_value" : 12.35,
"top_hits" : [
{
"value" : 7.3,
"count" : 478
},
{
"value" : 8.3,
"count" : 443
},
{
"value" : 8.8,
"count" : 420
},
{
"value" : 6.8,
"count" : 406
},
{
"value" : 7.8,
"count" : 405
},
{
"value" : 6.3,
"count" : 371
},
{
"value" : 9.8,
"count" : 368
},
{
"value" : 5.8,
"count" : 362
},
{
"value" : 9.3,
"count" : 332
},
{
"value" : 10.3,
"count" : 332
}
]
},
"tpep_dropoff_datetime" : {
"count" : 19998,
"cardinality" : 9066,
"top_hits" : [
{
"value" : "2018-06-01 01:12:12",
"count" : 10
},
{
"value" : "2018-06-01 00:32:15",
"count" : 9
},
{
"value" : "2018-06-01 00:44:27",
"count" : 9
},
{
"value" : "2018-06-01 00:46:42",
"count" : 9
},
{
"value" : "2018-06-01 01:03:22",
"count" : 9
},
{
"value" : "2018-06-01 01:05:13",
"count" : 9
},
{
"value" : "2018-06-01 00:11:20",
"count" : 8
},
{
"value" : "2018-06-01 00:16:03",
"count" : 8
},
{
"value" : "2018-06-01 00:19:47",
"count" : 8
},
{
"value" : "2018-06-01 00:25:17",
"count" : 8
}
]
},
"tpep_pickup_datetime" : {
"count" : 19998,
"cardinality" : 8760,
"top_hits" : [
{
"value" : "2018-06-01 00:01:23",
"count" : 12
},
{
"value" : "2018-06-01 00:04:31",
"count" : 10
},
{
"value" : "2018-06-01 00:05:38",
"count" : 10
},
{
"value" : "2018-06-01 00:09:50",
"count" : 10
},
{
"value" : "2018-06-01 00:12:01",
"count" : 10
},
{
"value" : "2018-06-01 00:14:17",
"count" : 10
},
{
"value" : "2018-06-01 00:00:34",
"count" : 9
},
{
"value" : "2018-06-01 00:00:40",
"count" : 9
},
{
"value" : "2018-06-01 00:02:53",
"count" : 9
},
{
"value" : "2018-06-01 00:05:40",
"count" : 9
}
]
},
"trip_distance" : {
"count" : 19998,
"cardinality" : 1687,
"min_value" : 0,
"max_value" : 64.63,
"mean_value" : 3.6521062106210715,
"median_value" : 2.16,
"top_hits" : [
{
"value" : 0.9,
"count" : 335
},
{
"value" : 0.8,
"count" : 320
},
{
"value" : 1.1,
"count" : 316
},
{
"value" : 0.7,
"count" : 304
},
{
"value" : 1.2,
"count" : 303
},
{
"value" : 1,
"count" : 296
},
{
"value" : 1.3,
"count" : 280
},
{
"value" : 1.5,
"count" : 268
},
{
"value" : 1.6,
"count" : 268
},
{
"value" : 0.6,
"count" : 256
}
]
}
}
}
----
// NOTCONSOLE
<1> `num_messages_analyzed` is 2 lower than `num_lines_analyzed` because only
data records count as messages. The first line contains the column names
and in this sample the second line is blank.
<2> Unlike the first example, in this case the `format` has been identified as
`delimited`.
<3> Because the `format` is `delimited`, the `column_names` field in the output
lists the column names in the order they appear in the sample.
<4> `has_header_row` indicates that for this sample the column names were in
the first row of the sample. (If they hadn't been then it would have been
a good idea to specify them in the `column_names` query parameter.)
<5> The `delimiter` for this sample is a comma, as it's a CSV file.
<6> The `quote` character is the default double quote. (The structure finder
does not attempt to deduce any other quote character, so if you have a
delimited file that's quoted with some other character you must specify it
using the `quote` query parameter.)
<7> The `timestamp_field` has been chosen to be `tpep_pickup_datetime`.
`tpep_dropoff_datetime` would work just as well, but `tpep_pickup_datetime`
was chosen because it comes first in the column order. If you prefer
`tpep_dropoff_datetime` then force it to be chosen using the
`timestamp_field` query parameter.
<8> `joda_timestamp_formats` are used to tell Logstash and Ingest pipeline how
to parse timestamps.
<9> `java_timestamp_formats` are the Java time formats recognized in the time
fields. In future Ingest pipeline will switch to use this format.
<10> The timestamp format in this sample doesn't specify a timezone, so to
accurately convert them to UTC timestamps to store in Elasticsearch it's
necessary to supply the timezone they relate to. `need_client_timezone`
will be `false` for timestamp formats that include the timezone.

View File

@ -99,7 +99,8 @@ public class FileStructure implements ToXContentObject, Writeable {
public static final ParseField SHOULD_TRIM_FIELDS = new ParseField("should_trim_fields");
public static final ParseField GROK_PATTERN = new ParseField("grok_pattern");
public static final ParseField TIMESTAMP_FIELD = new ParseField("timestamp_field");
public static final ParseField TIMESTAMP_FORMATS = new ParseField("timestamp_formats");
public static final ParseField JODA_TIMESTAMP_FORMATS = new ParseField("joda_timestamp_formats");
public static final ParseField JAVA_TIMESTAMP_FORMATS = new ParseField("java_timestamp_formats");
public static final ParseField NEED_CLIENT_TIMEZONE = new ParseField("need_client_timezone");
public static final ParseField MAPPINGS = new ParseField("mappings");
public static final ParseField FIELD_STATS = new ParseField("field_stats");
@ -123,7 +124,8 @@ public class FileStructure implements ToXContentObject, Writeable {
PARSER.declareBoolean(Builder::setShouldTrimFields, SHOULD_TRIM_FIELDS);
PARSER.declareString(Builder::setGrokPattern, GROK_PATTERN);
PARSER.declareString(Builder::setTimestampField, TIMESTAMP_FIELD);
PARSER.declareStringArray(Builder::setTimestampFormats, TIMESTAMP_FORMATS);
PARSER.declareStringArray(Builder::setJodaTimestampFormats, JODA_TIMESTAMP_FORMATS);
PARSER.declareStringArray(Builder::setJavaTimestampFormats, JAVA_TIMESTAMP_FORMATS);
PARSER.declareBoolean(Builder::setNeedClientTimezone, NEED_CLIENT_TIMEZONE);
PARSER.declareObject(Builder::setMappings, (p, c) -> new TreeMap<>(p.map()), MAPPINGS);
PARSER.declareObject(Builder::setFieldStats, (p, c) -> {
@ -150,7 +152,8 @@ public class FileStructure implements ToXContentObject, Writeable {
private final Character quote;
private final Boolean shouldTrimFields;
private final String grokPattern;
private final List<String> timestampFormats;
private final List<String> jodaTimestampFormats;
private final List<String> javaTimestampFormats;
private final String timestampField;
private final boolean needClientTimezone;
private final SortedMap<String, Object> mappings;
@ -160,8 +163,9 @@ public class FileStructure implements ToXContentObject, Writeable {
public FileStructure(int numLinesAnalyzed, int numMessagesAnalyzed, String sampleStart, String charset, Boolean hasByteOrderMarker,
Format format, String multilineStartPattern, String excludeLinesPattern, List<String> columnNames,
Boolean hasHeaderRow, Character delimiter, Character quote, Boolean shouldTrimFields, String grokPattern,
String timestampField, List<String> timestampFormats, boolean needClientTimezone, Map<String, Object> mappings,
Map<String, FieldStats> fieldStats, List<String> explanation) {
String timestampField, List<String> jodaTimestampFormats, List<String> javaTimestampFormats,
boolean needClientTimezone, Map<String, Object> mappings, Map<String, FieldStats> fieldStats,
List<String> explanation) {
this.numLinesAnalyzed = numLinesAnalyzed;
this.numMessagesAnalyzed = numMessagesAnalyzed;
@ -178,7 +182,10 @@ public class FileStructure implements ToXContentObject, Writeable {
this.shouldTrimFields = shouldTrimFields;
this.grokPattern = grokPattern;
this.timestampField = timestampField;
this.timestampFormats = (timestampFormats == null) ? null : Collections.unmodifiableList(new ArrayList<>(timestampFormats));
this.jodaTimestampFormats =
(jodaTimestampFormats == null) ? null : Collections.unmodifiableList(new ArrayList<>(jodaTimestampFormats));
this.javaTimestampFormats =
(javaTimestampFormats == null) ? null : Collections.unmodifiableList(new ArrayList<>(javaTimestampFormats));
this.needClientTimezone = needClientTimezone;
this.mappings = Collections.unmodifiableSortedMap(new TreeMap<>(mappings));
this.fieldStats = Collections.unmodifiableSortedMap(new TreeMap<>(fieldStats));
@ -200,7 +207,8 @@ public class FileStructure implements ToXContentObject, Writeable {
quote = in.readBoolean() ? (char) in.readVInt() : null;
shouldTrimFields = in.readOptionalBoolean();
grokPattern = in.readOptionalString();
timestampFormats = in.readBoolean() ? Collections.unmodifiableList(in.readList(StreamInput::readString)) : null;
jodaTimestampFormats = in.readBoolean() ? Collections.unmodifiableList(in.readList(StreamInput::readString)) : null;
javaTimestampFormats = in.readBoolean() ? Collections.unmodifiableList(in.readList(StreamInput::readString)) : null;
timestampField = in.readOptionalString();
needClientTimezone = in.readBoolean();
mappings = Collections.unmodifiableSortedMap(new TreeMap<>(in.readMap()));
@ -239,11 +247,17 @@ public class FileStructure implements ToXContentObject, Writeable {
}
out.writeOptionalBoolean(shouldTrimFields);
out.writeOptionalString(grokPattern);
if (timestampFormats == null) {
if (jodaTimestampFormats == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeCollection(timestampFormats, StreamOutput::writeString);
out.writeCollection(jodaTimestampFormats, StreamOutput::writeString);
}
if (javaTimestampFormats == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeCollection(javaTimestampFormats, StreamOutput::writeString);
}
out.writeOptionalString(timestampField);
out.writeBoolean(needClientTimezone);
@ -312,8 +326,12 @@ public class FileStructure implements ToXContentObject, Writeable {
return timestampField;
}
public List<String> getTimestampFormats() {
return timestampFormats;
public List<String> getJodaTimestampFormats() {
return jodaTimestampFormats;
}
public List<String> getJavaTimestampFormats() {
return javaTimestampFormats;
}
public boolean needClientTimezone() {
@ -371,8 +389,11 @@ public class FileStructure implements ToXContentObject, Writeable {
if (timestampField != null && timestampField.isEmpty() == false) {
builder.field(TIMESTAMP_FIELD.getPreferredName(), timestampField);
}
if (timestampFormats != null && timestampFormats.isEmpty() == false) {
builder.field(TIMESTAMP_FORMATS.getPreferredName(), timestampFormats);
if (jodaTimestampFormats != null && jodaTimestampFormats.isEmpty() == false) {
builder.field(JODA_TIMESTAMP_FORMATS.getPreferredName(), jodaTimestampFormats);
}
if (javaTimestampFormats != null && javaTimestampFormats.isEmpty() == false) {
builder.field(JAVA_TIMESTAMP_FORMATS.getPreferredName(), javaTimestampFormats);
}
builder.field(NEED_CLIENT_TIMEZONE.getPreferredName(), needClientTimezone);
builder.field(MAPPINGS.getPreferredName(), mappings);
@ -396,7 +417,7 @@ public class FileStructure implements ToXContentObject, Writeable {
return Objects.hash(numLinesAnalyzed, numMessagesAnalyzed, sampleStart, charset, hasByteOrderMarker, format,
multilineStartPattern, excludeLinesPattern, columnNames, hasHeaderRow, delimiter, quote, shouldTrimFields, grokPattern,
timestampField, timestampFormats, needClientTimezone, mappings, fieldStats, explanation);
timestampField, jodaTimestampFormats, javaTimestampFormats, needClientTimezone, mappings, fieldStats, explanation);
}
@Override
@ -413,7 +434,6 @@ public class FileStructure implements ToXContentObject, Writeable {
FileStructure that = (FileStructure) other;
return this.numLinesAnalyzed == that.numLinesAnalyzed &&
this.numMessagesAnalyzed == that.numMessagesAnalyzed &&
this.needClientTimezone == that.needClientTimezone &&
Objects.equals(this.sampleStart, that.sampleStart) &&
Objects.equals(this.charset, that.charset) &&
Objects.equals(this.hasByteOrderMarker, that.hasByteOrderMarker) &&
@ -427,7 +447,9 @@ public class FileStructure implements ToXContentObject, Writeable {
Objects.equals(this.shouldTrimFields, that.shouldTrimFields) &&
Objects.equals(this.grokPattern, that.grokPattern) &&
Objects.equals(this.timestampField, that.timestampField) &&
Objects.equals(this.timestampFormats, that.timestampFormats) &&
Objects.equals(this.jodaTimestampFormats, that.jodaTimestampFormats) &&
Objects.equals(this.javaTimestampFormats, that.javaTimestampFormats) &&
this.needClientTimezone == that.needClientTimezone &&
Objects.equals(this.mappings, that.mappings) &&
Objects.equals(this.fieldStats, that.fieldStats) &&
Objects.equals(this.explanation, that.explanation);
@ -450,7 +472,8 @@ public class FileStructure implements ToXContentObject, Writeable {
private Boolean shouldTrimFields;
private String grokPattern;
private String timestampField;
private List<String> timestampFormats;
private List<String> jodaTimestampFormats;
private List<String> javaTimestampFormats;
private boolean needClientTimezone;
private Map<String, Object> mappings;
private Map<String, FieldStats> fieldStats = Collections.emptyMap();
@ -539,8 +562,13 @@ public class FileStructure implements ToXContentObject, Writeable {
return this;
}
public Builder setTimestampFormats(List<String> timestampFormats) {
this.timestampFormats = timestampFormats;
public Builder setJodaTimestampFormats(List<String> jodaTimestampFormats) {
this.jodaTimestampFormats = jodaTimestampFormats;
return this;
}
public Builder setJavaTimestampFormats(List<String> javaTimestampFormats) {
this.javaTimestampFormats = javaTimestampFormats;
return this;
}
@ -652,11 +680,21 @@ public class FileStructure implements ToXContentObject, Writeable {
throw new IllegalStateException("enum value [" + format + "] missing from switch.");
}
if ((timestampField == null) != (timestampFormats == null || timestampFormats.isEmpty())) {
throw new IllegalArgumentException("Timestamp field and timestamp formats must both be specified or neither be specified.");
boolean isTimestampFieldSpecified = (timestampField != null);
boolean isJodaTimestampFormatsSpecified = (jodaTimestampFormats != null && jodaTimestampFormats.isEmpty() == false);
boolean isJavaTimestampFormatsSpecified = (javaTimestampFormats != null && javaTimestampFormats.isEmpty() == false);
if (isTimestampFieldSpecified != isJodaTimestampFormatsSpecified) {
throw new IllegalArgumentException(
"Timestamp field and Joda timestamp formats must both be specified or neither be specified.");
}
if (needClientTimezone && timestampField == null) {
if (isTimestampFieldSpecified != isJavaTimestampFormatsSpecified) {
throw new IllegalArgumentException(
"Timestamp field and Java timestamp formats must both be specified or neither be specified.");
}
if (needClientTimezone && isTimestampFieldSpecified == false) {
throw new IllegalArgumentException("Client timezone cannot be needed if there is no timestamp field.");
}
@ -670,7 +708,7 @@ public class FileStructure implements ToXContentObject, Writeable {
return new FileStructure(numLinesAnalyzed, numMessagesAnalyzed, sampleStart, charset, hasByteOrderMarker, format,
multilineStartPattern, excludeLinesPattern, columnNames, hasHeaderRow, delimiter, quote, shouldTrimFields, grokPattern,
timestampField, timestampFormats, needClientTimezone, mappings, fieldStats, explanation);
timestampField, jodaTimestampFormats, javaTimestampFormats, needClientTimezone, mappings, fieldStats, explanation);
}
}
}

View File

@ -63,7 +63,8 @@ public class FileStructureTests extends AbstractSerializingTestCase<FileStructur
if (format == FileStructure.Format.SEMI_STRUCTURED_TEXT || randomBoolean()) {
builder.setTimestampField(randomAlphaOfLength(10));
builder.setTimestampFormats(Arrays.asList(generateRandomStringArray(3, 20, false, false)));
builder.setJodaTimestampFormats(Arrays.asList(generateRandomStringArray(3, 20, false, false)));
builder.setJavaTimestampFormats(Arrays.asList(generateRandomStringArray(3, 20, false, false)));
builder.setNeedClientTimezone(randomBoolean());
}

View File

@ -141,7 +141,8 @@ public class DelimitedFileStructureFinder implements FileStructureFinder {
}
structureBuilder.setTimestampField(timeField.v1())
.setTimestampFormats(timeField.v2().dateFormats)
.setJodaTimestampFormats(timeField.v2().jodaTimestampFormats)
.setJavaTimestampFormats(timeField.v2().javaTimestampFormats)
.setNeedClientTimezone(timeField.v2().hasTimezoneDependentParsing())
.setMultilineStartPattern(timeLineRegex);
}

View File

@ -240,7 +240,7 @@ public final class FileStructureUtils {
Iterator<String> iter = fieldValues.iterator();
TimestampMatch timestampMatch = TimestampFormatFinder.findFirstFullMatch(iter.next());
while (timestampMatch != null && iter.hasNext()) {
// To be mapped as type date all the values must match the same date format - it is
// To be mapped as type date all the values must match the same timestamp format - it is
// not acceptable for all values to be dates, but with different formats
if (timestampMatch.equals(TimestampFormatFinder.findFirstFullMatch(iter.next(), timestampMatch.candidateIndex)) == false) {
timestampMatch = null;

View File

@ -55,7 +55,8 @@ public class JsonFileStructureFinder implements FileStructureFinder {
Tuple<String, TimestampMatch> timeField = FileStructureUtils.guessTimestampField(explanation, sampleRecords, overrides);
if (timeField != null) {
structureBuilder.setTimestampField(timeField.v1())
.setTimestampFormats(timeField.v2().dateFormats)
.setJodaTimestampFormats(timeField.v2().jodaTimestampFormats)
.setJavaTimestampFormats(timeField.v2().javaTimestampFormats)
.setNeedClientTimezone(timeField.v2().hasTimezoneDependentParsing());
}

View File

@ -112,7 +112,8 @@ public class TextLogFileStructureFinder implements FileStructureFinder {
FileStructure structure = structureBuilder
.setTimestampField(interimTimestampField)
.setTimestampFormats(bestTimestamp.v1().dateFormats)
.setJodaTimestampFormats(bestTimestamp.v1().jodaTimestampFormats)
.setJavaTimestampFormats(bestTimestamp.v1().javaTimestampFormats)
.setNeedClientTimezone(bestTimestamp.v1().hasTimezoneDependentParsing())
.setGrokPattern(grokPattern)
.setMappings(mappings)
@ -147,8 +148,8 @@ public class TextLogFileStructureFinder implements FileStructureFinder {
for (String sampleLine : sampleLines) {
TimestampMatch match = TimestampFormatFinder.findFirstMatch(sampleLine, overrides.getTimestampFormat());
if (match != null) {
TimestampMatch pureMatch = new TimestampMatch(match.candidateIndex, "", match.dateFormats, match.simplePattern,
match.grokPatternName, "");
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)));

View File

@ -12,7 +12,6 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.regex.Matcher;
@ -30,8 +29,12 @@ public final class TimestampFormatFinder {
private static final String PREFACE = "preface";
private static final String EPILOGUE = "epilogue";
private static final Pattern FRACTIONAL_SECOND_INTERPRETER = Pattern.compile("([:.,])(\\d{3,9})");
private static final String FRACTIONAL_SECOND_SEPARATORS = ":.,";
private static final Pattern FRACTIONAL_SECOND_INTERPRETER = Pattern.compile("([" + FRACTIONAL_SECOND_SEPARATORS + "])(\\d{3,9})");
private static final char DEFAULT_FRACTIONAL_SECOND_SEPARATOR = ',';
private static final Pattern FRACTIONAL_SECOND_TIMESTAMP_FORMAT_PATTERN =
Pattern.compile("([" + FRACTIONAL_SECOND_SEPARATORS + "]S{3,9})");
private static final String DEFAULT_FRACTIONAL_SECOND_FORMAT = DEFAULT_FRACTIONAL_SECOND_SEPARATOR + "SSS";
/**
* The timestamp patterns are complex and it can be slow to prove they do not
@ -48,7 +51,9 @@ public final class TimestampFormatFinder {
// The end of some number (likely year or day) followed by a space then HH:mm
Pattern.compile("\\d \\d{2}:\\d{2}\\b"),
// HH:mm:ss surrounded by spaces
Pattern.compile(" \\d{2}:\\d{2}:\\d{2} ")
Pattern.compile(" \\d{2}:\\d{2}:\\d{2} "),
// Literal 'T' surrounded by numbers
Pattern.compile("\\dT\\d")
);
/**
@ -59,81 +64,107 @@ public final class TimestampFormatFinder {
// The TOMCAT_DATESTAMP format has to come before ISO8601 because it's basically ISO8601 but
// with a space before the timezone, and because the timezone is optional in ISO8601 it will
// be recognised as that with the timezone missed off if ISO8601 is checked first
new CandidateTimestampFormat("YYYY-MM-dd HH:mm:ss,SSS Z", "\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2},\\d{3}",
new CandidateTimestampFormat("YYYY-MM-dd HH:mm:ss,SSS Z", "yyyy-MM-dd HH:mm:ss,SSS XX",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2},\\d{3}",
"\\b20\\d{2}-%{MONTHNUM}-%{MONTHDAY} %{HOUR}:?%{MINUTE}:(?:[0-5][0-9]|60)[:.,][0-9]{3,9} (?:Z|[+-]%{HOUR}%{MINUTE})\\b",
"TOMCAT_DATESTAMP", Arrays.asList(0, 1)),
// The Elasticsearch ISO8601 parser requires a literal T between the date and time, so
// longhand formats are needed if there's a space instead
new CandidateTimestampFormat("YYYY-MM-dd HH:mm:ss,SSSZ", "\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2},\\d{3}",
new CandidateTimestampFormat("YYYY-MM-dd HH:mm:ss,SSSZ", "yyyy-MM-dd HH:mm:ss,SSSXX",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2},\\d{3}",
"\\b%{YEAR}-%{MONTHNUM}-%{MONTHDAY} %{HOUR}:?%{MINUTE}:(?:[0-5][0-9]|60)[:.,][0-9]{3,9}(?:Z|[+-]%{HOUR}%{MINUTE})\\b",
"TIMESTAMP_ISO8601", Arrays.asList(0, 1)),
new CandidateTimestampFormat("YYYY-MM-dd HH:mm:ss,SSSZZ", "\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2},\\d{3}",
new CandidateTimestampFormat("YYYY-MM-dd HH:mm:ss,SSSZZ", "yyyy-MM-dd HH:mm:ss,SSSXXX",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2},\\d{3}",
"\\b%{YEAR}-%{MONTHNUM}-%{MONTHDAY} %{HOUR}:?%{MINUTE}:(?:[0-5][0-9]|60)[:.,][0-9]{3,9}[+-]%{HOUR}:%{MINUTE}\\b",
"TIMESTAMP_ISO8601", Arrays.asList(0, 1)),
new CandidateTimestampFormat("YYYY-MM-dd HH:mm:ss,SSS", "\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2},\\d{3}",
new CandidateTimestampFormat("YYYY-MM-dd HH:mm:ss,SSS", "yyyy-MM-dd HH:mm:ss,SSS",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2},\\d{3}",
"\\b%{YEAR}-%{MONTHNUM}-%{MONTHDAY} %{HOUR}:?%{MINUTE}:(?:[0-5][0-9]|60)[:.,][0-9]{3,9}\\b", "TIMESTAMP_ISO8601",
Arrays.asList(0, 1)),
new CandidateTimestampFormat("YYYY-MM-dd HH:mm:ssZ", "\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}",
new CandidateTimestampFormat("YYYY-MM-dd HH:mm:ssZ", "yyyy-MM-dd HH:mm:ssXX", "\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}",
"\\b%{YEAR}-%{MONTHNUM}-%{MONTHDAY} %{HOUR}:?%{MINUTE}:(?:[0-5][0-9]|60)(?:Z|[+-]%{HOUR}%{MINUTE})\\b", "TIMESTAMP_ISO8601",
Arrays.asList(0, 1)),
new CandidateTimestampFormat("YYYY-MM-dd HH:mm:ssZZ", "\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}",
new CandidateTimestampFormat("YYYY-MM-dd HH:mm:ssZZ", "yyyy-MM-dd HH:mm:ssXXX", "\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}",
"\\b%{YEAR}-%{MONTHNUM}-%{MONTHDAY} %{HOUR}:?%{MINUTE}:(?:[0-5][0-9]|60)[+-]%{HOUR}:%{MINUTE}\\b", "TIMESTAMP_ISO8601",
Arrays.asList(0, 1)),
new CandidateTimestampFormat("YYYY-MM-dd HH:mm:ss", "\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}",
new CandidateTimestampFormat("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}",
"\\b%{YEAR}-%{MONTHNUM}-%{MONTHDAY} %{HOUR}:?%{MINUTE}:(?:[0-5][0-9]|60)\\b", "TIMESTAMP_ISO8601",
Arrays.asList(0, 1)),
new CandidateTimestampFormat("ISO8601", "\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", "\\b%{TIMESTAMP_ISO8601}\\b",
"TIMESTAMP_ISO8601"),
new CandidateTimestampFormat("EEE MMM dd YYYY HH:mm:ss zzz",
// When using Java time the Elasticsearch ISO8601 parser for fractional time requires that the fractional
// separator match the current JVM locale, which is too restrictive for arbitrary log file parsing
new CandidateTimestampFormat("ISO8601", "yyyy-MM-dd'T'HH:mm:ss,SSSXX",
"\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}",
"\\b%{YEAR}-%{MONTHNUM}-%{MONTHDAY}T%{HOUR}:?%{MINUTE}:(?:[0-5][0-9]|60)[:.,][0-9]{3,9}(?:Z|[+-]%{HOUR}%{MINUTE})\\b",
"TIMESTAMP_ISO8601", Collections.singletonList(3)),
new CandidateTimestampFormat("ISO8601", "yyyy-MM-dd'T'HH:mm:ss,SSSXXX",
"\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}",
"\\b%{YEAR}-%{MONTHNUM}-%{MONTHDAY}T%{HOUR}:?%{MINUTE}:(?:[0-5][0-9]|60)[:.,][0-9]{3,9}[+-]%{HOUR}:%{MINUTE}\\b",
"TIMESTAMP_ISO8601", Collections.singletonList(3)),
new CandidateTimestampFormat("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}",
"\\b%{YEAR}-%{MONTHNUM}-%{MONTHDAY}T%{HOUR}:?%{MINUTE}:(?:[0-5][0-9]|60)[:.,][0-9]{3,9}\\b", "TIMESTAMP_ISO8601",
Collections.singletonList(3)),
new CandidateTimestampFormat("ISO8601", "ISO8601", "\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", "\\b%{TIMESTAMP_ISO8601}\\b",
"TIMESTAMP_ISO8601", Collections.singletonList(3)),
new CandidateTimestampFormat("EEE MMM dd YYYY HH:mm:ss zzz", "EEE MMM dd yyyy HH:mm:ss zzz",
"\\b[A-Z]\\S{2,8} [A-Z]\\S{2,8} \\d{1,2} \\d{4} \\d{2}:\\d{2}:\\d{2} ",
"\\b%{DAY} %{MONTH} %{MONTHDAY} %{YEAR} %{HOUR}:%{MINUTE}:(?:[0-5][0-9]|60) %{TZ}\\b", "DATESTAMP_RFC822", Arrays.asList(1, 2)),
new CandidateTimestampFormat("EEE MMM dd YYYY HH:mm zzz", "\\b[A-Z]\\S{2,8} [A-Z]\\S{2,8} \\d{1,2} \\d{4} \\d{2}:\\d{2} ",
new CandidateTimestampFormat("EEE MMM dd YYYY HH:mm zzz", "EEE MMM dd yyyy HH:mm zzz",
"\\b[A-Z]\\S{2,8} [A-Z]\\S{2,8} \\d{1,2} \\d{4} \\d{2}:\\d{2} ",
"\\b%{DAY} %{MONTH} %{MONTHDAY} %{YEAR} %{HOUR}:%{MINUTE} %{TZ}\\b", "DATESTAMP_RFC822", Collections.singletonList(1)),
new CandidateTimestampFormat("EEE, dd MMM YYYY HH:mm:ss ZZ",
new CandidateTimestampFormat("EEE, dd MMM YYYY HH:mm:ss ZZ", "EEE, dd MMM yyyy HH:mm:ss XXX",
"\\b[A-Z]\\S{2,8}, \\d{1,2} [A-Z]\\S{2,8} \\d{4} \\d{2}:\\d{2}:\\d{2} ",
"\\b%{DAY}, %{MONTHDAY} %{MONTH} %{YEAR} %{HOUR}:%{MINUTE}:(?:[0-5][0-9]|60) (?:Z|[+-]%{HOUR}:%{MINUTE})\\b",
"DATESTAMP_RFC2822", Arrays.asList(1, 2)),
new CandidateTimestampFormat("EEE, dd MMM YYYY HH:mm:ss Z",
new CandidateTimestampFormat("EEE, dd MMM YYYY HH:mm:ss Z", "EEE, dd MMM yyyy HH:mm:ss XX",
"\\b[A-Z]\\S{2,8}, \\d{1,2} [A-Z]\\S{2,8} \\d{4} \\d{2}:\\d{2}:\\d{2} ",
"\\b%{DAY}, %{MONTHDAY} %{MONTH} %{YEAR} %{HOUR}:%{MINUTE}:(?:[0-5][0-9]|60) (?:Z|[+-]%{HOUR}%{MINUTE})\\b",
"DATESTAMP_RFC2822", Arrays.asList(1, 2)),
new CandidateTimestampFormat("EEE, dd MMM YYYY HH:mm ZZ", "\\b[A-Z]\\S{2,8}, \\d{1,2} [A-Z]\\S{2,8} \\d{4} \\d{2}:\\d{2} ",
new CandidateTimestampFormat("EEE, dd MMM YYYY HH:mm ZZ", "EEE, dd MMM yyyy HH:mm XXX",
"\\b[A-Z]\\S{2,8}, \\d{1,2} [A-Z]\\S{2,8} \\d{4} \\d{2}:\\d{2} ",
"\\b%{DAY}, %{MONTHDAY} %{MONTH} %{YEAR} %{HOUR}:%{MINUTE} (?:Z|[+-]%{HOUR}:%{MINUTE})\\b", "DATESTAMP_RFC2822",
Collections.singletonList(1)),
new CandidateTimestampFormat("EEE, dd MMM YYYY HH:mm Z", "\\b[A-Z]\\S{2,8}, \\d{1,2} [A-Z]\\S{2,8} \\d{4} \\d{2}:\\d{2} ",
new CandidateTimestampFormat("EEE, dd MMM YYYY HH:mm Z", "EEE, dd MMM yyyy HH:mm XX",
"\\b[A-Z]\\S{2,8}, \\d{1,2} [A-Z]\\S{2,8} \\d{4} \\d{2}:\\d{2} ",
"\\b%{DAY}, %{MONTHDAY} %{MONTH} %{YEAR} %{HOUR}:%{MINUTE} (?:Z|[+-]%{HOUR}%{MINUTE})\\b", "DATESTAMP_RFC2822",
Collections.singletonList(1)),
new CandidateTimestampFormat("EEE MMM dd HH:mm:ss zzz YYYY",
new CandidateTimestampFormat("EEE MMM dd HH:mm:ss zzz YYYY", "EEE MMM dd HH:mm:ss zzz yyyy",
"\\b[A-Z]\\S{2,8} [A-Z]\\S{2,8} \\d{1,2} \\d{2}:\\d{2}:\\d{2} [A-Z]{3,4} \\d{4}\\b",
"\\b%{DAY} %{MONTH} %{MONTHDAY} %{HOUR}:%{MINUTE}:(?:[0-5][0-9]|60) %{TZ} %{YEAR}\\b", "DATESTAMP_OTHER",
Arrays.asList(1, 2)),
new CandidateTimestampFormat("EEE MMM dd HH:mm zzz YYYY",
new CandidateTimestampFormat("EEE MMM dd HH:mm zzz YYYY", "EEE MMM dd HH:mm zzz yyyy",
"\\b[A-Z]\\S{2,8} [A-Z]\\S{2,8} \\d{1,2} \\d{2}:\\d{2} [A-Z]{3,4} \\d{4}\\b",
"\\b%{DAY} %{MONTH} %{MONTHDAY} %{HOUR}:%{MINUTE} %{TZ} %{YEAR}\\b", "DATESTAMP_OTHER", Collections.singletonList(1)),
new CandidateTimestampFormat("YYYYMMddHHmmss", "\\b\\d{14}\\b",
new CandidateTimestampFormat("YYYYMMddHHmmss", "yyyyMMddHHmmss", "\\b\\d{14}\\b",
"\\b20\\d{2}%{MONTHNUM2}(?:(?:0[1-9])|(?:[12][0-9])|(?:3[01]))(?:2[0123]|[01][0-9])%{MINUTE}(?:[0-5][0-9]|60)\\b",
"DATESTAMP_EVENTLOG"),
new CandidateTimestampFormat("EEE MMM dd HH:mm:ss YYYY",
new CandidateTimestampFormat("EEE MMM dd HH:mm:ss YYYY", "EEE MMM dd HH:mm:ss yyyy",
"\\b[A-Z]\\S{2,8} [A-Z]\\S{2,8} \\d{1,2} \\d{2}:\\d{2}:\\d{2} \\d{4}\\b",
"\\b%{DAY} %{MONTH} %{MONTHDAY} %{HOUR}:%{MINUTE}:(?:[0-5][0-9]|60) %{YEAR}\\b", "HTTPDERROR_DATE", Arrays.asList(1, 2)),
new CandidateTimestampFormat(Arrays.asList("MMM dd HH:mm:ss,SSS", "MMM d HH:mm:ss,SSS"),
Arrays.asList("MMM dd HH:mm:ss,SSS", "MMM d HH:mm:ss,SSS"),
"\\b[A-Z]\\S{2,8} {1,2}\\d{1,2} \\d{2}:\\d{2}:\\d{2},\\d{3}",
"%{MONTH} +%{MONTHDAY} %{HOUR}:%{MINUTE}:(?:[0-5][0-9]|60)[:.,][0-9]{3,9}\\b", "SYSLOGTIMESTAMP",
Collections.singletonList(1)),
new CandidateTimestampFormat(Arrays.asList("MMM dd HH:mm:ss", "MMM d HH:mm:ss"),
Arrays.asList("MMM dd HH:mm:ss", "MMM d HH:mm:ss"),
"\\b[A-Z]\\S{2,8} {1,2}\\d{1,2} \\d{2}:\\d{2}:\\d{2}\\b", "%{MONTH} +%{MONTHDAY} %{HOUR}:%{MINUTE}:(?:[0-5][0-9]|60)\\b",
"SYSLOGTIMESTAMP", Collections.singletonList(1)),
new CandidateTimestampFormat("dd/MMM/YYYY:HH:mm:ss Z", "\\b\\d{2}/[A-Z]\\S{2}/\\d{4}:\\d{2}:\\d{2}:\\d{2} ",
new CandidateTimestampFormat("dd/MMM/YYYY:HH:mm:ss Z", "dd/MMM/yyyy:HH:mm:ss XX",
"\\b\\d{2}/[A-Z]\\S{2}/\\d{4}:\\d{2}:\\d{2}:\\d{2} ",
"\\b%{MONTHDAY}/%{MONTH}/%{YEAR}:%{HOUR}:%{MINUTE}:(?:[0-5][0-9]|60) [+-]?%{HOUR}%{MINUTE}\\b", "HTTPDATE"),
new CandidateTimestampFormat("MMM dd, YYYY K:mm:ss a", "\\b[A-Z]\\S{2,8} \\d{1,2}, \\d{4} \\d{1,2}:\\d{2}:\\d{2} [AP]M\\b",
new CandidateTimestampFormat("MMM dd, YYYY h:mm:ss a", "MMM dd, yyyy h:mm:ss a",
"\\b[A-Z]\\S{2,8} \\d{1,2}, \\d{4} \\d{1,2}:\\d{2}:\\d{2} [AP]M\\b",
"%{MONTH} %{MONTHDAY}, 20\\d{2} %{HOUR}:%{MINUTE}:(?:[0-5][0-9]|60) (?:AM|PM)\\b", "CATALINA_DATESTAMP"),
new CandidateTimestampFormat(Arrays.asList("MMM dd YYYY HH:mm:ss", "MMM d YYYY HH:mm:ss"),
Arrays.asList("MMM dd yyyy HH:mm:ss", "MMM d yyyy HH:mm:ss"),
"\\b[A-Z]\\S{2,8} {1,2}\\d{1,2} \\d{4} \\d{2}:\\d{2}:\\d{2}\\b",
"%{MONTH} +%{MONTHDAY} %{YEAR} %{HOUR}:%{MINUTE}:(?:[0-5][0-9]|60)\\b", "CISCOTIMESTAMP", Collections.singletonList(1)),
new CandidateTimestampFormat("UNIX_MS", "\\b\\d{13}\\b", "\\b\\d{13}\\b", "POSINT"),
new CandidateTimestampFormat("UNIX", "\\b\\d{10}\\.\\d{3,9}\\b", "\\b\\d{10}\\.(?:\\d{3}){1,3}\\b", "NUMBER"),
new CandidateTimestampFormat("UNIX", "\\b\\d{10}\\b", "\\b\\d{10}\\b", "POSINT"),
new CandidateTimestampFormat("TAI64N", "\\b[0-9A-Fa-f]{24}\\b", "\\b[0-9A-Fa-f]{24}\\b", "BASE16NUM")
new CandidateTimestampFormat("UNIX_MS", "UNIX_MS", "\\b\\d{13}\\b", "\\b\\d{13}\\b", "POSINT"),
new CandidateTimestampFormat("UNIX", "UNIX", "\\b\\d{10}\\.\\d{3,9}\\b", "\\b\\d{10}\\.(?:\\d{3}){1,3}\\b", "NUMBER"),
new CandidateTimestampFormat("UNIX", "UNIX", "\\b\\d{10}\\b", "\\b\\d{10}\\b", "POSINT"),
new CandidateTimestampFormat("TAI64N", "TAI64N", "\\b[0-9A-Fa-f]{24}\\b", "\\b[0-9A-Fa-f]{24}\\b", "BASE16NUM")
);
private TimestampFormatFinder() {
@ -151,7 +182,7 @@ public final class TimestampFormatFinder {
/**
* Find the first timestamp format that matches part of the supplied value.
* @param text The value that the returned timestamp format must exist within.
* @param requiredFormat A date format that any returned match must support.
* @param requiredFormat A timestamp format that any returned match must support.
* @return The timestamp format, or <code>null</code> if none matches.
*/
public static TimestampMatch findFirstMatch(String text, String requiredFormat) {
@ -174,14 +205,19 @@ public final class TimestampFormatFinder {
* excluding a specified number of candidate formats.
* @param text The value that the returned timestamp format must exist within.
* @param ignoreCandidates The number of candidate formats to exclude from the search.
* @param requiredFormat A date format that any returned match must support.
* @param requiredFormat A timestamp format that any returned match must support.
* @return The timestamp format, or <code>null</code> if none matches.
*/
public static TimestampMatch findFirstMatch(String text, int ignoreCandidates, String requiredFormat) {
if (ignoreCandidates >= ORDERED_CANDIDATE_FORMATS.size()) {
return null;
}
Boolean[] quickRuleoutMatches = new Boolean[QUICK_RULE_OUT_PATTERNS.size()];
int index = ignoreCandidates;
String adjustedRequiredFormat = adjustRequiredFormat(requiredFormat);
for (CandidateTimestampFormat candidate : ORDERED_CANDIDATE_FORMATS.subList(ignoreCandidates, ORDERED_CANDIDATE_FORMATS.size())) {
if (requiredFormat == null || candidate.dateFormats.contains(requiredFormat)) {
if (adjustedRequiredFormat == null || candidate.jodaTimestampFormats.contains(adjustedRequiredFormat) ||
candidate.javaTimestampFormats.contains(adjustedRequiredFormat)) {
boolean quicklyRuledOut = false;
for (Integer quickRuleOutIndex : candidate.quickRuleOutIndices) {
if (quickRuleoutMatches[quickRuleOutIndex] == null) {
@ -219,7 +255,7 @@ public final class TimestampFormatFinder {
/**
* Find the best timestamp format for matching an entire field value.
* @param text The value that the returned timestamp format must match in its entirety.
* @param requiredFormat A date format that any returned match must support.
* @param requiredFormat A timestamp format that any returned match must support.
* @return The timestamp format, or <code>null</code> if none matches.
*/
public static TimestampMatch findFirstFullMatch(String text, String requiredFormat) {
@ -242,13 +278,18 @@ public final class TimestampFormatFinder {
* excluding a specified number of candidate formats.
* @param text The value that the returned timestamp format must match in its entirety.
* @param ignoreCandidates The number of candidate formats to exclude from the search.
* @param requiredFormat A date format that any returned match must support.
* @param requiredFormat A timestamp format that any returned match must support.
* @return The timestamp format, or <code>null</code> if none matches.
*/
public static TimestampMatch findFirstFullMatch(String text, int ignoreCandidates, String requiredFormat) {
if (ignoreCandidates >= ORDERED_CANDIDATE_FORMATS.size()) {
return null;
}
int index = ignoreCandidates;
String adjustedRequiredFormat = adjustRequiredFormat(requiredFormat);
for (CandidateTimestampFormat candidate : ORDERED_CANDIDATE_FORMATS.subList(ignoreCandidates, ORDERED_CANDIDATE_FORMATS.size())) {
if (requiredFormat == null || candidate.dateFormats.contains(requiredFormat)) {
if (adjustedRequiredFormat == null || candidate.jodaTimestampFormats.contains(adjustedRequiredFormat) ||
candidate.javaTimestampFormats.contains(adjustedRequiredFormat)) {
Map<String, Object> captures = candidate.strictFullMatchGrok.captures(text);
if (captures != null) {
return makeTimestampMatch(candidate, index, "", text, "");
@ -259,16 +300,32 @@ public final class TimestampFormatFinder {
return null;
}
/**
* If a required timestamp format contains a fractional seconds component, adjust it to the
* fractional seconds format that's in the candidate timestamp formats, i.e. ",SSS". So, for
* example, "YYYY-MM-dd HH:mm:ss.SSSSSSSSS Z" would get adjusted to "YYYY-MM-dd HH:mm:ss,SSS Z".
*/
static String adjustRequiredFormat(String requiredFormat) {
return (requiredFormat == null) ? null :
FRACTIONAL_SECOND_TIMESTAMP_FORMAT_PATTERN.matcher(requiredFormat).replaceFirst(DEFAULT_FRACTIONAL_SECOND_FORMAT);
}
private static TimestampMatch makeTimestampMatch(CandidateTimestampFormat chosenTimestampFormat, int chosenIndex,
String preface, String matchedDate, String epilogue) {
Tuple<Character, Integer> fractionalSecondsInterpretation = interpretFractionalSeconds(matchedDate);
List<String> dateFormats = chosenTimestampFormat.dateFormats;
List<String> jodaTimestampFormats = chosenTimestampFormat.jodaTimestampFormats;
List<String> javaTimestampFormats = chosenTimestampFormat.javaTimestampFormats;
Pattern simplePattern = chosenTimestampFormat.simplePattern;
char separator = fractionalSecondsInterpretation.v1();
if (separator != DEFAULT_FRACTIONAL_SECOND_SEPARATOR) {
dateFormats = dateFormats.stream().map(dateFormat -> dateFormat.replace(DEFAULT_FRACTIONAL_SECOND_SEPARATOR, separator))
jodaTimestampFormats = jodaTimestampFormats.stream()
.map(jodaTimestampFormat -> jodaTimestampFormat.replace(DEFAULT_FRACTIONAL_SECOND_SEPARATOR, separator))
.collect(Collectors.toList());
if (dateFormats.stream().noneMatch(dateFormat -> dateFormat.startsWith("UNIX"))) {
javaTimestampFormats = javaTimestampFormats.stream()
.map(javaTimestampFormat -> javaTimestampFormat.replace(DEFAULT_FRACTIONAL_SECOND_SEPARATOR, separator))
.collect(Collectors.toList());
if (jodaTimestampFormats.stream().noneMatch(jodaTimestampFormat -> jodaTimestampFormat.startsWith("UNIX"))) {
String patternStr = simplePattern.pattern();
int separatorPos = patternStr.lastIndexOf(DEFAULT_FRACTIONAL_SECOND_SEPARATOR);
if (separatorPos >= 0) {
@ -281,11 +338,15 @@ public final class TimestampFormatFinder {
int numberOfDigitsInFractionalComponent = fractionalSecondsInterpretation.v2();
if (numberOfDigitsInFractionalComponent > 3) {
String fractionalSecondsFormat = "SSSSSSSSS".substring(0, numberOfDigitsInFractionalComponent);
dateFormats = dateFormats.stream().map(dateFormat -> dateFormat.replace("SSS", fractionalSecondsFormat))
jodaTimestampFormats = jodaTimestampFormats.stream()
.map(jodaTimestampFormat -> jodaTimestampFormat.replace("SSS", fractionalSecondsFormat))
.collect(Collectors.toList());
javaTimestampFormats = javaTimestampFormats.stream()
.map(javaTimestampFormat -> javaTimestampFormat.replace("SSS", fractionalSecondsFormat))
.collect(Collectors.toList());
}
return new TimestampMatch(chosenIndex, preface, dateFormats, simplePattern, chosenTimestampFormat.standardGrokPatternName,
epilogue);
return new TimestampMatch(chosenIndex, preface, jodaTimestampFormats, javaTimestampFormats, simplePattern,
chosenTimestampFormat.standardGrokPatternName, epilogue);
}
/**
@ -323,7 +384,12 @@ public final class TimestampFormatFinder {
/**
* Time format specifier(s) that will work with Logstash and Ingest pipeline date parsers.
*/
public final List<String> dateFormats;
public final List<String> jodaTimestampFormats;
/**
* Time format specifier(s) that will work with Logstash and Ingest pipeline date parsers.
*/
public final List<String> javaTimestampFormats;
/**
* A simple regex that will work in many languages to detect whether the timestamp format
@ -341,25 +407,25 @@ public final class TimestampFormatFinder {
*/
public final String epilogue;
TimestampMatch(int candidateIndex, String preface, String dateFormat, String simpleRegex, String grokPatternName, String epilogue) {
this(candidateIndex, preface, Collections.singletonList(dateFormat), simpleRegex, grokPatternName, epilogue);
TimestampMatch(int candidateIndex, String preface, String jodaTimestampFormat, String javaTimestampFormat, String simpleRegex,
String grokPatternName, String epilogue) {
this(candidateIndex, preface, Collections.singletonList(jodaTimestampFormat), Collections.singletonList(javaTimestampFormat),
simpleRegex, grokPatternName, epilogue);
}
TimestampMatch(int candidateIndex, String preface, String dateFormat, String simpleRegex, String grokPatternName, String epilogue,
boolean hasFractionalComponentSmallerThanMillisecond) {
this(candidateIndex, preface, Collections.singletonList(dateFormat), simpleRegex, grokPatternName, epilogue);
TimestampMatch(int candidateIndex, String preface, List<String> jodaTimestampFormats, List<String> javaTimestampFormats,
String simpleRegex, String grokPatternName, String epilogue) {
this(candidateIndex, preface, jodaTimestampFormats, javaTimestampFormats, Pattern.compile(simpleRegex), grokPatternName,
epilogue);
}
TimestampMatch(int candidateIndex, String preface, List<String> dateFormats, String simpleRegex, String grokPatternName,
String epilogue) {
this(candidateIndex, preface, dateFormats, Pattern.compile(simpleRegex), grokPatternName, epilogue);
}
TimestampMatch(int candidateIndex, String preface, List<String> dateFormats, Pattern simplePattern, String grokPatternName,
TimestampMatch(int candidateIndex, String preface, List<String> jodaTimestampFormats, List<String> javaTimestampFormats,
Pattern simplePattern, String grokPatternName,
String epilogue) {
this.candidateIndex = candidateIndex;
this.preface = preface;
this.dateFormats = dateFormats;
this.jodaTimestampFormats = Collections.unmodifiableList(jodaTimestampFormats);
this.javaTimestampFormats = Collections.unmodifiableList(javaTimestampFormats);
this.simplePattern = simplePattern;
this.grokPatternName = grokPatternName;
this.epilogue = epilogue;
@ -370,8 +436,8 @@ public final class TimestampFormatFinder {
* I.e., does the textual representation NOT define the timezone?
*/
public boolean hasTimezoneDependentParsing() {
return dateFormats.stream()
.anyMatch(dateFormat -> dateFormat.contains("HH") && dateFormat.toLowerCase(Locale.ROOT).indexOf('z') == -1);
return javaTimestampFormats.stream().anyMatch(javaTimestampFormat ->
javaTimestampFormat.indexOf('X') == -1 && javaTimestampFormat.indexOf('z') == -1 && javaTimestampFormat.contains("mm"));
}
/**
@ -380,13 +446,13 @@ public final class TimestampFormatFinder {
* and possibly also a "format" setting.
*/
public Map<String, String> getEsDateMappingTypeWithFormat() {
if (dateFormats.contains("TAI64N")) {
// There's no format for TAI64N in the date formats used in mappings
if (jodaTimestampFormats.contains("TAI64N")) {
// There's no format for TAI64N in the timestamp formats used in mappings
return Collections.singletonMap(FileStructureUtils.MAPPING_TYPE_SETTING, "keyword");
}
Map<String, String> mapping = new LinkedHashMap<>();
mapping.put(FileStructureUtils.MAPPING_TYPE_SETTING, "date");
String formats = dateFormats.stream().flatMap(format -> {
String formats = jodaTimestampFormats.stream().flatMap(format -> {
switch (format) {
case "ISO8601":
return Stream.empty();
@ -406,7 +472,8 @@ public final class TimestampFormatFinder {
@Override
public int hashCode() {
return Objects.hash(candidateIndex, preface, dateFormats, simplePattern.pattern(), grokPatternName, epilogue);
return Objects.hash(candidateIndex, preface, jodaTimestampFormats, javaTimestampFormats, simplePattern.pattern(),
grokPatternName, epilogue);
}
@Override
@ -421,7 +488,8 @@ public final class TimestampFormatFinder {
TimestampMatch that = (TimestampMatch) other;
return this.candidateIndex == that.candidateIndex &&
Objects.equals(this.preface, that.preface) &&
Objects.equals(this.dateFormats, that.dateFormats) &&
Objects.equals(this.jodaTimestampFormats, that.jodaTimestampFormats) &&
Objects.equals(this.javaTimestampFormats, that.javaTimestampFormats) &&
Objects.equals(this.simplePattern.pattern(), that.simplePattern.pattern()) &&
Objects.equals(this.grokPatternName, that.grokPatternName) &&
Objects.equals(this.epilogue, that.epilogue);
@ -430,7 +498,8 @@ public final class TimestampFormatFinder {
@Override
public String toString() {
return "index = " + candidateIndex + (preface.isEmpty() ? "" : ", preface = '" + preface + "'") +
", date formats = " + dateFormats.stream().collect(Collectors.joining("', '", "[ '", "' ]")) +
", Joda timestamp formats = " + jodaTimestampFormats.stream().collect(Collectors.joining("', '", "[ '", "' ]")) +
", Java timestamp formats = " + javaTimestampFormats.stream().collect(Collectors.joining("', '", "[ '", "' ]")) +
", simple pattern = '" + simplePattern.pattern() + "', grok pattern = '" + grokPatternName + "'" +
(epilogue.isEmpty() ? "" : ", epilogue = '" + epilogue + "'");
}
@ -438,29 +507,36 @@ public final class TimestampFormatFinder {
static final class CandidateTimestampFormat {
final List<String> dateFormats;
final List<String> jodaTimestampFormats;
final List<String> javaTimestampFormats;
final Pattern simplePattern;
final Grok strictSearchGrok;
final Grok strictFullMatchGrok;
final String standardGrokPatternName;
final List<Integer> quickRuleOutIndices;
CandidateTimestampFormat(String dateFormat, String simpleRegex, String strictGrokPattern, String standardGrokPatternName) {
this(Collections.singletonList(dateFormat), simpleRegex, strictGrokPattern, standardGrokPatternName);
CandidateTimestampFormat(String jodaTimestampFormat, String javaTimestampFormat, String simpleRegex, String strictGrokPattern,
String standardGrokPatternName) {
this(Collections.singletonList(jodaTimestampFormat), Collections.singletonList(javaTimestampFormat), simpleRegex,
strictGrokPattern, standardGrokPatternName);
}
CandidateTimestampFormat(String dateFormat, String simpleRegex, String strictGrokPattern, String standardGrokPatternName,
List<Integer> quickRuleOutIndices) {
this(Collections.singletonList(dateFormat), simpleRegex, strictGrokPattern, standardGrokPatternName, quickRuleOutIndices);
CandidateTimestampFormat(String jodaTimestampFormat, String javaTimestampFormat, String simpleRegex, String strictGrokPattern,
String standardGrokPatternName, List<Integer> quickRuleOutIndices) {
this(Collections.singletonList(jodaTimestampFormat), Collections.singletonList(javaTimestampFormat), simpleRegex,
strictGrokPattern, standardGrokPatternName, quickRuleOutIndices);
}
CandidateTimestampFormat(List<String> dateFormats, String simpleRegex, String strictGrokPattern, String standardGrokPatternName) {
this(dateFormats, simpleRegex, strictGrokPattern, standardGrokPatternName, Collections.emptyList());
CandidateTimestampFormat(List<String> jodaTimestampFormats, List<String> javaTimestampFormats, String simpleRegex,
String strictGrokPattern, String standardGrokPatternName) {
this(jodaTimestampFormats, javaTimestampFormats, simpleRegex, strictGrokPattern, standardGrokPatternName,
Collections.emptyList());
}
CandidateTimestampFormat(List<String> dateFormats, String simpleRegex, String strictGrokPattern, String standardGrokPatternName,
List<Integer> quickRuleOutIndices) {
this.dateFormats = dateFormats;
CandidateTimestampFormat(List<String> jodaTimestampFormats, List<String> javaTimestampFormats, String simpleRegex,
String strictGrokPattern, String standardGrokPatternName, List<Integer> quickRuleOutIndices) {
this.jodaTimestampFormats = jodaTimestampFormats;
this.javaTimestampFormats = javaTimestampFormats;
this.simplePattern = Pattern.compile(simpleRegex, Pattern.MULTILINE);
// The (?m) here has the Ruby meaning, which is equivalent to (?s) in Java
this.strictSearchGrok = new Grok(Grok.getBuiltinPatterns(), "(?m)%{DATA:" + PREFACE + "}" + strictGrokPattern +

View File

@ -93,7 +93,8 @@ public class XmlFileStructureFinder implements FileStructureFinder {
Tuple<String, TimestampMatch> timeField = FileStructureUtils.guessTimestampField(explanation, sampleRecords, overrides);
if (timeField != null) {
structureBuilder.setTimestampField(timeField.v1())
.setTimestampFormats(timeField.v2().dateFormats)
.setJodaTimestampFormats(timeField.v2().jodaTimestampFormats)
.setJavaTimestampFormats(timeField.v2().javaTimestampFormats)
.setNeedClientTimezone(timeField.v2().hasTimezoneDependentParsing());
}

View File

@ -50,7 +50,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
assertEquals(Arrays.asList("time", "message"), structure.getColumnNames());
assertNull(structure.getGrokPattern());
assertEquals("time", structure.getTimestampField());
assertEquals(Collections.singletonList("ISO8601"), structure.getTimestampFormats());
assertEquals(Collections.singletonList("ISO8601"), structure.getJodaTimestampFormats());
}
public void testCreateConfigsGivenCompleteCsvAndColumnNamesOverride() throws Exception {
@ -84,7 +84,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
assertEquals(Arrays.asList("my_time", "my_message"), structure.getColumnNames());
assertNull(structure.getGrokPattern());
assertEquals("my_time", structure.getTimestampField());
assertEquals(Collections.singletonList("ISO8601"), structure.getTimestampFormats());
assertEquals(Collections.singletonList("ISO8601"), structure.getJodaTimestampFormats());
}
public void testCreateConfigsGivenCompleteCsvAndHasHeaderRowOverride() throws Exception {
@ -120,7 +120,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
assertEquals(Arrays.asList("column1", "column2"), structure.getColumnNames());
assertNull(structure.getGrokPattern());
assertNull(structure.getTimestampField());
assertNull(structure.getTimestampFormats());
assertNull(structure.getJodaTimestampFormats());
}
public void testCreateConfigsGivenCsvWithIncompleteLastRecord() throws Exception {
@ -153,7 +153,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
assertEquals(Arrays.asList("message", "time", "count"), structure.getColumnNames());
assertNull(structure.getGrokPattern());
assertEquals("time", structure.getTimestampField());
assertEquals(Collections.singletonList("ISO8601"), structure.getTimestampFormats());
assertEquals(Collections.singletonList("ISO8601"), structure.getJodaTimestampFormats());
}
public void testCreateConfigsGivenCsvWithTrailingNulls() throws Exception {
@ -193,7 +193,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
"tip_amount", "tolls_amount", "improvement_surcharge", "total_amount", "column18", "column19"), structure.getColumnNames());
assertNull(structure.getGrokPattern());
assertEquals("tpep_pickup_datetime", structure.getTimestampField());
assertEquals(Collections.singletonList("YYYY-MM-dd HH:mm:ss"), structure.getTimestampFormats());
assertEquals(Collections.singletonList("YYYY-MM-dd HH:mm:ss"), structure.getJodaTimestampFormats());
}
public void testCreateConfigsGivenCsvWithTrailingNullsAndOverriddenTimeField() throws Exception {
@ -237,7 +237,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
"tip_amount", "tolls_amount", "improvement_surcharge", "total_amount", "column18", "column19"), structure.getColumnNames());
assertNull(structure.getGrokPattern());
assertEquals("tpep_dropoff_datetime", structure.getTimestampField());
assertEquals(Collections.singletonList("YYYY-MM-dd HH:mm:ss"), structure.getTimestampFormats());
assertEquals(Collections.singletonList("YYYY-MM-dd HH:mm:ss"), structure.getJodaTimestampFormats());
}
public void testCreateConfigsGivenCsvWithTrailingNullsExceptHeader() throws Exception {
@ -277,7 +277,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
"tip_amount", "tolls_amount", "improvement_surcharge", "total_amount"), structure.getColumnNames());
assertNull(structure.getGrokPattern());
assertEquals("tpep_pickup_datetime", structure.getTimestampField());
assertEquals(Collections.singletonList("YYYY-MM-dd HH:mm:ss"), structure.getTimestampFormats());
assertEquals(Collections.singletonList("YYYY-MM-dd HH:mm:ss"), structure.getJodaTimestampFormats());
}
public void testCreateConfigsGivenCsvWithTrailingNullsExceptHeaderAndColumnNamesOverride() throws Exception {
@ -324,7 +324,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
structure.getColumnNames());
assertNull(structure.getGrokPattern());
assertEquals("my_tpep_pickup_datetime", structure.getTimestampField());
assertEquals(Collections.singletonList("YYYY-MM-dd HH:mm:ss"), structure.getTimestampFormats());
assertEquals(Collections.singletonList("YYYY-MM-dd HH:mm:ss"), structure.getJodaTimestampFormats());
}
public void testCreateConfigsGivenCsvWithTimeLastColumn() throws Exception {
@ -357,7 +357,7 @@ public class DelimitedFileStructureFinderTests extends FileStructureTestCase {
assertEquals(Arrays.asList("pos_id", "trip_id", "latitude", "longitude", "altitude", "timestamp"), structure.getColumnNames());
assertNull(structure.getGrokPattern());
assertEquals("timestamp", structure.getTimestampField());
assertEquals(Collections.singletonList("YYYY-MM-dd HH:mm:ss.SSSSSS"), structure.getTimestampFormats());
assertEquals(Collections.singletonList("YYYY-MM-dd HH:mm:ss.SSSSSS"), structure.getJodaTimestampFormats());
}
public void testFindHeaderFromSampleGivenHeaderInSample() throws IOException {

View File

@ -39,7 +39,7 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
FileStructureUtils.guessTimestampField(explanation, Collections.singletonList(sample), EMPTY_OVERRIDES);
assertNotNull(match);
assertEquals("field1", match.v1());
assertThat(match.v2().dateFormats, contains("ISO8601"));
assertThat(match.v2().jodaTimestampFormats, contains("ISO8601"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName);
}
@ -52,7 +52,7 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
FileStructureUtils.guessTimestampField(explanation, Collections.singletonList(sample), overrides);
assertNotNull(match);
assertEquals("field1", match.v1());
assertThat(match.v2().dateFormats, contains("ISO8601"));
assertThat(match.v2().jodaTimestampFormats, contains("ISO8601"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName);
}
@ -76,7 +76,7 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
FileStructureUtils.guessTimestampField(explanation, Collections.singletonList(sample), overrides);
assertNotNull(match);
assertEquals("field1", match.v1());
assertThat(match.v2().dateFormats, contains("ISO8601"));
assertThat(match.v2().jodaTimestampFormats, contains("ISO8601"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName);
}
@ -99,7 +99,7 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), EMPTY_OVERRIDES);
assertNotNull(match);
assertEquals("field1", match.v1());
assertThat(match.v2().dateFormats, contains("ISO8601"));
assertThat(match.v2().jodaTimestampFormats, contains("ISO8601"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName);
}
@ -128,7 +128,7 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
FileStructureUtils.guessTimestampField(explanation, Collections.singletonList(sample), EMPTY_OVERRIDES);
assertNotNull(match);
assertEquals("time", match.v1());
assertThat(match.v2().dateFormats, contains("YYYY-MM-dd HH:mm:ss,SSS"));
assertThat(match.v2().jodaTimestampFormats, contains("YYYY-MM-dd HH:mm:ss,SSS"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName);
}
@ -145,7 +145,7 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), EMPTY_OVERRIDES);
assertNotNull(match);
assertEquals("time", match.v1());
assertThat(match.v2().dateFormats, contains("YYYY-MM-dd HH:mm:ss,SSS"));
assertThat(match.v2().jodaTimestampFormats, contains("YYYY-MM-dd HH:mm:ss,SSS"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName);
}
@ -176,7 +176,7 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), EMPTY_OVERRIDES);
assertNotNull(match);
assertEquals("time", match.v1());
assertThat(match.v2().dateFormats, contains("YYYY-MM-dd HH:mm:ss,SSS"));
assertThat(match.v2().jodaTimestampFormats, contains("YYYY-MM-dd HH:mm:ss,SSS"));
assertEquals("TIMESTAMP_ISO8601", match.v2().grokPatternName);
}
@ -193,7 +193,7 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), EMPTY_OVERRIDES);
assertNotNull(match);
assertEquals("time", match.v1());
assertThat(match.v2().dateFormats, contains("MMM dd YYYY HH:mm:ss", "MMM d YYYY HH:mm:ss"));
assertThat(match.v2().jodaTimestampFormats, contains("MMM dd YYYY HH:mm:ss", "MMM d YYYY HH:mm:ss"));
assertEquals("CISCOTIMESTAMP", match.v2().grokPatternName);
}
@ -226,7 +226,7 @@ public class FileStructureUtilsTests extends FileStructureTestCase {
FileStructureUtils.guessTimestampField(explanation, Arrays.asList(sample1, sample2), EMPTY_OVERRIDES);
assertNotNull(match);
assertEquals("time2", match.v1());
assertThat(match.v2().dateFormats, contains("MMM dd YYYY HH:mm:ss", "MMM d YYYY HH:mm:ss"));
assertThat(match.v2().jodaTimestampFormats, contains("MMM dd YYYY HH:mm:ss", "MMM d YYYY HH:mm:ss"));
assertEquals("CISCOTIMESTAMP", match.v2().grokPatternName);
}

View File

@ -38,6 +38,6 @@ public class JsonFileStructureFinderTests extends FileStructureTestCase {
assertNull(structure.getShouldTrimFields());
assertNull(structure.getGrokPattern());
assertEquals("timestamp", structure.getTimestampField());
assertEquals(Collections.singletonList("UNIX_MS"), structure.getTimestampFormats());
assertEquals(Collections.singletonList("UNIX_MS"), structure.getJodaTimestampFormats());
}
}

View File

@ -119,14 +119,14 @@ public class TextLogFileStructureFinderTests extends FileStructureTestCase {
assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker());
}
assertNull(structure.getExcludeLinesPattern());
assertEquals("^\\[\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern());
assertEquals("^\\[\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}", structure.getMultilineStartPattern());
assertNull(structure.getDelimiter());
assertNull(structure.getQuote());
assertNull(structure.getHasHeaderRow());
assertNull(structure.getShouldTrimFields());
assertEquals("\\[%{TIMESTAMP_ISO8601:timestamp}\\]\\[%{LOGLEVEL:loglevel} \\]\\[.*", structure.getGrokPattern());
assertEquals("timestamp", structure.getTimestampField());
assertEquals(Collections.singletonList("ISO8601"), structure.getTimestampFormats());
assertEquals(Collections.singletonList("ISO8601"), structure.getJodaTimestampFormats());
}
public void testCreateConfigsGivenElasticsearchLogAndTimestampFieldOverride() throws Exception {
@ -149,14 +149,14 @@ public class TextLogFileStructureFinderTests extends FileStructureTestCase {
assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker());
}
assertNull(structure.getExcludeLinesPattern());
assertEquals("^\\[\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern());
assertEquals("^\\[\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}", structure.getMultilineStartPattern());
assertNull(structure.getDelimiter());
assertNull(structure.getQuote());
assertNull(structure.getHasHeaderRow());
assertNull(structure.getShouldTrimFields());
assertEquals("\\[%{TIMESTAMP_ISO8601:my_time}\\]\\[%{LOGLEVEL:loglevel} \\]\\[.*", structure.getGrokPattern());
assertEquals("my_time", structure.getTimestampField());
assertEquals(Collections.singletonList("ISO8601"), structure.getTimestampFormats());
assertEquals(Collections.singletonList("ISO8601"), structure.getJodaTimestampFormats());
}
public void testCreateConfigsGivenElasticsearchLogAndGrokPatternOverride() throws Exception {
@ -180,7 +180,7 @@ public class TextLogFileStructureFinderTests extends FileStructureTestCase {
assertEquals(hasByteOrderMarker, structure.getHasByteOrderMarker());
}
assertNull(structure.getExcludeLinesPattern());
assertEquals("^\\[\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", structure.getMultilineStartPattern());
assertEquals("^\\[\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}", structure.getMultilineStartPattern());
assertNull(structure.getDelimiter());
assertNull(structure.getQuote());
assertNull(structure.getHasHeaderRow());
@ -188,7 +188,7 @@ public class TextLogFileStructureFinderTests extends FileStructureTestCase {
assertEquals("\\[%{TIMESTAMP_ISO8601:timestamp}\\]\\[%{LOGLEVEL:loglevel} *\\]" +
"\\[%{JAVACLASS:class} *\\] \\[%{HOSTNAME:node}\\] %{JAVALOGMESSAGE:message}", structure.getGrokPattern());
assertEquals("timestamp", structure.getTimestampField());
assertEquals(Collections.singletonList("ISO8601"), structure.getTimestampFormats());
assertEquals(Collections.singletonList("ISO8601"), structure.getJodaTimestampFormats());
}
public void testCreateConfigsGivenElasticsearchLogAndImpossibleGrokPatternOverride() {
@ -312,8 +312,8 @@ public class TextLogFileStructureFinderTests extends FileStructureTestCase {
Tuple<TimestampMatch, Set<String>> mostLikelyMatch =
TextLogFileStructureFinder.mostLikelyTimestamp(sample.split("\n"), FileStructureOverrides.EMPTY_OVERRIDES);
assertNotNull(mostLikelyMatch);
assertEquals(new TimestampMatch(7, "", "ISO8601", "\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", "TIMESTAMP_ISO8601", ""),
mostLikelyMatch.v1());
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() {
@ -325,8 +325,8 @@ public class TextLogFileStructureFinderTests extends FileStructureTestCase {
// 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(7, "", "ISO8601", "\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", "TIMESTAMP_ISO8601", ""),
mostLikelyMatch.v1());
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() {
@ -338,8 +338,8 @@ public class TextLogFileStructureFinderTests extends FileStructureTestCase {
assertNotNull(mostLikelyMatch);
// The override should force the seemingly inferior choice of timestamp
assertEquals(new TimestampMatch(6, "", "YYYY-MM-dd HH:mm:ss", "\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", "TIMESTAMP_ISO8601",
""), mostLikelyMatch.v1());
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() {

View File

@ -6,14 +6,11 @@
package org.elasticsearch.xpack.ml.filestructurefinder;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.time.DateFormatters;
import org.elasticsearch.xpack.ml.filestructurefinder.TimestampFormatFinder.TimestampMatch;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
import org.joda.time.format.ISODateTimeFormat;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
public class TimestampFormatFinderTests extends FileStructureTestCase {
@ -28,199 +25,206 @@ public class TimestampFormatFinderTests extends FileStructureTestCase {
public void testFindFirstMatchGivenOnlyIso8601() {
TimestampMatch expected = new TimestampMatch(7, "", "ISO8601", "\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", "TIMESTAMP_ISO8601",
"");
validateTimestampMatch(new TimestampMatch(7, "", "ISO8601", "yyyy-MM-dd'T'HH:mm:ss,SSSXX",
"\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}", "TIMESTAMP_ISO8601", ""), "2018-05-15T16:14:56,374Z",
1526400896374L);
validateTimestampMatch(new TimestampMatch(7, "", "ISO8601", "yyyy-MM-dd'T'HH:mm:ss,SSSXX",
"\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}", "TIMESTAMP_ISO8601", ""), "2018-05-15T17:14:56,374+0100",
1526400896374L);
validateTimestampMatch(new TimestampMatch(8, "", "ISO8601", "yyyy-MM-dd'T'HH:mm:ss,SSSXXX",
"\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2},\\d{3}", "TIMESTAMP_ISO8601", ""), "2018-05-15T17:14:56,374+01:00",
1526400896374L);
validateTimestampMatch(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", ""), "2018-05-15T17:14:56,374", 1526400896374L);
checkAndValidateDateFormat(expected, "2018-05-15T16:14:56,374Z", 1526400896374L);
checkAndValidateDateFormat(expected, "2018-05-15T17:14:56,374+0100", 1526400896374L);
checkAndValidateDateFormat(expected, "2018-05-15T17:14:56,374+01:00", 1526400896374L);
checkAndValidateDateFormat(expected, "2018-05-15T17:14:56,374", 1526400896374L);
checkAndValidateDateFormat(expected, "2018-05-15T16:14:56Z", 1526400896000L);
checkAndValidateDateFormat(expected, "2018-05-15T17:14:56+0100", 1526400896000L);
checkAndValidateDateFormat(expected, "2018-05-15T17:14:56+01:00", 1526400896000L);
checkAndValidateDateFormat(expected, "2018-05-15T17:14:56", 1526400896000L);
TimestampMatch pureIso8601Expected = new TimestampMatch(10, "", "ISO8601", "ISO8601",
"\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", "TIMESTAMP_ISO8601", "");
checkAndValidateDateFormat(new TimestampMatch(1, "", "YYYY-MM-dd HH:mm:ss,SSSZ",
validateTimestampMatch(pureIso8601Expected, "2018-05-15T16:14:56Z", 1526400896000L);
validateTimestampMatch(pureIso8601Expected, "2018-05-15T17:14:56+0100", 1526400896000L);
validateTimestampMatch(pureIso8601Expected, "2018-05-15T17:14:56+01:00", 1526400896000L);
validateTimestampMatch(pureIso8601Expected, "2018-05-15T17:14:56", 1526400896000L);
validateTimestampMatch(new TimestampMatch(1, "", "YYYY-MM-dd HH:mm:ss,SSSZ", "yyyy-MM-dd HH:mm:ss,SSSXX",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2},\\d{3}", "TIMESTAMP_ISO8601", ""), "2018-05-15 16:14:56,374Z",
1526400896374L);
checkAndValidateDateFormat(new TimestampMatch(1, "", "YYYY-MM-dd HH:mm:ss,SSSZ",
validateTimestampMatch(new TimestampMatch(1, "", "YYYY-MM-dd HH:mm:ss,SSSZ", "yyyy-MM-dd HH:mm:ss,SSSXX",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2},\\d{3}", "TIMESTAMP_ISO8601", ""), "2018-05-15 17:14:56,374+0100",
1526400896374L);
checkAndValidateDateFormat(new TimestampMatch(2, "", "YYYY-MM-dd HH:mm:ss,SSSZZ",
validateTimestampMatch(new TimestampMatch(2, "", "YYYY-MM-dd HH:mm:ss,SSSZZ", "yyyy-MM-dd HH:mm:ss,SSSXXX",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2},\\d{3}", "TIMESTAMP_ISO8601", ""), "2018-05-15 17:14:56,374+01:00",
1526400896374L);
checkAndValidateDateFormat(new TimestampMatch(3, "", "YYYY-MM-dd HH:mm:ss,SSS",
validateTimestampMatch(new TimestampMatch(3, "", "YYYY-MM-dd HH:mm:ss,SSS", "yyyy-MM-dd HH:mm:ss,SSS",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2},\\d{3}", "TIMESTAMP_ISO8601", ""), "2018-05-15 17:14:56,374", 1526400896374L);
checkAndValidateDateFormat(new TimestampMatch(4, "", "YYYY-MM-dd HH:mm:ssZ",
validateTimestampMatch(new TimestampMatch(4, "", "YYYY-MM-dd HH:mm:ssZ", "yyyy-MM-dd HH:mm:ssXX",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", "TIMESTAMP_ISO8601", ""), "2018-05-15 16:14:56Z", 1526400896000L);
checkAndValidateDateFormat(new TimestampMatch(4, "", "YYYY-MM-dd HH:mm:ssZ",
validateTimestampMatch(new TimestampMatch(4, "", "YYYY-MM-dd HH:mm:ssZ", "yyyy-MM-dd HH:mm:ssXX",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", "TIMESTAMP_ISO8601", ""), "2018-05-15 17:14:56+0100", 1526400896000L);
checkAndValidateDateFormat(new TimestampMatch(5, "", "YYYY-MM-dd HH:mm:ssZZ",
validateTimestampMatch(new TimestampMatch(5, "", "YYYY-MM-dd HH:mm:ssZZ", "yyyy-MM-dd HH:mm:ssXXX",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", "TIMESTAMP_ISO8601", ""), "2018-05-15 17:14:56+01:00", 1526400896000L);
checkAndValidateDateFormat(new TimestampMatch(6, "", "YYYY-MM-dd HH:mm:ss",
validateTimestampMatch(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", ""), "2018-05-15 17:14:56", 1526400896000L);
}
public void testFindFirstMatchGivenOnlyKnownDateFormat() {
public void testFindFirstMatchGivenOnlyKnownTimestampFormat() {
// Note: some of the time formats give millisecond accuracy, some second accuracy and some minute accuracy
checkAndValidateDateFormat(new TimestampMatch(0, "", "YYYY-MM-dd HH:mm:ss,SSS Z",
validateTimestampMatch(new TimestampMatch(0, "", "YYYY-MM-dd HH:mm:ss,SSS Z", "yyyy-MM-dd HH:mm:ss,SSS XX",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2},\\d{3}", "TOMCAT_DATESTAMP", ""), "2018-05-15 17:14:56,374 +0100",
1526400896374L);
checkAndValidateDateFormat(new TimestampMatch(8, "", "EEE MMM dd YYYY HH:mm:ss zzz",
validateTimestampMatch(new TimestampMatch(11, "", "EEE MMM dd YYYY HH:mm:ss zzz", "EEE MMM dd yyyy HH:mm:ss zzz",
"\\b[A-Z]\\S{2,8} [A-Z]\\S{2,8} \\d{1,2} \\d{4} \\d{2}:\\d{2}:\\d{2} ", "DATESTAMP_RFC822", ""),
"Tue May 15 2018 16:14:56 UTC", 1526400896000L);
checkAndValidateDateFormat(new TimestampMatch(9, "", "EEE MMM dd YYYY HH:mm zzz",
validateTimestampMatch(new TimestampMatch(12, "", "EEE MMM dd YYYY HH:mm zzz", "EEE MMM dd yyyy HH:mm zzz",
"\\b[A-Z]\\S{2,8} [A-Z]\\S{2,8} \\d{1,2} \\d{4} \\d{2}:\\d{2} ", "DATESTAMP_RFC822", ""),
"Tue May 15 2018 16:14 UTC", 1526400840000L);
checkAndValidateDateFormat(new TimestampMatch(10, "", "EEE, dd MMM YYYY HH:mm:ss ZZ",
validateTimestampMatch(new TimestampMatch(13, "", "EEE, dd MMM YYYY HH:mm:ss ZZ", "EEE, dd MMM yyyy HH:mm:ss XXX",
"\\b[A-Z]\\S{2,8}, \\d{1,2} [A-Z]\\S{2,8} \\d{4} \\d{2}:\\d{2}:\\d{2} ", "DATESTAMP_RFC2822", ""),
"Tue, 15 May 2018 17:14:56 +01:00", 1526400896000L);
checkAndValidateDateFormat(new TimestampMatch(11, "", "EEE, dd MMM YYYY HH:mm:ss Z",
validateTimestampMatch(new TimestampMatch(14, "", "EEE, dd MMM YYYY HH:mm:ss Z", "EEE, dd MMM yyyy HH:mm:ss XX",
"\\b[A-Z]\\S{2,8}, \\d{1,2} [A-Z]\\S{2,8} \\d{4} \\d{2}:\\d{2}:\\d{2} ", "DATESTAMP_RFC2822", ""),
"Tue, 15 May 2018 17:14:56 +0100", 1526400896000L);
checkAndValidateDateFormat(new TimestampMatch(12, "", "EEE, dd MMM YYYY HH:mm ZZ",
validateTimestampMatch(new TimestampMatch(15, "", "EEE, dd MMM YYYY HH:mm ZZ", "EEE, dd MMM yyyy HH:mm XXX",
"\\b[A-Z]\\S{2,8}, \\d{1,2} [A-Z]\\S{2,8} \\d{4} \\d{2}:\\d{2} ", "DATESTAMP_RFC2822", ""),
"Tue, 15 May 2018 17:14 +01:00", 1526400840000L);
checkAndValidateDateFormat(new TimestampMatch(13, "", "EEE, dd MMM YYYY HH:mm Z",
validateTimestampMatch(new TimestampMatch(16, "", "EEE, dd MMM YYYY HH:mm Z", "EEE, dd MMM yyyy HH:mm XX",
"\\b[A-Z]\\S{2,8}, \\d{1,2} [A-Z]\\S{2,8} \\d{4} \\d{2}:\\d{2} ", "DATESTAMP_RFC2822", ""), "Tue, 15 May 2018 17:14 +0100",
1526400840000L);
checkAndValidateDateFormat(new TimestampMatch(14, "", "EEE MMM dd HH:mm:ss zzz YYYY",
validateTimestampMatch(new TimestampMatch(17, "", "EEE MMM dd HH:mm:ss zzz YYYY", "EEE MMM dd HH:mm:ss zzz yyyy",
"\\b[A-Z]\\S{2,8} [A-Z]\\S{2,8} \\d{1,2} \\d{2}:\\d{2}:\\d{2} [A-Z]{3,4} \\d{4}\\b", "DATESTAMP_OTHER", ""),
"Tue May 15 16:14:56 UTC 2018", 1526400896000L);
checkAndValidateDateFormat(new TimestampMatch(15, "", "EEE MMM dd HH:mm zzz YYYY",
validateTimestampMatch(new TimestampMatch(18, "", "EEE MMM dd HH:mm zzz YYYY", "EEE MMM dd HH:mm zzz yyyy",
"\\b[A-Z]\\S{2,8} [A-Z]\\S{2,8} \\d{1,2} \\d{2}:\\d{2} [A-Z]{3,4} \\d{4}\\b", "DATESTAMP_OTHER", ""),
"Tue May 15 16:14 UTC 2018", 1526400840000L);
checkAndValidateDateFormat(new TimestampMatch(16, "", "YYYYMMddHHmmss", "\\b\\d{14}\\b", "DATESTAMP_EVENTLOG", ""),
validateTimestampMatch(new TimestampMatch(19, "", "YYYYMMddHHmmss", "yyyyMMddHHmmss", "\\b\\d{14}\\b",
"DATESTAMP_EVENTLOG", ""),
"20180515171456", 1526400896000L);
checkAndValidateDateFormat(new TimestampMatch(17, "", "EEE MMM dd HH:mm:ss YYYY",
validateTimestampMatch(new TimestampMatch(20, "", "EEE MMM dd HH:mm:ss YYYY", "EEE MMM dd HH:mm:ss yyyy",
"\\b[A-Z]\\S{2,8} [A-Z]\\S{2,8} \\d{1,2} \\d{2}:\\d{2}:\\d{2} \\d{4}\\b", "HTTPDERROR_DATE", ""),
"Tue May 15 17:14:56 2018", 1526400896000L);
checkAndValidateDateFormat(new TimestampMatch(18, "", Arrays.asList("MMM dd HH:mm:ss.SSS", "MMM d HH:mm:ss.SSS"),
validateTimestampMatch(new TimestampMatch(21, "", Arrays.asList("MMM dd HH:mm:ss.SSS", "MMM d HH:mm:ss.SSS"),
Arrays.asList("MMM dd HH:mm:ss.SSS", "MMM d HH:mm:ss.SSS"),
"\\b[A-Z]\\S{2,8} {1,2}\\d{1,2} \\d{2}:\\d{2}:\\d{2}\\.\\d{3}", "SYSLOGTIMESTAMP", ""), "May 15 17:14:56.725", 1526400896725L);
checkAndValidateDateFormat(new TimestampMatch(19, "", Arrays.asList("MMM dd HH:mm:ss", "MMM d HH:mm:ss"),
validateTimestampMatch(new TimestampMatch(22, "", Arrays.asList("MMM dd HH:mm:ss", "MMM d HH:mm:ss"),
Arrays.asList("MMM dd HH:mm:ss", "MMM d HH:mm:ss"),
"\\b[A-Z]\\S{2,8} {1,2}\\d{1,2} \\d{2}:\\d{2}:\\d{2}\\b", "SYSLOGTIMESTAMP", ""), "May 15 17:14:56", 1526400896000L);
checkAndValidateDateFormat(new TimestampMatch(20, "", "dd/MMM/YYYY:HH:mm:ss Z",
validateTimestampMatch(new TimestampMatch(23, "", "dd/MMM/YYYY:HH:mm:ss Z", "dd/MMM/yyyy:HH:mm:ss XX",
"\\b\\d{2}/[A-Z]\\S{2}/\\d{4}:\\d{2}:\\d{2}:\\d{2} ", "HTTPDATE", ""), "15/May/2018:17:14:56 +0100", 1526400896000L);
checkAndValidateDateFormat(new TimestampMatch(21, "", "MMM dd, YYYY K:mm:ss a",
validateTimestampMatch(new TimestampMatch(24, "", "MMM dd, YYYY h:mm:ss a", "MMM dd, yyyy h:mm:ss a",
"\\b[A-Z]\\S{2,8} \\d{1,2}, \\d{4} \\d{1,2}:\\d{2}:\\d{2} [AP]M\\b", "CATALINA_DATESTAMP", ""), "May 15, 2018 5:14:56 PM",
1526400896000L);
checkAndValidateDateFormat(new TimestampMatch(22, "", Arrays.asList("MMM dd YYYY HH:mm:ss", "MMM d YYYY HH:mm:ss"),
validateTimestampMatch(new TimestampMatch(25, "", Arrays.asList("MMM dd YYYY HH:mm:ss", "MMM d YYYY HH:mm:ss"),
Arrays.asList("MMM dd yyyy HH:mm:ss", "MMM d yyyy HH:mm:ss"),
"\\b[A-Z]\\S{2,8} {1,2}\\d{1,2} \\d{4} \\d{2}:\\d{2}:\\d{2}\\b", "CISCOTIMESTAMP", ""), "May 15 2018 17:14:56",
1526400896000L);
}
public void testFindFirstMatchGivenOnlySystemDate() {
assertEquals(new TimestampMatch(23, "", "UNIX_MS", "\\b\\d{13}\\b", "POSINT", ""),
assertEquals(new TimestampMatch(26, "", "UNIX_MS", "UNIX_MS", "\\b\\d{13}\\b", "POSINT", ""),
TimestampFormatFinder.findFirstMatch("1526400896374"));
assertEquals(new TimestampMatch(23, "", "UNIX_MS", "\\b\\d{13}\\b", "POSINT", ""),
assertEquals(new TimestampMatch(26, "", "UNIX_MS", "UNIX_MS", "\\b\\d{13}\\b", "POSINT", ""),
TimestampFormatFinder.findFirstFullMatch("1526400896374"));
assertEquals(new TimestampMatch(24, "", "UNIX", "\\b\\d{10}\\.\\d{3,9}\\b", "NUMBER", ""),
assertEquals(new TimestampMatch(27, "", "UNIX", "UNIX", "\\b\\d{10}\\.\\d{3,9}\\b", "NUMBER", ""),
TimestampFormatFinder.findFirstMatch("1526400896.736"));
assertEquals(new TimestampMatch(24, "", "UNIX", "\\b\\d{10}\\.\\d{3,9}\\b", "NUMBER", ""),
assertEquals(new TimestampMatch(27, "", "UNIX", "UNIX", "\\b\\d{10}\\.\\d{3,9}\\b", "NUMBER", ""),
TimestampFormatFinder.findFirstFullMatch("1526400896.736"));
assertEquals(new TimestampMatch(25, "", "UNIX", "\\b\\d{10}\\b", "POSINT", ""),
assertEquals(new TimestampMatch(28, "", "UNIX", "UNIX", "\\b\\d{10}\\b", "POSINT", ""),
TimestampFormatFinder.findFirstMatch("1526400896"));
assertEquals(new TimestampMatch(25, "", "UNIX", "\\b\\d{10}\\b", "POSINT", ""),
assertEquals(new TimestampMatch(28, "", "UNIX", "UNIX", "\\b\\d{10}\\b", "POSINT", ""),
TimestampFormatFinder.findFirstFullMatch("1526400896"));
assertEquals(new TimestampMatch(26, "", "TAI64N", "\\b[0-9A-Fa-f]{24}\\b", "BASE16NUM", ""),
assertEquals(new TimestampMatch(29, "", "TAI64N", "TAI64N", "\\b[0-9A-Fa-f]{24}\\b", "BASE16NUM", ""),
TimestampFormatFinder.findFirstMatch("400000005afb159a164ac980"));
assertEquals(new TimestampMatch(26, "", "TAI64N", "\\b[0-9A-Fa-f]{24}\\b", "BASE16NUM", ""),
assertEquals(new TimestampMatch(29, "", "TAI64N", "TAI64N", "\\b[0-9A-Fa-f]{24}\\b", "BASE16NUM", ""),
TimestampFormatFinder.findFirstFullMatch("400000005afb159a164ac980"));
}
private void checkAndValidateDateFormat(TimestampMatch expected, String text, long expectedEpochMs) {
assertEquals(expected, TimestampFormatFinder.findFirstMatch(text));
assertEquals(expected, TimestampFormatFinder.findFirstFullMatch(text));
// All the test times are for Tue May 15 2018 16:14:56 UTC, which is 17:14:56 in London
DateTimeZone zone = DateTimeZone.forID("Europe/London");
DateTime parsed;
for (int i = 0; i < expected.dateFormats.size(); ++i) {
try {
String dateFormat = expected.dateFormats.get(i);
switch (dateFormat) {
case "ISO8601":
parsed = ISODateTimeFormat.dateTimeParser().withZone(zone).withDefaultYear(2018).parseDateTime(text);
break;
default:
DateTimeFormatter parser = DateTimeFormat.forPattern(dateFormat).withZone(zone).withLocale(Locale.UK);
parsed = parser.withDefaultYear(2018).parseDateTime(text);
break;
}
if (expectedEpochMs == parsed.getMillis()) {
break;
}
// If the last one isn't right then propagate
if (i == expected.dateFormats.size() - 1) {
assertEquals(expectedEpochMs, parsed.getMillis());
}
} catch (RuntimeException e) {
// If the last one throws then propagate
if (i == expected.dateFormats.size() - 1) {
throw e;
}
}
}
assertTrue(expected.simplePattern.matcher(text).find());
}
public void testFindFirstMatchGivenRealLogMessages() {
assertEquals(new TimestampMatch(7, "[", "ISO8601", "\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}", "TIMESTAMP_ISO8601",
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",
"][INFO ][o.e.e.NodeEnvironment ] [node-0] heap size [3.9gb], compressed ordinary object pointers [true]"),
TimestampFormatFinder.findFirstMatch("[2018-05-11T17:07:29,553][INFO ][o.e.e.NodeEnvironment ] [node-0] " +
"heap size [3.9gb], compressed ordinary object pointers [true]"));
assertEquals(new TimestampMatch(20, "192.168.62.101 - - [", "dd/MMM/YYYY:HH:mm:ss Z",
assertEquals(new TimestampMatch(23, "192.168.62.101 - - [", "dd/MMM/YYYY:HH:mm:ss Z", "dd/MMM/yyyy:HH:mm:ss XX",
"\\b\\d{2}/[A-Z]\\S{2}/\\d{4}:\\d{2}:\\d{2}:\\d{2} ", "HTTPDATE",
"] \"POST //apiserv:8080/engine/v2/jobs HTTP/1.1\" 201 42 \"-\" \"curl/7.46.0\" 384"),
TimestampFormatFinder.findFirstMatch("192.168.62.101 - - [29/Jun/2016:12:11:31 +0000] " +
"\"POST //apiserv:8080/engine/v2/jobs HTTP/1.1\" 201 42 \"-\" \"curl/7.46.0\" 384"));
assertEquals(new TimestampMatch(21, "", "MMM dd, YYYY K:mm:ss a",
assertEquals(new TimestampMatch(24, "", "MMM dd, YYYY h:mm:ss a", "MMM dd, yyyy h:mm:ss a",
"\\b[A-Z]\\S{2,8} \\d{1,2}, \\d{4} \\d{1,2}:\\d{2}:\\d{2} [AP]M\\b", "CATALINA_DATESTAMP",
" org.apache.tomcat.util.http.Parameters processParameters"),
TimestampFormatFinder.findFirstMatch("Aug 29, 2009 12:03:57 AM org.apache.tomcat.util.http.Parameters processParameters"));
assertEquals(new TimestampMatch(19, "", Arrays.asList("MMM dd HH:mm:ss", "MMM d HH:mm:ss"),
assertEquals(new TimestampMatch(22, "", Arrays.asList("MMM dd HH:mm:ss", "MMM d HH:mm:ss"),
Arrays.asList("MMM dd HH:mm:ss", "MMM d HH:mm:ss"),
"\\b[A-Z]\\S{2,8} {1,2}\\d{1,2} \\d{2}:\\d{2}:\\d{2}\\b", "SYSLOGTIMESTAMP", " esxi1.acme.com Vpxa: " +
"[3CB3FB90 verbose 'vpxavpxaInvtVm' opID=WFU-33d82c31] [VpxaInvtVmChangeListener] Guest DiskInfo Changed"),
TimestampFormatFinder.findFirstMatch("Oct 19 17:04:44 esxi1.acme.com Vpxa: [3CB3FB90 verbose 'vpxavpxaInvtVm' " +
"opID=WFU-33d82c31] [VpxaInvtVmChangeListener] Guest DiskInfo Changed"));
assertEquals(new TimestampMatch(7, "559550912540598297\t", "ISO8601", "\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}",
assertEquals(new TimestampMatch(10, "559550912540598297\t", "ISO8601", "ISO8601", "\\b\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}:\\d{2}",
"TIMESTAMP_ISO8601",
"\t2016-04-20T21:06:53Z\t38545844\tserv02nw07\t192.168.114.28\tAuthpriv\tInfo\tsshd\tsubsystem request for sftp"),
TimestampFormatFinder.findFirstMatch("559550912540598297\t2016-04-20T14:06:53\t2016-04-20T21:06:53Z\t38545844\tserv02nw07\t" +
"192.168.114.28\tAuthpriv\tInfo\tsshd\tsubsystem request for sftp"));
assertEquals(new TimestampMatch(19, "", Arrays.asList("MMM dd HH:mm:ss", "MMM d HH:mm:ss"),
assertEquals(new TimestampMatch(22, "", Arrays.asList("MMM dd HH:mm:ss", "MMM d HH:mm:ss"),
Arrays.asList("MMM dd HH:mm:ss", "MMM d HH:mm:ss"),
"\\b[A-Z]\\S{2,8} {1,2}\\d{1,2} \\d{2}:\\d{2}:\\d{2}\\b", "SYSLOGTIMESTAMP",
" dnsserv named[22529]: error (unexpected RCODE REFUSED) resolving 'www.elastic.co/A/IN': 95.110.68.206#53"),
TimestampFormatFinder.findFirstMatch("Sep 8 11:55:35 dnsserv named[22529]: error (unexpected RCODE REFUSED) resolving " +
"'www.elastic.co/A/IN': 95.110.68.206#53"));
assertEquals(new TimestampMatch(3, "", "YYYY-MM-dd HH:mm:ss.SSSSSS", "\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}\\.\\d{3}",
"TIMESTAMP_ISO8601",
assertEquals(new TimestampMatch(3, "", "YYYY-MM-dd HH:mm:ss.SSSSSS", "yyyy-MM-dd HH:mm:ss.SSSSSS",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}\\.\\d{3}", "TIMESTAMP_ISO8601",
"|INFO |VirtualServer |1 |client 'User1'(id:2) was added to channelgroup 'Channel Admin'(id:5) by client " +
"'User1'(id:2) in channel '3er Instanz'(id:2)"),
TimestampFormatFinder.findFirstMatch("2018-01-06 19:22:20.106822|INFO |VirtualServer |1 |client " +
" 'User1'(id:2) was added to channelgroup 'Channel Admin'(id:5) by client 'User1'(id:2) in channel '3er Instanz'(id:2)"));
// Differs from the above as the required format is specified
assertEquals(new TimestampMatch(3, "", "YYYY-MM-dd HH:mm:ss.SSSSSS", "yyyy-MM-dd HH:mm:ss.SSSSSS",
"\\b\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}\\.\\d{3}", "TIMESTAMP_ISO8601",
"|INFO |VirtualServer |1 |client 'User1'(id:2) was added to channelgroup 'Channel Admin'(id:5) by client " +
"'User1'(id:2) in channel '3er Instanz'(id:2)"),
TimestampFormatFinder.findFirstMatch("2018-01-06 19:22:20.106822|INFO |VirtualServer |1 |client " +
" 'User1'(id:2) was added to channelgroup 'Channel Admin'(id:5) by client 'User1'(id:2) in channel '3er Instanz'(id:2)",
randomFrom("YYYY-MM-dd HH:mm:ss.SSSSSS", "yyyy-MM-dd HH:mm:ss.SSSSSS")));
// Non-matching required format specified
assertNull(TimestampFormatFinder.findFirstMatch("2018-01-06 19:22:20.106822|INFO |VirtualServer |1 |client " +
" 'User1'(id:2) was added to channelgroup 'Channel Admin'(id:5) by client 'User1'(id:2) in channel '3er Instanz'(id:2)",
randomFrom("UNIX", "EEE MMM dd YYYY HH:mm zzz")));
}
public void testAdjustRequiredFormat() {
assertEquals("YYYY-MM-dd HH:mm:ss,SSS Z", TimestampFormatFinder.adjustRequiredFormat("YYYY-MM-dd HH:mm:ss,SSS Z"));
assertEquals("YYYY-MM-dd HH:mm:ss,SSS Z", TimestampFormatFinder.adjustRequiredFormat("YYYY-MM-dd HH:mm:ss,SSSSSS Z"));
assertEquals("YYYY-MM-dd HH:mm:ss,SSS Z", TimestampFormatFinder.adjustRequiredFormat("YYYY-MM-dd HH:mm:ss,SSSSSSSSS Z"));
assertEquals("YYYY-MM-dd HH:mm:ss,SSS Z", TimestampFormatFinder.adjustRequiredFormat("YYYY-MM-dd HH:mm:ss.SSS Z"));
assertEquals("YYYY-MM-dd HH:mm:ss,SSS Z", TimestampFormatFinder.adjustRequiredFormat("YYYY-MM-dd HH:mm:ss.SSSSSS Z"));
assertEquals("YYYY-MM-dd HH:mm:ss,SSS Z", TimestampFormatFinder.adjustRequiredFormat("YYYY-MM-dd HH:mm:ss.SSSSSSSSS Z"));
assertEquals("YYYY-MM-dd HH:mm:ss,SSS", TimestampFormatFinder.adjustRequiredFormat("YYYY-MM-dd HH:mm:ss,SSS"));
assertEquals("YYYY-MM-dd HH:mm:ss,SSS", TimestampFormatFinder.adjustRequiredFormat("YYYY-MM-dd HH:mm:ss,SSSSSS"));
assertEquals("YYYY-MM-dd HH:mm:ss,SSS", TimestampFormatFinder.adjustRequiredFormat("YYYY-MM-dd HH:mm:ss,SSSSSSSSS"));
assertEquals("YYYY-MM-dd HH:mm:ss,SSS", TimestampFormatFinder.adjustRequiredFormat("YYYY-MM-dd HH:mm:ss.SSS"));
assertEquals("YYYY-MM-dd HH:mm:ss,SSS", TimestampFormatFinder.adjustRequiredFormat("YYYY-MM-dd HH:mm:ss.SSSSSS"));
assertEquals("YYYY-MM-dd HH:mm:ss,SSS", TimestampFormatFinder.adjustRequiredFormat("YYYY-MM-dd HH:mm:ss.SSSSSSSSS"));
}
public void testInterpretFractionalSeconds() {
@ -239,4 +243,112 @@ public class TimestampFormatFinderTests extends FileStructureTestCase {
assertEquals(new Tuple<>(',', 3), TimestampFormatFinder.interpretFractionalSeconds("2018-01-06T17:21:25,764 Z"));
assertEquals(new Tuple<>('.', 3), TimestampFormatFinder.interpretFractionalSeconds("2018-01-06T17:21:25.764 Z"));
}
private void validateTimestampMatch(TimestampMatch expected, String text, long expectedEpochMs) {
assertEquals(expected, TimestampFormatFinder.findFirstMatch(text));
assertEquals(expected, TimestampFormatFinder.findFirstFullMatch(text));
assertEquals(expected, TimestampFormatFinder.findFirstMatch(text, expected.candidateIndex));
assertEquals(expected, TimestampFormatFinder.findFirstFullMatch(text, expected.candidateIndex));
assertNull(TimestampFormatFinder.findFirstMatch(text, Integer.MAX_VALUE));
assertNull(TimestampFormatFinder.findFirstFullMatch(text, Integer.MAX_VALUE));
assertEquals(expected, TimestampFormatFinder.findFirstMatch(text, randomFrom(expected.jodaTimestampFormats)));
assertEquals(expected, TimestampFormatFinder.findFirstFullMatch(text, randomFrom(expected.jodaTimestampFormats)));
assertEquals(expected, TimestampFormatFinder.findFirstMatch(text, randomFrom(expected.javaTimestampFormats)));
assertEquals(expected, TimestampFormatFinder.findFirstFullMatch(text, randomFrom(expected.javaTimestampFormats)));
assertNull(TimestampFormatFinder.findFirstMatch(text, "wrong format"));
assertNull(TimestampFormatFinder.findFirstFullMatch(text, "wrong format"));
validateJodaTimestampFormats(expected.jodaTimestampFormats, text, expectedEpochMs);
validateJavaTimestampFormats(expected.javaTimestampFormats, text, expectedEpochMs);
assertTrue(expected.simplePattern.matcher(text).find());
}
private void validateJodaTimestampFormats(List<String> jodaTimestampFormats, String text, long expectedEpochMs) {
// All the test times are for Tue May 15 2018 16:14:56 UTC, which is 17:14:56 in London.
// This is the timezone that will be used for any text representations that don't include it.
org.joda.time.DateTimeZone defaultZone = org.joda.time.DateTimeZone.forID("Europe/London");
org.joda.time.DateTime parsed;
for (int i = 0; i < jodaTimestampFormats.size(); ++i) {
try {
String timestampFormat = jodaTimestampFormats.get(i);
switch (timestampFormat) {
case "ISO8601":
parsed = org.joda.time.format.ISODateTimeFormat.dateTimeParser()
.withZone(defaultZone).withDefaultYear(2018).parseDateTime(text);
break;
default:
org.joda.time.format.DateTimeFormatter parser =
org.joda.time.format.DateTimeFormat.forPattern(timestampFormat).withZone(defaultZone).withLocale(Locale.ROOT);
parsed = parser.withDefaultYear(2018).parseDateTime(text);
break;
}
if (expectedEpochMs == parsed.getMillis()) {
break;
}
// If the last one isn't right then propagate
if (i == jodaTimestampFormats.size() - 1) {
assertEquals(expectedEpochMs, parsed.getMillis());
}
} catch (RuntimeException e) {
// If the last one throws then propagate
if (i == jodaTimestampFormats.size() - 1) {
throw e;
}
}
}
}
private void validateJavaTimestampFormats(List<String> javaTimestampFormats, String text, long expectedEpochMs) {
// All the test times are for Tue May 15 2018 16:14:56 UTC, which is 17:14:56 in London.
// This is the timezone that will be used for any text representations that don't include it.
java.time.ZoneId defaultZone = java.time.ZoneId.of("Europe/London");
java.time.temporal.TemporalAccessor parsed;
for (int i = 0; i < javaTimestampFormats.size(); ++i) {
try {
String timestampFormat = javaTimestampFormats.get(i);
switch (timestampFormat) {
case "ISO8601":
parsed = DateFormatters.forPattern("strict_date_optional_time_nanos").withZone(defaultZone).parse(text);
break;
default:
java.time.format.DateTimeFormatter parser = new java.time.format.DateTimeFormatterBuilder()
.appendPattern(timestampFormat).parseDefaulting(java.time.temporal.ChronoField.YEAR_OF_ERA, 2018)
.toFormatter(Locale.ROOT);
// This next line parses the textual date without any default timezone, so if
// the text doesn't contain the timezone then the resulting temporal accessor
// will be incomplete (i.e. impossible to convert to an Instant). You would
// hope that it would be possible to specify a timezone to be used only in this
// case, and in Java 9 and 10 it is, by adding withZone(zone) before the
// parse(text) call. However, with Java 8 this overrides any timezone parsed
// from the text. The solution is to parse twice, once without a default
// timezone and then again with a default timezone if the first parse didn't
// find one in the text.
parsed = parser.parse(text);
if (parsed.query(java.time.temporal.TemporalQueries.zone()) == null) {
// TODO: when Java 8 is no longer supported remove the two
// lines and comment above and the closing brace below
parsed = parser.withZone(defaultZone).parse(text);
}
break;
}
long actualEpochMs = java.time.Instant.from(parsed).toEpochMilli();
if (expectedEpochMs == actualEpochMs) {
break;
}
// If the last one isn't right then propagate
if (i == javaTimestampFormats.size() - 1) {
assertEquals(expectedEpochMs, actualEpochMs);
}
} catch (RuntimeException e) {
// If the last one throws then propagate
if (i == javaTimestampFormats.size() - 1) {
throw e;
}
}
}
}
}

View File

@ -38,6 +38,6 @@ public class XmlFileStructureFinderTests extends FileStructureTestCase {
assertNull(structure.getShouldTrimFields());
assertNull(structure.getGrokPattern());
assertEquals("timestamp", structure.getTimestampField());
assertEquals(Collections.singletonList("UNIX_MS"), structure.getTimestampFormats());
assertEquals(Collections.singletonList("UNIX_MS"), structure.getJodaTimestampFormats());
}
}

View File

@ -49,7 +49,7 @@
},
"timestamp_format": {
"type": "string",
"description": "Optional parameter to specify the timestamp format in the file"
"description": "Optional parameter to specify the timestamp format in the file - may be either a Joda or Java time format"
},
"explain": {
"type": "boolean",

View File

@ -27,7 +27,8 @@
- match: { has_byte_order_marker: false }
- match: { format: json }
- match: { timestamp_field: time }
- match: { timestamp_formats.0: UNIX }
- match: { joda_timestamp_formats.0: UNIX }
- match: { java_timestamp_formats.0: UNIX }
- match: { need_client_timezone: false }
- match: { mappings.airline.type: keyword }
- match: { mappings.responsetime.type: double }
@ -83,7 +84,8 @@
- match: { has_byte_order_marker: false }
- match: { format: json }
- match: { timestamp_field: time }
- match: { timestamp_formats.0: UNIX }
- match: { joda_timestamp_formats.0: UNIX }
- match: { java_timestamp_formats.0: UNIX }
- match: { need_client_timezone: false }
- match: { mappings.airline.type: keyword }
- match: { mappings.responsetime.type: double }