tajo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jihoon...@apache.org
Subject tajo git commit: TAJO-1877: Adopt try-resource statement to AbstractDBStore.
Date Tue, 27 Oct 2015 13:23:34 GMT
Repository: tajo
Updated Branches:
  refs/heads/master 6947bc5a9 -> 8491b5dfa


TAJO-1877: Adopt try-resource statement to AbstractDBStore.

Closes #835

Signed-off-by: Jihoon Son <jihoonson@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/8491b5df
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/8491b5df
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/8491b5df

Branch: refs/heads/master
Commit: 8491b5dfa1ea000f75fcb2d4db4d0f53fe34fa36
Parents: 6947bc5
Author: Dongkyu Hwangbo <hwangbodk@gmail.com>
Authored: Tue Oct 27 22:22:34 2015 +0900
Committer: Jihoon Son <jihoonson@apache.org>
Committed: Tue Oct 27 22:23:15 2015 +0900

----------------------------------------------------------------------
 CHANGES                                         |   3 +
 .../tajo/catalog/store/AbstractDBStore.java     | 554 +++++++------------
 2 files changed, 188 insertions(+), 369 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/8491b5df/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 3cc6c83..5b2ff4e 100644
--- a/CHANGES
+++ b/CHANGES
@@ -56,6 +56,9 @@ Release 0.12.0 - unreleased
 
   SUB TASKS
 
+    TAJO-1877: Adopt try-resource statement to AbstractDBStore. 
+    (Contributed by Dongkyu Hwangbo, Committed by jihoon)
+
     TAJO-1653: Add 'INSERT INTO VALUES' statement for OLTP-like storages.
     (hyunsik)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/8491b5df/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
index e86bbba..9904057 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java
@@ -239,9 +239,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   }
 
   private int getSchemaVersion() {
-    Connection conn = null;
-    PreparedStatement pstmt = null;
-    ResultSet result = null;
     int schemaVersion = -1;
     
     String sql = "SELECT version FROM META";
@@ -249,18 +246,13 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
       LOG.debug(sql.toString());
     }
 
-    try {
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
-      result = pstmt.executeQuery();
-
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql);
+         ResultSet result = pstmt.executeQuery()) {
       if (result.next()) {
         schemaVersion = result.getInt("VERSION");
       }
     } catch (SQLException e) {
       throw new TajoInternalError(e);
-    } finally {
-      CatalogUtil.closeQuietly(pstmt, result);
     }
     
     return schemaVersion;
@@ -293,17 +285,11 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
    * Insert the version of the current catalog schema
    */
   protected void insertSchemaVersion() {
-    Connection conn;
-    PreparedStatement pstmt = null;
-    try {
-      conn = getConnection();
-      pstmt = conn.prepareStatement("INSERT INTO META VALUES (?)");
+    try (PreparedStatement pstmt = getConnection().prepareStatement("INSERT INTO META VALUES
(?)")) {
       pstmt.setInt(1, getDriverVersion());
       pstmt.executeUpdate();
     } catch (SQLException se) {
       throw new TajoInternalError(se);
-    } finally {
-      CatalogUtil.closeQuietly(pstmt);
     }
   }
 
@@ -355,27 +341,23 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
 
   @Override
   public boolean existTablespace(String tableSpaceName) {
-    Connection conn = null;
-    PreparedStatement pstmt = null;
     ResultSet res = null;
     boolean exist = false;
 
-    try {
-      StringBuilder sql = new StringBuilder();
-      sql.append("SELECT SPACE_NAME FROM " + TB_SPACES + " WHERE SPACE_NAME = ?");
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql.toString());
-      }
+    StringBuilder sql = new StringBuilder();
+    sql.append("SELECT SPACE_NAME FROM " + TB_SPACES + " WHERE SPACE_NAME = ?");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql.toString());
+    }
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql.toString());
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql.toString())) {
       pstmt.setString(1, tableSpaceName);
       res = pstmt.executeQuery();
       exist = res.next();
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
 
     return exist;
@@ -427,29 +409,21 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   }
 
   private Collection<String> getAllTablespaceNamesInternal(@Nullable String whereCondition)
{
-    Connection conn = null;
-    PreparedStatement pstmt = null;
-    ResultSet resultSet = null;
-
     List<String> tablespaceNames = new ArrayList<>();
 
-    try {
-      String sql = "SELECT SPACE_NAME FROM " + TB_SPACES;
+    String sql = "SELECT SPACE_NAME FROM " + TB_SPACES;
 
-      if (whereCondition != null) {
-        sql += " WHERE " + whereCondition;
-      }
+    if (whereCondition != null) {
+      sql += " WHERE " + whereCondition;
+    }
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
-      resultSet = pstmt.executeQuery();
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql);
+         ResultSet resultSet = pstmt.executeQuery()) {
       while (resultSet.next()) {
         tablespaceNames.add(resultSet.getString(1));
       }
     } catch (SQLException se) {
       throw new TajoInternalError(se);
-    } finally {
-      CatalogUtil.closeQuietly(pstmt, resultSet);
     }
 
     return tablespaceNames;
