You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jv...@apache.org on 2011/07/20 21:54:15 UTC

svn commit: r1148921 - in /hive/trunk: cli/src/java/org/apache/hadoop/hive/cli/ contrib/src/java/org/apache/hadoop/hive/contrib/util/typedbytes/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/io/

Author: jvs
Date: Wed Jul 20 19:54:14 2011
New Revision: 1148921

URL: http://svn.apache.org/viewvc?rev=1148921&view=rev
Log:
HIVE-1884. Potential risk of resource leaks in Hive
(Chinna Rao Lalam via jvs)


Modified:
    hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
    hive/trunk/contrib/src/java/org/apache/hadoop/hive/contrib/util/typedbytes/TypedBytesWritableInput.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileInputFormat.java

Modified: hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
URL: http://svn.apache.org/viewvc/hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java?rev=1148921&r1=1148920&r2=1148921&view=diff
==============================================================================
--- hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java (original)
+++ hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java Wed Jul 20 19:54:14 2011
@@ -61,6 +61,7 @@ import org.apache.hadoop.hive.ql.session
 import org.apache.hadoop.hive.service.HiveClient;
 import org.apache.hadoop.hive.service.HiveServerException;
 import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.thrift.TException;
 
 import sun.misc.Signal;
@@ -390,14 +391,18 @@ public class CliDriver {
 
   public int processFile(String fileName) throws IOException {
     FileReader fileReader = null;
+    BufferedReader bufferReader = null;
+    int rc = 0;
     try {
       fileReader = new FileReader(fileName);
-      return processReader(new BufferedReader(fileReader));
+      bufferReader = new BufferedReader(fileReader);
+      rc = processReader(bufferReader);
+      bufferReader.close();
+      bufferReader = null;
     } finally {
-      if (fileReader != null) {
-        fileReader.close();
-      }
+      IOUtils.closeStream(bufferReader);
     }
+    return rc;
   }
 
   public void processInitFiles(CliSessionState ss) throws IOException {

Modified: hive/trunk/contrib/src/java/org/apache/hadoop/hive/contrib/util/typedbytes/TypedBytesWritableInput.java
URL: http://svn.apache.org/viewvc/hive/trunk/contrib/src/java/org/apache/hadoop/hive/contrib/util/typedbytes/TypedBytesWritableInput.java?rev=1148921&r1=1148920&r2=1148921&view=diff
==============================================================================
--- hive/trunk/contrib/src/java/org/apache/hadoop/hive/contrib/util/typedbytes/TypedBytesWritableInput.java (original)
+++ hive/trunk/contrib/src/java/org/apache/hadoop/hive/contrib/util/typedbytes/TypedBytesWritableInput.java Wed Jul 20 19:54:14 2011
@@ -32,6 +32,7 @@ import org.apache.hadoop.io.ArrayWritabl
 import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.MapWritable;
@@ -380,22 +381,29 @@ public class TypedBytesWritableInput imp
   }
 
   public Writable readWritable(Writable writable) throws IOException {
-    ByteArrayInputStream bais = new ByteArrayInputStream(in.readBytes());
-    DataInputStream dis = new DataInputStream(bais);
-    String className = WritableUtils.readString(dis);
-    if (writable == null) {
-      try {
-        Class<? extends Writable> cls = conf.getClassByName(className)
-            .asSubclass(Writable.class);
-        writable = (Writable) ReflectionUtils.newInstance(cls, conf);
-      } catch (ClassNotFoundException e) {
-        throw new IOException(e);
+    DataInputStream dis = null;
+    try {
+      ByteArrayInputStream bais = new ByteArrayInputStream(in.readBytes());
+      dis = new DataInputStream(bais);
+      String className = WritableUtils.readString(dis);
+      if (writable == null) {
+        try {
+          Class<? extends Writable> cls = conf.getClassByName(className)
+              .asSubclass(Writable.class);
+          writable = (Writable) ReflectionUtils.newInstance(cls, conf);
+        } catch (ClassNotFoundException e) {
+          throw new IOException(e);
+        }
+      } else if (!writable.getClass().getName().equals(className)) {
+        throw new IOException("wrong Writable class given");
       }
-    } else if (!writable.getClass().getName().equals(className)) {
-      throw new IOException("wrong Writable class given");
+      writable.readFields(dis);
+      dis.close();
+      dis = null;
+      return writable;
+    } finally {
+      IOUtils.closeStream(dis);
     }
-    writable.readFields(dis);
-    return writable;
   }
 
   public Writable readWritable() throws IOException {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java?rev=1148921&r1=1148920&r2=1148921&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java Wed Jul 20 19:54:14 2011
@@ -37,10 +37,10 @@ import java.util.Date;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.Map.Entry;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -96,7 +96,6 @@ import org.apache.hadoop.hive.ql.plan.Ad
 import org.apache.hadoop.hive.ql.plan.AlterDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.AlterIndexDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
-import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
 import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc;
 import org.apache.hadoop.hive.ql.plan.CreateDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.CreateIndexDesc;
@@ -129,6 +128,7 @@ import org.apache.hadoop.hive.ql.plan.Sh
 import org.apache.hadoop.hive.ql.plan.ShowTablesDesc;
 import org.apache.hadoop.hive.ql.plan.SwitchDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.UnlockTableDesc;
+import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
 import org.apache.hadoop.hive.ql.security.authorization.Privilege;
 import org.apache.hadoop.hive.serde.Constants;
@@ -141,6 +141,7 @@ import org.apache.hadoop.hive.serde2.dyn
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.ToolRunner;
 
 /**
@@ -435,10 +436,11 @@ public class DDLTask extends Task<DDLWor
   }
 
   private int showGrants(ShowGrantDesc showGrantDesc) throws HiveException {
+    DataOutput outStream = null;
     try {
       Path resFile = new Path(showGrantDesc.getResFile());
       FileSystem fs = resFile.getFileSystem(conf);
-      DataOutput outStream = fs.create(resFile);
+      outStream = fs.create(resFile);
       PrincipalDesc principalDesc = showGrantDesc.getPrincipalDesc();
       PrivilegeObjectDesc hiveObjectDesc = showGrantDesc.getHiveObj();
       String principalName = principalDesc.getName();
@@ -581,6 +583,7 @@ public class DDLTask extends Task<DDLWor
         }
       }
       ((FSDataOutputStream) outStream).close();
+      outStream = null;
     } catch (FileNotFoundException e) {
       LOG.info("show table status: " + stringifyException(e));
       return 1;
@@ -590,6 +593,8 @@ public class DDLTask extends Task<DDLWor
     } catch (Exception e) {
       e.printStackTrace();
       throw new HiveException(e);
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream) outStream);
     }
     return 0;
   }
@@ -732,6 +737,7 @@ public class DDLTask extends Task<DDLWor
 
   private int roleDDL(RoleDDLDesc roleDDLDesc) {
     RoleDDLDesc.RoleOperation operation = roleDDLDesc.getOperation();
+    DataOutput outStream = null;
     try {
       if (operation.equals(RoleDDLDesc.RoleOperation.CREATE_ROLE)) {
         db.createRole(roleDDLDesc.getName(), roleDDLDesc.getRoleOwnerName());
@@ -743,12 +749,13 @@ public class DDLTask extends Task<DDLWor
         if (roles != null && roles.size() > 0) {
           Path resFile = new Path(roleDDLDesc.getResFile());
           FileSystem fs = resFile.getFileSystem(conf);
-          DataOutput outStream = fs.create(resFile);
+          outStream = fs.create(resFile);
           for (Role role : roles) {
             outStream.writeBytes("role name:" + role.getRoleName());
             outStream.write(terminator);
           }
           ((FSDataOutputStream) outStream).close();
+          outStream = null;
         }
       } else {
         throw new HiveException("Unkown role operation "
@@ -762,6 +769,8 @@ public class DDLTask extends Task<DDLWor
     } catch (IOException e) {
       LOG.info("role ddl exception: " + stringifyException(e));
       return 1;
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream) outStream);
     }
 
     return 0;
@@ -1620,10 +1629,11 @@ public class DDLTask extends Task<DDLWor
     }
 
     // write the results in the file
+    DataOutput outStream = null;
     try {
       Path resFile = new Path(showParts.getResFile());
       FileSystem fs = resFile.getFileSystem(conf);
-      DataOutput outStream = fs.create(resFile);
+      outStream = fs.create(resFile);
       Iterator<String> iterParts = parts.iterator();
 
       while (iterParts.hasNext()) {
@@ -1632,6 +1642,7 @@ public class DDLTask extends Task<DDLWor
         outStream.write(terminator);
       }
       ((FSDataOutputStream) outStream).close();
+      outStream = null;
     } catch (FileNotFoundException e) {
       LOG.info("show partitions: " + stringifyException(e));
       throw new HiveException(e.toString());
@@ -1640,6 +1651,8 @@ public class DDLTask extends Task<DDLWor
       throw new HiveException(e.toString());
     } catch (Exception e) {
       throw new HiveException(e.toString());
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream) outStream);
     }
 
     return 0;
@@ -1667,10 +1680,11 @@ public class DDLTask extends Task<DDLWor
     indexes = db.getIndexes(tbl.getDbName(), tbl.getTableName(), (short) -1);
 
     // write the results in the file
+    DataOutput outStream = null;
     try {
       Path resFile = new Path(showIndexes.getResFile());
       FileSystem fs = resFile.getFileSystem(conf);
-      DataOutput outStream = fs.create(resFile);
+      outStream = fs.create(resFile);
 
       if (showIndexes.isFormatted()) {
         // column headers
@@ -1685,6 +1699,7 @@ public class DDLTask extends Task<DDLWor
       }
 
       ((FSDataOutputStream) outStream).close();
+      outStream = null;
 
     } catch (FileNotFoundException e) {
       LOG.info("show indexes: " + stringifyException(e));
@@ -1694,6 +1709,8 @@ public class DDLTask extends Task<DDLWor
       throw new HiveException(e.toString());
     } catch (Exception e) {
       throw new HiveException(e.toString());
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream) outStream);
     }
 
     return 0;
@@ -1720,10 +1737,11 @@ public class DDLTask extends Task<DDLWor
     LOG.info("results : " + databases.size());
 
     // write the results in the file
+    DataOutput outStream = null;
     try {
       Path resFile = new Path(showDatabasesDesc.getResFile());
       FileSystem fs = resFile.getFileSystem(conf);
-      DataOutput outStream = fs.create(resFile);
+      outStream = fs.create(resFile);
 
       for (String database : databases) {
         // create a row per database name
@@ -1731,6 +1749,7 @@ public class DDLTask extends Task<DDLWor
         outStream.write(terminator);
       }
       ((FSDataOutputStream) outStream).close();
+      outStream = null;
     } catch (FileNotFoundException e) {
       LOG.warn("show databases: " + stringifyException(e));
       return 1;
@@ -1739,6 +1758,8 @@ public class DDLTask extends Task<DDLWor
       return 1;
     } catch (Exception e) {
       throw new HiveException(e.toString());
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream) outStream);
     }
     return 0;
   }
@@ -1772,10 +1793,11 @@ public class DDLTask extends Task<DDLWor
     }
 
     // write the results in the file
+    DataOutput outStream = null;
     try {
       Path resFile = new Path(showTbls.getResFile());
       FileSystem fs = resFile.getFileSystem(conf);
-      DataOutput outStream = fs.create(resFile);
+      outStream = fs.create(resFile);
       SortedSet<String> sortedTbls = new TreeSet<String>(tbls);
       Iterator<String> iterTbls = sortedTbls.iterator();
 
@@ -1785,6 +1807,7 @@ public class DDLTask extends Task<DDLWor
         outStream.write(terminator);
       }
       ((FSDataOutputStream) outStream).close();
+      outStream = null;
     } catch (FileNotFoundException e) {
       LOG.warn("show table: " + stringifyException(e));
       return 1;
@@ -1793,6 +1816,8 @@ public class DDLTask extends Task<DDLWor
       return 1;
     } catch (Exception e) {
       throw new HiveException(e.toString());
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream) outStream);
     }
     return 0;
   }
@@ -1818,10 +1843,11 @@ public class DDLTask extends Task<DDLWor
     }
 
     // write the results in the file
+    DataOutput outStream = null;
     try {
       Path resFile = new Path(showFuncs.getResFile());
       FileSystem fs = resFile.getFileSystem(conf);
-      DataOutput outStream = fs.create(resFile);
+      outStream = fs.create(resFile);
       SortedSet<String> sortedFuncs = new TreeSet<String>(funcs);
       Iterator<String> iterFuncs = sortedFuncs.iterator();
 
@@ -1831,6 +1857,7 @@ public class DDLTask extends Task<DDLWor
         outStream.write(terminator);
       }
       ((FSDataOutputStream) outStream).close();
+      outStream = null;
     } catch (FileNotFoundException e) {
       LOG.warn("show function: " + stringifyException(e));
       return 1;
@@ -1839,6 +1866,8 @@ public class DDLTask extends Task<DDLWor
       return 1;
     } catch (Exception e) {
       throw new HiveException(e.toString());
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream) outStream);
     }
     return 0;
   }
@@ -1861,10 +1890,11 @@ public class DDLTask extends Task<DDLWor
     }
 
     // write the results in the file
+    DataOutput outStream = null;
     try {
       Path resFile = new Path(showLocks.getResFile());
       FileSystem fs = resFile.getFileSystem(conf);
-      DataOutput outStream = fs.create(resFile);
+      outStream = fs.create(resFile);
       List<HiveLock> locks = null;
 
       if (showLocks.getTableName() == null) {
@@ -1915,6 +1945,7 @@ public class DDLTask extends Task<DDLWor
         outStream.write(terminator);
       }
       ((FSDataOutputStream) outStream).close();
+      outStream = null;
     } catch (FileNotFoundException e) {
       LOG.warn("show function: " + stringifyException(e));
       return 1;
@@ -1923,6 +1954,8 @@ public class DDLTask extends Task<DDLWor
       return 1;
     } catch (Exception e) {
       throw new HiveException(e.toString());
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream) outStream);
     }
     return 0;
   }
@@ -2040,10 +2073,11 @@ public class DDLTask extends Task<DDLWor
     String funcName = descFunc.getName();
 
     // write the results in the file
+    DataOutput outStream = null;
     try {
       Path resFile = new Path(descFunc.getResFile());
       FileSystem fs = resFile.getFileSystem(conf);
-      DataOutput outStream = fs.create(resFile);
+      outStream = fs.create(resFile);
 
       // get the function documentation
       Description desc = null;
@@ -2079,6 +2113,7 @@ public class DDLTask extends Task<DDLWor
       outStream.write(terminator);
 
       ((FSDataOutputStream) outStream).close();
+      outStream = null;
     } catch (FileNotFoundException e) {
       LOG.warn("describe function: " + stringifyException(e));
       return 1;
@@ -2087,15 +2122,18 @@ public class DDLTask extends Task<DDLWor
       return 1;
     } catch (Exception e) {
       throw new HiveException(e.toString());
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream) outStream);
     }
     return 0;
   }
 
   private int descDatabase(DescDatabaseDesc descDatabase) throws HiveException {
+    DataOutput outStream = null;
     try {
       Path resFile = new Path(descDatabase.getResFile());
       FileSystem fs = resFile.getFileSystem(conf);
-      DataOutput outStream = fs.create(resFile);
+      outStream = fs.create(resFile);
 
       Database database = db.getDatabase(descDatabase.getDatabaseName());
 
@@ -2123,6 +2161,7 @@ public class DDLTask extends Task<DDLWor
       outStream.write(terminator);
 
       ((FSDataOutputStream) outStream).close();
+      outStream = null;
 
     } catch (FileNotFoundException e) {
       LOG.warn("describe database: " + stringifyException(e));
@@ -2132,6 +2171,8 @@ public class DDLTask extends Task<DDLWor
       return 1;
     } catch (Exception e) {
       throw new HiveException(e.toString());
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream) outStream);
     }
     return 0;
   }
@@ -2174,10 +2215,11 @@ public class DDLTask extends Task<DDLWor
     }
 
     // write the results in the file
+    DataOutput outStream = null;
     try {
       Path resFile = new Path(showTblStatus.getResFile());
       FileSystem fs = resFile.getFileSystem(conf);
-      DataOutput outStream = fs.create(resFile);
+      outStream = fs.create(resFile);
 
       Iterator<Table> iterTables = tbls.iterator();
       while (iterTables.hasNext()) {
@@ -2256,6 +2298,7 @@ public class DDLTask extends Task<DDLWor
         outStream.write(terminator);
       }
       ((FSDataOutputStream) outStream).close();
+      outStream = null;
     } catch (FileNotFoundException e) {
       LOG.info("show table status: " + stringifyException(e));
       return 1;
@@ -2264,6 +2307,8 @@ public class DDLTask extends Task<DDLWor
       return 1;
     } catch (Exception e) {
       throw new HiveException(e);
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream) outStream);
     }
     return 0;
   }
@@ -2287,25 +2332,28 @@ public class DDLTask extends Task<DDLWor
     // describe the table - populate the output stream
     Table tbl = db.getTable(tableName, false);
     Partition part = null;
+    DataOutput outStream = null;
     try {
       Path resFile = new Path(descTbl.getResFile());
       if (tbl == null) {
         FileSystem fs = resFile.getFileSystem(conf);
-        DataOutput outStream = (DataOutput) fs.open(resFile);
+        outStream = (DataOutput) fs.open(resFile);
         String errMsg = "Table " + tableName + " does not exist";
         outStream.write(errMsg.getBytes("UTF-8"));
         ((FSDataOutputStream) outStream).close();
+        outStream = null;
         return 0;
       }
       if (descTbl.getPartSpec() != null) {
         part = db.getPartition(tbl, descTbl.getPartSpec(), false);
         if (part == null) {
           FileSystem fs = resFile.getFileSystem(conf);
-          DataOutput outStream = (DataOutput) fs.open(resFile);
+          outStream = (DataOutput) fs.open(resFile);
           String errMsg = "Partition " + descTbl.getPartSpec() + " for table "
               + tableName + " does not exist";
           outStream.write(errMsg.getBytes("UTF-8"));
           ((FSDataOutputStream) outStream).close();
+          outStream = null;
           return 0;
         }
         tbl = part.getTable();
@@ -2316,6 +2364,8 @@ public class DDLTask extends Task<DDLWor
     } catch (IOException e) {
       LOG.info("describe table: " + stringifyException(e));
       return 1;
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream) outStream);
     }
 
     try {
@@ -2324,7 +2374,7 @@ public class DDLTask extends Task<DDLWor
 
       Path resFile = new Path(descTbl.getResFile());
       FileSystem fs = resFile.getFileSystem(conf);
-      DataOutput outStream = fs.create(resFile);
+      outStream = fs.create(resFile);
 
       if (colPath.equals(tableName)) {
         if (!descTbl.isFormatted()) {
@@ -2380,6 +2430,7 @@ public class DDLTask extends Task<DDLWor
 
       LOG.info("DDLTask: written data for " + tbl.getTableName());
       ((FSDataOutputStream) outStream).close();
+      outStream = null;
 
     } catch (FileNotFoundException e) {
       LOG.info("describe table: " + stringifyException(e));
@@ -2389,6 +2440,8 @@ public class DDLTask extends Task<DDLWor
       return 1;
     } catch (Exception e) {
       throw new HiveException(e);
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream) outStream);
     }
 
     return 0;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileInputFormat.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileInputFormat.java?rev=1148921&r1=1148920&r2=1148921&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileInputFormat.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileInputFormat.java Wed Jul 20 19:54:14 2011
@@ -64,12 +64,18 @@ public class RCFileInputFormat<K extends
       return false;
     }
     for (int fileId = 0; fileId < files.size(); fileId++) {
+      RCFile.Reader reader = null;
       try {
-        RCFile.Reader reader = new RCFile.Reader(fs, files.get(fileId)
+        reader = new RCFile.Reader(fs, files.get(fileId)
             .getPath(), conf);
         reader.close();
+        reader = null;
       } catch (IOException e) {
         return false;
+      } finally {
+        if (null != reader) {
+          reader.close();
+        }
       }
     }
     return true;