You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2017/11/22 22:52:04 UTC

[02/12] drill git commit: DRILL-5771: Fix serDe errors for format plugins

DRILL-5771: Fix serDe errors for format plugins

1. Fix various serde issues for format plugins described in DRILL-5771.
2. Throw meaninful exception instead of NPE when table is not found when table function is used.
3. Added unit tests for all format plugins for ensure serde is checked (physical plan is generated in json format and then submitted).
4. Fix physical plan submission on Windows (DRILL-4640).

This closes #1014


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/7506cfbb
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/7506cfbb
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/7506cfbb

Branch: refs/heads/master
Commit: 7506cfbb5c8522d371c12dbdc2268d48a9449a48
Parents: d4c61ca
Author: Arina Ielchiieva <ar...@gmail.com>
Authored: Thu Oct 26 16:09:00 2017 +0300
Committer: Parth Chandra <pa...@apache.org>
Committed: Wed Nov 22 10:27:25 2017 -0800

----------------------------------------------------------------------
 .../drill/exec/store/avro/AvroFormatPlugin.java |  11 +-
 .../exec/store/dfs/BasicFormatMatcher.java      |   8 +-
 .../drill/exec/store/dfs/FileSelection.java     |   5 +-
 .../drill/exec/store/dfs/FileSystemPlugin.java  |  27 +++--
 .../exec/store/dfs/WorkspaceSchemaFactory.java  |   9 +-
 .../drill/exec/store/dfs/easy/EasySubScan.java  |  11 +-
 .../exec/store/httpd/HttpdLogFormatPlugin.java  |  55 ++++++---
 .../exec/store/parquet/ParquetGroupScan.java    |   4 +-
 .../exec/store/parquet/ParquetRowGroupScan.java |  28 +++--
 .../drill/exec/store/pcap/PcapFormatConfig.java |  10 ++
 .../drill/exec/store/pcap/PcapFormatPlugin.java |   9 +-
 .../java/org/apache/drill/PlanTestBase.java     |  13 +++
 .../org/apache/drill/TestSelectWithOption.java  |  17 ++-
 .../drill/TestTpchDistributedConcurrent.java    |   2 +-
 .../drill/exec/store/FormatPluginSerDeTest.java | 113 +++++++++++++++++++
 15 files changed, 243 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
index 07b99f1..fd6e59b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -37,7 +37,6 @@ import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.apache.drill.exec.store.dfs.FormatMatcher;
 import org.apache.drill.exec.store.dfs.FormatSelection;
 import org.apache.drill.exec.store.dfs.MagicString;
-import org.apache.drill.exec.store.dfs.NamedFormatPluginConfig;
 import org.apache.drill.exec.store.dfs.easy.EasyFormatPlugin;
 import org.apache.drill.exec.store.dfs.easy.EasyWriter;
 import org.apache.drill.exec.store.dfs.easy.FileWork;
