You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2014/02/26 17:22:47 UTC

svn commit: r1572141 [1/2] - in /hive/trunk: itests/hive-unit/src/test/java/org/apache/hive/jdbc/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/ ql/src/java/org/apache/hadoop/hive/ql/session/ serv...

Author: hashutosh
Date: Wed Feb 26 16:22:47 2014
New Revision: 1572141

URL: http://svn.apache.org/r1572141
Log:
HIVE-4545 : HS2 should return describe table results without space padding (Vaibhav Gumashta & Thejas Nair via Ashutosh Chauhan)

Modified:
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java?rev=1572141&r1=1572140&r2=1572141&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java Wed Feb 26 16:22:47 2014
@@ -1071,7 +1071,7 @@ public class TestJdbcDriver2 {
     tests.put("", new Object[]{});
 
     for (String checkPattern: tests.keySet()) {
-      ResultSet rs = (ResultSet)con.getMetaData().getTables("default", null, checkPattern, null);
+      ResultSet rs = con.getMetaData().getTables("default", null, checkPattern, null);
       ResultSetMetaData resMeta = rs.getMetaData();
       assertEquals(5, resMeta.getColumnCount());
       assertEquals("TABLE_CAT", resMeta.getColumnName(1));
@@ -1100,7 +1100,7 @@ public class TestJdbcDriver2 {
     }
 
     // only ask for the views.
-    ResultSet rs = (ResultSet)con.getMetaData().getTables("default", null, null
+    ResultSet rs = con.getMetaData().getTables("default", null, null
         , new String[]{viewTypeName});
     int cnt=0;
     while (rs.next()) {
@@ -1112,7 +1112,7 @@ public class TestJdbcDriver2 {
 
   @Test
   public void testMetaDataGetCatalogs() throws SQLException {
-    ResultSet rs = (ResultSet)con.getMetaData().getCatalogs();
+    ResultSet rs = con.getMetaData().getCatalogs();
     ResultSetMetaData resMeta = rs.getMetaData();
     assertEquals(1, resMeta.getColumnCount());
     assertEquals("TABLE_CAT", resMeta.getColumnName(1));
@@ -1122,7 +1122,7 @@ public class TestJdbcDriver2 {
 
   @Test
   public void testMetaDataGetSchemas() throws SQLException {
-    ResultSet rs = (ResultSet)con.getMetaData().getSchemas();
+    ResultSet rs = con.getMetaData().getSchemas();
     ResultSetMetaData resMeta = rs.getMetaData();
     assertEquals(2, resMeta.getColumnCount());
     assertEquals("TABLE_SCHEM", resMeta.getColumnName(1));
@@ -1172,7 +1172,7 @@ public class TestJdbcDriver2 {
    */
   private void metaDataGetTableTypeTest(Set<String> tabletypes)
       throws SQLException {
-    ResultSet rs = (ResultSet)con.getMetaData().getTableTypes();
+    ResultSet rs = con.getMetaData().getTableTypes();
 
     int cnt = 0;
     while (rs.next()) {
@@ -1237,7 +1237,7 @@ public class TestJdbcDriver2 {
    */
   @Test
   public void testMetaDataGetColumnsMetaData() throws SQLException {
-    ResultSet rs = (ResultSet)con.getMetaData().getColumns(null, null
+    ResultSet rs = con.getMetaData().getColumns(null, null
         , "testhivejdbcdriver\\_table", null);
 
     ResultSetMetaData rsmd = rs.getMetaData();
@@ -1301,18 +1301,34 @@ public class TestJdbcDriver2 {
     ResultSet res = stmt.executeQuery("describe " + tableName);
 
     res.next();
-    assertEquals("Column name 'under_col' not found", "under_col", res.getString(1).trim());
+    assertEquals("Column name 'under_col' not found", "under_col", res.getString(1));
     assertEquals("Column type 'under_col' for column under_col not found", "int", res
-        .getString(2).trim());
+        .getString(2));
     res.next();
-    assertEquals("Column name 'value' not found", "value", res.getString(1).trim());
+    assertEquals("Column name 'value' not found", "value", res.getString(1));
     assertEquals("Column type 'string' for column key not found", "string", res
-        .getString(2).trim());
+        .getString(2));
 
     assertFalse("More results found than expected", res.next());
   }
 
   @Test
+  public void testShowColumns() throws SQLException {
+    Statement stmt = con.createStatement();
+    assertNotNull("Statement is null", stmt);
+
+    ResultSet res = stmt.executeQuery("show columns in " + tableName);
+    res.next();
+    assertEquals("Column name 'under_col' not found",
+        "under_col", res.getString(1));
+
+    res.next();
+    assertEquals("Column name 'value' not found",
+        "value", res.getString(1));
+    assertFalse("More results found than expected", res.next());
+  }
+
+  @Test
   public void testDatabaseMetaData() throws SQLException {
     DatabaseMetaData meta = con.getMetaData();
 
@@ -1881,7 +1897,7 @@ public class TestJdbcDriver2 {
   public void testUnsupportedFetchTypes() throws Exception {
     try {
       con.createStatement(ResultSet.TYPE_SCROLL_SENSITIVE,
-        ResultSet.CONCUR_READ_ONLY);
+          ResultSet.CONCUR_READ_ONLY);
       fail("createStatement with TYPE_SCROLL_SENSITIVE should fail");
     } catch(SQLException e) {
       assertEquals("HYC00", e.getSQLState().trim());
@@ -1889,7 +1905,7 @@ public class TestJdbcDriver2 {
 
     try {
       con.createStatement(ResultSet.TYPE_SCROLL_INSENSITIVE,
-        ResultSet.CONCUR_UPDATABLE);
+          ResultSet.CONCUR_UPDATABLE);
       fail("createStatement with CONCUR_UPDATABLE should fail");
     } catch(SQLException e) {
       assertEquals("HYC00", e.getSQLState().trim());
@@ -1924,7 +1940,7 @@ public class TestJdbcDriver2 {
   private void execFetchFirst(String sqlStmt, String colName, boolean oneRowOnly)
       throws Exception {
     Statement stmt = con.createStatement(ResultSet.TYPE_SCROLL_INSENSITIVE,
-          ResultSet.CONCUR_READ_ONLY);
+        ResultSet.CONCUR_READ_ONLY);
     ResultSet res = stmt.executeQuery(sqlStmt);
 
     List<String> results = new ArrayList<String> ();

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=1572141&r1=1572140&r2=1572141&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 Feb 26 16:22:47 2014
@@ -216,11 +216,11 @@ public class DDLTask extends Task<DDLWor
     // normal human readable output or a json object.
     formatter = MetaDataFormatUtils.getFormatter(conf);
     INTERMEDIATE_ARCHIVED_DIR_SUFFIX =
-      HiveConf.getVar(conf, ConfVars.METASTORE_INT_ARCHIVED);
+        HiveConf.getVar(conf, ConfVars.METASTORE_INT_ARCHIVED);
     INTERMEDIATE_ORIGINAL_DIR_SUFFIX =
-      HiveConf.getVar(conf, ConfVars.METASTORE_INT_ORIGINAL);
+        HiveConf.getVar(conf, ConfVars.METASTORE_INT_ORIGINAL);
     INTERMEDIATE_EXTRACTED_DIR_SUFFIX =
-      HiveConf.getVar(conf, ConfVars.METASTORE_INT_EXTRACTED);
+        HiveConf.getVar(conf, ConfVars.METASTORE_INT_EXTRACTED);
   }
 
   @Override
@@ -447,7 +447,7 @@ public class DDLTask extends Task<DDLWor
       }
 
       AlterTableExchangePartition alterTableExchangePartition =
-        work.getAlterTableExchangePartition();
+          work.getAlterTableExchangePartition();
       if (alterTableExchangePartition != null) {
         return exchangeTablePartition(db, alterTableExchangePartition);
       }
@@ -508,7 +508,7 @@ public class DDLTask extends Task<DDLWor
           if (grantRole) {
             db.grantRole(roleName, userName, principal.getType(),
                 grantOrRevokeRoleDDL.getGrantor(), grantOrRevokeRoleDDL
-                    .getGrantorType(), grantOrRevokeRoleDDL.isGrantOption());
+                .getGrantorType(), grantOrRevokeRoleDDL.isGrantOption());
           } else {
             db.revokeRole(roleName, userName, principal.getType());
           }
@@ -556,7 +556,7 @@ public class DDLTask extends Task<DDLWor
     try {
       if (hiveObjectDesc == null) {
         privs.addAll(db.showPrivilegeGrant(HiveObjectType.GLOBAL, principalName, type,
-          null, null, null, null));
+            null, null, null, null));
       } else if (hiveObjectDesc != null && hiveObjectDesc.getObject() == null) {
         privs.addAll(db.showPrivilegeGrant(null, principalName, type, null, null, null, null));
       } else {
@@ -656,9 +656,9 @@ public class DDLTask extends Task<DDLWor
 
         //only grantInfo is used
         HiveObjectPrivilege thriftObjectPriv = new HiveObjectPrivilege(new HiveObjectRef(
-          AuthorizationUtils.getThriftHiveObjType(privObj.getType()),privObj.getDbname(),
-          privObj.getTableViewURI(),null,null), principal.getName(),
-          AuthorizationUtils.getThriftPrincipalType(principal.getType()), grantInfo);
+            AuthorizationUtils.getThriftHiveObjType(privObj.getType()),privObj.getDbname(),
+            privObj.getTableViewURI(),null,null), principal.getName(),
+            AuthorizationUtils.getThriftPrincipalType(principal.getType()), grantInfo);
         privList.add(thriftObjectPriv);
       }
       boolean testMode = conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST);
@@ -792,9 +792,9 @@ public class DDLTask extends Task<DDLWor
                         tableName, partValues, null), null, null,  new PrivilegeGrantInfo(priv.toString(), 0, grantor, grantorType, grantOption)));
               } else {
                 privBag
-                    .addToPrivileges(new HiveObjectPrivilege(
-                        new HiveObjectRef(HiveObjectType.TABLE, dbName,
-                            tableName, null, null), null, null, new PrivilegeGrantInfo(priv.toString(), 0, grantor, grantorType, grantOption)));
+                .addToPrivileges(new HiveObjectPrivilege(
+                    new HiveObjectRef(HiveObjectType.TABLE, dbName,
+                        tableName, null, null), null, null, new PrivilegeGrantInfo(priv.toString(), 0, grantor, grantorType, grantOption)));
               }
             } else {
               privBag.addToPrivileges(new HiveObjectPrivilege(
@@ -1040,7 +1040,7 @@ public class DDLTask extends Task<DDLWor
     }
 
     db
-        .createIndex(
+    .createIndex(
         crtIndex.getTableName(), crtIndex.getIndexName(), crtIndex.getIndexTypeHandlerClass(),
         crtIndex.getIndexedCols(), crtIndex.getIndexTableName(), crtIndex.getDeferredRebuild(),
         crtIndex.getInputFormat(), crtIndex.getOutputFormat(), crtIndex.getSerde(),
@@ -1049,12 +1049,12 @@ public class DDLTask extends Task<DDLWor
         crtIndex.getLineDelim(), crtIndex.getMapKeyDelim(), crtIndex.getIndexComment()
         );
     if (HiveUtils.getIndexHandler(conf, crtIndex.getIndexTypeHandlerClass()).usesIndexTable()) {
-        String indexTableName =
-            crtIndex.getIndexTableName() != null ? crtIndex.getIndexTableName() :
+      String indexTableName =
+          crtIndex.getIndexTableName() != null ? crtIndex.getIndexTableName() :
             MetaStoreUtils.getIndexTableName(SessionState.get().getCurrentDatabase(),
-            crtIndex.getTableName(), crtIndex.getIndexName());
-        Table indexTable = db.getTable(indexTableName);
-        work.getOutputs().add(new WriteEntity(indexTable));
+                crtIndex.getTableName(), crtIndex.getIndexName());
+          Table indexTable = db.getTable(indexTableName);
+          work.getOutputs().add(new WriteEntity(indexTable));
     }
     return 0;
   }
@@ -1066,54 +1066,54 @@ public class DDLTask extends Task<DDLWor
     Index idx = db.getIndex(dbName, baseTableName, indexName);
 
     switch(alterIndex.getOp()) {
-      case ADDPROPS:
-        idx.getParameters().putAll(alterIndex.getProps());
-        break;
-      case UPDATETIMESTAMP:
-        try {
-          Map<String, String> props = new HashMap<String, String>();
-          Map<Map<String, String>, Long> basePartTs = new HashMap<Map<String, String>, Long>();
+    case ADDPROPS:
+      idx.getParameters().putAll(alterIndex.getProps());
+      break;
+    case UPDATETIMESTAMP:
+      try {
+        Map<String, String> props = new HashMap<String, String>();
+        Map<Map<String, String>, Long> basePartTs = new HashMap<Map<String, String>, Long>();
 
-          Table baseTbl = db.getTable(SessionState.get().getCurrentDatabase(),
-              baseTableName);
+        Table baseTbl = db.getTable(SessionState.get().getCurrentDatabase(),
+            baseTableName);
 
-          if (baseTbl.isPartitioned()) {
-            List<Partition> baseParts;
-            if (alterIndex.getSpec() != null) {
-              baseParts = db.getPartitions(baseTbl, alterIndex.getSpec());
-            } else {
-              baseParts = db.getPartitions(baseTbl);
-            }
-            if (baseParts != null) {
-              for (Partition p : baseParts) {
-                FileSystem fs = p.getDataLocation().getFileSystem(db.getConf());
-                FileStatus fss = fs.getFileStatus(p.getDataLocation());
-                basePartTs.put(p.getSpec(), fss.getModificationTime());
-              }
-            }
+        if (baseTbl.isPartitioned()) {
+          List<Partition> baseParts;
+          if (alterIndex.getSpec() != null) {
+            baseParts = db.getPartitions(baseTbl, alterIndex.getSpec());
           } else {
-            FileSystem fs = baseTbl.getPath().getFileSystem(db.getConf());
-            FileStatus fss = fs.getFileStatus(baseTbl.getPath());
-            basePartTs.put(null, fss.getModificationTime());
-          }
-          for (Map<String, String> spec : basePartTs.keySet()) {
-            if (spec != null) {
-              props.put(spec.toString(), basePartTs.get(spec).toString());
-            } else {
-              props.put("base_timestamp", basePartTs.get(null).toString());
+            baseParts = db.getPartitions(baseTbl);
+          }
+          if (baseParts != null) {
+            for (Partition p : baseParts) {
+              FileSystem fs = p.getDataLocation().getFileSystem(db.getConf());
+              FileStatus fss = fs.getFileStatus(p.getDataLocation());
+              basePartTs.put(p.getSpec(), fss.getModificationTime());
             }
           }
-          idx.getParameters().putAll(props);
-        } catch (HiveException e) {
-          throw new HiveException("ERROR: Failed to update index timestamps");
-        } catch (IOException e) {
-          throw new HiveException("ERROR: Failed to look up timestamps on filesystem");
+        } else {
+          FileSystem fs = baseTbl.getPath().getFileSystem(db.getConf());
+          FileStatus fss = fs.getFileStatus(baseTbl.getPath());
+          basePartTs.put(null, fss.getModificationTime());
+        }
+        for (Map<String, String> spec : basePartTs.keySet()) {
+          if (spec != null) {
+            props.put(spec.toString(), basePartTs.get(spec).toString());
+          } else {
+            props.put("base_timestamp", basePartTs.get(null).toString());
+          }
         }
+        idx.getParameters().putAll(props);
+      } catch (HiveException e) {
+        throw new HiveException("ERROR: Failed to update index timestamps");
+      } catch (IOException e) {
+        throw new HiveException("ERROR: Failed to look up timestamps on filesystem");
+      }
 
-        break;
-      default:
-        console.printError("Unsupported Alter commnad");
-        return 1;
+      break;
+    default:
+      console.printError("Unsupported Alter commnad");
+      return 1;
     }
 
     // set last modified by properties
@@ -1178,17 +1178,17 @@ public class DDLTask extends Task<DDLWor
   }
 
   /**
-  * Alter partition column type in a table
-  *
-  * @param db
-  *          Database to rename the partition.
-  * @param alterPartitionDesc
-  *          change partition column type.
-  * @return Returns 0 when execution succeeds and above 0 if it fails.
-  * @throws HiveException
-  */
+   * Alter partition column type in a table
+   *
+   * @param db
+   *          Database to rename the partition.
+   * @param alterPartitionDesc
+   *          change partition column type.
+   * @return Returns 0 when execution succeeds and above 0 if it fails.
+   * @throws HiveException
+   */
   private int alterTableAlterPart(Hive db, AlterTableAlterPartDesc alterPartitionDesc)
-    throws HiveException {
+      throws HiveException {
 
     Table tbl = db.getTable(alterPartitionDesc.getDbName(), alterPartitionDesc.getTableName());
     String tabName = alterPartitionDesc.getTableName();
@@ -1389,7 +1389,7 @@ public class DDLTask extends Task<DDLWor
 
   private int archive(Hive db, AlterTableSimpleDesc simpleDesc,
       DriverContext driverContext)
-      throws HiveException {
+          throws HiveException {
     String dbName = simpleDesc.getDbName();
     String tblName = simpleDesc.getTableName();
 
@@ -1417,7 +1417,7 @@ public class DDLTask extends Task<DDLWor
       for(Partition p: partitions){
         if(partitionInCustomLocation(tbl, p)) {
           String message = String.format("ARCHIVE cannot run for partition " +
-                      "groups with custom locations like %s", p.getLocation());
+              "groups with custom locations like %s", p.getLocation());
           throw new HiveException(message);
         }
       }
@@ -1505,10 +1505,10 @@ public class DDLTask extends Task<DDLWor
       // First create the archive in a tmp dir so that if the job fails, the
       // bad files don't pollute the filesystem
       Path tmpPath = new Path(driverContext.getCtx()
-                    .getExternalTmpPath(originalDir.toUri()), "partlevel");
+          .getExternalTmpPath(originalDir.toUri()), "partlevel");
 
       console.printInfo("Creating " + archiveName +
-                        " for " + originalDir.toString());
+          " for " + originalDir.toString());
       console.printInfo("in " + tmpPath);
       console.printInfo("Please wait... (this may take a while)");
 
@@ -1517,7 +1517,7 @@ public class DDLTask extends Task<DDLWor
       try {
         int maxJobNameLen = conf.getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH);
         String jobname = String.format("Archiving %s@%s",
-          tbl.getTableName(), partSpecInfo.getName());
+            tbl.getTableName(), partSpecInfo.getName());
         jobname = Utilities.abbreviate(jobname, maxJobNameLen - 6);
         conf.setVar(HiveConf.ConfVars.HADOOPJOBNAME, jobname);
         ret = shim.createHadoopArchive(conf, originalDir, tmpPath, archiveName);
@@ -1542,7 +1542,7 @@ public class DDLTask extends Task<DDLWor
     } else {
       if (pathExists(intermediateArchivedDir)) {
         console.printInfo("Intermediate archive directory " + intermediateArchivedDir +
-        " already exists. Assuming it contains an archived version of the partition");
+            " already exists. Assuming it contains an archived version of the partition");
       }
     }
 
@@ -1650,7 +1650,7 @@ public class DDLTask extends Task<DDLWor
       for(Partition p: partitions){
         if(partitionInCustomLocation(tbl, p)) {
           String message = String.format("UNARCHIVE cannot run for partition " +
-                      "groups with custom locations like %s", p.getLocation());
+              "groups with custom locations like %s", p.getLocation());
           throw new HiveException(message);
         }
       }
@@ -1780,7 +1780,7 @@ public class DDLTask extends Task<DDLWor
       }
     } else {
       console.printInfo(intermediateArchivedDir + " already exists. " +
-      "Assuming it contains the archived version of the partition");
+          "Assuming it contains the archived version of the partition");
     }
 
     // If there is a failure from here to until when the metadata is changed,
@@ -1798,7 +1798,7 @@ public class DDLTask extends Task<DDLWor
       }
     } else {
       console.printInfo(originalDir + " already exists. " +
-      "Assuming it contains the extracted files in the partition");
+          "Assuming it contains the extracted files in the partition");
     }
 
     for(Partition p: partitions) {
@@ -2164,27 +2164,27 @@ public class DDLTask extends Task<DDLWor
           if (delims.containsKey(serdeConstants.FIELD_DELIM)) {
             tbl_row_format += "  FIELDS TERMINATED BY '" +
                 escapeHiveCommand(StringEscapeUtils.escapeJava(delims.get(
-                serdeConstants.FIELD_DELIM))) + "' \n";
+                    serdeConstants.FIELD_DELIM))) + "' \n";
           }
           if (delims.containsKey(serdeConstants.COLLECTION_DELIM)) {
             tbl_row_format += "  COLLECTION ITEMS TERMINATED BY '" +
                 escapeHiveCommand(StringEscapeUtils.escapeJava(delims.get(
-                serdeConstants.COLLECTION_DELIM))) + "' \n";
+                    serdeConstants.COLLECTION_DELIM))) + "' \n";
           }
           if (delims.containsKey(serdeConstants.MAPKEY_DELIM)) {
             tbl_row_format += "  MAP KEYS TERMINATED BY '" +
                 escapeHiveCommand(StringEscapeUtils.escapeJava(delims.get(
-                serdeConstants.MAPKEY_DELIM))) + "' \n";
+                    serdeConstants.MAPKEY_DELIM))) + "' \n";
           }
           if (delims.containsKey(serdeConstants.LINE_DELIM)) {
             tbl_row_format += "  LINES TERMINATED BY '" +
                 escapeHiveCommand(StringEscapeUtils.escapeJava(delims.get(
-                serdeConstants.LINE_DELIM))) + "' \n";
+                    serdeConstants.LINE_DELIM))) + "' \n";
           }
           if (delims.containsKey(serdeConstants.SERIALIZATION_NULL_FORMAT)) {
             tbl_row_format += "  NULL DEFINED AS '" +
                 escapeHiveCommand(StringEscapeUtils.escapeJava(delims.get(
-                serdeConstants.SERIALIZATION_NULL_FORMAT))) + "' \n";
+                    serdeConstants.SERIALIZATION_NULL_FORMAT))) + "' \n";
           }
         }
         else {
@@ -2400,7 +2400,7 @@ public class DDLTask extends Task<DDLWor
   }
 
   public int showColumns(Hive db, ShowColumnsDesc showCols)
