You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by so...@apache.org on 2019/03/17 07:38:02 UTC

[drill] branch master updated: DRILL-7021: HTTPD Throws NPE and Doesn't Recognize Timeformat

This is an automated email from the ASF dual-hosted git repository.

sorabh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git


The following commit(s) were added to refs/heads/master by this push:
     new bf1bdec  DRILL-7021: HTTPD Throws NPE and Doesn't Recognize Timeformat
bf1bdec is described below

commit bf1bdec6069f6fdd2132608450357edea47d328c
Author: Charles S. Givre <cg...@gmail.com>
AuthorDate: Tue Mar 12 08:57:20 2019 -0400

    DRILL-7021: HTTPD Throws NPE and Doesn't Recognize Timeformat
---
 exec/java-exec/pom.xml                             |   2 +-
 .../exec/store/httpd/HttpdLogFormatConfig.java     |  80 +++++
 .../exec/store/httpd/HttpdLogFormatPlugin.java     | 114 ++----
 .../drill/exec/store/httpd/HttpdLogRecord.java     |  92 +++--
 .../apache/drill/exec/store/httpd/HttpdParser.java | 396 ++++++++++++++-------
 .../drill/exec/store/httpd/HttpdParserTest.java    |  50 ---
 .../main/resources/bootstrap-storage-plugins.json  | 178 ++++-----
 .../drill/exec/store/FormatPluginSerDeTest.java    |   4 +-
 .../drill/exec/store/httpd/TestHTTPDLogReader.java | 237 ++++++++++++
 .../resources/httpd/hackers-access-small.httpd     |  10 +
 10 files changed, 794 insertions(+), 369 deletions(-)

diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 79b84ab..a341ab9 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -458,7 +458,7 @@
     <dependency>
       <groupId>nl.basjes.parse.httpdlog</groupId>
       <artifactId>httpdlog-parser</artifactId>
-      <version>2.4</version>
+      <version>5.2</version>
       <exclusions>
         <exclusion>
           <groupId>commons-codec</groupId>
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatConfig.java
new file mode 100644
index 0000000..c4c34b6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatConfig.java
@@ -0,0 +1,80 @@
+/*
+ * 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.drill.exec.store.httpd;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.logical.FormatPluginConfig;
+
+@JsonTypeName("httpd")
+@JsonInclude(JsonInclude.Include.NON_DEFAULT)
+public class HttpdLogFormatConfig implements FormatPluginConfig {
+
+  public String logFormat;
+  public String timestampFormat = "dd/MMM/yyyy:HH:mm:ss ZZ";
+
+  /**
+   * @return the log formatting string.  This string is the config string from httpd.conf or similar config file.
+   */
+  public String getLogFormat() {
+    return logFormat;
+  }
+
+  public void setLogFormat(String format) {
+    this.logFormat = format;
+  }
+
+  /**
+   * @return the timestampFormat
+   */
+  public String getTimestampFormat() {
+    return timestampFormat;
+  }
+
+  /**
+   * Sets the time stamp format
+   * @param timestamp
+   */
+  public void setTimestampFormat(String timestamp) {
+    this.timestampFormat = timestamp;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = logFormat != null ? logFormat.hashCode() : 0;
+    result = 31 * result + (timestampFormat != null ? timestampFormat.hashCode() : 0);
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    HttpdLogFormatConfig that = (HttpdLogFormatConfig) o;
+
+    if (logFormat != null ? !logFormat.equals(that.logFormat) : that.logFormat != null) {
+      return false;
+    }
+    return timestampFormat != null ? timestampFormat.equals(that.timestampFormat) : that.timestampFormat == null;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
index 5a78732..4785da1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
@@ -18,9 +18,9 @@
 package org.apache.drill.exec.store.httpd;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.List;
 
-import com.fasterxml.jackson.annotation.JsonInclude;
 import nl.basjes.parse.core.exceptions.DissectionFailure;
 import nl.basjes.parse.core.exceptions.InvalidDissectorException;
 import nl.basjes.parse.core.exceptions.MissingDissectorsException;
@@ -28,7 +28,6 @@ import nl.basjes.parse.core.exceptions.MissingDissectorsException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.FragmentContext;
@@ -55,26 +54,23 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.LineRecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextInputFormat;
-
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import java.util.Collections;
 import java.util.Map;
 import org.apache.drill.exec.store.RecordReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.HttpdLogFormatConfig> {
+public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatConfig> {
 
   private static final Logger LOG = LoggerFactory.getLogger(HttpdLogFormatPlugin.class);
   private static final String PLUGIN_EXTENSION = "httpd";
   private static final int VECTOR_MEMORY_ALLOCATION = 4095;
 
   public HttpdLogFormatPlugin(final String name, final DrillbitContext context, final Configuration fsConf,
-      final StoragePluginConfig storageConfig, final HttpdLogFormatConfig formatConfig) {
+                              final StoragePluginConfig storageConfig, final HttpdLogFormatConfig formatConfig) {
 
     super(name, context, fsConf, storageConfig, formatConfig, true, false, true, true,
-        Lists.newArrayList(PLUGIN_EXTENSION), PLUGIN_EXTENSION);
+            Collections.singletonList(PLUGIN_EXTENSION), PLUGIN_EXTENSION);
   }
 
   @Override
@@ -93,55 +89,6 @@ public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.
   }
 
   /**
-   * This class is a POJO to hold the configuration for the HttpdLogFormat Parser. This is automatically
-   * serialized/deserialized from JSON format.
-   */
-  @JsonTypeName(PLUGIN_EXTENSION) @JsonInclude(JsonInclude.Include.NON_DEFAULT)
-  public static class HttpdLogFormatConfig implements FormatPluginConfig {
-
-    public String logFormat;
-    public String timestampFormat;
-
-    /**
-     * @return the logFormat
-     */
-    public String getLogFormat() {
-      return logFormat;
-    }
-
-    /**
-     * @return the timestampFormat
-     */
-    public String getTimestampFormat() {
-      return timestampFormat;
-    }
-
-    @Override
-    public int hashCode() {
-      int result = logFormat != null ? logFormat.hashCode() : 0;
-      result = 31 * result + (timestampFormat != null ? timestampFormat.hashCode() : 0);
-      return result;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) {
-        return true;
-      }
-      if (o == null || getClass() != o.getClass()) {
-        return false;
-      }
-
-      HttpdLogFormatConfig that = (HttpdLogFormatConfig) o;
-
-      if (logFormat != null ? !logFormat.equals(that.logFormat) : that.logFormat != null) {
-        return false;
-      }
-      return timestampFormat != null ? timestampFormat.equals(that.timestampFormat) : that.timestampFormat == null;
-    }
-  }
-
-  /**
    * This class performs the work for the plugin. This is where all logic goes to read records. In this case httpd logs
    * are lines terminated with a new line character.
    */
@@ -169,11 +116,15 @@ public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.
      * @return Map with Drill field names as a key and Parser Field names as a value
      */
     private Map<String, String> makeParserFields() {
-      final Map<String, String> fieldMapping = Maps.newHashMap();
+      Map<String, String> fieldMapping = new HashMap<>();
       for (final SchemaPath sp : getColumns()) {
-        final String drillField = sp.getRootSegment().getPath();
-        final String parserField = HttpdParser.parserFormattedFieldName(drillField);
-        fieldMapping.put(drillField, parserField);
+        String drillField = sp.getRootSegment().getPath();
+        try {
+          String parserField = HttpdParser.parserFormattedFieldName(drillField);
+          fieldMapping.put(drillField, parserField);
+        } catch (Exception e) {
+          LOG.info("Putting field: " + drillField + " into map", e);
+        }
       }
       return fieldMapping;
     }
@@ -187,10 +138,11 @@ public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.
          */
         final Map<String, String> fieldMapping = !isStarQuery() ? makeParserFields() : null;
         writer = new VectorContainerWriter(output);
+
         parser = new HttpdParser(writer.rootAsMap(), context.getManagedBuffer(),
-            HttpdLogFormatPlugin.this.getConfig().getLogFormat(),
-            HttpdLogFormatPlugin.this.getConfig().getTimestampFormat(),
-            fieldMapping);
+                HttpdLogFormatPlugin.this.getConfig().getLogFormat(),
+                HttpdLogFormatPlugin.this.getConfig().getTimestampFormat(),
+                fieldMapping);
 
         final Path path = fs.makeQualified(work.getPath());
         FileSplit split = new FileSplit(path, work.getStart(), work.getLength(), new String[]{""});
@@ -200,23 +152,21 @@ public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.
         job.setInputFormat(inputFormat.getClass());
         lineReader = (LineRecordReader) inputFormat.getRecordReader(split, job, Reporter.NULL);
         lineNumber = lineReader.createKey();
