You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2011/07/16 08:43:21 UTC
svn commit: r1147365 -
/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
Author: namit
Date: Sat Jul 16 06:43:21 2011
New Revision: 1147365
URL: http://svn.apache.org/viewvc?rev=1147365&view=rev
Log:
HIVE-2252 Display a sample of partitions created when Fatal Error
occurred due to too many partitioned created (Ning Zhang via namit)
Modified:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java?rev=1147365&r1=1147364&r2=1147365&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java Sat Jul 16 06:43:21 2011
@@ -26,8 +26,10 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.Stack;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.common.FileUtils;
@@ -636,8 +638,58 @@ public class FileSinkOperator extends Te
protected void fatalErrorMessage(StringBuilder errMsg, long counterCode) {
errMsg.append("Operator ").append(getOperatorId()).append(" (id=").append(id).append("): ");
errMsg.append(counterCode > FATAL_ERR_MSG.length - 1 ?
- "fatal error" :
+ "fatal error." :
FATAL_ERR_MSG[(int) counterCode]);
+ // number of partitions exceeds limit, list all the partition names
+ if (counterCode > 0) {
+ errMsg.append(lsDir());
+ }
+ }
+
+ // sample the partitions that are generated so that users have a sense of what's causing the error
+ private String lsDir() {
+ String specPath = conf.getDirName();
+ // need to get a JobConf here because it's not passed through at client side
+ JobConf jobConf = new JobConf(ExecDriver.class);
+ Path tmpPath = Utilities.toTempPath(specPath);
+ StringBuilder sb = new StringBuilder("\n");
+ try {
+ DynamicPartitionCtx dpCtx = conf.getDynPartCtx();
+ int numDP = dpCtx.getNumDPCols();
+ FileSystem fs = tmpPath.getFileSystem(jobConf);
+ FileStatus[] status = Utilities.getFileStatusRecurse(tmpPath, numDP, fs);
+ sb.append("Sample of ")
+ .append(Math.min(status.length, 100))
+ .append(" partitions created under ")
+ .append(tmpPath.toString())
+ .append(":\n");
+ for (int i = 0; i < status.length; ++i) {
+ sb.append("\t.../");
+ sb.append(getPartitionSpec(status[i].getPath(), numDP))
+ .append("\n");
+ }
+ sb.append("...\n");
+ } catch (Exception e) {
+ // cannot get the subdirectories, just return the root directory
+ sb.append(tmpPath).append("...\n").append(e.getMessage());
+ e.printStackTrace();
+ } finally {
+ return sb.toString();
+ }
+ }
+
+ private String getPartitionSpec(Path path, int level) {
+ Stack<String> st = new Stack<String>();
+ Path p = path;
+ for (int i = 0; i < level; ++i) {
+ st.push(p.getName());
+ p = p.getParent();
+ }
+ StringBuilder sb = new StringBuilder();
+ while (!st.empty()) {
+ sb.append(st.pop());
+ }
+ return sb.toString();
}
@Override