-                         throws HiveException {
+      throws HiveException {
 
     String dbName = showCols.getDbName();
     String tableName = showCols.getTableName();
@@ -2421,8 +2421,11 @@ public class DDLTask extends Task<DDLWor
 
       List<FieldSchema> cols = table.getCols();
       cols.addAll(table.getPartCols());
-      outStream.writeBytes(
-          MetaDataFormatUtils.getAllColumnsInformation(cols, false));
+      // In case the query is served by HiveServer2, don't pad it with spaces,
+      // as HiveServer2 output is consumed by JDBC/ODBC clients.
+      boolean isOutputPadded = !SessionState.get().isHiveServerQuery();
+      outStream.writeBytes(MetaDataFormatUtils.getAllColumnsInformation(
+          cols, false, isOutputPadded));
       outStream.close();
       outStream = null;
     } catch (IOException e) {
@@ -2515,8 +2518,8 @@ public class DDLTask extends Task<DDLWor
       }
       else {
         locks = lockMgr.getLocks(getHiveObject(showLocks.getTableName(),
-                                               showLocks.getPartSpec()),
-                                 true, isExt);
+            showLocks.getPartSpec()),
+            true, isExt);
       }
 
       Collections.sort(locks, new Comparator<HiveLock>() {
@@ -2602,10 +2605,10 @@ public class DDLTask extends Task<DDLWor
 
     Map<String, String> partSpec = lockTbl.getPartSpec();
     HiveLockObjectData lockData =
-      new HiveLockObjectData(lockTbl.getQueryId(),
-                             String.valueOf(System.currentTimeMillis()),
-                             "EXPLICIT",
-                             lockTbl.getQueryStr());
+        new HiveLockObjectData(lockTbl.getQueryId(),
+            String.valueOf(System.currentTimeMillis()),
+            "EXPLICIT",
+            lockTbl.getQueryStr());
 
     if (partSpec == null) {
       HiveLock lck = lockMgr.lock(new HiveLockObject(tbl, lockData), mode, true);
@@ -2651,9 +2654,9 @@ public class DDLTask extends Task<DDLWor
     }
 
     HiveLockObjectData lockData =
-      new HiveLockObjectData(lockDb.getQueryId(),
-                             String.valueOf(System.currentTimeMillis()),
-                             "EXPLICIT", lockDb.getQueryStr());
+        new HiveLockObjectData(lockDb.getQueryId(),
+            String.valueOf(System.currentTimeMillis()),
+            "EXPLICIT", lockDb.getQueryStr());
 
     HiveLock lck = lockMgr.lock(new HiveLockObject(dbObj.getName(), lockData), mode, true);
     if (lck == null) {
@@ -2699,7 +2702,7 @@ public class DDLTask extends Task<DDLWor
   }
 
   private HiveLockObject getHiveObject(String tabName,
-                                       Map<String, String> partSpec) throws HiveException {
+      Map<String, String> partSpec) throws HiveException {
     Table  tbl = db.getTable(tabName);
     if (tbl == null) {
       throw new HiveException("Table " + tabName + " does not exist ");
@@ -2830,12 +2833,12 @@ public class DDLTask extends Task<DDLWor
       if (database == null) {
         throw new HiveException(ErrorMsg.DATABASE_NOT_EXISTS, descDatabase.getDatabaseName());
       } else {
-          Map<String, String> params = null;
-          if(descDatabase.isExt()) {
-            params = database.getParameters();
-          }
-          PrincipalType ownerType = database.getOwnerType();
-          formatter.showDatabaseDescription(outStream, database.getName(),
+        Map<String, String> params = null;
+        if(descDatabase.isExt()) {
+          params = database.getParameters();
+        }
+        PrincipalType ownerType = database.getOwnerType();
+        formatter.showDatabaseDescription(outStream, database.getName(),
             database.getDescription(), database.getLocationUri(),
             database.getOwnerName(), (null == ownerType) ? null : ownerType.name(), params);
       }
@@ -3018,7 +3021,7 @@ public class DDLTask extends Task<DDLWor
           outStream.close();
           outStream = null;
           throw new HiveException(ErrorMsg.INVALID_PARTITION,
-                  StringUtils.join(descTbl.getPartSpec().keySet(), ','), tableName);
+              StringUtils.join(descTbl.getPartSpec().keySet(), ','), tableName);
         }
         tbl = part.getTable();
       }
@@ -3039,19 +3042,22 @@ public class DDLTask extends Task<DDLWor
         cols = (part == null || tbl.getTableType() == TableType.VIRTUAL_VIEW) ?
             tbl.getCols() : part.getCols();
 
-        if (!descTbl.isFormatted()) {
-          if (tableName.equals(colPath)) {
-            cols.addAll(tbl.getPartCols());
-          }
-        }
+            if (!descTbl.isFormatted()) {
+              if (tableName.equals(colPath)) {
+                cols.addAll(tbl.getPartCols());
+              }
+            }
       } else {
         cols = Hive.getFieldsFromDeserializer(colPath, tbl.getDeserializer());
       }
 
       fixDecimalColumnTypeName(cols);
-
-      formatter.describeTable(outStream, colPath, tableName, tbl, part, cols,
-                              descTbl.isFormatted(), descTbl.isExt(), descTbl.isPretty());
+      // In case the query is served by HiveServer2, don't pad it with spaces,
+      // as HiveServer2 output is consumed by JDBC/ODBC clients.
+      boolean isOutputPadded = !SessionState.get().isHiveServerQuery();
+      formatter.describeTable(outStream, colPath, tableName, tbl, part,
+          cols, descTbl.isFormatted(), descTbl.isExt(),
+          descTbl.isPretty(), isOutputPadded);
 
       LOG.info("DDLTask: written data for " + tbl.getTableName());
       outStream.close();
@@ -3195,7 +3201,7 @@ public class DDLTask extends Task<DDLWor
         part = db.getPartition(tbl, alterTbl.getPartSpec(), false);
         if (part == null) {
           throw new HiveException(ErrorMsg.INVALID_PARTITION,
-                  StringUtils.join(alterTbl.getPartSpec().keySet(), ',') + " for table " + alterTbl.getOldName());
+              StringUtils.join(alterTbl.getPartSpec().keySet(), ',') + " for table " + alterTbl.getOldName());
         }
       }
       else {
@@ -3213,7 +3219,7 @@ public class DDLTask extends Task<DDLWor
       if (tbl.getSerializationLib().equals(
           "org.apache.hadoop.hive.serde.thrift.columnsetSerDe")) {
         console
-            .printInfo("Replacing columns for columnsetSerDe and changing to LazySimpleSerDe");
+        .printInfo("Replacing columns for columnsetSerDe and changing to LazySimpleSerDe");
         tbl.setSerializationLib(LazySimpleSerDe.class.getName());
         tbl.getTTable().getSd().setCols(newCols);
       } else {
@@ -3302,7 +3308,7 @@ public class DDLTask extends Task<DDLWor
       if (tbl.getSerializationLib().equals(
           "org.apache.hadoop.hive.serde.thrift.columnsetSerDe")) {
         console
-            .printInfo("Replacing columns for columnsetSerDe and changing to LazySimpleSerDe");
+        .printInfo("Replacing columns for columnsetSerDe and changing to LazySimpleSerDe");
         tbl.setSerializationLib(LazySimpleSerDe.class.getName());
       } else if (!tbl.getSerializationLib().equals(
           MetadataTypedColumnsetSerDe.class.getName())
@@ -3343,7 +3349,7 @@ public class DDLTask extends Task<DDLWor
               alterTbl.getProps());
         }
         tbl.setFields(Hive.getFieldsFromDeserializer(tbl.getTableName(), tbl.
-              getDeserializer()));
+            getDeserializer()));
       }
     } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.ADDFILEFORMAT) {
       if(part != null) {
@@ -3580,7 +3586,7 @@ public class DDLTask extends Task<DDLWor
             return;
           }
           throw new HiveException(
-            "Cannot drop a base table with DROP VIEW");
+              "Cannot drop a base table with DROP VIEW");
         }
       }
     }