@@ -457,17 +431,12 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   
   @Override
   public List<TablespaceProto> getTablespaces() {
-    Connection conn = null;
-    Statement stmt = null;
-    ResultSet resultSet = null;
     List<TablespaceProto> tablespaces = TUtil.newList();
 
-    try {
-      String sql = "SELECT SPACE_ID, SPACE_NAME, SPACE_HANDLER, SPACE_URI FROM " + TB_SPACES
;
-      conn = getConnection();
-      stmt = conn.createStatement();
-      resultSet = stmt.executeQuery(sql);
+    String sql = "SELECT SPACE_ID, SPACE_NAME, SPACE_HANDLER, SPACE_URI FROM " + TB_SPACES
;
 
+    try (Statement stmt = getConnection().createStatement();
+         ResultSet resultSet = stmt.executeQuery(sql)) {
       while (resultSet.next()) {
         TablespaceProto.Builder builder = TablespaceProto.newBuilder();
         builder.setId(resultSet.getInt("SPACE_ID"));
@@ -481,21 +450,16 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
 
     } catch (SQLException se) {
       throw new TajoInternalError(se);
-    } finally {
-      CatalogUtil.closeQuietly(stmt, resultSet);
     }
   }
 
   @Override
   public TablespaceProto getTablespace(String spaceName) throws UndefinedTablespaceException
{
-    Connection conn = null;
-    PreparedStatement pstmt = null;
     ResultSet resultSet = null;
 
-    try {
-      String sql = "SELECT * FROM " + TB_SPACES + " WHERE SPACE_NAME=?";
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
+    String sql = "SELECT * FROM " + TB_SPACES + " WHERE SPACE_NAME=?";
+
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
       pstmt.setString(1, spaceName);
       resultSet = pstmt.executeQuery();
 
@@ -515,11 +479,10 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
           setUri(uri).
           build();
 
-
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, resultSet);
+      CatalogUtil.closeQuietly(resultSet);
     }
   }
 
@@ -609,27 +572,23 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
 
   @Override
   public boolean existDatabase(String databaseName) {
-    Connection conn = null;
-    PreparedStatement pstmt = null;
     ResultSet res = null;
     boolean exist = false;
 
-    try {
-      StringBuilder sql = new StringBuilder();
-      sql.append("SELECT DB_NAME FROM " + TB_DATABASES + " WHERE DB_NAME = ?");
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql.toString());
-      }
+    StringBuilder sql = new StringBuilder();
+    sql.append("SELECT DB_NAME FROM " + TB_DATABASES + " WHERE DB_NAME = ?");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql.toString());
+    }
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql.toString());
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql.toString())) {
       pstmt.setString(1, databaseName);
       res = pstmt.executeQuery();
       exist = res.next();
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
 
     return exist;
@@ -678,29 +637,21 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   }
 
   private Collection<String> getAllDatabaseNamesInternal(@Nullable String whereCondition)
{
-    Connection conn = null;
-    PreparedStatement pstmt = null;
-    ResultSet resultSet = null;
-
     List<String> databaseNames = new ArrayList<>();
 
-    try {
-      String sql = "SELECT DB_NAME FROM " + TB_DATABASES;
+    String sql = "SELECT DB_NAME FROM " + TB_DATABASES;
 
-      if (whereCondition != null) {
-        sql += " WHERE " + whereCondition;
-      }
+    if (whereCondition != null) {
+      sql += " WHERE " + whereCondition;
+    }
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
-      resultSet = pstmt.executeQuery();
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql);
+         ResultSet resultSet = pstmt.executeQuery()) {
       while (resultSet.next()) {
         databaseNames.add(resultSet.getString(1));
       }
     } catch (SQLException se) {
       throw new TajoInternalError(se);
-    } finally {
-      CatalogUtil.closeQuietly(pstmt, resultSet);
     }
 
     return databaseNames;
