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 [2/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...

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java?rev=1572141&r1=1572140&r2=1572141&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java Wed Feb 26 16:22:47 2014
@@ -48,416 +48,415 @@ import org.apache.hadoop.hive.ql.session
  * simple lines of text.
  */
 class TextMetaDataFormatter implements MetaDataFormatter {
-    private static final Log LOG = LogFactory.getLog(TextMetaDataFormatter.class);
+  private static final Log LOG = LogFactory.getLog(TextMetaDataFormatter.class);
 
-    private static final int separator = Utilities.tabCode;
-    private static final int terminator = Utilities.newLineCode;
+  private static final int separator = Utilities.tabCode;
+  private static final int terminator = Utilities.newLineCode;
 
-    /** The number of columns to be used in pretty formatting metadata output.
-     * If -1, then the current terminal width is auto-detected and used.
-     */
-    private final int prettyOutputNumCols;
+  /** The number of columns to be used in pretty formatting metadata output.
+   * If -1, then the current terminal width is auto-detected and used.
+   */
+  private final int prettyOutputNumCols;
 
-    public TextMetaDataFormatter(int prettyOutputNumCols) {
-      this.prettyOutputNumCols = prettyOutputNumCols;
-    }
+  public TextMetaDataFormatter(int prettyOutputNumCols) {
+    this.prettyOutputNumCols = prettyOutputNumCols;
+  }
 
-    /**
-     * Write an error message.
-     */
-    @Override
-    public void error(OutputStream out, String msg, int errorCode, String sqlState)
-        throws HiveException
-    {
-        error(out, msg, errorCode, sqlState, null);
-    }
+  /**
+   * 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
-    {
-      try {
-        out.write(errorMessage.getBytes("UTF-8"));
-        if(errorDetail != null) {
-          out.write(errorDetail.getBytes("UTF-8"));
-        }
-        out.write(errorCode);
-        if(sqlState != null) {
-          out.write(sqlState.getBytes("UTF-8"));//this breaks all the tests in .q files
-        }
-        out.write(terminator);
-      } catch (Exception e) {
-          throw new HiveException(e);
-        }
+  @Override
+  public void error(OutputStream out, String errorMessage, int errorCode, String sqlState, String errorDetail)
+      throws HiveException
+      {
+    try {
+      out.write(errorMessage.getBytes("UTF-8"));
+      if(errorDetail != null) {
+        out.write(errorDetail.getBytes("UTF-8"));
+      }
+      out.write(errorCode);
+      if(sqlState != null) {
+        out.write(sqlState.getBytes("UTF-8"));//this breaks all the tests in .q files
+      }
+      out.write(terminator);
+    } catch (Exception e) {
+      throw new HiveException(e);
     }
-    /**
-     * Show a list of tables.
-     */
-    @Override
-    public void showTables(DataOutputStream out, Set<String> tables)
-        throws HiveException
-    {
-        Iterator<String> iterTbls = tables.iterator();
-
-        try {
-            while (iterTbls.hasNext()) {
-                // create a row per table name
-                out.writeBytes(iterTbls.next());
-                out.write(terminator);
-            }
-        } catch (IOException e) {
-           throw new HiveException(e);
-        }
+      }
+  /**
+   * Show a list of tables.
+   */
+  @Override
+  public void showTables(DataOutputStream out, Set<String> tables)
+      throws HiveException
+      {
+    Iterator<String> iterTbls = tables.iterator();
+
+    try {
+      while (iterTbls.hasNext()) {
+        // create a row per table name
+        out.writeBytes(iterTbls.next());
+        out.write(terminator);
+      }
+    } catch (IOException e) {
+      throw new HiveException(e);
     }
+      }
 
-    @Override
-    public void describeTable(DataOutputStream outStream,
-                              String colPath, String tableName,
-                              Table tbl, Partition part, List<FieldSchema> cols,
-                              boolean isFormatted, boolean isExt, boolean isPretty)
-         throws HiveException {
-        try {
-          String output;
-          if (colPath.equals(tableName)) {
-            List<FieldSchema> partCols = tbl.isPartitioned() ? tbl.getPartCols() : null;
-            output = isPretty ?
-                MetaDataPrettyFormatUtils.getAllColumnsInformation(
-                    cols, partCols, prettyOutputNumCols)
+  @Override
+  public void describeTable(DataOutputStream outStream,  String colPath,
+      String tableName, Table tbl, Partition part, List<FieldSchema> cols,
+      boolean isFormatted, boolean isExt, boolean isPretty,
+      boolean isOutputPadded) throws HiveException {
+    try {
+      String output;
+      if (colPath.equals(tableName)) {
+        List<FieldSchema> partCols = tbl.isPartitioned() ? tbl.getPartCols() : null;
+        output = isPretty ?
+            MetaDataPrettyFormatUtils.getAllColumnsInformation(
+                cols, partCols, prettyOutputNumCols)
                 :
-                MetaDataFormatUtils.getAllColumnsInformation(cols, partCols, isFormatted);
+                  MetaDataFormatUtils.getAllColumnsInformation(cols, partCols, isFormatted, isOutputPadded);
+      } else {
+        output = MetaDataFormatUtils.getAllColumnsInformation(cols, isFormatted, isOutputPadded);
+      }
+      outStream.write(output.getBytes("UTF-8"));
+
+      if (tableName.equals(colPath)) {
+        if (isFormatted) {
+          if (part != null) {
+            output = MetaDataFormatUtils.getPartitionInformation(part);
           } else {
-            output = MetaDataFormatUtils.getAllColumnsInformation(cols, isFormatted);
+            output = MetaDataFormatUtils.getTableInformation(tbl);
           }
           outStream.write(output.getBytes("UTF-8"));
+        }
 
-          if (tableName.equals(colPath)) {
-            if (isFormatted) {
-              if (part != null) {
-                output = MetaDataFormatUtils.getPartitionInformation(part);
-              } else {
-                output = MetaDataFormatUtils.getTableInformation(tbl);
-              }
-              outStream.write(output.getBytes("UTF-8"));
-            }
-
-          // if extended desc table then show the complete details of the table
-            if (isExt) {
-              // add empty line
-              outStream.write(terminator);
-              if (part != null) {
-                // show partition information
-                outStream.writeBytes("Detailed Partition Information");
-                outStream.write(separator);
-                outStream.write(part.getTPartition().toString().getBytes("UTF-8"));
-                outStream.write(separator);
-                // comment column is empty
-                outStream.write(terminator);
-              } else {
-                // show table information
-                outStream.writeBytes("Detailed Table Information");
-                outStream.write(separator);
-                outStream.write(tbl.getTTable().toString().getBytes("UTF-8"));
-                outStream.write(separator);
-                outStream.write(terminator);
-              }
-            }
+        // if extended desc table then show the complete details of the table
+        if (isExt) {
+          // add empty line
+          outStream.write(terminator);
+          if (part != null) {
+            // show partition information
+            outStream.writeBytes("Detailed Partition Information");
+            outStream.write(separator);
+            outStream.write(part.getTPartition().toString().getBytes("UTF-8"));
+            outStream.write(separator);
+            // comment column is empty
+            outStream.write(terminator);
+          } else {
+            // show table information
+            outStream.writeBytes("Detailed Table Information");
+            outStream.write(separator);
+            outStream.write(tbl.getTTable().toString().getBytes("UTF-8"));
+            outStream.write(separator);
+            outStream.write(terminator);
           }
-        } catch (IOException e) {
-          throw new HiveException(e);
         }
+      }
+    } catch (IOException e) {
+      throw new HiveException(e);
     }
+  }
 
-    @Override
-    public void showTableStatus(DataOutputStream outStream,
-                                Hive db,
-                                HiveConf conf,
-                                List<Table> tbls,
-                                Map<String, String> part,
-                                Partition par)
-        throws HiveException
-    {
-        try {
-            Iterator<Table> iterTables = tbls.iterator();
-            while (iterTables.hasNext()) {
-              // create a row per table name
-              Table tbl = iterTables.next();
-              String tableName = tbl.getTableName();
-              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();
-              }
-
-              String owner = tbl.getOwner();
-              List<FieldSchema> cols = tbl.getCols();
-              String ddlCols = MetaStoreUtils.getDDLFromFieldSchema("columns", cols);
-              boolean isPartitioned = tbl.isPartitioned();
-              String partitionCols = "";
-              if (isPartitioned) {
-                partitionCols = MetaStoreUtils.getDDLFromFieldSchema(
-                    "partition_columns", tbl.getPartCols());
-              }
+  @Override
+  public void showTableStatus(DataOutputStream outStream,
+      Hive db,
+      HiveConf conf,
+      List<Table> tbls,
+      Map<String, String> part,
+      Partition par)
+          throws HiveException
+          {
+    try {
+      Iterator<Table> iterTables = tbls.iterator();
+      while (iterTables.hasNext()) {
+        // create a row per table name
+        Table tbl = iterTables.next();
+        String tableName = tbl.getTableName();
+        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();
+        }
 
-              outStream.writeBytes("tableName:" + tableName);
-              outStream.write(terminator);
-              outStream.writeBytes("owner:" + owner);
-              outStream.write(terminator);
-              outStream.writeBytes("location:" + tblLoc);
-              outStream.write(terminator);
-              outStream.writeBytes("inputformat:" + inputFormattCls);
-              outStream.write(terminator);
-              outStream.writeBytes("outputformat:" + outputFormattCls);
-              outStream.write(terminator);
-              outStream.writeBytes("columns:" + ddlCols);
-              outStream.write(terminator);
-              outStream.writeBytes("partitioned:" + isPartitioned);
-              outStream.write(terminator);
-              outStream.writeBytes("partitionColumns:" + partitionCols);
-              outStream.write(terminator);
-              // output file system information
-              Path tblPath = tbl.getPath();
-              List<Path> locations = new ArrayList<Path>();
-              if (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);
-                }
+        String owner = tbl.getOwner();
+        List<FieldSchema> cols = tbl.getCols();
+        String ddlCols = MetaStoreUtils.getDDLFromFieldSchema("columns", cols);
+        boolean isPartitioned = tbl.isPartitioned();
+        String partitionCols = "";
+        if (isPartitioned) {
+          partitionCols = MetaStoreUtils.getDDLFromFieldSchema(
+              "partition_columns", tbl.getPartCols());
+        }
+
+        outStream.writeBytes("tableName:" + tableName);
+        outStream.write(terminator);
+        outStream.writeBytes("owner:" + owner);
+        outStream.write(terminator);
+        outStream.writeBytes("location:" + tblLoc);
+        outStream.write(terminator);
+        outStream.writeBytes("inputformat:" + inputFormattCls);
+        outStream.write(terminator);
+        outStream.writeBytes("outputformat:" + outputFormattCls);
+        outStream.write(terminator);
+        outStream.writeBytes("columns:" + ddlCols);
+        outStream.write(terminator);
+        outStream.writeBytes("partitioned:" + isPartitioned);
+        outStream.write(terminator);
+        outStream.writeBytes("partitionColumns:" + partitionCols);
+        outStream.write(terminator);
+        // output file system information
+        Path tblPath = tbl.getPath();
+        List<Path> locations = new ArrayList<Path>();
+        if (isPartitioned) {
+          if (par == null) {
+            for (Partition curPart : db.getPartitions(tbl)) {
+              if (curPart.getLocation() != null) {
+                locations.add(new Path(curPart.getLocation()));
               }
-              if (!locations.isEmpty()) {
-                writeFileSystemStats(outStream, conf, locations, tblPath, false, 0);
-              }
-
-              outStream.write(terminator);
             }
-        } catch (IOException e) {
-            throw new HiveException(e);
+          } else {
+            if (par.getLocation() != null) {
+              locations.add(new Path(par.getLocation()));
+            }
+          }
+        } else {
+          if (tblPath != null) {
+            locations.add(tblPath);
+          }
+        }
+        if (!locations.isEmpty()) {
+          writeFileSystemStats(outStream, conf, locations, tblPath, false, 0);
         }
+
+        outStream.write(terminator);
+      }
+    } catch (IOException e) {
+      throw new HiveException(e);
     }
+          }
 
-    private void writeFileSystemStats(DataOutputStream outStream,
-                                      HiveConf conf,
-                                      List<Path> locations,
-                                      Path tblPath, boolean partSpecified, int indent)
-        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();
-        if (partSpecified) {
-          // check whether the part exists or not in fs
-          tmpStatus = fs.getFileStatus(locations.get(0));
-        }
-      } 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()) {
+  private void writeFileSystemStats(DataOutputStream outStream,
+      HiveConf conf,
+      List<Path> locations,
+      Path tblPath, boolean partSpecified, int indent)
+          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();
+      if (partSpecified) {
+        // check whether the part exists or not in fs
+        tmpStatus = fs.getFileStatus(locations.get(0));
+      }
+    } 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()) {
+            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
         }
       }
-      String unknownString = "unknown";
+    }
+    String unknownString = "unknown";
 
-      for (int k = 0; k < indent; k++) {
-        outStream.writeBytes(Utilities.INDENT);
-      }
-      outStream.writeBytes("totalNumberFiles:");
-      outStream.writeBytes(unknown ? unknownString : "" + numOfFiles);
-      outStream.write(terminator);
+    for (int k = 0; k < indent; k++) {
+      outStream.writeBytes(Utilities.INDENT);
+    }
+    outStream.writeBytes("totalNumberFiles:");
+    outStream.writeBytes(unknown ? unknownString : "" + numOfFiles);
+    outStream.write(terminator);
 
-      for (int k = 0; k < indent; k++) {
-        outStream.writeBytes(Utilities.INDENT);
-      }
-      outStream.writeBytes("totalFileSize:");
-      outStream.writeBytes(unknown ? unknownString : "" + totalFileSize);
-      outStream.write(terminator);
+    for (int k = 0; k < indent; k++) {
+      outStream.writeBytes(Utilities.INDENT);
+    }
+    outStream.writeBytes("totalFileSize:");
+    outStream.writeBytes(unknown ? unknownString : "" + totalFileSize);
+    outStream.write(terminator);
 
-      for (int k = 0; k < indent; k++) {
-        outStream.writeBytes(Utilities.INDENT);
-      }
-      outStream.writeBytes("maxFileSize:");
-      outStream.writeBytes(unknown ? unknownString : "" + maxFileSize);
-      outStream.write(terminator);
+    for (int k = 0; k < indent; k++) {
+      outStream.writeBytes(Utilities.INDENT);
+    }
+    outStream.writeBytes("maxFileSize:");
+    outStream.writeBytes(unknown ? unknownString : "" + maxFileSize);
+    outStream.write(terminator);
 
-      for (int k = 0; k < indent; k++) {
-        outStream.writeBytes(Utilities.INDENT);
-      }
-      outStream.writeBytes("minFileSize:");
-      if (numOfFiles > 0) {
-        outStream.writeBytes(unknown ? unknownString : "" + minFileSize);
-      } else {
-        outStream.writeBytes(unknown ? unknownString : "" + 0);
-      }
-      outStream.write(terminator);
+    for (int k = 0; k < indent; k++) {
+      outStream.writeBytes(Utilities.INDENT);
+    }
+    outStream.writeBytes("minFileSize:");
+    if (numOfFiles > 0) {
+      outStream.writeBytes(unknown ? unknownString : "" + minFileSize);
+    } else {
+      outStream.writeBytes(unknown ? unknownString : "" + 0);
+    }
+    outStream.write(terminator);
 
-      for (int k = 0; k < indent; k++) {
-        outStream.writeBytes(Utilities.INDENT);
-      }
-      outStream.writeBytes("lastAccessTime:");
-      outStream.writeBytes((unknown || lastAccessTime < 0) ? unknownString : ""
-          + lastAccessTime);
-      outStream.write(terminator);
+    for (int k = 0; k < indent; k++) {
+      outStream.writeBytes(Utilities.INDENT);
+    }
+    outStream.writeBytes("lastAccessTime:");
+    outStream.writeBytes((unknown || lastAccessTime < 0) ? unknownString : ""
+        + lastAccessTime);
+    outStream.write(terminator);
 
-      for (int k = 0; k < indent; k++) {
-        outStream.writeBytes(Utilities.INDENT);
-      }
-      outStream.writeBytes("lastUpdateTime:");
-      outStream.writeBytes(unknown ? unknownString : "" + lastUpdateTime);
-      outStream.write(terminator);
-  }
+    for (int k = 0; k < indent; k++) {
+      outStream.writeBytes(Utilities.INDENT);
+    }
+    outStream.writeBytes("lastUpdateTime:");
+    outStream.writeBytes(unknown ? unknownString : "" + lastUpdateTime);
+    outStream.write(terminator);
+          }
 