-      }
-      catch (NoSuchMethodException | MissingDissectorsException | InvalidDissectorException e) {
+      } catch (NoSuchMethodException | MissingDissectorsException | InvalidDissectorException e) {
         throw handleAndGenerate("Failure creating HttpdParser", e);
-      }
-      catch (IOException e) {
+      } catch (IOException e) {
         throw handleAndGenerate("Failure creating HttpdRecordReader", e);
       }
     }
 
     private RuntimeException handleAndGenerate(final String s, final Exception e) {
       throw UserException.dataReadError(e)
-          .message(s + "\n%s", e.getMessage())
-          .addContext("Path", work.getPath())
-          .addContext("Split Start", work.getStart())
-          .addContext("Split Length", work.getLength())
-          .addContext("Local Line Number", lineNumber.get())
-          .build(LOG);
+              .message(s + "\n%s", e.getMessage())
+              .addContext("Path", work.getPath())
+              .addContext("Split Start", work.getStart())
+              .addContext("Split Length", work.getLength())
+              .addContext("Local Line Number", lineNumber.get())
+              .build(LOG);
     }
 
     /**
@@ -241,8 +191,7 @@ public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.
         writer.setValueCount(recordCount);
 
         return recordCount;
-      }
-      catch (DissectionFailure | InvalidDissectorException | MissingDissectorsException | IOException e) {
+      } catch (DissectionFailure | InvalidDissectorException | MissingDissectorsException | IOException e) {
         throw handleAndGenerate("Failure while parsing log record.", e);
       }
     }
@@ -253,8 +202,7 @@ public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.
         if (lineReader != null) {
           lineReader.close();
         }
-      }
-      catch (IOException e) {
+      } catch (IOException e) {
         LOG.warn("Failure while closing Httpd reader.", e);
       }
     }
@@ -262,10 +210,10 @@ public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.
     @Override
     public String toString() {
       return "HttpdLogRecordReader[Path=" + work.getPath()
-          + ", Start=" + work.getStart()
-          + ", Length=" + work.getLength()
-          + ", Line=" + lineNumber.get()
-          + "]";
+              + ", Start=" + work.getStart()
+              + ", Length=" + work.getLength()
+              + ", Line=" + lineNumber.get()
+              + "]";
     }
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogRecord.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogRecord.java
index 55fc13d..95917cb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogRecord.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogRecord.java
@@ -20,31 +20,43 @@ package org.apache.drill.exec.store.httpd;
 import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
 import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 import io.netty.buffer.DrillBuf;
+
 import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.Map;
+
 import nl.basjes.parse.core.Casts;
 import nl.basjes.parse.core.Parser;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
 import org.apache.drill.exec.vector.complex.writer.BigIntWriter;
 import org.apache.drill.exec.vector.complex.writer.Float8Writer;
 import org.apache.drill.exec.vector.complex.writer.VarCharWriter;
+import org.apache.drill.exec.vector.complex.writer.TimeStampWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
 public class HttpdLogRecord {
 
   private static final Logger LOG = LoggerFactory.getLogger(HttpdLogRecord.class);
   private final Map<String, VarCharWriter> strings = Maps.newHashMap();
   private final Map<String, BigIntWriter> longs = Maps.newHashMap();
   private final Map<String, Float8Writer> doubles = Maps.newHashMap();
+  private final Map<String, TimeStampWriter> times = new HashMap<>();
   private final Map<String, MapWriter> wildcards = Maps.newHashMap();
   private final Map<String, String> cleanExtensions = Maps.newHashMap();
   private final Map<String, MapWriter> startedWildcards = Maps.newHashMap();
   private final Map<String, MapWriter> wildcardWriters = Maps.newHashMap();
+  private final SimpleDateFormat dateFormatter;
   private DrillBuf managedBuffer;
+  private String timeFormat;
 
-  public HttpdLogRecord(final DrillBuf managedBuffer) {
+  public HttpdLogRecord(final DrillBuf managedBuffer, final String timeFormat) {
     this.managedBuffer = managedBuffer;
+    this.timeFormat = timeFormat;
+    this.dateFormatter = new SimpleDateFormat(this.timeFormat);
   }
 
   /**
@@ -66,7 +78,7 @@ public class HttpdLogRecord {
     return managedBuffer;
   }
 
-  private void writeString(final VarCharWriter writer, final String value) {
+  private void writeString(VarCharWriter writer, String value) {
     final byte[] stringBytes = value.getBytes(Charsets.UTF_8);
     final DrillBuf stringBuffer = buf(stringBytes.length);
     stringBuffer.clear();
@@ -82,14 +94,13 @@ public class HttpdLogRecord {
    * @param value value of field
    */
   @SuppressWarnings("unused")
