KYLIN-1323 further fix on read-write separation
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/aa7558ab Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/aa7558ab Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/aa7558ab Branch: refs/heads/master Commit: aa7558ab7b53f94b1b926035eba7c66cabbd5779 Parents: c96f827 Author: Li Yang <liy...@apache.org> Authored: Fri May 20 12:37:39 2016 +0800 Committer: Li Yang <liy...@apache.org> Committed: Fri May 20 12:37:39 2016 +0800 ---------------------------------------------------------------------- .../kylin/storage/hbase/steps/CreateHTableJob.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kylin/blob/aa7558ab/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java ---------------------------------------------------------------------- diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java index 29bcb62..7a32e37 100644 --- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java +++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CreateHTableJob.java @@ -30,7 +30,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.io.*; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.ToolRunner; @@ -44,7 +48,6 @@ import org.apache.kylin.cube.CubeManager; import org.apache.kylin.cube.CubeSegment; import org.apache.kylin.cube.kv.RowConstants; import org.apache.kylin.cube.model.CubeDesc; -import org.apache.kylin.engine.mr.HadoopUtil; import org.apache.kylin.engine.mr.common.AbstractHadoopJob; import org.apache.kylin.engine.mr.common.CubeStatsReader; import org.apache.kylin.engine.mr.common.CuboidShardUtil; @@ -280,7 +283,9 @@ public class CreateHTableJob extends AbstractHadoopJob { return; } - FileSystem fs = FileSystem.get(HBaseConnection.getCurrentHBaseConfiguration()); + // note read-write separation, respect HBase FS here + Configuration hbaseConf = HBaseConnection.getCurrentHBaseConfiguration(); + FileSystem fs = FileSystem.get(hbaseConf); if (fs.exists(outputFolder) == false) { fs.mkdirs(outputFolder); } @@ -294,7 +299,7 @@ public class CreateHTableJob extends AbstractHadoopJob { if (hfileSizeMB > 0.0 && kylinConfig.isDevEnv()) { hfileSizeMB = mbPerRegion / 2; } - int compactionThreshold = Integer.valueOf(HBaseConnection.getCurrentHBaseConfiguration().get("hbase.hstore.compactionThreshold", "3")); + int compactionThreshold = Integer.valueOf(hbaseConf.get("hbase.hstore.compactionThreshold", "3")); logger.info("hbase.hstore.compactionThreshold is " + compactionThreshold); if (hfileSizeMB > 0.0 && hfileSizeMB * compactionThreshold < mbPerRegion) { hfileSizeMB = mbPerRegion / compactionThreshold; @@ -338,8 +343,7 @@ public class CreateHTableJob extends AbstractHadoopJob { } - Configuration conf = HadoopUtil.getCurrentConfiguration(); - SequenceFile.Writer hfilePartitionWriter = SequenceFile.createWriter(conf, SequenceFile.Writer.file(hfilePartitionFile), SequenceFile.Writer.keyClass(ImmutableBytesWritable.class), SequenceFile.Writer.valueClass(NullWritable.class)); + SequenceFile.Writer hfilePartitionWriter = SequenceFile.createWriter(hbaseConf, SequenceFile.Writer.file(hfilePartitionFile), SequenceFile.Writer.keyClass(ImmutableBytesWritable.class), SequenceFile.Writer.valueClass(NullWritable.class)); for (int i = 0; i < splits.size(); i++) { hfilePartitionWriter.append(new ImmutableBytesWritable(splits.get(i)), NullWritable.get());