zwangsheng commented on code in PR #1830:
URL:
https://github.com/apache/incubator-celeborn/pull/1830#discussion_r1309573072
##########
build/make-distribution.sh:
##########
@@ -218,22 +218,43 @@ function build_flink_client {
cp
"$PROJECT_DIR"/client-flink/flink-$FLINK_BINARY_VERSION-shaded/target/celeborn-client-flink-${FLINK_BINARY_VERSION}-shaded_$SCALA_VERSION-$VERSION.jar
"$DIST_DIR/flink/"
}
+function build_mr_client {
+ HADOOP_VERSION=$("$MVN" help:evaluate -Dexpression=hadoop.version $@
2>/dev/null \
+ | grep -v "INFO" \
+ | grep -v "WARNING" \
+ | tail -n 1)
+ BUILD_COMMAND=("$MVN" clean package $MVN_DIST_OPT -pl
:celeborn-client-mr-shaded-${HADOOP_VERSION} -am $@)
+
+ # Actually build the jar
+ echo -e "\nBuilding with..."
+ echo -e "\$ ${BUILD_COMMAND[@]}\n"
+
+ "${BUILD_COMMAND[@]}"
+
+ ## flink spark client jars
+ mkdir -p "$DIST_DIR/mr"
+ cp
"$PROJECT_DIR"/client-mr/mr-shaded/target/celeborn-client-mr-shaded-${HADOOP_VERSION}-$VERSION.jar
"$DIST_DIR/mr/"
+}
+
if [ "$RELEASE" == "true" ]; then
build_service
build_spark_client -Pspark-2.4
build_spark_client -Pspark-3.4
build_flink_client -Pflink-1.14
build_flink_client -Pflink-1.15
build_flink_client -Pflink-1.17
+ build_mr_client mr
else
## build release package on demand
build_service $@
- if [[ $@ != *"spark"* && $@ != *"flink"* ]]; then
+ if [[ $@ != *"spark"* && $@ != *"flink"* && $@ != *"mr"* ]]; then
echo "Skip building client."
elif [[ $@ == *"spark"* && $@ != *"flink"* ]]; then
build_spark_client $@
elif [[ $@ == *"flink"* && $@ != *"spark"* ]]; then
build_flink_client $@
+ elif [[ $@ == *"mr"* ]]; then
Review Comment:
If we build with `-Pspark-3.4 -Pmr`, will only build spark client
##########
client-mr/mr/src/main/java/org/apache/celeborn/mapreduce/v2/app/MRAppMasterWithCeleborn.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.celeborn.mapreduce.v2.app;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.ipc.CallerContext;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.JobSubmissionFiles;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.v2.app.MRAppMaster;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+import org.apache.hadoop.mapreduce.v2.util.MRWebAppUtil;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.client.LifecycleManager;
+import org.apache.celeborn.common.CelebornConf;
+import org.apache.celeborn.common.exception.CelebornIOException;
+import org.apache.celeborn.util.HadoopUtils;
+
+public class MRAppMasterWithCeleborn extends MRAppMaster {
+ private static final Logger logger =
LoggerFactory.getLogger(MRAppMasterWithCeleborn.class);
+
+ public MRAppMasterWithCeleborn(
+ ApplicationAttemptId applicationAttemptId,
+ ContainerId containerId,
+ String nmHost,
+ int nmPort,
+ int nmHttpPort,
+ long appSubmitTime,
+ JobConf jobConf)
+ throws CelebornIOException {
+ super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort,
appSubmitTime);
+
+ int numReducers = jobConf.getInt(MRJobConfig.NUM_REDUCES, 0);
+ if (numReducers > 0) {
+ CelebornConf conf = HadoopUtils.fromYarnConf(jobConf);
+ LifecycleManager lifecycleManager =
+ new LifecycleManager(applicationAttemptId.toString(), conf);
+ String lcHost = lifecycleManager.getHost();
+ int lcPort = lifecycleManager.getPort();
+ logger.info("RMAppMaster initialized with {} {} {}", lcHost, lcPort,
applicationAttemptId);
+ JobConf lcConf = new JobConf();
+ lcConf.clear();
+ lcConf.set(HadoopUtils.MR_CELEBORN_LC_HOST, lcHost);
+ lcConf.set(HadoopUtils.MR_CELEBORN_LC_PORT, lcPort + "");
+ lcConf.set(HadoopUtils.MR_CELEBORN_APPLICATION_ID,
applicationAttemptId.toString());
+ writeLifecycleManagerConfToTask(jobConf, lcConf);
+ }
+ }
+
+ void writeLifecycleManagerConfToTask(JobConf conf, JobConf lcConf) throws
CelebornIOException {
+ try {
+ FileSystem fs = new Cluster(conf).getFileSystem();
+ String jobDirStr = conf.get(MRJobConfig.MAPREDUCE_JOB_DIR);
+ Path celebornExtraConf = new Path(jobDirStr,
HadoopUtils.MR_CELEBORN_CONF);
+
+ try (FSDataOutputStream out =
+ FileSystem.create(
+ fs, celebornExtraConf, new
FsPermission(JobSubmissionFiles.JOB_FILE_PERMISSION))) {
+ lcConf.writeXml(out);
+ }
+ FileStatus status = fs.getFileStatus(celebornExtraConf);
+ long currentTs = status.getModificationTime();
+ String uri = fs.getUri() + Path.SEPARATOR + celebornExtraConf.toUri();
+ String files = conf.get(MRJobConfig.CACHE_FILES);
+ conf.set(MRJobConfig.CACHE_FILES, files == null ? uri : uri + "," +
files);
+ String ts = conf.get(MRJobConfig.CACHE_FILE_TIMESTAMPS);
+ conf.set(
+ MRJobConfig.CACHE_FILE_TIMESTAMPS,
+ ts == null ? String.valueOf(currentTs) : currentTs + "," + ts);
+ String vis = conf.get(MRJobConfig.CACHE_FILE_VISIBILITIES);
+ conf.set(MRJobConfig.CACHE_FILE_VISIBILITIES, vis == null ? "false" :
"false" + "," + vis);
+ long size = status.getLen();
+ String sizes = conf.get(MRJobConfig.CACHE_FILES_SIZES);
+ conf.set(
+ MRJobConfig.CACHE_FILES_SIZES, sizes == null ? String.valueOf(size)
: size + "," + sizes);
+ } catch (InterruptedException | IOException e) {
+ logger.error("Upload extra conf exception", e);
+ throw new CelebornIOException("Upload extra conf exception ", e);
+ }
+ }
+
+ private static void validateInputParam(String value, String param) throws
IOException {
+ if (value == null) {
+ String msg = param + " is null";
+ logger.error(msg);
+ throw new IOException(msg);
Review Comment:
Why using IOException? How about using `CelebornRuntimeException`
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]