Repository: kylin Updated Branches: refs/heads/yang21-hbase1.x 471c1e74e -> a3fd7d5e4 (forced update)
KYLIN-2159 Redistribution Hive Table Step always requires row_count filename as 000000_0 Project: http://git-wip-us.apache.org/repos/asf/kylin/repo Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/1720f1eb Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/1720f1eb Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/1720f1eb Branch: refs/heads/yang21-hbase1.x Commit: 1720f1eb8aea8bf6807b654558508b261d679bf7 Parents: d9bd93c Author: shaofengshi <shaofeng...@apache.org> Authored: Tue Nov 8 12:41:05 2016 +0800 Committer: shaofengshi <shaofeng...@apache.org> Committed: Tue Nov 8 12:42:34 2016 +0800 ---------------------------------------------------------------------- .../org/apache/kylin/source/hive/HiveMRInput.java | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kylin/blob/1720f1eb/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java ---------------------------------------------------------------------- diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java index 95828a0..52f474a 100644 --- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java +++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java @@ -24,6 +24,7 @@ import java.util.Set; import org.apache.commons.io.IOUtils; import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; @@ -310,10 +311,22 @@ public class HiveMRInput implements IMRInput { KylinConfig config = getCubeSpecificConfig(); try { - computeRowCount(config.getCliCommandExecutor()); + Path rowCountFile = null; + Path rowCountFolder = new Path(getRowCountOutputDir()); + FileSystem fs = FileSystem.get(rowCountFolder.toUri(), HadoopUtil.getCurrentConfiguration()); + for (FileStatus stat : fs.listStatus(rowCountFolder)) { + if (stat.isDirectory() == false && stat.getPath().getName().startsWith("0000")) { + rowCountFile = stat.getPath(); + logger.debug("Finding file " + rowCountFile); + break; + } + } + + if (rowCountFile == null) { + return new ExecuteResult(ExecuteResult.State.ERROR, "No row count file found in '" + getRowCountOutputDir() + "'"); + } - Path rowCountFile = new Path(getRowCountOutputDir(), "000000_0"); long rowCount = readRowCountFromFile(rowCountFile); if (!config.isEmptySegmentAllowed() && rowCount == 0) { stepLogger.log("Detect upstream hive table is empty, " + "fail the job because \"kylin.job.allow.empty.segment\" = \"false\"");