@@ -3591,7 +3597,7 @@ public class DDLTask extends Task<DDLWor
     }
 
     int partitionBatchSize = HiveConf.getIntVar(conf,
-      ConfVars.METASTORE_BATCH_RETRIEVE_TABLE_PARTITION_MAX);
+        ConfVars.METASTORE_BATCH_RETRIEVE_TABLE_PARTITION_MAX);
 
     // We should check that all the partitions of the table can be dropped
     if (tbl != null && tbl.isPartitioned()) {
@@ -3599,13 +3605,13 @@ public class DDLTask extends Task<DDLWor
 
       for(int i=0; i < partitionNames.size(); i+= partitionBatchSize) {
         List<String> partNames = partitionNames.subList(i, Math.min(i+partitionBatchSize,
-          partitionNames.size()));
+            partitionNames.size()));
         List<Partition> listPartitions = db.getPartitionsByNames(tbl, partNames);
         for (Partition p: listPartitions) {
           if (!p.canDrop()) {
             throw new HiveException("Table " + tbl.getTableName() +
-              " Partition" + p.getName() +
-              " is protected from being dropped");
+                " Partition" + p.getName() +
+                " is protected from being dropped");
           }
         }
       }
@@ -3641,7 +3647,7 @@ public class DDLTask extends Task<DDLWor
     try {
 
       Deserializer d = ReflectionUtils.newInstance(conf.getClassByName(serdeName).
-        asSubclass(Deserializer.class), conf);
+          asSubclass(Deserializer.class), conf);
       if (d != null) {
         LOG.debug("Found class for " + serdeName);
       }