@@ -708,18 +659,12 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   
   @Override
   public List<DatabaseProto> getAllDatabases() {
-    Connection conn = null;
-    Statement stmt = null;
-    ResultSet resultSet = null;
-
     List<DatabaseProto> databases = new ArrayList<>();
 
-    try {
-      String sql = "SELECT DB_ID, DB_NAME, SPACE_ID FROM " + TB_DATABASES;
+    String sql = "SELECT DB_ID, DB_NAME, SPACE_ID FROM " + TB_DATABASES;
 
-      conn = getConnection();
-      stmt = conn.createStatement();
-      resultSet = stmt.executeQuery(sql);
+    try (Statement stmt = getConnection().createStatement();
+         ResultSet resultSet = stmt.executeQuery(sql)) {
       while (resultSet.next()) {
         DatabaseProto.Builder builder = DatabaseProto.newBuilder();
         
@@ -731,8 +676,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
       }
     } catch (SQLException se) {
       throw new TajoInternalError(se);
-    } finally {
-      CatalogUtil.closeQuietly(stmt, resultSet);
     }
     
     return databases;
@@ -763,14 +706,11 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   }
 
   private TableSpaceInternal getTableSpaceInfo(String spaceName) throws UndefinedTablespaceException
{
-    Connection conn = null;
-    PreparedStatement pstmt = null;
     ResultSet res = null;
 
-    try {
-      String sql = "SELECT SPACE_ID, SPACE_URI, SPACE_HANDLER from " + TB_SPACES + " WHERE
SPACE_NAME = ?";
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
+    String sql = "SELECT SPACE_ID, SPACE_URI, SPACE_HANDLER from " + TB_SPACES + " WHERE
SPACE_NAME = ?";
+
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
       pstmt.setString(1, spaceName);
       res = pstmt.executeQuery();
       if (!res.next()) {
@@ -780,19 +720,16 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
   }
 
   private int getTableId(int databaseId, String databaseName, String tableName) throws UndefinedTableException
{
-    Connection conn = null;
-    PreparedStatement pstmt = null;
     ResultSet res = null;
 
-    try {
-      String tidSql = "SELECT TID from TABLES WHERE db_id = ? AND " + COL_TABLES_NAME + "=?";
-      conn = getConnection();
-      pstmt = conn.prepareStatement(tidSql);
+    String tidSql = "SELECT TID from TABLES WHERE db_id = ? AND " + COL_TABLES_NAME + "=?";
+
+    try (PreparedStatement pstmt = getConnection().prepareStatement(tidSql)) {
       pstmt.setInt(1, databaseId);
       pstmt.setString(2, tableName);
       res = pstmt.executeQuery();
@@ -803,7 +740,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
   }
 
@@ -1105,16 +1042,12 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   }
 
   private Map<String, String> getTableOptions(final int tableId) {
-    Connection conn = null;
-    PreparedStatement pstmt = null;
     ResultSet res = null;
     Map<String, String> options = new HashMap<>();
 
-    try {
-      String tidSql = "SELECT key_, value_ FROM " + TB_OPTIONS + " WHERE TID=?";
+    String tidSql = "SELECT key_, value_ FROM " + TB_OPTIONS + " WHERE TID=?";
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(tidSql);
+    try (PreparedStatement pstmt = getConnection().prepareStatement(tidSql)) {
       pstmt.setInt(1, tableId);
       res = pstmt.executeQuery();
 
@@ -1124,7 +1057,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
 
     return options;
@@ -1181,21 +1114,14 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
       LOG.debug(updtaeRenameTableSql);
     }
 
-    Connection conn;
-    PreparedStatement pstmt = null;
+    try (PreparedStatement pstmt = getConnection().prepareStatement(updtaeRenameTableSql))
{
 
-    try {
-
-      conn = getConnection();
-      pstmt = conn.prepareStatement(updtaeRenameTableSql);
       pstmt.setString(1, tableName);
       pstmt.setInt(2, tableId);
       pstmt.executeUpdate();
 
     } catch (SQLException se) {
       throw new TajoInternalError(se);
-    } finally {
-      CatalogUtil.closeQuietly(pstmt);
     }
   }
 
@@ -1208,13 +1134,8 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
       LOG.debug(updtaeRenameTableSql);
     }
 
-    Connection conn;
-    PreparedStatement pstmt = null;
+    try (PreparedStatement pstmt = getConnection().prepareStatement(updtaeRenameTableSql))
{
 
-    try {
-
-      conn = getConnection();
-      pstmt = conn.prepareStatement(updtaeRenameTableSql);
       pstmt.setString(1, tableName);
       pstmt.setString(2, newTablePath);
       pstmt.setInt(3, tableId);
@@ -1222,8 +1143,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
 
     } catch (SQLException se) {
       throw new TajoInternalError(se);
-    } finally {
-      CatalogUtil.closeQuietly(pstmt);
     }
   }
 
@@ -1459,13 +1378,9 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
       LOG.debug(sql);
     }
 
-    Connection conn = null;
-    PreparedStatement pstmt = null;
     ResultSet res = null;
 
-    try {
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
       pstmt.setString(1, databaseName);
       res = pstmt.executeQuery();
       if (!res.next()) {
@@ -1476,28 +1391,23 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException e) {
       throw new TajoInternalError(e);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
   }
 
   @Override
   public boolean existTable(String databaseName, final String tableName) throws UndefinedDatabaseException
{
-    Connection conn = null;
-    PreparedStatement pstmt = null;
     ResultSet res = null;
     boolean exist = false;
 
-    try {
-      int dbid = getDatabaseId(databaseName);
-
-      String sql = "SELECT TID FROM TABLES WHERE DB_ID = ? AND " + COL_TABLES_NAME + "=?";
+    String sql = "SELECT TID FROM TABLES WHERE DB_ID = ? AND " + COL_TABLES_NAME + "=?";
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql.toString());
-      }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql.toString());
+    }
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql.toString());
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql.toString())) {
+      int dbid = getDatabaseId(databaseName);
 
       pstmt.setInt(1, dbid);
       pstmt.setString(2, tableName);
@@ -1506,7 +1416,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
 
     return exist;
@@ -1637,13 +1547,9 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
       LOG.debug(sql);
     }
 