-    /**
-     * Show the table partitions.
-     */
-    @Override
-    public void showTablePartitons(DataOutputStream outStream, List<String> parts)
-        throws HiveException
-    {
-        try {
-            for (String part : parts) {
-                // Partition names are URL encoded. We decode the names unless Hive
-                // is configured to use the encoded names.
-                SessionState ss = SessionState.get();
-                if (ss != null && ss.getConf() != null &&
-                      !ss.getConf().getBoolVar(HiveConf.ConfVars.HIVE_DECODE_PARTITION_NAME)) {
-                    outStream.writeBytes(part);
-                } else {
-                    outStream.writeBytes(FileUtils.unescapePathName(part));
-                }
-                outStream.write(terminator);
-            }
-        } catch (IOException e) {
-            throw new HiveException(e);
+  /**
+   * Show the table partitions.
+   */
+  @Override
+  public void showTablePartitons(DataOutputStream outStream, List<String> parts)
+      throws HiveException
+      {
+    try {
+      for (String part : parts) {
+        // Partition names are URL encoded. We decode the names unless Hive
+        // is configured to use the encoded names.
+        SessionState ss = SessionState.get();
+        if (ss != null && ss.getConf() != null &&
+            !ss.getConf().getBoolVar(HiveConf.ConfVars.HIVE_DECODE_PARTITION_NAME)) {
+          outStream.writeBytes(part);
+        } else {
+          outStream.writeBytes(FileUtils.unescapePathName(part));
         }
+        outStream.write(terminator);
+      }
+    } catch (IOException e) {
+      throw new HiveException(e);
     }
+      }
 
-    /**
-     * Show the list of databases
-     */
-    @Override
-    public void showDatabases(DataOutputStream outStream, List<String> databases)
-        throws HiveException
-        {
-        try {
-            for (String database : databases) {
-                // create a row per database name
-                outStream.writeBytes(database);
-                outStream.write(terminator);
-              }
-        } catch (IOException e) {
-            throw new HiveException(e);
-        }
+  /**
+   * Show the list of databases
+   */
+  @Override
+  public void showDatabases(DataOutputStream outStream, List<String> databases)
+      throws HiveException
+      {
+    try {
+      for (String database : databases) {
+        // create a row per database name
+        outStream.writeBytes(database);
+        outStream.write(terminator);
+      }
+    } catch (IOException e) {
+      throw new HiveException(e);
     }
+      }
 
-    /**
-     * Describe a database
-     */
-    @Override
-    public void showDatabaseDescription(DataOutputStream outStream, String database, String comment,
+  /**
+   * Describe a database
+   */
+  @Override
+  public void showDatabaseDescription(DataOutputStream outStream, String database, String comment,
       String location, String ownerName, String ownerType, Map<String, String> params)
-      throws HiveException {
-        try {
-            outStream.writeBytes(database);
-            outStream.write(separator);
-            if (comment != null) {
-              outStream.write(comment.getBytes("UTF-8"));
-            }
-            outStream.write(separator);
-            if (location != null) {
-              outStream.writeBytes(location);
-            }
-            outStream.write(separator);
-            if (ownerName != null) {
-              outStream.writeBytes(ownerName);
-            }
-            outStream.write(separator);
-            if (ownerType != null) {
-              outStream.writeBytes(ownerType);
-            }
-            outStream.write(separator);
-            if (params != null && !params.isEmpty()) {
-                outStream.writeBytes(params.toString());
-            }
-            outStream.write(terminator);
-        } catch (IOException e) {
-            throw new HiveException(e);
-        }
+          throws HiveException {
+    try {
+      outStream.writeBytes(database);
+      outStream.write(separator);
+      if (comment != null) {
+        outStream.write(comment.getBytes("UTF-8"));
+      }
+      outStream.write(separator);
+      if (location != null) {
+        outStream.writeBytes(location);
+      }
+      outStream.write(separator);
+      if (ownerName != null) {
+        outStream.writeBytes(ownerName);
+      }
+      outStream.write(separator);
+      if (ownerType != null) {
+        outStream.writeBytes(ownerType);
+      }
+      outStream.write(separator);
+      if (params != null && !params.isEmpty()) {
+        outStream.writeBytes(params.toString());
+      }
+      outStream.write(terminator);
+    } catch (IOException e) {
+      throw new HiveException(e);
     }
+  }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java?rev=1572141&r1=1572140&r2=1572141&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java Wed Feb 26 16:22:47 2014
@@ -88,6 +88,11 @@ public class SessionState {
    */
   protected boolean isVerbose;
 
+  /**
+   * Is the query served from HiveServer2
+   */
+  private boolean isHiveServerQuery = false;
+
   /*
    * HiveHistory Object
    */
@@ -193,6 +198,10 @@ public class SessionState {
     }
   }
 
+  public boolean isHiveServerQuery() {
+    return this.isHiveServerQuery;
+  }
+
   public void setIsSilent(boolean isSilent) {
     if(conf != null) {
       conf.setBoolVar(HiveConf.ConfVars.HIVESESSIONSILENT, isSilent);
@@ -208,6 +217,10 @@ public class SessionState {
     this.isVerbose = isVerbose;
   }
 
+  public void setIsHiveServerQuery(boolean isHiveServerQuery) {
+    this.isHiveServerQuery = isHiveServerQuery;
+  }
+
   public SessionState(HiveConf conf) {
     this(conf, null);
   }
@@ -330,7 +343,7 @@ public class SessionState {
         throw new RuntimeException(e);
       }
     } else {
-       LOG.info("No Tez session required at this point. hive.execution.engine=mr.");
+      LOG.info("No Tez session required at this point. hive.execution.engine=mr.");
     }
     return startSs;
   }
@@ -383,7 +396,7 @@ public class SessionState {
     if(LOG.isDebugEnabled()){
       Object authorizationClass = getAuthorizationMode() == AuthorizationMode.V1 ?
           getAuthorizer() : getAuthorizerV2();
-      LOG.debug("Session is using authorization class " + authorizationClass.getClass());
+          LOG.debug("Session is using authorization class " + authorizationClass.getClass());
     }
     return;
   }

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java?rev=1572141&r1=1572140&r2=1572141&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java Wed Feb 26 16:22:47 2014
@@ -85,6 +85,7 @@ public class HiveSessionImpl implements 
     this.sessionHandle = new SessionHandle(protocol);
     this.hiveConf = new HiveConf(serverhiveConf);
 
+    //set conf properties specified by user from client side
     if (sessionConfMap != null) {
       for (Map.Entry<String, String> entry : sessionConfMap.entrySet()) {
         hiveConf.set(entry.getKey(), entry.getValue());
@@ -98,6 +99,7 @@ public class HiveSessionImpl implements 
         FetchFormatter.ThriftFormatter.class.getName());
     hiveConf.setInt(ListSinkOperator.OUTPUT_PROTOCOL, protocol.getValue());
     sessionState = new SessionState(hiveConf, username);
+    sessionState.setIsHiveServerQuery(true);
     SessionState.start(sessionState);
   }
 
@@ -210,12 +212,12 @@ public class HiveSessionImpl implements 
 
   private OperationHandle executeStatementInternal(String statement, Map<String, String> confOverlay,
       boolean runAsync)
-      throws HiveSQLException {
+          throws HiveSQLException {
     acquire();
 
     OperationManager operationManager = getOperationManager();
     ExecuteStatementOperation operation = operationManager
-          .newExecuteStatementOperation(getSession(), statement, confOverlay, runAsync);
+        .newExecuteStatementOperation(getSession(), statement, confOverlay, runAsync);
     OperationHandle opHandle = operation.getHandle();
     try {
       operation.run();
@@ -297,7 +299,7 @@ public class HiveSessionImpl implements 
   @Override
   public OperationHandle getTables(String catalogName, String schemaName, String tableName,
       List<String> tableTypes)
-      throws HiveSQLException {
+          throws HiveSQLException {
     acquire();
 
     OperationManager operationManager = getOperationManager();
@@ -346,9 +348,9 @@ public class HiveSessionImpl implements 
         catalogName, schemaName, tableName, columnName);
     OperationHandle opHandle = operation.getHandle();
     try {
-    operation.run();
-    opHandleSet.add(opHandle);
-    return opHandle;
+      operation.run();
+      opHandleSet.add(opHandle);
+      return opHandle;
     } catch (HiveSQLException e) {
       operationManager.closeOperation(opHandle);
       throw e;