From 9e3f0fedd25ea52eccdbb4eb1820a7753cb7a853 Mon Sep 17 00:00:00 2001 From: xy720 Date: Mon, 17 Aug 2020 21:29:02 +0800 Subject: [PATCH 01/14] save code --- .../apache/doris/catalog/SparkResource.java | 20 ++ .../java/org/apache/doris/common/Config.java | 9 + .../apache/doris/load/loadv2/ConfigFile.java | 8 + .../doris/load/loadv2/SparkEtlJobHandler.java | 161 +++++++++---- .../doris/load/loadv2/SparkRepository.java | 2 +- .../load/loadv2/SparkYarnConfigFiles.java | 213 ++++++++++++++++++ .../load/loadv2/YarnApplicationReport.java | 88 ++++++++ 7 files changed, 462 insertions(+), 39 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/load/loadv2/ConfigFile.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/load/loadv2/YarnApplicationReport.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java index 4f0f95ea6456ca..a217e8804820d0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java @@ -24,6 +24,7 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.proc.BaseProcResult; import org.apache.doris.load.loadv2.SparkRepository; +import org.apache.doris.load.loadv2.SparkYarnConfigFiles; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -66,6 +67,7 @@ public class SparkResource extends Resource { private static final String SPARK_CONFIG_PREFIX = "spark."; private static final String BROKER_PROPERTY_PREFIX = "broker."; // spark uses hadoop configs in the form of spark.hadoop.* + private static final String SPARK_HADOOP_CONFIG_PREFIX = "spark.hadoop."; private static final String SPARK_YARN_RESOURCE_MANAGER_ADDRESS = "spark.hadoop.yarn.resourcemanager.address"; private static final String SPARK_FS_DEFAULT_FS = "spark.hadoop.fs.defaultFS"; private static final String YARN_RESOURCE_MANAGER_ADDRESS = "yarn.resourcemanager.address"; @@ -166,6 +168,14 @@ public synchronized SparkRepository.SparkArchive prepareArchive() throws LoadExc return archive; } + // Each SparkResource has and only has one yarn config to run yarn command + // This method will write all the configuration into config files in a specific directory + public synchronized String prepareYarnConfig() throws LoadException { + SparkYarnConfigFiles yarnConfigFiles = new SparkYarnConfigFiles(name, getSparkHadoopConfigs(sparkConfigs)); + yarnConfigFiles.prepare(); + return yarnConfigFiles.getConfigDir(); + } + public boolean isYarnMaster() { return getMaster().equalsIgnoreCase(YARN_MASTER); } @@ -243,6 +253,16 @@ private Map getSparkConfigs(Map properties) { return sparkConfigs; } + private Map getSparkHadoopConfigs(Map properties) { + Map sparkConfigs = Maps.newHashMap(); + for (Map.Entry entry : properties.entrySet()) { + if (entry.getKey().startsWith(SPARK_HADOOP_CONFIG_PREFIX)) { + sparkConfigs.put(entry.getKey(), entry.getValue()); + } + } + return sparkConfigs; + } + private Map getBrokerProperties(Map properties) { Map brokerProperties = Maps.newHashMap(); for (Map.Entry entry : properties.entrySet()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java index 26fccc0c42541f..b9db19af7a2caa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java @@ -543,6 +543,15 @@ public class Config extends ConfigBase { @ConfField public static String spark_resource_path = ""; + /** + * Default yarn client path + */ + @ConfField(mutable = true, masterOnly = true) + public static String yarn_client_path = PaloFe.DORIS_HOME_DIR + "/lib/yarn-client/hadoop/bin/yarn"; + + @ConfField + public static String yarn_config_dir = PaloFe.DORIS_HOME_DIR + "/lib/yarn-config"; + /** * Default number of waiting jobs for routine load and version 2 of load * This is a desired number. diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/ConfigFile.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/ConfigFile.java new file mode 100644 index 00000000000000..6b85e12ae4ba3c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/ConfigFile.java @@ -0,0 +1,8 @@ +package org.apache.doris.load.loadv2; + +import org.apache.doris.common.LoadException; + +public interface ConfigFile { + public void createFile() throws LoadException; + public String getFilePath(); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java index 428e88c3cae693..177e78e3df0d2e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java @@ -17,6 +17,7 @@ package org.apache.doris.load.loadv2; +import org.apache.doris.PaloFe; import org.apache.doris.analysis.BrokerDesc; import org.apache.doris.catalog.SparkResource; import org.apache.doris.common.Config; @@ -24,6 +25,8 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.BrokerUtil; +import org.apache.doris.common.util.CommandResult; +import org.apache.doris.common.util.Util; import org.apache.doris.load.EtlStatus; import org.apache.doris.load.loadv2.dpp.DppResult; import org.apache.doris.load.loadv2.etl.EtlJobConfig; @@ -43,9 +46,6 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.spark.launcher.SparkAppHandle; @@ -76,6 +76,13 @@ public class SparkEtlJobHandler { // 5min private static final int GET_APPID_MAX_RETRY_TIMES = 300; private static final int GET_APPID_SLEEP_MS = 1000; + // 30s + private static final long EXEC_CMD_TIMEOUT_MS = 30000L; + // yarn client path + private static final String YARN_CLIENT = PaloFe.DORIS_HOME_DIR + Config.yarn_client_path; + // yarn command + private static final String YARN_STATUS_CMD = "%s --config %s application -status %s"; + private static final String YARN_KILL_CMD = "%s --config %s application -kill %s"; class SparkAppListener implements Listener { @Override @@ -195,38 +202,47 @@ public void submitEtlJob(long loadJobId, String loadLabel, EtlJobConfig etlJobCo } public EtlStatus getEtlJobStatus(SparkAppHandle handle, String appId, long loadJobId, String etlOutputPath, - SparkResource resource, BrokerDesc brokerDesc) { + SparkResource resource, BrokerDesc brokerDesc) throws LoadException { EtlStatus status = new EtlStatus(); if (resource.isYarnMaster()) { - // state from yarn Preconditions.checkState(appId != null && !appId.isEmpty()); - YarnClient client = startYarnClient(resource); - try { - ApplicationReport report = client.getApplicationReport(ConverterUtils.toApplicationId(appId)); - LOG.info("yarn application -status {}. load job id: {}, result: {}", appId, loadJobId, report); - - YarnApplicationState state = report.getYarnApplicationState(); - FinalApplicationStatus faStatus = report.getFinalApplicationStatus(); - status.setState(fromYarnState(state, faStatus)); - if (status.getState() == TEtlState.CANCELLED) { - if (state == YarnApplicationState.FINISHED) { - status.setFailMsg("spark app state: " + faStatus.toString()); - } else { - status.setFailMsg("yarn app state: " + state.toString()); + // prepare yarn config + String configDir = resource.prepareYarnConfig(); + // yarn --config configDir application -status command + String yarnStatusCmd = String.format(YARN_STATUS_CMD, YARN_CLIENT, configDir, appId); + LOG.info(yarnStatusCmd); + String[] envp = { "LC_ALL=" + Config.locale }; + CommandResult result = Util.executeCommand(yarnStatusCmd, envp, EXEC_CMD_TIMEOUT_MS); + if (result.getReturnCode() != 0) { + String stderr = result.getStderr(); + if (stderr != null) { + // case application not exists + if (stderr.contains("doesn't exist in RM")) { + LOG.warn("spark app not found. spark app id: {}, load job id: {}", appId, loadJobId); + status.setState(TEtlState.CANCELLED); } } - status.setTrackingUrl(report.getTrackingUrl()); - status.setProgress((int) (report.getProgress() * 100)); - } catch (ApplicationNotFoundException e) { - LOG.warn("spark app not found. spark app id: {}, load job id: {}", appId, loadJobId, e); + LOG.warn("yarn application status failed. spark app id: {}, load job id: {}, timeout: {}, msg: {}", + appId, loadJobId, EXEC_CMD_TIMEOUT_MS, stderr); status.setState(TEtlState.CANCELLED); - status.setFailMsg(e.getMessage()); - } catch (YarnException | IOException e) { - LOG.warn("yarn application status failed. spark app id: {}, load job id: {}", appId, loadJobId, e); - } finally { - stopYarnClient(client); + return status; } + LOG.info(result.getStdout()); + ApplicationReport report = new YarnApplicationReport(result.getStdout()).getReport(); + LOG.info("yarn application -status {}. load job id: {}, result: {}", appId, loadJobId, report); + YarnApplicationState state = report.getYarnApplicationState(); + FinalApplicationStatus faStatus = report.getFinalApplicationStatus(); + status.setState(fromYarnState(state, faStatus)); + if (status.getState() == TEtlState.CANCELLED) { + if (state == YarnApplicationState.FINISHED) { + status.setFailMsg("spark app state: " + faStatus.toString()); + } else { + status.setFailMsg("yarn app state: " + state.toString()); + } + } + status.setTrackingUrl(report.getTrackingUrl()); + status.setProgress((int) (report.getProgress() * 100)); } else { // state from handle if (handle == null) { @@ -262,19 +278,88 @@ public EtlStatus getEtlJobStatus(SparkAppHandle handle, String appId, long loadJ return status; } - public void killEtlJob(SparkAppHandle handle, String appId, long loadJobId, SparkResource resource) { +// public EtlStatus getEtlJobStatus(SparkAppHandle handle, String appId, long loadJobId, String etlOutputPath, +// SparkResource resource, BrokerDesc brokerDesc) { +// EtlStatus status = new EtlStatus(); +// +// if (resource.isYarnMaster()) { +// // state from yarn +// Preconditions.checkState(appId != null && !appId.isEmpty()); +// YarnClient client = startYarnClient(resource); +// try { +// ApplicationReport report = client.getApplicationReport(ConverterUtils.toApplicationId(appId)); +// LOG.info("yarn application -status {}. load job id: {}, result: {}", appId, loadJobId, report); +// +// YarnApplicationState state = report.getYarnApplicationState(); +// FinalApplicationStatus faStatus = report.getFinalApplicationStatus(); +// status.setState(fromYarnState(state, faStatus)); +// if (status.getState() == TEtlState.CANCELLED) { +// if (state == YarnApplicationState.FINISHED) { +// status.setFailMsg("spark app state: " + faStatus.toString()); +// } else { +// status.setFailMsg("yarn app state: " + state.toString()); +// } +// } +// status.setTrackingUrl(report.getTrackingUrl()); +// status.setProgress((int) (report.getProgress() * 100)); +// } catch (ApplicationNotFoundException e) { +// LOG.warn("spark app not found. spark app id: {}, load job id: {}", appId, loadJobId, e); +// status.setState(TEtlState.CANCELLED); +// status.setFailMsg(e.getMessage()); +// } catch (YarnException | IOException e) { +// LOG.warn("yarn application status failed. spark app id: {}, load job id: {}", appId, loadJobId, e); +// } finally { +// stopYarnClient(client); +// } +// } else { +// // state from handle +// if (handle == null) { +// status.setFailMsg("spark app handle is null"); +// status.setState(TEtlState.CANCELLED); +// return status; +// } +// +// State state = handle.getState(); +// status.setState(fromSparkState(state)); +// if (status.getState() == TEtlState.CANCELLED) { +// status.setFailMsg("spark app state: " + state.toString()); +// } +// LOG.info("spark app id: {}, load job id: {}, app state: {}", appId, loadJobId, state); +// } +// +// if (status.getState() == TEtlState.FINISHED || status.getState() == TEtlState.CANCELLED) { +// // get dpp result +// String dppResultFilePath = EtlJobConfig.getDppResultFilePath(etlOutputPath); +// try { +// byte[] data = BrokerUtil.readFile(dppResultFilePath, brokerDesc); +// String dppResultStr = new String(data, "UTF-8"); +// DppResult dppResult = new Gson().fromJson(dppResultStr, DppResult.class); +// status.setDppResult(dppResult); +// if (status.getState() == TEtlState.CANCELLED && !Strings.isNullOrEmpty(dppResult.failedReason)) { +// status.setFailMsg(dppResult.failedReason); +// } +// } catch (UserException | JsonSyntaxException | UnsupportedEncodingException e) { +// LOG.warn("read broker file failed. path: {}", dppResultFilePath, e); +// } +// } +// +// return status; +// } + + public void killEtlJob(SparkAppHandle handle, String appId, long loadJobId, SparkResource resource) throws LoadException { if (resource.isYarnMaster()) { Preconditions.checkNotNull(appId); - YarnClient client = startYarnClient(resource); - try { - try { - client.killApplication(ConverterUtils.toApplicationId(appId)); - LOG.info("yarn application -kill {}", appId); - } catch (YarnException | IOException e) { - LOG.warn("yarn application kill failed. app id: {}, load job id: {}", appId, loadJobId, e); - } - } finally { - stopYarnClient(client); + // prepare yarn config + String configDir = resource.prepareYarnConfig(); + // yarn --config configDir application -status command + String yarnKillCmd = String.format(YARN_KILL_CMD, YARN_CLIENT, configDir, appId); + LOG.info(yarnKillCmd); + String[] envp = { "LC_ALL=" + Config.locale }; + CommandResult result = Util.executeCommand(yarnKillCmd, envp, EXEC_CMD_TIMEOUT_MS); + LOG.info("yarn application -kill {}", appId); + if (result.getReturnCode() != 0) { + String stderr = result.getStderr(); + LOG.warn("yarn application kill failed. app id: {}, load job id: {}, msg: {}", appId, loadJobId, stderr); } } else { if (handle != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkRepository.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkRepository.java index 18f8504886bbf8..3fc8bce63a3233 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkRepository.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkRepository.java @@ -98,7 +98,7 @@ public void prepare() throws LoadException { } private void initRepository() throws LoadException { - LOG.info("start to init remote repositoryi. local dpp: {}", this.localDppPath); + LOG.info("start to init remote repository. local dpp: {}", this.localDppPath); boolean needUpload = false; boolean needReplace = false; CHECK: { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java new file mode 100644 index 00000000000000..3fec5a9d4d4bde --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java @@ -0,0 +1,213 @@ +package org.apache.doris.load.loadv2; + +import org.apache.doris.common.Config; +import org.apache.doris.common.LoadException; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.w3c.dom.Document; +import org.w3c.dom.Element; +import org.w3c.dom.Node; + +import java.io.File; +import java.util.List; +import java.util.Map; +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.transform.OutputKeys; +import javax.xml.transform.Transformer; +import javax.xml.transform.TransformerFactory; +import javax.xml.transform.dom.DOMSource; +import javax.xml.transform.stream.StreamResult; + +public class SparkYarnConfigFiles { + private static final Logger LOG = LogManager.getLogger(SparkYarnConfigFiles.class); + + private static final String HADOOP_CONF_FILE = "core-site.xml"; + private static final String YARN_CONF_FILE = "yarn-site.xml"; + private static final String SPARK_HADOOP_PREFIX = "spark.hadoop."; + private static final String HADOOP_PREFIX = "hadoop."; + private static final String YARN_PREFIX = "yarn."; + + private String configDir; + private List configFiles; + + public String getConfigDir() { + return this.configDir; + } + + public SparkYarnConfigFiles(String resourceName, Map properties) { + this.configDir = Config.yarn_config_dir + "/" + resourceName; + this.configFiles = Lists.newArrayList(); + createConfigFiles(properties); + } + + private void createConfigFiles(Map properties) { + configFiles.add(new XMLConfigFile(configDir + "/" + HADOOP_CONF_FILE, + getPropertiesByPrefix(properties, HADOOP_PREFIX))); + configFiles.add(new XMLConfigFile(configDir + "/" + YARN_CONF_FILE, + getPropertiesByPrefix(properties, YARN_PREFIX))); + } + + public void prepare() throws LoadException { + initConfigFile(); + } + + private void initConfigFile() throws LoadException { + LOG.info("start to init config file. config dir: {}", this.configDir); + Preconditions.checkState(!Strings.isNullOrEmpty(configDir)); + + boolean needUpdate = false; + boolean needReplace = false; + CHECK: { + if (!checkConfigDirExists(this.configDir)) { + needUpdate = true; + break CHECK; + } + + for (ConfigFile configFile : configFiles) { + String filePath = configFile.getFilePath(); + if (!checkConfigFileExists(filePath)) { + needUpdate = true; + needReplace = true; + break CHECK; + } + } + } + + if (needUpdate) { + updateConfig(needReplace); + } + LOG.info("init spark yarn config success, config dir={}, config file size={}", + configDir, configFiles.size()); + } + + private boolean checkConfigDirExists(String dir) { + boolean result = true; + File configDir = new File(dir); + if (!configDir.exists() || !configDir.isDirectory()) { + result = false; + } + return result; + } + + private boolean checkConfigFileExists(String filePath) { + boolean result = true; + File configFile = new File(filePath); + if (!configFile.exists() || !configFile.isFile()) { + result = false; + } + return result; + } + + private void updateConfig(boolean needReplace) throws LoadException { + if (needReplace) { + clearAndDelete(this.configDir); + } + mkdir(this.configDir); + for (ConfigFile configFile : configFiles) { + configFile.createFile(); + } + } + + private Map getPropertiesByPrefix(Map properties, String prefix) { + Map result = Maps.newHashMap(); + for (Map.Entry property : properties.entrySet()) { + String key = property.getKey(); + if (key.startsWith(SPARK_HADOOP_PREFIX)) { + String newKey = key.substring(SPARK_HADOOP_PREFIX.length()); + if (newKey.startsWith(prefix)) { + result.put(newKey, property.getValue()); + properties.remove(key); + } + } + } + return result; + } + + private void clearAndDelete(String deletePath) { + File file = new File(deletePath); + if (!file.exists()) { + return; + } + if (file.isFile()) { + file.delete(); + return; + } + File[] files = file.listFiles(); + for (File file1 : files) { + clearAndDelete(file1.getAbsolutePath()); + } + file.delete(); + } + + private void mkdir(String configDir) { + File file = new File(configDir); + file.mkdir(); + } + + public static class XMLConfigFile implements ConfigFile { + private static final String CONFIGURATION = "configuration"; + private static final String PROPERTY = "property"; + private static final String NAME = "name"; + private static final String VALUE = "value"; + + private String filePath; + private Map configProperties; + + public XMLConfigFile(String filePath, Map configProperties) { + this.filePath = filePath; + this.configProperties = configProperties; + } + + @Override + public String getFilePath() { + return filePath; + } + + @Override + public void createFile() throws LoadException { + createXML(this.filePath, this.configProperties); + } + + private void createXML(String filePath, Map properties) throws LoadException { + try { + DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance(); + DocumentBuilder db = factory.newDocumentBuilder(); + Document document = db.newDocument(); + document.setXmlStandalone(true); + Element configuration = (Element) appendNode(document, CONFIGURATION, null); + for (Map.Entry pair : properties.entrySet()) { + Element property = (Element) appendNode(configuration, PROPERTY, null); + appendNode(property, NAME, pair.getKey()); + appendNode(property, VALUE, pair.getValue()); + } + + TransformerFactory tff = TransformerFactory.newInstance(); + Transformer tf = tff.newTransformer(); + + tf.setOutputProperty(OutputKeys.INDENT, "yes"); + tf.transform(new DOMSource(document), new StreamResult(new File(filePath))); + } catch (Exception e) { + throw new LoadException(e.getMessage()); + } + } + + private Node appendNode(Node parent, String tag, String content) { + Element child = null; + if (parent instanceof Document) { + child = ((Document)parent).createElement(tag); + } else { + child = parent.getOwnerDocument().createElement(tag); + } + if (content != null && !content.equals("")) { + child.setTextContent(content); + } + return parent.appendChild(child); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/YarnApplicationReport.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/YarnApplicationReport.java new file mode 100644 index 00000000000000..b46a14b17a5df0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/YarnApplicationReport.java @@ -0,0 +1,88 @@ +package org.apache.doris.load.loadv2; + +import com.google.common.base.Splitter; +import com.google.common.collect.Maps; + +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationReportPBImpl; +import org.apache.hadoop.yarn.util.ConverterUtils; + +import java.util.List; +import java.util.Map; + +/** + * Covert output string of command `yarn application -status` to application report. + * Input sample: + * ------------------- + * Application Report : + * Application-Id : application_1573630236805_6763648 + * Application-Name : doris_label_test + * Application-Type : SPARK-2.4.1 + * User : test + * Queue : test-queue + * Start-Time : 1597654469958 + * Finish-Time : 1597654801939 + * Progress : 100% + * State : FINISHED + * Final-State : SUCCEEDED + * Tracking-URL : 127.0.0.1:8004/history/application_1573630236805_6763648/1 + * RPC Port : 40236 + * AM Host : host-name + * ------------------ + * + * Output: + * ApplicationReport + */ +public class YarnApplicationReport { + private ApplicationReport report; + private static final String APPLICATION_ID = "Application-Id"; + private static final String APPLICATION_TYPE = "Application-Type"; + private static final String APPLICATION_NAME = "Application-Name"; + private static final String USER = "User"; + private static final String QUEUE = "Queue"; + private static final String START_TIME = "Start-Time"; + private static final String FINISH_TIME = "Finish-Time"; + private static final String PROGRESS = "Progress"; + private static final String STATE = "State"; + private static final String FINAL_STATE = "Final-State"; + private static final String TRACKING_URL = "Tracking-URL"; + private static final String RPC_PORT = "RPC Port"; + private static final String AM_HOST = "AM Host"; + + public YarnApplicationReport(String output) { + this.report = new ApplicationReportPBImpl(); + getReportFromOutput(output); + } + + public ApplicationReport getReport() { + return report; + } + + private void getReportFromOutput(String output) { + Map reportMap = Maps.newHashMap(); + List lines = Splitter.onPattern(",").trimResults().splitToList(output); + for (String line : lines) { + List entry = Splitter.onPattern(":").trimResults().splitToList(line); + if (entry.size() != 2) { + continue; + } + reportMap.put(entry.get(0), entry.get(1)); + } + + report.setApplicationId(ConverterUtils.toApplicationId(reportMap.get(APPLICATION_ID))); + report.setName(reportMap.get(APPLICATION_NAME)); + report.setApplicationType(reportMap.get(APPLICATION_TYPE)); + report.setUser(reportMap.get(USER)); + report.setQueue(reportMap.get(QUEUE)); + report.setStartTime(Long.parseLong(reportMap.get(START_TIME))); + report.setFinishTime(Long.parseLong(reportMap.get(FINISH_TIME))); + report.setProgress(Float.parseFloat(reportMap.get(PROGRESS))); + report.setYarnApplicationState(YarnApplicationState.valueOf(reportMap.get(STATE))); + report.setFinalApplicationStatus(FinalApplicationStatus.valueOf(reportMap.get(FINAL_STATE))); + report.setTrackingUrl(reportMap.get(TRACKING_URL)); + report.setRpcPort(Integer.parseInt(reportMap.get(RPC_PORT))); + report.setHost(reportMap.get(AM_HOST)); + } +} From b806b0b93add0f96c2f85ad989fe12d9fe2c1113 Mon Sep 17 00:00:00 2001 From: xy720 Date: Tue, 18 Aug 2020 15:52:35 +0800 Subject: [PATCH 02/14] save code --- .../apache/doris/load/loadv2/ConfigFile.java | 17 ++++ .../doris/load/loadv2/SparkEtlJobHandler.java | 89 +++---------------- .../load/loadv2/SparkYarnConfigFiles.java | 30 ++++++- .../load/loadv2/YarnApplicationReport.java | 77 +++++++++++----- 4 files changed, 110 insertions(+), 103 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/ConfigFile.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/ConfigFile.java index 6b85e12ae4ba3c..de2536f489e2b2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/ConfigFile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/ConfigFile.java @@ -1,3 +1,20 @@ +// 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.doris.load.loadv2; import org.apache.doris.common.LoadException; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java index 177e78e3df0d2e..126aa883940d8b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java @@ -17,7 +17,6 @@ package org.apache.doris.load.loadv2; -import org.apache.doris.PaloFe; import org.apache.doris.analysis.BrokerDesc; import org.apache.doris.catalog.SparkResource; import org.apache.doris.common.Config; @@ -78,8 +77,6 @@ public class SparkEtlJobHandler { private static final int GET_APPID_SLEEP_MS = 1000; // 30s private static final long EXEC_CMD_TIMEOUT_MS = 30000L; - // yarn client path - private static final String YARN_CLIENT = PaloFe.DORIS_HOME_DIR + Config.yarn_client_path; // yarn command private static final String YARN_STATUS_CMD = "%s --config %s application -status %s"; private static final String YARN_KILL_CMD = "%s --config %s application -kill %s"; @@ -209,8 +206,10 @@ public EtlStatus getEtlJobStatus(SparkAppHandle handle, String appId, long loadJ Preconditions.checkState(appId != null && !appId.isEmpty()); // prepare yarn config String configDir = resource.prepareYarnConfig(); - // yarn --config configDir application -status command - String yarnStatusCmd = String.format(YARN_STATUS_CMD, YARN_CLIENT, configDir, appId); + // yarn client path + String yarnClient = Config.yarn_client_path; + // command: yarn --config configDir application -status appId + String yarnStatusCmd = String.format(YARN_STATUS_CMD, yarnClient, configDir, appId); LOG.info(yarnStatusCmd); String[] envp = { "LC_ALL=" + Config.locale }; CommandResult result = Util.executeCommand(yarnStatusCmd, envp, EXEC_CMD_TIMEOUT_MS); @@ -228,9 +227,9 @@ public EtlStatus getEtlJobStatus(SparkAppHandle handle, String appId, long loadJ status.setState(TEtlState.CANCELLED); return status; } - LOG.info(result.getStdout()); ApplicationReport report = new YarnApplicationReport(result.getStdout()).getReport(); - LOG.info("yarn application -status {}. load job id: {}, result: {}", appId, loadJobId, report); + LOG.info("yarn application -status {}. load job id: {}, output: {}, report: {}", + appId, loadJobId, result.getStdout(), report); YarnApplicationState state = report.getYarnApplicationState(); FinalApplicationStatus faStatus = report.getFinalApplicationStatus(); status.setState(fromYarnState(state, faStatus)); @@ -278,85 +277,19 @@ public EtlStatus getEtlJobStatus(SparkAppHandle handle, String appId, long loadJ return status; } -// public EtlStatus getEtlJobStatus(SparkAppHandle handle, String appId, long loadJobId, String etlOutputPath, -// SparkResource resource, BrokerDesc brokerDesc) { -// EtlStatus status = new EtlStatus(); -// -// if (resource.isYarnMaster()) { -// // state from yarn -// Preconditions.checkState(appId != null && !appId.isEmpty()); -// YarnClient client = startYarnClient(resource); -// try { -// ApplicationReport report = client.getApplicationReport(ConverterUtils.toApplicationId(appId)); -// LOG.info("yarn application -status {}. load job id: {}, result: {}", appId, loadJobId, report); -// -// YarnApplicationState state = report.getYarnApplicationState(); -// FinalApplicationStatus faStatus = report.getFinalApplicationStatus(); -// status.setState(fromYarnState(state, faStatus)); -// if (status.getState() == TEtlState.CANCELLED) { -// if (state == YarnApplicationState.FINISHED) { -// status.setFailMsg("spark app state: " + faStatus.toString()); -// } else { -// status.setFailMsg("yarn app state: " + state.toString()); -// } -// } -// status.setTrackingUrl(report.getTrackingUrl()); -// status.setProgress((int) (report.getProgress() * 100)); -// } catch (ApplicationNotFoundException e) { -// LOG.warn("spark app not found. spark app id: {}, load job id: {}", appId, loadJobId, e); -// status.setState(TEtlState.CANCELLED); -// status.setFailMsg(e.getMessage()); -// } catch (YarnException | IOException e) { -// LOG.warn("yarn application status failed. spark app id: {}, load job id: {}", appId, loadJobId, e); -// } finally { -// stopYarnClient(client); -// } -// } else { -// // state from handle -// if (handle == null) { -// status.setFailMsg("spark app handle is null"); -// status.setState(TEtlState.CANCELLED); -// return status; -// } -// -// State state = handle.getState(); -// status.setState(fromSparkState(state)); -// if (status.getState() == TEtlState.CANCELLED) { -// status.setFailMsg("spark app state: " + state.toString()); -// } -// LOG.info("spark app id: {}, load job id: {}, app state: {}", appId, loadJobId, state); -// } -// -// if (status.getState() == TEtlState.FINISHED || status.getState() == TEtlState.CANCELLED) { -// // get dpp result -// String dppResultFilePath = EtlJobConfig.getDppResultFilePath(etlOutputPath); -// try { -// byte[] data = BrokerUtil.readFile(dppResultFilePath, brokerDesc); -// String dppResultStr = new String(data, "UTF-8"); -// DppResult dppResult = new Gson().fromJson(dppResultStr, DppResult.class); -// status.setDppResult(dppResult); -// if (status.getState() == TEtlState.CANCELLED && !Strings.isNullOrEmpty(dppResult.failedReason)) { -// status.setFailMsg(dppResult.failedReason); -// } -// } catch (UserException | JsonSyntaxException | UnsupportedEncodingException e) { -// LOG.warn("read broker file failed. path: {}", dppResultFilePath, e); -// } -// } -// -// return status; -// } - public void killEtlJob(SparkAppHandle handle, String appId, long loadJobId, SparkResource resource) throws LoadException { if (resource.isYarnMaster()) { Preconditions.checkNotNull(appId); // prepare yarn config String configDir = resource.prepareYarnConfig(); - // yarn --config configDir application -status command - String yarnKillCmd = String.format(YARN_KILL_CMD, YARN_CLIENT, configDir, appId); + // yarn client path + String yarnClient = Config.yarn_client_path; + // command: yarn --config configDir application -kill appId + String yarnKillCmd = String.format(YARN_KILL_CMD, yarnClient, configDir, appId); LOG.info(yarnKillCmd); String[] envp = { "LC_ALL=" + Config.locale }; CommandResult result = Util.executeCommand(yarnKillCmd, envp, EXEC_CMD_TIMEOUT_MS); - LOG.info("yarn application -kill {}", appId); + LOG.info("yarn application -kill {}, output: {}", appId, result.getStdout()); if (result.getReturnCode() != 0) { String stderr = result.getStderr(); LOG.warn("yarn application kill failed. app id: {}, load job id: {}, msg: {}", appId, loadJobId, stderr); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java index 3fec5a9d4d4bde..b2195428d5882e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java @@ -1,3 +1,20 @@ +// 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.doris.load.loadv2; import org.apache.doris.common.Config; @@ -14,6 +31,7 @@ import org.w3c.dom.Node; import java.io.File; +import java.util.Iterator; import java.util.List; import java.util.Map; import javax.xml.parsers.DocumentBuilder; @@ -47,6 +65,7 @@ public SparkYarnConfigFiles(String resourceName, Map properties) } private void createConfigFiles(Map properties) { + LOG.info("create config file, properties size: {}", properties.size()); configFiles.add(new XMLConfigFile(configDir + "/" + HADOOP_CONF_FILE, getPropertiesByPrefix(properties, HADOOP_PREFIX))); configFiles.add(new XMLConfigFile(configDir + "/" + YARN_CONF_FILE, @@ -92,6 +111,7 @@ private boolean checkConfigDirExists(String dir) { if (!configDir.exists() || !configDir.isDirectory()) { result = false; } + LOG.info("check yarn client config dir exists, result: {}", result); return result; } @@ -101,6 +121,7 @@ private boolean checkConfigFileExists(String filePath) { if (!configFile.exists() || !configFile.isFile()) { result = false; } + LOG.info("check yarn client config file path exists, result: {}, path: {}", result, filePath); return result; } @@ -112,17 +133,20 @@ private void updateConfig(boolean needReplace) throws LoadException { for (ConfigFile configFile : configFiles) { configFile.createFile(); } + LOG.info("finished to update yarn client config dir, dir={}", configDir); } private Map getPropertiesByPrefix(Map properties, String prefix) { Map result = Maps.newHashMap(); - for (Map.Entry property : properties.entrySet()) { + Iterator> iterator = properties.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry property = iterator.next(); String key = property.getKey(); if (key.startsWith(SPARK_HADOOP_PREFIX)) { String newKey = key.substring(SPARK_HADOOP_PREFIX.length()); if (newKey.startsWith(prefix)) { result.put(newKey, property.getValue()); - properties.remove(key); + iterator.remove(); } } } @@ -147,7 +171,7 @@ private void clearAndDelete(String deletePath) { private void mkdir(String configDir) { File file = new File(configDir); - file.mkdir(); + file.mkdirs(); } public static class XMLConfigFile implements ConfigFile { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/YarnApplicationReport.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/YarnApplicationReport.java index b46a14b17a5df0..3270fd7d9701ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/YarnApplicationReport.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/YarnApplicationReport.java @@ -1,5 +1,24 @@ +// 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.doris.load.loadv2; +import org.apache.doris.common.LoadException; +import com.google.common.base.Preconditions; import com.google.common.base.Splitter; import com.google.common.collect.Maps; @@ -9,6 +28,8 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationReportPBImpl; import org.apache.hadoop.yarn.util.ConverterUtils; +import java.text.NumberFormat; +import java.text.ParseException; import java.util.List; import java.util.Map; @@ -36,7 +57,6 @@ * ApplicationReport */ public class YarnApplicationReport { - private ApplicationReport report; private static final String APPLICATION_ID = "Application-Id"; private static final String APPLICATION_TYPE = "Application-Type"; private static final String APPLICATION_NAME = "Application-Name"; @@ -50,39 +70,52 @@ public class YarnApplicationReport { private static final String TRACKING_URL = "Tracking-URL"; private static final String RPC_PORT = "RPC Port"; private static final String AM_HOST = "AM Host"; + private static final String DIAGNOSTICS = "Diagnostics"; - public YarnApplicationReport(String output) { + private ApplicationReport report; + + public YarnApplicationReport(String output) throws LoadException { this.report = new ApplicationReportPBImpl(); - getReportFromOutput(output); + parseFromOutput(output); } public ApplicationReport getReport() { return report; } - private void getReportFromOutput(String output) { + private void parseFromOutput(String output) throws LoadException { Map reportMap = Maps.newHashMap(); - List lines = Splitter.onPattern(",").trimResults().splitToList(output); + List lines = Splitter.onPattern("\n").trimResults().splitToList(output); + // Application-Id : application_1573630236805_6763648 ==> (Application-Id, application_1573630236805_6763648) for (String line : lines) { - List entry = Splitter.onPattern(":").trimResults().splitToList(line); - if (entry.size() != 2) { - continue; + List entry = Splitter.onPattern(":").limit(2).trimResults().splitToList(line); + Preconditions.checkState(entry.size() <= 2); + if (entry.size() > 1) { + reportMap.put(entry.get(0), entry.get(1)); + } else { + reportMap.put(entry.get(0), ""); } - reportMap.put(entry.get(0), entry.get(1)); } - report.setApplicationId(ConverterUtils.toApplicationId(reportMap.get(APPLICATION_ID))); - report.setName(reportMap.get(APPLICATION_NAME)); - report.setApplicationType(reportMap.get(APPLICATION_TYPE)); - report.setUser(reportMap.get(USER)); - report.setQueue(reportMap.get(QUEUE)); - report.setStartTime(Long.parseLong(reportMap.get(START_TIME))); - report.setFinishTime(Long.parseLong(reportMap.get(FINISH_TIME))); - report.setProgress(Float.parseFloat(reportMap.get(PROGRESS))); - report.setYarnApplicationState(YarnApplicationState.valueOf(reportMap.get(STATE))); - report.setFinalApplicationStatus(FinalApplicationStatus.valueOf(reportMap.get(FINAL_STATE))); - report.setTrackingUrl(reportMap.get(TRACKING_URL)); - report.setRpcPort(Integer.parseInt(reportMap.get(RPC_PORT))); - report.setHost(reportMap.get(AM_HOST)); + try { + report.setApplicationId(ConverterUtils.toApplicationId(reportMap.get(APPLICATION_ID))); + report.setName(reportMap.get(APPLICATION_NAME)); + report.setApplicationType(reportMap.get(APPLICATION_TYPE)); + report.setUser(reportMap.get(USER)); + report.setQueue(reportMap.get(QUEUE)); + report.setStartTime(Long.parseLong(reportMap.get(START_TIME))); + report.setFinishTime(Long.parseLong(reportMap.get(FINISH_TIME))); + report.setProgress(NumberFormat.getPercentInstance().parse(reportMap.get(PROGRESS)).floatValue()); + report.setYarnApplicationState(YarnApplicationState.valueOf(reportMap.get(STATE))); + report.setFinalApplicationStatus(FinalApplicationStatus.valueOf(reportMap.get(FINAL_STATE))); + report.setTrackingUrl(reportMap.get(TRACKING_URL)); + report.setRpcPort(Integer.parseInt(reportMap.get(RPC_PORT))); + report.setHost(reportMap.get(AM_HOST)); + report.setDiagnostics(reportMap.get(DIAGNOSTICS)); + } catch (NumberFormatException | ParseException e) { + throw new LoadException(e.getMessage()); + } catch (Exception e) { + throw new LoadException(e.getMessage()); + } } } From 6ea1493fa405059ba4a1dce8f0574326e25e7c61 Mon Sep 17 00:00:00 2001 From: xy720 Date: Wed, 19 Aug 2020 16:03:00 +0800 Subject: [PATCH 03/14] add ut --- .../apache/doris/catalog/SparkResource.java | 2 +- .../doris/load/loadv2/SparkEtlJobHandler.java | 19 --- .../load/loadv2/SparkYarnConfigFiles.java | 7 + .../load/loadv2/YarnApplicationReport.java | 2 +- .../load/loadv2/SparkEtlJobHandlerTest.java | 159 +++++++++++++++--- .../load/loadv2/SparkYarnConfigFilesTest.java | 72 ++++++++ .../loadv2/YarnApplicationReportTest.java | 50 ++++++ 7 files changed, 264 insertions(+), 47 deletions(-) create mode 100644 fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkYarnConfigFilesTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/load/loadv2/YarnApplicationReportTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java index a217e8804820d0..cd03a21b009ceb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java @@ -169,7 +169,7 @@ public synchronized SparkRepository.SparkArchive prepareArchive() throws LoadExc } // Each SparkResource has and only has one yarn config to run yarn command - // This method will write all the configuration into config files in a specific directory + // This method will write all the configuration start with "spark.hadoop." into config files in a specific directory public synchronized String prepareYarnConfig() throws LoadException { SparkYarnConfigFiles yarnConfigFiles = new SparkYarnConfigFiles(name, getSparkHadoopConfigs(sparkConfigs)); yarnConfigFiles.prepare(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java index 126aa883940d8b..48941e8cbca2db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java @@ -21,7 +21,6 @@ import org.apache.doris.catalog.SparkResource; import org.apache.doris.common.Config; import org.apache.doris.common.LoadException; -import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.BrokerUtil; import org.apache.doris.common.util.CommandResult; @@ -39,12 +38,9 @@ import com.google.gson.Gson; import com.google.gson.JsonSyntaxException; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.client.api.YarnClient; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.spark.launcher.SparkAppHandle; @@ -332,21 +328,6 @@ public void deleteEtlOutputPath(String outputPath, BrokerDesc brokerDesc) { } } - private YarnClient startYarnClient(SparkResource resource) { - YarnClient client = YarnClient.createYarnClient(); - Configuration conf = new YarnConfiguration(); - // set yarn.resourcemanager.address - Pair pair = resource.getYarnResourcemanagerAddressPair(); - conf.set(pair.first, pair.second); - client.init(conf); - client.start(); - return client; - } - - private void stopYarnClient(YarnClient client) { - client.stop(); - } - private TEtlState fromYarnState(YarnApplicationState state, FinalApplicationStatus faStatus) { switch (state) { case FINISHED: diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java index b2195428d5882e..927a6434adcbc8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java @@ -64,6 +64,13 @@ public SparkYarnConfigFiles(String resourceName, Map properties) createConfigFiles(properties); } + // for unit test + public SparkYarnConfigFiles(String resourceName, String parentDir, Map properties) { + this.configDir = parentDir + "/" + resourceName; + this.configFiles = Lists.newArrayList(); + createConfigFiles(properties); + } + private void createConfigFiles(Map properties) { LOG.info("create config file, properties size: {}", properties.size()); configFiles.add(new XMLConfigFile(configDir + "/" + HADOOP_CONF_FILE, diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/YarnApplicationReport.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/YarnApplicationReport.java index 3270fd7d9701ef..cd540c280e5325 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/YarnApplicationReport.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/YarnApplicationReport.java @@ -89,7 +89,7 @@ private void parseFromOutput(String output) throws LoadException { // Application-Id : application_1573630236805_6763648 ==> (Application-Id, application_1573630236805_6763648) for (String line : lines) { List entry = Splitter.onPattern(":").limit(2).trimResults().splitToList(line); - Preconditions.checkState(entry.size() <= 2); + Preconditions.checkState(entry.size() <= 2, line); if (entry.size() > 1) { reportMap.put(entry.get(0), entry.get(1)); } else { diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkEtlJobHandlerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkEtlJobHandlerTest.java index e011062fbbc0e0..32306402f01de4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkEtlJobHandlerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkEtlJobHandlerTest.java @@ -33,6 +33,8 @@ import org.apache.doris.common.LoadException; import org.apache.doris.common.UserException; import org.apache.doris.common.util.BrokerUtil; +import org.apache.doris.common.util.CommandResult; +import org.apache.doris.common.util.Util; import org.apache.doris.load.EtlStatus; import org.apache.doris.load.loadv2.etl.EtlJobConfig; import org.apache.doris.thrift.TBrokerFileStatus; @@ -46,12 +48,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.client.api.YarnClient; -import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.spark.launcher.SparkAppHandle; import org.apache.spark.launcher.SparkAppHandle.State; import org.apache.spark.launcher.SparkLauncher; @@ -76,6 +72,52 @@ public class SparkEtlJobHandlerTest { private String remoteArchivePath; private SparkRepository.SparkArchive archive; + private final String runningReport = "Application Report :\n" + + "Application-Id : application_15888888888_0088\n" + + "Application-Name : label0\n" + + "Application-Type : SPARK-2.4.1\n" + + "User : test\n" + + "Queue : test-queue\n" + + "Start-Time : 1597654469958\n" + + "Finish-Time : 0\n" + + "Progress : 50%\n" + + "State : RUNNING\n" + + "Final-State : UNDEFINED\n" + + "Tracking-URL : http://127.0.0.1:8080/proxy/application_1586619723848_0088/\n" + + "RPC Port : 40236\n" + + "AM Host : host-name"; + + private final String failedReport = "Application Report :\n" + + "Application-Id : application_15888888888_0088\n" + + "Application-Name : label0\n" + + "Application-Type : SPARK-2.4.1\n" + + "User : test\n" + + "Queue : test-queue\n" + + "Start-Time : 1597654469958\n" + + "Finish-Time : 1597654801939\n" + + "Progress : 100%\n" + + "State : FINISHED\n" + + "Final-State : FAILED\n" + + "Tracking-URL : http://127.0.0.1:8080/proxy/application_1586619723848_0088/\n" + + "RPC Port : 40236\n" + + "AM Host : host-name"; + + private final String finishReport = "Application Report :\n" + + "Application-Id : application_15888888888_0088\n" + + "Application-Name : label0\n" + + "Application-Type : SPARK-2.4.1\n" + + "User : test\n" + + "Queue : test-queue\n" + + "Start-Time : 1597654469958\n" + + "Finish-Time : 1597654801939\n" + + "Progress : 100%\n" + + "State : FINISHED\n" + + "Final-State : SUCCEEDED\n" + + "Tracking-URL : http://127.0.0.1:8080/proxy/application_1586619723848_0088/\n" + + "RPC Port : 40236\n" + + "AM Host : host-name"; + + @Before public void setUp() { loadJobId = 0L; @@ -166,23 +208,32 @@ public void testSubmitEtlJobFailed(@Mocked BrokerUtil brokerUtil, @Mocked SparkL } @Test - public void testGetEtlJobStatus(@Mocked BrokerUtil brokerUtil, @Mocked YarnClient client, - @Injectable ApplicationReport report) - throws IOException, YarnException, UserException { + public void testGetEtlJobStatus(@Mocked BrokerUtil brokerUtil, @Mocked Util util, @Mocked CommandResult commandResult, + @Mocked SparkYarnConfigFiles sparkYarnConfigFiles) + throws IOException, UserException { + + new Expectations() { + { + sparkYarnConfigFiles.prepare(); + sparkYarnConfigFiles.getConfigDir(); + result = "./yarn_config"; + } + }; + + new Expectations() { + { + commandResult.getReturnCode(); + result = 0; + commandResult.getStdout(); + returns(runningReport, runningReport, failedReport, failedReport, finishReport, finishReport); + } + }; + new Expectations() { { - YarnClient.createYarnClient(); - result = client; - client.getApplicationReport((ApplicationId) any); - result = report; - report.getYarnApplicationState(); - returns(YarnApplicationState.RUNNING, YarnApplicationState.FINISHED, YarnApplicationState.FINISHED); - report.getFinalApplicationStatus(); - returns(FinalApplicationStatus.UNDEFINED, FinalApplicationStatus.FAILED, FinalApplicationStatus.SUCCEEDED); - report.getTrackingUrl(); - result = trackingUrl; - report.getProgress(); - returns(0.5f, 1f, 1f); + Util.executeCommand(anyString, (String[]) any); + minTimes = 0; + result = commandResult; BrokerUtil.readFile(anyString, (BrokerDesc) any); result = "{'normal_rows': 10, 'abnormal_rows': 0, 'failed_reason': 'etl job failed'}"; } @@ -217,13 +268,69 @@ public void testGetEtlJobStatus(@Mocked BrokerUtil brokerUtil, @Mocked YarnClien } @Test - public void testKillEtlJob(@Mocked YarnClient client) throws IOException, YarnException { + public void testGetEtlJobStatusFailed(@Mocked Util util, @Mocked CommandResult commandResult, + @Mocked SparkYarnConfigFiles sparkYarnConfigFiles) + throws IOException, UserException { + new Expectations() { { - YarnClient.createYarnClient(); - result = client; - client.killApplication((ApplicationId) any); - times = 1; + sparkYarnConfigFiles.prepare(); + sparkYarnConfigFiles.getConfigDir(); + result = "./yarn_config"; + } + }; + + new Expectations() { + { + commandResult.getReturnCode(); + result = -1; + } + }; + + new Expectations() { + { + Util.executeCommand(anyString, (String[]) any); + minTimes = 0; + result = commandResult; + } + }; + + SparkResource resource = new SparkResource(resourceName); + Map sparkConfigs = resource.getSparkConfigs(); + sparkConfigs.put("spark.master", "yarn"); + sparkConfigs.put("spark.submit.deployMode", "cluster"); + sparkConfigs.put("spark.hadoop.yarn.resourcemanager.address", "127.0.0.1:9999"); + BrokerDesc brokerDesc = new BrokerDesc(broker, Maps.newHashMap()); + SparkEtlJobHandler handler = new SparkEtlJobHandler(); + + // yarn finished and spark failed + EtlStatus status = handler.getEtlJobStatus(null, appId, loadJobId, etlOutputPath, resource, brokerDesc); + Assert.assertEquals(TEtlState.CANCELLED, status.getState()); + } + + @Test + public void testKillEtlJob(@Mocked Util util, @Mocked CommandResult commandResult, + @Mocked SparkYarnConfigFiles sparkYarnConfigFiles) throws IOException, UserException { + new Expectations() { + { + sparkYarnConfigFiles.prepare(); + sparkYarnConfigFiles.getConfigDir(); + result = "./yarn_config"; + } + }; + + new Expectations() { + { + commandResult.getReturnCode(); + result = 0; + } + }; + + new Expectations() { + { + Util.executeCommand(anyString, (String[]) any); + minTimes = 0; + result = commandResult; } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkYarnConfigFilesTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkYarnConfigFilesTest.java new file mode 100644 index 00000000000000..09e4f62403971b --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkYarnConfigFilesTest.java @@ -0,0 +1,72 @@ +package org.apache.doris.load.loadv2; + +import mockit.Mocked; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.common.LoadException; +import com.google.common.collect.Maps; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.util.Map; + +public class SparkYarnConfigFilesTest { + private static final String RESOURCE_NAME = "spark0"; + private static final String SPARK_HADOOP_PREFIX = "spark.hadoop."; + private static final String YARN_CONFIG_DIR = "./yarn_config"; + + private Map properties; + + @Mocked + Catalog catalog; + + @Before + public void setUp() { + properties = Maps.newHashMap(); + properties.put(SPARK_HADOOP_PREFIX + "hadoop.job.ugi", "test,test"); + properties.put(SPARK_HADOOP_PREFIX + "hadoop.security.authentication", "simple"); + properties.put(SPARK_HADOOP_PREFIX + "yarn.resourcemanager.address", "host:port"); + properties.put(SPARK_HADOOP_PREFIX + "yarn.resourcemanager.scheduler.address", "host:port"); + } + + @Test + public void testNormal() { + SparkYarnConfigFiles sparkYarnConfigFiles = new SparkYarnConfigFiles(RESOURCE_NAME, YARN_CONFIG_DIR, properties); + try { + // prepare config files + sparkYarnConfigFiles.prepare(); + // get config files' parent directory + String configDir = sparkYarnConfigFiles.getConfigDir(); + File dir = new File(configDir); + File[] configFiles = dir.listFiles(); + Assert.assertEquals(2, configFiles.length); + } catch (LoadException e) { + Assert.fail(); + } + } + + @After + public void clear() { + delete(YARN_CONFIG_DIR); + } + + private void delete(String deletePath) { + File file = new File(deletePath); + if (!file.exists()) { + return; + } + if (file.isFile()) { + file.delete(); + return; + } + File[] files = file.listFiles(); + for (File file1 : files) { + delete(file1.getAbsolutePath()); + } + file.delete(); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/YarnApplicationReportTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/YarnApplicationReportTest.java new file mode 100644 index 00000000000000..2815d5f306ae78 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/YarnApplicationReportTest.java @@ -0,0 +1,50 @@ +package org.apache.doris.load.loadv2; + +import org.apache.doris.common.LoadException; + +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.junit.Assert; +import org.junit.Test; + +public class YarnApplicationReportTest { + private final String runningReport = "Application Report :\n" + + "Application-Id : application_15888888888_0088\n" + + "Application-Name : label0\n" + + "Application-Type : SPARK-2.4.1\n" + + "User : test\n" + + "Queue : test-queue\n" + + "Start-Time : 1597654469958\n" + + "Finish-Time : 0\n" + + "Progress : 50%\n" + + "State : RUNNING\n" + + "Final-State : UNDEFINED\n" + + "Tracking-URL : http://127.0.0.1:8080/proxy/application_1586619723848_0088/\n" + + "RPC Port : 40236\n" + + "AM Host : host-name"; + + @Test + public void testParseToReport() { + try { + YarnApplicationReport yarnReport = new YarnApplicationReport(runningReport); + ApplicationReport report = yarnReport.getReport(); + Assert.assertEquals("application_15888888888_0088", report.getApplicationId().toString()); + Assert.assertEquals("label0", report.getName()); + Assert.assertEquals("test", report.getUser()); + Assert.assertEquals("test-queue", report.getQueue()); + Assert.assertEquals(1597654469958L, report.getStartTime()); + Assert.assertEquals(0L, report.getFinishTime()); + Assert.assertTrue(report.getProgress() == 0.5f); + Assert.assertEquals(YarnApplicationState.RUNNING, report.getYarnApplicationState()); + Assert.assertEquals(FinalApplicationStatus.UNDEFINED, report.getFinalApplicationStatus()); + Assert.assertEquals("http://127.0.0.1:8080/proxy/application_1586619723848_0088/", report.getTrackingUrl()); + Assert.assertEquals(40236, report.getRpcPort()); + Assert.assertEquals("host-name", report.getHost()); + + } catch (LoadException e) { + e.printStackTrace(); + Assert.fail(); + } + } +} From c7f00c00bd736231392289d8db7712a86114f068 Mon Sep 17 00:00:00 2001 From: xy720 Date: Tue, 25 Aug 2020 18:26:08 +0800 Subject: [PATCH 04/14] save code --- .../java/org/apache/doris/common/Config.java | 1 - .../doris/load/loadv2/SparkEtlJobHandler.java | 63 +++--- .../load/loadv2/SparkLauncherMonitors.java | 199 ++++++++++++++++++ .../doris/load/loadv2/SparkLoadAppHandle.java | 148 +++++++++++++ .../doris/load/loadv2/SparkLoadJob.java | 3 +- .../loadv2/SparkPendingTaskAttachment.java | 8 +- 6 files changed, 378 insertions(+), 44 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadAppHandle.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java index b9db19af7a2caa..df94bf9c8977b0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java @@ -546,7 +546,6 @@ public class Config extends ConfigBase { /** * Default yarn client path */ - @ConfField(mutable = true, masterOnly = true) public static String yarn_client_path = PaloFe.DORIS_HOME_DIR + "/lib/yarn-client/hadoop/bin/yarn"; @ConfField diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java index 48941e8cbca2db..ce03d0caa07bc8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java @@ -26,6 +26,7 @@ import org.apache.doris.common.util.CommandResult; import org.apache.doris.common.util.Util; import org.apache.doris.load.EtlStatus; +import org.apache.doris.load.loadv2.SparkLoadAppHandle.State; import org.apache.doris.load.loadv2.dpp.DppResult; import org.apache.doris.load.loadv2.etl.EtlJobConfig; import org.apache.doris.load.loadv2.etl.SparkEtlJob; @@ -45,10 +46,8 @@ import org.apache.logging.log4j.Logger; import org.apache.spark.launcher.SparkAppHandle; import org.apache.spark.launcher.SparkAppHandle.Listener; -import org.apache.spark.launcher.SparkAppHandle.State; import org.apache.spark.launcher.SparkLauncher; -import java.io.File; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.util.List; @@ -69,8 +68,9 @@ public class SparkEtlJobHandler { private static final String JOB_CONFIG_DIR = "configs"; private static final String ETL_JOB_NAME = "doris__%s"; // 5min - private static final int GET_APPID_MAX_RETRY_TIMES = 300; - private static final int GET_APPID_SLEEP_MS = 1000; + // private static final int GET_APPID_MAX_RETRY_TIMES = 300; + // private static final int GET_APPID_SLEEP_MS = 1000; + private static final int GET_APPID_TIMEOUT_MS = 300000; // 30s private static final long EXEC_CMD_TIMEOUT_MS = 30000L; // yarn command @@ -140,8 +140,8 @@ public void submitEtlJob(long loadJobId, String loadLabel, EtlJobConfig etlJobCo .setAppName(String.format(ETL_JOB_NAME, loadLabel)) .setSparkHome(sparkHome) .addAppArgs(jobConfigHdfsPath) - .redirectError() - .redirectOutput(new File(Config.sys_log_dir + "/spark-submitter.log")); + .redirectError(); + //.redirectOutput(new File(Config.sys_log_dir + "/spark-submitter.log")); // spark configs for (Map.Entry entry : resource.getSparkConfigs().entrySet()) { @@ -149,44 +149,35 @@ public void submitEtlJob(long loadJobId, String loadLabel, EtlJobConfig etlJobCo } // start app - SparkAppHandle handle = null; + SparkLoadAppHandle handle = null; State state = null; String appId = null; - int retry = 0; String errMsg = "start spark app failed. error: "; try { - handle = launcher.startApplication(new SparkAppListener()); + Process process = launcher.launch(); + handle = new SparkLoadAppHandle(process); + SparkLauncherMonitors.LogMonitor logMonitor = SparkLauncherMonitors.createLogMonitor(handle); + logMonitor.start(); + try { + logMonitor.join(); + appId = handle.getAppId(); + state = handle.getState(); + } catch (InterruptedException e) { + logMonitor.interrupt(); + throw new LoadException(errMsg + e.getMessage()); + } } catch (IOException e) { LOG.warn(errMsg, e); throw new LoadException(errMsg + e.getMessage()); } - while (retry++ < GET_APPID_MAX_RETRY_TIMES) { - appId = handle.getAppId(); - if (appId != null) { - break; - } - - // check state and retry - state = handle.getState(); - if (fromSparkState(state) == TEtlState.CANCELLED) { - throw new LoadException(errMsg + "spark app state: " + state.toString()); - } - if (retry >= GET_APPID_MAX_RETRY_TIMES) { - throw new LoadException(errMsg + "wait too much time for getting appid. spark app state: " - + state.toString()); - } + if (fromSparkState(state) == TEtlState.CANCELLED) { + throw new LoadException(errMsg + "spark app state: " + state.toString() + ", loadJobId:" + loadJobId); + } - // log - if (retry % 10 == 0) { - LOG.info("spark appid that handle get is null. load job id: {}, state: {}, retry times: {}", - loadJobId, state.toString(), retry); - } - try { - Thread.sleep(GET_APPID_SLEEP_MS); - } catch (InterruptedException e) { - LOG.warn(e.getMessage()); - } + if (appId == null) { + throw new LoadException(errMsg + "Failed to get appId from handle. spark app state: " + + state.toString() + ", loadJobId:" + loadJobId); } // success @@ -194,7 +185,7 @@ public void submitEtlJob(long loadJobId, String loadLabel, EtlJobConfig etlJobCo attachment.setHandle(handle); } - public EtlStatus getEtlJobStatus(SparkAppHandle handle, String appId, long loadJobId, String etlOutputPath, + public EtlStatus getEtlJobStatus(SparkLoadAppHandle handle, String appId, long loadJobId, String etlOutputPath, SparkResource resource, BrokerDesc brokerDesc) throws LoadException { EtlStatus status = new EtlStatus(); @@ -273,7 +264,7 @@ public EtlStatus getEtlJobStatus(SparkAppHandle handle, String appId, long loadJ return status; } - public void killEtlJob(SparkAppHandle handle, String appId, long loadJobId, SparkResource resource) throws LoadException { + public void killEtlJob(SparkLoadAppHandle handle, String appId, long loadJobId, SparkResource resource) throws LoadException { if (resource.isYarnMaster()) { Preconditions.checkNotNull(appId); // prepare yarn config diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java new file mode 100644 index 00000000000000..6d7a96e442b078 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java @@ -0,0 +1,199 @@ +package org.apache.doris.load.loadv2; + +import com.google.common.base.Preconditions; +import com.google.common.base.Splitter; +import com.google.common.base.Strings; + +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class SparkLauncherMonitors { + private static final Logger LOG = LogManager.getLogger(SparkLauncherMonitors.class); + // 5min + private static final long SUBMIT_APP_TIMEOUT_MS = 300 * 1000; + + private LogMonitor logMonitor; + + public static LogMonitor createLogMonitor(SparkLoadAppHandle handle) { + return new LogMonitor(handle); + } + + private static SparkLoadAppHandle.State fromYarnState(YarnApplicationState yarnState) { + switch (yarnState) { + case SUBMITTED: + case ACCEPTED: + return SparkLoadAppHandle.State.SUBMITTED; + case RUNNING: + return SparkLoadAppHandle.State.RUNNING; + case FINISHED: + return SparkLoadAppHandle.State.FINISHED; + case FAILED: + return SparkLoadAppHandle.State.FAILED; + case KILLED: + return SparkLoadAppHandle.State.KILLED; + default: + // NEW NEW_SAVING + return SparkLoadAppHandle.State.UNKNOWN; + } + } + + public static class LogMonitor extends Thread { + private final Process process; + private SparkLoadAppHandle handle; + private long submitTimeoutMs; + private boolean isStop; + + private static final String STATE = "state"; + private static final String QUEUE = "queue"; + private static final String START_TIME = "start time"; + private static final String FINAL_STATUS = "final status"; + private static final String URL = "tracking URL"; + private static final String USER = "user"; + + public LogMonitor(SparkLoadAppHandle handle) { + this.handle = handle; + this.process = handle.getProcess(); + this.isStop = false; + } + + public void setSubmitTimeoutMs(long submitTimeoutMs) { + this.submitTimeoutMs = submitTimeoutMs; + } + + // Monitor the process's output + @Override + public void run() { + BufferedReader outReader = null; + String line = null; + long startTime = System.currentTimeMillis(); + try { + Preconditions.checkState(process.isAlive()); + outReader = new BufferedReader(new InputStreamReader(process.getInputStream())); + while (!isStop && (line = outReader.readLine()) != null) { + LOG.info("Monitor Log: " + line); + // parse state and appId + if (line.contains(STATE)) { + SparkLoadAppHandle.State oldState = handle.getState(); + SparkLoadAppHandle.State newState = oldState; + // 1. state + String state = regexGetState(line); + if (state != null) { + YarnApplicationState yarnState = YarnApplicationState.valueOf(state); + newState = fromYarnState(yarnState); + if (newState != oldState) { + handle.setState(newState); + } + } + // 2. appId + String appId = regexGetAppId(line); + if (appId != null) { + if (!appId.equals(handle.getAppId())) { + handle.setAppId(appId); + } + } + + LOG.info("spark appId that handle get is {}, state: {}", handle.getAppId(), handle.getState().toString()); + switch (newState) { + case UNKNOWN: + case CONNECTED: + case SUBMITTED: + // If the app stays in the UNKNOWN/CONNECTED/SUBMITTED state for more than submitTimeoutMs + // stop monitoring and kill the process + if (System.currentTimeMillis() - startTime > submitTimeoutMs) { + isStop = true; + handle.kill(); + } + break; + case RUNNING: + case FINISHED: + // There's no need to parse all logs of handle process to get all the information. + // As soon as the state changes to RUNNING/KILLED/FAILED/FINISHED/LOST, + // stop monitoring but keep the process alive. + isStop = true; + break; + case KILLED: + case FAILED: + case LOST: + // If the state changes to KILLED/FAILED/LOST, + // stop monitoring and kill the process + isStop = true; + handle.kill(); + break; + default: + Preconditions.checkState(false, "wrong spark app state"); + } + } + // parse other values + else if (line.contains(QUEUE) || line.contains(START_TIME) || line.contains(FINAL_STATUS) || + line.contains(URL) || line.contains(USER)) { + String value = getValue(line); + if (!Strings.isNullOrEmpty(value)) { + try { + if (line.contains(QUEUE)) { + handle.setQueue(value); + } else if (line.contains(START_TIME)) { + handle.setStartTime(Long.parseLong(value)); + } else if (line.contains(FINAL_STATUS)) { + handle.setFinalStatus(FinalApplicationStatus.valueOf(value)); + } else if (line.contains(URL)) { + handle.setUrl(value); + } else if (line.contains(USER)) { + handle.setUser(value); + } + } catch (IllegalArgumentException e) { + LOG.warn("parse log encounter an error, line: {}, msg: {}", line, e.getMessage()); + } + } + } + } + } catch (Exception e) { + LOG.warn("Exception monitoring process.", e); + } finally { + try { + if (outReader != null) { + outReader.close(); + } + } catch (IOException e) { + LOG.warn("close buffered reader error", e); + } + } + } + + private static String getValue(String line) { + String result = null; + List entry = Splitter.onPattern(":").trimResults().splitToList(line); + if (entry.size() == 2) { + result = entry.get(1); + } + return result; + } + + // Regex convert str such as "XXX (state: ACCEPTED)" to "ACCEPTED" + private static String regexGetState(String line) { + String result = null; + Matcher stateMatcher = Pattern.compile("(?<=\\(state: )(.+?)(?=\\))").matcher(line); + if (stateMatcher.find()) { + result = stateMatcher.group(); + } + return result; + } + + private static String regexGetAppId(String line) { + String result = null; + Matcher appIdMatcher = Pattern.compile("application_[0-9]+_[0-9]+").matcher(line); + if (appIdMatcher.find()) { + result = appIdMatcher.group(); + } + return result; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadAppHandle.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadAppHandle.java new file mode 100644 index 00000000000000..2807af9e16f83e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadAppHandle.java @@ -0,0 +1,148 @@ +package org.apache.doris.load.loadv2; + +import com.google.common.collect.Lists; + +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Iterator; +import java.util.List; + +public class SparkLoadAppHandle { + private static final Logger LOG = LogManager.getLogger(SparkLoadAppHandle.class); + // 5min + private static final long SUBMIT_APP_TIMEOUT_MS = 300 * 1000; + + private Process process; + + private String appId; + private State state; + private String queue; + private long startTime; + private FinalApplicationStatus finalStatus; + private String trackingUrl; + private String user; + + private List listeners; + + public interface Listener { + void stateChanged(SparkLoadAppHandle handle); + + void infoChanged(SparkLoadAppHandle handle); + } + + public static enum State { + UNKNOWN(false), + CONNECTED(false), + SUBMITTED(false), + RUNNING(false), + FINISHED(true), + FAILED(true), + KILLED(true), + LOST(true); + + private final boolean isFinal; + + private State(boolean isFinal) { + this.isFinal = isFinal; + } + + public boolean isFinal() { + return this.isFinal; + } + } + + public SparkLoadAppHandle(Process process) { + this.state = State.UNKNOWN; + this.process = process; + } + + public void addListener(Listener listener) { + if (this.listeners == null) { + this.listeners = Lists.newArrayList(); + } + + this.listeners.add(listener); + } + + public void stop() { + } + + public void kill() { + this.setState(State.KILLED); + if (this.process != null) { + if (this.process.isAlive()) { + this.process.destroyForcibly(); + } + this.process = null; + } + } + + public State getState() { return this.state; } + + public String getAppId() { return this.appId; } + + public String getQueue() { return this.queue; } + + public Process getProcess() { return this.process; } + + public long getStartTime() { return this.startTime; } + + public FinalApplicationStatus getFinalStatus() { return this.finalStatus; } + + public String getUrl() { return this.trackingUrl; } + + public String getUser() { return this.user; } + + public void setState(State state) { + this.state = state; + this.fireEvent(false); + } + + public void setAppId(String appId) { + this.appId = appId; + this.fireEvent(true); + } + + public void setQueue(String queue) { + this.queue = queue; + this.fireEvent(true); + } + + public void setStartTime(long startTime) { + this.startTime = startTime; + this.fireEvent(true); + } + + public void setFinalStatus(FinalApplicationStatus status) { + this.finalStatus = status; + this.fireEvent(true); + } + + public void setUrl(String url) { + this.trackingUrl = url; + this.fireEvent(true); + } + + public void setUser(String user) { + this.user = user; + this.fireEvent(true); + } + + private void fireEvent(boolean isInfoChanged) { + if (this.listeners != null) { + Iterator iterator = this.listeners.iterator(); + + while (iterator.hasNext()) { + Listener l = (Listener)iterator.next(); + if (isInfoChanged) { + l.infoChanged(this); + } else { + l.stateChanged(this); + } + } + } + + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java index f1dff69c78399a..4f3266194e4ba3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java @@ -85,7 +85,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.spark.launcher.SparkAppHandle; import com.google.common.base.Preconditions; import com.google.common.base.Strings; @@ -127,7 +126,7 @@ public class SparkLoadJob extends BulkLoadJob { // --- members below not persist --- private ResourceDesc resourceDesc; // for spark standalone - private SparkAppHandle sparkAppHandle; + private SparkLoadAppHandle sparkAppHandle; // for straggler wait long time to commit transaction private long quorumFinishTimestamp = -1; // below for push task diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkPendingTaskAttachment.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkPendingTaskAttachment.java index 311ca3bab70380..4189defdb693a7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkPendingTaskAttachment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkPendingTaskAttachment.java @@ -17,10 +17,8 @@ package org.apache.doris.load.loadv2; -import org.apache.spark.launcher.SparkAppHandle; - public class SparkPendingTaskAttachment extends TaskAttachment { - private SparkAppHandle handle; + private SparkLoadAppHandle handle; private String appId; private String outputPath; @@ -28,7 +26,7 @@ public SparkPendingTaskAttachment(long taskId) { super(taskId); } - public SparkAppHandle getHandle() { + public SparkLoadAppHandle getHandle() { return handle; } @@ -40,7 +38,7 @@ public void setAppId(String appId) { this.appId = appId; } - public void setHandle(SparkAppHandle handle) { + public void setHandle(SparkLoadAppHandle handle) { this.handle = handle; } From 635e1152170e14daec5170b79607beb3f93a6d97 Mon Sep 17 00:00:00 2001 From: xy720 Date: Tue, 25 Aug 2020 18:36:35 +0800 Subject: [PATCH 05/14] license --- .../load/loadv2/SparkLauncherMonitors.java | 17 +++++++++++++++++ .../doris/load/loadv2/SparkLoadAppHandle.java | 17 +++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java index 6d7a96e442b078..83806f3bf8fd68 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java @@ -1,3 +1,20 @@ +// 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.doris.load.loadv2; import com.google.common.base.Preconditions; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadAppHandle.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadAppHandle.java index 2807af9e16f83e..5d75ae9392a6f1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadAppHandle.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadAppHandle.java @@ -1,3 +1,20 @@ +// 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.doris.load.loadv2; import com.google.common.collect.Lists; From 1eb1a7143a58f5257066180a7f036b7ad54917a3 Mon Sep 17 00:00:00 2001 From: xy720 Date: Tue, 25 Aug 2020 18:49:34 +0800 Subject: [PATCH 06/14] save code --- .../doris/load/loadv2/SparkEtlJobHandler.java | 11 +- .../load/loadv2/SparkEtlJobHandlerTest.java | 143 +++++++++--------- .../doris/load/loadv2/SparkLoadJobTest.java | 22 +-- 3 files changed, 84 insertions(+), 92 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java index ce03d0caa07bc8..617ec444c9e873 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java @@ -44,8 +44,6 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.spark.launcher.SparkAppHandle; -import org.apache.spark.launcher.SparkAppHandle.Listener; import org.apache.spark.launcher.SparkLauncher; import java.io.IOException; @@ -68,8 +66,6 @@ public class SparkEtlJobHandler { private static final String JOB_CONFIG_DIR = "configs"; private static final String ETL_JOB_NAME = "doris__%s"; // 5min - // private static final int GET_APPID_MAX_RETRY_TIMES = 300; - // private static final int GET_APPID_SLEEP_MS = 1000; private static final int GET_APPID_TIMEOUT_MS = 300000; // 30s private static final long EXEC_CMD_TIMEOUT_MS = 30000L; @@ -77,14 +73,14 @@ public class SparkEtlJobHandler { private static final String YARN_STATUS_CMD = "%s --config %s application -status %s"; private static final String YARN_KILL_CMD = "%s --config %s application -kill %s"; - class SparkAppListener implements Listener { + class SparkAppListener implements SparkLoadAppHandle.Listener { @Override - public void stateChanged(SparkAppHandle sparkAppHandle) { + public void stateChanged(SparkLoadAppHandle sparkAppHandle) { LOG.info("get spark state changed: {}, app id: {}", sparkAppHandle.getState(), sparkAppHandle.getAppId()); } @Override - public void infoChanged(SparkAppHandle sparkAppHandle) { + public void infoChanged(SparkLoadAppHandle sparkAppHandle) { LOG.info("get spark info changed: {}, app id: {}", sparkAppHandle.getState(), sparkAppHandle.getAppId()); } } @@ -157,6 +153,7 @@ public void submitEtlJob(long loadJobId, String loadLabel, EtlJobConfig etlJobCo Process process = launcher.launch(); handle = new SparkLoadAppHandle(process); SparkLauncherMonitors.LogMonitor logMonitor = SparkLauncherMonitors.createLogMonitor(handle); + logMonitor.setSubmitTimeoutMs(GET_APPID_TIMEOUT_MS); logMonitor.start(); try { logMonitor.join(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkEtlJobHandlerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkEtlJobHandlerTest.java index 32306402f01de4..e70c63f0c06531 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkEtlJobHandlerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkEtlJobHandlerTest.java @@ -30,13 +30,11 @@ import org.apache.doris.catalog.SparkResource; import org.apache.doris.common.Config; import org.apache.doris.common.GenericPool; -import org.apache.doris.common.LoadException; import org.apache.doris.common.UserException; import org.apache.doris.common.util.BrokerUtil; import org.apache.doris.common.util.CommandResult; import org.apache.doris.common.util.Util; import org.apache.doris.load.EtlStatus; -import org.apache.doris.load.loadv2.etl.EtlJobConfig; import org.apache.doris.thrift.TBrokerFileStatus; import org.apache.doris.thrift.TBrokerListPathRequest; import org.apache.doris.thrift.TBrokerListResponse; @@ -48,9 +46,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import org.apache.spark.launcher.SparkAppHandle; -import org.apache.spark.launcher.SparkAppHandle.State; -import org.apache.spark.launcher.SparkLauncher; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -137,75 +132,75 @@ public void setUp() { .SparkLibrary("", "", SparkRepository.SparkLibrary.LibType.SPARK2X, 0L)); } - @Test - public void testSubmitEtlJob(@Mocked BrokerUtil brokerUtil, @Mocked SparkLauncher launcher, - @Injectable SparkAppHandle handle) throws IOException, LoadException { - new Expectations() { - { - launcher.startApplication((SparkAppHandle.Listener) any); - result = handle; - handle.getAppId(); - returns(null, null, appId); - handle.getState(); - returns(State.CONNECTED, State.SUBMITTED, State.RUNNING); - } - }; - - EtlJobConfig etlJobConfig = new EtlJobConfig(Maps.newHashMap(), etlOutputPath, label, null); - SparkResource resource = new SparkResource(resourceName); - new Expectations(resource) { - { - resource.prepareArchive(); - result = archive; - } - }; - - Map sparkConfigs = resource.getSparkConfigs(); - sparkConfigs.put("spark.master", "yarn"); - sparkConfigs.put("spark.submit.deployMode", "cluster"); - sparkConfigs.put("spark.hadoop.yarn.resourcemanager.address", "127.0.0.1:9999"); - BrokerDesc brokerDesc = new BrokerDesc(broker, Maps.newHashMap()); - SparkPendingTaskAttachment attachment = new SparkPendingTaskAttachment(pendingTaskId); - SparkEtlJobHandler handler = new SparkEtlJobHandler(); - handler.submitEtlJob(loadJobId, label, etlJobConfig, resource, brokerDesc, attachment); - - // check submit etl job success - Assert.assertEquals(appId, attachment.getAppId()); - Assert.assertEquals(handle, attachment.getHandle()); - } - - @Test(expected = LoadException.class) - public void testSubmitEtlJobFailed(@Mocked BrokerUtil brokerUtil, @Mocked SparkLauncher launcher, - @Injectable SparkAppHandle handle) throws IOException, LoadException { - new Expectations() { - { - launcher.startApplication((SparkAppHandle.Listener) any); - result = handle; - handle.getAppId(); - result = null; - handle.getState(); - returns(State.CONNECTED, State.SUBMITTED, State.FAILED); - } - }; - - EtlJobConfig etlJobConfig = new EtlJobConfig(Maps.newHashMap(), etlOutputPath, label, null); - SparkResource resource = new SparkResource(resourceName); - new Expectations(resource) { - { - resource.prepareArchive(); - result = archive; - } - }; - - Map sparkConfigs = resource.getSparkConfigs(); - sparkConfigs.put("spark.master", "yarn"); - sparkConfigs.put("spark.submit.deployMode", "cluster"); - sparkConfigs.put("spark.hadoop.yarn.resourcemanager.address", "127.0.0.1:9999"); - BrokerDesc brokerDesc = new BrokerDesc(broker, Maps.newHashMap()); - SparkPendingTaskAttachment attachment = new SparkPendingTaskAttachment(pendingTaskId); - SparkEtlJobHandler handler = new SparkEtlJobHandler(); - handler.submitEtlJob(loadJobId, label, etlJobConfig, resource, brokerDesc, attachment); - } +// @Test +// public void testSubmitEtlJob(@Mocked BrokerUtil brokerUtil, @Mocked SparkLauncher launcher, +// @Injectable SparkLoadAppAppHandle handle) throws IOException, LoadException { +// new Expectations() { +// { +// launcher.startApplication((SparkLoadAppAppHandle.Listener) any); +// result = handle; +// handle.getAppId(); +// returns(null, null, appId); +// handle.getState(); +// returns(SparkLoadAppHandle.State.CONNECTED, SparkLoadAppHandle.State.SUBMITTED, SparkLoadAppHandle.State.RUNNING); +// } +// }; +// +// EtlJobConfig etlJobConfig = new EtlJobConfig(Maps.newHashMap(), etlOutputPath, label, null); +// SparkResource resource = new SparkResource(resourceName); +// new Expectations(resource) { +// { +// resource.prepareArchive(); +// result = archive; +// } +// }; +// +// Map sparkConfigs = resource.getSparkConfigs(); +// sparkConfigs.put("spark.master", "yarn"); +// sparkConfigs.put("spark.submit.deployMode", "cluster"); +// sparkConfigs.put("spark.hadoop.yarn.resourcemanager.address", "127.0.0.1:9999"); +// BrokerDesc brokerDesc = new BrokerDesc(broker, Maps.newHashMap()); +// SparkPendingTaskAttachment attachment = new SparkPendingTaskAttachment(pendingTaskId); +// SparkEtlJobHandler handler = new SparkEtlJobHandler(); +// handler.submitEtlJob(loadJobId, label, etlJobConfig, resource, brokerDesc, attachment); +// +// // check submit etl job success +// Assert.assertEquals(appId, attachment.getAppId()); +// Assert.assertEquals(handle, attachment.getHandle()); +// } + +// @Test(expected = LoadException.class) +// public void testSubmitEtlJobFailed(@Mocked BrokerUtil brokerUtil, @Mocked SparkLauncher launcher, +// @Injectable SparkAppHandle handle) throws IOException, LoadException { +// new Expectations() { +// { +// launcher.startApplication((SparkAppHandle.Listener) any); +// result = handle; +// handle.getAppId(); +// result = null; +// handle.getState(); +// returns(State.CONNECTED, State.SUBMITTED, State.FAILED); +// } +// }; +// +// EtlJobConfig etlJobConfig = new EtlJobConfig(Maps.newHashMap(), etlOutputPath, label, null); +// SparkResource resource = new SparkResource(resourceName); +// new Expectations(resource) { +// { +// resource.prepareArchive(); +// result = archive; +// } +// }; +// +// Map sparkConfigs = resource.getSparkConfigs(); +// sparkConfigs.put("spark.master", "yarn"); +// sparkConfigs.put("spark.submit.deployMode", "cluster"); +// sparkConfigs.put("spark.hadoop.yarn.resourcemanager.address", "127.0.0.1:9999"); +// BrokerDesc brokerDesc = new BrokerDesc(broker, Maps.newHashMap()); +// SparkPendingTaskAttachment attachment = new SparkPendingTaskAttachment(pendingTaskId); +// SparkEtlJobHandler handler = new SparkEtlJobHandler(); +// handler.submitEtlJob(loadJobId, label, etlJobConfig, resource, brokerDesc, attachment); +// } @Test public void testGetEtlJobStatus(@Mocked BrokerUtil brokerUtil, @Mocked Util util, @Mocked CommandResult commandResult, diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java index 671de5a73516d8..e86cad0bdb9b9d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java @@ -17,15 +17,18 @@ package org.apache.doris.load.loadv2; +import mockit.Expectations; +import mockit.Injectable; +import mockit.Mocked; + import org.apache.doris.analysis.BrokerDesc; import org.apache.doris.analysis.DataDescription; -import org.apache.doris.analysis.ResourceDesc; import org.apache.doris.analysis.LabelName; import org.apache.doris.analysis.LoadStmt; +import org.apache.doris.analysis.ResourceDesc; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.ResourceMgr; import org.apache.doris.catalog.MaterializedIndex; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; @@ -33,6 +36,7 @@ import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.RangePartitionInfo; import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.ResourceMgr; import org.apache.doris.catalog.SparkResource; import org.apache.doris.catalog.Tablet; import org.apache.doris.common.AnalysisException; @@ -57,13 +61,9 @@ import org.apache.doris.transaction.TabletCommitInfo; import org.apache.doris.transaction.TransactionState; import org.apache.doris.transaction.TransactionState.LoadJobSourceType; - import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import mockit.Expectations; -import mockit.Injectable; -import mockit.Mocked; -import org.apache.spark.launcher.SparkAppHandle; + import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -269,7 +269,7 @@ public void testUpdateEtlStatusRunning(@Mocked Catalog catalog, @Injectable Stri new Expectations() { { - handler.getEtlJobStatus((SparkAppHandle) any, appId, anyLong, etlOutputPath, + handler.getEtlJobStatus((SparkLoadAppHandle) any, appId, anyLong, etlOutputPath, (SparkResource) any, (BrokerDesc) any); result = status; } @@ -292,7 +292,7 @@ public void testUpdateEtlStatusCancelled(@Mocked Catalog catalog, @Injectable St new Expectations() { { - handler.getEtlJobStatus((SparkAppHandle) any, appId, anyLong, etlOutputPath, + handler.getEtlJobStatus((SparkLoadAppHandle) any, appId, anyLong, etlOutputPath, (SparkResource) any, (BrokerDesc) any); result = status; } @@ -312,7 +312,7 @@ public void testUpdateEtlStatusFinishedQualityFailed(@Mocked Catalog catalog, @I new Expectations() { { - handler.getEtlJobStatus((SparkAppHandle) any, appId, anyLong, etlOutputPath, + handler.getEtlJobStatus((SparkLoadAppHandle) any, appId, anyLong, etlOutputPath, (SparkResource) any, (BrokerDesc) any); result = status; } @@ -343,7 +343,7 @@ public void testUpdateEtlStatusFinishedAndCommitTransaction( new Expectations() { { - handler.getEtlJobStatus((SparkAppHandle) any, appId, anyLong, etlOutputPath, + handler.getEtlJobStatus((SparkLoadAppHandle) any, appId, anyLong, etlOutputPath, (SparkResource) any, (BrokerDesc) any); result = status; handler.getEtlFilePaths(etlOutputPath, (BrokerDesc) any); From f85f6926f33caf0b0297180466ec98b5ff8176e0 Mon Sep 17 00:00:00 2001 From: xy720 Date: Tue, 25 Aug 2020 19:06:43 +0800 Subject: [PATCH 07/14] save code --- .../java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java index 617ec444c9e873..9e8dda78824530 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java @@ -224,7 +224,7 @@ public EtlStatus getEtlJobStatus(SparkLoadAppHandle handle, String appId, long l status.setFailMsg("yarn app state: " + state.toString()); } } - status.setTrackingUrl(report.getTrackingUrl()); + status.setTrackingUrl(handle.getUrl()); status.setProgress((int) (report.getProgress() * 100)); } else { // state from handle From 77deaef42479f84d2b3ee4240e541036996455c3 Mon Sep 17 00:00:00 2001 From: xy720 Date: Wed, 26 Aug 2020 17:08:35 +0800 Subject: [PATCH 08/14] add ut --- .../apache/doris/catalog/SparkResource.java | 11 + .../doris/load/loadv2/SparkEtlJobHandler.java | 33 ++- .../load/loadv2/SparkLauncherMonitors.java | 41 ++- .../doris/load/loadv2/SparkLoadJob.java | 23 +- .../load/loadv2/SparkEtlJobHandlerTest.java | 186 ++++++------ .../loadv2/SparkLauncherMonitorsTest.java | 74 +++++ .../resources/data/spark_launcher_monitor.log | 264 ++++++++++++++++++ 7 files changed, 510 insertions(+), 122 deletions(-) create mode 100644 fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorsTest.java create mode 100644 fe/fe-core/src/test/resources/data/spark_launcher_monitor.log diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java index cd03a21b009ceb..e3bb1cb8f8e035 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.BrokerDesc; import org.apache.doris.analysis.ResourceDesc; +import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.LoadException; import org.apache.doris.common.Pair; @@ -30,6 +31,7 @@ import com.google.common.collect.Maps; import com.google.gson.annotations.SerializedName; +import java.io.File; import java.util.Map; /** @@ -176,6 +178,15 @@ public synchronized String prepareYarnConfig() throws LoadException { return yarnConfigFiles.getConfigDir(); } + public String getYarnClientPath() throws LoadException { + String yarnClientPath = Config.yarn_client_path; + File file = new File(yarnClientPath); + if (!file.exists() || !file.isFile()) { + throw new LoadException("yarn client does not exist in path: " + yarnClientPath); + } + return yarnClientPath; + } + public boolean isYarnMaster() { return getMaster().equalsIgnoreCase(YARN_MASTER); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java index 9e8dda78824530..52de11da6babc7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java @@ -20,6 +20,7 @@ import org.apache.doris.analysis.BrokerDesc; import org.apache.doris.catalog.SparkResource; import org.apache.doris.common.Config; +import org.apache.doris.common.FeConstants; import org.apache.doris.common.LoadException; import org.apache.doris.common.UserException; import org.apache.doris.common.util.BrokerUtil; @@ -66,7 +67,7 @@ public class SparkEtlJobHandler { private static final String JOB_CONFIG_DIR = "configs"; private static final String ETL_JOB_NAME = "doris__%s"; // 5min - private static final int GET_APPID_TIMEOUT_MS = 300000; + private static final long GET_APPID_TIMEOUT_MS = 300000L; // 30s private static final long EXEC_CMD_TIMEOUT_MS = 30000L; // yarn command @@ -137,7 +138,6 @@ public void submitEtlJob(long loadJobId, String loadLabel, EtlJobConfig etlJobCo .setSparkHome(sparkHome) .addAppArgs(jobConfigHdfsPath) .redirectError(); - //.redirectOutput(new File(Config.sys_log_dir + "/spark-submitter.log")); // spark configs for (Map.Entry entry : resource.getSparkConfigs().entrySet()) { @@ -152,17 +152,20 @@ public void submitEtlJob(long loadJobId, String loadLabel, EtlJobConfig etlJobCo try { Process process = launcher.launch(); handle = new SparkLoadAppHandle(process); - SparkLauncherMonitors.LogMonitor logMonitor = SparkLauncherMonitors.createLogMonitor(handle); - logMonitor.setSubmitTimeoutMs(GET_APPID_TIMEOUT_MS); - logMonitor.start(); - try { - logMonitor.join(); - appId = handle.getAppId(); - state = handle.getState(); - } catch (InterruptedException e) { - logMonitor.interrupt(); - throw new LoadException(errMsg + e.getMessage()); + handle.addListener(new SparkAppListener()); + if (!FeConstants.runningUnitTest) { + SparkLauncherMonitors.LogMonitor logMonitor = SparkLauncherMonitors.createLogMonitor(handle); + logMonitor.setSubmitTimeoutMs(GET_APPID_TIMEOUT_MS); + logMonitor.start(); + try { + logMonitor.join(); + } catch (InterruptedException e) { + logMonitor.interrupt(); + throw new LoadException(errMsg + e.getMessage()); + } } + appId = handle.getAppId(); + state = handle.getState(); } catch (IOException e) { LOG.warn(errMsg, e); throw new LoadException(errMsg + e.getMessage()); @@ -191,7 +194,7 @@ public EtlStatus getEtlJobStatus(SparkLoadAppHandle handle, String appId, long l // prepare yarn config String configDir = resource.prepareYarnConfig(); // yarn client path - String yarnClient = Config.yarn_client_path; + String yarnClient = resource.getYarnClientPath(); // command: yarn --config configDir application -status appId String yarnStatusCmd = String.format(YARN_STATUS_CMD, yarnClient, configDir, appId); LOG.info(yarnStatusCmd); @@ -224,7 +227,7 @@ public EtlStatus getEtlJobStatus(SparkLoadAppHandle handle, String appId, long l status.setFailMsg("yarn app state: " + state.toString()); } } - status.setTrackingUrl(handle.getUrl()); + status.setTrackingUrl(handle.getUrl() != null? handle.getUrl() : report.getTrackingUrl()); status.setProgress((int) (report.getProgress() * 100)); } else { // state from handle @@ -267,7 +270,7 @@ public void killEtlJob(SparkLoadAppHandle handle, String appId, long loadJobId, // prepare yarn config String configDir = resource.prepareYarnConfig(); // yarn client path - String yarnClient = Config.yarn_client_path; + String yarnClient = resource.getYarnClientPath(); // command: yarn --config configDir application -kill appId String yarnKillCmd = String.format(YARN_KILL_CMD, yarnClient, configDir, appId); LOG.info(yarnKillCmd); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java index 83806f3bf8fd68..fc38d344eea522 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java @@ -35,10 +35,6 @@ public class SparkLauncherMonitors { private static final Logger LOG = LogManager.getLogger(SparkLauncherMonitors.class); - // 5min - private static final long SUBMIT_APP_TIMEOUT_MS = 300 * 1000; - - private LogMonitor logMonitor; public static LogMonitor createLogMonitor(SparkLoadAppHandle handle) { return new LogMonitor(handle); @@ -63,6 +59,9 @@ private static SparkLoadAppHandle.State fromYarnState(YarnApplicationState yarnS } } + // This monitor is use for monitoring the spark launcher process. + // User can use this monitor to get real-time `appId`, `state` and `tracking-url` + // of spark launcher by reading and analyze the output of process. public static class LogMonitor extends Thread { private final Process process; private SparkLoadAppHandle handle; @@ -76,31 +75,39 @@ public static class LogMonitor extends Thread { private static final String URL = "tracking URL"; private static final String USER = "user"; + // 5min + private static final long DEFAULT_SUBMIT_TIMEOUT_MS = 300000L; + public LogMonitor(SparkLoadAppHandle handle) { this.handle = handle; this.process = handle.getProcess(); this.isStop = false; + setSubmitTimeoutMs(DEFAULT_SUBMIT_TIMEOUT_MS); } public void setSubmitTimeoutMs(long submitTimeoutMs) { this.submitTimeoutMs = submitTimeoutMs; } - // Monitor the process's output + // Normally, log monitor will automatically stop if the spark app state changes + // to RUNNING. + // But if the spark app state changes to FAILED/KILLED/LOST, log monitor will stop + // and kill the spark launcher process. + // There is a `submitTimeout` for preventing the spark app state from staying in + // UNKNOWN/SUBMITTED for a long time. @Override public void run() { BufferedReader outReader = null; String line = null; long startTime = System.currentTimeMillis(); try { - Preconditions.checkState(process.isAlive()); outReader = new BufferedReader(new InputStreamReader(process.getInputStream())); while (!isStop && (line = outReader.readLine()) != null) { - LOG.info("Monitor Log: " + line); + LOG.info("monitor log: " + line); + SparkLoadAppHandle.State oldState = handle.getState(); + SparkLoadAppHandle.State newState = oldState; // parse state and appId if (line.contains(STATE)) { - SparkLoadAppHandle.State oldState = handle.getState(); - SparkLoadAppHandle.State newState = oldState; // 1. state String state = regexGetState(line); if (state != null) { @@ -118,7 +125,7 @@ public void run() { } } - LOG.info("spark appId that handle get is {}, state: {}", handle.getAppId(), handle.getState().toString()); + LOG.debug("spark appId that handle get is {}, state: {}", handle.getAppId(), handle.getState().toString()); switch (newState) { case UNKNOWN: case CONNECTED: @@ -133,7 +140,7 @@ public void run() { case RUNNING: case FINISHED: // There's no need to parse all logs of handle process to get all the information. - // As soon as the state changes to RUNNING/KILLED/FAILED/FINISHED/LOST, + // As soon as the state changes to RUNNING/FINISHED, // stop monitoring but keep the process alive. isStop = true; break; @@ -185,16 +192,21 @@ else if (line.contains(QUEUE) || line.contains(START_TIME) || line.contains(FINA } } + // e.g. + // input: "final status: SUCCEEDED" + // output: "SUCCEEDED" private static String getValue(String line) { String result = null; - List entry = Splitter.onPattern(":").trimResults().splitToList(line); + List entry = Splitter.onPattern(":").trimResults().limit(2).splitToList(line); if (entry.size() == 2) { result = entry.get(1); } return result; } - // Regex convert str such as "XXX (state: ACCEPTED)" to "ACCEPTED" + // e.g. + // input: "Application report for application_1573630236805_6864759 (state: ACCEPTED)" + // output: "ACCEPTED" private static String regexGetState(String line) { String result = null; Matcher stateMatcher = Pattern.compile("(?<=\\(state: )(.+?)(?=\\))").matcher(line); @@ -204,6 +216,9 @@ private static String regexGetState(String line) { return result; } + // e.g. + // input: "Application report for application_1573630236805_6864759 (state: ACCEPTED)" + // output: "application_1573630236805_6864759" private static String regexGetAppId(String line) { String result = null; Matcher appIdMatcher = Pattern.compile("application_[0-9]+_[0-9]+").matcher(line); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java index 4f3266194e4ba3..6576be232f6b9a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java @@ -20,15 +20,14 @@ import org.apache.doris.analysis.BrokerDesc; import org.apache.doris.analysis.CastExpr; import org.apache.doris.analysis.DescriptorTable; -import org.apache.doris.analysis.ResourceDesc; import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.ResourceDesc; import org.apache.doris.analysis.SlotDescriptor; import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.Resource; import org.apache.doris.catalog.FsBroker; import org.apache.doris.catalog.MaterializedIndex; import org.apache.doris.catalog.MaterializedIndex.IndexExtState; @@ -36,6 +35,7 @@ import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.Resource; import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.SparkResource; import org.apache.doris.catalog.Tablet; @@ -82,10 +82,6 @@ import org.apache.doris.transaction.TransactionState.LoadJobSourceType; import org.apache.doris.transaction.TransactionState.TxnCoordinator; import org.apache.doris.transaction.TransactionState.TxnSourceType; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Lists; @@ -93,6 +89,9 @@ import com.google.common.collect.Sets; import com.google.gson.annotations.SerializedName; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; @@ -126,7 +125,7 @@ public class SparkLoadJob extends BulkLoadJob { // --- members below not persist --- private ResourceDesc resourceDesc; // for spark standalone - private SparkLoadAppHandle sparkAppHandle; + private SparkLoadAppHandle sparkLoadAppHandle; // for straggler wait long time to commit transaction private long quorumFinishTimestamp = -1; // below for push task @@ -229,7 +228,7 @@ private void onPendingTaskFinished(SparkPendingTaskAttachment attachment) { // add task id into finishedTaskIds finishedTaskIds.add(attachment.getTaskId()); - sparkAppHandle = attachment.getHandle(); + sparkLoadAppHandle = attachment.getHandle(); appId = attachment.getAppId(); etlOutputPath = attachment.getOutputPath(); @@ -277,7 +276,7 @@ public void updateEtlStatus() throws Exception { // get etl status SparkEtlJobHandler handler = new SparkEtlJobHandler(); - EtlStatus status = handler.getEtlJobStatus(sparkAppHandle, appId, id, etlOutputPath, sparkResource, brokerDesc); + EtlStatus status = handler.getEtlJobStatus(sparkLoadAppHandle, appId, id, etlOutputPath, sparkResource, brokerDesc); writeLock(); try { switch (status.getState()) { @@ -629,9 +628,9 @@ private void clearJob() { LOG.debug("kill etl job and delete etl files. id: {}, state: {}", id, state); SparkEtlJobHandler handler = new SparkEtlJobHandler(); if (state == JobState.CANCELLED) { - if ((!Strings.isNullOrEmpty(appId) && sparkResource.isYarnMaster()) || sparkAppHandle != null) { + if ((!Strings.isNullOrEmpty(appId) && sparkResource.isYarnMaster()) || sparkLoadAppHandle != null) { try { - handler.killEtlJob(sparkAppHandle, appId, id, sparkResource); + handler.killEtlJob(sparkLoadAppHandle, appId, id, sparkResource); } catch (Exception e) { LOG.warn("kill etl job failed. id: {}, state: {}", id, state, e); } @@ -660,7 +659,7 @@ private void clearJob() { } } // clear job infos that not persist - sparkAppHandle = null; + sparkLoadAppHandle = null; resourceDesc = null; tableToLoadPartitions.clear(); indexToPushBrokerReaderParams.clear(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkEtlJobHandlerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkEtlJobHandlerTest.java index e70c63f0c06531..904a8b1718ba00 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkEtlJobHandlerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkEtlJobHandlerTest.java @@ -29,12 +29,15 @@ import org.apache.doris.catalog.FsBroker; import org.apache.doris.catalog.SparkResource; import org.apache.doris.common.Config; +import org.apache.doris.common.FeConstants; import org.apache.doris.common.GenericPool; +import org.apache.doris.common.LoadException; import org.apache.doris.common.UserException; import org.apache.doris.common.util.BrokerUtil; import org.apache.doris.common.util.CommandResult; import org.apache.doris.common.util.Util; import org.apache.doris.load.EtlStatus; +import org.apache.doris.load.loadv2.etl.EtlJobConfig; import org.apache.doris.thrift.TBrokerFileStatus; import org.apache.doris.thrift.TBrokerListPathRequest; import org.apache.doris.thrift.TBrokerListResponse; @@ -46,6 +49,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import org.apache.spark.launcher.SparkLauncher; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -115,6 +119,7 @@ public class SparkEtlJobHandlerTest { @Before public void setUp() { + FeConstants.runningUnitTest = true; loadJobId = 0L; label = "label0"; resourceName = "spark0"; @@ -132,79 +137,78 @@ public void setUp() { .SparkLibrary("", "", SparkRepository.SparkLibrary.LibType.SPARK2X, 0L)); } -// @Test -// public void testSubmitEtlJob(@Mocked BrokerUtil brokerUtil, @Mocked SparkLauncher launcher, -// @Injectable SparkLoadAppAppHandle handle) throws IOException, LoadException { -// new Expectations() { -// { -// launcher.startApplication((SparkLoadAppAppHandle.Listener) any); -// result = handle; -// handle.getAppId(); -// returns(null, null, appId); -// handle.getState(); -// returns(SparkLoadAppHandle.State.CONNECTED, SparkLoadAppHandle.State.SUBMITTED, SparkLoadAppHandle.State.RUNNING); -// } -// }; -// -// EtlJobConfig etlJobConfig = new EtlJobConfig(Maps.newHashMap(), etlOutputPath, label, null); -// SparkResource resource = new SparkResource(resourceName); -// new Expectations(resource) { -// { -// resource.prepareArchive(); -// result = archive; -// } -// }; -// -// Map sparkConfigs = resource.getSparkConfigs(); -// sparkConfigs.put("spark.master", "yarn"); -// sparkConfigs.put("spark.submit.deployMode", "cluster"); -// sparkConfigs.put("spark.hadoop.yarn.resourcemanager.address", "127.0.0.1:9999"); -// BrokerDesc brokerDesc = new BrokerDesc(broker, Maps.newHashMap()); -// SparkPendingTaskAttachment attachment = new SparkPendingTaskAttachment(pendingTaskId); -// SparkEtlJobHandler handler = new SparkEtlJobHandler(); -// handler.submitEtlJob(loadJobId, label, etlJobConfig, resource, brokerDesc, attachment); -// -// // check submit etl job success -// Assert.assertEquals(appId, attachment.getAppId()); -// Assert.assertEquals(handle, attachment.getHandle()); -// } - -// @Test(expected = LoadException.class) -// public void testSubmitEtlJobFailed(@Mocked BrokerUtil brokerUtil, @Mocked SparkLauncher launcher, -// @Injectable SparkAppHandle handle) throws IOException, LoadException { -// new Expectations() { -// { -// launcher.startApplication((SparkAppHandle.Listener) any); -// result = handle; -// handle.getAppId(); -// result = null; -// handle.getState(); -// returns(State.CONNECTED, State.SUBMITTED, State.FAILED); -// } -// }; -// -// EtlJobConfig etlJobConfig = new EtlJobConfig(Maps.newHashMap(), etlOutputPath, label, null); -// SparkResource resource = new SparkResource(resourceName); -// new Expectations(resource) { -// { -// resource.prepareArchive(); -// result = archive; -// } -// }; -// -// Map sparkConfigs = resource.getSparkConfigs(); -// sparkConfigs.put("spark.master", "yarn"); -// sparkConfigs.put("spark.submit.deployMode", "cluster"); -// sparkConfigs.put("spark.hadoop.yarn.resourcemanager.address", "127.0.0.1:9999"); -// BrokerDesc brokerDesc = new BrokerDesc(broker, Maps.newHashMap()); -// SparkPendingTaskAttachment attachment = new SparkPendingTaskAttachment(pendingTaskId); -// SparkEtlJobHandler handler = new SparkEtlJobHandler(); -// handler.submitEtlJob(loadJobId, label, etlJobConfig, resource, brokerDesc, attachment); -// } + @Test + public void testSubmitEtlJob(@Mocked BrokerUtil brokerUtil, @Mocked SparkLauncher launcher, @Injectable Process process, + @Mocked SparkLoadAppHandle handle ) throws IOException, LoadException { + new Expectations() { + { + launcher.launch(); + result = process; + handle.getAppId(); + result = appId; + handle.getState(); + result = SparkLoadAppHandle.State.RUNNING; + } + }; + + EtlJobConfig etlJobConfig = new EtlJobConfig(Maps.newHashMap(), etlOutputPath, label, null); + SparkResource resource = new SparkResource(resourceName); + new Expectations(resource) { + { + resource.prepareArchive(); + result = archive; + } + }; + + Map sparkConfigs = resource.getSparkConfigs(); + sparkConfigs.put("spark.master", "yarn"); + sparkConfigs.put("spark.submit.deployMode", "cluster"); + sparkConfigs.put("spark.hadoop.yarn.resourcemanager.address", "127.0.0.1:9999"); + BrokerDesc brokerDesc = new BrokerDesc(broker, Maps.newHashMap()); + SparkPendingTaskAttachment attachment = new SparkPendingTaskAttachment(pendingTaskId); + SparkEtlJobHandler handler = new SparkEtlJobHandler(); + handler.submitEtlJob(loadJobId, label, etlJobConfig, resource, brokerDesc, attachment); + + // check submit etl job success + Assert.assertEquals(appId, attachment.getAppId()); + } + + @Test(expected = LoadException.class) + public void testSubmitEtlJobFailed(@Mocked BrokerUtil brokerUtil, @Mocked SparkLauncher launcher, @Injectable Process process, + @Mocked SparkLoadAppHandle handle) throws IOException, LoadException { + new Expectations() { + { + launcher.launch(); + result = process; + handle.getAppId(); + result = appId; + handle.getState(); + result = SparkLoadAppHandle.State.FAILED; + } + }; + + EtlJobConfig etlJobConfig = new EtlJobConfig(Maps.newHashMap(), etlOutputPath, label, null); + SparkResource resource = new SparkResource(resourceName); + new Expectations(resource) { + { + resource.prepareArchive(); + result = archive; + } + }; + + Map sparkConfigs = resource.getSparkConfigs(); + sparkConfigs.put("spark.master", "yarn"); + sparkConfigs.put("spark.submit.deployMode", "cluster"); + sparkConfigs.put("spark.hadoop.yarn.resourcemanager.address", "127.0.0.1:9999"); + BrokerDesc brokerDesc = new BrokerDesc(broker, Maps.newHashMap()); + SparkPendingTaskAttachment attachment = new SparkPendingTaskAttachment(pendingTaskId); + SparkEtlJobHandler handler = new SparkEtlJobHandler(); + handler.submitEtlJob(loadJobId, label, etlJobConfig, resource, brokerDesc, attachment); + } @Test public void testGetEtlJobStatus(@Mocked BrokerUtil brokerUtil, @Mocked Util util, @Mocked CommandResult commandResult, - @Mocked SparkYarnConfigFiles sparkYarnConfigFiles) + @Mocked SparkYarnConfigFiles sparkYarnConfigFiles, @Mocked SparkLoadAppHandle handle) throws IOException, UserException { new Expectations() { @@ -212,15 +216,14 @@ public void testGetEtlJobStatus(@Mocked BrokerUtil brokerUtil, @Mocked Util util sparkYarnConfigFiles.prepare(); sparkYarnConfigFiles.getConfigDir(); result = "./yarn_config"; - } - }; - new Expectations() { - { commandResult.getReturnCode(); result = 0; commandResult.getStdout(); returns(runningReport, runningReport, failedReport, failedReport, finishReport, finishReport); + + handle.getUrl(); + result = trackingUrl; } }; @@ -229,6 +232,7 @@ public void testGetEtlJobStatus(@Mocked BrokerUtil brokerUtil, @Mocked Util util Util.executeCommand(anyString, (String[]) any); minTimes = 0; result = commandResult; + BrokerUtil.readFile(anyString, (BrokerDesc) any); result = "{'normal_rows': 10, 'abnormal_rows': 0, 'failed_reason': 'etl job failed'}"; } @@ -239,22 +243,30 @@ public void testGetEtlJobStatus(@Mocked BrokerUtil brokerUtil, @Mocked Util util sparkConfigs.put("spark.master", "yarn"); sparkConfigs.put("spark.submit.deployMode", "cluster"); sparkConfigs.put("spark.hadoop.yarn.resourcemanager.address", "127.0.0.1:9999"); + new Expectations(resource) { + { + resource.getYarnClientPath(); + result = Config.yarn_client_path; + } + }; + BrokerDesc brokerDesc = new BrokerDesc(broker, Maps.newHashMap()); SparkEtlJobHandler handler = new SparkEtlJobHandler(); // running - EtlStatus status = handler.getEtlJobStatus(null, appId, loadJobId, etlOutputPath, resource, brokerDesc); + EtlStatus status = handler.getEtlJobStatus(handle, appId, loadJobId, etlOutputPath, resource, brokerDesc); Assert.assertEquals(TEtlState.RUNNING, status.getState()); Assert.assertEquals(50, status.getProgress()); + Assert.assertEquals(trackingUrl, status.getTrackingUrl()); // yarn finished and spark failed - status = handler.getEtlJobStatus(null, appId, loadJobId, etlOutputPath, resource, brokerDesc); + status = handler.getEtlJobStatus(handle, appId, loadJobId, etlOutputPath, resource, brokerDesc); Assert.assertEquals(TEtlState.CANCELLED, status.getState()); Assert.assertEquals(100, status.getProgress()); Assert.assertEquals("etl job failed", status.getDppResult().failedReason); // finished - status = handler.getEtlJobStatus(null, appId, loadJobId, etlOutputPath, resource, brokerDesc); + status = handler.getEtlJobStatus(handle, appId, loadJobId, etlOutputPath, resource, brokerDesc); Assert.assertEquals(TEtlState.FINISHED, status.getState()); Assert.assertEquals(100, status.getProgress()); Assert.assertEquals(trackingUrl, status.getTrackingUrl()); @@ -264,7 +276,7 @@ public void testGetEtlJobStatus(@Mocked BrokerUtil brokerUtil, @Mocked Util util @Test public void testGetEtlJobStatusFailed(@Mocked Util util, @Mocked CommandResult commandResult, - @Mocked SparkYarnConfigFiles sparkYarnConfigFiles) + @Mocked SparkYarnConfigFiles sparkYarnConfigFiles, @Mocked SparkLoadAppHandle handle) throws IOException, UserException { new Expectations() { @@ -272,11 +284,7 @@ public void testGetEtlJobStatusFailed(@Mocked Util util, @Mocked CommandResult c sparkYarnConfigFiles.prepare(); sparkYarnConfigFiles.getConfigDir(); result = "./yarn_config"; - } - }; - new Expectations() { - { commandResult.getReturnCode(); result = -1; } @@ -295,6 +303,13 @@ public void testGetEtlJobStatusFailed(@Mocked Util util, @Mocked CommandResult c sparkConfigs.put("spark.master", "yarn"); sparkConfigs.put("spark.submit.deployMode", "cluster"); sparkConfigs.put("spark.hadoop.yarn.resourcemanager.address", "127.0.0.1:9999"); + new Expectations(resource) { + { + resource.getYarnClientPath(); + result = Config.yarn_client_path; + } + }; + BrokerDesc brokerDesc = new BrokerDesc(broker, Maps.newHashMap()); SparkEtlJobHandler handler = new SparkEtlJobHandler(); @@ -334,6 +349,13 @@ public void testKillEtlJob(@Mocked Util util, @Mocked CommandResult commandResul sparkConfigs.put("spark.master", "yarn"); sparkConfigs.put("spark.submit.deployMode", "cluster"); sparkConfigs.put("spark.hadoop.yarn.resourcemanager.address", "127.0.0.1:9999"); + new Expectations(resource) { + { + resource.getYarnClientPath(); + result = Config.yarn_client_path; + } + }; + SparkEtlJobHandler handler = new SparkEtlJobHandler(); try { handler.killEtlJob(null, appId, loadJobId, resource); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorsTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorsTest.java new file mode 100644 index 00000000000000..d091f29f680247 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorsTest.java @@ -0,0 +1,74 @@ +// 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.doris.load.loadv2; + +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.net.URL; + +public class SparkLauncherMonitorsTest { + private String appId; + private SparkLoadAppHandle.State state; + private String queue; + private long startTime; + private FinalApplicationStatus finalApplicationStatus; + private String trackingUrl; + private String user; + + @Before + public void setUp() { + appId = "application_1573630236805_6864759"; + state = SparkLoadAppHandle.State.RUNNING; + queue = "spark-queue"; + startTime = 1597916263384L; + finalApplicationStatus = FinalApplicationStatus.UNDEFINED; + trackingUrl = "http://myhost:8388/proxy/application_1573630236805_6864759/"; + user = "testugi"; + } + + @Test + public void testLogMonitorNormal() { + URL log = getClass().getClassLoader().getResource("data/spark_launcher_monitor.log"); + String cmd = "cat " + log.getPath(); + SparkLoadAppHandle handle = null; + try { + Process process = Runtime.getRuntime().exec(cmd); + handle = new SparkLoadAppHandle(process); + SparkLauncherMonitors.LogMonitor logMonitor = SparkLauncherMonitors.createLogMonitor(handle); + logMonitor.start(); + try { + logMonitor.join(); + } catch (InterruptedException e) { + } + } catch (IOException e) { + Assert.fail(); + } + + Assert.assertEquals(appId, handle.getAppId()); + Assert.assertEquals(state, handle.getState()); + Assert.assertEquals(queue, handle.getQueue()); + Assert.assertEquals(startTime, handle.getStartTime()); + Assert.assertEquals(finalApplicationStatus, handle.getFinalStatus()); + Assert.assertEquals(trackingUrl, handle.getUrl()); + Assert.assertEquals(user, handle.getUser()); + } +} diff --git a/fe/fe-core/src/test/resources/data/spark_launcher_monitor.log b/fe/fe-core/src/test/resources/data/spark_launcher_monitor.log new file mode 100644 index 00000000000000..4b9e5df1ce16b8 --- /dev/null +++ b/fe/fe-core/src/test/resources/data/spark_launcher_monitor.log @@ -0,0 +1,264 @@ +WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +WARN DependencyUtils: Skip remote jar hdfs://myhost:54310/user/etl/cluster_id/__spark_repository__test8/__archive_1.0.0/__lib_bea2ecd751343b3e1651556e85e98f89_spark-dpp-1.0.0-jar-with-dependencies.jar. +INFO RMProxy: Connecting to ResourceManager at myhost/127.0.0.1:8350 +INFO Client: Requesting a new application from cluster with 0 NodeManagers +INFO Client: Verifying our application has not requested more than the maximum memory capability of the cluster (204800 MB per container) +INFO Client: Will allocate AM container, with 1408 MB memory including 384 MB overhead +INFO Client: Setting up container launch context for our AM +INFO Client: Setting up the launch environment for our AM container +INFO Client: Preparing resources for our AM container +* 0 [521082][9458][src/wrapper/hdfs_filesystem_wrapper.cpp:204] connect to cluster: hdfs://myhost:54310 +* 0 [522398][9458][src/afs/afs.cpp:171] connect to afs cluster: hdfs://myhost:54310 +INFO Client: Source and destination file systems are the same. Not copying hdfs://myhost:54310/user/etl/cluster_id/__spark_repository__test8/__archive_1.0.0/__lib_4d29048a6e58c5dd21d9deb9589573be_spark-2x.zip +INFO Client: Source and destination file systems are the same. Not copying hdfs://myhost:54310/user/etl/cluster_id/__spark_repository__test8/__archive_1.0.0/__lib_bea2ecd751343b3e1651556e85e98f89_spark-dpp-1.0.0-jar-with-dependencies.jar +* 0 [714727][9458][src/afs/dfs_client_impl.cpp:1759] fail to get path info, path:/user/etl/.sparkStaging/application_1573630236805_6864759/__dce_sessions__, ret:-10011 +INFO Client: Uploading resource file:/tmp/spark-7448982a-b4c2-4224-9f14-32b145deb920/__dce__2334649440686670150.zip -> hdfs://myhost:54310/user/etl/.sparkStaging/application_1573630236805_6864759/__dce__.zip +* 0 [517842][9458][src/afs/dfs_client_impl.cpp:1759] fail to get path info, path:/user/etl/.sparkStagig/application_1573630236805_6864759/__dce__.zip, ret:-10011 +* 0 [522307][9458][src/afs/dfs_client_impl.cpp:477] Avg write speed: 170393 +INFO Client: Uploading resource file:/tmp/spark-7448982a-b4c2-4224-9f14-32b145deb920/__spark_conf__1476443680705568032.zip -> hdfs://myhost:54310/user/etl/.sparkStaging/application_1573630236805_6864759/__spark_conf__.zip +* 0 [388983][9458][src/afs/dfs_client_impl.cpp:1759] fail to get path info, path:/user/etl/.sparkStaging/application_1573630236805_6864759/__spark_conf__.zip, ret:-10011 +INFO SecurityManager: Changing view acls to: test,testugi +INFO SecurityManager: Changing modify acls to: test,testugi +INFO SecurityManager: Changing view acls groups to: +INFO SecurityManager: Changing modify acls groups to: +INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: Set(test, testugi); groups with view permissions: Set(); users with modify permissions: Set(test, testugi); groups with modify permissions: Set() +WARN Client: priority = 30 +INFO Client: Submitting application application_1573630236805_6864759 to ResourceManager +INFO YarnClientImpl: Submitted application application_1573630236805_6864759 +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: +client token: N/A +diagnostics: N/A +applicationMaster host: N/A +applicationMaster RPC port: -1 +queue: spark-queue +start time: 1597916263384 +final status: UNDEFINED +tracking URL: http://myhost:8388/proxy/application_1573630236805_6864759/ +user: testugi +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: ACCEPTED) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: +client token: N/A +diagnostics: N/A +applicationMaster host: applicationMasterHost +applicationMaster RPC port: 48297 +queue: spark-queue +start time: 1597916263384 +final status: UNDEFINED +tracking URL: http://myhost:8388/proxy/application_1573630236805_6864759/ +user: testugi +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: RUNNING) +INFO Client: Application report for application_1573630236805_6864759 (state: FINISHED) +INFO Client: +client token: N/A +diagnostics: N/A +applicationMaster host: applicationMasterHost +applicationMaster RPC port: 48297 +queue: spark-queue +start time: 1597916263384 +final status: SUCCEEDED +tracking URL: http://myhost:8388/proxy/application_1573630236805_6864759/A +user: testugi +* 0 [689781][9458][src/afs/dfs_client_impl.cpp:1473] fail to del, path:/user/etl/.sparkStaging/application_1573630236805_6864759, ret:-10011 +* 0 [689781][9458][src/afs/dfs_client_impl.cpp:1473] fail to del, path:/user/etl/.sparkStaging/application_1573630236805_6864759, ret:-10011 +WARN DFileSystem: Delete src not found! path: hdfs://myhost:54310/user/etl/.sparkStaging/application_1573630236805_6864759 +INFO ShutdownHookManager: Shutdown hook called +INFO ShutdownHookManager: Deleting directory /tmp/spark-7448982a-b4c2-4224-9f14-32b145deb920 +INFO ShutdownHookManager: Deleting directory /tmp/spark-5c1c8a31-3db5-4042-925a-405509cd21da From fefbf7c860ac3584f59cb7974e86641dcac1533b Mon Sep 17 00:00:00 2001 From: xy720 Date: Wed, 26 Aug 2020 17:56:04 +0800 Subject: [PATCH 09/14] add comments --- .../main/java/org/apache/doris/catalog/SparkResource.java | 6 +++--- .../main/java/org/apache/doris/load/loadv2/ConfigFile.java | 3 +++ .../org/apache/doris/load/loadv2/SparkEtlJobHandler.java | 2 +- .../org/apache/doris/load/loadv2/SparkYarnConfigFiles.java | 1 + 4 files changed, 8 insertions(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java index e3bb1cb8f8e035..35822cb57cebe0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java @@ -265,13 +265,13 @@ private Map getSparkConfigs(Map properties) { } private Map getSparkHadoopConfigs(Map properties) { - Map sparkConfigs = Maps.newHashMap(); + Map sparkConfig = Maps.newHashMap(); for (Map.Entry entry : properties.entrySet()) { if (entry.getKey().startsWith(SPARK_HADOOP_CONFIG_PREFIX)) { - sparkConfigs.put(entry.getKey(), entry.getValue()); + sparkConfig.put(entry.getKey(), entry.getValue()); } } - return sparkConfigs; + return sparkConfig; } private Map getBrokerProperties(Map properties) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/ConfigFile.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/ConfigFile.java index de2536f489e2b2..17384a72cbd07f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/ConfigFile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/ConfigFile.java @@ -19,6 +19,9 @@ import org.apache.doris.common.LoadException; +// The config file required to run the yarn command. +// Each time before running the yarn command, we need to check that the +// config file exists in the specified path, and if not, create them. public interface ConfigFile { public void createFile() throws LoadException; public String getFilePath(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java index 52de11da6babc7..ed2dc23387013a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java @@ -189,8 +189,8 @@ public EtlStatus getEtlJobStatus(SparkLoadAppHandle handle, String appId, long l SparkResource resource, BrokerDesc brokerDesc) throws LoadException { EtlStatus status = new EtlStatus(); + Preconditions.checkState(appId != null && !appId.isEmpty()); if (resource.isYarnMaster()) { - Preconditions.checkState(appId != null && !appId.isEmpty()); // prepare yarn config String configDir = resource.prepareYarnConfig(); // yarn client path diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java index 927a6434adcbc8..c980d74b0c7d84 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkYarnConfigFiles.java @@ -181,6 +181,7 @@ private void mkdir(String configDir) { file.mkdirs(); } + // xml config file public static class XMLConfigFile implements ConfigFile { private static final String CONFIGURATION = "configuration"; private static final String PROPERTY = "property"; From 0b77bab05dd9fc99bca5f5d9c43136e9c5fe5983 Mon Sep 17 00:00:00 2001 From: xy720 Date: Wed, 26 Aug 2020 18:02:26 +0800 Subject: [PATCH 10/14] remove s --- .../java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java | 2 +- .../{SparkLauncherMonitors.java => SparkLauncherMonitor.java} | 4 ++-- ...auncherMonitorsTest.java => SparkLauncherMonitorTest.java} | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) rename fe/fe-core/src/main/java/org/apache/doris/load/loadv2/{SparkLauncherMonitors.java => SparkLauncherMonitor.java} (99%) rename fe/fe-core/src/test/java/org/apache/doris/load/loadv2/{SparkLauncherMonitorsTest.java => SparkLauncherMonitorTest.java} (94%) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java index ed2dc23387013a..0749fbf8bdbe7a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java @@ -154,7 +154,7 @@ public void submitEtlJob(long loadJobId, String loadLabel, EtlJobConfig etlJobCo handle = new SparkLoadAppHandle(process); handle.addListener(new SparkAppListener()); if (!FeConstants.runningUnitTest) { - SparkLauncherMonitors.LogMonitor logMonitor = SparkLauncherMonitors.createLogMonitor(handle); + SparkLauncherMonitor.LogMonitor logMonitor = SparkLauncherMonitor.createLogMonitor(handle); logMonitor.setSubmitTimeoutMs(GET_APPID_TIMEOUT_MS); logMonitor.start(); try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitor.java similarity index 99% rename from fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java rename to fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitor.java index fc38d344eea522..628037d68895ce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitors.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLauncherMonitor.java @@ -33,8 +33,8 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -public class SparkLauncherMonitors { - private static final Logger LOG = LogManager.getLogger(SparkLauncherMonitors.class); +public class SparkLauncherMonitor { + private static final Logger LOG = LogManager.getLogger(SparkLauncherMonitor.class); public static LogMonitor createLogMonitor(SparkLoadAppHandle handle) { return new LogMonitor(handle); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorsTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorTest.java similarity index 94% rename from fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorsTest.java rename to fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorTest.java index d091f29f680247..d619a37504bfcc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorTest.java @@ -25,7 +25,7 @@ import java.io.IOException; import java.net.URL; -public class SparkLauncherMonitorsTest { +public class SparkLauncherMonitorTest { private String appId; private SparkLoadAppHandle.State state; private String queue; @@ -53,7 +53,7 @@ public void testLogMonitorNormal() { try { Process process = Runtime.getRuntime().exec(cmd); handle = new SparkLoadAppHandle(process); - SparkLauncherMonitors.LogMonitor logMonitor = SparkLauncherMonitors.createLogMonitor(handle); + SparkLauncherMonitor.LogMonitor logMonitor = SparkLauncherMonitor.createLogMonitor(handle); logMonitor.start(); try { logMonitor.join(); From 86077e20b5056dd86b45f8e21c4b80e1dfa6063f Mon Sep 17 00:00:00 2001 From: xy720 Date: Wed, 26 Aug 2020 18:44:48 +0800 Subject: [PATCH 11/14] save code --- .../main/java/org/apache/doris/common/Config.java | 5 +++++ .../doris/load/loadv2/SparkEtlJobHandler.java | 13 ------------- .../doris/load/loadv2/SparkLauncherMonitorTest.java | 2 +- .../resources/{data => }/spark_launcher_monitor.log | 0 4 files changed, 6 insertions(+), 14 deletions(-) rename fe/fe-core/src/test/resources/{data => }/spark_launcher_monitor.log (100%) diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java index df94bf9c8977b0..72c5ff8553aed5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java @@ -548,6 +548,11 @@ public class Config extends ConfigBase { */ public static String yarn_client_path = PaloFe.DORIS_HOME_DIR + "/lib/yarn-client/hadoop/bin/yarn"; + /** + * Default yarn config file directory + * Each time before running the yarn command, we need to check that the + * config file exists under this path, and if not, create them. + */ @ConfField public static String yarn_config_dir = PaloFe.DORIS_HOME_DIR + "/lib/yarn-config"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java index 0749fbf8bdbe7a..3b53851e97c2ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java @@ -74,18 +74,6 @@ public class SparkEtlJobHandler { private static final String YARN_STATUS_CMD = "%s --config %s application -status %s"; private static final String YARN_KILL_CMD = "%s --config %s application -kill %s"; - class SparkAppListener implements SparkLoadAppHandle.Listener { - @Override - public void stateChanged(SparkLoadAppHandle sparkAppHandle) { - LOG.info("get spark state changed: {}, app id: {}", sparkAppHandle.getState(), sparkAppHandle.getAppId()); - } - - @Override - public void infoChanged(SparkLoadAppHandle sparkAppHandle) { - LOG.info("get spark info changed: {}, app id: {}", sparkAppHandle.getState(), sparkAppHandle.getAppId()); - } - } - public void submitEtlJob(long loadJobId, String loadLabel, EtlJobConfig etlJobConfig, SparkResource resource, BrokerDesc brokerDesc, SparkPendingTaskAttachment attachment) throws LoadException { // delete outputPath @@ -152,7 +140,6 @@ public void submitEtlJob(long loadJobId, String loadLabel, EtlJobConfig etlJobCo try { Process process = launcher.launch(); handle = new SparkLoadAppHandle(process); - handle.addListener(new SparkAppListener()); if (!FeConstants.runningUnitTest) { SparkLauncherMonitor.LogMonitor logMonitor = SparkLauncherMonitor.createLogMonitor(handle); logMonitor.setSubmitTimeoutMs(GET_APPID_TIMEOUT_MS); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorTest.java index d619a37504bfcc..8e55d4138db6bd 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLauncherMonitorTest.java @@ -47,7 +47,7 @@ public void setUp() { @Test public void testLogMonitorNormal() { - URL log = getClass().getClassLoader().getResource("data/spark_launcher_monitor.log"); + URL log = getClass().getClassLoader().getResource("spark_launcher_monitor.log"); String cmd = "cat " + log.getPath(); SparkLoadAppHandle handle = null; try { diff --git a/fe/fe-core/src/test/resources/data/spark_launcher_monitor.log b/fe/fe-core/src/test/resources/spark_launcher_monitor.log similarity index 100% rename from fe/fe-core/src/test/resources/data/spark_launcher_monitor.log rename to fe/fe-core/src/test/resources/spark_launcher_monitor.log From 7f9651688bae81bcc0978b4c2148716322c6beb2 Mon Sep 17 00:00:00 2001 From: xy720 Date: Wed, 26 Aug 2020 19:01:08 +0800 Subject: [PATCH 12/14] add comments --- .../java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java index 3b53851e97c2ed..1d9eedc5ae7d3b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkEtlJobHandler.java @@ -163,7 +163,7 @@ public void submitEtlJob(long loadJobId, String loadLabel, EtlJobConfig etlJobCo } if (appId == null) { - throw new LoadException(errMsg + "Failed to get appId from handle. spark app state: " + throw new LoadException(errMsg + "Waiting too much time to get appId from handle. spark app state: " + state.toString() + ", loadJobId:" + loadJobId); } @@ -252,8 +252,8 @@ public EtlStatus getEtlJobStatus(SparkLoadAppHandle handle, String appId, long l } public void killEtlJob(SparkLoadAppHandle handle, String appId, long loadJobId, SparkResource resource) throws LoadException { + Preconditions.checkNotNull(appId); if (resource.isYarnMaster()) { - Preconditions.checkNotNull(appId); // prepare yarn config String configDir = resource.prepareYarnConfig(); // yarn client path From f5fc44f863a70fc5040ee8c6438c6dd4af007ab2 Mon Sep 17 00:00:00 2001 From: xy720 Date: Wed, 26 Aug 2020 19:12:27 +0800 Subject: [PATCH 13/14] remove s --- .../org/apache/doris/catalog/SparkResource.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java index 35822cb57cebe0..0614bd34ebf9de 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SparkResource.java @@ -173,7 +173,7 @@ public synchronized SparkRepository.SparkArchive prepareArchive() throws LoadExc // Each SparkResource has and only has one yarn config to run yarn command // This method will write all the configuration start with "spark.hadoop." into config files in a specific directory public synchronized String prepareYarnConfig() throws LoadException { - SparkYarnConfigFiles yarnConfigFiles = new SparkYarnConfigFiles(name, getSparkHadoopConfigs(sparkConfigs)); + SparkYarnConfigFiles yarnConfigFiles = new SparkYarnConfigFiles(name, getSparkHadoopConfig(sparkConfigs)); yarnConfigFiles.prepare(); return yarnConfigFiles.getConfigDir(); } @@ -203,7 +203,7 @@ public void update(ResourceDesc resourceDesc) throws DdlException { if (properties.containsKey(SPARK_MASTER)) { throw new DdlException("Cannot change spark master"); } - sparkConfigs.putAll(getSparkConfigs(properties)); + sparkConfigs.putAll(getSparkConfig(properties)); // update working dir and broker if (properties.containsKey(WORKING_DIR)) { @@ -220,7 +220,7 @@ protected void setProperties(Map properties) throws DdlException Preconditions.checkState(properties != null); // get spark configs - sparkConfigs = getSparkConfigs(properties); + sparkConfigs = getSparkConfig(properties); // check master and deploy mode if (getMaster() == null) { throw new DdlException("Missing " + SPARK_MASTER + " in properties"); @@ -254,17 +254,17 @@ && isYarnMaster()) { brokerProperties = getBrokerProperties(properties); } - private Map getSparkConfigs(Map properties) { - Map sparkConfigs = Maps.newHashMap(); + private Map getSparkConfig(Map properties) { + Map sparkConfig = Maps.newHashMap(); for (Map.Entry entry : properties.entrySet()) { if (entry.getKey().startsWith(SPARK_CONFIG_PREFIX)) { - sparkConfigs.put(entry.getKey(), entry.getValue()); + sparkConfig.put(entry.getKey(), entry.getValue()); } } - return sparkConfigs; + return sparkConfig; } - private Map getSparkHadoopConfigs(Map properties) { + private Map getSparkHadoopConfig(Map properties) { Map sparkConfig = Maps.newHashMap(); for (Map.Entry entry : properties.entrySet()) { if (entry.getKey().startsWith(SPARK_HADOOP_CONFIG_PREFIX)) { From f3b51caeaa2da9b19784769f2fc01518e33ca397 Mon Sep 17 00:00:00 2001 From: xy720 Date: Wed, 26 Aug 2020 19:45:21 +0800 Subject: [PATCH 14/14] save code --- fe/fe-core/src/main/java/org/apache/doris/common/Config.java | 1 + 1 file changed, 1 insertion(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java index 72c5ff8553aed5..2b4ee22868f021 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java @@ -546,6 +546,7 @@ public class Config extends ConfigBase { /** * Default yarn client path */ + @ConfField public static String yarn_client_path = PaloFe.DORIS_HOME_DIR + "/lib/yarn-client/hadoop/bin/yarn"; /**