-    Connection conn = null;
-    PreparedStatement pstmt = null;
     ResultSet res = null;
 
-    try {
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
       pstmt.setString(1, databaseName);
       res = pstmt.executeQuery();
       if (!res.next()) {
@@ -1654,7 +1560,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException e) {
       throw new TajoInternalError(e);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
   }
 
@@ -1814,24 +1720,19 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
 
   @Override
   public List<String> getAllTableNames(String databaseName) throws UndefinedDatabaseException
{
-    Connection conn = null;
-    PreparedStatement pstmt = null;
     ResultSet res = null;
 
     List<String> tables = new ArrayList<>();
 
-    try {
-
-      int dbid = getDatabaseId(databaseName);
+    String sql = "SELECT " + COL_TABLES_NAME + " FROM TABLES WHERE DB_ID = ?";
 
-      String sql = "SELECT " + COL_TABLES_NAME + " FROM TABLES WHERE DB_ID = ?";
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql);
+    }
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql);
-      }
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
+      int dbid = getDatabaseId(databaseName);
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
       pstmt.setInt(1, dbid);
       res = pstmt.executeQuery();
       while (res.next()) {
@@ -1840,27 +1741,21 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
     return tables;
   }
   
   @Override
   public List<TableDescriptorProto> getAllTables() {
-    Connection conn = null;
-    Statement stmt = null;
-    ResultSet resultSet = null;
-
     List<TableDescriptorProto> tables = new ArrayList<>();
 
-    try {
-      String sql = "SELECT t.TID, t.DB_ID, t." + COL_TABLES_NAME + ", t.TABLE_TYPE, t.PATH,
t.DATA_FORMAT, " +
-          " s.SPACE_URI FROM " + TB_TABLES + " t, " + TB_DATABASES + " d, " + TB_SPACES +
-          " s WHERE t.DB_ID = d.DB_ID AND d.SPACE_ID = s.SPACE_ID";
+    String sql = "SELECT t.TID, t.DB_ID, t." + COL_TABLES_NAME + ", t.TABLE_TYPE, t.PATH,
t.DATA_FORMAT, " +
+            " s.SPACE_URI FROM " + TB_TABLES + " t, " + TB_DATABASES + " d, " + TB_SPACES
+
+            " s WHERE t.DB_ID = d.DB_ID AND d.SPACE_ID = s.SPACE_ID";
 
-      conn = getConnection();
-      stmt = conn.createStatement();
-      resultSet = stmt.executeQuery(sql);
+    try (Statement stmt = getConnection().createStatement();
+         ResultSet resultSet = stmt.executeQuery(sql)) {
       while (resultSet.next()) {
         TableDescriptorProto.Builder builder = TableDescriptorProto.newBuilder();
         
@@ -1887,8 +1782,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
       }
     } catch (SQLException se) {
       throw new TajoInternalError(se);
-    } finally {
-      CatalogUtil.closeQuietly(stmt, resultSet);
     }
     
     return tables;
@@ -1896,18 +1789,12 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   
   @Override
   public List<TableOptionProto> getAllTableProperties() {
-    Connection conn = null;
-    Statement stmt = null;
-    ResultSet resultSet = null;
-
     List<TableOptionProto> options = new ArrayList<>();
 
-    try {
-      String sql = "SELECT tid, key_, value_ FROM " + TB_OPTIONS;
+    String sql = "SELECT tid, key_, value_ FROM " + TB_OPTIONS;
 
-      conn = getConnection();
-      stmt = conn.createStatement();
-      resultSet = stmt.executeQuery(sql);
+    try (Statement stmt = getConnection().createStatement();
+         ResultSet resultSet = stmt.executeQuery(sql)) {
       while (resultSet.next()) {
         TableOptionProto.Builder builder = TableOptionProto.newBuilder();
         
@@ -1922,8 +1809,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
       }
     } catch (SQLException se) {
       throw new TajoInternalError(se);
-    } finally {
-      CatalogUtil.closeQuietly(stmt, resultSet);
     }
     
     return options;
@@ -1931,18 +1816,12 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   
   @Override
   public List<TableStatsProto> getAllTableStats() {
-    Connection conn = null;
-    Statement stmt = null;
-    ResultSet resultSet = null;
-
     List<TableStatsProto> stats = new ArrayList<>();
 
-    try {
-      String sql = "SELECT tid, num_rows, num_bytes FROM " + TB_STATISTICS;
+    String sql = "SELECT tid, num_rows, num_bytes FROM " + TB_STATISTICS;
 
-      conn = getConnection();
-      stmt = conn.createStatement();
-      resultSet = stmt.executeQuery(sql);
+    try (Statement stmt = getConnection().createStatement();
+         ResultSet resultSet = stmt.executeQuery(sql)) {
       while (resultSet.next()) {
         TableStatsProto.Builder builder = TableStatsProto.newBuilder();
         
@@ -1954,8 +1833,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
       }
     } catch (SQLException se) {
       throw new TajoInternalError(se);
-    } finally {
-      CatalogUtil.closeQuietly(stmt, resultSet);
     }
     
     return stats;
@@ -1971,8 +1848,8 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
 
     try {
       String sql =
-          "SELECT TID, COLUMN_NAME, ORDINAL_POSITION, NESTED_FIELD_NUM, DATA_TYPE, TYPE_LENGTH
FROM " + TB_COLUMNS +
-          " ORDER BY TID ASC, ORDINAL_POSITION ASC";
+              "SELECT TID, COLUMN_NAME, ORDINAL_POSITION, NESTED_FIELD_NUM, DATA_TYPE, TYPE_LENGTH
FROM " + TB_COLUMNS +
+                      " ORDER BY TID ASC, ORDINAL_POSITION ASC";
 
       conn = getConnection();
       stmt = conn.createStatement();
@@ -2014,24 +1891,20 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   public CatalogProtos.PartitionMethodProto getPartitionMethod(String databaseName, String
tableName) throws
       UndefinedDatabaseException, UndefinedTableException, UndefinedPartitionMethodException
{
 
-    Connection conn = null;
     ResultSet res = null;
-    PreparedStatement pstmt = null;
 
     final int databaseId = getDatabaseId(databaseName);
     final int tableId = getTableId(databaseId, databaseName, tableName);
     ensurePartitionTable(tableName, tableId);
 
-    try {
-      String sql = "SELECT partition_type, expression, expression_schema FROM " + TB_PARTITION_METHODS
+
-          " WHERE " + COL_TABLES_PK + " = ? ";
+    String sql = "SELECT partition_type, expression, expression_schema FROM " + TB_PARTITION_METHODS
+
+            " WHERE " + COL_TABLES_PK + " = ? ";
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql);
-      }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql);
+    }
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
       pstmt.setInt(1, tableId);
       res = pstmt.executeQuery();
 
@@ -2044,7 +1917,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (Throwable se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
   }
 
@@ -2052,24 +1925,20 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   public boolean existPartitionMethod(String databaseName, String tableName)
       throws UndefinedDatabaseException, UndefinedTableException {
 
-    Connection conn = null;
     ResultSet res = null;
-    PreparedStatement pstmt = null;
     boolean exist = false;
 
-    try {
-      String sql = "SELECT partition_type, expression, expression_schema FROM " + TB_PARTITION_METHODS
+
-          " WHERE " + COL_TABLES_PK + "= ?";
+    String sql = "SELECT partition_type, expression, expression_schema FROM " + TB_PARTITION_METHODS
+
+            " WHERE " + COL_TABLES_PK + "= ?";
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql);
-      }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql);
+    }
 
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
       int databaseId = getDatabaseId(databaseName);
       int tableId = getTableId(databaseId, databaseName, tableName);
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
       pstmt.setInt(1, tableId);
       res = pstmt.executeQuery();
 
@@ -2077,7 +1946,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
     return exist;
   }
@@ -2093,21 +1962,17 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
    */
   private void ensurePartitionTable(String tbName, int tableId)
       throws UndefinedTableException, UndefinedDatabaseException, UndefinedPartitionMethodException
{
-
-    Connection conn;
+    
     ResultSet res = null;
-    PreparedStatement pstmt = null;
 
-    try {
-      String sql = "SELECT partition_type, expression, expression_schema FROM " + TB_PARTITION_METHODS
+
-          " WHERE " + COL_TABLES_PK + "= ?";
+    String sql = "SELECT partition_type, expression, expression_schema FROM " + TB_PARTITION_METHODS
+
+            " WHERE " + COL_TABLES_PK + "= ?";
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql);
-      }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql);
+    }
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
       pstmt.setInt(1, tableId);
       res = pstmt.executeQuery();
 
