-
Notifications
You must be signed in to change notification settings - Fork 1.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KYLIN-4833 create hfile on hadoop hdfs and distcp to hbase hdfs #1494
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -27,6 +27,7 @@ | |
import org.apache.kylin.cube.CubeInstance; | ||
import org.apache.kylin.cube.CubeSegment; | ||
import org.apache.kylin.cube.cuboid.CuboidModeEnum; | ||
import org.apache.kylin.engine.mr.CubingJob; | ||
import org.apache.kylin.engine.mr.JobBuilderSupport; | ||
import org.apache.kylin.engine.mr.common.AbstractHadoopJob; | ||
import org.apache.kylin.engine.mr.common.BatchConstants; | ||
|
@@ -63,6 +64,13 @@ public HadoopShellExecutable createCreateHTableStep(String jobId, CuboidModeEnum | |
appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid()); | ||
appendExecCmdParameters(cmd, BatchConstants.ARG_PARTITION, | ||
getRowkeyDistributionOutputPath(jobId) + "/part-r-00000"); | ||
if(this.seg.getConfig().isHFileDistCP()){ | ||
String partitionOutputPath = getRealizationRootPath(jobId) + "/rowkey_stats/part-r-00000_hfile"; | ||
appendExecCmdParameters(cmd, BatchConstants.ARG_PARTITION, partitionOutputPath); | ||
}else { | ||
appendExecCmdParameters(cmd, BatchConstants.ARG_PARTITION, | ||
getRowkeyDistributionOutputPath(jobId) + "/part-r-00000"); | ||
} | ||
appendExecCmdParameters(cmd, BatchConstants.ARG_CUBOID_MODE, cuboidMode.toString()); | ||
appendExecCmdParameters(cmd, BatchConstants.ARG_HBASE_CONF_PATH, getHBaseConfFilePath(jobId)); | ||
|
||
|
@@ -72,6 +80,26 @@ public HadoopShellExecutable createCreateHTableStep(String jobId, CuboidModeEnum | |
return createHtableStep; | ||
} | ||
|
||
public AbstractExecutable createDistcpHFileStep(String jobId){ | ||
String inputPath = getRealizationRootPath(jobId) + "/hfile"; | ||
MapReduceExecutable createHFilesStep = new MapReduceExecutable(); | ||
createHFilesStep.setName(ExecutableConstants.STEP_NAME_HFILE_DISTCP); | ||
StringBuilder cmd = new StringBuilder(); | ||
|
||
appendMapReduceParameters(cmd); | ||
appendExecCmdParameters(cmd, BatchConstants.ARG_CUBE_NAME, seg.getRealization().getName()); | ||
// appendExecCmdParameters(cmd, BatchConstants.ARG_PARTITION, getRowkeyDistributionOutputPath(jobId) + "/part-r-00000_hfile"); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. remove commented code if not needed , will help in code readability going forward. |
||
appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, inputPath); | ||
appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, getHFilePath(jobId)); | ||
appendExecCmdParameters(cmd, BatchConstants.ARG_HTABLE_NAME, seg.getStorageLocationIdentifier()); | ||
appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME, "Kylin_HFile_Distcp_" + seg.getRealization().getName() + "_Step"); | ||
|
||
createHFilesStep.setMapReduceParams(cmd.toString()); | ||
createHFilesStep.setMapReduceJobClass(HFileDistcpJob.class); | ||
createHFilesStep.setCounterSaveAs(",," + CubingJob.CUBE_SIZE_BYTES); | ||
return createHFilesStep; | ||
} | ||
|
||
// TODO make it abstract | ||
public MapReduceExecutable createMergeCuboidDataStep(CubeSegment seg, List<CubeSegment> mergingSegments, | ||
String jobID, Class<? extends AbstractHadoopJob> clazz) { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -91,7 +91,8 @@ private void dropHdfsPathOnCluster(List<String> oldHdfsPaths, FileSystem fileSys | |
} | ||
// If hbase was deployed on another cluster, the job dir is empty and should be dropped, | ||
// because of rowkey_stats and hfile dirs are both dropped. | ||
if (fileSystem.listStatus(oldPath.getParent()).length == 0) { | ||
Path parentPath = oldPath.getParent(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. good to use final if variable is not mutated : final Path parentPath |
||
if (fileSystem.exists(parentPath) && fileSystem.listStatus(parentPath).length == 0) { | ||
Path emptyJobPath = new Path(JobBuilderSupport.getJobWorkingDir(config, getJobId())); | ||
emptyJobPath = Path.getPathWithoutSchemeAndAuthority(emptyJobPath); | ||
if (fileSystem.exists(emptyJobPath)) { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,106 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.kylin.storage.hbase.steps; | ||
|
||
import com.google.common.collect.Lists; | ||
import org.apache.commons.cli.Options; | ||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.FileSystem; | ||
import org.apache.hadoop.fs.Path; | ||
import org.apache.hadoop.mapred.JobContext; | ||
import org.apache.hadoop.tools.DistCp; | ||
import org.apache.hadoop.tools.DistCpOptions; | ||
import org.apache.hadoop.util.ToolRunner; | ||
import org.apache.kylin.common.KylinConfig; | ||
import org.apache.kylin.common.util.HadoopUtil; | ||
import org.apache.kylin.cube.CubeInstance; | ||
import org.apache.kylin.cube.CubeManager; | ||
import org.apache.kylin.engine.mr.common.AbstractHadoopJob; | ||
import org.apache.kylin.engine.mr.common.BatchConstants; | ||
import org.apache.kylin.storage.hbase.HBaseConnection; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.util.List; | ||
|
||
/** | ||
* @author fengpod | ||
*/ | ||
public class HFileDistcpJob extends AbstractHadoopJob { | ||
|
||
protected static final Logger logger = LoggerFactory.getLogger(HFileDistcpJob.class); | ||
|
||
public int run(String[] args) throws Exception { | ||
Options options = new Options(); | ||
|
||
try { | ||
options.addOption(OPTION_CUBE_NAME); | ||
options.addOption(OPTION_JOB_NAME); | ||
options.addOption(OPTION_INPUT_PATH); | ||
options.addOption(OPTION_OUTPUT_PATH); | ||
options.addOption(OPTION_HTABLE_NAME); | ||
parseOptions(options, args); | ||
|
||
// use current hbase configuration | ||
Configuration configuration = new Configuration(); | ||
HBaseConnection.addHBaseClusterNNHAConfiguration(configuration); | ||
|
||
Path input = new Path(getOptionValue(OPTION_INPUT_PATH)); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. good to change to final |
||
Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH)); | ||
FileSystem fs = HadoopUtil.getFileSystem(output, configuration); | ||
if (fs.exists(output) == false) { | ||
fs.mkdirs(output); | ||
} | ||
|
||
String cubeName = getOptionValue(OPTION_CUBE_NAME); | ||
CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()); | ||
CubeInstance cube = cubeMgr.getCube(cubeName); | ||
|
||
List<Path> sourceList = Lists.newArrayList(); | ||
sourceList.add(input); | ||
DistCpOptions distCpOptions = new DistCpOptions(sourceList, output); | ||
distCpOptions.setMapBandwidth(cube.getConfig().getDistCPMapBandWidth()); | ||
distCpOptions.setMaxMaps(cube.getConfig().getDistCPMaxMapNum()); | ||
distCpOptions.setOverwrite(true); | ||
distCpOptions.setBlocking(true); | ||
|
||
configuration.set(JobContext.JOB_NAME, getOptionValue(OPTION_JOB_NAME)); | ||
DistCp distCp = new DistCp(configuration, distCpOptions); | ||
|
||
job = distCp.execute(); | ||
|
||
// set job configuration | ||
job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName); | ||
// add metadata to distributed cache | ||
attachCubeMetadata(cube, job.getConfiguration()); | ||
} catch (Exception e){ | ||
throw e; | ||
}finally { | ||
if (job != null) | ||
cleanupTempConfFile(job.getConfiguration()); | ||
} | ||
return 0; | ||
} | ||
|
||
public static void main(String[] args) throws Exception { | ||
int exitCode = ToolRunner.run(new HFileDistcpJob(), args); | ||
System.exit(exitCode); | ||
} | ||
|
||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Append paramters twice for
BatchConstants.ARG_PARTITION
.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
new commit fixed this