KYLIN-1426: fix the bug: fail to locate kylin.properties during cardinality calculation
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/4656bd56 Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/4656bd56 Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/4656bd56 Branch: refs/heads/1.4-rc Commit: 4656bd5643e09c67ebf81564a630d4163d61664f Parents: 147986b Author: yangzhong <yangzh...@ebay.com> Authored: Thu Feb 18 09:44:18 2016 +0800 Committer: shaofengshi <shaofeng...@apache.org> Committed: Fri Feb 19 16:22:42 2016 +0800 ---------------------------------------------------------------------- .../main/java/org/apache/kylin/rest/service/CubeService.java | 7 ++++--- .../source/hive/cardinality/HiveColumnCardinalityJob.java | 6 ++++++ 2 files changed, 10 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kylin/blob/4656bd56/server/src/main/java/org/apache/kylin/rest/service/CubeService.java ---------------------------------------------------------------------- diff --git a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java index 3f894ec..b543ef7 100644 --- a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java +++ b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java @@ -41,6 +41,7 @@ import org.apache.kylin.engine.EngineFactory; import org.apache.kylin.engine.mr.CubingJob; import org.apache.kylin.engine.mr.HadoopUtil; import org.apache.kylin.engine.mr.common.HadoopShellExecutable; +import org.apache.kylin.engine.mr.common.MapReduceExecutable; import org.apache.kylin.job.exception.JobException; import org.apache.kylin.job.execution.DefaultChainedExecutable; import org.apache.kylin.job.execution.ExecutableState; @@ -499,10 +500,10 @@ public class CubeService extends BasicService { String outPath = HiveColumnCardinalityJob.OUTPUT_PATH + "/" + tableName; String param = "-table " + tableName + " -output " + outPath; - HadoopShellExecutable step1 = new HadoopShellExecutable(); + MapReduceExecutable step1 = new MapReduceExecutable(); - step1.setJobClass(HiveColumnCardinalityJob.class); - step1.setJobParams(param); + step1.setMapReduceJobClass(HiveColumnCardinalityJob.class); + step1.setMapReduceParams(param); job.addTask(step1); http://git-wip-us.apache.org/repos/asf/kylin/blob/4656bd56/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/HiveColumnCardinalityJob.java ---------------------------------------------------------------------- diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/HiveColumnCardinalityJob.java b/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/HiveColumnCardinalityJob.java index 2b35755..2b12273 100644 --- a/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/HiveColumnCardinalityJob.java +++ b/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/HiveColumnCardinalityJob.java @@ -29,10 +29,13 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.apache.kylin.common.KylinConfig; import org.apache.kylin.engine.mr.IMRInput.IMRTableInputFormat; import org.apache.kylin.engine.mr.MRUtil; import org.apache.kylin.engine.mr.common.AbstractHadoopJob; import org.apache.kylin.engine.mr.common.BatchConstants; +import org.apache.kylin.metadata.MetadataManager; +import org.apache.kylin.metadata.model.TableDesc; /** * This hadoop job will scan all rows of the hive table and then calculate the cardinality on each column. @@ -94,6 +97,9 @@ public class HiveColumnCardinalityJob extends AbstractHadoopJob { this.deletePath(job.getConfiguration(), output); logger.info("Going to submit HiveColumnCardinalityJob for table '" + table + "'"); + + TableDesc tableDesc = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(table); + attachKylinPropsAndMetadata(tableDesc, job.getConfiguration()); int result = waitForCompletion(job); return result;