@@ -2117,7 +1982,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
   }
 
@@ -2130,22 +1995,18 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     final int databaseId = getDatabaseId(databaseName);
     final int tableId = getTableId(databaseId, databaseName, tableName);
     ensurePartitionTable(tableName, tableId);
-
-    Connection conn = null;
+    
     ResultSet res = null;
-    PreparedStatement pstmt = null;
     PartitionDescProto.Builder builder = null;
 
-    try {
-      String sql = "SELECT PATH, " + COL_PARTITIONS_PK  + ", " + COL_PARTITION_BYTES + "
FROM " + TB_PARTTIONS +
-        " WHERE " + COL_TABLES_PK + " = ? AND PARTITION_NAME = ? ";
+    String sql = "SELECT PATH, " + COL_PARTITIONS_PK  + ", " + COL_PARTITION_BYTES + " FROM
" + TB_PARTTIONS +
+            " WHERE " + COL_TABLES_PK + " = ? AND PARTITION_NAME = ? ";
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql);
-      }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql);
+    }
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
       pstmt.setInt(1, tableId);
       pstmt.setString(2, partitionName);
       res = pstmt.executeQuery();
@@ -2163,22 +2024,18 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
     return builder.build();
   }
 
   private void setPartitionKeys(int pid, PartitionDescProto.Builder partitionDesc) {
-    Connection conn = null;
     ResultSet res = null;
-    PreparedStatement pstmt = null;
 
-    try {
-      String sql = "SELECT "+ COL_COLUMN_NAME  + " , "+ COL_PARTITION_VALUE
-        + " FROM " + TB_PARTTION_KEYS + " WHERE " + COL_PARTITIONS_PK + " = ? ";
+    String sql = "SELECT "+ COL_COLUMN_NAME  + " , "+ COL_PARTITION_VALUE
+            + " FROM " + TB_PARTTION_KEYS + " WHERE " + COL_PARTITIONS_PK + " = ? ";
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
       pstmt.setInt(1, pid);
       res = pstmt.executeQuery();
 
@@ -2191,16 +2048,14 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
   }
 
   @Override
   public List<PartitionDescProto> getPartitionsOfTable(String databaseName, String
tableName)
       throws UndefinedDatabaseException, UndefinedTableException, UndefinedPartitionMethodException
{
-    Connection conn = null;
     ResultSet res = null;
-    PreparedStatement pstmt = null;
     PartitionDescProto.Builder builder = null;
     List<PartitionDescProto> partitions = new ArrayList<>();
 
@@ -2208,16 +2063,14 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     final int tableId = getTableId(databaseId, databaseName, tableName);
     ensurePartitionTable(tableName, tableId);
 
-    try {
-      String sql = "SELECT PATH, PARTITION_NAME, " + COL_PARTITIONS_PK + ", " + COL_PARTITION_BYTES
-        + " FROM " + TB_PARTTIONS +" WHERE " + COL_TABLES_PK + " = ?  ";
+    String sql = "SELECT PATH, PARTITION_NAME, " + COL_PARTITIONS_PK + ", " + COL_PARTITION_BYTES
+            + " FROM " + TB_PARTTIONS +" WHERE " + COL_TABLES_PK + " = ?  ";
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql);
-      }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql);
+    }
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
       pstmt.setInt(1, tableId);
       res = pstmt.executeQuery();
 