@@ -3755,8 +3761,8 @@ public class DDLTask extends Task<DDLWor
 
     if (crtTbl.getStorageHandler() != null) {
       tbl.setProperty(
-        org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE,
-        crtTbl.getStorageHandler());
+          org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE,
+          crtTbl.getStorageHandler());
     }
     HiveStorageHandler storageHandler = tbl.getStorageHandler();
 
@@ -3773,7 +3779,7 @@ public class DDLTask extends Task<DDLWor
       } else {
         String serDeClassName = storageHandler.getSerDeClass().getName();
         LOG.info("Use StorageHandler-supplied " + serDeClassName
-          + " for table " + crtTbl.getTableName());
+            + " for table " + crtTbl.getTableName());
         tbl.setSerializationLib(serDeClassName);
       }
     } else {
@@ -3804,7 +3810,7 @@ public class DDLTask extends Task<DDLWor
     }
     if (crtTbl.getSerdeProps() != null) {
       Iterator<Entry<String, String>> iter = crtTbl.getSerdeProps().entrySet()
-        .iterator();
+          .iterator();
       while (iter.hasNext()) {
         Entry<String, String> m = iter.next();
         tbl.setSerdeParam(m.getKey(), m.getValue());
@@ -3840,9 +3846,9 @@ public class DDLTask extends Task<DDLWor
     tbl.setOutputFormatClass(crtTbl.getOutputFormat());
 
     tbl.getTTable().getSd().setInputFormat(
-      tbl.getInputFormatClass().getName());
+        tbl.getInputFormatClass().getName());
     tbl.getTTable().getSd().setOutputFormat(
-      tbl.getOutputFormatClass().getName());
+        tbl.getOutputFormatClass().getName());
 
     if (crtTbl.isExternal()) {
       tbl.setProperty("EXTERNAL", "TRUE");
@@ -3937,7 +3943,7 @@ public class DDLTask extends Task<DDLWor
 
       if (crtTbl.getDefaultSerdeProps() != null) {
         Iterator<Entry<String, String>> iter = crtTbl.getDefaultSerdeProps().entrySet()
-          .iterator();
+            .iterator();
         while (iter.hasNext()) {
           Entry<String, String> m = iter.next();
           tbl.setSerdeParam(m.getKey(), m.getValue());

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java?rev=1572141&r1=1572140&r2=1572141&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/JsonMetaDataFormatter.java Wed Feb 26 16:22:47 2014
@@ -47,371 +47,344 @@ import org.codehaus.jackson.map.ObjectMa
  * json.
  */
 public class JsonMetaDataFormatter implements MetaDataFormatter {
-    private static final Log LOG = LogFactory.getLog(JsonMetaDataFormatter.class);
+  private static final Log LOG = LogFactory.getLog(JsonMetaDataFormatter.class);
 
-    /**
-     * Convert the map to a JSON string.
-     */
-    private void asJson(OutputStream out, Map<String, Object> data)
-        throws HiveException
-    {
-        try {
-            new ObjectMapper().writeValue(out, data);
-        } catch (IOException e) {
-            throw new HiveException("Unable to convert to json", e);
-        }
+  /**
+   * Convert the map to a JSON string.
+   */
+  private void asJson(OutputStream out, Map<String, Object> data)
+      throws HiveException
+      {
+    try {
+      new ObjectMapper().writeValue(out, data);
+    } catch (IOException e) {
+      throw new HiveException("Unable to convert to json", e);
     }
+      }
 
-    /**
-     * Write an error message.
-     */
-    @Override
-    public void error(OutputStream out, String msg, int errorCode, String sqlState)
-        throws HiveException
-    {
-        error(out, msg, errorCode, sqlState, null);
-    }
-    @Override
-    public void error(OutputStream out, String errorMessage, int errorCode, String sqlState, String errorDetail) throws HiveException {
-        MapBuilder mb = MapBuilder.create().put("error", errorMessage);
-        if(errorDetail != null) {
-            mb.put("errorDetail", errorDetail);
-        }
-        mb.put("errorCode", errorCode);
-        if(sqlState != null) {
-          mb.put("sqlState", sqlState);
-        }
-        asJson(out,mb.build());
+  /**
+   * Write an error message.
+   */
+  @Override
+  public void error(OutputStream out, String msg, int errorCode, String sqlState)
+      throws HiveException
+      {
+    error(out, msg, errorCode, sqlState, null);
+      }
+  @Override
+  public void error(OutputStream out, String errorMessage, int errorCode, String sqlState, String errorDetail) throws HiveException {
+    MapBuilder mb = MapBuilder.create().put("error", errorMessage);
+    if(errorDetail != null) {
+      mb.put("errorDetail", errorDetail);
+    }
+    mb.put("errorCode", errorCode);
+    if(sqlState != null) {
+      mb.put("sqlState", sqlState);
+    }
+    asJson(out,mb.build());
+  }
+
+  /**
+   * Show a list of tables.
+   */
+  @Override
+  public void showTables(DataOutputStream out, Set<String> tables)
+      throws HiveException {
+    asJson(out, MapBuilder.create().put("tables", tables).build());
+  }
+
+  /**
+   * Describe table.
+   */
+  @Override
+  public void describeTable(DataOutputStream out, String colPath,
+      String tableName, Table tbl, Partition part, List<FieldSchema> cols,
+      boolean isFormatted, boolean isExt, boolean isPretty,
+      boolean isOutputPadded) throws HiveException {
+    MapBuilder builder = MapBuilder.create();
+    builder.put("columns", makeColsUnformatted(cols));
+
+    if (isExt) {
+      if (part != null) {
+        builder.put("partitionInfo", part.getTPartition());
+      }
+      else {
+        builder.put("tableInfo", tbl.getTTable());
+      }
     }
 
-    /**
-     * Show a list of tables.
-     */
-    @Override
-    public void showTables(DataOutputStream out, Set<String> tables)
-        throws HiveException
-    {
-        asJson(out,
-               MapBuilder.create()
-               .put("tables", tables)
-               .build());
-    }
-
-    /**
-     * Describe table.
-     */
-    @Override
-    public void describeTable(DataOutputStream out,
-                              String colPath, String tableName,
-                              Table tbl, Partition part, List<FieldSchema> cols,
-                              boolean isFormatted, boolean isExt,
-                              boolean isPretty)
-        throws HiveException
-    {
-        MapBuilder builder = MapBuilder.create();
-
-        builder.put("columns", makeColsUnformatted(cols));
-
-        if (isExt) {
-            if (part != null)
-                builder.put("partitionInfo", part.getTPartition());
-            else
-                builder.put("tableInfo", tbl.getTTable());
-        }
+    asJson(out, builder.build());
+  }
 
-        asJson(out, builder.build());
+  private List<Map<String, Object>> makeColsUnformatted(List<FieldSchema> cols) {
+    ArrayList<Map<String, Object>> res = new ArrayList<Map<String, Object>>();
+    for (FieldSchema col : cols) {
+      res.add(makeOneColUnformatted(col));
+    }
+    return res;
+  }
+
+  private Map<String, Object> makeOneColUnformatted(FieldSchema col) {
+    return MapBuilder.create()
+        .put("name", col.getName())
+        .put("type", col.getType())
+        .put("comment", col.getComment())
+        .build();
+  }
+
+  @Override
+  public void showTableStatus(DataOutputStream out, Hive db, HiveConf conf,
+      List<Table> tbls, Map<String, String> part, Partition par)
+          throws HiveException {
+    asJson(out, MapBuilder.create().put(
+        "tables", makeAllTableStatus(db, conf, tbls, part, par)).build());
+  }
+
+  private List<Map<String, Object>> makeAllTableStatus(Hive db, HiveConf conf,
+      List<Table> tbls, Map<String, String> part, Partition par)
+          throws HiveException {
+    try {
+      ArrayList<Map<String, Object>> res = new ArrayList<Map<String, Object>>();
+      for (Table tbl : tbls) {
+        res.add(makeOneTableStatus(tbl, db, conf, part, par));
+      }
+      return res;
+    } catch(IOException e) {
+      throw new HiveException(e);
+    }
+  }
+
+  private Map<String, Object> makeOneTableStatus(Table tbl, Hive db,
+      HiveConf conf, Map<String, String> part, Partition par)
+          throws HiveException, IOException {
+    String tblLoc = null;
+    String inputFormattCls = null;
+    String outputFormattCls = null;
+    if (part != null) {
+      if (par != null) {
+        if (par.getLocation() != null) {
+          tblLoc = par.getDataLocation().toString();
+        }
+        inputFormattCls = par.getInputFormatClass().getName();
+        outputFormattCls = par.getOutputFormatClass().getName();
+      }
+    } else {
+      if (tbl.getPath() != null) {
+        tblLoc = tbl.getDataLocation().toString();
+      }
+      inputFormattCls = tbl.getInputFormatClass().getName();
+      outputFormattCls = tbl.getOutputFormatClass().getName();
     }
 
-    private List<Map<String, Object>> makeColsUnformatted(List<FieldSchema> cols) {
-        ArrayList<Map<String, Object>> res = new ArrayList<Map<String, Object>>();
-        for (FieldSchema col : cols)
-            res.add(makeOneColUnformatted(col));
-        return res;
-    }
-
-    private Map<String, Object> makeOneColUnformatted(FieldSchema col) {
-        return MapBuilder.create()
-            .put("name", col.getName())
-            .put("type", col.getType())
-            .put("comment", col.getComment())
-            .build();
-    }
-
-    @Override
-    public void showTableStatus(DataOutputStream out,
-                                Hive db,
-                                HiveConf conf,
-                                List<Table> tbls,
-                                Map<String, String> part,
-                                Partition par)
-        throws HiveException
-    {
-        asJson(out, MapBuilder
-               .create()
-               .put("tables", makeAllTableStatus(db, conf,
-                                                 tbls, part, par))
-               .build());
-    }
-
-    private List<Map<String, Object>> makeAllTableStatus(Hive db,
-                                    HiveConf conf,
-                                    List<Table> tbls,
-                                    Map<String, String> part,
-                                    Partition par)
-        throws HiveException
-    {
-        try {
-            ArrayList<Map<String, Object>> res = new ArrayList<Map<String, Object>>();
-            for (Table tbl : tbls)
-                res.add(makeOneTableStatus(tbl, db, conf, part, par));
-            return res;
-        } catch(IOException e) {
-            throw new HiveException(e);
-        }
+    MapBuilder builder = MapBuilder.create();
+
+    builder.put("tableName", tbl.getTableName());
+    builder.put("owner", tbl.getOwner());
+    builder.put("location", tblLoc);
+    builder.put("inputFormat", inputFormattCls);
+    builder.put("outputFormat", outputFormattCls);
+    builder.put("columns", makeColsUnformatted(tbl.getCols()));
+
+    builder.put("partitioned", tbl.isPartitioned());
+    if (tbl.isPartitioned()) {
+      builder.put("partitionColumns", makeColsUnformatted(tbl.getPartCols()));
     }
 
-    private Map<String, Object> makeOneTableStatus(Table tbl,
-                                   Hive db,
-                                   HiveConf conf,
-                                   Map<String, String> part,
-                                   Partition par)
-        throws HiveException, IOException
-    {
-        String tblLoc = null;
-        String inputFormattCls = null;
-        String outputFormattCls = null;
-        if (part != null) {
-          if (par != null) {
-            if (par.getLocation() != null) {
-              tblLoc = par.getDataLocation().toString();
-            }
-            inputFormattCls = par.getInputFormatClass().getName();
-            outputFormattCls = par.getOutputFormatClass().getName();
-          }
-        } else {
-          if (tbl.getPath() != null) {
-            tblLoc = tbl.getDataLocation().toString();
-          }
-          inputFormattCls = tbl.getInputFormatClass().getName();
-          outputFormattCls = tbl.getOutputFormatClass().getName();
-        }
+    putFileSystemsStats(builder, makeTableStatusLocations(tbl, db, par),
+        conf, tbl.getPath());
 
-        MapBuilder builder = MapBuilder.create();
+    return builder.build();
+  }
 
-        builder.put("tableName", tbl.getTableName());
-        builder.put("owner", tbl.getOwner());
-        builder.put("location", tblLoc);
-        builder.put("inputFormat", inputFormattCls);
-        builder.put("outputFormat", outputFormattCls);
-        builder.put("columns", makeColsUnformatted(tbl.getCols()));
-
-        builder.put("partitioned", tbl.isPartitioned());
-        if (tbl.isPartitioned())
-            builder.put("partitionColumns", makeColsUnformatted(tbl.getPartCols()));
-
-        putFileSystemsStats(builder, makeTableStatusLocations(tbl, db, par),
-                            conf, tbl.getPath());
-
-        return builder.build();
-    }
-
-    private List<Path> makeTableStatusLocations(Table tbl, Hive db, Partition par)
-        throws HiveException
-    {
-        // output file system information
-        Path tblPath = tbl.getPath();
-        List<Path> locations = new ArrayList<Path>();
-        if (tbl.isPartitioned()) {
-          if (par == null) {
-            for (Partition curPart : db.getPartitions(tbl)) {
-              if (curPart.getLocation() != null) {
-                locations.add(new Path(curPart.getLocation()));
-              }
-            }
-          } else {
-            if (par.getLocation() != null) {
-              locations.add(new Path(par.getLocation()));
-            }
-          }
-        } else {
-          if (tblPath != null) {
-            locations.add(tblPath);
+  private List<Path> makeTableStatusLocations(Table tbl, Hive db, Partition par)
+      throws HiveException {
+    // output file system information
+    Path tblPath = tbl.getPath();
+    List<Path> locations = new ArrayList<Path>();
+    if (tbl.isPartitioned()) {
+      if (par == null) {
+        for (Partition curPart : db.getPartitions(tbl)) {
+          if (curPart.getLocation() != null) {
+            locations.add(new Path(curPart.getLocation()));
           }
         }
+      } else {
+        if (par.getLocation() != null) {
+          locations.add(new Path(par.getLocation()));
+        }
+      }
+    } else {
+      if (tblPath != null) {
+        locations.add(tblPath);
+      }
+    }
+
+    return locations;
+  }
 
-        return locations;
+  // Duplicates logic in TextMetaDataFormatter
+  private void putFileSystemsStats(MapBuilder builder, List<Path> locations,
+      HiveConf conf, Path tblPath)
+          throws IOException {
+    long totalFileSize = 0;
+    long maxFileSize = 0;
+    long minFileSize = Long.MAX_VALUE;
+    long lastAccessTime = 0;
+    long lastUpdateTime = 0;
+    int numOfFiles = 0;
+
+    boolean unknown = false;
+    FileSystem fs = tblPath.getFileSystem(conf);
+    // in case all files in locations do not exist
+    try {
+      FileStatus tmpStatus = fs.getFileStatus(tblPath);
+      lastAccessTime = tmpStatus.getAccessTime();
+      lastUpdateTime = tmpStatus.getModificationTime();
+    } catch (IOException e) {
+      LOG.warn(
+          "Cannot access File System. File System status will be unknown: ", e);
+      unknown = true;
     }
 
-    // Duplicates logic in TextMetaDataFormatter
-    private void putFileSystemsStats(MapBuilder builder, List<Path> locations,
-                                     HiveConf conf, Path tblPath)
-        throws IOException
-    {
-      long totalFileSize = 0;
-      long maxFileSize = 0;
-      long minFileSize = Long.MAX_VALUE;
-      long lastAccessTime = 0;
-      long lastUpdateTime = 0;
-      int numOfFiles = 0;
-
-      boolean unknown = false;
-      FileSystem fs = tblPath.getFileSystem(conf);
-      // in case all files in locations do not exist
-      try {
-        FileStatus tmpStatus = fs.getFileStatus(tblPath);
-        lastAccessTime = tmpStatus.getAccessTime();
-        lastUpdateTime = tmpStatus.getModificationTime();
-      } catch (IOException e) {
-        LOG.warn(
-            "Cannot access File System. File System status will be unknown: ", e);
-        unknown = true;
-      }
-
-      if (!unknown) {
-        for (Path loc : locations) {
-          try {
-            FileStatus status = fs.getFileStatus(tblPath);
-            FileStatus[] files = fs.listStatus(loc);
-            long accessTime = status.getAccessTime();
-            long updateTime = status.getModificationTime();
-            // no matter loc is the table location or part location, it must be a
-            // directory.
-            if (!status.isDir()) {
+    if (!unknown) {
+      for (Path loc : locations) {
+        try {
+          FileStatus status = fs.getFileStatus(tblPath);
+          FileStatus[] files = fs.listStatus(loc);
+          long accessTime = status.getAccessTime();
+          long updateTime = status.getModificationTime();
+          // no matter loc is the table location or part location, it must be a
+          // directory.
+          if (!status.isDir()) {
+            continue;
+          }
+          if (accessTime > lastAccessTime) {
+            lastAccessTime = accessTime;
+          }
+          if (updateTime > lastUpdateTime) {
+            lastUpdateTime = updateTime;
+          }
+          for (FileStatus currentStatus : files) {
+            if (currentStatus.isDir()) {
               continue;
             }
+            numOfFiles++;
+            long fileLen = currentStatus.getLen();
+            totalFileSize += fileLen;
+            if (fileLen > maxFileSize) {
+              maxFileSize = fileLen;
+            }
+            if (fileLen < minFileSize) {
+              minFileSize = fileLen;
+            }
+            accessTime = currentStatus.getAccessTime();
+            updateTime = currentStatus.getModificationTime();
             if (accessTime > lastAccessTime) {
               lastAccessTime = accessTime;
             }
             if (updateTime > lastUpdateTime) {
               lastUpdateTime = updateTime;
             }
-            for (FileStatus currentStatus : files) {
-              if (currentStatus.isDir()) {
-                continue;
-              }
-              numOfFiles++;
-              long fileLen = currentStatus.getLen();
-              totalFileSize += fileLen;
-              if (fileLen > maxFileSize) {
-                maxFileSize = fileLen;
-              }
-              if (fileLen < minFileSize) {
-                minFileSize = fileLen;
-              }
-              accessTime = currentStatus.getAccessTime();
-              updateTime = currentStatus.getModificationTime();
-              if (accessTime > lastAccessTime) {
-                lastAccessTime = accessTime;
-              }
-              if (updateTime > lastUpdateTime) {
-                lastUpdateTime = updateTime;
-              }
-            }
-          } catch (IOException e) {
-            // ignore
           }
+        } catch (IOException e) {
+          // ignore
         }
       }
-
-      builder
-          .put("totalNumberFiles", numOfFiles, ! unknown)
-          .put("totalFileSize",    totalFileSize, ! unknown)
-          .put("maxFileSize",      maxFileSize, ! unknown)
-          .put("minFileSize",      numOfFiles > 0 ? minFileSize : 0, ! unknown)
-          .put("lastAccessTime",   lastAccessTime, ! (unknown  || lastAccessTime < 0))
-          .put("lastUpdateTime",   lastUpdateTime, ! unknown);
-    }
-
-    /**
-     * Show the table partitions.
-     */
-    @Override
-    public void showTablePartitons(DataOutputStream out, List<String> parts)
-        throws HiveException
-    {
-        asJson(out,
-               MapBuilder.create()
-               .put("partitions", makeTablePartions(parts))
-               .build());
-    }
-
-    private List<Map<String, Object>> makeTablePartions(List<String> parts)
-        throws HiveException
-    {
-        try {
-            ArrayList<Map<String, Object>> res = new ArrayList<Map<String, Object>>();
-            for (String part : parts)
-                res.add(makeOneTablePartition(part));
-            return res;
-        } catch (UnsupportedEncodingException e) {
-            throw new HiveException(e);
-        }
     }
 
-    // This seems like a very wrong implementation.
-    private Map<String, Object> makeOneTablePartition(String partIdent)
-        throws UnsupportedEncodingException
-    {
-        ArrayList<Map<String, Object>> res = new ArrayList<Map<String, Object>>();
-
-        ArrayList<String> names = new ArrayList<String>();
-        for (String part : StringUtils.split(partIdent, "/")) {
-            String name = part;
-            String val = null;
-            String[] kv = StringUtils.split(part, "=", 2);
-            if (kv != null) {
-                name = kv[0];
-                if (kv.length > 1)
-                    val = URLDecoder.decode(kv[1], "UTF-8");
-            }
-            if (val != null)
-                names.add(name + "='" + val + "'");
-            else
-                names.add(name);
-
-            res.add(MapBuilder.create()
-                    .put("columnName", name)
-                    .put("columnValue", val)
-                    .build());
-        }
+    builder
+    .put("totalNumberFiles", numOfFiles, ! unknown)
+    .put("totalFileSize",    totalFileSize, ! unknown)
+    .put("maxFileSize",      maxFileSize, ! unknown)
+    .put("minFileSize",      numOfFiles > 0 ? minFileSize : 0, ! unknown)
+    .put("lastAccessTime",   lastAccessTime, ! (unknown  || lastAccessTime < 0))
+    .put("lastUpdateTime",   lastUpdateTime, ! unknown);
+  }
+
+  /**
+   * Show the table partitions.
+   */
+  @Override
+  public void showTablePartitons(DataOutputStream out, List<String> parts)
+      throws HiveException {
+    asJson(out, MapBuilder.create().put("partitions",
+        makeTablePartions(parts)).build());
+  }
 
-        return MapBuilder.create()
-            .put("name", StringUtils.join(names, ","))
-            .put("values", res)
-            .build();
-    }
-
-    /**
-     * Show a list of databases
-     */
-    @Override
-    public void showDatabases(DataOutputStream out, List<String> databases)
-        throws HiveException
-    {
-        asJson(out,
-               MapBuilder.create()
-               .put("databases", databases)
-               .build());
-    }
-
-    /**
-     * Show the description of a database
-     */
-    @Override
-    public void showDatabaseDescription(DataOutputStream out, String database, String comment,
-      String location, String ownerName, String ownerType, Map<String, String> params)
+  private List<Map<String, Object>> makeTablePartions(List<String> parts)
       throws HiveException {
-      MapBuilder builder = MapBuilder.create().put("database", database).put("comment", comment)
-        .put("location", location);
-      if (null != ownerName) {
-        builder.put("owner", ownerName);
+    try {
+      ArrayList<Map<String, Object>> res = new ArrayList<Map<String, Object>>();
+      for (String part : parts) {
+        res.add(makeOneTablePartition(part));
+      }
+      return res;
+    } catch (UnsupportedEncodingException e) {
+      throw new HiveException(e);
+    }
+  }
+
+  // This seems like a very wrong implementation.
+  private Map<String, Object> makeOneTablePartition(String partIdent)
+      throws UnsupportedEncodingException {
+    ArrayList<Map<String, Object>> res = new ArrayList<Map<String, Object>>();
+
+    ArrayList<String> names = new ArrayList<String>();
+    for (String part : StringUtils.split(partIdent, "/")) {
+      String name = part;
+      String val = null;
+      String[] kv = StringUtils.split(part, "=", 2);
+      if (kv != null) {
+        name = kv[0];
+        if (kv.length > 1)
+          val = URLDecoder.decode(kv[1], "UTF-8");
       }
-      if (null != ownerType) {
-        builder.put("ownerType", ownerType);
+      if (val != null) {
+        names.add(name + "='" + val + "'");
       }
-      if (null != params && !params.isEmpty()) {
-        builder.put("params", params);
+      else {
+        names.add(name);
       }
-      asJson(out, builder.build());
+
+      res.add(MapBuilder.create()
+          .put("columnName", name)
+          .put("columnValue", val)
+          .build());
+    }
+
+    return MapBuilder.create()
+        .put("name", StringUtils.join(names, ","))
+        .put("values", res)
+        .build();
+  }
+
+  /**
+   * Show a list of databases
+   */
+  @Override
+  public void showDatabases(DataOutputStream out, List<String> databases)
+      throws HiveException {
+    asJson(out, MapBuilder.create().put("databases", databases).build());
+  }
+
+  /**
+   * Show the description of a database
+   */
+  @Override
+  public void showDatabaseDescription(DataOutputStream out, String database, String comment,
+      String location, String ownerName, String ownerType, Map<String, String> params)
+          throws HiveException {
+    MapBuilder builder = MapBuilder.create().put("database", database).put("comment", comment)
+        .put("location", location);
+    if (null != ownerName) {
+      builder.put("owner", ownerName);
+    }
+    if (null != ownerType) {
+      builder.put("ownerType", ownerType);
+    }
+    if (null != params && !params.isEmpty()) {
+      builder.put("params", params);
     }
+    asJson(out, builder.build());
+  }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java?rev=1572141&r1=1572140&r2=1572141&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java Wed Feb 26 16:22:47 2014
@@ -63,40 +63,81 @@ public final class MetaDataFormatUtils {
     columnInformation.append(LINE_DELIM);
   }
 
+  /**
+   * Write formatted information about the given columns to a string
+   * @param cols - list of columns
+   * @param printHeader - if header should be included
+   * @param isOutputPadded - make it more human readable by setting indentation
+   *        with spaces. Turned off for use by HiveServer2
+   * @return string with formatted column information
+   */
   public static String getAllColumnsInformation(List<FieldSchema> cols,
-      boolean printHeader) {
+      boolean printHeader, boolean isOutputPadded) {
     StringBuilder columnInformation = new StringBuilder(DEFAULT_STRINGBUILDER_SIZE);
     if(printHeader){
       formatColumnsHeader(columnInformation);
     }
-    formatAllFields(columnInformation, cols);
+    formatAllFields(columnInformation, cols, isOutputPadded);
     return columnInformation.toString();
   }
 
-  public static String getAllColumnsInformation(List<FieldSchema> cols, List<FieldSchema> partCols,
-      boolean printHeader) {
+  /**
+   * Write formatted information about the given columns, including partition
+   * columns to a string
+   * @param cols - list of columns
+   * @param partCols - list of partition columns
+   * @param printHeader - if header should be included
+   * @param isOutputPadded - make it more human readable by setting indentation
+   *        with spaces. Turned off for use by HiveServer2
+   * @return string with formatted column information
+   */
+  public static String getAllColumnsInformation(List<FieldSchema> cols,
+      List<FieldSchema> partCols, boolean printHeader, boolean isOutputPadded) {
     StringBuilder columnInformation = new StringBuilder(DEFAULT_STRINGBUILDER_SIZE);
     if(printHeader){
       formatColumnsHeader(columnInformation);
     }
-    formatAllFields(columnInformation, cols);
+    formatAllFields(columnInformation, cols, isOutputPadded);
 
     if ((partCols != null) && (!partCols.isEmpty())) {
       columnInformation.append(LINE_DELIM).append("# Partition Information")
-        .append(LINE_DELIM);
+      .append(LINE_DELIM);
       formatColumnsHeader(columnInformation);
-      formatAllFields(columnInformation, partCols);
+      formatAllFields(columnInformation, partCols, isOutputPadded);
     }
 
     return columnInformation.toString();
   }
 
-  private static void formatAllFields(StringBuilder tableInfo, List<FieldSchema> cols) {
+  /**
+   * Write formatted column information into given StringBuilder
+   * @param tableInfo - StringBuilder to append column information into
+   * @param cols - list of columns
+   * @param isOutputPadded - make it more human readable by setting indentation
+   *        with spaces. Turned off for use by HiveServer2
+   */
+  private static void formatAllFields(StringBuilder tableInfo,
+      List<FieldSchema> cols, boolean isOutputPadded) {
     for (FieldSchema col : cols) {
-      formatOutput(col.getName(), col.getType(), getComment(col), tableInfo);
+      if(isOutputPadded) {
+        formatWithIndentation(col.getName(), col.getType(), getComment(col), tableInfo);
+      }
+      else {
+        formatWithoutIndentation(col.getName(), col.getType(), col.getComment(), tableInfo);
+      }
     }
   }
 
+  private static void formatWithoutIndentation(String name, String type, String comment,
+      StringBuilder colBuffer) {
+    colBuffer.append(name);
+    colBuffer.append(FIELD_DELIM);
+    colBuffer.append(type);
+    colBuffer.append(FIELD_DELIM);
+    colBuffer.append(comment == null ? "" : comment);
+    colBuffer.append(LINE_DELIM);
+  }
+
   public static String getAllColumnsInformation(Index index) {
     StringBuilder indexInfo = new StringBuilder(DEFAULT_STRINGBUILDER_SIZE);
 
@@ -133,7 +174,7 @@ public final class MetaDataFormatUtils {
     formatOutput(indexColumns.toArray(new String[0]), indexInfo);
 
     return indexInfo.toString();
-}
+  }
 
   public static String getPartitionInformation(Partition part) {
     StringBuilder tableInfo = new StringBuilder(DEFAULT_STRINGBUILDER_SIZE);
@@ -176,7 +217,7 @@ public final class MetaDataFormatUtils {
   }
 
   private static void getStorageDescriptorInfo(StringBuilder tableInfo,
-                                               StorageDescriptor storageDesc) {
+      StorageDescriptor storageDesc) {
 
     formatOutput("SerDe Library:", storageDesc.getSerdeInfo().getSerializationLib(), tableInfo);
     formatOutput("InputFormat:", storageDesc.getInputFormat(), tableInfo);
@@ -293,13 +334,13 @@ public final class MetaDataFormatUtils {
   }
 
   private static void formatOutput(String name, String value,
-                                   StringBuilder tableInfo) {
+      StringBuilder tableInfo) {
     tableInfo.append(String.format("%-" + ALIGNMENT + "s", name)).append(FIELD_DELIM);
     tableInfo.append(String.format("%-" + ALIGNMENT + "s", value)).append(LINE_DELIM);
   }
 
-  private static void formatOutput(String colName, String colType, String colComment,
-                                   StringBuilder tableInfo) {
+  private static void formatWithIndentation(String colName, String colType, String colComment,
+      StringBuilder tableInfo) {
     tableInfo.append(String.format("%-" + ALIGNMENT + "s", colName)).append(FIELD_DELIM);
     tableInfo.append(String.format("%-" + ALIGNMENT + "s", colType)).append(FIELD_DELIM);
 
@@ -313,7 +354,7 @@ public final class MetaDataFormatUtils {
     int colTypeLength = ALIGNMENT > colType.length() ? ALIGNMENT : colType.length();
     for (int i = 1; i < commentSegments.length; i++) {
       tableInfo.append(String.format("%" + colNameLength + "s" + FIELD_DELIM + "%"
-        + colTypeLength + "s" + FIELD_DELIM + "%s", "", "", commentSegments[i])).append(LINE_DELIM);
+          + colTypeLength + "s" + FIELD_DELIM + "%s", "", "", commentSegments[i])).append(LINE_DELIM);
     }
   }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java?rev=1572141&r1=1572140&r2=1572141&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatter.java Wed Feb 26 16:22:47 2014
@@ -23,13 +23,13 @@ import java.io.OutputStream;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
-import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
 
 /**
  * Interface to format table and index information.  We can format it
@@ -37,64 +37,75 @@ import org.apache.hadoop.hive.ql.session
  * (json).
  */
 public interface MetaDataFormatter {
-    /**
-     * Write an error message.
-     * @param sqlState if {@code null}, will be ignored
-     */
-    public void error(OutputStream out, String msg, int errorCode, String sqlState)
-        throws HiveException;
+  /**
+   * Write an error message.
+   * @param sqlState if {@code null}, will be ignored
+   */
+  public void error(OutputStream out, String msg, int errorCode, String sqlState)
+      throws HiveException;
 
   /**
    * @param sqlState if {@code null}, will be skipped in output
    * @param errorDetail usually string version of some Exception, if {@code null}, will be ignored
    */
-    public void error(OutputStream out, String errorMessage, int errorCode, String sqlState, String errorDetail)
+  public void error(OutputStream out, String errorMessage, int errorCode, String sqlState, String errorDetail)
+      throws HiveException;
+
+  /**
+   * Show a list of tables.
+   */
+  public void showTables(DataOutputStream out, Set<String> tables)
+      throws HiveException;
+
+  /**
+   * Describe table.
+   * @param out
+   * @param colPath
+   * @param tableName
+   * @param tbl
+   * @param part
+   * @param cols
+   * @param isFormatted - describe with formatted keyword
+   * @param isExt
+   * @param isPretty
+   * @param isOutputPadded - if true, add spacing and indentation
+   * @throws HiveException
+   */
+  public void describeTable(DataOutputStream out, String colPath,
+      String tableName, Table tbl, Partition part, List<FieldSchema> cols,
+      boolean isFormatted, boolean isExt, boolean isPretty,
+      boolean isOutputPadded)
           throws HiveException;
 
-    /**
-     * Show a list of tables.
-     */
-    public void showTables(DataOutputStream out, Set<String> tables)
-        throws HiveException;
-
-    /**
-     * Describe table.
-     */
-    public void describeTable(DataOutputStream out,
-                              String colPath, String tableName,
-                              Table tbl, Partition part, List<FieldSchema> cols,
-                              boolean isFormatted, boolean isExt, boolean isPretty)
-        throws HiveException;
-
-   /**
-     * Show the table status.
-     */
-    public void showTableStatus(DataOutputStream out,
-                                Hive db,
-                                HiveConf conf,
-                                List<Table> tbls,
-                                Map<String, String> part,
-                                Partition par)
-        throws HiveException;
-
-    /**
-     * Show the table partitions.
-     */
-    public void showTablePartitons(DataOutputStream out,
-                                   List<String> parts)
-        throws HiveException;
-
-    /**
-     * Show the databases
-     */
-    public void showDatabases(DataOutputStream out, List<String> databases)
-        throws HiveException;
-
-    /**
-     * Describe a database.
-     */
-    public void showDatabaseDescription (DataOutputStream out, String database, String comment,
+  /**
+   * Show the table status.
+   */
+  public void showTableStatus(DataOutputStream out,
+      Hive db,
+      HiveConf conf,
+      List<Table> tbls,
+      Map<String, String> part,
+      Partition par)
+          throws HiveException;
+
+  /**
+   * Show the table partitions.
+   */
+  public void showTablePartitons(DataOutputStream out,
+      List<String> parts)
+          throws HiveException;
+
+  /**
+   * Show the databases
+   */
+  public void showDatabases(DataOutputStream out, List<String> databases)
+      throws HiveException;
+
+  /**
+   * Describe a database.
+   */
+  public void showDatabaseDescription (DataOutputStream out, String database, String comment,
       String location, String ownerName, String ownerType, Map<String, String> params)
-     throws HiveException;
+          throws HiveException;
 }