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/*