@@ -2232,7 +2085,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
     return partitions;
   }
@@ -2242,28 +2095,24 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     UndefinedTableException, UndefinedPartitionMethodException {
 
     String sql = null;
-    Connection conn = null;
     ResultSet res = null;
-    PreparedStatement pstmt = null;
     boolean result = false;
 
     final int databaseId = getDatabaseId(databaseName);
     final int tableId = getTableId(databaseId, databaseName, tableName);
     ensurePartitionTable(tableName, tableId);
 
-    try {
-      if (this instanceof DerbyStore) {
-        sql = "SELECT 1 FROM " + TB_PARTTIONS +" WHERE " + COL_TABLES_PK + " = ? FETCH FIRST
ROW ONLY ";
-      } else {
-        sql = "SELECT 1 FROM " + TB_PARTTIONS +" WHERE " + COL_TABLES_PK + " = ? LIMIT 1
";
-      }
+    if (this instanceof DerbyStore) {
+      sql = "SELECT 1 FROM " + TB_PARTTIONS +" WHERE " + COL_TABLES_PK + " = ? FETCH FIRST
ROW ONLY ";
+    } else {
+      sql = "SELECT 1 FROM " + TB_PARTTIONS +" WHERE " + COL_TABLES_PK + " = ? LIMIT 1 ";
+    }
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql);
-      }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql);
+    }
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
       pstmt.setInt(1, tableId);
       res = pstmt.executeQuery();
 
