diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DurationInfo.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DurationInfo.java index 490562bf54ece..e511543aff13e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DurationInfo.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DurationInfo.java @@ -93,7 +93,7 @@ public String toString() { @Override public void close() { - finished(); + super.close(); if (logAtInfo) { log.info("{}", this); } else { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java index 1fb920e99f08e..bfee812d35d29 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java @@ -22,13 +22,14 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.DurationTracker; /** * Little duration counter. */ @InterfaceAudience.Public @InterfaceStability.Unstable -public class OperationDuration { +public class OperationDuration implements DurationTracker { /** * Time in millis when the operation started. @@ -65,6 +66,16 @@ public void finished() { finished = time(); } + @Override + public void failed() { + finished(); + } + + @Override + public void close() { + finished(); + } + /** * Return the duration as {@link #humanTime(long)}. * @return a printable duration. diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml index 11410a24c1ad2..39661ee002792 100644 --- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml +++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml @@ -86,4 +86,9 @@ + + + + + diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index f10eea7a4d3bf..ed4a88ce0542f 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -419,6 +419,7 @@ org.apache.hadoop.fs.s3a.commit.impl.* org.apache.hadoop.fs.s3a.commit.magic.* org.apache.hadoop.fs.s3a.commit.staging.* + org.apache.hadoop.fs.s3a.audit.mapreduce.* org.apache.hadoop.mapreduce.** @@ -440,6 +441,25 @@ + + + + org.apache.avro + avro-maven-plugin + + + generate-avro-sources + generate-sources + + schema + + + + + src/main/avro + ${project.build.directory}/generated-sources/avro + + diff --git a/hadoop-tools/hadoop-aws/src/main/avro/AvroDataSchema.avsc b/hadoop-tools/hadoop-aws/src/main/avro/AvroDataSchema.avsc new file mode 100644 index 0000000000000..860796460a86c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/avro/AvroDataSchema.avsc @@ -0,0 +1,48 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +{ + "type" : "record", "name" : "AvroS3LogEntryRecord", + "namespace" : "org.apache.hadoop.fs.s3a.audit", + "fields" : [ + { "name" : "owner", "type" : "string" }, + { "name" : "bucket", "type" : "string" }, + { "name" : "timestamp", "type" : "string" }, + { "name" : "remoteip", "type" : "string" }, + { "name" : "requester", "type" : "string" }, + { "name" : "requestid", "type" : "string" }, + { "name" : "verb", "type" : "string" }, + { "name" : "key", "type" : "string" }, + { "name" : "requesturi", "type" : "string" }, + { "name" : "http", "type" : "string" }, + { "name" : "awserrorcode", "type" : "string" }, + { "name" : "bytessent", "type" : ["long", "null"] }, + { "name" : "objectsize", "type" : ["long", "null"] }, + { "name" : "totaltime", "type" : ["long", "null"] }, + { "name" : "turnaroundtime" , "type" : ["long", "null"] }, + { "name" : "referrer", "type" : "string" }, + { "name" : "useragent", "type" : "string" }, + { "name" : "version", "type" : "string" }, + { "name" : "hostid", "type" : "string" }, + { "name" : "sigv", "type" : "string" }, + { "name" : "cypher", "type" : "string" }, + { "name" : "auth", "type" : "string" }, + { "name" : "endpoint", "type" : "string" }, + { "name" : "tls", "type" : "string" }, + { "name" : "tail", "type" : "string" }, + { "name" : "referrerMap", "type" : {"type": "map", "values": "string"} } + ] +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditTool.java new file mode 100644 index 0000000000000..9c5337fc2316b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AuditTool.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +import java.io.IOException; +import java.io.PrintStream; +import java.util.Arrays; +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.audit.mapreduce.S3AAuditLogMergerAndParser; +import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; +import org.apache.hadoop.util.ExitUtil; + +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_COMMAND_ARGUMENT_ERROR; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_FAIL; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_SUCCESS; + +/** + * AuditTool is a Command Line Interface. + * Its functionality is to parse the audit log files + * and generate avro file. + */ +public class AuditTool extends S3GuardTool { + + private static final Logger LOG = LoggerFactory.getLogger(AuditTool.class); + + /** + * Name of audit tool: {@value}. + */ + public static final String AUDIT = "audit"; + + + /** + * Name of this tool: {@value}. + */ + public static final String AUDIT_TOOL = + "org.apache.hadoop.fs.s3a.audit.AuditTool"; + + /** + * Purpose of this tool: {@value}. + */ + public static final String PURPOSE = + "\n\nUSAGE:\nMerge and parse audit log files and convert into avro files " + + "for " + + "better " + + "visualization"; + + // Exit codes + private static final int SUCCESS = EXIT_SUCCESS; + + private static final int FAILURE = EXIT_FAIL; + + private static final int INVALID_ARGUMENT = EXIT_COMMAND_ARGUMENT_ERROR; + + private static final int SAMPLE = 500; + + private static final String USAGE = + "hadoop " + AUDIT_TOOL + + " " + + " " + + "\n"; + + private PrintStream out; + + public AuditTool(final Configuration conf) { + super(conf); + } + + /** + * Tells us the usage of the AuditTool by commands. + * @return the string USAGE + */ + public String getUsage() { + return USAGE + PURPOSE; + } + + public String getName() { + return AUDIT_TOOL; + } + + /** + * This run method in AuditTool takes source and destination path of bucket, + * and checks if there are directories and pass these paths to merge and + * parse audit log files. + * @param args argument list + * @param stream output stream + * @return SUCCESS i.e, '0', which is an exit code + * @throws Exception on any failure. + */ + @Override + public int run(final String[] args, final PrintStream stream) + throws ExitUtil.ExitException, Exception { + + this.out = stream; + + preConditionArgsSizeCheck(args); + List paths = Arrays.asList(args); + + // Path of audit log files + Path logsPath = new Path(paths.get(0)); + + // Path of destination file + Path destPath = new Path(paths.get(1)); + + final S3AAuditLogMergerAndParser auditLogMergerAndParser = + new S3AAuditLogMergerAndParser(getConf(), SAMPLE); + + // Calls S3AAuditLogMergerAndParser for implementing merging, passing of + // audit log files and converting into avro file + boolean mergeAndParseResult = + auditLogMergerAndParser.mergeAndParseAuditLogFiles( + logsPath, destPath); + if (!mergeAndParseResult) { + return FAILURE; + } + + return SUCCESS; + } + + private void preConditionArgsSizeCheck(String[] args) { + if (args.length != 2) { + errorln(getUsage()); + throw invalidArgs("Invalid number of arguments"); + } + } + + + /** + * Flush all active output channels, including {@code System.err}, + * so as to stay in sync with any JRE log messages. + */ + private void flush() { + if (out != null) { + out.flush(); + } else { + System.out.flush(); + } + System.err.flush(); + } + + + public void closeOutput() throws IOException { + flush(); + if (out != null) { + out.close(); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/ParsedAuditLogEntry.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/ParsedAuditLogEntry.java new file mode 100644 index 0000000000000..83849d8d9a506 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/ParsedAuditLogEntry.java @@ -0,0 +1,608 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableUtils; + +import static org.apache.hadoop.io.WritableUtils.writeString; + +/** + * Log entry which can be serialized as java Serializable or hadoop Writable. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class ParsedAuditLogEntry implements Serializable, Writable { + + private static final long serialVersionUID = 0xf00f_f00fL; + + private static final int VERSION = 0xf00f; + + private String owner; + + private String bucket; + + private String timestamp; + + private String remoteip; + + private String requester; + + private String requestid; + + private String verb; + + private String key; + + private String requesturi; + + private String http; + + private String awserrorcode; + + private long bytessent; + + private long objectsize; + + private long totaltime; + + private long turnaroundtime; + + private String referrer; + + private String useragent; + + private String version; + + private String hostid; + + private String sigv; + + private String cypher; + + private String auth; + + private String endpoint; + + private String tls; + + private String tail; + + private Map referrerMap; + + public ParsedAuditLogEntry() { + } + + /** + * Build from an avro record. + * @param record record + */ + public ParsedAuditLogEntry(AvroS3LogEntryRecord record) { + fromAvro(record); + } + + + public String getOwner() { + return owner; + } + + public void setOwner(final String owner) { + this.owner = owner; + } + + public String getAuth() { + return auth; + } + + public void setAuth(final String auth) { + this.auth = auth; + } + + public String getAwserrorcode() { + return awserrorcode; + } + + public void setAwserrorcode(final String awserrorcode) { + this.awserrorcode = awserrorcode; + } + + public String getBucket() { + return bucket; + } + + public void setBucket(final String bucket) { + this.bucket = bucket; + } + + public long getBytessent() { + return bytessent; + } + + public void setBytessent(final long bytessent) { + this.bytessent = bytessent; + } + + public String getCypher() { + return cypher; + } + + public void setCypher(final String cypher) { + this.cypher = cypher; + } + + public String getEndpoint() { + return endpoint; + } + + public void setEndpoint(final String endpoint) { + this.endpoint = endpoint; + } + + public String getHostid() { + return hostid; + } + + public void setHostid(final String hostid) { + this.hostid = hostid; + } + + public String getHttp() { + return http; + } + + public void setHttp(final String http) { + this.http = http; + } + + public String getKey() { + return key; + } + + public void setKey(final String key) { + this.key = key; + } + + public long getObjectsize() { + return objectsize; + } + + public void setObjectsize(final long objectsize) { + this.objectsize = objectsize; + } + + public String getReferrer() { + return referrer; + } + + public void setReferrer(final String referrer) { + this.referrer = referrer; + } + + public Map getReferrerMap() { + return referrerMap; + } + + public void setReferrerMap(final Map referrerMap) { + this.referrerMap = referrerMap; + } + + /** + * Get an audit entry, returning the default value if the referrer map is + * null or there is no entry of that name. + * @param name entry name + * @param defVal default value + * @return the value or the default + */ + public String getAuditEntry(String name, String defVal) { + if (referrerMap == null) { + return null; + } + return referrerMap.getOrDefault(name, defVal); + } + + /** + * Get an audit entry, returning the default value if the referrer map is + * null or there is no entry of that name. + * @param name entry name + * @param defVal default value + * @return the value or the default + */ + public boolean hasAuditEntry(String name) { + return referrerMap != null && + referrerMap.containsKey(name); + } + + public String getRemoteip() { + return remoteip; + } + + public void setRemoteip(final String remoteip) { + this.remoteip = remoteip; + } + + public String getRequester() { + return requester; + } + + public void setRequester(final String requester) { + this.requester = requester; + } + + public String getRequestid() { + return requestid; + } + + public void setRequestid(final String requestid) { + this.requestid = requestid; + } + + public String getRequesturi() { + return requesturi; + } + + public void setRequesturi(final String requesturi) { + this.requesturi = requesturi; + } + + public String getSigv() { + return sigv; + } + + public void setSigv(final String sigv) { + this.sigv = sigv; + } + + public String getTail() { + return tail; + } + + public void setTail(final String tail) { + this.tail = tail; + } + + public String getTimestamp() { + return timestamp; + } + + public void setTimestamp(final String timestamp) { + this.timestamp = timestamp; + } + + public String getTls() { + return tls; + } + + public void setTls(final String tls) { + this.tls = tls; + } + + public long getTotaltime() { + return totaltime; + } + + public void setTotaltime(final long totaltime) { + this.totaltime = totaltime; + } + + public long getTurnaroundtime() { + return turnaroundtime; + } + + public void setTurnaroundtime(final long turnaroundtime) { + this.turnaroundtime = turnaroundtime; + } + + public String getUseragent() { + return useragent; + } + + public void setUseragent(final String useragent) { + this.useragent = useragent; + } + + public String getVerb() { + return verb; + } + + public void setVerb(final String verb) { + this.verb = verb; + } + + public String getVersion() { + return version; + } + + public void setVersion(final String version) { + this.version = version; + } + + /** + * Build from an avro record. + *

+ * All {@code CharSequence} fields are converted to strings, + * and the referrer map is rebuilt. + * If any of the {@code Long} fields are null, they are set to 0. + * @param record source record + */ + public void fromAvro(AvroS3LogEntryRecord record) { + + owner = String.valueOf(record.getOwner()); + bucket = String.valueOf(record.getBucket()); + timestamp = String.valueOf(record.getTimestamp()); + remoteip = String.valueOf(record.getRemoteip()); + requester = String.valueOf(record.getRequester()); + requestid = String.valueOf(record.getRequestid()); + verb = String.valueOf(record.getVerb()); + key = String.valueOf(record.getKey()); + requesturi = String.valueOf(record.getRequesturi()); + http = String.valueOf(record.getHttp()); + awserrorcode = String.valueOf(record.getAwserrorcode()); + bytessent = longValue(record.getBytessent()); + objectsize = longValue(record.getObjectsize()); + totaltime = longValue(record.getTotaltime()); + turnaroundtime = longValue(record.getTurnaroundtime()); + referrer = String.valueOf(record.getReferrer()); + useragent = String.valueOf(record.getUseragent()); + version = String.valueOf(record.getVersion()); + hostid = String.valueOf(record.getHostid()); + sigv = String.valueOf(record.getSigv()); + cypher = String.valueOf(record.getCypher()); + auth = String.valueOf(record.getAuth()); + endpoint = String.valueOf(record.getEndpoint()); + tls = String.valueOf(record.getTls()); + tail = String.valueOf(record.getTail()); + // copy the entries + final Map entries = record.getReferrerMap(); + referrerMap = new HashMap<>(entries.size()); + entries.forEach((k, v) -> + referrerMap.put(String.valueOf(k), String.valueOf(v))); + } + + /** + * get a long value if the source is not null. + * @param l Long source + * @return either the long value or 0 + */ + private long longValue(Long l) { + return l != null ? l : 0; + } + + /** + * Fill in an avro record. + * @param record record to update + * @return the record + */ + public AvroS3LogEntryRecord toAvro(AvroS3LogEntryRecord record) { + record.setOwner(owner); + record.setBucket(bucket); + record.setTimestamp(timestamp); + record.setRemoteip(remoteip); + record.setRequester(requester); + record.setRequestid(requestid); + record.setVerb(verb); + record.setKey(key); + record.setRequesturi(requesturi); + record.setHttp(http); + record.setAwserrorcode(awserrorcode); + record.setBytessent(bytessent); + record.setObjectsize(objectsize); + record.setTotaltime(totaltime); + record.setTurnaroundtime(turnaroundtime); + record.setReferrer(referrer); + record.setUseragent(useragent); + record.setVersion(version); + record.setHostid(hostid); + record.setSigv(sigv); + record.setCypher(cypher); + record.setAuth(auth); + record.setEndpoint(endpoint); + record.setTls(tls); + record.setTail(tail); + Map entries = new HashMap<>(referrerMap.size()); + entries.putAll(referrerMap); + record.setReferrerMap(entries); + return record; + } + + @Override + public void readFields(DataInput in) throws IOException { + long ver = in.readLong(); + if (ver != serialVersionUID) { + throw new IOException("Unknown version of ParsedAuditLogEntry: " + + ver); + } + owner = readStr(in); + bucket = readStr(in); + timestamp = readStr(in); + remoteip = readStr(in); + requester = readStr(in); + requestid = readStr(in); + verb = readStr(in); + key = readStr(in); + requesturi = readStr(in); + http = readStr(in); + awserrorcode = readStr(in); + bytessent = in.readLong(); + objectsize = in.readLong(); + totaltime = in.readLong(); + turnaroundtime = in.readLong(); + referrer = readStr(in); + useragent = readStr(in); + version = readStr(in); + hostid = readStr(in); + sigv = readStr(in); + cypher = readStr(in); + auth = readStr(in); + endpoint = readStr(in); + tls = readStr(in); + tail = readStr(in); + // read the referrer map + final int size = in.readInt(); + referrerMap = new HashMap<>(size); + for (int i = 0; i < size; i++) { + referrerMap.put(readStr(in), readStr(in)); + } + } + + + @Override + public void write(final DataOutput out) throws IOException { + + out.writeLong(serialVersionUID); + final String s = owner; + write(out, s); + write(out, bucket); + write(out, timestamp); + write(out, remoteip); + write(out, requester); + write(out, requestid); + write(out, verb); + write(out, key); + write(out, requesturi); + write(out, http); + write(out, awserrorcode); + out.writeLong(bytessent); + out.writeLong(objectsize); + out.writeLong(totaltime); + out.writeLong(turnaroundtime); + write(out, referrer); + write(out, useragent); + write(out, version); + write(out, hostid); + write(out, sigv); + write(out, cypher); + write(out, auth); + write(out, endpoint); + write(out, tls); + write(out, tail); + + // write the referrer map + out.writeInt(referrerMap.size()); + for (Map.Entry entry : referrerMap.entrySet()) { + write(out, entry.getKey()); + write(out, entry.getValue()); + } + + } + + private static void write(final DataOutput out, final String s) throws IOException { + writeString(out, s); + } + + private String readStr(final DataInput in) throws IOException { + return WritableUtils.readString(in); + } + + /** + * Deep equality test on all values. + * @param that the object to test against + * @return true iff everything matches + */ + public boolean deepEquals(final ParsedAuditLogEntry that) { + if (this == that) { + return true; + } + return bytessent == that.bytessent + && objectsize == that.objectsize + && totaltime == that.totaltime + && turnaroundtime == that.turnaroundtime + && Objects.equals(owner, that.owner) + && Objects.equals(bucket, that.bucket) + && Objects.equals(timestamp, that.timestamp) + && Objects.equals(remoteip, that.remoteip) + && Objects.equals(requester, that.requester) + && Objects.equals(requestid, that.requestid) + && Objects.equals(verb, that.verb) + && Objects.equals(key, that.key) + && Objects.equals(requesturi, that.requesturi) + && Objects.equals(http, that.http) + && Objects.equals(awserrorcode, that.awserrorcode) + && Objects.equals(referrer, that.referrer) + && Objects.equals(useragent, that.useragent) + && Objects.equals(version, that.version) + && Objects.equals(hostid, that.hostid) + && Objects.equals(sigv, that.sigv) + && Objects.equals(cypher, that.cypher) + && Objects.equals(auth, that.auth) + && Objects.equals(endpoint, that.endpoint) + && Objects.equals(tls, that.tls) + && Objects.equals(tail, that.tail) + && Objects.equals(referrerMap, that.referrerMap); // relies on hashmap equality + } + + /** + * Equality is based on the request ID only. + * @param o other object + * @return true iff the request IDs match + */ + @Override + public boolean equals(final Object o) { + if (this == o) {return true;} + if (!(o instanceof ParsedAuditLogEntry)) {return false;} + ParsedAuditLogEntry that = (ParsedAuditLogEntry) o; + return Objects.equals(requestid, that.requestid); + } + + /** + * Hashcode is based on the request ID only. + * @return hash code + */ + @Override + public int hashCode() { + return Objects.hashCode(requestid); + } + + @Override + public String toString() { + return "ParsedAuditLogEntry{" + + "auth='" + auth + '\'' + + ", awserrorcode='" + awserrorcode + '\'' + + ", bucket='" + bucket + '\'' + + ", bytessent=" + bytessent + + ", endpoint='" + endpoint + '\'' + + ", hostid='" + hostid + '\'' + + ", http='" + http + '\'' + + ", key='" + key + '\'' + + ", objectsize=" + objectsize + + ", owner='" + owner + '\'' + + ", referrer='" + referrer + '\'' + + ", referrerMap=" + referrerMap + + ", remoteip='" + remoteip + '\'' + + ", requester='" + requester + '\'' + + ", requestid='" + requestid + '\'' + + ", requesturi='" + requesturi + '\'' + + ", tail='" + tail + '\'' + + ", timestamp='" + timestamp + '\'' + + ", totaltime=" + totaltime + + ", turnaroundtime=" + turnaroundtime + + ", useragent='" + useragent + '\'' + + ", verb='" + verb + '\'' + + '}'; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogParser.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogParser.java index ac29234a771e9..f755ff572c4b4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogParser.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/S3LogParser.java @@ -29,7 +29,7 @@ /** * Class to help parse AWS S3 Logs. * see https://docs.aws.amazon.com/AmazonS3/latest/userguide/LogFormat.html - * + *

* Getting the regexp right is surprisingly hard; this class does it * explicitly and names each group in the process. * All group names are included in {@link #AWS_LOG_REGEXP_GROUPS} in the order diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/mapreduce/S3AAuditLogMergerAndParser.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/mapreduce/S3AAuditLogMergerAndParser.java new file mode 100644 index 0000000000000..a2979f674e2b2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/mapreduce/S3AAuditLogMergerAndParser.java @@ -0,0 +1,348 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit.mapreduce; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.s3a.audit.AvroS3LogEntryRecord; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.LineRecordReader; +import org.apache.hadoop.util.DurationInfo; + +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE; +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE; +import static org.apache.hadoop.fs.s3a.audit.S3LogParser.AWS_LOG_REGEXP_GROUPS; +import static org.apache.hadoop.fs.s3a.audit.S3LogParser.BYTESSENT_GROUP; +import static org.apache.hadoop.fs.s3a.audit.S3LogParser.LOG_ENTRY_PATTERN; +import static org.apache.hadoop.fs.s3a.audit.S3LogParser.OBJECTSIZE_GROUP; +import static org.apache.hadoop.fs.s3a.audit.S3LogParser.TOTALTIME_GROUP; +import static org.apache.hadoop.fs.s3a.audit.S3LogParser.TURNAROUNDTIME_GROUP; +import static org.apache.hadoop.util.functional.FutureIO.awaitFuture; + +/** + * Merge all the audit logs present in a directory of + * multiple audit log files into a single audit log file. + */ +public class S3AAuditLogMergerAndParser { + + /** + * Max length of a line in the audit log: {@value}. + */ + public static final int MAX_LINE_LENGTH = 32000; + + /** + * List of fields in a log record which are of type long. + */ + public static final List FIELDS_OF_TYPE_LONG = + Arrays.asList(TURNAROUNDTIME_GROUP, BYTESSENT_GROUP, + OBJECTSIZE_GROUP, TOTALTIME_GROUP); + + private static final Logger LOG = + LoggerFactory.getLogger(S3AAuditLogMergerAndParser.class); + + /** + * Referrer header key from the regexp, not the actual header name + * which is spelled differently: {@value}. + */ + private static final String REFERRER_HEADER_KEY = "referrer"; + + /** + * Value to use when a long value cannot be parsed: {@value}. + */ + public static final long FAILED_TO_PARSE_LONG = -1L; + + /** + * Key for the referrer map in the Avro record: {@value}. + */ + public static final String REFERRER_MAP = "referrerMap"; + + private final Configuration conf; + /* + * Number of records to process before giving a status update. + */ + private final int sample; + + // Basic parsing counters. + + /** + * Number of audit log files parsed. + */ + private long logFilesParsed = 0; + + /** + * Number of log entries parsed. + */ + private long auditLogsParsed = 0; + + /** + * How many referrer headers were parsed. + */ + private long referrerHeaderLogParsed = 0; + + /** + * How many records were skipped due to lack of referrer or other reason. + */ + private long recordsSkipped = 0; + + public S3AAuditLogMergerAndParser(final Configuration conf, final int sample) { + this.conf = conf; + + this.sample = sample; + } + + /** + * parseAuditLog method helps in parsing the audit log + * into key-value pairs using regular expressions. + * @param singleAuditLog this is single audit log from merged audit log file + * @return it returns a map i.e, auditLogMap which contains key-value pairs of a single audit log + */ + public HashMap parseAuditLog(String singleAuditLog) { + HashMap auditLogMap = new HashMap<>(); + if (singleAuditLog == null || singleAuditLog.isEmpty()) { + LOG.debug("This is an empty string or null string, expected a valid string to parse"); + return auditLogMap; + } + final Matcher matcher = LOG_ENTRY_PATTERN.matcher(singleAuditLog); + boolean patternMatched = matcher.matches(); + if (patternMatched) { + for (String key : AWS_LOG_REGEXP_GROUPS) { + try { + final String value = matcher.group(key); + auditLogMap.put(key, value); + } catch (IllegalStateException e) { + LOG.debug("Skipping key :{} due to no match with the audit log " + + "pattern :", key); + LOG.debug(String.valueOf(e)); + } + } + } + return auditLogMap; + } + + /** + * Parses the http referrer header. + * which is one of the key-value pair of audit log. + * @param referrerHeader this is the http referrer header of a particular + * audit log. + * @return returns a map which contains key-value pairs of referrer headers; an empty map + * there was no referrer header or parsing failed + */ + public static HashMap parseAudit(String referrerHeader) { + HashMap referrerHeaderMap = new HashMap<>(); + if (StringUtils.isEmpty(referrerHeader) + || referrerHeader.equals("-")) { + + LOG.debug("This is an empty string or null string, expected a valid string to parse"); + return referrerHeaderMap; + } + + // '?' used as the split point between the headers and the url. This + // returns the first occurrence of '?' + int indexOfQuestionMark = referrerHeader.indexOf("?"); + String httpReferrer = referrerHeader.substring(indexOfQuestionMark + 1, + referrerHeader.length() - 1); + + int lengthOfReferrer = httpReferrer.length(); + int start = 0; + LOG.debug("HttpReferrer headers string: {}", httpReferrer); + while (start < lengthOfReferrer) { + // splits "key" and "value" of each header + int equals = httpReferrer.indexOf("=", start); + // no match : break, no header left + if (equals == -1) { + break; + } + // key represents the string between "start" and index of "=". + String key = httpReferrer.substring(start, equals); + // splits between different headers, this also helps in ignoring "=" + // inside values since we set the end till we find '&' + int end = httpReferrer.indexOf("&", equals); + // or end of string + if (end == -1) { + end = lengthOfReferrer; + } + // value represents the string between index of "=" + 1 and the "end" + String value = httpReferrer.substring(equals + 1, end); + referrerHeaderMap.put(key, value); + start = end + 1; + } + + return referrerHeaderMap; + } + + /** + * Merge and parse all the audit log files and convert data into avro file. + * @param logsPath source path of logs + * @param destFile destination path of merged log file + * @return true + * @throws IOException on any failure + */ + public boolean mergeAndParseAuditLogFiles( + final Path logsPath, + final Path destFile) throws IOException { + + // List source log files + final FileSystem sourceFS = logsPath.getFileSystem(conf); + RemoteIterator listOfLogFiles = + sourceFS.listFiles(logsPath, true); + + final FileSystem destFS = destFile.getFileSystem(conf); + final FSDataOutputStreamBuilder builder = destFS.createFile(destFile) + .recursive() + .overwrite(true); + + // this has a broken return type; a java typesystem quirk. + builder.opt(FS_S3A_CREATE_PERFORMANCE, true); + + FSDataOutputStream fsDataOutputStream = builder.build(); + + // Instantiate DatumWriter class + DatumWriter datumWriter = + new SpecificDatumWriter<>( + AvroS3LogEntryRecord.class); + try (DataFileWriter dataFileWriter = + new DataFileWriter<>(datumWriter); + DataFileWriter avroWriter = + dataFileWriter.create(AvroS3LogEntryRecord.getClassSchema(), + fsDataOutputStream);) { + + // Iterating over the list of files to merge and parse + while (listOfLogFiles.hasNext()) { + logFilesParsed++; + FileStatus fileStatus = listOfLogFiles.next(); + int fileLength = (int) fileStatus.getLen(); + + try (DurationInfo duration = new DurationInfo(LOG, "Processing %s", fileStatus.getPath()); + FSDataInputStream fsDataInputStream = + awaitFuture(sourceFS.openFile(fileStatus.getPath()) + .withFileStatus(fileStatus) + .opt(FS_OPTION_OPENFILE_READ_POLICY, + FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE) + .build())) { + + // Reading the file data using LineRecordReader + LineRecordReader lineRecordReader = + new LineRecordReader(fsDataInputStream, 0L, fileLength, + MAX_LINE_LENGTH); + LongWritable longWritable = new LongWritable(); + Text singleAuditLog = new Text(); + + // Parse each and every audit log from list of logs + while (lineRecordReader.next(longWritable, singleAuditLog)) { + // Parse audit log + HashMap auditLogMap = + parseAuditLog(singleAuditLog.toString()); + auditLogsParsed++; + + // Insert data according to schema + + // Instantiating generated AvroDataRecord class + AvroS3LogEntryRecord avroDataRecord = buildLogRecord(auditLogMap); + if (!avroDataRecord.getReferrerMap().isEmpty()) { + referrerHeaderLogParsed++; + } + avroWriter.append(avroDataRecord); + } + dataFileWriter.flush(); + } + } + } + + LOG.info("Successfully parsed :{} audit logs and {} referrer header " + + "in the logs", auditLogsParsed, referrerHeaderLogParsed); + return true; + } + + public long getAuditLogsParsed() { + return auditLogsParsed; + } + + public long getReferrerHeaderLogParsed() { + return referrerHeaderLogParsed; + } + + /** + * Build log record from a parsed audit log entry. + * + * @param auditLogMap parsed audit log entry. + * @return the Avro record. + */ + public AvroS3LogEntryRecord buildLogRecord(Map auditLogMap) { + + // Instantiating generated AvroDataRecord class + AvroS3LogEntryRecord avroDataRecord = new AvroS3LogEntryRecord(); + for (Map.Entry entry : auditLogMap.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue().trim(); + + // if value == '-' and key is not in arraylist then put '-' or else '-1' + // if key is in arraylist of long values then parse the long value + // while parsing do it in try-catch block, + // in catch block need to log exception and set value as '-1' + try { + if (FIELDS_OF_TYPE_LONG.contains(key)) { + if (value.equals("-")) { + avroDataRecord.put(key, null); + } else { + try { + avroDataRecord.put(key, Long.parseLong(value)); + } catch (NumberFormatException e) { + // failed to parse the long value. + LOG.debug("Failed to parse long value for key {} : {}", key, value); + avroDataRecord.put(key, FAILED_TO_PARSE_LONG); + } + } + } else { + avroDataRecord.put(key, value); + } + } catch (Exception e) { + avroDataRecord.put(key, null); + } + } + + // Parse the audit header + HashMap referrerHeaderMap = + parseAudit(auditLogMap.get(REFERRER_HEADER_KEY)); + avroDataRecord.put(REFERRER_MAP, referrerHeaderMap); + return avroDataRecord; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/mapreduce/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/mapreduce/package-info.java new file mode 100644 index 0000000000000..cbe907beaf7ca --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/mapreduce/package-info.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * It requires mapreduce on the class path. + */ + +@InterfaceAudience.Public +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.audit.mapreduce; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index e1b995de68df4..d4e95d4e762bb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -49,6 +49,7 @@ import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.WriteOperationHelper; +import org.apache.hadoop.fs.s3a.audit.AuditTool; import org.apache.hadoop.fs.s3a.auth.RolePolicies; import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens; import org.apache.hadoop.fs.s3a.commit.CommitConstants; @@ -976,6 +977,9 @@ public static int run(Configuration conf, String... args) throws } switch (subCommand) { + case AuditTool.AUDIT: + command = new AuditTool(conf); + break; case BucketInfo.NAME: command = new BucketInfo(conf); break; diff --git a/hadoop-tools/hadoop-aws/src/main/shellprofile.d/hadoop-s3a.sh b/hadoop-tools/hadoop-aws/src/main/shellprofile.d/hadoop-s3a.sh new file mode 100644 index 0000000000000..912308d731ad5 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/shellprofile.d/hadoop-s3a.sh @@ -0,0 +1,37 @@ +#!/usr/bin/env bash + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +if ! declare -f hadoop_subcommand_s3a >/dev/null 2>/dev/null; then + + if [[ "${HADOOP_SHELL_EXECNAME}" = hadoop ]]; then + hadoop_add_subcommand "s3a" client "S3 Commands" + fi + + # this can't be indented otherwise shelldocs won't get it + +## @description s3a command for hadoop +## @audience public +## @stability stable +## @replaceable yes +function hadoop_subcommand_s3a +{ + # shellcheck disable=SC2034 + HADOOP_CLASSNAME=org.apache.hadoop.fs.s3a.s3guard.S3GuardTool + hadoop_add_to_classpath_tools hadoop-aws +} + +fi diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditTool.java new file mode 100644 index 0000000000000..adeedfd3dc289 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestAuditTool.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +import java.io.File; +import java.io.FileWriter; +import java.nio.file.Files; + +import org.assertj.core.api.Assertions; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.avro.file.DataFileReader; +import org.apache.avro.io.DatumReader; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.test.HadoopTestBase; + +import static org.apache.hadoop.fs.s3a.audit.S3LogParser.BUCKET_GROUP; +import static org.apache.hadoop.fs.s3a.audit.S3LogParser.REMOTEIP_GROUP; +import static org.apache.hadoop.fs.s3a.audit.TestS3AAuditLogMergerAndParser.SAMPLE_LOG_ENTRY; + +/** + * AuditTool tests. + */ +public class TestAuditTool extends HadoopTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestAuditTool.class); + + private final Configuration conf = new Configuration(); + + /** + * The audit tool. + */ + private AuditTool auditTool; + + /** + * Temporary directory to store the sample files; should be under target/ + * though IDEs may put it elsewhere. + */ + private File sampleDir; + + /** + * Sample directories and files to test. + */ + private File sampleFile; + + private File sampleDestDir; + + @Before + public void setup() throws Exception { + auditTool = new AuditTool(conf); + } + + /** + * Testing run method in AuditTool class by passing source and destination + * paths. + */ + @Test + public void testRun() throws Exception { + sampleDir = Files.createTempDirectory("sampleDir").toFile(); + sampleFile = File.createTempFile("sampleFile", ".txt", sampleDir); + try (FileWriter fw = new FileWriter(sampleFile)) { + fw.write(SAMPLE_LOG_ENTRY); + fw.flush(); + } + sampleDestDir = Files.createTempDirectory("sampleDestDir").toFile(); + Path logsPath = new Path(sampleDir.toURI()); + Path destPath = new Path(sampleDestDir.toURI()); + String[] args = {logsPath.toString(), destPath.toString()}; + auditTool.run(args); + FileSystem fileSystem = destPath.getFileSystem(conf); + RemoteIterator listOfDestFiles = + fileSystem.listFiles(destPath, true); + Path expectedPath = new Path(destPath, "AvroData.avro"); + fileSystem.open(expectedPath); + + File avroFile = new File(expectedPath.toUri()); + + //DeSerializing the objects + DatumReader datumReader = + new SpecificDatumReader<>(AvroS3LogEntryRecord.class); + + //Instantiating DataFileReader + DataFileReader dataFileReader = + new DataFileReader<>(avroFile, datumReader); + + AvroS3LogEntryRecord record = new AvroS3LogEntryRecord(); + while (dataFileReader.hasNext()) { + record = dataFileReader.next(record); + Assertions.assertThat(record.get(BUCKET_GROUP)) + .describedAs(BUCKET_GROUP) + .extracting(Object::toString) + .isEqualTo("bucket-london"); + + //verifying the remoteip from generated avro data + Assertions.assertThat(record.get(REMOTEIP_GROUP)) + .describedAs(BUCKET_GROUP) + .extracting(Object::toString) + .isEqualTo("109.157.171.174"); + } + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestS3AAuditLogMergerAndParser.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestS3AAuditLogMergerAndParser.java new file mode 100644 index 0000000000000..a99fc8851564c --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/TestS3AAuditLogMergerAndParser.java @@ -0,0 +1,326 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a.audit; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Map; + +import org.assertj.core.api.AbstractStringAssert; +import org.assertj.core.api.Assertions; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.audit.mapreduce.S3AAuditLogMergerAndParser; +import org.apache.hadoop.test.AbstractHadoopTestBase; +import org.apache.hadoop.util.Preconditions; + +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PATH; +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_PRINCIPAL; +import static org.apache.hadoop.fs.s3a.audit.mapreduce.S3AAuditLogMergerAndParser.parseAudit; + +/** + * Unit tests on {@link S3AAuditLogMergerAndParser} class. + */ +public class TestS3AAuditLogMergerAndParser extends AbstractHadoopTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestS3AAuditLogMergerAndParser.class); + + @Rule + public TestName methodName = new TestName(); + + @BeforeClass + public static void nameTestThread() { + Thread.currentThread().setName("JUnit"); + } + + @Before + public void nameThread() { + Thread.currentThread().setName("JUnit-" + getMethodName()); + } + + protected String getMethodName() { + return methodName.getMethodName(); + } + + /** + * A real log entry. + * This is derived from a real log entry on a test run. + * If this needs to be updated, please do it from a real log. + * Splitting this up across lines has a tendency to break things, so + * be careful making changes. + */ + static final String SAMPLE_LOG_ENTRY = + "183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000" + + " bucket-london" + + " [13/May/2021:11:26:06 +0000]" + + " 109.157.171.174" + + " arn:aws:iam::152813717700:user/dev" + + " M7ZB7C4RTKXJKTM9" + + " REST.PUT.OBJECT" + + " fork-0001/test/testParseBrokenCSVFile" + + " \"PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1\"" + + " 200" + + " -" + + " -" + + " 794" + + " 55" + + " 17" + + " \"https://audit.example.org/hadoop/1/op_create/" + + "e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/" + + "?op=op_create" + + "&p1=fork-0001/test/testParseBrokenCSVFile" + + "&pr=alice" + + "&ps=2eac5a04-2153-48db-896a-09bc9a2fd132" + + "&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154" + + "&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&" + + "ts=1620905165700\"" + + " \"Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK\"" + + " -" + + " TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0=" + + " SigV4" + + " ECDHE-RSA-AES128-GCM-SHA256" + + " AuthHeader" + + " bucket-london.s3.eu-west-2.amazonaws.com" + + " TLSv1.2" + "\n"; + + static final String SAMPLE_LOG_ENTRY_1 = + "01234567890123456789" + + " bucket-london1" + + " [13/May/2021:11:26:06 +0000]" + + " 109.157.171.174" + + " arn:aws:iam::152813717700:user/dev" + + " M7ZB7C4RTKXJKTM9" + + " REST.PUT.OBJECT" + + " fork-0001/test/testParseBrokenCSVFile" + + " \"PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1\"" + + " 200" + + " -" + + " -" + + " 794" + + " 55" + + " 17" + + " \"https://audit.example.org/hadoop/1/op_create/" + + "e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/" + + "?op=op_create" + + "&p1=fork-0001/test/testParseBrokenCSVFile" + + "&pr=alice" + + "&ps=2eac5a04-2153-48db-896a-09bc9a2fd132" + + "&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154" + + "&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&" + + "ts=1620905165700\"" + + " \"Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK\"" + + " -" + + " TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0=" + + " SigV4" + + " ECDHE-RSA-AES128-GCM-SHA256" + + " AuthHeader" + + " bucket-london.s3.eu-west-2.amazonaws.com" + + " TLSv1.2" + "\n"; + + /** + * A real referrer header entry. + * This is derived from a real log entry on a test run. + * If this needs to be updated, please do it from a real log. + * Splitting this up across lines has a tendency to break things, so + * be careful making changes. + */ + private final String sampleReferrerHeader = + "\"https://audit.example.org/hadoop/1/op_create/" + + "e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?" + + "op=op_create" + + "&p1=fork-0001/test/testParseBrokenCSVFile" + + "&pr=alice" + + "&ps=2eac5a04-2153-48db-896a-09bc9a2fd132" + + "&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154" + + "&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156" + + "&ts=1620905165700\""; + + private File sampleDir; + + private S3AAuditLogMergerAndParser s3AAuditLogMergerAndParser = + new S3AAuditLogMergerAndParser(new Configuration(), 1); + + + /** + * Testing parseAuditLog method in parser class by passing sample audit log + * entry and checks if the log is parsed correctly. + */ + @Test + public void testParseAuditLog() { + Map parseAuditLogResult = + s3AAuditLogMergerAndParser.parseAuditLog(SAMPLE_LOG_ENTRY); + Assertions.assertThat(parseAuditLogResult) + .describedAs("the result of parseAuditLogResult of %s", SAMPLE_LOG_ENTRY) + .isNotNull(); + + //verifying the bucket from parsed audit log + assertFieldValue(parseAuditLogResult, "bucket", "bucket-london"); + + //verifying the remoteip from parsed audit log + assertFieldValue(parseAuditLogResult, "remoteip", "109.157.171.174"); + } + + /** + * Assert that a field in the parsed audit log matches the expected value. + * @param parseAuditLogResult parsed audit log + * @param field field name + * @param expected expected value + * @return the ongoing assert + */ + private static AbstractStringAssert assertFieldValue(final Map parseAuditLogResult, + final String field, + final String expected) { + return Assertions.assertThat(parseAuditLogResult.get(field)) + .describedAs("Mismatch in the field %s parsed from the audit", field) + .isEqualTo(expected); + } + + /** + * Testing parseAuditLog method in parser class by passing empty string and + * null and checks if the result is empty. + */ + @Test + public void testParseAuditLogEmptyAndNull() { + Map parseAuditLogResultEmpty = + s3AAuditLogMergerAndParser.parseAuditLog(""); + Assertions.assertThat(parseAuditLogResultEmpty) + .describedAs("Audit log map should be empty") + .isEmpty(); + Map parseAuditLogResultNull = + s3AAuditLogMergerAndParser.parseAuditLog(null); + Assertions.assertThat(parseAuditLogResultNull) + .describedAs("Audit log map of null record should be empty") + .isEmpty(); + } + + /** + * Testing parseReferrerHeader method in parser class by passing + * sample referrer header taken from sample audit log and checks if the + * referrer header is parsed correctly. + */ + @Test + public void testParseAudit() { + Map parseReferrerHeaderResult = + parseAudit(sampleReferrerHeader); + //verifying the path 'p1' from parsed referrer header + Assertions.assertThat(parseReferrerHeaderResult) + .describedAs("Mismatch in the path parsed from the referrer") + .isNotNull() + .containsEntry(PARAM_PATH, "fork-0001/test/testParseBrokenCSVFile") + .describedAs("Referrer header map should contain the principal") + .containsEntry(PARAM_PRINCIPAL, "alice"); + + } + + /** + * Testing parseReferrerHeader method in parser class by passing empty + * string and null string and checks if the result is empty. + */ + @Test + public void testParseAuditEmptyAndNull() { + Assertions.assertThat(parseAudit("")) + .isEmpty(); + Assertions.assertThat(parseAudit(null)) + .isEmpty(); + } + + /** + * Testing mergeAndParseAuditLogFiles method by passing filesystem, source + * and destination paths. + */ + @Test + public void testMergeAndParseAuditLogFiles() throws IOException { + // Getting the audit dir and file path from test/resources/ + String auditLogDir = this.getClass().getClassLoader().getResource( + "TestAuditLogs").toString(); + String auditSingleFile = this.getClass().getClassLoader().getResource( + "TestAuditLogs/sampleLog1").toString(); + Preconditions.checkArgument(!StringUtils.isAnyBlank(auditLogDir, + auditSingleFile), String.format("Audit path should not be empty. Check " + + "test/resources. auditLogDir : %s, auditLogSingleFile :%s", + auditLogDir, auditSingleFile)); + Path auditDirPath = new Path(auditLogDir); + Path auditFilePath = new Path(auditSingleFile); + + final Path destPath = tempAvroPath(); + boolean mergeAndParseResult = + s3AAuditLogMergerAndParser.mergeAndParseAuditLogFiles( + auditDirPath, destPath); + Assertions.assertThat(mergeAndParseResult) + .describedAs("The merge and parse failed for the audit log") + .isTrue(); + // 36 audit logs with referrer in each of the 2 sample files. + Assertions.assertThat(s3AAuditLogMergerAndParser.getAuditLogsParsed()) + .describedAs("Mismatch in the number of audit logs parsed") + .isEqualTo(36 + 36); + Assertions.assertThat(s3AAuditLogMergerAndParser.getReferrerHeaderLogParsed()) + .describedAs("Mismatch in the number of referrer headers parsed") + .isEqualTo(36 + 36); + } + + private Path tempAvroPath() throws IOException { + File destFile = Files.createTempFile(getMethodName(), ".avro").toFile(); + return new Path(destFile.toURI()); + } + + /** + * Testing mergeAndParseAuditLogCounter method by passing filesystem, + * sample files source and destination paths. + */ + @Test + public void testMergeAndParseAuditLogCounter() throws IOException { + sampleDir = Files.createTempDirectory("sampleDir").toFile(); + File firstSampleFile = + File.createTempFile("sampleFile1", ".txt", sampleDir); + File secondSampleFile = + File.createTempFile("sampleFile2", ".txt", sampleDir); + File thirdSampleFile = + File.createTempFile("sampleFile3", ".txt", sampleDir); + try (FileWriter fw = new FileWriter(firstSampleFile); + FileWriter fw1 = new FileWriter(secondSampleFile); + FileWriter fw2 = new FileWriter(thirdSampleFile)) { + fw.write(SAMPLE_LOG_ENTRY); + fw1.write(SAMPLE_LOG_ENTRY); + fw2.write(SAMPLE_LOG_ENTRY_1); + } + Path logsPath = new Path(sampleDir.toURI()); + Path destPath = tempAvroPath(); + boolean mergeAndParseResult = + s3AAuditLogMergerAndParser.mergeAndParseAuditLogFiles( + logsPath, destPath); + Assertions.assertThat(mergeAndParseResult) + .describedAs("Merge and parsing of the audit logs files was unsuccessful") + .isTrue(); + Assertions.assertThat(s3AAuditLogMergerAndParser.getAuditLogsParsed()) + .describedAs("Mismatch in the number of audit logs parsed") + .isEqualTo(3); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/resources/TestAuditLogs/sampleLog1 b/hadoop-tools/hadoop-aws/src/test/resources/TestAuditLogs/sampleLog1 new file mode 100644 index 0000000000000..8e3cb81175c94 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/resources/TestAuditLogs/sampleLog1 @@ -0,0 +1,36 @@ +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/test/resources/TestAuditLogs/sampleLog2 b/hadoop-tools/hadoop-aws/src/test/resources/TestAuditLogs/sampleLog2 new file mode 100644 index 0000000000000..8e3cb81175c94 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/resources/TestAuditLogs/sampleLog2 @@ -0,0 +1,36 @@ +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; +183c9826b45486e485693808f38e2c4071004bf5dfd4c3ab210f0a21a4000000 bucket-london [13/May/2021:11:26:06 +0000] 109.157.171.174 arn:aws:iam::152813717700:user/dev M7ZB7C4RTKXJKTM9 REST.PUT.OBJECT fork-0001/test/testParseBrokenCSVFile "PUT /fork-0001/test/testParseBrokenCSVFile HTTP/1.1" 200 - - 794 55 17 "https://audit.example.org/hadoop/1/op_create/e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278/?op=op_create&p1=fork-0001/test/testParseBrokenCSVFile&pr=alice&ps=2eac5a04-2153-48db-896a-09bc9a2fd132&id=e8ede3c7-8506-4a43-8268-fe8fcbb510a4-00000278&t0=154&fs=e8ede3c7-8506-4a43-8268-fe8fcbb510a4&t1=156&ts=1620905165700" "Hadoop 3.4.0-SNAPSHOT, java/1.8.0_282 vendor/AdoptOpenJDK" - TrIqtEYGWAwvu0h1N9WJKyoqM0TyHUaY+ZZBwP2yNf2qQp1Z/0= SigV4 ECDHE-RSA-AES128-GCM-SHA256 AuthHeader bucket-london.s3.eu-west-2.amazonaws.com TLSv1.2"; \ No newline at end of file diff --git a/pom.xml b/pom.xml index 38fb70046fb5c..d8cf67a67e0f6 100644 --- a/pom.xml +++ b/pom.xml @@ -586,6 +586,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x dev-support/docker/pkg-resolver/packages.json dev-support/docker/pkg-resolver/platforms.json **/target/** + **/TestAuditLogs/*