@@ -105,13 +104,7 @@ public class AvroFormatPlugin extends EasyFormatPlugin<AvroFormatConfig> {
         FileSelection selection, FileSystemPlugin fsPlugin,
         String storageEngineName, String userName) throws IOException {
       if (isFileReadable(fs, selection.getFirstPath(fs))) {
-        if (plugin.getName() != null) {
-          NamedFormatPluginConfig namedConfig = new NamedFormatPluginConfig();
-          namedConfig.name = plugin.getName();
-          return new AvroDrillTable(storageEngineName, fsPlugin, userName, new FormatSelection(namedConfig, selection));
-        } else {
-          return new AvroDrillTable(storageEngineName, fsPlugin, userName, new FormatSelection(plugin.getConfig(), selection));
-        }
+        return new AvroDrillTable(storageEngineName, fsPlugin, userName, new FormatSelection(plugin.getConfig(), selection));
       }
       return null;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
index fb51bfc..6526073 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
@@ -77,13 +77,7 @@ public class BasicFormatMatcher extends FormatMatcher{
       FileSelection selection, FileSystemPlugin fsPlugin,
       String storageEngineName, String userName) throws IOException {
     if (isFileReadable(fs, selection.getFirstPath(fs))) {
-      if (plugin.getName() != null) {
-        NamedFormatPluginConfig namedConfig = new NamedFormatPluginConfig();
-        namedConfig.name = plugin.getName();
-        return new DynamicDrillTable(fsPlugin, storageEngineName, userName, new FormatSelection(namedConfig, selection));
-      } else {
-        return new DynamicDrillTable(fsPlugin, storageEngineName, userName, new FormatSelection(plugin.getConfig(), selection));
-      }
+      return new DynamicDrillTable(fsPlugin, storageEngineName, userName, new FormatSelection(plugin.getConfig(), selection));
     }
     return null;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java
index 293259f..6aff1dd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.fs.Path;
  */
 public class FileSelection {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FileSelection.class);
-  private static final String PATH_SEPARATOR = System.getProperty("file.separator");
   private static final String WILD_CARD = "*";
 
   private List<FileStatus> statuses;
@@ -224,7 +223,7 @@ public class FileSelection {
     int shortest = Integer.MAX_VALUE;
     for (int i = 0; i < total; i++) {
       final Path path = new Path(files.get(i));
-      folders[i] = Path.getPathWithoutSchemeAndAuthority(path).toString().split(PATH_SEPARATOR);
+      folders[i] = Path.getPathWithoutSchemeAndAuthority(path).toString().split(Path.SEPARATOR);
       shortest = Math.min(shortest, folders[i].length);
     }
 
@@ -247,7 +246,7 @@ public class FileSelection {
   private static String buildPath(final String[] path, final int folderIndex) {
     final StringBuilder builder = new StringBuilder();
     for (int i=0; i<folderIndex; i++) {
-      builder.append(path[i]).append(PATH_SEPARATOR);
+      builder.append(path[i]).append(Path.SEPARATOR);
     }
     builder.deleteCharAt(builder.length()-1);
     return builder.toString();

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
index 4cb1b45..5d382fe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -133,15 +133,7 @@ public class FileSystemPlugin extends AbstractStoragePlugin {
   public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection, List<SchemaPath> columns)
       throws IOException {
     FormatSelection formatSelection = selection.getWith(lpPersistance, FormatSelection.class);
-    FormatPlugin plugin;
-    if (formatSelection.getFormat() instanceof NamedFormatPluginConfig) {
-      plugin = formatCreator.getFormatPluginByName( ((NamedFormatPluginConfig) formatSelection.getFormat()).name);
-    } else {
-      plugin = formatPluginsByConfig.get(formatSelection.getFormat());
-    }
-    if (plugin == null) {
-      plugin = formatCreator.newFormatPlugin(formatSelection.getFormat());
-    }
+    FormatPlugin plugin = getFormatPlugin(formatSelection.getFormat());
     return plugin.getGroupScan(userName, formatSelection.getSelection(), columns);
   }
 
@@ -154,12 +146,23 @@ public class FileSystemPlugin extends AbstractStoragePlugin {
     return formatCreator.getFormatPluginByName(name);
   }
 
+  /**
+   * If format plugin configuration is for named format plugin, will return format plugin from pre-loaded list by name.
+   * For other cases will try to find format plugin by its configuration, if not present will attempt to create one.
+   *
+   * @param config format plugin configuration
+   * @return format plugin for given configuration if found, null otherwise
+   */
   public FormatPlugin getFormatPlugin(FormatPluginConfig config) {
     if (config instanceof NamedFormatPluginConfig) {
       return formatCreator.getFormatPluginByName(((NamedFormatPluginConfig) config).name);
-    } else {
-      return formatPluginsByConfig.get(config);
     }
+
+    FormatPlugin plugin = formatPluginsByConfig.get(config);
+    if (plugin == null) {
+      plugin = formatCreator.newFormatPlugin(config);
+    }
+    return plugin;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
index 605ba18..6629fc4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
@@ -251,7 +251,14 @@ public class WorkspaceSchemaFactory {
 
     @Override
     public TranslatableTable apply(List<Object> arguments) {
-      return new DrillTranslatableTable(schema.getDrillTable(new TableInstance(sig, arguments)));
+      DrillTable drillTable = schema.getDrillTable(new TableInstance(sig, arguments));
+      if (drillTable == null) {
+        throw UserException
+            .validationError()
+            .message("Unable to find table [%s] in schema [%s]", sig.name, schema.getFullSchemaName())
+            .build(logger);
+      }
+      return new DrillTranslatableTable(drillTable);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
index 5fd5039..a6af1ac 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -26,7 +26,6 @@ import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractSubScan;
 import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.apache.drill.exec.store.dfs.NamedFormatPluginConfig;
 import org.apache.drill.exec.store.schedule.CompleteFileWork.FileWorkImpl;
 
 import com.fasterxml.jackson.annotation.JacksonInject;
@@ -94,13 +93,7 @@ public class EasySubScan extends AbstractSubScan{
 
   @JsonProperty("format")
   public FormatPluginConfig getFormatConfig(){
-    if (formatPlugin.getName() != null) {
-      NamedFormatPluginConfig namedConfig = new NamedFormatPluginConfig();
-      namedConfig.name = formatPlugin.getName();
-      return namedConfig;
-    } else {
-      return formatPlugin.getConfig();
-    }
+    return formatPlugin.getConfig();
   }
 
   @JsonProperty("columns")

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
----------------------------------------------------------------------
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 a6dc7d9..cee9a89 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
@@ -1,21 +1,26 @@
-
-/**
- * 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
+/*
+ * 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.
+ * 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 java.io.IOException;
 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;
@@ -73,11 +78,11 @@ 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)
+  @JsonTypeName(PLUGIN_EXTENSION) @JsonInclude(JsonInclude.Include.NON_DEFAULT)
   public static class HttpdLogFormatConfig implements FormatPluginConfig {
 
-    private String logFormat;
-    private String timestampFormat;
+    public String logFormat;
+    public String timestampFormat;
 
     /**
      * @return the logFormat
@@ -92,6 +97,30 @@ public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.
     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;
+    }
   }
 
   /**
@@ -119,7 +148,7 @@ public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.
      * The query fields passed in are formatted in a way that Drill requires. Those must be cleaned up to work with the
      * parser.
      *
-     * @return Map<DrillFieldNames, ParserFieldNames>
+     * @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();

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
index 972332c..75b18df 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
@@ -164,9 +164,6 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
   ) throws IOException, ExecutionSetupException {
     super(ImpersonationUtil.resolveUserName(userName));
     this.columns = columns;
-    if (formatConfig == null) {
-      formatConfig = new ParquetFormatConfig();
-    }
     Preconditions.checkNotNull(storageConfig);
     Preconditions.checkNotNull(formatConfig);
     this.formatPlugin = (ParquetFormatPlugin) engineRegistry.getFormatPlugin(storageConfig, formatConfig);
@@ -345,6 +342,7 @@ public class ParquetGroupScan extends AbstractFileGroupScan {
     return true;
   }
 
+  @JsonIgnore
   @Override
   public Collection<String> getFiles() {
     return fileSet;

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
index 5da6aef..f1fb1e9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -65,9 +65,12 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
       @JsonProperty("selectionRoot") String selectionRoot, //
       @JsonProperty("filter") LogicalExpression filter
   ) throws ExecutionSetupException {
-    this(userName, (ParquetFormatPlugin) registry.getFormatPlugin(Preconditions.checkNotNull(storageConfig),
-            formatConfig == null ? new ParquetFormatConfig() : formatConfig),
-        rowGroupReadEntries, columns, selectionRoot, filter);
+    this(userName,
+        (ParquetFormatPlugin) registry.getFormatPlugin(Preconditions.checkNotNull(storageConfig), Preconditions.checkNotNull(formatConfig)),
+        rowGroupReadEntries,
+        columns,
+        selectionRoot,
+        filter);
   }
 
   public ParquetRowGroupScan( //
@@ -79,7 +82,7 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
       LogicalExpression filter
   ) {
     super(userName);
-    this.formatPlugin = Preconditions.checkNotNull(formatPlugin);
+    this.formatPlugin = Preconditions.checkNotNull(formatPlugin, "Could not find format config for the given configuration");
     this.formatConfig = formatPlugin.getConfig();
     this.rowGroupReadEntries = rowGroupReadEntries;
     this.columns = columns == null ? GroupScan.ALL_COLUMNS : columns;
@@ -97,6 +100,14 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
     return formatPlugin.getStorageConfig();
   }
 
+  /**
+   * @return Parquet plugin format config
+   */
+  @JsonProperty("format")
+  public ParquetFormatConfig getFormatConfig() {
+    return formatConfig;
+  }
+
   public String getSelectionRoot() {
     return selectionRoot;
   }
@@ -140,11 +151,4 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
     return CoreOperatorType.PARQUET_ROW_GROUP_SCAN_VALUE;
   }
 
-  /**
-   * @return Parquet plugin format config
-   */
-  public ParquetFormatConfig getFormatConfig() {
-    return formatConfig;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatConfig.java
index 4e44839..89b56ad 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatConfig.java
@@ -21,4 +21,14 @@ import org.apache.drill.common.logical.FormatPluginConfig;
 
 @JsonTypeName("pcap")
 public class PcapFormatConfig implements FormatPluginConfig {
+
+  @Override
+  public int hashCode() {
+    return 0;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    return obj instanceof PcapFormatConfig;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java
index 2d311cd..65ff238 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pcap/PcapFormatPlugin.java
@@ -34,7 +34,6 @@ import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.apache.drill.exec.store.dfs.FormatMatcher;
 import org.apache.drill.exec.store.dfs.FormatSelection;
 import org.apache.drill.exec.store.dfs.MagicString;
-import org.apache.drill.exec.store.dfs.NamedFormatPluginConfig;
 import org.apache.drill.exec.store.dfs.easy.EasyFormatPlugin;
 import org.apache.drill.exec.store.dfs.easy.EasyWriter;
 import org.apache.drill.exec.store.dfs.easy.FileWork;
@@ -99,13 +98,7 @@ public class PcapFormatPlugin extends EasyFormatPlugin<PcapFormatConfig> {
                                  FileSelection selection, FileSystemPlugin fsPlugin,
                                  String storageEngineName, String userName) throws IOException {
       if (isFileReadable(fs, selection.getFirstPath(fs))) {
-        if (plugin.getName() != null) {
-          NamedFormatPluginConfig namedConfig = new NamedFormatPluginConfig();
-          namedConfig.name = plugin.getName();
-          return new PcapDrillTable(storageEngineName, fsPlugin, userName, new FormatSelection(namedConfig, selection));
-        } else {
-          return new PcapDrillTable(storageEngineName, fsPlugin, userName, new FormatSelection(plugin.getConfig(), selection));
-        }
+        return new PcapDrillTable(storageEngineName, fsPlugin, userName, new FormatSelection(plugin.getConfig(), selection));
       }
       return null;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
index 091d567..22b734b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
@@ -281,6 +281,19 @@ public class PlanTestBase extends BaseTestQuery {
     }
   }
 
+
+  /**
+   * Creates physical plan for the given query and then executes this plan.
+   * This method is useful for testing serialization / deserialization issues.
+   *
+   * @param query query string
+   */
+  public static void testPhysicalPlanExecutionBasedOnQuery(String query) throws Exception {
+    query = "EXPLAIN PLAN for " + QueryTestUtil.normalizeQuery(query);
+    String plan = getPlanInString(query, JSON_FORMAT);
+    testPhysical(plan);
+  }
+
   /*
    * This will get the plan (either logical or physical) in Optiq RelNode
    * format, based on SqlExplainLevel and Depth.

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/test/java/org/apache/drill/TestSelectWithOption.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestSelectWithOption.java b/exec/java-exec/src/test/java/org/apache/drill/TestSelectWithOption.java
index 5a49ad2..a6dff74 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestSelectWithOption.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestSelectWithOption.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -19,12 +19,15 @@ package org.apache.drill;
 
 import static java.lang.String.format;
 import static org.apache.drill.test.TestBuilder.listOf;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertThat;
 
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
 
 import org.apache.drill.categories.SqlTest;
+import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.exec.store.dfs.WorkspaceSchemaFactory;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.test.TestBuilder;
@@ -276,4 +279,16 @@ public class TestSelectWithOption extends BaseTestQuery {
       test("use sys");
     }
   }
+
+  @Test(expected = UserRemoteException.class)
+  public void testAbsentTable() throws Exception {
+    String schema = "cp.default";
+    String tableName = "absent_table";
+    try {
+      test("select * from table(`%s`.`%s`(type=>'parquet'))", schema, tableName);
+    } catch (UserRemoteException e) {
+      assertThat(e.getMessage(), containsString(String.format("Unable to find table [%s] in schema [%s]", tableName, schema)));
+      throw e;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributedConcurrent.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributedConcurrent.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributedConcurrent.java
index 917890b..f096d55 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributedConcurrent.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributedConcurrent.java
@@ -177,7 +177,7 @@ public class TestTpchDistributedConcurrent extends BaseTestQuery {
     }
   }
 
-  @Test
+  //@Test
   public void testConcurrentQueries() throws Exception {
     QueryTestUtil.testRunAndPrint(client, UserBitShared.QueryType.SQL, alterSession);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/7506cfbb/exec/java-exec/src/test/java/org/apache/drill/exec/store/FormatPluginSerDeTest.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..ca81eaa
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/FormatPluginSerDeTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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;
+
+import org.apache.drill.PlanTestBase;
+import org.apache.drill.exec.store.avro.AvroTestUtil;
+import org.junit.Test;
+
+import java.nio.file.Paths;
+
+public class FormatPluginSerDeTest extends PlanTestBase {
+
+  @Test
+  public void testParquet() throws Exception {
+    test("alter session set `planner.slice_target` = 1");
+    testPhysicalPlanSubmission(
+        String.format("select * from table(cp.`%s`(type=>'parquet'))", "parquet/alltypes_required.parquet"),
+        String.format("select * from table(cp.`%s`(type=>'parquet', autoCorrectCorruptDates=>false))", "parquet/alltypes_required.parquet"),
+        String.format("select * from table(cp.`%s`(type=>'parquet', autoCorrectCorruptDates=>true))", "parquet/alltypes_required.parquet")
+    );
+  }
+
+  @Test
+  public void testAvro() throws Exception {
+    AvroTestUtil.AvroTestRecordWriter testSetup = AvroTestUtil.generateSimplePrimitiveSchema_NoNullValues(5);
+    String file = testSetup.getFileName();
+    testPhysicalPlanSubmission(
+        String.format("select * from dfs.`%s`", file),
+        String.format("select * from table(dfs.`%s`(type=>'avro'))", file)
+    );
+  }
+
+  @Test
+  public void testSequenceFile() throws Exception {
+    String path = "sequencefiles/simple.seq";
+    dirTestWatcher.copyResourceToRoot(Paths.get(path));
+    testPhysicalPlanSubmission(
+        String.format("select * from dfs.`%s`", path),
+        String.format("select * from table(dfs.`%s`(type=>'sequencefile'))", path)
+    );
+  }
+
+  @Test
+  public void testPcap() throws Exception {
+    String path = "store/pcap/tcp-1.pcap";
+    dirTestWatcher.copyResourceToRoot(Paths.get(path));
+    testPhysicalPlanSubmission(
+        String.format("select * from dfs.`%s`", path),
+        String.format("select * from table(dfs.`%s`(type=>'pcap'))", path)
+    );
+  }
+
+  @Test
+  public void testHttpd() throws Exception {
+    String path = "store/httpd/dfs-bootstrap.httpd";
+    dirTestWatcher.copyResourceToRoot(Paths.get(path));
+    String logFormat = "%h %t \"%r\" %>s %b \"%{Referer}i\"";
+    String timeStampFormat = "dd/MMM/yyyy:HH:mm:ss ZZ";
+    testPhysicalPlanSubmission(
+        String.format("select * from dfs.`%s`", path),
+        String.format("select * from table(dfs.`%s`(type=>'httpd', logFormat=>'%s'))", path, logFormat),
+        String.format("select * from table(dfs.`%s`(type=>'httpd', logFormat=>'%s', timestampFormat=>'%s'))", path, logFormat, timeStampFormat)
+    );
+  }
+
+  @Test
+  public void testJson() throws Exception {
+    testPhysicalPlanSubmission(
+        "select * from cp.`donuts.json`",
+        "select * from table(cp.`donuts.json`(type=>'json'))"
+    );
+  }
+
+  @Test
+  public void testText() throws Exception {
+    String path = "store/text/data/regions.csv";
+    dirTestWatcher.copyResourceToRoot(Paths.get(path));
+    testPhysicalPlanSubmission(
+        String.format("select * from table(dfs.`%s`(type => 'text'))", path),
+        String.format("select * from table(dfs.`%s`(type => 'text', extractHeader => false, fieldDelimiter => 'A'))", path)
+    );
+  }
+
+  @Test
+  public void testNamed() throws Exception {
+    String path = "store/text/WithQuote.tbl";
+    dirTestWatcher.copyResourceToRoot(Paths.get(path));
+    String query = String.format("select * from table(dfs.`%s`(type=>'named', name=>'psv'))", path);
+    testPhysicalPlanSubmission(query);
+  }
+
+  private void testPhysicalPlanSubmission(String...queries) throws Exception {
+    for (String query : queries) {
+      PlanTestBase.testPhysicalPlanExecutionBasedOnQuery(query);
+    }
+  }
+
+}
+