You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pr...@apache.org on 2015/11/13 21:35:01 UTC

hive git commit: HIVE-11120: Generic interface for file format validation (Prasanth Jayachandran reviewed by Xuefu Zhang)

Repository: hive
Updated Branches:
  refs/heads/master f18849b11 -> 82fd1bdbe


HIVE-11120: Generic interface for file format validation (Prasanth Jayachandran reviewed by Xuefu Zhang)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/82fd1bdb
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/82fd1bdb
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/82fd1bdb

Branch: refs/heads/master
Commit: 82fd1bdbe70acbbdf9c9fc5b227f111005f9d87a
Parents: f18849b
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Fri Nov 13 14:34:48 2015 -0600
Committer: Prasanth Jayachandran <j....@gmail.com>
Committed: Fri Nov 13 14:34:48 2015 -0600

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |  63 ++++---
 .../hadoop/hive/ql/io/HiveFileFormatUtils.java  | 166 +++++++++----------
 .../ql/io/SequenceFileInputFormatChecker.java   |   3 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   4 +
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |  49 +++---
 .../clientnegative/archive_corrupt.q.out        |  14 +-
 .../clientnegative/load_orc_negative1.q.out     |   2 +-
 .../clientnegative/load_orc_negative2.q.out     |   2 +-
 .../clientnegative/load_orc_negative3.q.out     |   2 +-
 .../clientnegative/load_orc_negative_part.q.out |   2 +-
 .../clientnegative/load_wrong_fileformat.q.out  |   7 +-
 .../load_wrong_fileformat_rc_seq.q.out          |   7 +-
 .../load_wrong_fileformat_txt_seq.q.out         |   7 +-
 13 files changed, 153 insertions(+), 175 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/82fd1bdb/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
index 5ac523a..e9cd450 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
@@ -18,8 +18,16 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.io.Serializable;
+import java.security.AccessControlException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -62,17 +70,8 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.util.StringUtils;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.security.AccessControlException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * MoveTask implementation.
@@ -295,13 +294,39 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
             throw new HiveException(
                 "addFiles: filesystem error in check phase", e);
           }