@@ -2273,7 +2122,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
     return result;
   }
@@ -2302,7 +2151,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
       TableDescProto tableDesc = getTable(request.getDatabaseName(), request.getTableName());
 
       pair = getSelectStatementAndPartitionFilterSet(tableDesc.getTableName(), tableDesc.getPartition()
-        .getExpressionSchema().getFieldsList(), request.getAlgebra());
+              .getExpressionSchema().getFieldsList(), request.getAlgebra());
 
       selectStatement = pair.getFirst();
       filterSets = pair.getSecond();
@@ -2489,19 +2338,13 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
 
   @Override
   public List<TablePartitionProto> getAllPartitions() {
-    Connection conn = null;
-    Statement stmt = null;
-    ResultSet resultSet = null;
-
     List<TablePartitionProto> partitions = new ArrayList<>();
 
-    try {
-      String sql = "SELECT " + COL_PARTITIONS_PK + ", " + COL_TABLES_PK + ", PARTITION_NAME,
" +
-        " PATH FROM " + TB_PARTTIONS;
+    String sql = "SELECT " + COL_PARTITIONS_PK + ", " + COL_TABLES_PK + ", PARTITION_NAME,
" +
+            " PATH FROM " + TB_PARTTIONS;
 
-      conn = getConnection();
-      stmt = conn.createStatement();
-      resultSet = stmt.executeQuery(sql);
+    try (Statement stmt = getConnection().createStatement();
+         ResultSet resultSet = stmt.executeQuery(sql)) {
       while (resultSet.next()) {
         TablePartitionProto.Builder builder = TablePartitionProto.newBuilder();
 
@@ -2514,8 +2357,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
       }
     } catch (SQLException se) {
       throw new TajoInternalError(se);
-    } finally {
-      CatalogUtil.closeQuietly(stmt, resultSet);
     }
 
     return partitions;
@@ -2761,11 +2602,9 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
 
   public static String getTableName(Connection conn, int tableId) throws SQLException {
     ResultSet res = null;
-    PreparedStatement pstmt = null;
 
-    try {
-      pstmt =
-          conn.prepareStatement("SELECT " + COL_TABLES_NAME + " FROM " + TB_TABLES + " WHERE
" + COL_TABLES_PK + "=?");
+    try (PreparedStatement pstmt =
+                 conn.prepareStatement("SELECT " + COL_TABLES_NAME + " FROM " + TB_TABLES
+ " WHERE " + COL_TABLES_PK + "=?")) {
       pstmt.setInt(1, tableId);
       res = pstmt.executeQuery();
       if (!res.next()) {
@@ -2773,7 +2612,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
       }
       return res.getString(1);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
   }
 
@@ -2861,29 +2700,25 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   public IndexDescProto getIndexByColumns(String databaseName, String tableName, String[]
columnNames)
       throws UndefinedDatabaseException, UndefinedTableException, UndefinedIndexException
{
 
-    Connection conn = null;
     ResultSet res = null;
-    PreparedStatement pstmt = null;
     IndexDescProto proto = null;
 
-    try {
+    String sql = GET_INDEXES_SQL + " WHERE " + COL_DATABASES_PK + "=? AND " +
+            COL_TABLES_PK + "=? AND COLUMN_NAMES=?";
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql);
+    }
+
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
       int databaseId = getDatabaseId(databaseName);
       int tableId = getTableId(databaseId, databaseName, tableName);
       TableDescProto tableDescProto = getTable(databaseName, tableName);
 
-      String sql = GET_INDEXES_SQL + " WHERE " + COL_DATABASES_PK + "=? AND " +
-          COL_TABLES_PK + "=? AND COLUMN_NAMES=?";
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql);
-      }
-
       // Since the column names in the unified name are always sorted
       // in order of occurrence position in the relation schema,
       // they can be uniquely identified.
       String unifiedName = CatalogUtil.getUnifiedSimpleColumnName(new Schema(tableDescProto.getSchema()),
columnNames);
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
       pstmt.setInt(1, databaseId);
       pstmt.setInt(2, tableId);
       pstmt.setString(3, unifiedName);
@@ -2900,7 +2735,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
 
     return proto;
@@ -2908,24 +2743,18 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
 
   @Override
   public boolean existIndexByName(String databaseName, final String indexName) throws UndefinedDatabaseException
{
-    Connection conn = null;
     ResultSet res = null;
-    PreparedStatement pstmt = null;
-
     boolean exist = false;
 
-    try {
-      int databaseId = getDatabaseId(databaseName);
-
-      String sql =
-          "SELECT INDEX_NAME FROM " + TB_INDEXES + " WHERE " + COL_DATABASES_PK + "=? AND
INDEX_NAME=?";
+    String sql =
+            "SELECT INDEX_NAME FROM " + TB_INDEXES + " WHERE " + COL_DATABASES_PK + "=? AND
INDEX_NAME=?";
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql);
-      }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql);
+    }
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
+      int databaseId = getDatabaseId(databaseName);
       pstmt.setInt(1, databaseId);
       pstmt.setString(2, indexName);
       res = pstmt.executeQuery();
@@ -2933,7 +2762,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
 
     return exist;
@@ -2942,32 +2771,26 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   @Override
   public boolean existIndexByColumns(String databaseName, String tableName, String[] columnNames)
       throws UndefinedDatabaseException, UndefinedTableException {
-
-    Connection conn = null;
     ResultSet res = null;
-    PreparedStatement pstmt = null;
-
     boolean exist = false;
 
-    try {
+    String sql =
+            "SELECT " + COL_INDEXES_PK + " FROM " + TB_INDEXES +
+                    " WHERE " + COL_DATABASES_PK + "=? AND " + COL_TABLES_PK + "=? AND COLUMN_NAMES=?";
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql);
+    }
+
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql)) {
       int databaseId = getDatabaseId(databaseName);
       int tableId = getTableId(databaseId, databaseName, tableName);
       Schema relationSchema = new Schema(getTable(databaseName, tableName).getSchema());
 
-      String sql =
-          "SELECT " + COL_INDEXES_PK + " FROM " + TB_INDEXES +
-              " WHERE " + COL_DATABASES_PK + "=? AND " + COL_TABLES_PK + "=? AND COLUMN_NAMES=?";
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql);
-      }
-
       // Since the column names in the unified name are always sorted
       // in order of occurrence position in the relation schema,
       // they can be uniquely identified.
       String unifiedName = CatalogUtil.getUnifiedSimpleColumnName(new Schema(relationSchema),
columnNames);
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
       pstmt.setInt(1, databaseId);
       pstmt.setInt(2, tableId);
       pstmt.setString(3, unifiedName);
