Index: ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java (revision 1147261) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java (working copy) @@ -26,8 +26,10 @@ 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,10 +638,60 @@ 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 st = new Stack(); + 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 public void closeOp(boolean abort) throws HiveException {