-  public void set(final String field, final String value) {
+  public void set(String field, String value) {
     if (value != null) {
       final VarCharWriter w = strings.get(field);
       if (w != null) {
         LOG.trace("Parsed field: {}, as string: {}", field, value);
         writeString(w, value);
-      }
-      else {
+      } else {
         LOG.warn("No 'string' writer found for field: {}", field);
       }
     }
@@ -103,14 +114,13 @@ public class HttpdLogRecord {
    * @param value value of field
    */
   @SuppressWarnings("unused")
-  public void set(final String field, final Long value) {
+  public void set(String field, Long value) {
     if (value != null) {
       final BigIntWriter w = longs.get(field);
       if (w != null) {
         LOG.trace("Parsed field: {}, as long: {}", field, value);
         w.writeBigInt(value);
-      }
-      else {
+      } else {
         LOG.warn("No 'long' writer found for field: {}", field);
       }
     }
@@ -118,20 +128,48 @@ public class HttpdLogRecord {
 
   /**
    * This method is referenced and called via reflection. This is added as a parsing target for the parser. It will get
+   * called when the value of a log field is a timesstamp data type.
+   *
+   * @param field name of field
+   * @param value value of field
+   */
+  @SuppressWarnings("unused")
+  public void setTimestamp(String field, String value) {
+    if (value != null) {
+      //Convert the date string into a long
+      long ts = 0;
+      try {
+        Date d = this.dateFormatter.parse(value);
+        ts = d.getTime();
+      } catch (Exception e) {
+        //If the date formatter does not successfully create a date, the timestamp will fall back to zero
+        //Do not throw exception
+      }
+      final TimeStampWriter tw = times.get(field);
+      if (tw != null) {
+        LOG.trace("Parsed field: {}, as time: {}", field, value);
+        tw.writeTimeStamp(ts);
+      } else {
+        LOG.warn("No 'timestamp' writer found for field: {}", field);
+      }
+    }
+  }
+
+  /**
+   * This method is referenced and called via reflection. This is added as a parsing target for the parser. It will get
    * called when the value of a log field is a Double data type.
    *
    * @param field name of field
    * @param value value of field
    */
   @SuppressWarnings("unused")
-  public void set(final String field, final Double value) {
+  public void set(String field, Double value) {
     if (value != null) {
       final Float8Writer w = doubles.get(field);
       if (w != null) {
         LOG.trace("Parsed field: {}, as double: {}", field, value);
         w.writeFloat8(value);
-      }
-      else {
+      } else {
         LOG.warn("No 'double' writer found for field: {}", field);
       }
     }
@@ -146,7 +184,7 @@ public class HttpdLogRecord {
    * @param value value of field
    */
   @SuppressWarnings("unused")
-  public void setWildcard(final String field, final String value) {
+  public void setWildcard(String field, String value) {
     if (value != null) {
       final MapWriter mapWriter = getWildcardWriter(field);
       LOG.trace("Parsed wildcard field: {}, as string: {}", field, value);
@@ -164,7 +202,7 @@ public class HttpdLogRecord {
    * @param value value of field
    */
   @SuppressWarnings("unused")
-  public void setWildcard(final String field, final Long value) {
+  public void setWildcard(String field, Long value) {
     if (value != null) {
       final MapWriter mapWriter = getWildcardWriter(field);
       LOG.trace("Parsed wildcard field: {}, as long: {}", field, value);
@@ -182,7 +220,7 @@ public class HttpdLogRecord {
    * @param value value of field
    */
   @SuppressWarnings("unused")
-  public void setWildcard(final String field, final Double value) {
+  public void setWildcard(String field, Double value) {
     if (value != null) {
       final MapWriter mapWriter = getWildcardWriter(field);
       LOG.trace("Parsed wildcard field: {}, as double: {}", field, value);
@@ -199,7 +237,7 @@ public class HttpdLogRecord {
    * @param field like HTTP.URI:request.firstline.uri.query.old where 'old' is one of many different parameter names.
    * @return the writer to be used for this field.
    */
-  private MapWriter getWildcardWriter(final String field) {
+  private MapWriter getWildcardWriter(String field) {
     MapWriter writer = startedWildcards.get(field);
     if (writer == null) {
       for (Map.Entry<String, MapWriter> entry : wildcards.entrySet()) {
@@ -212,7 +250,7 @@ public class HttpdLogRecord {
            * unsafe characters in it.
            */
           if (!cleanExtensions.containsKey(field)) {
-            final String extension = field.substring(root.length() + 1, field.length());
+            final String extension = field.substring(root.length() + 1);
             final String cleanExtension = HttpdParser.drillFormattedFieldName(extension);
             cleanExtensions.put(field, cleanExtension);
             LOG.debug("Added extension: field='{}' with cleanExtension='{}'", field, cleanExtension);
@@ -255,6 +293,10 @@ public class HttpdLogRecord {
     return doubles;
   }
 
+  public Map<String, TimeStampWriter> getTimes() {
+    return times;
+  }
+
   /**
    * This record will be used with a single parser. For each field that is to be parsed a setter will be called. It
    * registers a setter method for each field being parsed. It also builds the data writers to hold the data beings
@@ -281,21 +323,23 @@ public class HttpdLogRecord {
       parser.addParseTarget(this.getClass().getMethod("setWildcard", String.class, Double.class), parserFieldName);
       parser.addParseTarget(this.getClass().getMethod("setWildcard", String.class, Long.class), parserFieldName);
       wildcards.put(cleanName, mapWriter.map(drillFieldName));
-    }
-    else if (type.contains(Casts.DOUBLE)) {
+    } else if (type.contains(Casts.DOUBLE)) {
       LOG.debug("Adding DOUBLE parse target: {}, with field name: {}", parserFieldName, drillFieldName);
       parser.addParseTarget(this.getClass().getMethod("set", String.class, Double.class), parserFieldName);
       doubles.put(parserFieldName, mapWriter.float8(drillFieldName));
-    }
-    else if (type.contains(Casts.LONG)) {
+    } else if (type.contains(Casts.LONG)) {
       LOG.debug("Adding LONG parse target: {}, with field name: {}", parserFieldName, drillFieldName);
       parser.addParseTarget(this.getClass().getMethod("set", String.class, Long.class), parserFieldName);
       longs.put(parserFieldName, mapWriter.bigInt(drillFieldName));
-    }
-    else {
+    } else {
       LOG.debug("Adding STRING parse target: {}, with field name: {}", parserFieldName, drillFieldName);
-      parser.addParseTarget(this.getClass().getMethod("set", String.class, String.class), parserFieldName);
-      strings.put(parserFieldName, mapWriter.varChar(drillFieldName));
+      if (parserFieldName.startsWith("TIME.STAMP:")) {
+        parser.addParseTarget(this.getClass().getMethod("setTimestamp", String.class, String.class), parserFieldName);
+        times.put(parserFieldName, mapWriter.timeStamp(drillFieldName));
+      } else {
+        parser.addParseTarget(this.getClass().getMethod("set", String.class, String.class), parserFieldName);
+        strings.put(parserFieldName, mapWriter.varChar(drillFieldName));
+      }
     }
   }
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParser.java
index ba72177..5d3d7c0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParser.java
@@ -45,112 +45,262 @@ public class HttpdParser {
   private final Parser<HttpdLogRecord> parser;
   private final HttpdLogRecord record;
 
-    public static final HashMap<String, String> LOGFIELDS = new HashMap<String, String>();
-    static
-    {
-        LOGFIELDS.put("request_receive_time_weekyear__utc", "TIME_YEAR:request_receive_time_weekyear__utc");
-        LOGFIELDS.put("request_referer_ref", "HTTP_REF:request_referer_ref");
-        LOGFIELDS.put("request_referer_protocol", "HTTP_PROTOCOL:request_referer_protocol");
-        LOGFIELDS.put("request_receive_time_timezone", "TIME_ZONE:request_receive_time_timezone");
-        LOGFIELDS.put("connection_client_host", "IP:connection_client_host");
-        LOGFIELDS.put("connection_client_ip", "IP:connection_client_ip");
-        LOGFIELDS.put("connection_client_peerip", "IP:connection_client_peerip");
-        LOGFIELDS.put("connection_server_ip", "IP:connection_server_ip");
-        LOGFIELDS.put("request_receive_time_day", "TIME_DAY:request_receive_time_day");
-        LOGFIELDS.put("request_receive_time_minute__utc", "TIME_MINUTE:request_receive_time_minute__utc");
-        LOGFIELDS.put("request_referer_query_$", "STRING:request_referer_query_$");
-        LOGFIELDS.put("request_receive_time_millisecond__utc", "TIME_MILLISECOND:request_receive_time_millisecond__utc");
-        LOGFIELDS.put("request_firstline_uri_port", "HTTP_PORT:request_firstline_uri_port");
-        LOGFIELDS.put("request_referer_userinfo", "HTTP_USERINFO:request_referer_userinfo");
-        LOGFIELDS.put("request_receive_time_second__utc", "TIME_SECOND:request_receive_time_second__utc");
-        LOGFIELDS.put("request_firstline_uri_protocol", "HTTP_PROTOCOL:request_firstline_uri_protocol");
-        LOGFIELDS.put("request_receive_time_month", "TIME_MONTH:request_receive_time_month");
-        LOGFIELDS.put("request_firstline_uri_query", "HTTP_QUERYSTRING:request_firstline_uri_query");
-        LOGFIELDS.put("request_firstline_uri_path", "HTTP_PATH:request_firstline_uri_path");
-        LOGFIELDS.put("request_receive_time_hour__utc", "TIME_HOUR:request_receive_time_hour__utc");
-        LOGFIELDS.put("request_receive_time_monthname", "TIME_MONTHNAME:request_receive_time_monthname");
-        LOGFIELDS.put("request_receive_time_year__utc", "TIME_YEAR:request_receive_time_year__utc");
-        LOGFIELDS.put("request_receive_time_second", "TIME_SECOND:request_receive_time_second");
-        LOGFIELDS.put("request_referer", "HTTP_URI:request_referer");
-        LOGFIELDS.put("request_receive_time_monthname__utc", "TIME_MONTHNAME:request_receive_time_monthname__utc");
-        LOGFIELDS.put("request_referer_path", "HTTP_PATH:request_referer_path");
-        LOGFIELDS.put("request_receive_time_weekyear", "TIME_YEAR:request_receive_time_weekyear");
-        LOGFIELDS.put("request_firstline_protocol", "HTTP_PROTOCOL:request_firstline_protocol");
-        LOGFIELDS.put("request_referer_port", "HTTP_PORT:request_referer_port");
-        LOGFIELDS.put("request_receive_time_minute", "TIME_MINUTE:request_receive_time_minute");
-        LOGFIELDS.put("request_status_last", "STRING:request_status_last");
-        LOGFIELDS.put("request_receive_time_hour", "TIME_HOUR:request_receive_time_hour");
-        LOGFIELDS.put("request_firstline_protocol_version", "HTTP_PROTOCOL_VERSION:request_firstline_protocol_version");
-        LOGFIELDS.put("request_receive_time", "TIME_STAMP:request_receive_time");
-        LOGFIELDS.put("request_firstline_method", "HTTP_METHOD:request_firstline_method");
-        LOGFIELDS.put("request_receive_time_epoch", "TIME_EPOCH:request_receive_time_epoch");
-        LOGFIELDS.put("request_receive_time_weekofweekyear", "TIME_WEEK:request_receive_time_weekofweekyear");
-        LOGFIELDS.put("request_firstline_uri_host", "HTTP_HOST:request_firstline_uri_host");
-        LOGFIELDS.put("request_referer_query", "HTTP_QUERYSTRING:request_referer_query");
-        LOGFIELDS.put("request_firstline_uri_userinfo", "HTTP_USERINFO:request_firstline_uri_userinfo");
-        LOGFIELDS.put("response_body_bytes", "BYTES:response_body_bytes");
-        LOGFIELDS.put("response_body_bytesclf", "BYTES:response_body_bytesclf");
-        LOGFIELDS.put("request_referer_host", "HTTP_HOST:request_referer_host");
-        LOGFIELDS.put("request_receive_time_weekofweekyear__utc", "TIME_WEEK:request_receive_time_weekofweekyear__utc");
-        LOGFIELDS.put("request_firstline_uri", "HTTP_URI:request_firstline_uri");
-        LOGFIELDS.put("request_firstline_uri_ref", "HTTP_REF:request_firstline_uri_ref");
-        LOGFIELDS.put("request_receive_time_year", "TIME_YEAR:request_receive_time_year");
-        LOGFIELDS.put("request_firstline", "HTTP_FIRSTLINE:request_firstline");
-        LOGFIELDS.put("request_user-agent", "HTTP_USERAGENT:request_user-agent");
-        LOGFIELDS.put("request_cookies", "HTTP_COOKIE:request_cookies");
-        LOGFIELDS.put("server_process_time", "MICROSECONDS:server_process_time");
-        LOGFIELDS.put("request_cookies_$", "HTTP_COOKIE:request_cookies_$");
-        LOGFIELDS.put("server_environment_$", "VARIABLE:server_environment_$");
-        LOGFIELDS.put("server_filename", "FILENAME:server_filename");
-        LOGFIELDS.put("request_protocol", "PROTOCOL:request_protocol");
-        LOGFIELDS.put("request_header_", "HTTP_HEADER:request_header_");
-        LOGFIELDS.put("connection_keepalivecount", "NUMBER:connection_keepalivecount");
-        LOGFIELDS.put("connection_client_logname", "NUMBER:connection_client_logname");
-        LOGFIELDS.put("request_errorlogid", "STRING:request_errorlogid");
-        LOGFIELDS.put("request_method", "HTTP_METHOD:request_method");
-        LOGFIELDS.put("server_module_note_$", "STRING:server_module_note_$");
-        LOGFIELDS.put("response_header_$", "HTTP_HEADER:response_header_$");
-        LOGFIELDS.put("request_server_port_canonical", "PORT:request_server_port_canonical");
-        LOGFIELDS.put("connection_server_port_canonical", "PORT:connection_server_port_canonical");
-        LOGFIELDS.put("connection_server_port", "PORT:connection_server_port");
-        LOGFIELDS.put("connection_client_port", "PORT:connection_client_port");
-        LOGFIELDS.put("connection_server_child_processid", "NUMBER:connection_server_child_processid");
-        LOGFIELDS.put("connection_server_child_threadid", "NUMBER:connection_server_child_threadid");
-        LOGFIELDS.put("connection_server_child_hexthreadid", "NUMBER:connection_server_child_hexthreadid");
-        LOGFIELDS.put("request_querystring", "HTTP_QUERYSTRING:request_querystring");
-        LOGFIELDS.put("request_handler", "STRING:request_handler");
-        LOGFIELDS.put("request_status_original", "STRING:request_status_original");
-        LOGFIELDS.put("request_status_last", "STRING:request_status_last");
-        LOGFIELDS.put("request_receive_time_begin_msec", "TIME_EPOCH:request_receive_time_begin_msec");
-        LOGFIELDS.put("request_receive_time_end_msec", "TIME_EPOCH:request_receive_time_end_msec");
-        LOGFIELDS.put("request_receive_time_begin_usec", "TIME_EPOCH_USEC:request_receive_time_begin_usec");
-        LOGFIELDS.put("request_receive_time_begin_usec", "TIME_EPOCH_USEC:request_receive_time_begin_usec");
-        LOGFIELDS.put("request_receive_time_end_usec", "TIME_EPOCH_USEC:request_receive_time_end_usec");
-        LOGFIELDS.put("request_receive_time_begin_msec_frac", "TIME_EPOCH:request_receive_time_begin_msec_frac");
-        LOGFIELDS.put("request_receive_time_begin_msec_frac", "TIME_EPOCH:request_receive_time_begin_msec_frac");
-        LOGFIELDS.put("request_receive_time_end_msec_frac", "TIME_EPOCH:request_receive_time_end_msec_frac");
-        LOGFIELDS.put("request_receive_time_begin_usec_frac", "TIME_EPOCH_USEC_FRAC:request_receive_time_begin_usec_frac");
-        LOGFIELDS.put("request_receive_time_begin_usec_frac", "TIME_EPOCH_USEC_FRAC:request.receive.time.begin.usec_frac");
-        LOGFIELDS.put("request_receive_time_end_usec_frac", "TIME_EPOCH_USEC_FRAC:request_receive_time_end_usec_frac");
-        LOGFIELDS.put("response_server_processing_time", "SECONDS:response_server_processing_time");
-        LOGFIELDS.put("connection_client_user", "STRING:connection_client_user");
-        LOGFIELDS.put("request_urlpath", "URI:request_urlpath");
-        LOGFIELDS.put("connection_server_name_canonical", "STRING:connection_server_name_canonical");
-        LOGFIELDS.put("connection_server_name", "STRING:connection_server_name");
-        LOGFIELDS.put("response_connection_status", "HTTP_CONNECTSTATUS:response_connection_status");
-        LOGFIELDS.put("request_bytes", "BYTES:request_bytes");
-        LOGFIELDS.put("response_bytes", "BYTES:response_bytes");
-    }
+  public static final HashMap<String, String> LOGFIELDS = new HashMap<String, String>();
 
-    //Map map = Collections.synchronizedMap(LOGFIELDS);
+  static {
+    LOGFIELDS.put("connection.client.ip", "IP:connection.client.ip");
+    LOGFIELDS.put("connection.client.ip.last", "IP:connection.client.ip.last");
+    LOGFIELDS.put("connection.client.ip.original", "IP:connection.client.ip.original");
+    LOGFIELDS.put("connection.client.ip.last", "IP:connection.client.ip.last");
+    LOGFIELDS.put("connection.client.peerip", "IP:connection.client.peerip");
+    LOGFIELDS.put("connection.client.peerip.last", "IP:connection.client.peerip.last");
+    LOGFIELDS.put("connection.client.peerip.original", "IP:connection.client.peerip.original");
+    LOGFIELDS.put("connection.client.peerip.last", "IP:connection.client.peerip.last");
+    LOGFIELDS.put("connection.server.ip", "IP:connection.server.ip");
+    LOGFIELDS.put("connection.server.ip.last", "IP:connection.server.ip.last");
+    LOGFIELDS.put("connection.server.ip.original", "IP:connection.server.ip.original");
+    LOGFIELDS.put("connection.server.ip.last", "IP:connection.server.ip.last");
+    LOGFIELDS.put("response.body.bytes", "BYTES:response.body.bytes");
+    LOGFIELDS.put("response.body.bytes.last", "BYTES:response.body.bytes.last");
+    LOGFIELDS.put("response.body.bytes.original", "BYTES:response.body.bytes.original");
+    LOGFIELDS.put("response.body.bytes.last", "BYTES:response.body.bytes.last");
+    LOGFIELDS.put("response.body.bytesclf", "BYTES:response.body.bytesclf");
+    LOGFIELDS.put("response.body.bytes", "BYTESCLF:response.body.bytes");
+    LOGFIELDS.put("response.body.bytes.last", "BYTESCLF:response.body.bytes.last");
+    LOGFIELDS.put("response.body.bytes.original", "BYTESCLF:response.body.bytes.original");
+    LOGFIELDS.put("response.body.bytes.last", "BYTESCLF:response.body.bytes.last");
+    LOGFIELDS.put("request.cookies.foobar", "HTTP.COOKIE:request.cookies.foobar");
+    LOGFIELDS.put("server.environment.foobar", "VARIABLE:server.environment.foobar");
+    LOGFIELDS.put("server.filename", "FILENAME:server.filename");
+    LOGFIELDS.put("server.filename.last", "FILENAME:server.filename.last");
+    LOGFIELDS.put("server.filename.original", "FILENAME:server.filename.original");
+    LOGFIELDS.put("server.filename.last", "FILENAME:server.filename.last");
+    LOGFIELDS.put("connection.client.host", "IP:connection.client.host");
+    LOGFIELDS.put("connection.client.host.last", "IP:connection.client.host.last");
+    LOGFIELDS.put("connection.client.host.original", "IP:connection.client.host.original");
+    LOGFIELDS.put("connection.client.host.last", "IP:connection.client.host.last");
+    LOGFIELDS.put("request.protocol", "PROTOCOL:request.protocol");
+    LOGFIELDS.put("request.protocol.last", "PROTOCOL:request.protocol.last");
+    LOGFIELDS.put("request.protocol.original", "PROTOCOL:request.protocol.original");
+    LOGFIELDS.put("request.protocol.last", "PROTOCOL:request.protocol.last");
+    LOGFIELDS.put("request.header.foobar", "HTTP.HEADER:request.header.foobar");
+    LOGFIELDS.put("request.trailer.foobar", "HTTP.TRAILER:request.trailer.foobar");
+    LOGFIELDS.put("connection.keepalivecount", "NUMBER:connection.keepalivecount");
+    LOGFIELDS.put("connection.keepalivecount.last", "NUMBER:connection.keepalivecount.last");
+    LOGFIELDS.put("connection.keepalivecount.original", "NUMBER:connection.keepalivecount.original");
+    LOGFIELDS.put("connection.keepalivecount.last", "NUMBER:connection.keepalivecount.last");
+    LOGFIELDS.put("connection.client.logname", "NUMBER:connection.client.logname");
+    LOGFIELDS.put("connection.client.logname.last", "NUMBER:connection.client.logname.last");
+    LOGFIELDS.put("connection.client.logname.original", "NUMBER:connection.client.logname.original");
+    LOGFIELDS.put("connection.client.logname.last", "NUMBER:connection.client.logname.last");
+    LOGFIELDS.put("request.errorlogid", "STRING:request.errorlogid");
+    LOGFIELDS.put("request.errorlogid.last", "STRING:request.errorlogid.last");
+    LOGFIELDS.put("request.errorlogid.original", "STRING:request.errorlogid.original");
+    LOGFIELDS.put("request.errorlogid.last", "STRING:request.errorlogid.last");
+    LOGFIELDS.put("request.method", "HTTP.METHOD:request.method");
+    LOGFIELDS.put("request.method.last", "HTTP.METHOD:request.method.last");
+    LOGFIELDS.put("request.method.original", "HTTP.METHOD:request.method.original");
+    LOGFIELDS.put("request.method.last", "HTTP.METHOD:request.method.last");
+    LOGFIELDS.put("server.module_note.foobar", "STRING:server.module_note.foobar");
+    LOGFIELDS.put("response.header.foobar", "HTTP.HEADER:response.header.foobar");
+    LOGFIELDS.put("response.trailer.foobar", "HTTP.TRAILER:response.trailer.foobar");
+    LOGFIELDS.put("request.server.port.canonical", "PORT:request.server.port.canonical");
+    LOGFIELDS.put("request.server.port.canonical.last", "PORT:request.server.port.canonical.last");
+    LOGFIELDS.put("request.server.port.canonical.original", "PORT:request.server.port.canonical.original");
+    LOGFIELDS.put("request.server.port.canonical.last", "PORT:request.server.port.canonical.last");
+    LOGFIELDS.put("connection.server.port.canonical", "PORT:connection.server.port.canonical");
+    LOGFIELDS.put("connection.server.port.canonical.last", "PORT:connection.server.port.canonical.last");
+    LOGFIELDS.put("connection.server.port.canonical.original", "PORT:connection.server.port.canonical.original");
+    LOGFIELDS.put("connection.server.port.canonical.last", "PORT:connection.server.port.canonical.last");
+    LOGFIELDS.put("connection.server.port", "PORT:connection.server.port");
+    LOGFIELDS.put("connection.server.port.last", "PORT:connection.server.port.last");
+    LOGFIELDS.put("connection.server.port.original", "PORT:connection.server.port.original");
+    LOGFIELDS.put("connection.server.port.last", "PORT:connection.server.port.last");
+    LOGFIELDS.put("connection.client.port", "PORT:connection.client.port");
+    LOGFIELDS.put("connection.client.port.last", "PORT:connection.client.port.last");
+    LOGFIELDS.put("connection.client.port.original", "PORT:connection.client.port.original");
+    LOGFIELDS.put("connection.client.port.last", "PORT:connection.client.port.last");
+    LOGFIELDS.put("connection.server.child.processid", "NUMBER:connection.server.child.processid");
+    LOGFIELDS.put("connection.server.child.processid.last", "NUMBER:connection.server.child.processid.last");
+    LOGFIELDS.put("connection.server.child.processid.original", "NUMBER:connection.server.child.processid.original");
+    LOGFIELDS.put("connection.server.child.processid.last", "NUMBER:connection.server.child.processid.last");
+    LOGFIELDS.put("connection.server.child.processid", "NUMBER:connection.server.child.processid");
+    LOGFIELDS.put("connection.server.child.processid.last", "NUMBER:connection.server.child.processid.last");
+    LOGFIELDS.put("connection.server.child.processid.original", "NUMBER:connection.server.child.processid.original");
+    LOGFIELDS.put("connection.server.child.processid.last", "NUMBER:connection.server.child.processid.last");
+    LOGFIELDS.put("connection.server.child.threadid", "NUMBER:connection.server.child.threadid");
+    LOGFIELDS.put("connection.server.child.threadid.last", "NUMBER:connection.server.child.threadid.last");
+    LOGFIELDS.put("connection.server.child.threadid.original", "NUMBER:connection.server.child.threadid.original");
+    LOGFIELDS.put("connection.server.child.threadid.last", "NUMBER:connection.server.child.threadid.last");
+    LOGFIELDS.put("connection.server.child.hexthreadid", "NUMBER:connection.server.child.hexthreadid");
+    LOGFIELDS.put("connection.server.child.hexthreadid.last", "NUMBER:connection.server.child.hexthreadid.last");
+    LOGFIELDS.put("connection.server.child.hexthreadid.original", "NUMBER:connection.server.child.hexthreadid.original");
+    LOGFIELDS.put("connection.server.child.hexthreadid.last", "NUMBER:connection.server.child.hexthreadid.last");
+    LOGFIELDS.put("request.querystring", "HTTP.QUERYSTRING:request.querystring");
+    LOGFIELDS.put("request.querystring.last", "HTTP.QUERYSTRING:request.querystring.last");
+    LOGFIELDS.put("request.querystring.original", "HTTP.QUERYSTRING:request.querystring.original");
+    LOGFIELDS.put("request.querystring.last", "HTTP.QUERYSTRING:request.querystring.last");
+    LOGFIELDS.put("request.firstline", "HTTP.FIRSTLINE:request.firstline");
+    LOGFIELDS.put("request.firstline.original", "HTTP.FIRSTLINE:request.firstline.original");
+    LOGFIELDS.put("request.firstline.original", "HTTP.FIRSTLINE:request.firstline.original");
+    LOGFIELDS.put("request.firstline.last", "HTTP.FIRSTLINE:request.firstline.last");
+    LOGFIELDS.put("request.handler", "STRING:request.handler");
+    LOGFIELDS.put("request.handler.last", "STRING:request.handler.last");
+    LOGFIELDS.put("request.handler.original", "STRING:request.handler.original");
+    LOGFIELDS.put("request.handler.last", "STRING:request.handler.last");
+    LOGFIELDS.put("request.status", "STRING:request.status");
+    LOGFIELDS.put("request.status.original", "STRING:request.status.original");
+    LOGFIELDS.put("request.status.original", "STRING:request.status.original");
+    LOGFIELDS.put("request.status.last", "STRING:request.status.last");
+    LOGFIELDS.put("request.receive.time", "TIME.STAMP:request.receive.time");
+    LOGFIELDS.put("request.receive.time.last", "TIME.STAMP:request.receive.time.last");
+    LOGFIELDS.put("request.receive.time.original", "TIME.STAMP:request.receive.time.original");
+    LOGFIELDS.put("request.receive.time.last", "TIME.STAMP:request.receive.time.last");
+    LOGFIELDS.put("request.receive.time.year", "TIME.YEAR:request.receive.time.year");
+    LOGFIELDS.put("request.receive.time.begin.year", "TIME.YEAR:request.receive.time.begin.year");
+    LOGFIELDS.put("request.receive.time.end.year", "TIME.YEAR:request.receive.time.end.year");
+    LOGFIELDS.put("request.receive.time.sec", "TIME.SECONDS:request.receive.time.sec");
+    LOGFIELDS.put("request.receive.time.sec", "TIME.SECONDS:request.receive.time.sec");
+    LOGFIELDS.put("request.receive.time.sec.original", "TIME.SECONDS:request.receive.time.sec.original");
+    LOGFIELDS.put("request.receive.time.sec.last", "TIME.SECONDS:request.receive.time.sec.last");
+    LOGFIELDS.put("request.receive.time.begin.sec", "TIME.SECONDS:request.receive.time.begin.sec");
+    LOGFIELDS.put("request.receive.time.begin.sec.last", "TIME.SECONDS:request.receive.time.begin.sec.last");
+    LOGFIELDS.put("request.receive.time.begin.sec.original", "TIME.SECONDS:request.receive.time.begin.sec.original");
+    LOGFIELDS.put("request.receive.time.begin.sec.last", "TIME.SECONDS:request.receive.time.begin.sec.last");
+    LOGFIELDS.put("request.receive.time.end.sec", "TIME.SECONDS:request.receive.time.end.sec");
+    LOGFIELDS.put("request.receive.time.end.sec.last", "TIME.SECONDS:request.receive.time.end.sec.last");
+    LOGFIELDS.put("request.receive.time.end.sec.original", "TIME.SECONDS:request.receive.time.end.sec.original");
+    LOGFIELDS.put("request.receive.time.end.sec.last", "TIME.SECONDS:request.receive.time.end.sec.last");
+    LOGFIELDS.put("request.receive.time.begin.msec", "TIME.EPOCH:request.receive.time.begin.msec");
+    LOGFIELDS.put("request.receive.time.msec", "TIME.EPOCH:request.receive.time.msec");
+    LOGFIELDS.put("request.receive.time.msec.last", "TIME.EPOCH:request.receive.time.msec.last");
+    LOGFIELDS.put("request.receive.time.msec.original", "TIME.EPOCH:request.receive.time.msec.original");
+    LOGFIELDS.put("request.receive.time.msec.last", "TIME.EPOCH:request.receive.time.msec.last");
+    LOGFIELDS.put("request.receive.time.begin.msec", "TIME.EPOCH:request.receive.time.begin.msec");
+    LOGFIELDS.put("request.receive.time.begin.msec.last", "TIME.EPOCH:request.receive.time.begin.msec.last");
+    LOGFIELDS.put("request.receive.time.begin.msec.original", "TIME.EPOCH:request.receive.time.begin.msec.original");
+    LOGFIELDS.put("request.receive.time.begin.msec.last", "TIME.EPOCH:request.receive.time.begin.msec.last");
+    LOGFIELDS.put("request.receive.time.end.msec", "TIME.EPOCH:request.receive.time.end.msec");
+    LOGFIELDS.put("request.receive.time.end.msec.last", "TIME.EPOCH:request.receive.time.end.msec.last");
+    LOGFIELDS.put("request.receive.time.end.msec.original", "TIME.EPOCH:request.receive.time.end.msec.original");
+    LOGFIELDS.put("request.receive.time.end.msec.last", "TIME.EPOCH:request.receive.time.end.msec.last");
+    LOGFIELDS.put("request.receive.time.begin.usec", "TIME.EPOCH.USEC:request.receive.time.begin.usec");
+    LOGFIELDS.put("request.receive.time.usec", "TIME.EPOCH.USEC:request.receive.time.usec");
+    LOGFIELDS.put("request.receive.time.usec.last", "TIME.EPOCH.USEC:request.receive.time.usec.last");
+    LOGFIELDS.put("request.receive.time.usec.original", "TIME.EPOCH.USEC:request.receive.time.usec.original");
+    LOGFIELDS.put("request.receive.time.usec.last", "TIME.EPOCH.USEC:request.receive.time.usec.last");
+    LOGFIELDS.put("request.receive.time.begin.usec", "TIME.EPOCH.USEC:request.receive.time.begin.usec");
+    LOGFIELDS.put("request.receive.time.begin.usec.last", "TIME.EPOCH.USEC:request.receive.time.begin.usec.last");
+    LOGFIELDS.put("request.receive.time.begin.usec.original", "TIME.EPOCH.USEC:request.receive.time.begin.usec.original");
+    LOGFIELDS.put("request.receive.time.begin.usec.last", "TIME.EPOCH.USEC:request.receive.time.begin.usec.last");
+    LOGFIELDS.put("request.receive.time.end.usec", "TIME.EPOCH.USEC:request.receive.time.end.usec");
+    LOGFIELDS.put("request.receive.time.end.usec.last", "TIME.EPOCH.USEC:request.receive.time.end.usec.last");
+    LOGFIELDS.put("request.receive.time.end.usec.original", "TIME.EPOCH.USEC:request.receive.time.end.usec.original");
+    LOGFIELDS.put("request.receive.time.end.usec.last", "TIME.EPOCH.USEC:request.receive.time.end.usec.last");
+    LOGFIELDS.put("request.receive.time.begin.msec_frac", "TIME.EPOCH:request.receive.time.begin.msec_frac");
+    LOGFIELDS.put("request.receive.time.msec_frac", "TIME.EPOCH:request.receive.time.msec_frac");
+    LOGFIELDS.put("request.receive.time.msec_frac.last", "TIME.EPOCH:request.receive.time.msec_frac.last");
+    LOGFIELDS.put("request.receive.time.msec_frac.original", "TIME.EPOCH:request.receive.time.msec_frac.original");
+    LOGFIELDS.put("request.receive.time.msec_frac.last", "TIME.EPOCH:request.receive.time.msec_frac.last");
+    LOGFIELDS.put("request.receive.time.begin.msec_frac", "TIME.EPOCH:request.receive.time.begin.msec_frac");
+    LOGFIELDS.put("request.receive.time.begin.msec_frac.last", "TIME.EPOCH:request.receive.time.begin.msec_frac.last");
+    LOGFIELDS.put("request.receive.time.begin.msec_frac.original", "TIME.EPOCH:request.receive.time.begin.msec_frac.original");
+    LOGFIELDS.put("request.receive.time.begin.msec_frac.last", "TIME.EPOCH:request.receive.time.begin.msec_frac.last");
+    LOGFIELDS.put("request.receive.time.end.msec_frac", "TIME.EPOCH:request.receive.time.end.msec_frac");
+    LOGFIELDS.put("request.receive.time.end.msec_frac.last", "TIME.EPOCH:request.receive.time.end.msec_frac.last");
+    LOGFIELDS.put("request.receive.time.end.msec_frac.original", "TIME.EPOCH:request.receive.time.end.msec_frac.original");
+    LOGFIELDS.put("request.receive.time.end.msec_frac.last", "TIME.EPOCH:request.receive.time.end.msec_frac.last");
+    LOGFIELDS.put("request.receive.time.begin.usec_frac", "FRAC:request.receive.time.begin.usec_frac");
+    LOGFIELDS.put("request.receive.time.usec_frac", "FRAC:request.receive.time.usec_frac");
+    LOGFIELDS.put("request.receive.time.usec_frac.last", "FRAC:request.receive.time.usec_frac.last");
+    LOGFIELDS.put("request.receive.time.usec_frac.original", "FRAC:request.receive.time.usec_frac.original");
+    LOGFIELDS.put("request.receive.time.usec_frac.last", "FRAC:request.receive.time.usec_frac.last");
+    LOGFIELDS.put("request.receive.time.begin.usec_frac", "FRAC:request.receive.time.begin.usec_frac");
+    LOGFIELDS.put("request.receive.time.begin.usec_frac.last", "FRAC:request.receive.time.begin.usec_frac.last");
+    LOGFIELDS.put("request.receive.time.begin.usec_frac.original", "FRAC:request.receive.time.begin.usec_frac.original");
+    LOGFIELDS.put("request.receive.time.begin.usec_frac.last", "FRAC:request.receive.time.begin.usec_frac.last");
+    LOGFIELDS.put("request.receive.time.end.usec_frac", "FRAC:request.receive.time.end.usec_frac");
+    LOGFIELDS.put("request.receive.time.end.usec_frac.last", "FRAC:request.receive.time.end.usec_frac.last");
+    LOGFIELDS.put("request.receive.time.end.usec_frac.original", "FRAC:request.receive.time.end.usec_frac.original");
+    LOGFIELDS.put("request.receive.time.end.usec_frac.last", "FRAC:request.receive.time.end.usec_frac.last");
+    LOGFIELDS.put("response.server.processing.time", "SECONDS:response.server.processing.time");
+    LOGFIELDS.put("response.server.processing.time.original", "SECONDS:response.server.processing.time.original");
+    LOGFIELDS.put("response.server.processing.time.original", "SECONDS:response.server.processing.time.original");
+    LOGFIELDS.put("response.server.processing.time.last", "SECONDS:response.server.processing.time.last");
+    LOGFIELDS.put("server.process.time", "MICROSECONDS:server.process.time");
+    LOGFIELDS.put("response.server.processing.time", "MICROSECONDS:response.server.processing.time");
+    LOGFIELDS.put("response.server.processing.time.original", "MICROSECONDS:response.server.processing.time.original");
+    LOGFIELDS.put("response.server.processing.time.original", "MICROSECONDS:response.server.processing.time.original");
+    LOGFIELDS.put("response.server.processing.time.last", "MICROSECONDS:response.server.processing.time.last");
+    LOGFIELDS.put("response.server.processing.time", "MICROSECONDS:response.server.processing.time");
+    LOGFIELDS.put("response.server.processing.time.original", "MICROSECONDS:response.server.processing.time.original");
+    LOGFIELDS.put("response.server.processing.time.original", "MICROSECONDS:response.server.processing.time.original");
+    LOGFIELDS.put("response.server.processing.time.last", "MICROSECONDS:response.server.processing.time.last");
+    LOGFIELDS.put("response.server.processing.time", "MILLISECONDS:response.server.processing.time");
+    LOGFIELDS.put("response.server.processing.time.original", "MILLISECONDS:response.server.processing.time.original");
+    LOGFIELDS.put("response.server.processing.time.original", "MILLISECONDS:response.server.processing.time.original");
+    LOGFIELDS.put("response.server.processing.time.last", "MILLISECONDS:response.server.processing.time.last");
+    LOGFIELDS.put("response.server.processing.time", "SECONDS:response.server.processing.time");
+    LOGFIELDS.put("response.server.processing.time.original", "SECONDS:response.server.processing.time.original");
+    LOGFIELDS.put("response.server.processing.time.original", "SECONDS:response.server.processing.time.original");
+    LOGFIELDS.put("response.server.processing.time.last", "SECONDS:response.server.processing.time.last");
+    LOGFIELDS.put("connection.client.user", "STRING:connection.client.user");
+    LOGFIELDS.put("connection.client.user.last", "STRING:connection.client.user.last");
+    LOGFIELDS.put("connection.client.user.original", "STRING:connection.client.user.original");
+    LOGFIELDS.put("connection.client.user.last", "STRING:connection.client.user.last");
+    LOGFIELDS.put("request.urlpath", "URI:request.urlpath");
+    LOGFIELDS.put("request.urlpath.original", "URI:request.urlpath.original");
+    LOGFIELDS.put("request.urlpath.original", "URI:request.urlpath.original");
+    LOGFIELDS.put("request.urlpath.last", "URI:request.urlpath.last");
+    LOGFIELDS.put("connection.server.name.canonical", "STRING:connection.server.name.canonical");
+    LOGFIELDS.put("connection.server.name.canonical.last", "STRING:connection.server.name.canonical.last");
+    LOGFIELDS.put("connection.server.name.canonical.original", "STRING:connection.server.name.canonical.original");
+    LOGFIELDS.put("connection.server.name.canonical.last", "STRING:connection.server.name.canonical.last");
+    LOGFIELDS.put("connection.server.name", "STRING:connection.server.name");
+    LOGFIELDS.put("connection.server.name.last", "STRING:connection.server.name.last");
+    LOGFIELDS.put("connection.server.name.original", "STRING:connection.server.name.original");
+    LOGFIELDS.put("connection.server.name.last", "STRING:connection.server.name.last");
+    LOGFIELDS.put("response.connection.status", "HTTP.CONNECTSTATUS:response.connection.status");
+    LOGFIELDS.put("response.connection.status.last", "HTTP.CONNECTSTATUS:response.connection.status.last");
+    LOGFIELDS.put("response.connection.status.original", "HTTP.CONNECTSTATUS:response.connection.status.original");
+    LOGFIELDS.put("response.connection.status.last", "HTTP.CONNECTSTATUS:response.connection.status.last");
+    LOGFIELDS.put("request.bytes", "BYTES:request.bytes");
+    LOGFIELDS.put("request.bytes.last", "BYTES:request.bytes.last");
+    LOGFIELDS.put("request.bytes.original", "BYTES:request.bytes.original");
+    LOGFIELDS.put("request.bytes.last", "BYTES:request.bytes.last");
+    LOGFIELDS.put("response.bytes", "BYTES:response.bytes");
+    LOGFIELDS.put("response.bytes.last", "BYTES:response.bytes.last");
+    LOGFIELDS.put("response.bytes.original", "BYTES:response.bytes.original");
+    LOGFIELDS.put("response.bytes.last", "BYTES:response.bytes.last");
+    LOGFIELDS.put("total.bytes", "BYTES:total.bytes");
+    LOGFIELDS.put("total.bytes.last", "BYTES:total.bytes.last");
+    LOGFIELDS.put("total.bytes.original", "BYTES:total.bytes.original");
+    LOGFIELDS.put("total.bytes.last", "BYTES:total.bytes.last");
+    LOGFIELDS.put("request.cookies", "HTTP.COOKIES:request.cookies");
+    LOGFIELDS.put("request.cookies.last", "HTTP.COOKIES:request.cookies.last");
+    LOGFIELDS.put("request.cookies.original", "HTTP.COOKIES:request.cookies.original");
+    LOGFIELDS.put("request.cookies.last", "HTTP.COOKIES:request.cookies.last");
+    LOGFIELDS.put("response.cookies", "HTTP.SETCOOKIES:response.cookies");
+    LOGFIELDS.put("response.cookies.last", "HTTP.SETCOOKIES:response.cookies.last");
+    LOGFIELDS.put("response.cookies.original", "HTTP.SETCOOKIES:response.cookies.original");
+    LOGFIELDS.put("response.cookies.last", "HTTP.SETCOOKIES:response.cookies.last");
+    LOGFIELDS.put("request.user-agent", "HTTP.USERAGENT:request.user-agent");
+    LOGFIELDS.put("request.user-agent.last", "HTTP.USERAGENT:request.user-agent.last");
+    LOGFIELDS.put("request.user-agent.original", "HTTP.USERAGENT:request.user-agent.original");
+    LOGFIELDS.put("request.user-agent.last", "HTTP.USERAGENT:request.user-agent.last");
+    LOGFIELDS.put("request.referer", "HTTP.URI:request.referer");
+    LOGFIELDS.put("request.referer.last", "HTTP.URI:request.referer.last");
+    LOGFIELDS.put("request.referer.original", "HTTP.URI:request.referer.original");
+    LOGFIELDS.put("request.referer.last", "HTTP.URI:request.referer.last");
+  }
 
   public HttpdParser(final MapWriter mapWriter, final DrillBuf managedBuffer, final String logFormat,
-      final String timestampFormat, final Map<String, String> fieldMapping)
-      throws NoSuchMethodException, MissingDissectorsException, InvalidDissectorException {
+                     final String timestampFormat, final Map<String, String> fieldMapping)
+          throws NoSuchMethodException, MissingDissectorsException, InvalidDissectorException {
 
     Preconditions.checkArgument(logFormat != null && !logFormat.trim().isEmpty(), "logFormat cannot be null or empty");
 
-    this.record = new HttpdLogRecord(managedBuffer);
+    this.record = new HttpdLogRecord(managedBuffer, timestampFormat);
     this.parser = new HttpdLoglineParser<>(HttpdLogRecord.class, logFormat, timestampFormat);
 
     setupParser(mapWriter, logFormat, fieldMapping);
@@ -167,7 +317,6 @@ public class HttpdParser {
    * We do not expose the underlying parser or the record which is used to manage the writers.
    *
    * @param line log line to tear apart.
-   *
    * @throws DissectionFailure
    * @throws InvalidDissectorException
    * @throws MissingDissectorsException
@@ -181,7 +330,7 @@ public class HttpdParser {
    * In order to define a type remapping the format of the field configuration will look like: <br/>
    * HTTP.URI:request.firstline.uri.query.[parameter name] <br/>
    *
-   * @param parser Add type remapping to this parser instance.
+   * @param parser    Add type remapping to this parser instance.
    * @param fieldName request.firstline.uri.query.[parameter_name]
    * @param fieldType HTTP.URI, etc..
    */
@@ -198,11 +347,17 @@ public class HttpdParser {
    * @param drillFieldName name to be cleansed.
    * @return
    */
-  public static String parserFormattedFieldName(final String drillFieldName) {
-      String tempFieldName;
-      tempFieldName = LOGFIELDS.get(drillFieldName);
-      return tempFieldName.replace(SAFE_WILDCARD, PARSER_WILDCARD).replaceAll(SAFE_SEPARATOR, ".").replaceAll("\\.\\.", "_");
+  public static String parserFormattedFieldName(String drillFieldName) {
 
+    //The Useragent fields contain a dash which causes potential problems if the field name is not escaped properly
+    //This removes the dash
+    if (drillFieldName.contains("useragent")) {
+      drillFieldName = drillFieldName.replace("useragent", "user-agent");
+    }
+
+    String tempFieldName;
+    tempFieldName = LOGFIELDS.get(drillFieldName);
+    return tempFieldName.replace(SAFE_WILDCARD, PARSER_WILDCARD).replaceAll(SAFE_SEPARATOR, ".").replaceAll("\\.\\.", "_");
   }
 
   /**
@@ -213,19 +368,24 @@ public class HttpdParser {
    * @param parserFieldName name to be cleansed.
    * @return
    */
-  public static String drillFormattedFieldName(final String parserFieldName) {
+  public static String drillFormattedFieldName(String parserFieldName) {
 
-      if (parserFieldName.contains(":") ) {
-        String[] fieldPart= parserFieldName.split(":");
-        return fieldPart[1].replaceAll("_", "__").replace(PARSER_WILDCARD, SAFE_WILDCARD).replaceAll("\\.", SAFE_SEPARATOR);
-        }
-    else{
+    //The Useragent fields contain a dash which causes potential problems if the field name is not escaped properly
+    //This removes the dash
+    if (parserFieldName.contains("user-agent")) {
+      parserFieldName = parserFieldName.replace("user-agent", "useragent");
+    }
+
+    if (parserFieldName.contains(":")) {
+      String[] fieldPart = parserFieldName.split(":");
+      return fieldPart[1].replaceAll("_", "__").replace(PARSER_WILDCARD, SAFE_WILDCARD).replaceAll("\\.", SAFE_SEPARATOR);
+    } else {
       return parserFieldName.replaceAll("_", "__").replace(PARSER_WILDCARD, SAFE_WILDCARD).replaceAll("\\.", SAFE_SEPARATOR);
     }
   }
 
   private void setupParser(final MapWriter mapWriter, final String logFormat, final Map<String, String> fieldMapping)
-      throws NoSuchMethodException, MissingDissectorsException, InvalidDissectorException {
+          throws NoSuchMethodException, MissingDissectorsException, InvalidDissectorException {
 
     /**
      * If the user has selected fields, then we will use them to configure the parser because this would be the most
@@ -236,8 +396,7 @@ public class HttpdParser {
     if (fieldMapping != null && !fieldMapping.isEmpty()) {
       LOG.debug("Using fields defined by user.");
       requestedPaths = fieldMapping;
-    }
-    else {
+    } else {
       /**
        * Use all possible paths that the parser has determined from the specified log format.
        */
@@ -255,7 +414,6 @@ public class HttpdParser {
      */
     Parser<Object> dummy = new HttpdLoglineParser<>(Object.class, logFormat);
     dummy.addParseTarget(String.class.getMethod("indexOf", String.class), allParserPaths);
-
     for (final Map.Entry<String, String> entry : requestedPaths.entrySet()) {
       final EnumSet<Casts> casts;
 
@@ -270,10 +428,8 @@ public class HttpdParser {
 
         final String[] pieces = entry.getValue().split(":");
         addTypeRemapping(parser, pieces[1], pieces[0]);
-
         casts = Casts.STRING_ONLY;
-      }
-      else {
+      } else {
         casts = dummy.getCasts(entry.getValue());
       }
 
@@ -281,4 +437,4 @@ public class HttpdParser {
       record.addField(parser, mapWriter, casts, entry.getValue(), entry.getKey());
     }
   }
-}
\ No newline at end of file
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java
deleted file mode 100644
index 3897136..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.drill.exec.store.httpd;
-
-import io.netty.buffer.DrillBuf;
-import java.util.Map;
-import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class HttpdParserTest {
-
-  private static final Logger LOG = LoggerFactory.getLogger(HttpdParserTest.class);
-
-  private void runTest(String logFormat, String logLine) throws Exception {
-    MapWriter mapWriter = null;
-    DrillBuf managedBuffer = null;
-    Map<String, String> configuredFields = null;
-    HttpdParser parser = new HttpdParser(mapWriter, managedBuffer, logFormat, null, configuredFields);
-    parser.parse(logLine);
-  }
-
-//  @Test
-  public void testFirstPattern() throws Exception {
-    LOG.info("testFirstPattern");
-//    final String format = "common";
-//    final String format = "%h %l %u %t \"%r\" %>s %b";
-    final String format = "%h %t \"%r\" %>s %b \"%{Referer}i\"";
-    final String line = "127.0.0.1 [31/Dec/2012:23:49:41 +0100] "
-        + "\"GET /foo HTTP/1.1\" 200 "
-        + "1213 \"http://localhost/index.php?mies=wim\"";
-    runTest(format, line);
-  }
-
-}
\ No newline at end of file
diff --git a/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json b/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
index 46f1620..afcf53d 100644
--- a/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
+++ b/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
@@ -1,70 +1,70 @@
 {
   "storage":{
-    dfs: {
-      type: "file",
-      connection: "file:///",
-      workspaces: {
+    "dfs": {
+      "type": "file",
+      "connection": "file:///",
+      "workspaces": {
         "root" : {
-          location: "/",
-          writable: false,
-          allowAccessOutsideWorkspace: false
+          "location": "/",
+          "writable": false,
+          "allowAccessOutsideWorkspace": false
         },
         "tmp" : {
-          location: "/tmp",
-          writable: true,
-          allowAccessOutsideWorkspace: false
+          "location": "/tmp",
+          "writable": true,
+          "allowAccessOutsideWorkspace": false
         }
       },
-      formats: {
+      "formats": {
         "psv" : {
-          type: "text",
-          extensions: [ "tbl" ],
-          delimiter: "|"
+          "type": "text",
+          "extensions": [ "tbl" ],
+          "delimiter": "|"
         },
         "csv" : {
-          type: "text",
-          extensions: [ "csv" ],
-          delimiter: ","
+          "type": "text",
+          "extensions": [ "csv" ],
+          "delimiter": ","
         },
         "tsv" : {
-          type: "text",
-          extensions: [ "tsv" ],
-          delimiter: "\t"
+          "type": "text",
+          "extensions": [ "tsv" ],
+          "delimiter": "\t"
         },
         "httpd" : {
-          type: "httpd",
-          logFormat: "%h %t \"%r\" %>s %b \"%{Referer}i\""
-          /* timestampFormat: "dd/MMM/yyyy:HH:mm:ss ZZ" */
+          "type": "httpd",
+          "logFormat": "%h %l %u %t \"%r\" %>s %b \"%{Referer}i\" \"%{User-agent}i\"",
+          "timestampFormat": "dd/MMM/yyyy:HH:mm:ss ZZ"
         },
         "parquet" : {
-          type: "parquet"
+          "type": "parquet"
         },
         "json" : {
-          type: "json",
-          extensions: [ "json" ]
+          "type": "json",
+          "extensions": [ "json" ]
         },
         "pcap" : {
-          type: "pcap"
+          "type": "pcap"
         },
         "pcapng" : {
-          type: "pcapng"
+          "type": "pcapng"
         },
         "avro" : {
-          type: "avro"
+          "type": "avro"
         },
         "sequencefile": {
-          type : "sequencefile",
-          extensions: [ "seq" ]
+          "type": "sequencefile",
+          "extensions": [ "seq" ]
         },
         "csvh" : {
-          type: "text",
-          extensions: [ "csvh" ],
-          delimiter: ",",
-          extractHeader: true
+          "type": "text",
+          "extensions": [ "csvh" ],
+          "delimiter": ",",
+          "extractHeader": true
         },
         "image" : {
-          type: "image",
-          extensions: [
+          "type": "image",
+          "extensions": [
             "jpg", "jpeg", "jpe", "tif", "tiff", "dng", "psd", "png", "bmp", "gif",
             "ico", "pcx", "wav", "wave", "avi", "webp", "mov", "mp4", "m4a", "m4p",
             "m4b", "m4r", "m4v", "3gp", "3g2", "eps", "epsf", "epsi", "ai", "arw",
@@ -72,99 +72,99 @@
           ]
         }
       },
-      enabled : true
+      "enabled" : true
     },
 
-    s3: {
-      type: "file",
-      connection: "s3a://my.bucket.location.com",
-      config : {
+    "s3": {
+      "type": "file",
+      "connection": "s3a://my.bucket.location.com",
+      "config" : {
         "fs.s3a.access.key": "ID",
         "fs.s3a.secret.key": "SECRET"
       },
-      workspaces: {
+      "workspaces": {
         "root" : {
-          location: "/",
-          writable: false
+          "location": "/",
+          "writable": false
         },
         "tmp" : {
-          location: "/tmp",
-          writable: true
+          "location": "/tmp",
+          "writable": true
         }
       },
-      formats: {
+      "formats": {
         "psv" : {
-          type: "text",
-          extensions: [ "tbl" ],
-          delimiter: "|"
+          "type": "text",
+          "extensions": [ "tbl" ],
+          "delimiter": "|"
         },
         "csv" : {
-          type: "text",
-          extensions: [ "csv" ],
-          delimiter: ","
+          "type": "text",
+          "extensions": [ "csv" ],
+          "delimiter": ","
         },
         "tsv" : {
-          type: "text",
-          extensions: [ "tsv" ],
-          delimiter: "\t"
+          "type": "text",
+          "extensions": [ "tsv" ],
+          "delimiter": "\t"
         },
         "parquet" : {
-          type: "parquet"
+          "type": "parquet"
         },
         "json" : {
-          type: "json",
-          extensions: [ "json" ]
+          "type": "json",
+          "extensions": [ "json" ]
         },
         "avro" : {
-          type: "avro"
+          "type": "avro"
         },
         "sequencefile": {
-          type : "sequencefile",
-          extensions: [ "seq" ]
+          "type": "sequencefile",
+          "extensions": [ "seq" ]
         },
         "csvh" : {
-          type: "text",
-          extensions: [ "csvh" ],
-          delimiter: ",",
-          extractHeader: true
+          "type": "text",
+          "extensions": [ "csvh" ],
+          "delimiter": ",",
+          "extractHeader": true
         }
       },
-      enabled : false
+      "enabled" : false
     },
 
-    cp: {
-      type: "file",
-      connection: "classpath:///",
-      formats: {
+    "cp": {
+      "type": "file",
+      "connection": "classpath:///",
+      "formats": {
         "csv" : {
-          type: "text",
-          extensions: [ "csv" ],
-          delimiter: ","
+          "type": "text",
+          "extensions": [ "csv" ],
+          "delimiter": ","
         },
         "tsv" : {
-                  type: "text",
-                  extensions: [ "tsv" ],
-                  delimiter: "\t"
+                  "type": "text",
+                  "extensions": [ "tsv" ],
+                  "delimiter": "\t"
         },
         "json" : {
-          type: "json",
-          extensions: [ "json" ]
+          "type": "json",
+          "extensions": [ "json" ]
         },
         "parquet" : {
-          type: "parquet"
+          "type": "parquet"
         },
         "avro" : {
-          type: "avro"
+          "type": "avro"
         },
         "csvh" : {
-          type: "text",
-          extensions: [ "csvh" ],
-          delimiter: ",",
-          extractHeader: true
+          "type": "text",
+          "extensions": [ "csvh" ],
+          "delimiter": ",",
+          "extractHeader": true
         },
         "image" : {
-          type: "image",
-          extensions: [
+          "type": "image",
+          "extensions": [
             "jpg", "jpeg", "jpe", "tif", "tiff", "dng", "psd", "png", "bmp", "gif",
             "ico", "pcx", "wav", "wave", "avi", "webp", "mov", "mp4", "m4a", "m4p",
             "m4b", "m4r", "m4v", "3gp", "3g2", "eps", "epsf", "epsi", "ai", "arw",
@@ -172,7 +172,7 @@
           ]
         }
       },
-      enabled : true
+      "enabled" : true
     }
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/FormatPluginSerDeTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/FormatPluginSerDeTest.java
index 7e68dd4..b4c8b14 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/FormatPluginSerDeTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/FormatPluginSerDeTest.java
@@ -93,9 +93,9 @@ public class FormatPluginSerDeTest extends PlanTestBase {
 
   @Test
   public void testHttpd() throws Exception {
-    String path = "store/httpd/dfs-bootstrap.httpd";
+    String path = "store/httpd/dfs-test-bootstrap-test.httpd";
     dirTestWatcher.copyResourceToRoot(Paths.get(path));
-    String logFormat = "%h %t \"%r\" %>s %b \"%{Referer}i\"";
+    String logFormat = "%h %l %u %t \"%r\" %>s %b \"%{Referer}i\" \"%{User-agent}i\"";
     String timeStampFormat = "dd/MMM/yyyy:HH:mm:ss ZZ";
     testPhysicalPlanSubmission(
         String.format("select * from dfs.`%s`", path),
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHTTPDLogReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHTTPDLogReader.java
new file mode 100644
index 0000000..ac85a92
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHTTPDLogReader.java
@@ -0,0 +1,237 @@
+/*
+ * 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.drill.exec.store.httpd;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.dfs.FileSystemConfig;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.test.BaseDirTestWatcher;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSetUtilities;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.time.LocalDateTime;
+import java.util.HashMap;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHTTPDLogReader extends ClusterTest {
+
+  @ClassRule
+  public static final BaseDirTestWatcher dirTestWatcher = new BaseDirTestWatcher();
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterTest.startCluster(ClusterFixture.builder(dirTestWatcher));
+    defineHTTPDPlugin();
+  }
+
+  private static void defineHTTPDPlugin() throws ExecutionSetupException {
+
+    // Create an instance of the regex config.
+    // Note: we can"t use the ".log" extension; the Drill .gitignore
+    // file ignores such files, so they"ll never get committed. Instead,
+    // make up a fake suffix.
+    HttpdLogFormatConfig sampleConfig = new HttpdLogFormatConfig();
+    sampleConfig.setLogFormat("%h %l %u %t \"%r\" %>s %b \"%{Referer}i\" \"%{User-agent}i\"");
+
+    // Define a temporary format plugin for the "cp" storage plugin.
+    Drillbit drillbit = cluster.drillbit();
+    final StoragePluginRegistry pluginRegistry = drillbit.getContext().getStorage();
+    final FileSystemPlugin plugin = (FileSystemPlugin) pluginRegistry.getPlugin("cp");
+    final FileSystemConfig pluginConfig = (FileSystemConfig) plugin.getConfig();
+    pluginConfig.getFormats().put("sample", sampleConfig);
+    pluginRegistry.createOrUpdate("cp", pluginConfig, false);
+  }
+
+  @Test
+  public void testDateField() throws RpcException {
+    String sql = "SELECT `request_receive_time` FROM cp.`httpd/hackers-access-small.httpd` LIMIT 5";
+    RowSet results = client.queryBuilder().sql(sql).rowSet();
+
+    BatchSchema expectedSchema = new SchemaBuilder()
+            .addNullable("request_receive_time", MinorType.TIMESTAMP)
+            .build();
+    RowSet expected = client.rowSetBuilder(expectedSchema)
+            .addRow(1445742685000L)
+            .addRow(1445742686000L)
+            .addRow(1445742687000L)
+            .addRow(1445743471000L)
+            .addRow(1445743472000L)
+            .build();
+
+    RowSetUtilities.verify(expected, results);
+  }
+
+  @Test
+  public void testSelectColumns() throws Exception {
+    String sql = "SELECT request_referer_ref,\n" +
+            "request_receive_time_last_time,\n" +
+            "request_firstline_uri_protocol,\n" +
+            "request_receive_time_microsecond,\n" +
+            "request_receive_time_last_microsecond__utc,\n" +
+            "request_firstline_original_protocol,\n" +
+            "request_firstline_original_uri_host,\n" +
+            "request_referer_host,\n" +
+            "request_receive_time_month__utc,\n" +
+            "request_receive_time_last_minute,\n" +
+            "request_firstline_protocol_version,\n" +
+            "request_receive_time_time__utc,\n" +
+            "request_referer_last_ref,\n" +
+            "request_receive_time_last_timezone,\n" +
+            "request_receive_time_last_weekofweekyear,\n" +
+            "request_referer_last,\n" +
+            "request_receive_time_minute,\n" +
+            "connection_client_host_last,\n" +
+            "request_receive_time_last_millisecond__utc,\n" +
+            "request_firstline_original_uri,\n" +
+            "request_firstline,\n" +
+            "request_receive_time_nanosecond,\n" +
+            "request_receive_time_last_millisecond,\n" +
+            "request_receive_time_day,\n" +
+            "request_referer_port,\n" +
+            "request_firstline_original_uri_port,\n" +
+            "request_receive_time_year,\n" +
+            "request_receive_time_last_date,\n" +
+            "request_receive_time_last_time__utc,\n" +
+            "request_receive_time_last_hour__utc,\n" +
+            "request_firstline_original_protocol_version,\n" +
+            "request_firstline_original_method,\n" +
+            "request_receive_time_last_year__utc,\n" +
+            "request_firstline_uri,\n" +
+            "request_referer_last_host,\n" +
+            "request_receive_time_last_minute__utc,\n" +
+            "request_receive_time_weekofweekyear,\n" +
+            "request_firstline_uri_userinfo,\n" +
+            "request_receive_time_epoch,\n" +
+            "connection_client_logname,\n" +
+            "response_body_bytes,\n" +
+            "request_receive_time_nanosecond__utc,\n" +
+            "request_firstline_protocol,\n" +
+            "request_receive_time_microsecond__utc,\n" +
+            "request_receive_time_hour,\n" +
+            "request_firstline_uri_host,\n" +
+            "request_referer_last_port,\n" +
+            "request_receive_time_last_epoch,\n" +
+            "request_receive_time_last_weekyear__utc,\n" +
+            "request_useragent,\n" +
+            "request_receive_time_weekyear,\n" +
+            "request_receive_time_timezone,\n" +
+            "response_body_bytesclf,\n" +
+            "request_receive_time_last_date__utc,\n" +
+            "request_receive_time_millisecond__utc,\n" +
+            "request_referer_last_protocol,\n" +
+            "request_status_last,\n" +
+            "request_firstline_uri_query,\n" +
+            "request_receive_time_minute__utc,\n" +
+            "request_firstline_original_uri_protocol,\n" +
+            "request_referer_query,\n" +
+            "request_receive_time_date,\n" +
+            "request_firstline_uri_port,\n" +
+            "request_receive_time_last_second__utc,\n" +
+            "request_referer_last_userinfo,\n" +
+            "request_receive_time_last_second,\n" +
+            "request_receive_time_last_monthname__utc,\n" +
+            "request_firstline_method,\n" +
+            "request_receive_time_last_month__utc,\n" +
+            "request_receive_time_millisecond,\n" +
+            "request_receive_time_day__utc,\n" +
+            "request_receive_time_year__utc,\n" +
+            "request_receive_time_weekofweekyear__utc,\n" +
+            "request_receive_time_second,\n" +
+            "request_firstline_original_uri_ref,\n" +
+            "connection_client_logname_last,\n" +
+            "request_receive_time_last_year,\n" +
+            "request_firstline_original_uri_path,\n" +
+            "connection_client_host,\n" +
+            "request_firstline_original_uri_query,\n" +
+            "request_referer_userinfo,\n" +
+            "request_receive_time_last_monthname,\n" +
+            "request_referer_path,\n" +
+            "request_receive_time_monthname,\n" +
+            "request_receive_time_last_month,\n" +
+            "request_referer_last_query,\n" +
+            "request_firstline_uri_ref,\n" +
+            "request_receive_time_last_day,\n" +
+            "request_receive_time_time,\n" +
+            "request_receive_time_last_weekofweekyear__utc,\n" +
+            "request_useragent_last,\n" +
+            "request_receive_time_last_weekyear,\n" +
+            "request_receive_time_last_microsecond,\n" +
+            "request_firstline_original,\n" +
+            "request_referer_last_path,\n" +
+            "request_receive_time_month,\n" +
+            "request_receive_time_last_day__utc,\n" +
+            "request_referer,\n" +
+            "request_referer_protocol,\n" +
+            "request_receive_time_monthname__utc,\n" +
+            "response_body_bytes_last,\n" +
+            "request_receive_time,\n" +
+            "request_receive_time_last_nanosecond,\n" +
+            "request_firstline_uri_path,\n" +
+            "request_firstline_original_uri_userinfo,\n" +
+            "request_receive_time_date__utc,\n" +
+            "request_receive_time_last,\n" +
+            "request_receive_time_last_nanosecond__utc,\n" +
+            "request_receive_time_last_hour,\n" +
+            "request_receive_time_hour__utc,\n" +
+            "request_receive_time_second__utc,\n" +
+            "connection_client_user_last,\n" +
+            "request_receive_time_weekyear__utc,\n" +
+            "connection_client_user\n" +
+            "FROM cp.`httpd/hackers-access-small.httpd`\n" +
+            "LIMIT 1";
+
+    testBuilder()
+            .sqlQuery(sql)
+            .unOrdered()
+            .baselineColumns("request_referer_ref", "request_receive_time_last_time", "request_firstline_uri_protocol", "request_receive_time_microsecond", "request_receive_time_last_microsecond__utc", "request_firstline_original_protocol", "request_firstline_original_uri_host", "request_referer_host", "request_receive_time_month__utc", "request_receive_time_last_minute", "request_firstline_protocol_version", "request_receive_time_time__utc", "request_referer_last_ref", "request_receive_ [...]
+            .baselineValues(null, "04:11:25", null, 0L, 0L, "HTTP", null, "howto.basjes.nl", 10L, 11L, "1.1", "03:11:25", null, null, 43L, "http://howto.basjes.nl/", 11L, "195.154.46.135", 0L, "/linux/doing-pxe-without-dhcp-control", "GET /linux/doing-pxe-without-dhcp-control HTTP/1.1", 0L, 0L, 25L, null, null, 2015L, "2015-10-25", "03:11:25", 3L, "1.1", "GET", 2015L, "/linux/doing-pxe-without-dhcp-control", "howto.basjes.nl", 11L, 43L, null, 1445742685000L, null, 24323L, 0L, "HTTP", 0L, [...]
+            .go();
+  }
+
+
+  @Test
+  public void testCount() throws Exception {
+    String sql = "SELECT COUNT(*) FROM cp.`httpd/hackers-access-small.httpd`";
+    long result = client.queryBuilder().sql(sql).singletonLong();
+    assertEquals(10, result);
+  }
+
+  @Test
+  public void testStar() throws Exception {
+    String sql = "SELECT * FROM cp.`httpd/hackers-access-small.httpd` LIMIT 1";
+
+    testBuilder()
+            .sqlQuery(sql)
+            .unOrdered()
+            .baselineColumns("request_referer_ref","request_receive_time_last_time","request_firstline_uri_protocol","request_receive_time_microsecond","request_receive_time_last_microsecond__utc","request_firstline_original_uri_query_$","request_firstline_original_protocol","request_firstline_original_uri_host","request_referer_host","request_receive_time_month__utc","request_receive_time_last_minute","request_firstline_protocol_version","request_receive_time_time__utc","request_referer [...]
+            .baselineValues(null,"04:11:25",null,0L,0L,new HashMap<>(),"HTTP",null,"howto.basjes.nl",10L,11L,"1.1","03:11:25",null,null,43L,"http://howto.basjes.nl/",11L,"195.154.46.135",0L,"/linux/doing-pxe-without-dhcp-control","GET /linux/doing-pxe-without-dhcp-control HTTP/1.1",0L,0L,25L,null,null,2015L,"2015-10-25",new HashMap<>(),"03:11:25",3L,"1.1","GET",2015L,"/linux/doing-pxe-without-dhcp-control","howto.basjes.nl",11L,43L,null,1445742685000L,null,24323L,0L,"HTTP",0L,4L,null,nul [...]
+            .go();
+  }
+}
diff --git a/exec/java-exec/src/test/resources/httpd/hackers-access-small.httpd b/exec/java-exec/src/test/resources/httpd/hackers-access-small.httpd
new file mode 100644
index 0000000..98af532
--- /dev/null
+++ b/exec/java-exec/src/test/resources/httpd/hackers-access-small.httpd
@@ -0,0 +1,10 @@
+195.154.46.135 - - [25/Oct/2015:04:11:25 +0100] "GET /linux/doing-pxe-without-dhcp-control HTTP/1.1" 200 24323 "http://howto.basjes.nl/" "Mozilla/5.0 (Windows NT 5.1; rv:35.0) Gecko/20100101 Firefox/35.0"
+23.95.237.180 - - [25/Oct/2015:04:11:26 +0100] "GET /join_form HTTP/1.0" 200 11114 "http://howto.basjes.nl/" "Mozilla/5.0 (Windows NT 5.1; rv:35.0) Gecko/20100101 Firefox/35.0"
+23.95.237.180 - - [25/Oct/2015:04:11:27 +0100] "POST /join_form HTTP/1.1" 302 9093 "http://howto.basjes.nl/join_form" "Mozilla/5.0 (Windows NT 5.1; rv:35.0) Gecko/20100101 Firefox/35.0"
+158.222.5.157 - - [25/Oct/2015:04:24:31 +0100] "GET /join_form HTTP/1.0" 200 11114 "http://howto.basjes.nl/" "Mozilla/5.0 (Windows NT 6.3; WOW64; rv:34.0) Gecko/20100101 Firefox/34.0 AlexaToolbar/alxf-2.21"
+158.222.5.157 - - [25/Oct/2015:04:24:32 +0100] "POST /join_form HTTP/1.1" 302 9093 "http://howto.basjes.nl/join_form" "Mozilla/5.0 (Windows NT 6.3; WOW64; rv:34.0) Gecko/20100101 Firefox/34.0 AlexaToolbar/alxf-2.21"
+158.222.5.157 - - [25/Oct/2015:04:24:37 +0100] "GET /acl_users/credentials_cookie_auth/require_login?came_from=http%3A//howto.basjes.nl/join_form HTTP/1.1" 200 10716 "http://howto.basjes.nl/join_form" "Mozilla/5.0 (Windows NT 6.3; WOW64; rv:34.0) Gecko/20100101 Firefox/34.0 AlexaToolbar/alxf-2.21"
+158.222.5.157 - - [25/Oct/2015:04:24:39 +0100] "GET /login_form HTTP/1.1" 200 10543 "http://howto.basjes.nl/" "Mozilla/5.0 (Windows NT 6.3; WOW64; rv:34.0) Gecko/20100101 Firefox/34.0 AlexaToolbar/alxf-2.21"
+158.222.5.157 - - [25/Oct/2015:04:24:41 +0100] "POST /login_form HTTP/1.1" 200 16810 "http://howto.basjes.nl/login_form" "Mozilla/5.0 (Windows NT 6.3; WOW64; rv:34.0) Gecko/20100101 Firefox/34.0 AlexaToolbar/alxf-2.21"
+5.39.5.5 - - [25/Oct/2015:04:32:22 +0100] "GET /join_form HTTP/1.1" 200 11114 "http://howto.basjes.nl/" "Mozilla/5.0 (Windows NT 5.1; rv:34.0) Gecko/20100101 Firefox/34.0"
+180.180.64.16 - - [25/Oct/2015:04:34:37 +0100] "GET /linux/doing-pxe-without-dhcp-control HTTP/1.1" 200 24323 "http://howto.basjes.nl/" "Mozilla/5.0 (Windows NT 5.1; rv:35.0) Gecko/20100101 Firefox/35.0"