You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2014/10/21 08:16:40 UTC

svn commit: r1633280 - /hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java

Author: navis
Date: Tue Oct 21 06:16:39 2014
New Revision: 1633280

URL: http://svn.apache.org/r1633280
Log:
HIVE-7951 : InputFormats implementing (Job)Configurable should not be cached (Navis reviewed by Ashutosh Chauhan)

Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java?rev=1633280&r1=1633279&r2=1633280&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java Tue Oct 21 06:16:39 2014
@@ -30,7 +30,7 @@ import java.util.Map;
 import org.apache.commons.lang3.StringEscapeUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -66,6 +66,7 @@ import org.apache.hadoop.io.WritableComp
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobConfigurable;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -219,22 +220,27 @@ public class FetchOperator implements Se
   /**
    * A cache of InputFormat instances.
    */
-  private static Map<Class, InputFormat<WritableComparable, Writable>> inputFormats = new HashMap<Class, InputFormat<WritableComparable, Writable>>();
+  private static final Map<String, InputFormat> inputFormats = new HashMap<String, InputFormat>();
 
   @SuppressWarnings("unchecked")
-  static InputFormat<WritableComparable, Writable> getInputFormatFromCache(Class inputFormatClass,
-      Configuration conf) throws IOException {
-    if (!inputFormats.containsKey(inputFormatClass)) {
+  static InputFormat getInputFormatFromCache(Class<? extends InputFormat> inputFormatClass,
+       JobConf conf) throws IOException {
+    if (Configurable.class.isAssignableFrom(inputFormatClass) ||
+        JobConfigurable.class.isAssignableFrom(inputFormatClass)) {
+      return (InputFormat<WritableComparable, Writable>) ReflectionUtils
+          .newInstance(inputFormatClass, conf);
+    }
+    InputFormat format = inputFormats.get(inputFormatClass.getName());
+    if (format == null) {
       try {
-        InputFormat<WritableComparable, Writable> newInstance = (InputFormat<WritableComparable, Writable>) ReflectionUtils
-            .newInstance(inputFormatClass, conf);
-        inputFormats.put(inputFormatClass, newInstance);
+        format = ReflectionUtils.newInstance(inputFormatClass, conf);
+        inputFormats.put(inputFormatClass.getName(), format);
       } catch (Exception e) {
         throw new IOException("Cannot create an instance of InputFormat class "
             + inputFormatClass.getName() + " as specified in mapredWork!", e);
       }
     }
-    return inputFormats.get(inputFormatClass);
+    return format;
   }
 
   private StructObjectInspector getRowInspectorFromTable(TableDesc table) throws Exception {