@@ -2976,7 +2799,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
     return exist;
   }
@@ -3022,20 +2845,19 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
       throws UndefinedDatabaseException, UndefinedTableException {
 
     ResultSet res = null;
-    PreparedStatement pstmt = null;
 
-    try {
-      final int databaseId = getDatabaseId(databaseName);
-      final int tableId = getTableId(databaseId, databaseName, tableName);
+    String sql = GET_INDEXES_SQL + " WHERE " + COL_DATABASES_PK + "=? AND " + COL_TABLES_PK
+ "=?";
 
-      String sql = GET_INDEXES_SQL + " WHERE " + COL_DATABASES_PK + "=? AND " + COL_TABLES_PK
+ "=?";
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql);
+    }
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql);
-      }
+    conn = getConnection();
+
+    try (PreparedStatement pstmt = conn.prepareStatement(sql)) {
+      final int databaseId = getDatabaseId(databaseName);
+      final int tableId = getTableId(databaseId, databaseName, tableName);
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql);
       pstmt.setInt(1, databaseId);
       pstmt.setInt(2, tableId);
       res = pstmt.executeQuery();
@@ -3044,7 +2866,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
   }
 
@@ -3179,22 +3001,16 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
   }
 
   private boolean existColumn(final int tableId, final String columnName) {
-    Connection conn ;
-    PreparedStatement pstmt = null;
     ResultSet res = null;
     boolean exist = false;
 
-    try {
-
-      String sql = "SELECT COLUMN_NAME FROM " + TB_COLUMNS + " WHERE TID = ? AND COLUMN_NAME
= ?";
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(sql.toString());
-      }
+    String sql = "SELECT COLUMN_NAME FROM " + TB_COLUMNS + " WHERE TID = ? AND COLUMN_NAME
= ?";
 
-      conn = getConnection();
-      pstmt = conn.prepareStatement(sql.toString());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(sql.toString());
+    }
 
+    try (PreparedStatement pstmt = getConnection().prepareStatement(sql.toString())) {
       pstmt.setInt(1, tableId);
       pstmt.setString(2, columnName);
       res = pstmt.executeQuery();
@@ -3202,7 +3018,7 @@ public abstract class AbstractDBStore extends CatalogConstants implements
Catalo
     } catch (SQLException se) {
       throw new TajoInternalError(se);
     } finally {
-      CatalogUtil.closeQuietly(pstmt, res);
+      CatalogUtil.closeQuietly(res);
     }
 
     return exist;


Mime
View raw message