+
+          // handle file format check for table level
           if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVECHECKFILEFORMAT)) {
-            // Check if the file format of the file matches that of the table.
-            boolean flag = HiveFileFormatUtils.checkInputFormat(
-                srcFs, conf, tbd.getTable().getInputFileFormatClass(), files);
-            if (!flag) {
-              throw new HiveException(
-                  "Wrong file format. Please check the file's format.");
+            boolean flag = true;
+            // work.checkFileFormat is set to true only for Load Task, so assumption here is
+            // dynamic partition context is null
+            if (tbd.getDPCtx() == null) {
+              if (tbd.getPartitionSpec() == null || tbd.getPartitionSpec().isEmpty()) {
+                // Check if the file format of the file matches that of the table.
+                flag = HiveFileFormatUtils.checkInputFormat(
+                    srcFs, conf, tbd.getTable().getInputFileFormatClass(), files);
+              } else {
+                // Check if the file format of the file matches that of the partition
+                Partition oldPart = db.getPartition(table, tbd.getPartitionSpec(), false);
+                if (oldPart == null) {
+                  // this means we have just created a table and are specifying partition in the
+                  // load statement (without pre-creating the partition), in which case lets use
+                  // table input format class. inheritTableSpecs defaults to true so when a new
+                  // partition is created later it will automatically inherit input format
+                  // from table object
+                  flag = HiveFileFormatUtils.checkInputFormat(
+                      srcFs, conf, tbd.getTable().getInputFileFormatClass(), files);
+                } else {
+                  flag = HiveFileFormatUtils.checkInputFormat(
+                      srcFs, conf, oldPart.getInputFormatClass(), files);
+                }
+              }
+              if (!flag) {
+                throw new HiveException(
+                    "Wrong file format. Please check the file's format.");
+              }
+            } else {
+              LOG.warn("Skipping file format check as dpCtx is not null");
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/82fd1bdb/ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java
index 0328a23..bc13862 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveFileFormatUtils.java
@@ -31,10 +31,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -44,7 +41,7 @@ import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter;
 import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -68,6 +65,12 @@ import org.apache.hadoop.mapred.TaskAttemptContext;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.util.Shell;
 import org.apache.hive.common.util.ReflectionUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.collect.ImmutableMap;
 
 /**
  * An util class for various Hive file format tasks.
@@ -79,30 +82,68 @@ import org.apache.hive.common.util.ReflectionUtil;
 public final class HiveFileFormatUtils {
   private static final Logger LOG = LoggerFactory.getLogger(HiveFileFormatUtils.class);
 
-  static {
-    outputFormatSubstituteMap =
-        new ConcurrentHashMap<Class<?>, Class<? extends OutputFormat>>();
-    HiveFileFormatUtils.registerOutputFormatSubstitute(
-        IgnoreKeyTextOutputFormat.class, HiveIgnoreKeyTextOutputFormat.class);
-    HiveFileFormatUtils.registerOutputFormatSubstitute(
-        SequenceFileOutputFormat.class, HiveSequenceFileOutputFormat.class);
-  }
+  public static class FileChecker {
+    // we don't have many file formats that implement InputFormatChecker. We won't be holding
+    // multiple instances of such classes
+    private static int MAX_CACHE_SIZE = 16;
 
-  @SuppressWarnings("unchecked")
-  private static Map<Class<?>, Class<? extends OutputFormat>>
-    outputFormatSubstituteMap;
+    // immutable maps
+    Map<Class<? extends InputFormat>, Class<? extends InputFormatChecker>> inputFormatCheckerMap;
+    Map<Class<?>, Class<? extends OutputFormat>> outputFormatSubstituteMap;
 
-  /**
-   * register a substitute.
-   *
-   * @param origin
-   *          the class that need to be substituted
-   * @param substitute
-   */
-  @SuppressWarnings("unchecked")
-  public static void registerOutputFormatSubstitute(Class<?> origin,
-      Class<? extends HiveOutputFormat> substitute) {
-    outputFormatSubstituteMap.put(origin, substitute);
+    // mutable thread-safe map to store instances
+    Cache<Class<? extends InputFormatChecker>, InputFormatChecker> inputFormatCheckerInstanceCache;
+
+    // classloader invokes this static block when its first loaded (lazy initialization).
+    // Class loading is thread safe.
+    private static class Factory {
+      static final FileChecker INSTANCE = new FileChecker();
+    }
+
+    public static FileChecker getInstance() {
+      return Factory.INSTANCE;
+    }
+
+    private FileChecker() {
+      // read-only maps (initialized once)
+      inputFormatCheckerMap = ImmutableMap
+          .<Class<? extends InputFormat>, Class<? extends InputFormatChecker>>builder()
+          .put(SequenceFileInputFormat.class, SequenceFileInputFormatChecker.class)
+          .put(RCFileInputFormat.class, RCFileInputFormat.class)
+          .put(OrcInputFormat.class, OrcInputFormat.class)
+          .build();
+      outputFormatSubstituteMap = ImmutableMap
+          .<Class<?>, Class<? extends OutputFormat>>builder()
+          .put(IgnoreKeyTextOutputFormat.class, HiveIgnoreKeyTextOutputFormat.class)
+          .put(SequenceFileOutputFormat.class, HiveSequenceFileOutputFormat.class)
+          .build();
+
+      // updatable map that holds instances of the class
+      inputFormatCheckerInstanceCache = CacheBuilder.newBuilder().maximumSize(MAX_CACHE_SIZE)
+          .build();
+    }
+
+    public Set<Class<? extends InputFormat>> registeredClasses() {
+      return inputFormatCheckerMap.keySet();
+    }
+
+    public Class<? extends OutputFormat> getOutputFormatSubstiture(Class<?> origin) {
+      return outputFormatSubstituteMap.get(origin);
+    }
+
+    public Class<? extends InputFormatChecker> getInputFormatCheckerClass(Class<?> inputFormat) {
+      return inputFormatCheckerMap.get(inputFormat);
+    }
+
+    public void putInputFormatCheckerInstance(
+        Class<? extends InputFormatChecker> checkerCls, InputFormatChecker instanceCls) {
+      inputFormatCheckerInstanceCache.put(checkerCls, instanceCls);
+    }
+
+    public InputFormatChecker getInputFormatCheckerInstance(
+        Class<? extends InputFormatChecker> checkerCls) {
+      return inputFormatCheckerInstanceCache.getIfPresent(checkerCls);
+    }
   }
 
   /**
@@ -114,7 +155,8 @@ public final class HiveFileFormatUtils {
     if (origin == null || HiveOutputFormat.class.isAssignableFrom(origin)) {
       return (Class<? extends OutputFormat>) origin;  // hive native
     }
-    Class<? extends OutputFormat> substitute = outputFormatSubstituteMap.get(origin);
+    Class<? extends OutputFormat> substitute = FileChecker.getInstance()
+        .getOutputFormatSubstiture(origin);
     if (substitute != null) {
       return substitute;  // substituted
     }
@@ -122,66 +164,6 @@ public final class HiveFileFormatUtils {
   }
 
   /**
-   * get the final output path of a given FileOutputFormat.
-   *
-   * @param parent
-   *          parent dir of the expected final output path
-   * @param jc
-   *          job configuration
-   * @deprecated
-   */
-  @Deprecated
-  public static Path getOutputFormatFinalPath(Path parent, String taskId, JobConf jc,
-      HiveOutputFormat<?, ?> hiveOutputFormat, boolean isCompressed,
-      Path defaultFinalPath) throws IOException {
-    if (hiveOutputFormat instanceof HiveIgnoreKeyTextOutputFormat) {
-      return new Path(parent, taskId
-          + Utilities.getFileExtension(jc, isCompressed));
-    }
-    return defaultFinalPath;
-  }
-
-  static {
-    inputFormatCheckerMap =
-        new HashMap<Class<? extends InputFormat>, Class<? extends InputFormatChecker>>();
-    HiveFileFormatUtils.registerInputFormatChecker(
-        SequenceFileInputFormat.class, SequenceFileInputFormatChecker.class);
-    HiveFileFormatUtils.registerInputFormatChecker(RCFileInputFormat.class,
-        RCFileInputFormat.class);
-    inputFormatCheckerInstanceCache =
-        new HashMap<Class<? extends InputFormatChecker>, InputFormatChecker>();
-  }
-
-  @SuppressWarnings("unchecked")
-  private static Map<Class<? extends InputFormat>, Class<? extends InputFormatChecker>> inputFormatCheckerMap;
-
-  private static Map<Class<? extends InputFormatChecker>, InputFormatChecker> inputFormatCheckerInstanceCache;
-
-  /**
-   * register an InputFormatChecker for a given InputFormat.
-   *
-   * @param format
-   *          the class that need to be substituted
-   * @param checker
-   */
-  @SuppressWarnings("unchecked")
-  public static synchronized void registerInputFormatChecker(
-      Class<? extends InputFormat> format,
-      Class<? extends InputFormatChecker> checker) {
-    inputFormatCheckerMap.put(format, checker);
-  }
-
-  /**
-   * get an InputFormatChecker for a file format.
-   */
-  public static synchronized Class<? extends InputFormatChecker> getInputFormatChecker(
-      Class<?> inputFormat) {
-    Class<? extends InputFormatChecker> result = inputFormatCheckerMap
-        .get(inputFormat);
-    return result;
-  }
-
-  /**
    * checks if files are in same format as the given input format.
    */
   @SuppressWarnings("unchecked")
@@ -189,7 +171,8 @@ public final class HiveFileFormatUtils {
       Class<? extends InputFormat> inputFormatCls, List<FileStatus> files)
       throws HiveException {
     if (files.isEmpty()) return false;
-    Class<? extends InputFormatChecker> checkerCls = getInputFormatChecker(inputFormatCls);
+    Class<? extends InputFormatChecker> checkerCls = FileChecker.getInstance()
+        .getInputFormatCheckerClass(inputFormatCls);
     if (checkerCls == null
         && inputFormatCls.isAssignableFrom(TextInputFormat.class)) {
       // we get a text input format here, we can not determine a file is text
@@ -200,11 +183,12 @@ public final class HiveFileFormatUtils {
     }
 
     if (checkerCls != null) {
-      InputFormatChecker checkerInstance = inputFormatCheckerInstanceCache.get(checkerCls);
+      InputFormatChecker checkerInstance = FileChecker.getInstance()
+          .getInputFormatCheckerInstance(checkerCls);
       try {
         if (checkerInstance == null) {
           checkerInstance = checkerCls.newInstance();
-          inputFormatCheckerInstanceCache.put(checkerCls, checkerInstance);
+          FileChecker.getInstance().putInputFormatCheckerInstance(checkerCls, checkerInstance);
         }
         return checkerInstance.validateInput(fs, conf, files);
       } catch (Exception e) {
@@ -228,7 +212,7 @@ public final class HiveFileFormatUtils {
       }
     }
     if (files2.isEmpty()) return true;
-    Set<Class<? extends InputFormat>> inputFormatter = inputFormatCheckerMap.keySet();
+    Set<Class<? extends InputFormat>> inputFormatter = FileChecker.getInstance().registeredClasses();
     for (Class<? extends InputFormat> reg : inputFormatter) {
       boolean result = checkInputFormat(fs, conf, reg, files2);
       if (result) {

http://git-wip-us.apache.org/repos/asf/hive/blob/82fd1bdb/ql/src/java/org/apache/hadoop/hive/ql/io/SequenceFileInputFormatChecker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/SequenceFileInputFormatChecker.java b/ql/src/java/org/apache/hadoop/hive/ql/io/SequenceFileInputFormatChecker.java
index 6cb46c9..f59b838 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/SequenceFileInputFormatChecker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/SequenceFileInputFormatChecker.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.ql.io;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.fs.FileStatus;
@@ -49,7 +48,7 @@ public class SequenceFileInputFormatChecker implements InputFormatChecker {
         reader = null;
       } catch (IOException e) {
         return false;
-      }finally{
+      } finally{
         IOUtils.closeStream(reader);
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/82fd1bdb/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
index af40137..bee0831 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
@@ -421,6 +421,10 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
       return false;
     }
     for (FileStatus file : files) {
+      // 0 length files cannot be ORC files
+      if (file.getLen() == 0) {
+        return false;
+      }
       try {
         OrcFile.createReader(file.getPath(),
             OrcFile.readerOptions(conf).filesystem(fs));

http://git-wip-us.apache.org/repos/asf/hive/blob/82fd1bdb/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
index c488029..b90616f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
@@ -41,9 +41,7 @@ import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
-import org.apache.hadoop.hive.ql.io.FileFormatException;
-import org.apache.hadoop.hive.ql.io.orc.OrcFile;
-import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
+import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
@@ -52,6 +50,8 @@ import org.apache.hadoop.hive.ql.plan.MoveWork;
 import org.apache.hadoop.hive.ql.plan.StatsWork;
 import org.apache.hadoop.mapred.InputFormat;
 
+import com.google.common.collect.Lists;
+
 /**
  * LoadSemanticAnalyzer.
  *
@@ -128,7 +128,7 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
     return new URI(fromScheme, fromAuthority, path, null, null);
   }
 
-  private FileStatus[] applyConstraintsAndGetFiles(URI fromURI, URI toURI, Tree ast,
+  private List<FileStatus> applyConstraintsAndGetFiles(URI fromURI, Tree ast,
       boolean isLocal) throws SemanticException {
 
     FileStatus[] srcs = null;
@@ -159,7 +159,7 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new SemanticException(ErrorMsg.INVALID_PATH.getMsg(ast), e);
     }
 
-    return srcs;
+    return Lists.newArrayList(srcs);
   }
 
   @Override
@@ -209,9 +209,6 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new SemanticException(ErrorMsg.LOAD_INTO_STORED_AS_DIR.getMsg());
     }
 
-    URI toURI = ((ts.partHandle != null) ? ts.partHandle.getDataLocation()
-        : ts.tableHandle.getDataLocation()).toUri();
-
     List<FieldSchema> parts = ts.tableHandle.getPartitionKeys();
     if ((parts != null && parts.size() > 0)
         && (ts.partSpec == null || ts.partSpec.size() == 0)) {
@@ -219,11 +216,12 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     // make sure the arguments make sense
-    FileStatus[] files = applyConstraintsAndGetFiles(fromURI, toURI, fromTree, isLocal);
+    List<FileStatus> files = applyConstraintsAndGetFiles(fromURI, fromTree, isLocal);
 
     // for managed tables, make sure the file formats match
-    if (TableType.MANAGED_TABLE.equals(ts.tableHandle.getTableType())) {
-      ensureFileFormatsMatch(ts, files);
+    if (TableType.MANAGED_TABLE.equals(ts.tableHandle.getTableType())
+        && conf.getBoolVar(HiveConf.ConfVars.HIVECHECKFILEFORMAT)) {
+      ensureFileFormatsMatch(ts, files, fromURI);
     }
     inputs.add(toReadEntity(new Path(fromURI)));
     Task<? extends Serializable> rTask = null;
@@ -317,7 +315,9 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
   }
 
-  private void ensureFileFormatsMatch(TableSpec ts, FileStatus[] fileStatuses) throws SemanticException {
+  private void ensureFileFormatsMatch(TableSpec ts, List<FileStatus> fileStatuses,
+      final URI fromURI)
+      throws SemanticException {
     final Class<? extends InputFormat> destInputFormat;
     try {
       if (ts.getPartSpec() == null || ts.getPartSpec().isEmpty()) {
@@ -329,23 +329,16 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new SemanticException(e);
     }
 
-    // Other file formats should do similar check to make sure file formats match
-    // when doing LOAD DATA .. INTO TABLE
-    if (OrcInputFormat.class.equals(destInputFormat)) {
-      for (FileStatus fileStatus : fileStatuses) {
-        try {
-          Path filePath = fileStatus.getPath();
-          FileSystem fs = FileSystem.get(filePath.toUri(), conf);
-          // just creating orc reader is going to do sanity checks to make sure its valid ORC file
-          OrcFile.createReader(fs, filePath);
-        } catch (FileFormatException e) {
-          throw new SemanticException(ErrorMsg.INVALID_FILE_FORMAT_IN_LOAD.getMsg("Destination" +
-              " table is stored as ORC but the file being loaded is not a valid ORC file."));
-        } catch (IOException e) {
-          throw new SemanticException("Unable to load data to destination table." +
-              " Error: " + e.getMessage());
-        }
+    try {
+      FileSystem fs = FileSystem.get(fromURI, conf);
+      boolean validFormat = HiveFileFormatUtils.checkInputFormat(fs, conf, destInputFormat,
+          fileStatuses);
+      if (!validFormat) {
+        throw new SemanticException(ErrorMsg.INVALID_FILE_FORMAT_IN_LOAD.getMsg());
       }
+    } catch (Exception e) {
+      throw new SemanticException("Unable to load data to destination table." +
+          " Error: " + e.getMessage());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/82fd1bdb/ql/src/test/results/clientnegative/archive_corrupt.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/archive_corrupt.q.out b/ql/src/test/results/clientnegative/archive_corrupt.q.out
index 56e8ec4..892fbac 100644
--- a/ql/src/test/results/clientnegative/archive_corrupt.q.out
+++ b/ql/src/test/results/clientnegative/archive_corrupt.q.out
@@ -16,16 +16,4 @@ POSTHOOK: query: create table tstsrcpart like srcpart
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@tstsrcpart
-PREHOOK: query: -- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20)
--- The version of GzipCodec that is provided in Hadoop 0.20 silently ignores
--- file format errors. However, versions of Hadoop that include
--- HADOOP-6835 (e.g. 0.23 and 1.x) cause a Wrong File Format exception
--- to be thrown during the LOAD step. This former behavior is tested
--- in clientpositive/archive_corrupt.q
-
-load data local inpath '../../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', hr='11')
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@tstsrcpart
-Failed with exception Wrong file format. Please check the file's format.
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.MoveTask
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/82fd1bdb/ql/src/test/results/clientnegative/load_orc_negative1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_orc_negative1.q.out b/ql/src/test/results/clientnegative/load_orc_negative1.q.out
index ca15a30..d103546 100644
--- a/ql/src/test/results/clientnegative/load_orc_negative1.q.out
+++ b/ql/src/test/results/clientnegative/load_orc_negative1.q.out
@@ -6,4 +6,4 @@ POSTHOOK: query: create table orc_test (userid bigint, string1 string, subtype d
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@orc_test
-FAILED: SemanticException [Error 30019]: The file that you are trying to load does not match the file format of the destination table. Destination table is stored as ORC but the file being loaded is not a valid ORC file.
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/82fd1bdb/ql/src/test/results/clientnegative/load_orc_negative2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_orc_negative2.q.out b/ql/src/test/results/clientnegative/load_orc_negative2.q.out
index 77fb50e..9b0cb69 100644
--- a/ql/src/test/results/clientnegative/load_orc_negative2.q.out
+++ b/ql/src/test/results/clientnegative/load_orc_negative2.q.out
@@ -22,4 +22,4 @@ POSTHOOK: query: create table orc_test (userid bigint, string1 string, subtype d
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@orc_test
-FAILED: SemanticException [Error 30019]: The file that you are trying to load does not match the file format of the destination table. Destination table is stored as ORC but the file being loaded is not a valid ORC file.
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/82fd1bdb/ql/src/test/results/clientnegative/load_orc_negative3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_orc_negative3.q.out b/ql/src/test/results/clientnegative/load_orc_negative3.q.out
index 77fb50e..9b0cb69 100644
--- a/ql/src/test/results/clientnegative/load_orc_negative3.q.out
+++ b/ql/src/test/results/clientnegative/load_orc_negative3.q.out
@@ -22,4 +22,4 @@ POSTHOOK: query: create table orc_test (userid bigint, string1 string, subtype d
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@orc_test
-FAILED: SemanticException [Error 30019]: The file that you are trying to load does not match the file format of the destination table. Destination table is stored as ORC but the file being loaded is not a valid ORC file.
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/82fd1bdb/ql/src/test/results/clientnegative/load_orc_negative_part.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_orc_negative_part.q.out b/ql/src/test/results/clientnegative/load_orc_negative_part.q.out
index 32dd627..2e8068d 100644
--- a/ql/src/test/results/clientnegative/load_orc_negative_part.q.out
+++ b/ql/src/test/results/clientnegative/load_orc_negative_part.q.out
@@ -49,4 +49,4 @@ POSTHOOK: query: alter table orc_test add partition(ds='11')
 POSTHOOK: type: ALTERTABLE_ADDPARTS
 POSTHOOK: Output: default@orc_test
 POSTHOOK: Output: default@orc_test@ds=11
-FAILED: SemanticException [Error 30019]: The file that you are trying to load does not match the file format of the destination table. Destination table is stored as ORC but the file being loaded is not a valid ORC file.
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/82fd1bdb/ql/src/test/results/clientnegative/load_wrong_fileformat.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_wrong_fileformat.q.out b/ql/src/test/results/clientnegative/load_wrong_fileformat.q.out
index 732eb22..8ec0058 100644
--- a/ql/src/test/results/clientnegative/load_wrong_fileformat.q.out
+++ b/ql/src/test/results/clientnegative/load_wrong_fileformat.q.out
@@ -14,9 +14,4 @@ CREATE TABLE load_wrong_fileformat_T1(name STRING) STORED AS SEQUENCEFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@load_wrong_fileformat_T1
-PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE load_wrong_fileformat_T1
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@load_wrong_fileformat_t1
-Failed with exception Wrong file format. Please check the file's format.
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.MoveTask
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/82fd1bdb/ql/src/test/results/clientnegative/load_wrong_fileformat_rc_seq.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_wrong_fileformat_rc_seq.q.out b/ql/src/test/results/clientnegative/load_wrong_fileformat_rc_seq.q.out
index b68b8e6..916eca4 100644
--- a/ql/src/test/results/clientnegative/load_wrong_fileformat_rc_seq.q.out
+++ b/ql/src/test/results/clientnegative/load_wrong_fileformat_rc_seq.q.out
@@ -14,9 +14,4 @@ CREATE TABLE T1(name STRING) STORED AS RCFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@T1
-PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T1
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@t1
-Failed with exception Wrong file format. Please check the file's format.
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.MoveTask
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.

http://git-wip-us.apache.org/repos/asf/hive/blob/82fd1bdb/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out b/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out
index 179a654..645ece6 100644
--- a/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out
+++ b/ql/src/test/results/clientnegative/load_wrong_fileformat_txt_seq.q.out
@@ -14,9 +14,4 @@ CREATE TABLE T1(name STRING) STORED AS TEXTFILE
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@T1
-PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T1
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@t1
-Failed with exception Wrong file format. Please check the file's format.
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.MoveTask
+FAILED: SemanticException Unable to load data to destination table. Error: The file that you are trying to load does not match the file format of the destination table.