|
@@ -25,9 +25,11 @@ import java.net.MalformedURLException;
|
|
|
import java.net.Proxy;
|
|
|
import java.net.URL;
|
|
|
|
|
|
+import org.apache.commons.lang3.StringUtils;
|
|
|
import org.apache.hadoop.conf.Configurable;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.mapred.JobContext;
|
|
|
+import org.apache.hadoop.mapreduce.CustomJobEndNotifier;
|
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
|
|
|
import org.eclipse.jetty.util.log.Log;
|
|
@@ -57,6 +59,9 @@ public class JobEndNotifier implements Configurable {
|
|
|
protected int timeout; // Timeout (ms) on the connection and notification
|
|
|
protected URL urlToNotify; //URL to notify read from the config
|
|
|
protected Proxy proxyToUse = Proxy.NO_PROXY; //Proxy to use for notification
|
|
|
+ // A custom notifier implementation
|
|
|
+ // (see org.apache.hadoop.mapreduce.CustomJobEndNotifier)
|
|
|
+ private String customJobEndNotifierClassName;
|
|
|
|
|
|
/**
|
|
|
* Parse the URL that needs to be notified of the end of the job, along
|
|
@@ -84,6 +89,9 @@ public class JobEndNotifier implements Configurable {
|
|
|
|
|
|
proxyConf = conf.get(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY);
|
|
|
|
|
|
+ customJobEndNotifierClassName = StringUtils.stripToNull(
|
|
|
+ conf.get(MRJobConfig.MR_JOB_END_NOTIFICATION_CUSTOM_NOTIFIER_CLASS));
|
|
|
+
|
|
|
//Configure the proxy to use if its set. It should be set like
|
|
|
//proxyType@proxyHostname:port
|
|
|
if(proxyConf != null && !proxyConf.equals("") &&
|
|
@@ -115,11 +123,22 @@ public class JobEndNotifier implements Configurable {
|
|
|
public Configuration getConf() {
|
|
|
return conf;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Notify the URL just once. Use best effort.
|
|
|
*/
|
|
|
protected boolean notifyURLOnce() {
|
|
|
+ if (customJobEndNotifierClassName == null) {
|
|
|
+ return notifyViaBuiltInNotifier();
|
|
|
+ } else {
|
|
|
+ return notifyViaCustomNotifier();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Uses a simple HttpURLConnection to do the Job end notification.
|
|
|
+ */
|
|
|
+ private boolean notifyViaBuiltInNotifier() {
|
|
|
boolean success = false;
|
|
|
try {
|
|
|
Log.getLog().info("Job end notification trying " + urlToNotify);
|
|
@@ -145,6 +164,36 @@ public class JobEndNotifier implements Configurable {
|
|
|
return success;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Uses the custom Job end notifier class to do the Job end notification.
|
|
|
+ */
|
|
|
+ private boolean notifyViaCustomNotifier() {
|
|
|
+ try {
|
|
|
+ Log.getLog().info("Will be using " + customJobEndNotifierClassName
|
|
|
+ + " for Job end notification");
|
|
|
+
|
|
|
+ final Class<? extends CustomJobEndNotifier> customJobEndNotifierClass =
|
|
|
+ Class.forName(customJobEndNotifierClassName)
|
|
|
+ .asSubclass(CustomJobEndNotifier.class);
|
|
|
+ final CustomJobEndNotifier customJobEndNotifier =
|
|
|
+ customJobEndNotifierClass.getDeclaredConstructor().newInstance();
|
|
|
+
|
|
|
+ boolean success = customJobEndNotifier.notifyOnce(urlToNotify, conf);
|
|
|
+ if (success) {
|
|
|
+ Log.getLog().info("Job end notification to " + urlToNotify
|
|
|
+ + " succeeded");
|
|
|
+ } else {
|
|
|
+ Log.getLog().warn("Job end notification to " + urlToNotify
|
|
|
+ + " failed");
|
|
|
+ }
|
|
|
+ return success;
|
|
|
+ } catch (Exception e) {
|
|
|
+ Log.getLog().warn("Job end notification to " + urlToNotify
|
|
|
+ + " failed", e);
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Notify a server of the completion of a submitted job. The user must have
|
|
|
* configured MRJobConfig.MR_JOB_END_NOTIFICATION_URL
|