aboutsummaryrefslogtreecommitdiff
path: root/drill-yarn/src/main/java/org/apache
diff options
context:
space:
mode:
authorPaul Rogers <progers@maprtech.com>2017-10-26 00:24:00 -0700
committerArina Ielchiieva <arina.yelchiyeva@gmail.com>2018-03-04 17:43:22 +0200
commitf2ac8749b42539ca6301024becbf3e7092e9511e (patch)
treefc6fa0a8eac3420504736e463633e5ee783956a9 /drill-yarn/src/main/java/org/apache
parentcf2478f7a48e66cbb0a7f29750c8a4360a271e9b (diff)
DRILL-1170: YARN integration for Drill
closes #1011
Diffstat (limited to 'drill-yarn/src/main/java/org/apache')
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMException.java30
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMRegistrar.java43
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMWrapperException.java25
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMYarnFacade.java91
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMYarnFacadeImpl.java288
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractDrillbitScheduler.java166
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractScheduler.java112
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractTaskManager.java50
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/BatchScheduler.java88
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterController.java206
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterControllerImpl.java785
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ControllerFactory.java30
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ControllerVisitor.java22
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Dispatcher.java345
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DispatcherAddOn.java30
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillApplicationMaster.java119
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillControllerFactory.java398
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillbitScheduler.java51
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/EventContext.java70
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/NodeInventory.java195
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PersistentTaskScheduler.java174
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Pollable.java28
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PulseRunnable.java79
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/RegistryHandler.java35
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Scheduler.java162
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerState.java79
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateActions.java101
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateImpl.java467
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Task.java323
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskLifecycleListener.java26
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskSpec.java40
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskState.java895
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskVisitor.java22
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/YarnFacadeException.java30
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AMSecurityManager.java34
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AMSecurityManagerImpl.java221
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AbstractTasksModel.java380
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AmRestApi.java296
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AuthDynamicFeature.java114
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/ControllerModel.java208
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/PageTree.java80
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/WebServer.java467
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/WebUiPageTree.java527
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/package-info.java22
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/package-info.java36
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/AMRunner.java368
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/CleanCommand.java89
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientCommand.java100
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientContext.java48
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientException.java34
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/CommandLineOptions.java230
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/DrillOnYarn.java176
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/FileUploader.java551
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/HelpCommand.java26
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/KillCommand.java48
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/PrintConfigCommand.java49
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/ResizeCommand.java115
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/SimpleRestClient.java66
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/StartCommand.java145
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/StatusCommand.java189
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/StopCommand.java223
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/client/package-info.java31
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/core/AppSpec.java169
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/core/ClusterDef.java212
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/core/ContainerRequestSpec.java125
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/core/DfsFacade.java345
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/core/DoYUtil.java189
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/core/DoyConfigException.java30
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/core/DrillOnYarnConfig.java841
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/core/LaunchSpec.java248
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/core/NameValuePair.java46
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/core/YarnClientException.java30
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/core/YarnRMClient.java207
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/core/package-info.java24
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/package-info.java35
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/zk/AMRegistry.java145
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinator.java318
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinatorDriver.java315
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKConfigException.java26
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRegistry.java582
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRuntimeException.java30
-rw-r--r--drill-yarn/src/main/java/org/apache/drill/yarn/zk/package-info.java31
82 files changed, 14426 insertions, 0 deletions
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMException.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMException.java
new file mode 100644
index 000000000..f4e740071
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMException.java
@@ -0,0 +1,30 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+public class AMException extends Exception {
+ private static final long serialVersionUID = 1L;
+
+ public AMException(String msg) {
+ super(msg);
+ }
+
+ public AMException(String msg, Exception e) {
+ super(msg, e);
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMRegistrar.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMRegistrar.java
new file mode 100644
index 000000000..fdce55bb3
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMRegistrar.java
@@ -0,0 +1,43 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+/**
+ * Interface to register the AM. Registration prevents two AMs from running for
+ * the same Drill cluster. A normal return means that this AM now "owns" the
+ * cluster. An exception means this is a duplicate AM (or something went wrong.)
+ * <p>
+ * Although the interface contains a deregister call, the implementation should
+ * automatically deregister on death of the AM to prevent zombie registrations.
+ * (The production system, ZK, handles this via ephemeral znodes.)
+ */
+
+public interface AMRegistrar {
+ public static class AMRegistrationException extends Exception {
+ private static final long serialVersionUID = 1L;
+
+ public AMRegistrationException(Exception e) {
+ super(e.getMessage(), e);
+ }
+ }
+
+ void register(String amHost, int amPort, String appId)
+ throws AMRegistrationException;
+
+ void deregister();
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMWrapperException.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMWrapperException.java
new file mode 100644
index 000000000..d445ec7ce
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMWrapperException.java
@@ -0,0 +1,25 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+@SuppressWarnings("serial")
+public class AMWrapperException extends Error {
+ public AMWrapperException(String msg, Exception e) {
+ super(msg, e);
+ }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMYarnFacade.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMYarnFacade.java
new file mode 100644
index 000000000..62ebdfc5d
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMYarnFacade.java
@@ -0,0 +1,91 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import java.util.List;
+
+import org.apache.drill.yarn.core.ContainerRequestSpec;
+import org.apache.drill.yarn.core.LaunchSpec;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
+import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
+
+/**
+ * Defines the interface between the Application Master and YARN. This interface
+ * enables the use of a mock implementation for testing as well as the actual
+ * implementation that works with YARN.
+ */
+
+public interface AMYarnFacade {
+ /**
+ * Provides a collection of web UI links for the YARN Resource Manager and the
+ * Node Manager that is running the Drill-on-YARN AM. This information is
+ * primarily for use in the AM's own web UI.
+ */
+
+ public static class YarnAppHostReport {
+ public String appId;
+ public String amHost;
+ public String rmHost;
+ public String rmUrl;
+ public String rmAppUrl;
+ public String nmHost;
+ public String nmUrl;
+ public String nmAppUrl;
+ }
+
+ void start(AMRMClientAsync.CallbackHandler resourceCallback,
+ NMClientAsync.CallbackHandler nodeCallback);
+
+ void register(String trackingUrl) throws YarnFacadeException;
+
+ String getTrackingUrl();
+
+ ContainerRequest requestContainer(ContainerRequestSpec containerSpec);
+
+ void removeContainerRequest(ContainerRequest containerRequest);
+
+ void launchContainer(Container container, LaunchSpec taskSpec)
+ throws YarnFacadeException;
+
+ void finish(boolean success, String msg) throws YarnFacadeException;
+
+ void releaseContainer(Container container);
+
+ void killContainer(Container container);
+
+ int getNodeCount();
+
+ Resource getResources();
+
+ RegisterApplicationMasterResponse getRegistrationResponse();
+
+ void blacklistNode(String nodeName);
+
+ void removeBlacklist(String nodeName);
+
+ List<NodeReport> getNodeReports() throws YarnFacadeException;
+
+ YarnAppHostReport getAppHostReport();
+
+ boolean supportsDiskResource();
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMYarnFacadeImpl.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMYarnFacadeImpl.java
new file mode 100644
index 000000000..ac25d0e72
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMYarnFacadeImpl.java
@@ -0,0 +1,288 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.yarn.core.ContainerRequestSpec;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.drill.yarn.core.LaunchSpec;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler;
+import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+
+/**
+ * Wrapper around the asynchronous versions of the YARN AM-RM and AM-NM
+ * interfaces. Allows strategy code to factor out the YARN-specific bits so that
+ * strategy code is simpler. Also allows replacing the actual YARN code with a
+ * mock for unit testing.
+ */
+
+public class AMYarnFacadeImpl implements AMYarnFacade {
+ private static final Log LOG = LogFactory.getLog(AMYarnFacadeImpl.class);
+
+ private YarnConfiguration conf;
+ private AMRMClientAsync<ContainerRequest> resourceMgr;
+ private NMClientAsync nodeMgr;
+ private RegisterApplicationMasterResponse registration;
+ private YarnClient client;
+ private int pollPeriodMs;
+
+ private String appMasterTrackingUrl;
+
+ private ApplicationId appId;
+
+ private ApplicationReport appReport;
+
+ private String amHost;
+
+ private boolean supportsDisks;
+
+ public AMYarnFacadeImpl(int pollPeriodMs) {
+ this.pollPeriodMs = pollPeriodMs;
+ }
+
+ @Override
+ public void start(CallbackHandler resourceCallback,
+ org.apache.hadoop.yarn.client.api.async.NMClientAsync.CallbackHandler nodeCallback ) {
+
+ conf = new YarnConfiguration();
+
+ resourceMgr = AMRMClientAsync.createAMRMClientAsync(pollPeriodMs, resourceCallback);
+ resourceMgr.init(conf);
+ resourceMgr.start();
+
+ // Create the asynchronous node manager client
+
+ nodeMgr = NMClientAsync.createNMClientAsync(nodeCallback);
+ nodeMgr.init(conf);
+ nodeMgr.start();
+
+ client = YarnClient.createYarnClient();
+ client.init(conf);
+ client.start();
+
+ String appIdStr = System.getenv(DrillOnYarnConfig.APP_ID_ENV_VAR);
+ if (appIdStr != null) {
+ appId = ConverterUtils.toApplicationId(appIdStr);
+ try {
+ appReport = client.getApplicationReport(appId);
+ } catch (YarnException | IOException e) {
+ LOG.error(
+ "Failed to get YARN applicaiton report for App ID: " + appIdStr, e);
+ }
+ }
+ }
+
+ @Override
+ public void register(String trackingUrl) throws YarnFacadeException {
+ String thisHostName = NetUtils.getHostname();
+ LOG.debug("Host Name from YARN: " + thisHostName);
+ if (trackingUrl != null) {
+ // YARN seems to provide multiple names: MACHNAME.local/10.250.56.235
+ // The second seems to be the IP address, which is what we want.
+ String names[] = thisHostName.split("/");
+ amHost = names[names.length - 1];
+ appMasterTrackingUrl = trackingUrl.replace("<host>", amHost);
+ LOG.info("Tracking URL: " + appMasterTrackingUrl);
+ }
+ try {
+ LOG.trace("Registering with YARN");
+ registration = resourceMgr.registerApplicationMaster(thisHostName, 0,
+ appMasterTrackingUrl);
+ } catch (YarnException | IOException e) {
+ throw new YarnFacadeException("Register AM failed", e);
+ }
+
+ // Some distributions (but not the stock YARN) support Disk
+ // resources. Since Drill compiles against Apache YARN, without disk
+ // resources, we have to use an indirect mechnanism to look for the
+ // disk enum at runtime when we don't have that enum value at compile time.
+
+ for (SchedulerResourceTypes type : registration
+ .getSchedulerResourceTypes()) {
+ if (type.name().equals("DISK")) {
+ supportsDisks = true;
+ }
+ }
+ }
+
+ @Override
+ public String getTrackingUrl( ) { return appMasterTrackingUrl; }
+
+ @Override
+ public boolean supportsDiskResource( ) { return supportsDisks; }
+
+ @Override
+ public ContainerRequest requestContainer(ContainerRequestSpec containerSpec) {
+ ContainerRequest request = containerSpec.makeRequest();
+ resourceMgr.addContainerRequest(containerSpec.makeRequest());
+ return request;
+ }
+
+ @Override
+ public void launchContainer(Container container, LaunchSpec taskSpec)
+ throws YarnFacadeException {
+ ContainerLaunchContext context = createLaunchContext(taskSpec);
+ startContainerAsync(container, context);
+ }
+
+ private ContainerLaunchContext createLaunchContext(LaunchSpec task)
+ throws YarnFacadeException {
+ try {
+ return task.createLaunchContext(conf);
+ } catch (IOException e) {
+ throw new YarnFacadeException("Failed to create launch context", e);
+ }
+ }
+
+ private void startContainerAsync(Container container,
+ ContainerLaunchContext context) {
+ nodeMgr.startContainerAsync(container, context);
+ }
+
+ @Override
+ public void finish(boolean succeeded, String msg) throws YarnFacadeException {
+ // Stop the Node Manager client.
+
+ nodeMgr.stop();
+
+ // Deregister the app from YARN.
+
+ String appMsg = "Drill Cluster Shut-Down";
+ FinalApplicationStatus status = FinalApplicationStatus.SUCCEEDED;
+ if (!succeeded) {
+ appMsg = "Drill Cluster Fatal Error - check logs";
+ status = FinalApplicationStatus.FAILED;
+ }
+ if (msg != null) {
+ appMsg = msg;
+ }
+ try {
+ resourceMgr.unregisterApplicationMaster(status, appMsg, "");
+ } catch (YarnException | IOException e) {
+ throw new YarnFacadeException("Deregister AM failed", e);
+ }
+
+ // Stop the Resource Manager client
+
+ resourceMgr.stop();
+ }
+
+ @Override
+ public void releaseContainer(Container container) {
+ resourceMgr.releaseAssignedContainer(container.getId());
+ }
+
+ @Override
+ public void killContainer(Container container) {
+ nodeMgr.stopContainerAsync(container.getId(), container.getNodeId());
+ }
+
+ @Override
+ public int getNodeCount() {
+ return resourceMgr.getClusterNodeCount();
+ }
+
+ @Override
+ public Resource getResources() {
+ return resourceMgr.getAvailableResources();
+ }
+
+ @Override
+ public void removeContainerRequest(ContainerRequest containerRequest) {
+ resourceMgr.removeContainerRequest(containerRequest);
+ }
+
+ @Override
+ public RegisterApplicationMasterResponse getRegistrationResponse() {
+ return registration;
+ }
+
+ @Override
+ public void blacklistNode(String nodeName) {
+ resourceMgr.updateBlacklist(Collections.singletonList(nodeName), null);
+ }
+
+ @Override
+ public void removeBlacklist(String nodeName) {
+ resourceMgr.updateBlacklist(null, Collections.singletonList(nodeName));
+ }
+
+ @Override
+ public List<NodeReport> getNodeReports() throws YarnFacadeException {
+ try {
+ return client.getNodeReports(NodeState.RUNNING);
+ } catch (Exception e) {
+ throw new YarnFacadeException("getNodeReports failed", e);
+ }
+ }
+
+ @Override
+ public YarnAppHostReport getAppHostReport() {
+ // Cobble together YARN links to simplify debugging.
+
+ YarnAppHostReport hostRpt = new YarnAppHostReport();
+ hostRpt.amHost = amHost;
+ if (appId != null) {
+ hostRpt.appId = appId.toString();
+ }
+ if (appReport == null) {
+ return hostRpt;
+ }
+ try {
+ String rmLink = appReport.getTrackingUrl();
+ URL url = new URL(rmLink);
+ hostRpt.rmHost = url.getHost();
+ hostRpt.rmUrl = "http://" + hostRpt.rmHost + ":" + url.getPort() + "/";
+ hostRpt.rmAppUrl = hostRpt.rmUrl + "cluster/app/" + appId.toString();
+ } catch (MalformedURLException e) {
+ return null;
+ }
+
+ hostRpt.nmHost = System.getenv("NM_HOST");
+ String nmPort = System.getenv("NM_HTTP_PORT");
+ if (hostRpt.nmHost != null || nmPort != null) {
+ hostRpt.nmUrl = "http://" + hostRpt.nmHost + ":" + nmPort + "/";
+ hostRpt.nmAppUrl = hostRpt.nmUrl + "node/application/" + hostRpt.appId;
+ }
+ return hostRpt;
+ }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractDrillbitScheduler.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractDrillbitScheduler.java
new file mode 100644
index 000000000..cf99eb36b
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractDrillbitScheduler.java
@@ -0,0 +1,166 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.yarn.zk.ZKRegistry;
+
+/**
+ * Base class for schedulers (pools) for Drillbits. Derived classes implement
+ * various policies for node selection. This class handles the common tasks such
+ * as holding the Drillbit launch specification, providing Drillbit- specific
+ * behaviors and so on.
+ * <p>
+ * The key purpose of this class is to abstract Drillbit-speicific code from the
+ * rest of the AM cluster controller. We do so for several reasons: ease of
+ * testing (we can use mock tasks), ability to handle additional server types in
+ * the future, and a way to keep each module focused on a single task (as the
+ * controller and its state machine is complex enough without mixing in Drillbit
+ * specifics.)
+ */
+
+public abstract class AbstractDrillbitScheduler
+ extends PersistentTaskScheduler {
+ /**
+ * Interface to provide Drill-bit specific behavior. Ideally, this class would
+ * provide the interface to gracefully shut down a Drillbit, but Drill has no
+ * API to do graceful shutdown in this release. (The only graceful shutdown is
+ * by issuing a SIGTERM from the node runing the Drillbit, but YARN has no way
+ * to do this, despite active discussions on several YARN JIRA entries.
+ */
+
+ public class DrillbitManager extends AbstractTaskManager {
+ /**
+ * Allow only one concurrent container request by default to ensure that the
+ * node blacklist mechanism works to ensure that the RM does not allocate
+ * two containers on the same node.
+ */
+
+ @Override
+ public int maxConcurrentAllocs() {
+ return 1;
+ }
+
+ @Override
+ public void allocated(EventContext context) {
+
+ // One drillbit per node, so reserve the node
+ // just allocated.
+
+ context.controller.getNodeInventory().reserve(context.task.container);
+ }
+
+ @Override
+ public void completed(EventContext context) {
+ // This method is called for all completed tasks, even those that
+ // completed (were cancelled) before a container was allocated.
+ // If we have no container, then we have nothing to tell the
+ // node inventory.
+
+ if (context.task.container != null) {
+ context.controller.getNodeInventory().release(context.task.container);
+ }
+ analyzeResult(context);
+ }
+
+ @Override
+ public boolean isLive(EventContext context) {
+ ZKRegistry reg = (ZKRegistry) context.controller.getProperty(ZKRegistry.CONTROLLER_PROPERTY);
+ return reg.isRegistered(context.task);
+ }
+
+ /**
+ * Analyze the result. Drillbits should not exit, but this one did. It might
+ * be because we asked it to exit, which is fine. Otherwise, the exit is
+ * unexpected and we should 1) provide the admin with an explanation, and 2)
+ * prevent retries after a few tries.
+ *
+ * @param context
+ */
+
+ private void analyzeResult(EventContext context) {
+ Task task = context.task;
+
+ // If we cancelled the Drill-bit, just unblacklist the
+ // host so we can run another drillbit on it later.
+
+ if (task.isCancelled()) {
+ return;
+ }
+
+ // The Drill-bit stopped on its own.
+ // Maybe the exit status will tell us something.
+
+ int exitCode = task.completionStatus.getExitStatus();
+
+ // We can also consider the runtime.
+
+ long duration = task.uptime() / 1000;
+
+ // The ZK state may also help.
+
+ boolean registered = task.trackingState != Task.TrackingState.NEW;
+
+ // If the exit code was 1, then the script probably found
+ // an error. Only retry once.
+
+ if (registered || task.getTryCount() < 2) {
+
+ // Use the default retry policy.
+
+ return;
+ }
+
+ // Seems to be a mis-configuration. The Drill-bit exited quickly and
+ // did not register in ZK. Also, we've tried twice now with no luck.
+ // Assume the node is bad.
+
+ String hostName = task.getHostName();
+ StringBuilder buf = new StringBuilder();
+ buf.append(task.getLabel()).append(" on host ").append(hostName)
+ .append(" failed with status ").append(exitCode).append(" after ")
+ .append(duration).append(" secs. with");
+ if (!registered) {
+ buf.append("out");
+ }
+ buf.append(" ZK registration");
+ if (duration < 60 && !registered) {
+ buf.append(
+ "\n Probable configuration problem, check Drill log file on host ")
+ .append(hostName).append(".");
+ }
+ LOG.error(buf.toString());
+ task.cancelled = true;
+
+ // Mark the host as permanently blacklisted. Leave it
+ // in YARN's blacklist.
+
+ context.controller.getNodeInventory().blacklist(hostName);
+ }
+ }
+
+ private static final Log LOG = LogFactory
+ .getLog(AbstractDrillbitScheduler.class);
+
+ public AbstractDrillbitScheduler(String type, String name, int quantity) {
+ super(type, name, quantity);
+ isTracked = true;
+ setTaskManager(new DrillbitManager());
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractScheduler.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractScheduler.java
new file mode 100644
index 000000000..01b89aede
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractScheduler.java
@@ -0,0 +1,112 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.yarn.core.ContainerRequestSpec;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+public abstract class AbstractScheduler implements Scheduler {
+ private static final Log LOG = LogFactory.getLog(AbstractScheduler.class);
+ private final String name;
+ private final String type;
+ protected TaskSpec taskSpec;
+ protected int priority;
+ protected int failCount;
+ protected TaskManager taskManager;
+ protected SchedulerState state;
+ protected boolean isTracked;
+
+ public AbstractScheduler(String type, String name) {
+ this.type = type;
+ this.name = name;
+ taskManager = new AbstractTaskManager();
+ }
+
+ public void setTaskManager(TaskManager taskManager) {
+ this.taskManager = taskManager;
+ }
+
+ @Override
+ public void registerState(SchedulerState state) {
+ this.state = state;
+ }
+
+ @Override
+ public void setPriority(int priority) {
+ this.priority = priority;
+ taskSpec.containerSpec.priority = priority;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public String getType() {
+ return type;
+ }
+
+ @Override
+ public TaskManager getTaskManager() {
+ return taskManager;
+ }
+
+ @Override
+ public void change(int delta) {
+ resize(getTarget() + delta);
+ }
+
+ protected void addTasks(int n) {
+ LOG.info( "[" + getName( ) + "] - Adding " + n + " tasks" );
+ for (int i = 0; i < n; i++) {
+ state.start(new Task(this, taskSpec));
+ }
+ }
+
+ @Override
+ public boolean isTracked() {
+ return isTracked;
+ }
+
+ @Override
+ public ContainerRequestSpec getResource() {
+ return taskSpec.containerSpec;
+ }
+
+ @Override
+ public void limitContainerSize(Resource maxResource) throws AMException {
+ if (taskSpec.containerSpec.memoryMb > maxResource.getMemory()) {
+ LOG.warn(taskSpec.name + " requires " + taskSpec.containerSpec.memoryMb
+ + " MB but the maximum YARN container size is "
+ + maxResource.getMemory() + " MB");
+ taskSpec.containerSpec.memoryMb = maxResource.getMemory();
+ }
+ if (taskSpec.containerSpec.vCores > maxResource.getVirtualCores()) {
+ LOG.warn(taskSpec.name + " requires " + taskSpec.containerSpec.vCores
+ + " vcores but the maximum YARN container size is "
+ + maxResource.getVirtualCores() + " vcores");
+ taskSpec.containerSpec.vCores = maxResource.getVirtualCores();
+ }
+ }
+
+ @Override
+ public int getRequestTimeoutSec() { return 0; }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractTaskManager.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractTaskManager.java
new file mode 100644
index 000000000..7acd40261
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractTaskManager.java
@@ -0,0 +1,50 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import org.apache.drill.yarn.appMaster.Scheduler.TaskManager;
+import org.apache.drill.yarn.core.LaunchSpec;
+
+/**
+ * Task manager that does nothing.
+ */
+
+public class AbstractTaskManager implements TaskManager {
+ @Override
+ public int maxConcurrentAllocs() {
+ return Integer.MAX_VALUE;
+ }
+
+ @Override
+ public void allocated(EventContext context) {
+ }
+
+ @Override
+ public LaunchSpec getLaunchSpec(Task task) {
+ return task.getLaunchSpec();
+ }
+
+ @Override
+ public boolean stop(Task task) { return false; }
+
+ @Override
+ public void completed(EventContext context) { }
+
+ @Override
+ public boolean isLive(EventContext context) { return true; }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/BatchScheduler.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/BatchScheduler.java
new file mode 100644
index 000000000..8f3aaabd9
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/BatchScheduler.java
@@ -0,0 +1,88 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+public class BatchScheduler extends AbstractScheduler {
+ private int quantity;
+ private int completedCount;
+
+ public BatchScheduler(String name, int quantity) {
+ super("batch", name);
+ this.quantity = quantity;
+ }
+
+ @Override
+ public void completed(Task task) {
+ completedCount++;
+ if (task.getDisposition() != Task.Disposition.COMPLETED) {
+ failCount++;
+ }
+ }
+
+ @Override
+ public int resize(int level) { quantity = level; return quantity; }
+
+ @Override
+ public int getTarget() { return quantity; }
+
+ @Override
+ public int[] getProgress() {
+ return new int[] { Math.min(completedCount, quantity), quantity };
+ }
+
+ @Override
+ public void adjust() {
+ int activeCount = state.getTaskCount();
+ int delta = quantity - activeCount - completedCount;
+ if (delta < 0) {
+ addTasks(-delta);
+ }
+ if (delta > 0) {
+ cancelTasks(delta);
+ }
+ }
+
+ /**
+ * Cancel any starting tasks. We don't cancel launched, in-flight tasks
+ * because there is no way to tell YARN to cancel tasks that are in the
+ * process of being launched: we have to wait for them to start
+ * before canceling.
+ *
+ * @param n
+ */
+
+ private void cancelTasks(int n) {
+ for (Task task : state.getStartingTasks()) {
+ state.cancel(task);
+ if (--n == 0) {
+ break;
+ }
+ }
+ }
+
+ @Override
+ public boolean hasMoreTasks() {
+ return completedCount < quantity;
+ }
+
+ @Override
+ public void requestTimedOut() {
+ // Not clear what to do here. Since this case is used only for testing,
+ // deal with this case later.
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterController.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterController.java
new file mode 100644
index 000000000..6aaa18b11
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterController.java
@@ -0,0 +1,206 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+
+/**
+ * Interface which identifies the cluster controller methods that are save to
+ * call from the {@link Dispatcher}. Methods here are either designed to be
+ * called before the event threads start or after they complete. The remainder
+ * synchronized to coordinate between event threads.
+ */
+
+public interface ClusterController extends RegistryHandler {
+ void enableFailureCheck(boolean flag);
+
+ void registerLifecycleListener(TaskLifecycleListener listener);
+
+ void registerScheduler(Scheduler resourceGroup);
+
+ void setProperty(String key, Object value);
+
+ Object getProperty(String key);
+
+ /**
+ * Called after the dispatcher has started YARN and other server
+ * components. The controller can now begin to spin up tasks.
+ */
+
+ void started( ) throws YarnFacadeException, AMException;
+
+ /**
+ * Called by the timer ("pulse") thread to trigger time-based events.
+ *
+ * @param curTime
+ */
+
+ void tick(long curTime);
+
+ /**
+ * The RM has allocated one or more containers in response to container
+ * requests submitted to the RM.
+ *
+ * @param containers
+ * the set of containers provided by YARN
+ * @return the set of tasks to launch
+ */
+
+ void containersAllocated(List<Container> containers);
+
+ /**
+ * The NM reports that a container has successfully started.
+ *
+ * @param containerId
+ * the container which started
+ */
+
+ void containerStarted(ContainerId containerId);
+
+ /**
+ * The RM API reports that an attempt to start a container has failed locally.
+ *
+ * @param containerId
+ * the container that failed to launch
+ * @param t
+ * the error that occurred
+ */
+
+ void taskStartFailed(ContainerId containerId, Throwable t);
+
+ /**
+ * The Node Manager reports that a container has stopped.
+ *
+ * @param containerId
+ */
+ void containerStopped(ContainerId containerId);
+
+ /**
+ * The Resource Manager reports that containers have completed with the given
+ * statuses. Find the task for each container and mark them as completed.
+ *
+ * @param statuses
+ */
+
+ void containersCompleted(List<ContainerStatus> statuses);
+
+ float getProgress();
+
+ /**
+ * The Node Manager API reports that a request sent to the NM to stop a task
+ * has failed.
+ *
+ * @param containerId
+ * the container that failed to stop
+ * @param t
+ * the reason that the stop request failed
+ */
+
+ void stopTaskFailed(ContainerId containerId, Throwable t);
+
+ /**
+ * Request to resize the Drill cluster by a relative amount.
+ *
+ * @param delta
+ * the amount of change. Can be positive (to grow) or negative (to
+ * shrink the cluster)
+ */
+
+ void resizeDelta(int delta);
+
+ /**
+ * Request to resize the Drill cluster to the given size.
+ *
+ * @param n
+ * the desired cluster size
+ */
+
+ int resizeTo(int n);
+
+ /**
+ * Indicates a request to gracefully shut down the cluster.
+ */
+
+ void shutDown();
+
+ /**
+ * Called by the main thread to wait for the normal shutdown of the
+ * controller. Such shutdown occurs when the admin sends a sutdown
+ * command from the UI or REST API.
+ *
+ * @return
+ */
+
+ boolean waitForCompletion();
+
+ void updateRMStatus();
+
+ void setMaxRetries(int value);
+
+ /**
+ * Allow an observer to see a consistent view of the controller's
+ * state by performing the visit in a synchronized block.
+ * @param visitor
+ */
+
+ void visit( ControllerVisitor visitor );
+
+ /**
+ * Allow an observer to see a consistent view of the controller's
+ * task state by performing the visit in a synchronized block.
+ *
+ * @param visitor
+ */
+
+ void visitTasks( TaskVisitor visitor );
+
+ /**
+ * Return the target number of tasks that the controller seeks to maintain.
+ * This is the sum across all pools.
+ *
+ * @return
+ */
+
+ int getTargetCount();
+
+ boolean isTaskLive(int id);
+
+ /**
+ * Cancels the given task, reducing the target task count. Called
+ * from the UI to allow the user to select the specific task to end
+ * when reducing cluster size.
+ *
+ * @param id
+ * @return
+ */
+
+ boolean cancelTask(int id);
+
+ /**
+ * Whether this distribution of YARN supports disk resources.
+ * @return
+ */
+
+ boolean supportsDiskResource();
+
+ int getFreeNodeCount();
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterControllerImpl.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterControllerImpl.java
new file mode 100644
index 000000000..3c011ec54
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterControllerImpl.java
@@ -0,0 +1,785 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.yarn.appMaster.TaskLifecycleListener.Event;
+import org.apache.drill.yarn.core.DoYUtil;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
+
+/**
+ * Controls the Drill cluster by representing the current cluster state with a
+ * desired state, taking corrective action to keep the cluster in the desired
+ * state. The cluster as a whole has a state, as do each task (node) within the
+ * cluster.
+ * <p>
+ * This class is designed to allow unit tests. In general, testing the
+ * controller on a live cluster is tedious. This class encapsulates the
+ * controller algorithm so it can be driven by a simulated cluster.
+ * <p>
+ * This object is shared between threads, thus synchronized.
+ */
+
+public class ClusterControllerImpl implements ClusterController {
+ /**
+ * Controller lifecycle state.
+ */
+
+ public enum State {
+ /**
+ * Cluster is starting. Things are in a partially-built state. No tasks are
+ * started until the cluster moves to LIVE.
+ */
+
+ START,
+
+ /**
+ * Normal operating state: the controller seeks to maintain the desired
+ * number of tasks.
+ */
+
+ LIVE,
+
+ /**
+ * Controller is shutting down. Tasks are gracefully (where possible) ended;
+ * no new tasks are started. (That is, when we detect the exit of a task,
+ * the controller no longer immediately tries to start a replacement.
+ */
+
+ ENDING,
+
+ /**
+ * The controller has shut down. All tasks and threads are stopped. The
+ * controller allows the main thread (which has been patiently waiting) to
+ * continue, allowing the AM itself to shut down. Thus, this is a very
+ * short-lived state.
+ */
+
+ ENDED,
+
+ /**
+ * Something bad happened on start-up; the AM can't start and must shut
+ * down.
+ */
+
+ FAILED
+ }
+
+ private final static int PRIORITY_OFFSET = 1;
+
+ private static final Log LOG = LogFactory.getLog(ClusterControllerImpl.class);
+
+ /**
+ * Signals the completion of the cluster run. The main program waits on this
+ * mutex until all tasks complete (batch) or the cluster is explicitly shut
+ * down (persistent tasks.)
+ */
+
+ private Object completionMutex = new Object();
+
+ /**
+ * Maximum number of retries for each task launch.
+ */
+
+ protected int maxRetries = 3;
+
+ /**
+ * Controller state.
+ *
+ * @see {@link State}
+ */
+
+ State state = State.START;
+
+ /**
+ * Definition of the task types that can be run by this controller, along with
+ * the target task levels for each.
+ */
+
+ private Map<String, SchedulerStateActions> taskPools = new HashMap<>();
+
+ /**
+ * List of task pools prioritized in the order in which tasks should start.
+ * DoY supports only one task pool at present. The idea is to, later, support
+ * multiple pools that represent, say, pool 1 as the minimum number of
+ * Drillbits to run at all times, with pool 2 as extra Drillbits to start up
+ * during peak demand.
+ * <p>
+ * The priority also gives rise to YARN request priorities which are the only
+ * tool the AM has to associate container grants with the requests to which
+ * they correspond.
+ */
+
+ private List<SchedulerStateActions> prioritizedGroups = new ArrayList<>();
+
+ /**
+ * Cluster-wide association of YARN container IDs to tasks.
+ */
+
+ private Set<ContainerId> allocatedContainers = new HashSet<>();
+
+ /**
+ * Cluster-wide list of active tasks. Allows lookup from container ID to task
+ * (and then from task to task type.)
+ */
+
+ private Map<ContainerId, Task> activeContainers = new HashMap<>();
+
+ /**
+ * Tracks the tasks that have completed: either successfully (state == ENDED)
+ * or failed (state == FAILED). Eventually store this information elsewhere to
+ * avoid cluttering memory with historical data. Entries here are static
+ * copies, preserving the state at the time that the task completed.
+ */
+
+ private List<Task> completedTasks = new LinkedList<>();
+
+ /**
+ * Wrapper around the YARN API. Abstracts the details of YARN operations.
+ */
+
+ private final AMYarnFacade yarn;
+
+ /**
+ * Maximum number of new tasks to start on each "pulse" tick.
+ */
+
+ private int maxRequestsPerTick = 2;
+
+ private int stopTimoutMs = 10_000;
+
+ /**
+ * Time (in ms) between request to YARN to get an updated list of the node
+ * "inventory".
+ */
+
+ private int configPollPeriod = 60_000;
+ private long nextResourcePollTime;
+
+ /**
+ * List of nodes available in the cluster. Necessary as part of the process of
+ * ensuring that we run one Drillbit per node. (The YARN blacklist only half
+ * works for this purpose.)
+ */
+
+ private NodeInventory nodeInventory;
+
+ private long lastFailureCheckTime;
+
+ private int failureCheckPeriodMs = 60_000;
+
+ private int taskCheckPeriodMs = 10_000;
+ private long lastTaskCheckTime;
+
+ /**
+ * To increase code modularity, add-ons (such as the ZK monitor) register as
+ * lifecycle listeners that are alerted to "interesting" lifecycle events.
+ */
+
+ private List<TaskLifecycleListener> lifecycleListeners = new ArrayList<>();
+
+ /**
+ * Handy mechanism for setting properties on this controller that are
+ * available to plugins and UI without cluttering this class with member
+ * variables.
+ */
+
+ private Map<String, Object> properties = new HashMap<>();
+
+ /**
+ * When enabled, allows the controller to check for failures that result in no
+ * drillbits running. The controller will then automatically exit as no useful
+ * work can be done. Disable this to make debugging easier on a single-node
+ * cluster (lets you, say, start a "stray" drill bit and see what happens
+ * without the AM exiting.)
+ */
+
+ private boolean enableFailureCheck = true;
+
+ public ClusterControllerImpl(AMYarnFacade yarn) {
+ this.yarn = yarn;
+ }
+
+ @Override
+ public void enableFailureCheck(boolean flag) {
+ this.enableFailureCheck = flag;
+ }
+
+ /**
+ * Define a task type. Registration order is important: the controller starts
+ * task in the order that they are registered. Must happen before the YARN
+ * callbacks start.
+ *
+ * @param scheduler
+ */
+
+ @Override
+ public void registerScheduler(Scheduler scheduler) {
+ assert !taskPools.containsKey(scheduler.getName());
+ scheduler.setPriority(taskPools.size() + PRIORITY_OFFSET);
+ SchedulerStateActions taskGroup = new SchedulerStateImpl(this, scheduler);
+ taskPools.put(taskGroup.getName(), taskGroup);
+ prioritizedGroups.add(taskGroup);
+ }
+
+ /**
+ * Called when the caller has completed start-up and the controller should
+ * become live.
+ */
+
+ @Override
+ public synchronized void started() throws YarnFacadeException, AMException {
+ nodeInventory = new NodeInventory(yarn);
+
+ // Verify that no resource seeks a container larger than
+ // what YARN can provide. Ensures a graceful exit in this
+ // case.
+
+ Resource maxResource = yarn.getRegistrationResponse()
+ .getMaximumResourceCapability();
+ for (SchedulerStateActions group : prioritizedGroups) {
+ group.getScheduler().limitContainerSize(maxResource);
+ }
+ state = State.LIVE;
+ }
+
+ @Override
+ public synchronized void tick(long curTime) {
+ if (state == State.LIVE) {
+ adjustTasks(curTime);
+ requestContainers();
+ }
+ if (state == State.LIVE || state == State.ENDING) {
+ checkTasks(curTime);
+ }
+ }
+
+ /**
+ * Adjust the number of running tasks to match the desired level.
+ *
+ * @param curTime
+ */
+
+ private void adjustTasks(long curTime) {
+ if (enableFailureCheck && getFreeNodeCount() == 0) {
+ checkForFailure(curTime);
+ }
+ if (state != State.LIVE) {
+ return;
+ }
+ for (SchedulerStateActions group : prioritizedGroups) {
+ group.adjustTasks();
+ }
+ }
+
+ /**
+ * Get the approximate number of free YARN nodes (those that can
+ * accept a task request.) Starts with the number of nodes from
+ * the node inventory, then subtracts any in-flight requests (which
+ * do not, by definition, have node allocated.)
+ * <p>
+ * This approximation <b>does not</b> consider whether the node
+ * has sufficient resources to run a task; only whether the node
+ * itself exists.
+ * @return
+ */
+
+ @Override
+ public int getFreeNodeCount( ) {
+ int count = nodeInventory.getFreeNodeCount();
+ for (SchedulerStateActions group : prioritizedGroups) {
+ count -= group.getRequestCount( );
+ }
+ return Math.max( 0, count );
+ }
+
+ /**
+ * Check if the controller is unable to run any tasks. If so, and the option
+ * is enabled, then automatically exit since no useful work can be done.
+ *
+ * @param curTime
+ */
+
+ private void checkForFailure(long curTime) {
+ if (lastFailureCheckTime + failureCheckPeriodMs > curTime) {
+ return;
+ }
+ lastFailureCheckTime = curTime;
+ for (SchedulerStateActions group : prioritizedGroups) {
+ if (group.getTaskCount() > 0) {
+ return;
+ }
+ }
+ LOG.error(
+ "Application failure: no tasks are running and no nodes are available -- exiting.");
+ terminate(State.FAILED);
+ }
+
+ /**
+ * Periodically check tasks, handling any timeout issues.
+ *
+ * @param curTime
+ */
+
+ private void checkTasks(long curTime) {
+
+ // Check periodically, not on every tick.
+
+ if (lastTaskCheckTime + taskCheckPeriodMs > curTime) {
+ return;
+ }
+ lastTaskCheckTime = curTime;
+
+ // Check for task timeouts in states that have a timeout.
+
+ EventContext context = new EventContext(this);
+ for (SchedulerStateActions group : prioritizedGroups) {
+ context.setGroup(group);
+ group.checkTasks(context, curTime);
+ }
+ }
+
+ /**
+ * Get an update from YARN on available resources.
+ */
+
+ @Override
+ public void updateRMStatus() {
+ long curTime = System.currentTimeMillis();
+ if (nextResourcePollTime > curTime) {
+ return;
+ }
+
+ // yarnNodeCount = yarn.getNodeCount();
+ // LOG.info("YARN reports " + yarnNodeCount + " nodes.");
+
+ // Resource yarnResources = yarn.getResources();
+ // if (yarnResources != null) {
+ // LOG.info("YARN reports " + yarnResources.getMemory() + " MB, " +
+ // yarnResources.getVirtualCores()
+ // + " vcores available.");
+ // }
+ nextResourcePollTime = curTime + configPollPeriod;
+ }
+
+ /**
+ * Request any containers that have accumulated.
+ */
+
+ private void requestContainers() {
+ EventContext context = new EventContext(this);
+ for (SchedulerStateActions group : prioritizedGroups) {
+ context.setGroup(group);
+ if (group.requestContainers(context, maxRequestsPerTick)) {
+ break;
+ }
+ }
+ }
+
+ @Override
+ public synchronized void containersAllocated(List<Container> containers) {
+ EventContext context = new EventContext(this);
+ for (Container container : containers) {
+ if (allocatedContainers.contains(container.getId())) {
+ continue;
+ }
+
+ // We should never get a container on a node in the blacklist we
+ // sent to YARN. If we do, something is wrong. Log the error and
+ // reject the container. Else, bad things happen further along as
+ // the tracking mechanisms assume one task per node.
+
+ String host = container.getNodeId().getHost();
+ if (nodeInventory.isInUse(host)) {
+ LOG.error( "Host is in use, but YARN allocated a container: " +
+ DoYUtil.labelContainer(container) + " - container rejected." );
+ yarn.releaseContainer(container);
+ continue;
+ }
+
+ // The container is fine.
+
+ allocatedContainers.add(container.getId());
+ int priority = container.getPriority().getPriority();
+ int offset = priority - PRIORITY_OFFSET;
+ if (offset < 0 || offset > prioritizedGroups.size()) {
+ LOG.error("Container allocated with unknown priority " + DoYUtil.labelContainer(container));
+ continue;
+ }
+ context.setGroup(prioritizedGroups.get(offset));
+ context.group.containerAllocated(context, container);
+ }
+ }
+
+ @Override
+ public synchronized void containerStarted(ContainerId containerId) {
+ Task task = getTask(containerId);
+ if (task == null) {
+ return;
+ }
+ EventContext context = new EventContext(this, task);
+ context.getState().containerStarted(context);
+ LOG.trace("Container started: " + containerId);
+ }
+
+ @Override
+ public synchronized void taskStartFailed(ContainerId containerId,
+ Throwable t) {
+ Task task = getTask(containerId);
+ if (task == null) {
+ return;
+ }
+ EventContext context = new EventContext(this, task);
+ context.getState().launchFailed(context, t);
+ }
+
+ private Task getTask(ContainerId containerId) {
+ return activeContainers.get(containerId);
+ }
+
+ @Override
+ public synchronized void containerStopped(ContainerId containerId) {
+ // Ignored because the node manager notification is very
+ // unreliable. Better to rely on the Resource Manager
+ // completion request.
+ // Task task = getTask(containerId);
+ // if (task == null) {
+ // return; }
+ // EventContext context = new EventContext(this, task);
+ // context.getState().containerStopped(context);
+ }
+
+ @Override
+ public synchronized void containersCompleted(List<ContainerStatus> statuses) {
+ EventContext context = new EventContext(this);
+ for (ContainerStatus status : statuses) {
+ Task task = getTask(status.getContainerId());
+ if (task == null) {
+ if (task == null) {
+ // Will occur if a container was allocated but rejected.
+ // Any other occurrence is unexpected and an error.
+
+ LOG.warn("Container completed but no associated task state: " + status.getContainerId() );
+ }
+ continue;
+ }
+ context.setTask(task);
+ context.getState().containerCompleted(context, status);
+ }
+ checkStatus();
+ }
+
+ @Override
+ public synchronized float getProgress() {
+ int numerator = 0;
+ int denominator = 0;
+ for (SchedulerStateActions group : taskPools.values()) {
+ Scheduler sched = group.getScheduler();
+ int[] progress = sched.getProgress();
+ numerator += progress[0];
+ denominator += progress[1];
+ }
+ if (numerator == 0) {
+ return 1;
+ }
+ return (float) denominator / (float) numerator;
+ }
+
+ @Override
+ public synchronized void stopTaskFailed(ContainerId containerId,
+ Throwable t) {
+ Task task = getTask(containerId);
+ if (task == null) {
+ return;
+ }
+ EventContext context = new EventContext(this, task);
+ context.getState().stopTaskFailed(context, t);
+ }
+
+ @Override
+ public synchronized void resizeDelta(int delta) {
+ // TODO: offer the delta to each scheduler in turn.
+ // For now, we support only one scheduler.
+
+ prioritizedGroups.get(0).getScheduler().change(delta);
+ }
+
+ @Override
+ public synchronized int resizeTo(int n) {
+ // TODO: offer the delta to each scheduler in turn.
+ // For now, we support only one scheduler.
+
+ return prioritizedGroups.get(0).getScheduler().resize(n);
+ }
+
+ @Override
+ public synchronized void shutDown() {
+ LOG.info("Shut down request received");
+ this.state = State.ENDING;
+ EventContext context = new EventContext(this);
+ for (SchedulerStateActions group : prioritizedGroups) {
+ group.shutDown(context);
+ }
+ checkStatus();
+ }
+
+ @Override
+ public boolean waitForCompletion() {
+ start();
+ synchronized (completionMutex) {
+ try {
+ completionMutex.wait();
+ LOG.info("Controller shut down completed");
+ } catch (InterruptedException e) {
+ // Should not happen
+ }
+ }
+ return succeeded();
+ }
+
+ private void start() {
+ yarnReport();
+ }
+
+ private void yarnReport() {
+ RegisterApplicationMasterResponse response = yarn.getRegistrationResponse();
+ LOG.info("YARN queue: " + response.getQueue());
+ Resource resource = response.getMaximumResourceCapability();
+ LOG.info("YARN max resource: " + resource.getMemory() + " MB, "
+ + resource.getVirtualCores() + " cores");
+ EnumSet<SchedulerResourceTypes> types = response
+ .getSchedulerResourceTypes();
+ StringBuilder buf = new StringBuilder();
+ String sep = "";
+ for (SchedulerResourceTypes type : types) {
+ buf.append(sep);
+ buf.append(type.toString());
+ sep = ", ";
+ }
+ LOG.info("YARN scheduler resource types: " + buf.toString());
+ }
+
+ /**
+ * Check for overall completion. We are done when either we've successfully
+ * run all tasks, or we've run some and given up on others. We're done when
+ * the number of completed or failed tasks reaches our target.
+ */
+
+ private void checkStatus() {
+ if (state != State.ENDING) {
+ return;
+ }
+ for (SchedulerStateActions group : prioritizedGroups) {
+ if (!group.isDone()) {
+ return;
+ }
+ }
+ terminate(State.ENDED);
+ }
+
+ private void terminate(State state) {
+ this.state = state;
+ synchronized (completionMutex) {
+ completionMutex.notify();
+ }
+ }
+
+ public boolean isLive() {
+ return state == State.LIVE;
+ }
+
+ public boolean succeeded() {
+ return state == State.ENDED;
+ }
+
+ public void containerAllocated(Task task) {
+ activeContainers.put(task.getContainerId(), task);
+ }
+
+ public AMYarnFacade getYarn() {
+ return yarn;
+ }
+
+ public void containerReleased(Task task) {
+ activeContainers.remove(task.getContainerId());
+ }
+
+ public void taskEnded(Task task) {
+ completedTasks.add(task);
+ }
+
+ public void taskRetried(Task task) {
+ Task copy = task.copy();
+ copy.disposition = Task.Disposition.RETRIED;
+ completedTasks.add(copy);
+ }
+
+ public void taskGroupCompleted(SchedulerStateActions taskGroup) {
+ checkStatus();
+ }
+
+ public int getMaxRetries() {
+ return maxRetries;
+ }
+
+ public int getStopTimeoutMs() {
+ return stopTimoutMs;
+ }
+
+ @Override
+ public synchronized void reserveHost(String hostName) {
+ nodeInventory.reserve(hostName);
+ }
+
+ @Override
+ public synchronized void releaseHost(String hostName) {
+ nodeInventory.release(hostName);
+ }
+
+ public NodeInventory getNodeInventory() {
+ return nodeInventory;
+ }
+
+ @Override
+ public void setProperty(String key, Object value) {
+ properties.put(key, value);
+ }
+
+ @Override
+ public Object getProperty(String key) {
+ return properties.get(key);
+ }
+
+ @Override
+ public void registerLifecycleListener(TaskLifecycleListener listener) {
+ lifecycleListeners.add(listener);
+ }
+
+ public void fireLifecycleChange(Event event, EventContext context) {
+ for (TaskLifecycleListener listener : lifecycleListeners) {
+ listener.stateChange(event, context);
+ }
+ }
+
+ @Override
+ public void setMaxRetries(int value) {
+ maxRetries = value;
+ }
+
+ @Override
+ public int getTargetCount() {
+ int count = 0;
+ for (SchedulerStateActions group : prioritizedGroups) {
+ count += group.getScheduler().getTarget();
+ }
+ return count;
+ }
+
+ public State getState() {
+ return state;
+ }
+
+ @Override
+ public synchronized void visit(ControllerVisitor visitor) {
+ visitor.visit(this);
+ }
+
+ public List<SchedulerStateActions> getPools() {
+ return prioritizedGroups;
+ }
+
+ @Override
+ public synchronized void visitTasks(TaskVisitor visitor) {
+ for (SchedulerStateActions pool : prioritizedGroups) {
+ pool.visitTaskModels(visitor);
+ }
+ }
+
+ public List<Task> getHistory() {
+ return completedTasks;
+ }
+
+ @Override
+ public boolean isTaskLive(int id) {
+ for (SchedulerStateActions group : prioritizedGroups) {
+ Task task = group.getTask(id);
+ if (task != null) {
+ return task.isLive();
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public synchronized boolean cancelTask(int id) {
+ for (SchedulerStateActions group : prioritizedGroups) {
+ Task task = group.getTask(id);
+ if (task != null) {
+ group.cancel(task);
+ group.getScheduler().change(-1);
+ return true;
+ }
+ }
+ LOG.warn( "Requested to cancel task, but no task found: " + id );
+ return false;
+ }
+
+ @Override
+ public synchronized void completionAck(Task task, String propertyKey) {
+ EventContext context = new EventContext(this);
+ context.setTask(task);
+ context.getState().completionAck(context);
+ if (propertyKey != null) {
+ task.properties.remove(propertyKey);
+ }
+ }
+
+ @Override
+ public synchronized void startAck(Task task, String propertyKey,
+ Object value) {
+ if (propertyKey != null && value != null) {
+ task.properties.put(propertyKey, value);
+ }
+ EventContext context = new EventContext(this);
+ context.setTask(task);
+ context.getState().startAck(context);
+ }
+
+ @Override
+ public boolean supportsDiskResource() {
+ return getYarn().supportsDiskResource();
+ }
+
+ @Override
+ public void registryDown() { shutDown( ); }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ControllerFactory.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ControllerFactory.java
new file mode 100644
index 000000000..b8d6e06fe
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ControllerFactory.java
@@ -0,0 +1,30 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+public interface ControllerFactory {
+ public static class ControllerFactoryException extends Exception {
+ private static final long serialVersionUID = 1L;
+
+ public ControllerFactoryException(String msg, Exception e) {
+ super(msg, e);
+ }
+ }
+
+ Dispatcher build() throws ControllerFactoryException;
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ControllerVisitor.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ControllerVisitor.java
new file mode 100644
index 000000000..5774d7d2b
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ControllerVisitor.java
@@ -0,0 +1,22 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+public interface ControllerVisitor {
+ void visit(ClusterController controller);
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Dispatcher.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Dispatcher.java
new file mode 100644
index 000000000..f5257e659
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Dispatcher.java
@@ -0,0 +1,345 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.yarn.appMaster.AMRegistrar.AMRegistrationException;
+import org.apache.drill.yarn.appMaster.AMYarnFacade.YarnAppHostReport;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
+import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
+
+/**
+ * Dispatches YARN, timer and ZooKeeper events to the cluster controller.
+ * Allows the controller to be independent of the plumbing needed to
+ * receive events. Divides work among
+ * various components to separate concerns. Three streams of events
+ * feed into an app master "strategy". The three streams are
+ * <ol>
+ * <li>Resource manager</li>
+ * <li>Node manager</li>
+ * <li>Timer</li>
+ * </ol>
+ * <p>
+ * This class is "lightly" multi-threaded: it responds to events
+ * from the RM, NM and timer threads. Within each of these, events
+ * are sequential. So, synchronization is needed across the three event
+ * types, but not within event types. (That is, we won't see two RM events,
+ * say, occurring at the same time from separate threads.)
+ */
+
+public class Dispatcher
+{
+ private static final Log LOG = LogFactory.getLog(Dispatcher.class);
+
+ /**
+ * Handle YARN Resource Manager events. This is a separate class to clarify
+ * which events are from the Resource Manager.
+ */
+
+ private class ResourceCallback implements AMRMClientAsync.CallbackHandler {
+ @Override
+ public void onContainersAllocated(List<Container> containers) {
+ LOG.trace("NM: Containers allocated: " + containers.size());
+ controller.containersAllocated(containers);
+ }
+
+ @Override
+ public void onContainersCompleted(List<ContainerStatus> statuses) {
+ LOG.trace("NM: Containers completed: " + statuses.size());
+ controller.containersCompleted(statuses);
+ }
+
+ @Override
+ public void onShutdownRequest() {
+ LOG.trace("RM: Shutdown request");
+ controller.shutDown();
+ }
+
+ @Override
+ public void onNodesUpdated(List<NodeReport> updatedNodes) {
+ LOG.trace("RM: Nodes updated, count= " + updatedNodes.size());
+ }
+
+ @Override
+ public float getProgress() {
+ // getProgress is called on each fetch from the NM response queue.
+ // This is a good time to update status, even if it looks a bit
+ // bizarre...
+
+ controller.updateRMStatus();
+ return controller.getProgress();
+ }
+
+ @Override
+ public void onError(Throwable e) {
+ LOG.error("Fatal RM Error: " + e.getMessage());
+ LOG.error("AM Shutting down!");
+ controller.shutDown();
+ }
+ }
+
+ /**
+ * Handle YARN Node Manager events. This is a separate class to clarify which
+ * events are, in fact, from the node manager.
+ */
+
+ public class NodeCallback implements NMClientAsync.CallbackHandler {
+ @Override
+ public void onStartContainerError(ContainerId containerId, Throwable t) {
+ LOG.trace("CNM: ontainer start error: " + containerId, t);
+ controller.taskStartFailed(containerId, t);
+ }
+
+ @Override
+ public void onContainerStarted(ContainerId containerId,
+ Map<String, ByteBuffer> allServiceResponse) {
+ LOG.trace("NM: Container started: " + containerId);
+ controller.containerStarted(containerId);
+ }
+
+ @Override
+ public void onContainerStatusReceived(ContainerId containerId,
+ ContainerStatus containerStatus) {
+ LOG.trace("NM: Container status: " + containerId + " - "
+ + containerStatus.toString());
+ }
+
+ @Override
+ public void onGetContainerStatusError(ContainerId containerId,
+ Throwable t) {
+ LOG.trace("NM: Container error: " + containerId, t);
+ }
+
+ @Override
+ public void onStopContainerError(ContainerId containerId, Throwable t) {
+ LOG.trace("NM: Stop container error: " + containerId, t);
+ controller.stopTaskFailed(containerId, t);
+ }
+
+ @Override
+ public void onContainerStopped(ContainerId containerId) {
+ LOG.trace("NM: Container stopped: " + containerId);
+ controller.containerStopped(containerId);
+ }
+ }
+
+ /**
+ * Handle timer events: a constant tick to handle time-based actions such as
+ * timeouts.
+ */
+
+ public class TimerCallback implements PulseRunnable.PulseCallback {
+ /**
+ * The lifecycle of each task is driven by RM and NM callbacks. We use the
+ * timer to start the process. While this is overkill here, in a real app,
+ * we'd check requested resource levels (which might change) and number of
+ * tasks (which might change if tasks die), and take corrective action:
+ * adding or removing tasks.
+ */
+
+ @Override
+ public void onTick(long curTime) {
+ for (Pollable pollable : pollables) {
+ pollable.tick(curTime);
+ }
+ controller.tick(curTime);
+ }
+ }
+
+ private AMYarnFacade yarn;
+ private ClusterController controller;
+
+ /**
+ * Add-on tools that are called once on each timer tick.
+ */
+
+ private List<Pollable> pollables = new ArrayList<>();
+
+ /**
+ * Add-ons for which the dispatcher should managed the start/end lifecycle.
+ */
+
+ private List<DispatcherAddOn> addOns = new ArrayList<>();
+ private String trackingUrl;
+ private AMRegistrar amRegistrar;
+ private int httpPort;
+ private PulseRunnable timer;
+ private Thread pulseThread;
+ private final int timerPeriodMs;
+
+ public Dispatcher(int timerPeriodMs) {
+ this.timerPeriodMs = timerPeriodMs;
+ }
+
+ public void setYarn(AMYarnFacade yarn) throws YarnFacadeException {
+ this.yarn = yarn;
+ controller = new ClusterControllerImpl(yarn);
+ }
+
+ public ClusterController getController() {
+ return controller;
+ }
+
+ public void registerPollable(Pollable pollable) {
+ pollables.add(pollable);
+ }
+
+ public void registerAddOn(DispatcherAddOn addOn) {
+ addOns.add(addOn);
+ }
+
+ public void setHttpPort(int port) {
+ httpPort = port;
+ }
+
+ public void setTrackingUrl(String trackingUrl) {
+ this.trackingUrl = trackingUrl;
+ }
+
+ public String getTrackingUrl() {
+ return yarn.getTrackingUrl();
+ }
+
+ public void setAMRegistrar(AMRegistrar registrar) {
+ amRegistrar = registrar;
+ }
+
+ /**
+ * Start the dispatcher by initializing YARN and registering the AM.
+ *
+ * @return true if successful, false if the dispatcher did not start.
+ */
+
+ public boolean start() throws YarnFacadeException {
+
+ // Start the connection to YARN to get information about this app, and to
+ // create a session we can use to report problems.
+
+ try {
+ setup();
+ } catch (AMException e) {
+ String msg = e.getMessage();
+ LOG.error("Fatal error: " + msg);
+ yarn.finish(false, msg);
+ return false;
+ }
+
+ // Ensure that this is the only AM. If not, shut down the AM,
+ // reporting to YARN that this is a failure and the message explaining
+ // the conflict. Report this as a SUCCESS run so that YARN does not
+ // attempt to retry the AM.
+
+ try {
+ register();
+ } catch (AMRegistrationException e) {
+ LOG.error(e.getMessage(), e);
+ yarn.finish(true, e.getMessage());
+ return false;
+ }
+ return true;
+ }
+
+ public void run() throws YarnFacadeException {
+ // Only if registration is successful do we start the pulse thread
+ // which will cause containers to be requested.
+
+ startTimer();
+
+ // Run until the controller decides to shut down.
+
+ LOG.trace("Running");
+ boolean success = controller.waitForCompletion();
+
+ // Shut down.
+
+ LOG.trace("Finishing");
+ finish(success, null);
+ }
+
+ private void setup() throws YarnFacadeException, AMException {
+ LOG.trace("Starting YARN agent");
+ yarn.start(new ResourceCallback(), new NodeCallback());
+ String url = trackingUrl.replace("<port>", Integer.toString(httpPort));
+ if (DrillOnYarnConfig.config().getBoolean(DrillOnYarnConfig.HTTP_ENABLE_SSL)) {
+ url = url.replace("http:", "https:");
+ }
+ LOG.trace("Registering YARN application, URL: " + url);
+ yarn.register(url);
+ controller.started();
+
+ for (DispatcherAddOn addOn : addOns) {
+ addOn.start(controller);
+ }
+ }
+
+ private void register() throws AMRegistrationException {
+ if (amRegistrar == null) {
+ LOG.warn(
+ "No AM Registrar provided: cannot check if this is the only AM for the Drill cluster.");
+ } else {
+ YarnAppHostReport rpt = yarn.getAppHostReport();
+ amRegistrar.register(rpt.amHost, httpPort, rpt.appId);
+ }
+ }
+
+ private void startTimer() {
+ timer = new PulseRunnable(timerPeriodMs, new TimerCallback());
+
+ // Start the pulse thread after registering so that we're in
+ // a state where we can interact with the RM.
+
+ pulseThread = new Thread(timer);
+ pulseThread.setName("Pulse");
+ pulseThread.start();
+ }
+
+ private void finish(boolean success, String msg) throws YarnFacadeException {
+ for (DispatcherAddOn addOn : addOns) {
+ addOn.finish(controller);
+ }
+
+ LOG.trace("Shutting down YARN agent");
+
+ // Stop the timer thread first. This ensures that the
+ // timer events don't try to use the YARN API during
+ // shutdown.
+
+ stopTimer();
+ yarn.finish(success, msg);
+ }
+
+ private void stopTimer() {
+ timer.stop();
+ try {
+ pulseThread.join();
+ } catch (InterruptedException e) {
+ // Ignore
+ }
+ }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DispatcherAddOn.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DispatcherAddOn.java
new file mode 100644
index 000000000..5c7100b93
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DispatcherAddOn.java
@@ -0,0 +1,30 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+/**
+ * Interface for an add-on to the dispatcher that
+ * should be started at start of the run and ended
+ * at the end of the run.
+ */
+
+public interface DispatcherAddOn {
+ void start(ClusterController controller);
+
+ void finish(ClusterController controller);
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillApplicationMaster.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillApplicationMaster.java
new file mode 100644
index 000000000..c0db9a1f8
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillApplicationMaster.java
@@ -0,0 +1,119 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.yarn.appMaster.ControllerFactory.ControllerFactoryException;
+import org.apache.drill.yarn.appMaster.http.WebServer;
+import org.apache.drill.yarn.core.DoyConfigException;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+
+/**
+ * Application Master for Drill. The name is visible when using the "jps"
+ * command and is chosen to make sense on a busy YARN node.
+ * <p>
+ * To debug this AM use the customized unmanaged AM launcher in this
+ * jar. (The "stock" YARN version does not give you time to attach
+ * the debugger.)
+ * <pre><code>
+ * TARGET_JAR=/your-git-folder/drill-yarn/target/drill-yarn-1.6-SNAPSHOT.jar
+ * TARGET_CLASS=org.apache.drill.yarn.appMaster.ApplicationMaster
+ * LAUNCHER_JAR=$TARGET_JAR
+ * LAUNCHER_CLASS=org.apache.drill.yarn.mock.UnmanagedAMLauncher
+ * $HH/bin/hadoop jar $LAUNCHER_JAR \
+ * $LAUNCHER_CLASS -classpath $TARGET_JAR \
+ * -cmd "java -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5005 \
+ * $TARGET_CLASS"
+ * </pre></code>
+ */
+
+public class DrillApplicationMaster {
+ private static final Log LOG = LogFactory
+ .getLog(DrillApplicationMaster.class);
+
+ public static void main(String[] args) {
+ LOG.trace("Drill Application Master starting.");
+
+ // Load the configuration. Assumes that the user's Drill-on-YARN
+ // configuration was archived along with the Drill software in
+ // the $DRILL_HOME/conf directory, and that $DRILL_HOME/conf is
+ // on the class-path.
+
+ try {
+ DrillOnYarnConfig.load().setAmDrillHome();
+ } catch (DoyConfigException e) {
+ System.err.println(e.getMessage());
+ System.exit(-1);
+ }
+
+ // Build the dispatcher using the Drillbit factory. Allows inserting
+ // other factories for testing, or if we need to manage a cluster of
+ // processes other than Drillbits.
+
+ // Dispatcher am = (new SimpleBatchFactory( )).build( );
+ // Dispatcher am = (new MockDrillbitFactory( )).build( );
+ Dispatcher dispatcher;
+ try {
+ dispatcher = (new DrillControllerFactory()).build();
+ } catch (ControllerFactoryException e) {
+ LOG.error("Setup failed, exiting: " + e.getMessage(), e);
+ System.exit(-1);
+ return;
+ }
+
+ // Start the Dispatcher. This will return false if this AM conflicts with
+ // a running AM.
+
+ try {
+ if (!dispatcher.start()) {
+ return;
+ }
+ } catch (Throwable e) {
+ LOG.error("Fatal error, exiting: " + e.getMessage(), e);
+ System.exit(-1);
+ }
+
+ // Create and start the web server. Do this after starting the AM
+ // so that we don't learn about a conflict via the a web server port
+ // conflict.
+
+ WebServer webServer = new WebServer(dispatcher);
+ try {
+ webServer.start();
+ } catch (Exception e) {
+ LOG.error("Web server setup failed, exiting: " + e.getMessage(), e);
+ System.exit(-1);
+ }
+
+ // Run the dispatcher until the cluster shuts down.
+
+ try {
+ dispatcher.run();
+ } catch (Throwable e) {
+ LOG.error("Fatal error, exiting: " + e.getMessage(), e);
+ System.exit(-1);
+ } finally {
+ try {
+ webServer.close();
+ } catch (Exception e) {
+ // Ignore
+ }
+ }
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillControllerFactory.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillControllerFactory.java
new file mode 100644
index 000000000..013fdba0c
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillControllerFactory.java
@@ -0,0 +1,398 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.yarn.core.ContainerRequestSpec;
+import org.apache.drill.yarn.core.DfsFacade;
+import org.apache.drill.yarn.core.DfsFacade.DfsFacadeException;
+import org.apache.drill.yarn.core.DoYUtil;
+import org.apache.drill.yarn.core.DoyConfigException;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.drill.yarn.core.LaunchSpec;
+import org.apache.drill.yarn.appMaster.http.AMSecurityManagerImpl;
+import org.apache.drill.yarn.core.ClusterDef;
+import org.apache.drill.yarn.zk.ZKClusterCoordinatorDriver;
+import org.apache.drill.yarn.zk.ZKRegistry;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+
+import com.typesafe.config.Config;
+
+/**
+ * Builds a controller for a cluster of Drillbits. The AM is designed to be
+ * mostly generic; only this class contains knowledge that the tasks being
+ * managed are drillbits. This design ensures that we can add other Drill
+ * components in the future without the need to make major changes to the AM
+ * logic.
+ * <p>
+ * The controller consists of a generic dispatcher and cluster controller, along
+ * with a Drill-specific scheduler and task launch specification. Drill also
+ * includes an interface to ZooKeeper to monitor Drillbits.
+ * <p>
+ * The AM is launched by YARN. All it knows is what is in its launch environment
+ * or configuration files. The client must set up all the information that the
+ * AM needs. Static information appears in configuration files. But, dynamic
+ * information (or that which is inconvenient to repeat in configuration files)
+ * must arrive in environment variables. See {@link DrillOnYarnConfig} for more
+ * information.
+ */
+
+public class DrillControllerFactory implements ControllerFactory {
+ private static final Log LOG = LogFactory.getLog(DrillControllerFactory.class);
+ private Config config = DrillOnYarnConfig.config();
+ private String drillArchivePath;
+ private String siteArchivePath;
+ private boolean localized;
+
+ @Override
+ public Dispatcher build() throws ControllerFactoryException {
+ LOG.info(
+ "Initializing AM for " + config.getString(DrillOnYarnConfig.APP_NAME));
+ Dispatcher dispatcher;
+ try {
+ Map<String, LocalResource> resources = prepareResources();
+
+ TaskSpec taskSpec = buildDrillTaskSpec(resources);
+
+ // Prepare dispatcher
+
+ int timerPeriodMs = config.getInt(DrillOnYarnConfig.AM_TICK_PERIOD_MS);
+ dispatcher = new Dispatcher(timerPeriodMs);
+ int pollPeriodMs = config.getInt(DrillOnYarnConfig.AM_POLL_PERIOD_MS);
+ AMYarnFacadeImpl yarn = new AMYarnFacadeImpl(pollPeriodMs);
+ dispatcher.setYarn(yarn);
+ dispatcher.getController()
+ .setMaxRetries(config.getInt(DrillOnYarnConfig.DRILLBIT_MAX_RETRIES));
+
+ int requestTimeoutSecs = DrillOnYarnConfig.config().getInt( DrillOnYarnConfig.DRILLBIT_REQUEST_TIMEOUT_SEC);
+ int maxExtraNodes = DrillOnYarnConfig.config().getInt(DrillOnYarnConfig.DRILLBIT_MAX_EXTRA_NODES);
+
+ // Assume basic scheduler for now.
+ ClusterDef.ClusterGroup pool = ClusterDef.getCluster(config, 0);
+ Scheduler testGroup = new DrillbitScheduler(pool.getName(), taskSpec,
+ pool.getCount(), requestTimeoutSecs, maxExtraNodes);
+ dispatcher.getController().registerScheduler(testGroup);
+ pool.modifyTaskSpec(taskSpec);
+
+ // ZooKeeper setup
+
+ buildZooKeeper(config, dispatcher);
+ } catch (YarnFacadeException | DoyConfigException e) {
+ throw new ControllerFactoryException("Drill AM intitialization failed", e);
+ }
+
+ // Tracking Url
+ // TODO: HTTPS support
+
+ dispatcher.setHttpPort(config.getInt(DrillOnYarnConfig.HTTP_PORT));
+ String trackingUrl = null;
+ if (config.getBoolean(DrillOnYarnConfig.HTTP_ENABLED)) {
+ trackingUrl = "http://<host>:<port>/redirect";
+ dispatcher.setTrackingUrl(trackingUrl);
+ }
+
+ // Enable/disable check for auto shutdown when no nodes are running.
+
+ dispatcher.getController().enableFailureCheck(
+ config.getBoolean(DrillOnYarnConfig.AM_ENABLE_AUTO_SHUTDOWN));
+
+ // Define the security manager
+
+ AMSecurityManagerImpl.setup();
+
+ return dispatcher;
+ }
+
+ /**
+ * Prepare the files ("resources" in YARN terminology) that YARN should
+ * download ("localize") for the Drillbit. We need both the Drill software and
+ * the user's site-specific configuration.
+ *
+ * @return
+ * @throws YarnFacadeException
+ */
+
+ private Map<String, LocalResource> prepareResources()
+ throws YarnFacadeException {
+ try {
+ DfsFacade dfs = new DfsFacade(config);
+ localized = dfs.isLocalized();
+ if (!localized) {
+ return null;
+ }
+ dfs.connect();
+ Map<String, LocalResource> resources = new HashMap<>();
+ DrillOnYarnConfig drillConfig = DrillOnYarnConfig.instance();
+
+ // Localize the Drill archive.
+
+ drillArchivePath = drillConfig.getDrillArchiveDfsPath();
+ DfsFacade.Localizer localizer = new DfsFacade.Localizer(dfs,
+ drillArchivePath);
+ String key = config.getString(DrillOnYarnConfig.DRILL_ARCHIVE_KEY);
+ localizer.defineResources(resources, key);
+ LOG.info("Localizing " + drillArchivePath + " with key \"" + key + "\"");
+
+ // Localize the site archive, if any.
+
+ siteArchivePath = drillConfig.getSiteArchiveDfsPath();
+ if (siteArchivePath != null) {
+ localizer = new DfsFacade.Localizer(dfs, siteArchivePath);
+ key = config.getString(DrillOnYarnConfig.SITE_ARCHIVE_KEY);
+ localizer.defineResources(resources, key);
+ LOG.info("Localizing " + siteArchivePath + " with key \"" + key + "\"");
+ }
+ return resources;
+ } catch (DfsFacadeException e) {
+ throw new YarnFacadeException(
+ "Failed to get DFS status for Drill archive", e);
+ }
+ }
+
+ /**
+ * Constructs the Drill launch command. The launch uses the YARN-specific
+ * yarn-drillbit.sh script, setting up the required input environment
+ * variables.
+ * <p>
+ * This is an exercise in getting many details just right. The code here sets
+ * the environment variables required by (and documented in) yarn-drillbit.sh.
+ * The easiest way to understand this code is to insert an "echo" statement in
+ * drill-bit.sh to echo the launch command there. Then, look in YARN's NM
+ * private container directory for the launch_container.sh script to see the
+ * command generated by the following code. Compare the two to validate that
+ * the code does the right thing.
+ * <p>
+ * This class is very Linux-specific. The usual adjustments must be made to
+ * adapt it to Windows.
+ *
+ * @param config
+ * @return
+ * @throws DoyConfigException
+ */
+
+ private TaskSpec buildDrillTaskSpec(Map<String, LocalResource> resources)
+ throws DoyConfigException {
+ DrillOnYarnConfig doyConfig = DrillOnYarnConfig.instance();
+
+ // Drillbit launch description
+
+ ContainerRequestSpec containerSpec = new ContainerRequestSpec();
+ containerSpec.memoryMb = config.getInt(DrillOnYarnConfig.DRILLBIT_MEMORY);
+ containerSpec.vCores = config.getInt(DrillOnYarnConfig.DRILLBIT_VCORES);
+ containerSpec.disks = config.getDouble(DrillOnYarnConfig.DRILLBIT_DISKS);
+
+ LaunchSpec drillbitSpec = new LaunchSpec();
+
+ // The drill home location is either a non-localized location,
+ // or, more typically, the expanded Drill directory under the
+ // container's working directory. When the localized directory,
+ // we rely on the fact that the current working directory is
+ // set to the container directory, so we just need the name
+ // of the Drill folder under the cwd.
+
+ String drillHome = doyConfig.getRemoteDrillHome();
+ drillbitSpec.env.put("DRILL_HOME", drillHome);
+ LOG.trace("Drillbit DRILL_HOME: " + drillHome);
+
+ // Heap memory
+
+ addIfSet(drillbitSpec, DrillOnYarnConfig.DRILLBIT_HEAP, "DRILL_HEAP");
+
+ // Direct memory
+
+ addIfSet(drillbitSpec, DrillOnYarnConfig.DRILLBIT_DIRECT_MEM,
+ "DRILL_MAX_DIRECT_MEMORY");
+
+ // Code cache
+
+ addIfSet(drillbitSpec, DrillOnYarnConfig.DRILLBIT_CODE_CACHE,
+ "DRILLBIT_CODE_CACHE_SIZE");
+
+ // Any additional VM arguments from the config file.
+
+ addIfSet(drillbitSpec, DrillOnYarnConfig.DRILLBIT_VM_ARGS,
+ "DRILL_JVM_OPTS");
+
+ // Any user-specified library path
+
+ addIfSet(drillbitSpec, DrillOnYarnConfig.JAVA_LIB_PATH,
+ DrillOnYarnConfig.DOY_LIBPATH_ENV_VAR);
+
+ // Drill logs.
+ // Relies on the LOG_DIR_EXPANSION_VAR marker which is replaced by
+ // the container log directory.
+
+ if (!config.getBoolean(DrillOnYarnConfig.DISABLE_YARN_LOGS)) {
+ drillbitSpec.env.put("DRILL_YARN_LOG_DIR",
+ ApplicationConstants.LOG_DIR_EXPANSION_VAR);
+ }
+
+ // Debug option.
+
+ if (config.getBoolean(DrillOnYarnConfig.DRILLBIT_DEBUG_LAUNCH)) {
+ drillbitSpec.env.put(DrillOnYarnConfig.DRILL_DEBUG_ENV_VAR, "1");
+ }
+
+ // Hadoop home should be set in drill-env.sh since it is needed
+ // for client launch as well as the AM.
+
+ // addIfSet( drillbitSpec, DrillOnYarnConfig.HADOOP_HOME, "HADOOP_HOME" );
+
+ // Garbage collection (gc) logging. In drillbit.sh logging can be
+ // configured to go anywhere. In YARN, all logs go to the YARN log
+ // directory; the gc log file is always called "gc.log".
+
+ if (config.getBoolean(DrillOnYarnConfig.DRILLBIT_LOG_GC)) {
+ drillbitSpec.env.put("ENABLE_GC_LOG", "1");
+ }
+
+ // Class path additions.
+
+ addIfSet(drillbitSpec, DrillOnYarnConfig.DRILLBIT_PREFIX_CLASSPATH,
+ DrillOnYarnConfig.DRILL_CLASSPATH_PREFIX_ENV_VAR);
+ addIfSet(drillbitSpec, DrillOnYarnConfig.DRILLBIT_CLASSPATH,
+ DrillOnYarnConfig.DRILL_CLASSPATH_ENV_VAR);
+
+ // Drill-config.sh has specific entries for Hadoop and Hbase. To prevent
+ // an endless number of such one-off cases, we add a general extension
+ // class path. But, we retain Hadoop and Hbase for backward compatibility.
+
+ addIfSet(drillbitSpec, DrillOnYarnConfig.DRILLBIT_EXTN_CLASSPATH,
+ "EXTN_CLASSPATH");
+ addIfSet(drillbitSpec, DrillOnYarnConfig.HADOOP_CLASSPATH,
+ "DRILL_HADOOP_CLASSPATH");
+ addIfSet(drillbitSpec, DrillOnYarnConfig.HBASE_CLASSPATH,
+ "DRILL_HBASE_CLASSPATH");
+
+ // Note that there is no equivalent of niceness for YARN: YARN controls
+ // the niceness of its child processes.
+
+ // Drillbit launch script under YARN
+ // Here we can use DRILL_HOME because all env vars are set before
+ // issuing this command.
+
+ drillbitSpec.command = "$DRILL_HOME/bin/yarn-drillbit.sh";
+
+ // Configuration (site directory), if given.
+
+ String siteDirPath = doyConfig.getRemoteSiteDir();
+ if (siteDirPath != null) {
+ drillbitSpec.cmdArgs.add("--site");
+ drillbitSpec.cmdArgs.add(siteDirPath);
+ }
+
+ // Localized resources
+
+ if (resources != null) {
+ drillbitSpec.resources.putAll(resources);
+ }
+
+ // Container definition.
+
+ TaskSpec taskSpec = new TaskSpec();
+ taskSpec.name = "Drillbit";
+ taskSpec.containerSpec = containerSpec;
+ taskSpec.launchSpec = drillbitSpec;
+ taskSpec.maxRetries = config.getInt(DrillOnYarnConfig.DRILLBIT_MAX_RETRIES);
+ return taskSpec;
+ }
+
+ /**
+ * Utility method to create an environment variable in the process launch
+ * specification if a given Drill-on-YARN configuration variable is set,
+ * copying the config value to the environment variable.
+ *
+ * @param spec
+ * @param configParam
+ * @param envVar
+ */
+
+ public void addIfSet(LaunchSpec spec, String configParam, String envVar) {
+ String value = config.getString(configParam);
+ if (!DoYUtil.isBlank(value)) {
+ spec.env.put(envVar, value);
+ }
+ }
+
+ public static class ZKRegistryAddOn implements DispatcherAddOn {
+ ZKRegistry zkRegistry;
+
+ public ZKRegistryAddOn(ZKRegistry zkRegistry) {
+ this.zkRegistry = zkRegistry;
+ }
+
+ @Override
+ public void start(ClusterController controller) {
+ zkRegistry.start(controller);
+ }
+
+ @Override
+ public void finish(ClusterController controller) {
+ zkRegistry.finish(controller);
+ }
+ }
+
+ /**
+ * Create the Drill-on-YARN version of the ZooKeeper cluster coordinator.
+ * Compared to the Drill version, this one takes its parameters via a builder
+ * pattern in the form of the cluster coordinator driver.
+ *
+ * @param config
+ * @param dispatcher
+ */
+
+ private void buildZooKeeper(Config config, Dispatcher dispatcher) {
+ String zkConnect = config.getString(DrillOnYarnConfig.ZK_CONNECT);
+ String zkRoot = config.getString(DrillOnYarnConfig.ZK_ROOT);
+ String clusterId = config.getString(DrillOnYarnConfig.CLUSTER_ID);
+ int failureTimeoutMs = config
+ .getInt(DrillOnYarnConfig.ZK_FAILURE_TIMEOUT_MS);
+ int retryCount = config.getInt(DrillOnYarnConfig.ZK_RETRY_COUNT);
+ int retryDelayMs = config.getInt(DrillOnYarnConfig.ZK_RETRY_DELAY_MS);
+ int userPort = config.getInt(DrillOnYarnConfig.DRILLBIT_USER_PORT);
+ int bitPort = config.getInt(DrillOnYarnConfig.DRILLBIT_BIT_PORT);
+ ZKClusterCoordinatorDriver driver = new ZKClusterCoordinatorDriver()
+ .setConnect(zkConnect, zkRoot, clusterId)
+ .setFailureTimoutMs(failureTimeoutMs)
+ .setRetryCount(retryCount)
+ .setRetryDelayMs(retryDelayMs)
+ .setPorts(userPort, bitPort, bitPort + 1);
+ ZKRegistry zkRegistry = new ZKRegistry(driver);
+ dispatcher.registerAddOn(new ZKRegistryAddOn(zkRegistry));
+
+ // The ZK driver is started and stopped in conjunction with the
+ // controller lifecycle.
+
+ dispatcher.getController().registerLifecycleListener(zkRegistry);
+
+ // The ZK driver also handles registering the AM for the cluster.
+
+ dispatcher.setAMRegistrar(driver);
+
+ // The UI needs access to ZK to report unmanaged drillbits. We use
+ // a property to avoid unnecessary code dependencies.
+
+ dispatcher.getController().setProperty(ZKRegistry.CONTROLLER_PROPERTY,
+ zkRegistry);
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillbitScheduler.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillbitScheduler.java
new file mode 100644
index 000000000..76936b58a
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillbitScheduler.java
@@ -0,0 +1,51 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+public class DrillbitScheduler extends AbstractDrillbitScheduler {
+ private int requestTimeoutSecs;
+ private int maxExtraNodes;
+
+
+ public DrillbitScheduler(String name, TaskSpec taskSpec, int quantity,
+ int requestTimeoutSecs, int maxExtraNodes) {
+ super("basic", name, quantity);
+ this.taskSpec = taskSpec;
+ this.requestTimeoutSecs = requestTimeoutSecs;
+ this.maxExtraNodes = maxExtraNodes;
+ }
+
+ /**
+ * Set the number of running tasks to the quantity given.
+ * Limits the quantity to only a small margin above the number
+ * of estimated free YARN nodes. This avoids a common users error
+ * where someone requests 20 nodes on a 5-node cluster.
+ */
+
+ @Override
+ public int resize(int level) {
+ int limit = quantity + state.getController().getFreeNodeCount( ) +
+ maxExtraNodes;
+ return super.resize( Math.min( limit, level ) );
+ }
+
+ @Override
+ public int getRequestTimeoutSec() {
+ return requestTimeoutSecs;
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/EventContext.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/EventContext.java
new file mode 100644
index 000000000..bec8cf935
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/EventContext.java
@@ -0,0 +1,70 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import org.apache.drill.yarn.appMaster.Scheduler.TaskManager;
+
+public class EventContext {
+ public final AMYarnFacade yarn;
+ public final ClusterControllerImpl controller;
+ public SchedulerStateImpl group;
+ public Task task;
+
+ public EventContext(ClusterControllerImpl controller) {
+ yarn = controller.getYarn();
+ this.controller = controller;
+ }
+
+ public EventContext(ClusterController controller) {
+ this((ClusterControllerImpl) controller);
+ }
+
+ public EventContext(ClusterControllerImpl controller, Task task) {
+ this(controller);
+ setTask(task);
+ }
+
+ /**
+ * For testing only, omits the controller and YARN.
+ *
+ * @param task
+ */
+
+ public EventContext(Task task) {
+ controller = null;
+ yarn = null;
+ this.task = task;
+ }
+
+ public void setTask(Task task) {
+ this.task = task;
+ group = task.getGroup();
+ }
+
+ public TaskState getState() {
+ return task.state;
+ }
+
+ public void setGroup(SchedulerStateActions group) {
+ this.group = (SchedulerStateImpl) group;
+ }
+
+ public TaskManager getTaskManager() {
+ return group.getScheduler().getTaskManager();
+ }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/NodeInventory.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/NodeInventory.java
new file mode 100644
index 000000000..ec20307d2
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/NodeInventory.java
@@ -0,0 +1,195 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+
+/**
+ * Creates an AM-side inventory of cluster nodes. Used to track node
+ * reservations (container allocations) to prevent requesting multiple
+ * containers on the same node. Tracks blacklisted nodes that have failed too
+ * often. Since YARN will discard our blacklist if we add to many nodes, tracks
+ * when a container is allocated on a blacklisted node and signals that the
+ * cluster is in a bad state.
+ */
+
+public class NodeInventory {
+ private static final Log LOG = LogFactory.getLog(NodeInventory.class);
+
+ /**
+ * Indicates the case in which we've failed so many nodes that YARN has
+ * cancelled some of our blacklist entries and we've received a container for
+ * a blacklisted node. At this point, we should stop adding new tasks else
+ * we'll get into a nasty loop.
+ */
+ private boolean failed;
+
+ private Map<String, String> nodeMap = new HashMap<>();
+
+ /**
+ * The set of nodes available that YARN reports are available.
+ * Not clear if these are all nodes in the cluster, or just those usable
+ * by the current app (when the app is associated to a queue that
+ * uses node labels.)
+ */
+
+ private Map<String, NodeReport> yarnNodes = new HashMap<>();
+
+ /**
+ * The set of nodes in use by Drill. Includes both nodes on which the AM
+ * has requested to run Drillbits, and those nodes found to be running
+ * "stray" Drillbits started outside of DoY.
+ */
+
+ private Set<String> nodesInUse = new HashSet<>();
+
+ /**
+ * Nodes that have failed (typically due to mis-configuration) and
+ * are to be excluded from future container requests.
+ */
+
+ private Set<String> blacklist = new HashSet<>();
+ private final AMYarnFacade yarn;
+
+ public NodeInventory(AMYarnFacade yarn) throws YarnFacadeException {
+ this.yarn = yarn;
+ buildNodeMap();
+ }
+
+ private void buildNodeMap() throws YarnFacadeException {
+ List<NodeReport> nodes = yarn.getNodeReports();
+ for (NodeReport node : nodes) {
+ String hostName = node.getNodeId().getHost();
+ nodeMap.put(hostName, node.getHttpAddress());
+ yarnNodes.put(hostName, node);
+ }
+ if (LOG.isInfoEnabled()) {
+ LOG.info("YARN Node report");
+ for (NodeReport node : nodes) {
+ LOG.info("Node: " + node.getHttpAddress() + ", Rack: "
+ + node.getRackName() + " has " + node.getCapability().getMemory()
+ + " MB, " + node.getCapability().getVirtualCores()
+ + " vcores, labels: " + node.getNodeLabels());
+ }
+ }
+ }
+
+ public boolean isFailed() {
+ return failed;
+ }
+
+ public void reserve(Container container) {
+ reserve(container.getNodeId().getHost());
+ }
+
+ public void reserve(String hostName) {
+ if (blacklist.contains(hostName)) {
+ LOG.error( "Node to be reserved is in the blacklist: " + hostName );
+ failed = true;
+ }
+ if (nodesInUse.contains(hostName)) {
+ LOG.error( "Node to be reserved is already in use: " + hostName );
+ return;
+ }
+ if (!yarnNodes.containsKey(hostName)) {
+ LOG.warn( "Node to be reserved was not in YARN node inventory: " + hostName );
+ }
+ nodesInUse.add(hostName);
+ yarn.blacklistNode(hostName);
+ }
+
+ public void release(Container container) {
+ release(container.getNodeId().getHost());
+ }
+
+ public void release(String hostName) {
+ if (!yarnNodes.containsKey(hostName)) {
+ return;
+ }
+ nodesInUse.remove(hostName);
+ yarn.removeBlacklist(hostName);
+ }
+
+ public void blacklist(String hostName) {
+ if (!yarnNodes.containsKey(hostName)) {
+ return;
+ }
+ assert !nodesInUse.contains(hostName);
+ blacklist.add(hostName);
+ yarn.blacklistNode(hostName);
+ LOG.info("Node blacklisted: " + hostName);
+ }
+
+ /**
+ * Determine the number of free nodes in the YARN cluster. The free set is the
+ * set of all YARN nodes minus those that are allocated and those that are
+ * blacklisted. Note that a node might be both in use and blacklisted if
+ * DoY blacklists a node, but then the user starts a "stray" Drillbit on
+ * that same node.
+ * <p>
+ * This number is an approximation: the set of nodes managed by YARN can
+ * change any time, and in-flight container requests will consume a node,
+ * but since the request is not yet completed, we don't know which node
+ * will be assigned, so the node does not yet appear in the in-use list.
+ *
+ * @return an approximation of the free node count
+ */
+
+ public int getFreeNodeCount() {
+ Set<String> free = new HashSet<>( );
+ free.addAll( yarnNodes.keySet() );
+ free.removeAll( nodesInUse );
+ free.removeAll( blacklist );
+ return free.size( );
+ }
+
+ /**
+ * Return a copy of the blacklist (list of failed nodes) for use in display
+ * to the user or similar purpose.
+ *
+ * @return a copy of the blacklist.
+ */
+
+ public List<String> getBlacklist() {
+ List<String> copy = new ArrayList<>( );
+ copy.addAll(blacklist);
+ return copy;
+ }
+
+ /**
+ * Report if the given host name is in use.
+ *
+ * @param hostName
+ * @return true if the host is reserved (in use by a container) or
+ * blacklisted (failed.)
+ */
+
+ public boolean isInUse(String hostName) {
+ return blacklist.contains(hostName) || nodesInUse.contains(hostName);
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PersistentTaskScheduler.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PersistentTaskScheduler.java
new file mode 100644
index 000000000..73a045fa6
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PersistentTaskScheduler.java
@@ -0,0 +1,174 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Abstract base class for schedulers that work with persistent
+ * (long-running) tasks. Such tasks are intended to run until
+ * explicitly shut down (unlike batch tasks that run until
+ * some expected completion.)
+ * <p>
+ * Provides a target quantity of tasks
+ * (see {@link #getTarget()}, along with operations to increase,
+ * decrease or set the target number.
+ * <p>
+ * The scheduler acts as a controller: starting new tasks as needed to
+ * match the desired target, or stopping tasks as needed when the
+ * target level is reduced.
+ */
+
+public abstract class PersistentTaskScheduler extends AbstractScheduler {
+ private static final Log LOG = LogFactory.getLog(PersistentTaskScheduler.class);
+ protected int quantity;
+
+ public PersistentTaskScheduler(String type, String name, int quantity) {
+ super(type, name);
+ this.quantity = quantity;
+ }
+
+ /**
+ * Set the number of running tasks to the quantity given.
+ *
+ * @param level
+ * the target number of tasks
+ */
+
+ @Override
+ public int resize(int level) {
+ quantity = level;
+ if (quantity < 0) {
+ quantity = 0;
+ }
+ return quantity;
+ }
+
+ @Override
+ public int getTarget() { return quantity; }
+
+ /**
+ * Indicate that a task is completed. Normally occurs only
+ * when shutting down excess tasks.
+ *
+ * @param task
+ */
+
+
+ @Override
+ public void completed(Task task) { }
+
+ /**
+ * Progress for persistent tasks defaults to the ratio of
+ * running tasks to target level. Thus, a persistent cluster
+ * will normally report 100% progress.
+ *
+ * @return
+ */
+
+ @Override
+ public int[] getProgress() {
+ int activeCount = state.getTaskCount();
+ return new int[] { Math.min(activeCount, quantity), quantity };
+ }
+
+ /**
+ * Adjust the number of running tasks to better match the target
+ * by starting or stopping tasks as needed.
+ */
+
+ @Override
+ public void adjust() {
+ int activeCount = state.getTaskCount();
+ int delta = quantity - activeCount;
+ if (delta > 0) {
+ addTasks(delta);
+ } else if (delta < 0) {
+ cancelTasks(activeCount);
+ }
+ }
+
+ /**
+ * Cancel the requested number of tasks. We exclude any tasks that are already
+ * in the process of being cancelled. Because we ignore those tasks, it might
+ * be that we want to reduce the task count, but there is nothing left to cancel.
+ *
+ * @param cancelCount
+ */
+
+ private void cancelTasks(int cancelCount) {
+ int cancelled = state.getCancelledTaskCount();
+ int cancellable = cancelCount - cancelled;
+ int n = cancellable - quantity;
+ LOG.info("[" + getName( ) + "] - Cancelling " + cancelCount +
+ " tasks. " + cancelled + " are already cancelled, " +
+ cancellable + " more will be cancelled.");
+ if (n <= 0) {
+ return;
+ }
+ for (Task task : state.getStartingTasks()) {
+ state.cancel(task);
+ if (--n == 0) {
+ return;
+ }
+ }
+ for (Task task : state.getActiveTasks()) {
+ state.cancel(task);
+ if (--n == 0) {
+ return;
+ }
+ }
+
+ // If we get here it means something has gotten out of whack.
+
+ LOG.error("Tried to cancel " + cancellable + " tasks, but " + n + " could not be cancelled.");
+ assert false;
+ }
+
+ /**
+ * The persistent scheduler has no fixed sequence of tasks to run, it launches
+ * a set and is never "done". For purposes of completion tracking claim we
+ * have no further tasks.
+ *
+ * @return false
+ */
+
+ @Override
+ public boolean hasMoreTasks() { return false; }
+
+ @Override
+ public void requestTimedOut() {
+
+ // We requested a node a while back, requested a container from YARN,
+ // but waited too long to receive it. Most likely cause is that we
+ // want a container on a node that either does not exist, or is too
+ // heavily loaded. (That is, we have a 3-node cluster and are requesting
+ // a 4th node. Or, we have 2 nodes but node 3 has insufficient resources.)
+ // In either case, we're not likely to ever get the container, so just
+ // reduce the target size to what we an get.
+
+ assert quantity > 0;
+ if (quantity == 0) {
+ LOG.error("Container timed out, but target quantity is already 0!");
+ } else {
+ quantity--;
+ LOG.info("Container request timed out. Reducing target container count by 1 to " + quantity);
+ }
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Pollable.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Pollable.java
new file mode 100644
index 000000000..7e1c9a3ef
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Pollable.java
@@ -0,0 +1,28 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+/**
+ * Interface for objects that are polled on each
+ * controller clock tick in order to perform
+ * time-based tasks.
+ */
+
+public interface Pollable {
+ public void tick(long curTime);
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PulseRunnable.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PulseRunnable.java
new file mode 100644
index 000000000..81d5a5d79
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PulseRunnable.java
@@ -0,0 +1,79 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Clock driver that calls a callback once each pulse period. Used to react to
+ * time-based events such as timeouts, checking for changed files, etc.
+ * This is called a "pulse" because it is periodic, like your pulse. But,
+ * unlike the "heartbeat" between the AM and YARN or the AM and ZK,
+ * this is purely internal.
+ */
+
+public class PulseRunnable implements Runnable {
+ private static final Log LOG = LogFactory.getLog(PulseRunnable.class);
+
+ /**
+ * Interface implemented to receive calls on each clock "tick."
+ */
+
+ public interface PulseCallback {
+ void onTick(long curTime);
+ }
+
+ private final int pulsePeriod;
+ private final PulseRunnable.PulseCallback callback;
+ public AtomicBoolean isLive = new AtomicBoolean(true);
+
+ public PulseRunnable(int pulsePeriodMS,
+ PulseRunnable.PulseCallback callback) {
+ pulsePeriod = pulsePeriodMS;
+ this.callback = callback;
+ }
+
+ @Override
+ public void run() {
+ while (isLive.get()) {
+ try {
+ Thread.sleep(pulsePeriod);
+ } catch (InterruptedException e) {
+ break;
+ }
+ try {
+ callback.onTick(System.currentTimeMillis());
+ } catch (Exception e) {
+
+ // Ignore exceptions. Seems strange, but is required to allow
+ // graceful shutdown of the AM when errors occur. For example, we
+ // start tasks on tick events. If those tasks fail, the timer
+ // goes down. But, the timer is also needed to time out failed
+ // requests in order to bring down the AM. So, just log the error
+ // and soldier on.
+
+ LOG.error("Timer thread caught, ignored an exception", e);
+ }
+ }
+ }
+
+ public void stop() { isLive.set(false); }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/RegistryHandler.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/RegistryHandler.java
new file mode 100644
index 000000000..ff29bdf1f
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/RegistryHandler.java
@@ -0,0 +1,35 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+/**
+ * Callback from the ZooKeeper registry to announce events
+ * related to Drillbit registration.
+ */
+
+public interface RegistryHandler {
+ void reserveHost(String hostName);
+
+ void releaseHost(String hostName);
+
+ void startAck(Task task, String propertyKey, Object value);
+
+ void completionAck(Task task, String endpointProperty);
+
+ void registryDown();
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Scheduler.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Scheduler.java
new file mode 100644
index 000000000..7f8be0c9b
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Scheduler.java
@@ -0,0 +1,162 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import org.apache.drill.yarn.core.ContainerRequestSpec;
+import org.apache.drill.yarn.core.LaunchSpec;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+/**
+ * The scheduler describes the set of tasks to run. It provides the details
+ * required to launch each task and optionally a specification of the containers
+ * required to run the task.
+ * <p>
+ * Schedulers can manage batch task (which do their job and complete), or
+ * persistent tasks (which run until terminated.)
+ * <p>
+ * The scheduler tracks task completion (for batch tasks) and task levels (for
+ * persistent tasks.)
+ */
+
+public interface Scheduler {
+ public interface TaskManager {
+ int maxConcurrentAllocs();
+
+ LaunchSpec getLaunchSpec(Task task);
+
+ void allocated(EventContext context);
+
+ boolean stop(Task task);
+
+ void completed(EventContext context);
+
+ boolean isLive(EventContext context);
+ }
+
+ /**
+ * Controller-assigned priority for this scheduler. Used to differentiate
+ * container requests by scheduler.
+ *
+ * @param priority
+ */
+
+ void setPriority(int priority);
+
+ /**
+ * Register the state object that tracks tasks launched by this scheduler.
+ *
+ * @param state
+ */
+
+ void registerState(SchedulerState state);
+
+ String getName();
+
+ String getType();
+
+ /**
+ * Whether tasks from this scheduler should incorporate app startup/shutdown
+ * acknowledgements (acks) into the task lifecycle.
+ *
+ * @return
+ */
+
+ boolean isTracked();
+
+ TaskManager getTaskManager();
+
+ /**
+ * Get the desired number of running tasks.
+ *
+ * @return
+ */
+ int getTarget();
+
+ /**
+ * Increase (positive) or decrease (negative) the number of desired tasks by
+ * the given amount.
+ *
+ * @param delta
+ */
+ void change(int delta);
+
+ /**
+ * Set the number of desired tasks to the given level.
+ *
+ * @param level
+ * @return the actual resize level, which may be lower than the requested
+ * level if the system cannot provide the requested level
+ */
+
+ int resize(int level);
+
+ void completed(Task task);
+
+ /**
+ * Adjust the number of running tasks to better track the desired number.
+ * Starts or stops tasks using the {@link SchedulerState} registered with
+ * {@link #registerState(SchedulerState)}.
+ */
+
+ void adjust();
+
+ /**
+ * Return an estimate of progress given as a ratio of (work completed, total
+ * work).
+ *
+ * @return
+ */
+ int[] getProgress();
+
+ /**
+ * If this is a batch scheduler, whether all tasks for the batch have
+ * completed. If this is a persistent task scheduler, always returns false.
+ *
+ * @return true if the scheduler has more tasks to run, false if the
+ * scheduler has no more tasks or manages a set of long-running tasks
+ */
+ boolean hasMoreTasks();
+
+ /**
+ * For reporting, get the YARN resources requested by processes in
+ * this pool.
+ * @return
+ */
+
+ ContainerRequestSpec getResource( );
+
+ void limitContainerSize(Resource maxResource) throws AMException;
+
+ /**
+ * Maximum amount of time to wait when cancelling a job in the REQUESTING
+ * state. YARN will happily wait forever for a resource, this setting
+ * forcibly cancels the request at timeout.
+ *
+ * @return the number of seconds to wait for timeout. 0 means no timeout
+ */
+
+ int getRequestTimeoutSec();
+
+ /**
+ * Informs the scheduler that a YARN resource request timed out. The scheduler
+ * can either retry or (more productively) assume that the requested node is
+ * not available and adjust its target size downward.
+ */
+
+ void requestTimedOut();
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerState.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerState.java
new file mode 100644
index 000000000..7a1f8bd02
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerState.java
@@ -0,0 +1,79 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import java.util.List;
+
+/**
+ * The cluster state for tasks managed by a scheduler. Abstracts away the
+ * details of managing tasks, allowing the scheduler to work only with overall
+ * number of tasks.
+ */
+
+public interface SchedulerState {
+ /**
+ * The number of tasks in any active (non-ended) lifecycle state.
+ *
+ * @return
+ */
+
+ int getTaskCount();
+
+ /**
+ * The number of active tasks that have been cancelled, but have not yet
+ * ended.
+ *
+ * @return
+ */
+
+ int getCancelledTaskCount();
+
+ /**
+ * Returns the list of tasks awaiting a container request to be sent to YARN
+ * or for which a container request has been sent to YARN, but no container
+ * allocation has yet been received. Such tasks are simple to cancel. The list
+ * does not contain any tasks in this state which have previously been
+ * cancelled.
+ *
+ * @return
+ */
+
+ List<Task> getStartingTasks();
+
+ /**
+ * Returns the list of active tasks that have not yet been cancelled. Active
+ * tasks are any task for which a container has been assigned, but has not yet
+ * received a RM container completion event.
+ *
+ * @return
+ */
+
+ List<Task> getActiveTasks();
+
+ /**
+ * Start the given task.
+ *
+ * @param task
+ */
+
+ void start(Task task);
+
+ void cancel(Task task);
+
+ ClusterController getController();
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateActions.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateActions.java
new file mode 100644
index 000000000..65e8f2adc
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateActions.java
@@ -0,0 +1,101 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import org.apache.hadoop.yarn.api.records.Container;
+
+/**
+ * Represents the set of commands called by the cluster controller to manage the
+ * state of tasks within a task group. Each task group is managed by a
+ * scheduler.
+ */
+
+public interface SchedulerStateActions {
+ /**
+ * Returns the name of the scheduler associated with this task action group.
+ *
+ * @return
+ */
+
+ String getName();
+
+ /**
+ * Returns the scheduler associated with this task group.
+ *
+ * @return
+ */
+
+ Scheduler getScheduler();
+
+ /**
+ * Adjust the number of running tasks as needed to balance the number of
+ * running tasks with the desired number. May result in no change it the
+ * cluster is already in balance (or is in the process of achieving balance.)
+ */
+
+ void adjustTasks();
+
+ /**
+ * Request a container the first task that we wish to start.
+ */
+
+ boolean requestContainers(EventContext context, int maxRequests);
+
+ /**
+ * A container request has been granted. Match the container up with the first
+ * task waiting for a container and launch the task.
+ *
+ * @param context
+ * @param container
+ */
+
+ void containerAllocated(EventContext context, Container container);
+
+ /**
+ * Shut down this task group by canceling all tasks not already cancelled.
+ *
+ * @param context
+ */
+
+ void shutDown(EventContext context);
+
+ /**
+ * Determine if this task group is done. It is done when there are no active
+ * tasks and the controller itself is shutting down. This latter check
+ * differentiates the start state (when no tasks are active) from the end
+ * state. The AM will not shut down until all task groups are done.
+ *
+ * @return
+ */
+
+ boolean isDone();
+
+ int getTaskCount( );
+
+ int getLiveCount();
+
+ int getRequestCount( );
+
+ void visitTaskModels( TaskVisitor visitor );
+
+ void checkTasks(EventContext context, long curTime);
+
+ void cancel(Task task);
+
+ Task getTask(int id);
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateImpl.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateImpl.java
new file mode 100644
index 000000000..4c85cf31c
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateImpl.java
@@ -0,0 +1,467 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.yarn.core.DoYUtil;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+
+/**
+ * Manages a the set of tasks associated with a scheduler. The scheduler decides
+ * which tasks to run or stop; the task group manages the life-cycle of the
+ * tasks for the given scheduler.
+ * <p>
+ * Schedulers, and hence their groups, define a priority. When starting, higher
+ * priority (lower priority value) groups run before lower priority groups.
+ * Similarly, when shrinking the cluster, lower priority groups shrink before
+ * higher priority groups.
+ */
+
+public final class SchedulerStateImpl
+ implements SchedulerState, SchedulerStateActions {
+ static final Log LOG = LogFactory.getLog(SchedulerStateImpl.class);
+
+ private final Scheduler scheduler;
+
+ private final ClusterControllerImpl controller;
+
+ /**
+ * Tracks the tasks to be started, but for which no work has yet been done.
+ * (State == PENDING).
+ */
+
+ protected List<Task> pendingTasks = new LinkedList<>();
+
+ /**
+ * Tracks the tasks for which containers have been requested. (State ==
+ * REQUESTED).
+ */
+
+ protected List<Task> allocatingTasks = new LinkedList<>();
+
+ /**
+ * Tracks running tasks: those that have been allocated containers and are
+ * starting, running, failed or ended. We use a map for this because, during
+ * these states, the task is identified by its container. (State == LAUNCHING,
+ * RUNNING or ENDING).
+ */
+
+ protected Map<ContainerId, Task> activeContainers = new HashMap<>();
+
+ public SchedulerStateImpl(ClusterControllerImpl controller,
+ Scheduler scheduler) {
+ this.controller = controller;
+ this.scheduler = scheduler;
+ scheduler.registerState(this);
+ }
+
+ @Override
+ public String getName() {
+ return scheduler.getName();
+ }
+
+ public int getMaxRetries() {
+ return controller.getMaxRetries();
+ }
+
+ public int getStopTimeoutMs() {
+ return controller.getStopTimeoutMs();
+ }
+
+ @Override
+ public Scheduler getScheduler() { return scheduler; }
+
+ /**
+ * Define a new task in this group. Adds it to the pending queue so that a
+ * container will be requested.
+ *
+ * @param task
+ */
+
+ @Override
+ public void start(Task task) {
+ assert task.getGroup() == null;
+ task.setGroup(this);
+ enqueuePendingRequest(task);
+ }
+
+ /**
+ * Put a task into the queue waiting to send a container request to YARN.
+ *
+ * @param task
+ */
+
+ public void enqueuePendingRequest(Task task) {
+ assert !activeContainers.containsValue(task);
+ assert !allocatingTasks.contains(task);
+ assert !pendingTasks.contains(task);
+ pendingTasks.add(task);
+
+ // Special initial-state notification
+
+ EventContext context = new EventContext(controller, task);
+ controller.fireLifecycleChange(TaskLifecycleListener.Event.CREATED,
+ context);
+ }
+
+ public int maxCurrentRequests() {
+ return this.scheduler.getTaskManager().maxConcurrentAllocs();
+ }
+
+ @Override
+ public boolean requestContainers(EventContext context, int maxRequests) {
+ if (pendingTasks.isEmpty()) {
+ return false;
+ }
+
+ // Limit the maximum number of requests to the limit set by
+ // the scheduler.
+
+ maxRequests = Math.min(maxRequests, maxCurrentRequests());
+
+ // Further limit requests to account for in-flight requests.
+
+ maxRequests -= allocatingTasks.size( );
+
+ // Request containers as long as there are pending tasks remaining.
+
+ for (int i = 0; i < maxRequests && !pendingTasks.isEmpty(); i++) {
+ context.setTask(pendingTasks.get(0));
+ context.getState().requestContainer(context);
+ }
+ return true;
+ }
+
+ /**
+ * Remove a task from the queue of tasks waiting to send a container request.
+ * The caller must put the task into the proper next state: the allocating
+ * queue or the completed task list.
+ *
+ * @param task
+ */
+
+ public void dequeuePendingRequest(Task task) {
+ assert !activeContainers.containsValue(task);
+ assert !allocatingTasks.contains(task);
+ assert pendingTasks.contains(task);
+ pendingTasks.remove(task);
+ }
+
+ /**
+ * Put a task onto the queue awaiting an allocation response from YARN.
+ *
+ * @param task
+ */
+
+ public void enqueueAllocatingTask(Task task) {
+ assert !activeContainers.containsValue(task);
+ assert !allocatingTasks.contains(task);
+ assert !pendingTasks.contains(task);
+ allocatingTasks.add(task);
+ }
+
+ @Override
+ public void containerAllocated(EventContext context, Container container) {
+ if (activeContainers.containsKey(container.getId())) {
+ LOG.error("Container allocated again: " + DoYUtil.labelContainer(container));
+ return;
+ }
+ if (allocatingTasks.isEmpty()) {
+
+ // Not sure why this happens. Maybe only in debug mode
+ // due stopping execution one thread while the RM
+ // heartbeat keeps sending our request over & over?
+ // One known case: the user requests a container. While YARN is
+ // considering the request, the user cancels the task.
+
+ LOG.warn("Releasing unwanted container: " + DoYUtil.labelContainer(container) );
+ context.yarn.releaseContainer(container);
+ return;
+ }
+ context.setTask(allocatingTasks.get(0));
+ context.getState().containerAllocated(context, container);
+ }
+
+ @Override
+ public void checkTasks(EventContext context, long curTime) {
+
+ // Iterate over tasks using a temporary list. The tick event may cause a timeout
+ // that turns around and modifies these lists.
+
+ List<Task> temp = new ArrayList<>( );
+ temp.addAll( allocatingTasks );
+ for (Task task : temp) {
+ context.setTask(task);
+ context.getState().tick(context, curTime);
+ }
+ temp.clear();
+ temp.addAll( pendingTasks );
+ for (Task task : temp) {
+ context.setTask(task);
+ context.getState().tick(context, curTime);
+ }
+ temp.clear();
+ temp.addAll( activeContainers.values( ) );
+ for (Task task : temp) {
+ context.setTask(task);
+ context.getState().tick(context, curTime);
+ }
+ }
+
+ /**
+ * Remove a task from the list of those waiting for a container allocation.
+ * The allocation may be done, or cancelled. The caller is responsible for
+ * moving the task to the next collection.
+ *
+ * @param task
+ */
+
+ public void dequeueAllocatingTask(Task task) {
+ assert allocatingTasks.contains(task);
+ allocatingTasks.remove(task);
+ }
+
+ /**
+ * Mark that a task has become active and should be tracked by its container
+ * ID. Prior to this, the task is not associated with a container.
+ *
+ * @param task
+ */
+
+ public void containerAllocated(Task task) {
+ assert !activeContainers.containsValue(task);
+ assert !allocatingTasks.contains(task);
+ assert !pendingTasks.contains(task);
+ activeContainers.put(task.getContainerId(), task);
+ controller.containerAllocated(task);
+ }
+
+ /**
+ * Mark that a task has completed: its container has expired or been revoked
+ * or the task has completed: successfully or a failure, as given by the
+ * task's disposition. The task can no longer be tracked by its container ID.
+ * If this is the last active task for this group, mark the group itself as
+ * completed.
+ *
+ * @param task
+ */
+
+ public void containerReleased(Task task) {
+ assert activeContainers.containsKey(task.getContainerId());
+ activeContainers.remove(task.getContainerId());
+ controller.containerReleased(task);
+ }
+
+ /**
+ * Mark that a task has completed successfully or a failure, as given by the
+ * task's disposition. If this is the last active task for this group, mark
+ * the group itself as completed.
+ *
+ * @param task
+ */
+
+ public void taskEnded(Task task) {
+ scheduler.completed(task);
+ controller.taskEnded(task);
+ if (isDone()) {
+ controller.taskGroupCompleted(this);
+ }
+ LOG.info(task.toString() + " - Task completed" );
+ }
+
+ /**
+ * Mark that a task is about to be retried. Task still retains its state from
+ * the current try.
+ *
+ * @param task
+ */
+
+ public void taskRetried(Task task) {
+ controller.taskRetried(task);
+ }
+
+ @Override
+ public void shutDown(EventContext context) {
+ for (Task task : getStartingTasks()) {
+ context.setTask(task);
+ context.getState().cancel(context);
+ }
+ for (Task task : getActiveTasks()) {
+ context.setTask(task);
+ context.getState().cancel(context);
+ }
+ }
+
+ /**
+ * Report if this task group has any tasks in the active part of their
+ * life-cycle: pending, allocating or active.
+ *
+ * @return
+ */
+
+ public boolean hasTasks() {
+ return getTaskCount() != 0;
+ }
+
+ @Override
+ public boolean isDone() {
+ return !hasTasks() && !scheduler.hasMoreTasks();
+ }
+
+ @Override
+ public void adjustTasks() {
+ scheduler.adjust();
+ }
+
+ /**
+ * Request a graceful stop of the task. Delegates to the task manager to do
+ * the actual work.
+ *
+ * @return true if the graceful stop request was sent, false if not, or if
+ * this task type has no graceful stop
+ */
+
+ public boolean requestStop(Task task) {
+ return scheduler.getTaskManager().stop(task);
+ }
+
+ @Override
+ public int getTaskCount() {
+ return pendingTasks.size() + allocatingTasks.size()
+ + activeContainers.size();
+ }
+
+ @Override
+ public int getCancelledTaskCount() {
+
+ // TODO Crude first cut. This value should be maintained
+ // as a count.
+
+ int count = 0;
+ for (Task task : pendingTasks) {
+ if (task.isCancelled()) {
+ count++;
+ }
+ }
+ for (Task task : allocatingTasks) {
+ if (task.isCancelled()) {
+ count++;
+ }
+ }
+ for (Task task : activeContainers.values()) {
+ if (task.isCancelled()) {
+ count++;
+ }
+ }
+ return count;
+ }
+
+ @Override
+ public List<Task> getStartingTasks() {
+ List<Task> tasks = new ArrayList<>();
+ for (Task task : pendingTasks) {
+ if (!task.isCancelled()) {
+ tasks.add(task);
+ }
+ }
+ for (Task task : allocatingTasks) {
+ if (!task.isCancelled()) {
+ tasks.add(task);
+ }
+ }
+ return tasks;
+ }
+
+ @Override
+ public List<Task> getActiveTasks() {
+ List<Task> tasks = new ArrayList<>();
+ for (Task task : activeContainers.values()) {
+ if (!task.isCancelled()) {
+ tasks.add(task);
+ }
+ }
+ return tasks;
+ }
+
+ @Override
+ public void cancel(Task task) {
+ EventContext context = new EventContext(controller, task);
+ LOG.info( task.getLabel() + " Task cancelled" );
+ context.getState().cancel(context);
+ }
+
+ @Override
+ public int getLiveCount() {
+ int count = 0;
+ for (Task task : activeContainers.values()) {
+ if (task.isLive()) {
+ count++;
+ }
+ }
+ return count;
+ }
+
+ @Override
+ public void visitTaskModels(TaskVisitor visitor) {
+ for (Task task : pendingTasks) {
+ visitor.visit(task);
+ }
+ for (Task task : allocatingTasks) {
+ visitor.visit(task);
+ }
+ for (Task task : activeContainers.values()) {
+ visitor.visit(task);
+ }
+ }
+
+ @Override
+ public Task getTask(int id) {
+ for (Task task : pendingTasks) {
+ if (task.getId() == id) {
+ return task;
+ }
+ }
+ for (Task task : allocatingTasks) {
+ if (task.getId() == id) {
+ return task;
+ }
+ }
+ for (Task task : activeContainers.values()) {
+ if (task.getId() == id) {
+ return task;
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public int getRequestCount() {
+ return allocatingTasks.size();
+ }
+
+ @Override
+ public ClusterController getController( ) { return controller; }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Task.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Task.java
new file mode 100644
index 000000000..147f5f7d3
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Task.java
@@ -0,0 +1,323 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.drill.yarn.core.ContainerRequestSpec;
+import org.apache.drill.yarn.core.LaunchSpec;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
+
+/**
+ * AM-side state of individual containers. This class is mostly
+ * a holder of state. Behavior is provided by the
+ * {@link TaskState} subclasses.
+ */
+
+public class Task {
+ /**
+ * Tracking plugin state. A task can be untracked, or moves
+ * though states<br>
+ * NEW --> START_ACK --> END_ACK
+ * <p>
+ * Tracking state is separate from, but integrated with,
+ * task state. This is because, due to latency, tracking
+ * events may be slightly out of sync with YARN events.
+ */
+
+ public enum TrackingState
+ {
+ UNTRACKED( "N/A" ),
+ NEW( "Waiting" ),
+ START_ACK( "OK" ),
+ END_ACK( "Deregistered" );
+
+ private String displayName;
+
+ private TrackingState( String displayName ) {
+ this.displayName = displayName;
+ }
+
+ public String getDisplayName( ) { return displayName; }
+ }
+
+ public enum Disposition
+ {
+ CANCELLED, LAUNCH_FAILED, RUN_FAILED, COMPLETED, TOO_MANY_RETRIES, RETRIED
+ }
+
+ /**
+ * Maximum amount of time to wait when canceling a job in the REQUESTING
+ * state. YARN will happily wait forever for a resource, this setting allows
+ * the user to request to cancel a task, give YARN a while to respond, then
+ * forcibly cancel the job at timeout.
+ */
+
+ public static final long MAX_CANCELLATION_TIME = 10_000; // ms = 10s
+
+ /**
+ * Tasks receive a sequential internal task ID. Since all task
+ * creation is single-threaded, no additional concurrency controls
+ * are needed to protect this value.
+ */
+
+ private static volatile int taskCounter = 0;
+
+ /**
+ * Internal identifier for the task.
+ */
+
+ public final int taskId;
+
+
+ public final Scheduler scheduler;
+
+ /**
+ * Identifies the type of container needed and the details of the task to run.
+ */
+
+ public TaskSpec taskSpec;
+
+ /**
+ * The scheduler group that manages this task.
+ */
+
+ public SchedulerStateImpl taskGroup;
+
+ /**
+ * Tracking state for an additional task tracker (such as using
+ * ZooKeeper to track Drill-bits.)
+ */
+
+ protected TrackingState trackingState;
+
+ /**
+ * Tracks the container request between request and allocation. We must pass
+ * the container request back to YARN to remove it once it is allocated.
+ */
+
+ public ContainerRequest containerRequest;
+
+ /**
+ * The YARN container assigned to this task. The container is set only during
+ * the ALLOCATED, LAUNCHING, RUNNING and ENDING states.
+ */
+
+ public Container container;
+
+ /**
+ * Life-cycle state of this task.
+ */
+
+ protected TaskState state;
+
+ /**
+ * True if the application has requested that the resource request or
+ * application run be cancelled. Cancelled tasks are not subject to retry.
+ */
+
+ protected boolean cancelled;
+
+ /**
+ * Disposition of a completed task: whether it was cancelled, succeeded or
+ * failed.
+ */
+
+ public Disposition disposition;
+
+ public Throwable error;
+
+ public int tryCount;
+
+ public ContainerStatus completionStatus;
+
+ public long launchTime;
+ public long stateStartTime;
+ public long completionTime;
+
+ long cancellationTime;
+
+ public Map<String,Object> properties = new HashMap<>( );
+
+ public Task(Scheduler scheduler, TaskSpec taskSpec) {
+ taskId = ++taskCounter;
+ this.scheduler = scheduler;
+ this.taskSpec = taskSpec;
+ state = TaskState.START;
+ resetTrackingState();
+ }
+
+ /**
+ * Special constructor to create a static copy of the current
+ * task. The copy is placed in the completed tasks list.
+ * @param task
+ */
+
+ private Task(Task task) {
+ taskId = task.taskId;
+ scheduler = task.scheduler;
+ taskSpec = task.taskSpec;
+ taskGroup = task.taskGroup;
+ trackingState = task.trackingState;
+ containerRequest = task.containerRequest;
+ container = task.container;
+ state = task.state;
+ cancelled = task.cancelled;
+ disposition = task.disposition;
+ error = task.error;
+ tryCount = task.tryCount;
+ completionStatus = task.completionStatus;
+ launchTime = task.launchTime;
+ stateStartTime = task.stateStartTime;
+ completionTime = task.completionTime;
+ cancellationTime = task.cancellationTime;
+ properties.putAll( task.properties );
+ }
+
+ public void resetTrackingState( ) {
+ trackingState = scheduler.isTracked() ? TrackingState.NEW : TrackingState.UNTRACKED;
+ }
+
+ public int getId( ) { return taskId; }
+ public ContainerRequestSpec getContainerSpec() { return taskSpec.containerSpec; }
+
+ public LaunchSpec getLaunchSpec() { return taskSpec.launchSpec; }
+
+ public TaskState getState() { return state; }
+
+ public ContainerId getContainerId() {
+ assert container != null;
+ return container.getId();
+ }
+
+ public Container getContainer() {
+ assert container != null;
+ return container;
+ }
+
+ public int getTryCount() { return tryCount; }
+
+ public boolean isFailed() {
+ return disposition != null && disposition != Disposition.COMPLETED;
+ }
+
+ public Disposition getDisposition() { return disposition; }
+
+ public SchedulerStateImpl getGroup() { return taskGroup; }
+
+ public void setGroup(SchedulerStateImpl taskGroup) { this.taskGroup = taskGroup; }
+
+ public boolean retryable() {
+ return !cancelled && disposition != Disposition.COMPLETED;
+ }
+
+ public boolean isCancelled() { return cancelled; }
+
+ /**
+ * Reset the task state in preparation for a retry.
+ * Note: state reset is done by the state class.
+ */
+
+ public void reset() {
+ assert !cancelled;
+ error = null;
+ disposition = null;
+ completionStatus = null;
+ launchTime = 0;
+ completionTime = 0;
+ cancellationTime = 0;
+ container = null;
+ resetTrackingState();
+ }
+
+ public long uptime() {
+ long endTime = completionTime;
+ if (endTime == 0) {
+ endTime = System.currentTimeMillis();
+ }
+ return endTime - launchTime;
+ }
+
+ public String getHostName() {
+ if (container == null) {
+ return null;
+ }
+ return container.getNodeId().getHost();
+ }
+
+ public TrackingState getTrackingState() {
+ return trackingState;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder buf = new StringBuilder();
+ buf.append("[id=")
+ .append(taskId)
+ .append(", type=");
+ // Scheduler is unset in some unit tests.
+ if (scheduler !=null ) {
+ buf.append(scheduler.getName());
+ }
+ buf.append(", name=")
+ .append(getName());
+ if (container != null) {
+ buf.append(", host=")
+ .append(getHostName());
+ }
+ buf.append(", state=")
+ .append(state.toString())
+ .append("]");
+ return buf.toString();
+ }
+
+ public boolean isLive() {
+ return state == TaskState.RUNNING && !cancelled;
+ }
+
+ public void cancel() {
+ cancelled = true;
+ cancellationTime = System.currentTimeMillis();
+ }
+
+ public Task copy() {
+ return new Task(this);
+ }
+
+ public String getName() {
+ return taskSpec == null ? null : taskSpec.name;
+ }
+
+ /**
+ * Label for this task displayed in log messages.
+ *
+ * @return
+ */
+
+ public String getLabel() {
+ return toString( );
+ }
+
+ public void setTrackingState(TrackingState tState) {
+ trackingState = tState;
+ }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskLifecycleListener.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskLifecycleListener.java
new file mode 100644
index 000000000..218cd9b0d
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskLifecycleListener.java
@@ -0,0 +1,26 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+public interface TaskLifecycleListener {
+ public enum Event {
+ CREATED, ALLOCATED, RUNNING, ENDED
+ }
+
+ void stateChange(Event event, EventContext context);
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskSpec.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskSpec.java
new file mode 100644
index 000000000..4399a8669
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskSpec.java
@@ -0,0 +1,40 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import org.apache.drill.yarn.core.ContainerRequestSpec;
+import org.apache.drill.yarn.core.LaunchSpec;
+
+public class TaskSpec {
+ /**
+ * Number of YARN vcores (virtual cores) and amount of memory (in MB) needed
+ * by this task.
+ */
+
+ public ContainerRequestSpec containerSpec;
+
+ /**
+ * Description of of the task process, environment and so on.
+ */
+
+ public LaunchSpec launchSpec;
+
+ public int maxRetries;
+
+ public String name;
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskState.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskState.java
new file mode 100644
index 000000000..3d52105e4
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskState.java
@@ -0,0 +1,895 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.yarn.appMaster.Task.Disposition;
+import org.apache.drill.yarn.core.DoYUtil;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+
+/**
+ * Represents the behaviors associated with each state in the lifecycle
+ * of a task.
+ * <p>
+ * Startup process:
+ * <dl>
+ * <dt>START --> REQUESTING<dt>
+ * <dd>New task sends a container request to YARN.</dd>
+ * <dt>REQUESTING --> LAUNCHING<dt>
+ * <dd>Container received from YARN, launching the tasks's process.</dd>
+ * <dt>LAUNCHING --> RUNNING<dt>
+ * <dd>Task launched and needs no start Ack.</dd>
+ * <dt>LAUNCHING --> WAIT_START_ACK<dt>
+ * <dd>Task launched and needs a start Ack.</dd>
+ * <dt>WAIT_START_ACK --> RUNNING<dt>
+ * <dd>Start Ack received.</dd>
+ * </dl>
+ * <p>
+ * Shutdown process:
+ * <dt>RUNNING --> WAIT_END_ACK | END<dt>
+ * <dd>The resource manager reported task completion.</dd>
+ * <dt>RUNNING --> ENDING<dt>
+ * <dd>Request sent to the task for a graceful shutdown.</dd>
+ * <dt>RUNNING --> KILLING<dt>
+ * <dd>Request sent to the node manager to forcibly kill the task.</dd>
+ * <dt>ENDING --> WAIT_END_ACK | END<dt>
+ * <dd>The task gracefully exited as reported by the resource manager.</dd>
+ * <dt>ENDING --> KILLING<dt>
+ * <dd>The wait for graceful exit timed out, a forced kill message
+ * sent to the node manager.</dd>
+ * <dt>KILLING --> WAIT_END_ACK | END<dt>
+ * <dd>The task exited as reported by the resource manager.</dd>
+ * <dt>END_ACK --> END<dt>
+ * <dd>The end-ack is received or the wait timed out.</dd>
+ * <dl>
+ * <p>
+ * This is a do-it-yourself enum. Java enums values are instances of a single
+ * class. In this version, each enum value is the sole instance of a separate
+ * class, allowing each state to have its own behavior.
+ */
+
+public abstract class TaskState {
+ /**
+ * Task that is newly created and needs a container allocated. No messages
+ * have yet been sent to YARN for the task.
+ */
+
+ private static class StartState extends TaskState {
+ protected StartState() { super(false, TaskLifecycleListener.Event.CREATED, true); }
+
+ @Override
+ public void requestContainer(EventContext context) {
+ Task task = context.task;
+ task.tryCount++;
+ context.group.dequeuePendingRequest(task);
+ if (task.cancelled) {
+ taskStartFailed(context, Disposition.CANCELLED);
+ } else {
+ transition(context, REQUESTING);
+ context.group.enqueueAllocatingTask(task);
+ task.containerRequest = context.yarn
+ .requestContainer(task.getContainerSpec());
+ }
+ }
+
+ /**
+ * Cancellation is trivial: just drop the task; no need to coordinate
+ * with YARN.
+ */
+
+ @Override
+ public void cancel(EventContext context) {
+ Task task = context.task;
+ assert !task.cancelled;
+ context.group.dequeuePendingRequest(task);
+ task.cancel();
+ taskStartFailed(context, Disposition.CANCELLED);
+ }
+ }
+
+ /**
+ * Task for which a container request has been sent but not yet received.
+ */
+
+ private static class RequestingState extends TaskState {
+ protected RequestingState() {
+ super(false, TaskLifecycleListener.Event.CREATED, true);
+ }
+
+ /**
+ * Handle REQUESING --> LAUNCHING. Indicates that we've asked YARN to start
+ * the task on the allocated container.
+ */
+
+ @Override
+ public void containerAllocated(EventContext context, Container container) {
+ Task task = context.task;
+ LOG.info(task.getLabel() + " - Received container: "
+ + DoYUtil.describeContainer(container));
+ context.group.dequeueAllocatingTask(task);
+
+ // No matter what happens below, we don't want to ask for this
+ // container again. The RM async API is a bit bizarre in this
+ // regard: it will keep asking for container over and over until
+ // we tell it to stop.
+
+ context.yarn.removeContainerRequest(task.containerRequest);
+
+ // The container is need both in the normal and in the cancellation
+ // path, so set it here.
+
+ task.container = container;
+ if (task.cancelled) {
+ context.yarn.releaseContainer(container);
+ taskStartFailed(context, Disposition.CANCELLED);
+ return;
+ }
+ task.error = null;
+ task.completionStatus = null;
+ transition(context, LAUNCHING);
+
+ // The pool that manages this task wants to know that we have
+ // a container. The task manager may want to do some task-
+ // specific setup.
+
+ context.group.containerAllocated(context.task);
+ context.getTaskManager().allocated(context);
+
+ // Go ahead and launch a task in the container using the launch
+ // specification provided by the task group (pool).
+
+ try {
+ context.yarn.launchContainer(container, task.getLaunchSpec());
+ task.launchTime = System.currentTimeMillis();
+ } catch (YarnFacadeException e) {
+ LOG.error("Container launch failed: " + task.getContainerId(), e);
+
+ // This may not be the right response. RM may still think
+ // we have the container if the above is a local failure.
+
+ task.error = e;
+ context.group.containerReleased(task);
+ task.container = null;
+ taskStartFailed(context, Disposition.LAUNCH_FAILED);
+ }
+ }
+
+ /**
+ * Cancel the container request. We must wait for the response from YARN to
+ * do the actual cancellation. For now, just mark the task as cancelled.
+ */
+
+ @Override
+ public void cancel(EventContext context) {
+ Task task = context.task;
+ context.task.cancel();
+ LOG.info(task.getLabel() + " - Cancelled at user request");
+ context.yarn.removeContainerRequest(task.containerRequest);
+ context.group.dequeueAllocatingTask(task);
+ task.disposition = Task.Disposition.CANCELLED;
+ task.completionTime = System.currentTimeMillis();
+ transition(context, END);
+ context.group.taskEnded(context.task);
+ }
+
+ /**
+ * The task is requesting a container. If the request takes too long,
+ * cancel the request and shrink the target task count. This event
+ * generally indicates that the user wants to run more tasks than
+ * the cluster has capacity.
+ */
+
+ @Override
+ public void tick(EventContext context, long curTime) {
+ Task task = context.task;
+ int timeoutSec = task.scheduler.getRequestTimeoutSec( );
+ if (timeoutSec == 0) {
+ return;
+ }
+ if (task.stateStartTime + timeoutSec * 1000 > curTime) {
+ return;
+ }
+ LOG.info(task.getLabel() + " - Request timed out after + "
+ + timeoutSec + " secs.");
+ context.yarn.removeContainerRequest(task.containerRequest);
+ context.group.dequeueAllocatingTask(task);
+ task.disposition = Task.Disposition.LAUNCH_FAILED;
+ task.completionTime = System.currentTimeMillis();
+ transition(context, END);
+ context.group.taskEnded(context.task);
+ task.scheduler.requestTimedOut();
+ }
+ }
+
+ /**
+ * Task for which a container has been allocated and the task launch request
+ * sent. Awaiting confirmation that the task is running.
+ */
+
+ private static class LaunchingState extends TaskState {
+ protected LaunchingState() {
+ super(true, TaskLifecycleListener.Event.ALLOCATED, true);
+ }
+
+ /**
+ * Handle launch failure. Results in a LAUNCHING --> END transition or
+ * restart.
+ * <p>
+ * This situation can occur, when debugging, if a timeout occurs after the
+ * allocation message, such as when, sitting in the debugger on the
+ * allocation event.
+ */
+
+ @Override
+ public void launchFailed(EventContext context, Throwable t) {
+ Task task = context.task;
+ LOG.info(task.getLabel() + " - Container start failed");
+ context.task.error = t;
+ launchFailed(context);
+ }
+
+ /**
+ * Handle LAUNCHING --> RUNNING/START_ACK. Indicates that YARN has confirmed
+ * that the task is, indeed, running.
+ */
+
+ @Override
+ public void containerStarted(EventContext context) {
+ Task task = context.task;
+
+ // If this task is tracked (that is, it is a Drillbit which
+ // we monitor using ZK) then we have to decide if we've
+ // seen the task in the tracker yet. If we have, then the
+ // task is fully running. If we haven't, then we need to
+ // wait for the start acknowledgement.
+
+ if (task.trackingState == Task.TrackingState.NEW) {
+ transition(context, WAIT_START_ACK);
+ } else {
+ transition(context, RUNNING);
+ }
+ task.error = null;
+
+ // If someone came along and marked the task as cancelled,
+ // we are now done waiting for YARN so we can immediately
+ // turn around and kill the task. (Can't kill the task,
+ // however, until YARN starts it, hence the need to wait
+ // for YARN to start the task before killing it.)
+
+ if (task.cancelled) {
+ transition(context, KILLING);
+ context.yarn.killContainer(task.getContainer());
+ }
+ }
+
+ /**
+ * Out-of-order start ACK, perhaps due to network latency. Handle by staying
+ * in this state, but later jump directly<br>
+ * LAUNCHING --> RUNNING
+ */
+
+ @Override
+ public void startAck(EventContext context) {
+ context.task.trackingState = Task.TrackingState.START_ACK;
+ }
+
+ @Override
+ public void containerCompleted(EventContext context,
+ ContainerStatus status) {
+ // Seen on Mac when putting machine to sleep.
+ // Handle by failing & retrying.
+ completed(context, status);
+ endOrAck(context);
+ }
+
+ @Override
+ public void cancel(EventContext context) {
+ context.task.cancel();
+ context.yarn.killContainer(context.task.getContainer());
+ }
+
+ @Override
+ public void tick(EventContext context, long curTime) {
+
+ // If we are canceling the task, and YARN has not reported container
+ // completion after some amount of time, just force failure.
+
+ Task task = context.task;
+ if (task.isCancelled()
+ && task.cancellationTime + Task.MAX_CANCELLATION_TIME < curTime) {
+ LOG.error(task.getLabel() + " - Launch timed out after "
+ + Task.MAX_CANCELLATION_TIME / 1000 + " secs.");
+ launchFailed(context);
+ }
+ }
+
+ private void launchFailed(EventContext context) {
+ Task task = context.task;
+ task.completionTime = System.currentTimeMillis();
+
+ // Not sure if releasing the container is needed...
+
+ context.yarn.releaseContainer(task.container);
+ context.group.containerReleased(task);
+ task.container = null;
+ taskStartFailed(context, Disposition.LAUNCH_FAILED);
+ }
+ }
+
+ /**
+ * Task has been launched, is tracked, but we've not yet received a start ack.
+ */
+
+ private static class WaitStartAckState extends TaskState {
+ protected WaitStartAckState() {
+ super(true, TaskLifecycleListener.Event.RUNNING, true);
+ }
+
+ @Override
+ public void startAck(EventContext context) {
+ context.task.trackingState = Task.TrackingState.START_ACK;
+ transition(context, RUNNING);
+ }
+
+ @Override
+ public void cancel(EventContext context) {
+ RUNNING.cancel(context);
+ }
+
+ // @Override
+ // public void containerStopped(EventContext context) {
+ // transition(context, WAIT_COMPLETE );
+ // }
+
+ @Override
+ public void containerCompleted(EventContext context,
+ ContainerStatus status) {
+ completed(context, status);
+ taskTerminated(context);
+ }
+
+ // TODO: Timeout in this state.
+ }
+
+ /**
+ * Task in the normal running state.
+ */
+
+ private static class RunningState extends TaskState {
+ protected RunningState() {
+ super(true, TaskLifecycleListener.Event.RUNNING, true);
+ }
+
+ /**
+ * Normal task completion. Implements the RUNNING --> END transition.
+ *
+ * @param status
+ */
+
+ @Override
+ public void containerCompleted(EventContext context,
+ ContainerStatus status) {
+ completed(context, status);
+ endOrAck(context);
+ }
+
+ @Override
+ public void cancel(EventContext context) {
+ Task task = context.task;
+ task.cancel();
+ if (context.group.requestStop(task)) {
+ transition(context, ENDING);
+ } else {
+ context.yarn.killContainer(task.container);
+ transition(context, KILLING);
+ }
+ }
+
+ /**
+ * The task claims that it is complete, but we think it is running. Assume
+ * that the task has started its own graceful shutdown (or the
+ * equivalent).<br>
+ * RUNNING --> ENDING
+ */
+
+ @Override
+ public void completionAck(EventContext context) {
+ context.task.trackingState = Task.TrackingState.END_ACK;
+ transition(context, ENDING);
+ }
+ }
+
+ /**
+ * Task for which a termination request has been sent to the Drill-bit, but
+ * confirmation has not yet been received from the Node Manager. (Not yet
+ * supported in the Drill-bit.
+ */
+
+ public static class EndingState extends TaskState {
+ protected EndingState() { super(true, TaskLifecycleListener.Event.RUNNING, false); }
+
+ /*
+ * Normal ENDING --> WAIT_COMPLETE transition, awaiting Resource Manager
+ * confirmation.
+ */
+
+// @Override
+// public void containerStopped(EventContext context) {
+// transition(context, WAIT_COMPLETE);
+// }
+
+ /**
+ * Normal ENDING --> WAIT_END_ACK | END transition.
+ *
+ * @param status
+ */
+
+ @Override
+ public void containerCompleted(EventContext context,
+ ContainerStatus status) {
+ completed(context, status);
+ endOrAck(context);
+ }
+
+ @Override
+ public void cancel(EventContext context) {
+ context.task.cancel();
+ }
+
+ /**
+ * If the graceful stop process exceeds the maximum timeout, go ahead and
+ * forcibly kill the process.
+ */
+
+ @Override
+ public void tick(EventContext context, long curTime) {
+ Task task = context.task;
+ if (curTime - task.stateStartTime > task.taskGroup.getStopTimeoutMs()) {
+ context.yarn.killContainer(task.container);
+ transition(context, KILLING);
+ }
+ }
+
+ @Override
+ public void completionAck(EventContext context) {
+ context.task.trackingState = Task.TrackingState.END_ACK;
+ }
+ }
+
+ /**
+ * Task for which a forced termination request has been sent to the Node
+ * Manager, but a stop message has not yet been received.
+ */
+
+ public static class KillingState extends TaskState {
+ protected KillingState() { super(true, TaskLifecycleListener.Event.RUNNING, false); }
+
+ /*
+ * Normal KILLING --> WAIT_COMPLETE transition, awaiting Resource Manager
+ * confirmation.
+ */
+
+// @Override
+// public void containerStopped(EventContext context) {
+// transition(context, WAIT_COMPLETE);
+// }
+
+ /**
+ * Normal KILLING --> WAIT_END_ACK | END transition.
+ *
+ * @param status
+ */
+
+ @Override
+ public void containerCompleted(EventContext context,
+ ContainerStatus status) {
+ completed(context, status);
+ endOrAck(context);
+ }
+
+ @Override
+ public void cancel(EventContext context) {
+ context.task.cancel();
+ }
+
+ @Override
+ public void startAck(EventContext context) {
+ // Better late than never... Happens during debugging sessions
+ // when order of messages is scrambled.
+
+ context.task.trackingState = Task.TrackingState.START_ACK;
+ }
+
+ @Override
+ public void completionAck(EventContext context) {
+ context.task.trackingState = Task.TrackingState.END_ACK;
+ }
+
+ @Override
+ public void stopTaskFailed(EventContext context, Throwable t) {
+ assert false;
+ // What to do?
+ }
+ }
+
+ /**
+ * Task exited, but we are waiting for confirmation from Zookeeper that
+ * the Drillbit registration has been removed. Required to associate
+ * ZK registrations with Drillbits. Ensures that we don't try to
+ * start a new Drillbit on a node until the previous Drillbit
+ * completely shut down, including dropping out of ZK.
+ */
+
+ private static class WaitEndAckState extends TaskState {
+ protected WaitEndAckState() {
+ super(false, TaskLifecycleListener.Event.RUNNING, false);
+ }
+
+ @Override
+ public void cancel(EventContext context) {
+ context.task.cancel();
+ }
+
+ @Override
+ public void completionAck(EventContext context) {
+ context.task.trackingState = Task.TrackingState.END_ACK;
+ taskTerminated(context);
+ }
+
+ /**
+ * Periodically check if the process is still live. We are supposed to
+ * receive events when the task becomes deregistered. But, we've seen
+ * cases where the task hangs in this state forever. Try to resolve
+ * the issue by polling periodically.
+ */
+
+ @Override
+ public void tick(EventContext context, long curTime) {
+ if(! context.getTaskManager().isLive(context)){
+ taskTerminated(context);
+ }
+ }
+ }
+
+ /**
+ * Task is completed or failed. The disposition field gives the details of the
+ * completion type. The task is not active on YARN, but could be retried.
+ */
+
+ private static class EndState extends TaskState {
+ protected EndState() {
+ super(false, TaskLifecycleListener.Event.ENDED, false);
+ }
+
+ /*
+ * Ignore out-of-order Node Manager completion notices.
+ */
+
+ // @Override
+ // public void containerStopped(EventContext context) {
+ // }
+
+ @Override
+ public void cancel(EventContext context) {
+ }
+ }
+
+ private static final Log LOG = LogFactory.getLog(TaskState.class);
+
+ public static final TaskState START = new StartState();
+ public static final TaskState REQUESTING = new RequestingState();
+ public static final TaskState LAUNCHING = new LaunchingState();
+ public static final TaskState WAIT_START_ACK = new WaitStartAckState();
+ public static final TaskState RUNNING = new RunningState();
+ public static final TaskState ENDING = new EndingState();
+ public static final TaskState KILLING = new KillingState();
+ public static final TaskState WAIT_END_ACK = new WaitEndAckState();
+ public static final TaskState END = new EndState();
+
+ protected final boolean hasContainer;
+ protected final TaskLifecycleListener.Event lifeCycleEvent;
+ protected final String label;
+ protected final boolean cancellable;
+
+ public TaskState(boolean hasContainer, TaskLifecycleListener.Event lcEvent,
+ boolean cancellable) {
+ this.hasContainer = hasContainer;
+ lifeCycleEvent = lcEvent;
+ this.cancellable = cancellable;
+ String name = toString();
+ name = name.replace("State", "");
+ name = name.replaceAll("([a-z]+)([A-Z])", "$1_$2");
+ label = name.toUpperCase();
+ }
+
+ protected void endOrAck(EventContext context) {
+ if (context.task.trackingState == Task.TrackingState.START_ACK) {
+ transition(context, WAIT_END_ACK);
+ } else {
+ taskTerminated(context);
+ }
+ }
+
+ public void requestContainer(EventContext context) {
+ illegalState(context, "requestContainer");
+ }
+
+ /**
+ * Resource Manager reports that the task has been allocated a container.
+ *
+ * @param context
+ * @param container
+ */
+
+ public void containerAllocated(EventContext context, Container container) {
+ illegalState(context, "containerAllocated");
+ }
+
+ /**
+ * The launch of the container failed.
+ *
+ * @param context
+ * @param t
+ */
+
+ public void launchFailed(EventContext context, Throwable t) {
+ illegalState(context, "launchFailed");
+ }
+
+ /**
+ * Node Manager reports that the task has started execution.
+ *
+ * @param context
+ */
+
+ public void containerStarted(EventContext context) {
+ illegalState(context, "containerStarted");
+ }
+
+ /**
+ * The monitoring plugin has detected that the task has confirmed that it is
+ * fully started.
+ */
+
+ public void startAck(EventContext context) {
+ illegalState(context, "startAck");
+ }
+
+ /**
+ * The node manager request to stop a task failed.
+ *
+ * @param context
+ * @param t
+ */
+
+ public void stopTaskFailed(EventContext context, Throwable t) {
+ illegalState(context, "stopTaskFailed");
+ }
+
+ /**
+ * The monitoring plugin has detected that the task has confirmed that it has
+ * started shutdown.
+ */
+
+ public void completionAck(EventContext context) {
+ illegalState(context, "completionAck");
+ }
+
+ /**
+ * Node Manager reports that the task has stopped execution. We don't yet know
+ * if this was a success or failure.
+ *
+ * @param context
+ */
+
+ public void containerStopped(EventContext context) {
+ illegalState(context, "containerStopped");
+ }
+
+ /**
+ * Resource Manager reports that the task has completed execution and provided
+ * the completion status.
+ *
+ * @param context
+ * @param status
+ */
+
+ public void containerCompleted(EventContext context, ContainerStatus status) {
+ completed(context, status);
+ illegalState(context, "containerCompleted");
+ }
+
+ /**
+ * Cluster manager wishes to cancel this task.
+ *
+ * @param context
+ */
+
+ public void cancel(EventContext context) {
+ illegalState(context, "cancel");
+ }
+
+ public void tick(EventContext context, long curTime) {
+ // Ignore by default
+ }
+
+ /**
+ * Implement a state transition, alerting any life cycle listeners and
+ * updating the log file. Marks the start time of the new state in support of
+ * states that implement a timeout.
+ *
+ * @param context
+ * @param newState
+ */
+
+ protected void transition(EventContext context, TaskState newState) {
+ TaskState oldState = context.task.state;
+ LOG.info(context.task.getLabel() + " " + oldState.toString() + " --> "
+ + newState.toString());
+ context.task.state = newState;
+ if (newState.lifeCycleEvent != oldState.lifeCycleEvent) {
+ context.controller.fireLifecycleChange(newState.lifeCycleEvent, context);
+ }
+ context.task.stateStartTime = System.currentTimeMillis();
+ }
+
+ /**
+ * Task failed when starting. No container has been allocated. The task
+ * will go from:<br>
+ * * --> END
+ * <p>
+ * If the run failed, and the task can be retried, it may
+ * then move from<br>
+ * END --> STARTING
+ * @param context
+ * @param disposition
+ */
+
+ protected void taskStartFailed(EventContext context,
+ Disposition disposition) {
+
+ // No container, so don't alert the task manager.
+
+ assert context.task.container == null;
+
+ context.getTaskManager().completed(context);
+ taskEnded(context, disposition);
+ retryTask(context);
+ }
+
+ /**
+ * A running task terminated. It may have succeeded or failed,
+ * this method will determine which.
+ * <p>
+ * Every task goes from:<br>
+ * * --> END
+ * <p>
+ * If the run failed, and the task can be retried, it may
+ * then move from<br>
+ * END --> STARTING
+ *
+ * @param context
+ */
+
+ protected void taskTerminated(EventContext context) {
+ Task task = context.task;
+
+ // Give the task manager a peek at the completed task.
+ // The task manager can override retry behavior. To
+ // cancel a task that would otherwise be retried, call
+ // cancel( ) on the task.
+
+ context.getTaskManager().completed(context);
+ context.group.containerReleased(task);
+ assert task.completionStatus != null;
+ if (task.completionStatus.getExitStatus() == 0) {
+ taskEnded(context, Disposition.COMPLETED);
+ context.group.taskEnded(context.task);
+ } else {
+ taskEnded(context, Disposition.RUN_FAILED);
+ retryTask(context);
+ }
+ }
+
+ /**
+ * Implements the details of marking a task as ended. Note, this method
+ * does not deregister the task with the scheduler state, we keep it
+ * registered in case we decide to retry.
+ *
+ * @param context
+ * @param disposition
+ */
+
+ private void taskEnded(EventContext context, Disposition disposition) {
+ Task task = context.task;
+ if (disposition == null) {
+ assert task.disposition != null;
+ } else {
+ task.disposition = disposition;
+ }
+ task.completionTime = System.currentTimeMillis();
+ transition(context, END);
+ }
+
+ /**
+ * Retry a task. Requires that the task currently be in the END state to provide
+ * clean state transitions. Will deregister the task if it cannot be retried
+ * because the cluster is ending or the task has failed too many times.
+ * Otherwise, starts the whole life cycle over again.
+ *
+ * @param context
+ */
+
+ private void retryTask(EventContext context) {
+ Task task = context.task;
+ assert task.state == END;
+ if (!context.controller.isLive() || !task.retryable()) {
+ context.group.taskEnded(task);
+ return;
+ }
+ if (task.tryCount > task.taskGroup.getMaxRetries()) {
+ LOG.error(task.getLabel() + " - Too many retries: " + task.tryCount);
+ task.disposition = Disposition.TOO_MANY_RETRIES;
+ context.group.taskEnded(task);
+ return;
+ }
+ LOG.info(task.getLabel() + " - Retrying task, try " + task.tryCount);
+ context.group.taskRetried(task);
+ task.reset();
+ transition(context, START);
+ context.group.enqueuePendingRequest(task);
+ }
+
+ /**
+ * An event is called in a state where it is not expected. Log it, ignore it
+ * and hope it goes away.
+ *
+ * @param action
+ */
+
+ private void illegalState(EventContext context, String action) {
+ // Intentionally assert: fails during debugging, soldiers on in production.
+
+ assert false;
+ LOG.error(context.task.getLabel() + " - Action " + action
+ + " in wrong state: " + toString(),
+ new IllegalStateException("Action in wrong state"));
+ }
+
+ protected void completed(EventContext context, ContainerStatus status) {
+ Task task = context.task;
+ String diag = status.getDiagnostics();
+ LOG.trace(
+ task.getLabel() + " Completed, exit status: " + status.getExitStatus()
+ + (DoYUtil.isBlank(diag) ? "" : ": " + status.getDiagnostics()));
+ task.completionStatus = status;
+ }
+
+ @Override
+ public String toString() { return getClass().getSimpleName(); }
+
+ public boolean hasContainer() { return hasContainer; }
+
+ public String getLabel() { return label; }
+
+ public boolean isCancellable() {
+ return cancellable;
+ }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskVisitor.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskVisitor.java
new file mode 100644
index 000000000..c90d4f8c9
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskVisitor.java
@@ -0,0 +1,22 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+public interface TaskVisitor {
+ void visit(Task task);
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/YarnFacadeException.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/YarnFacadeException.java
new file mode 100644
index 000000000..8ac0a5d0b
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/YarnFacadeException.java
@@ -0,0 +1,30 @@
+/*
+ * 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.drill.yarn.appMaster;
+
+/**
+ * Exceptions thrown from the YARN facade: the wrapper around the YARN AM
+ * interfaces.
+ */
+
+@SuppressWarnings("serial")
+public class YarnFacadeException extends Exception {
+ public YarnFacadeException(String msg, Exception e) {
+ super(msg, e);
+ }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AMSecurityManager.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AMSecurityManager.java
new file mode 100644
index 000000000..fbca171f5
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AMSecurityManager.java
@@ -0,0 +1,34 @@
+/*
+ * 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.drill.yarn.appMaster.http;
+
+/**
+ * Security manager for the Application Master. Allows a variety
+ * of security systems, including Drill's user authentication
+ * and DoY's static user/password, or an open AM web UI.
+ */
+
+public interface AMSecurityManager {
+ void init();
+
+ boolean requiresLogin();
+
+ boolean login(String user, String password);
+
+ void close();
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AMSecurityManagerImpl.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AMSecurityManagerImpl.java
new file mode 100644
index 000000000..d31690e1a
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AMSecurityManagerImpl.java
@@ -0,0 +1,221 @@
+/*
+ * 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.drill.yarn.appMaster.http;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticationException;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticatorFactory;
+import org.apache.drill.exec.util.ImpersonationUtil;
+import org.apache.drill.yarn.appMaster.AMWrapperException;
+import org.apache.drill.yarn.core.DoYUtil;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+
+import com.typesafe.config.Config;
+
+/**
+ * Implements the three supported AM security models: Drill,
+ * hard-coded user and password, and open access.
+ */
+
+public class AMSecurityManagerImpl implements AMSecurityManager {
+ private static final Log LOG = LogFactory.getLog(AMSecurityManagerImpl.class);
+
+ /**
+ * Thin layer around the Drill authentication system to adapt from
+ * Drill-on-YARN's environment to that expected by the Drill classes.
+ */
+ private static class DrillSecurityManager implements AMSecurityManager {
+ private UserAuthenticator authenticator;
+
+ @Override
+ public void init() {
+ try {
+ DrillOnYarnConfig doyConfig = DrillOnYarnConfig.instance();
+ DrillConfig config = doyConfig.getDrillConfig();
+ ScanResult classpathScan = doyConfig.getClassPathScan();
+ if (config.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED)) {
+ authenticator = UserAuthenticatorFactory.createAuthenticator(config,
+ classpathScan);
+ } else {
+ authenticator = null;
+ }
+ } catch (DrillbitStartupException e) {
+ LOG.info("Authentication initialization failed", e);
+ throw new AMWrapperException("Security init failed", e);
+ }
+ }
+
+ @Override
+ public boolean login(String user, String password) {
+ if (authenticator == null) {
+ return true;
+ }
+ try {
+ authenticator.authenticate(user, password);
+ } catch (UserAuthenticationException e) {
+ LOG.info("Authentication failed for user " + user, e);
+ return false;
+ }
+ return ImpersonationUtil.getProcessUserName().equals(user);
+ }
+
+ @Override
+ public void close() {
+ try {
+ if (authenticator != null) {
+ authenticator.close();
+ }
+ } catch (IOException e) {
+ LOG.info("Ignoring error on authenticator close", e);
+ }
+ }
+
+ @Override
+ public boolean requiresLogin() {
+ return authenticator != null;
+ }
+ }
+
+ /**
+ * Simple security manager: user name and password reside in the DoY config
+ * file.
+ */
+
+ private static class SimpleSecurityManager implements AMSecurityManager {
+
+ private String userName;
+ private String password;
+
+ @Override
+ public void init() {
+ Config config = DrillOnYarnConfig.config();
+ userName = config.getString(DrillOnYarnConfig.HTTP_USER_NAME);
+ password = config.getString(DrillOnYarnConfig.HTTP_PASSWORD);
+ if (DoYUtil.isBlank(userName)) {
+ LOG.warn("Simple HTTP authentication is enabled, but "
+ + DrillOnYarnConfig.HTTP_USER_NAME + " is blank.");
+ }
+ if (DoYUtil.isBlank(userName)) {
+ LOG.warn("Simple HTTP authentication is enabled, but "
+ + DrillOnYarnConfig.HTTP_PASSWORD + " is blank.");
+ }
+ }
+
+ @Override
+ public boolean requiresLogin() {
+ return !DoYUtil.isBlank(userName);
+ }
+
+ @Override
+ public boolean login(String user, String pwd) {
+ if (!requiresLogin()) {
+ return true;
+ }
+ boolean ok = userName.equals(user) && password.equals(pwd);
+ if (!ok) {
+ LOG.info(
+ "Failed login attempt with simple authorization for user " + user);
+ }
+ return ok;
+ }
+
+ @Override
+ public void close() {
+ // Nothing to do
+ }
+
+ }
+
+ private static AMSecurityManagerImpl instance;
+
+ private AMSecurityManager managerImpl;
+
+ private AMSecurityManagerImpl() {
+ }
+
+ public static void setup() {
+ instance = new AMSecurityManagerImpl();
+ instance.init();
+ }
+
+ /**
+ * Look at the DoY config file to decide which security system (if any) to
+ * use.
+ */
+
+ @Override
+ public void init() {
+ Config config = DrillOnYarnConfig.config();
+ String authType = config.getString(DrillOnYarnConfig.HTTP_AUTH_TYPE);
+ if (DrillOnYarnConfig.AUTH_TYPE_DRILL.equals(authType)) {
+ // Drill authentication. Requires both DoY to select Drill
+ // auth, and for Drill's auth to be enabled.
+ if(config.getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED)) {
+ managerImpl = new DrillSecurityManager();
+ managerImpl.init();
+ }
+ } else if (DrillOnYarnConfig.AUTH_TYPE_SIMPLE.equals(authType)) {
+ managerImpl = new SimpleSecurityManager();
+ managerImpl.init();
+ } else if (DoYUtil.isBlank(authType)
+ || DrillOnYarnConfig.AUTH_TYPE_NONE.equals(authType)) {
+ ;
+ } else {
+ LOG.error("Unrecognized authorization type for "
+ + DrillOnYarnConfig.HTTP_AUTH_TYPE + ": " + authType
+ + " - assuming no auth.");
+ }
+ }
+
+ @Override
+ public boolean login(String user, String password) {
+ if (managerImpl == null) {
+ return true;
+ }
+ return managerImpl.login(user, password);
+ }
+
+ @Override
+ public void close() {
+ if (managerImpl != null) {
+ managerImpl.close();
+ managerImpl = null;
+ }
+ }
+
+ @Override
+ public boolean requiresLogin() {
+ return managerImpl != null;
+ }
+
+ public static AMSecurityManager instance() {
+ return instance;
+ }
+
+ public static boolean isEnabled() {
+ return instance != null && instance.managerImpl != null;
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AbstractTasksModel.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AbstractTasksModel.java
new file mode 100644
index 000000000..bc4e8d64d
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AbstractTasksModel.java
@@ -0,0 +1,380 @@
+/*
+ * 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.drill.yarn.appMaster.http;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.yarn.appMaster.ClusterController;
+import org.apache.drill.yarn.appMaster.ClusterControllerImpl;
+import org.apache.drill.yarn.appMaster.ControllerVisitor;
+import org.apache.drill.yarn.appMaster.Task;
+import org.apache.drill.yarn.appMaster.Task.TrackingState;
+import org.apache.drill.yarn.appMaster.TaskState;
+import org.apache.drill.yarn.appMaster.TaskVisitor;
+import org.apache.drill.yarn.core.DoYUtil;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.drill.yarn.zk.ZKRegistry;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+public abstract class AbstractTasksModel {
+ public static class TaskModel {
+ public int id;
+ protected String groupName;
+ protected boolean isLive;
+ protected TaskState taskState;
+ protected String taskStateHint;
+ protected String state;
+ protected boolean cancelled;
+ protected String trackingState;
+ protected String trackingStateHint;
+ protected Container container;
+ protected DrillbitEndpoint endpoint;
+ protected long startTime;
+ protected int memoryMb;
+ protected int vcores;
+ protected double disks;
+ protected String containerId;
+ protected String nmLink;
+ protected long endTime;
+ protected String disposition;
+ protected int tryCount;
+
+ private Map<TaskState,String> stateHints = makeStateHints( );
+ private Map<TrackingState,String> trackingStateHints = makeTrackingStateHints( );
+
+ public TaskModel(Task task, boolean live) {
+ id = task.taskId;
+ groupName = task.scheduler.getName();
+ taskState = task.getState();
+ taskStateHint = stateHints.get(taskState);
+ state = taskState.getLabel();
+ cancelled = task.isCancelled();
+ isLive = live && taskState == TaskState.RUNNING;
+ TrackingState tState = task.getTrackingState();
+ trackingState = tState.getDisplayName();
+ trackingStateHint = trackingStateHints.get(tState);
+ container = task.container;
+ startTime = task.launchTime;
+ if (task.container != null) {
+ containerId = task.container.getId().toString();
+ Resource resource = task.container.getResource();
+ memoryMb = resource.getMemory();
+ vcores = resource.getVirtualCores();
+ disks = task.getContainerSpec().disks;
+
+ // Emulate the NM link. Used for debugging, gets us to
+ // the page on the NM UI for this container so we can see
+ // logs, etc.
+
+ nmLink = "http://" + task.container.getNodeHttpAddress();
+ } else {
+ memoryMb = task.scheduler.getResource().memoryMb;
+ vcores = task.scheduler.getResource().vCores;
+ }
+ endpoint = (DrillbitEndpoint) task.properties
+ .get(ZKRegistry.ENDPOINT_PROPERTY);
+ if (!live) {
+ endTime = task.completionTime;
+ tryCount = task.tryCount;
+
+ // Determine disposition from most general to most
+ // specific sources of information.
+
+ disposition = state;
+ if (task.disposition != null) {
+ disposition = task.disposition.toString();
+ }
+ if (task.completionStatus != null) {
+ disposition = reformatDiagnostics( task.completionStatus.getDiagnostics() );
+ }
+ if (task.error != null) {
+ disposition = task.error.getMessage();
+ }
+ }
+ }
+
+ private enum FormatState { PRE_STACK, IN_STACK, POST_STACK };
+
+ /**
+ * YARN diagnostics are verbose: they contain a stack trace of the YARN node
+ * manager thread (not Drill), and contain blank lines, the container ID,
+ * etc. Remove unnecessary cruft to make the diagnostics simpler and smaller
+ * in the Web UI.
+ *
+ * @param orig YARN diagnostics
+ * @return cleaned-up version.
+ */
+
+ public static String reformatDiagnostics( String orig ) {
+ try {
+ StringBuilder buf = new StringBuilder( );
+ BufferedReader reader = new BufferedReader( new StringReader( orig ) );
+ String line;
+ FormatState state = FormatState.PRE_STACK;
+ while ( (line = reader.readLine()) != null ) {
+ switch( state ) {
+ case PRE_STACK:
+ if ( line.startsWith( "Container id:") ) {
+ continue;
+ }
+ if ( line.startsWith( "Stack trace:" ) ) {
+ state = FormatState.IN_STACK;
+ continue;
+ }
+ break;
+ case IN_STACK:
+ if ( line.trim().isEmpty() ) {
+ state = FormatState.POST_STACK;
+ }
+ continue;
+ case POST_STACK:
+ default:
+ break;
+ }
+ if ( line.trim().isEmpty() ) {
+ continue;
+ }
+ buf.append( line );
+ buf.append( "<br/>\n" );
+ }
+ buf.append( "See log file for details." );
+ return buf.toString();
+ } catch (IOException e) {
+ // Will never occur. But, if the impossible happens, just return
+ // the original diagnostics.
+
+ return orig.replace("\n", "<br>\n");
+ }
+ }
+
+ private Map<TaskState, String> makeStateHints() {
+ Map<TaskState, String> hints = new HashMap<>();
+ hints.put(TaskState.START, "Queued to send a container request to YARN.");
+ hints.put(TaskState.REQUESTING, "Container request sent to YARN.");
+ hints.put(TaskState.LAUNCHING,
+ "YARN provided a container, send launch request.");
+ hints.put(TaskState.WAIT_START_ACK,
+ "Drillbit launched, waiting for ZooKeeper registration.");
+ hints.put(TaskState.RUNNING, "Drillbit is running normally.");
+ hints.put(TaskState.ENDING,
+ "Graceful shutdown request sent to the Drillbit.");
+ hints.put(TaskState.KILLING,
+ "Sent the YARN Node Manager a request to forcefully kill the Drillbit.");
+ hints.put(TaskState.WAIT_END_ACK,
+ "Drillbit has shut down; waiting for ZooKeeper to confirm.");
+ // The UI will never display the END state.
+ hints.put(TaskState.END, "The Drillbit has shut down.");
+ return hints;
+ }
+
+ private Map<TrackingState, String> makeTrackingStateHints() {
+ Map<TrackingState, String> hints = new HashMap<>();
+ // UNTRACKED state is not used by Drillbits.
+ hints.put(TrackingState.UNTRACKED, "Task is not tracked in ZooKeeper.");
+ hints.put(TrackingState.NEW,
+ "Drillbit has not yet registered with ZooKeeper.");
+ hints.put(TrackingState.START_ACK,
+ "Drillbit has registered normally with ZooKeeper.");
+ hints.put(TrackingState.END_ACK,
+ "Drillbit is no longer registered with ZooKeeper.");
+ return hints;
+ }
+
+ public String getTaskId() {
+ return Integer.toString(id);
+ }
+
+ public String getGroupName( ) { return groupName; }
+
+ public boolean isLive( ) {
+ return isLive;
+ }
+
+ public String getHost( ) {
+ if ( container == null ) {
+ return ""; }
+ return container.getNodeId().getHost();
+ }
+
+ public String getLink( ) {
+ if ( endpoint == null ) {
+ return ""; }
+ String port = DrillOnYarnConfig.config( ).getString( DrillOnYarnConfig.DRILLBIT_HTTP_PORT );
+ String protocol = "http:";
+ if ( DrillOnYarnConfig.config().getBoolean( DrillOnYarnConfig.DRILLBIT_USE_HTTPS ) ) {
+ protocol = "https:";
+ }
+ return protocol + "//" + endpoint.getAddress() + ":" + port + "/";
+ }
+
+ public String getState( ) { return state.toString(); }
+ public String getStateHint( ) { return taskStateHint; }
+ public boolean isCancelled( ) { return cancelled; }
+
+ public boolean isCancellable( ) {
+ return ! cancelled && taskState.isCancellable( );
+ }
+
+ public String getTrackingState( ) { return trackingState; }
+ public String getTrackingStateHint( ) { return trackingStateHint; }
+
+ public String getStartTime( ) {
+ if ( startTime == 0 ) {
+ return ""; }
+ return DoYUtil.toIsoTime( startTime );
+ }
+
+ public int getMemory( ) { return memoryMb; }
+ public int getVcores( ) { return vcores; }
+ public String getDisks( ) {
+ return String.format( "%.2f", disks );
+ }
+ public boolean hasContainer( ) { return containerId != null; }
+ public String getContainerId( ) { return displayString( containerId ); }
+ public String getNmLink( ) { return displayString( nmLink ); }
+ public String getDisposition( ) { return displayString( disposition ); }
+ public int getTryCount( ) { return tryCount; }
+ public String displayString( String value ) { return (value == null) ? "" : value; }
+
+ public String getEndTime( ) {
+ if ( endTime == 0 ) {
+ return ""; }
+ return DoYUtil.toIsoTime( endTime );
+ }
+ }
+
+ public static class UnmanagedDrillbitModel
+ {
+ protected String host;
+ protected String ports;
+
+ public UnmanagedDrillbitModel( String endpoint ) {
+ String parts[] = endpoint.split( ":" );
+ if ( parts.length < 4 ) {
+ // Should never occur, but better save than sorry.
+
+ host = endpoint;
+ ports = "";
+ }
+ else {
+ host = parts[0];
+ List<String> thePorts = new ArrayList<>( );
+ thePorts.add( parts[1] );
+ thePorts.add( parts[2] );
+ thePorts.add( parts[3] );
+ ports = DoYUtil.join( ", ", thePorts );
+ }
+ }
+
+ public String getHost( ) { return host; }
+ public String getPorts( ) { return ports; }
+ }
+
+ protected boolean supportsDisks;
+ protected List<TaskModel> results = new ArrayList<>( );
+
+ public List<TaskModel> getTasks( ) { return results; }
+ public boolean hasTasks( ) { return ! results.isEmpty(); }
+ public boolean supportsDiskResource( ) { return supportsDisks; }
+
+ public static class TasksModel extends AbstractTasksModel implements TaskVisitor
+ {
+ protected List<UnmanagedDrillbitModel> unmanaged;
+ protected List<String> blacklist;
+
+ @Override
+ public void visit(Task task) {
+ results.add( new TaskModel( task, true ) );
+ }
+
+ /**
+ * Sort tasks by Task ID.
+ */
+
+ public void sortTasks() {
+ Collections.sort( results, new Comparator<TaskModel>( ) {
+ @Override
+ public int compare(TaskModel t1, TaskModel t2) {
+ return Integer.compare( t1.id, t2.id );
+ }
+ });
+ }
+
+ /**
+ * List any anomalies: either stray Drillbits (those in ZK but not launched by DoY),
+ * or blacklisted nodes.
+ * <p>
+ * To avoid race conditions, do not use the controller visitor to invoke this method,
+ * we want to leave the controller unlocked and instead lock only the ZK registry.
+ *
+ * @param controller
+ */
+
+ public void listAnomalies(ClusterController controller) {
+ listUnmanaged(controller);
+ synchronized( controller ) {
+ blacklist = ((ClusterControllerImpl) controller).getNodeInventory().getBlacklist();
+ }
+ Collections.sort( blacklist );
+ }
+
+ private void listUnmanaged(ClusterController controller) {
+ ZKRegistry zkRegistry = (ZKRegistry) controller.getProperty( ZKRegistry.CONTROLLER_PROPERTY );
+ if ( zkRegistry == null ) {
+ return;
+ }
+ List<String> endpoints = zkRegistry.listUnmanagedDrillits( );
+ if ( endpoints.isEmpty() ) {
+ return; }
+ unmanaged = new ArrayList<>( );
+ for ( String endpoint : endpoints ) {
+ unmanaged.add( new UnmanagedDrillbitModel( endpoint ) );
+ }
+ }
+
+ public List<UnmanagedDrillbitModel>getUnnamaged( ) { return unmanaged; }
+ public boolean hasUnmanagedDrillbits( ) { return unmanaged != null; }
+ public int getUnmanagedDrillbitCount( ) {
+ return (unmanaged == null) ? 0 : unmanaged.size( );
+ }
+ public boolean hasBlacklist( ) { return ! blacklist.isEmpty(); }
+ public int getBlacklistCount( ) { return blacklist.size( ); }
+ public List<String> getBlacklist( ) { return blacklist; }
+ }
+
+ public static class HistoryModel extends AbstractTasksModel implements ControllerVisitor
+ {
+ @Override
+ public void visit(ClusterController controller) {
+ ClusterControllerImpl impl = (ClusterControllerImpl) controller;
+ for ( Task task : impl.getHistory( ) ) {
+ results.add( new TaskModel( task, false ) );
+ }
+ }
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AmRestApi.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AmRestApi.java
new file mode 100644
index 000000000..21ddc4b03
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AmRestApi.java
@@ -0,0 +1,296 @@
+/*
+ * 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.drill.yarn.appMaster.http;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.annotation.security.PermitAll;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+
+import org.apache.drill.yarn.appMaster.Dispatcher;
+import org.apache.drill.yarn.appMaster.http.AbstractTasksModel.TaskModel;
+import org.apache.drill.yarn.appMaster.http.ControllerModel.ClusterGroupModel;
+import org.apache.drill.yarn.core.DoYUtil;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.drill.yarn.core.NameValuePair;
+import org.apache.drill.yarn.zk.ZKClusterCoordinatorDriver;
+
+public class AmRestApi extends PageTree
+{
+ @Path("/config")
+ @PermitAll
+ public static class ConfigResource
+ {
+ @GET
+ @Produces(MediaType.APPLICATION_JSON)
+ public Map<String,Object> getConfig( ) {
+ Map<String, Object> map = new HashMap<>();
+ for (NameValuePair pair : DrillOnYarnConfig.instance().getPairs()) {
+ map.put(pair.getName(), pair.getValue());
+ }
+ return map;
+ }
+ }
+
+ /**
+ * Returns cluster status as a tree of JSON objects. Done as explicitly-defined
+ * maps to specify the key names (which must not change to avoid breaking
+ * compatibility) and to handle type conversions.
+ */
+
+ @Path("/status")
+ @PermitAll
+ public static class StatusResource
+ {
+ @GET
+ @Produces(MediaType.APPLICATION_JSON)
+ public Map<String,Object> getStatus( ) {
+ ControllerModel model = new ControllerModel( );
+ dispatcher.getController().visit( model );
+
+ Map<String,Object> root = new HashMap<>( );
+ root.put( "state", model.state.toString() );
+
+ Map<String, Object> summary = new HashMap<>();
+ summary.put("drillMemoryMb", model.totalDrillMemory);
+ summary.put("drillVcores", model.totalDrillVcores);
+ summary.put("yarnMemoryMb", model.yarnMemory);
+ summary.put("yarnVcores", model.yarnVcores);
+ summary.put("liveBitCount", model.liveCount);
+ summary.put("totalBitCount", model.taskCount);
+ summary.put("targetBitCount", model.targetCount);
+ summary.put("unmanagedCount", model.getUnmanagedCount());
+ summary.put("blackListCount", model.getBlacklistCount());
+ summary.put("freeNodeCount", model.getFreeNodeCount());
+ root.put("summary", summary);
+
+ List<Map<String, Object>> pools = new ArrayList<>();
+ for (ClusterGroupModel pool : model.groups) {
+ Map<String, Object> poolObj = new HashMap<>();
+ poolObj.put("name", pool.name);
+ poolObj.put("type", pool.type);
+ poolObj.put("liveBitCount", pool.liveCount);
+ poolObj.put("targetBitCount", pool.targetCount);
+ poolObj.put("totalBitCount", pool.taskCount);
+ poolObj.put("totalMemoryMb", pool.memory);
+ poolObj.put("totalVcores", pool.vcores);
+ pools.add(poolObj);
+ }
+ root.put("pools", pools);
+
+ AbstractTasksModel.TasksModel tasksModel = new AbstractTasksModel.TasksModel();
+ dispatcher.getController().visitTasks(tasksModel);
+ List<Map<String, Object>> bits = new ArrayList<>();
+ for (TaskModel task : tasksModel.results) {
+ Map<String, Object> bitObj = new HashMap<>();
+ bitObj.put("containerId", task.container.getId().toString());
+ bitObj.put("host", task.getHost());
+ bitObj.put("id", task.id);
+ bitObj.put("live", task.isLive());
+ bitObj.put("memoryMb", task.memoryMb);
+ bitObj.put("vcores", task.vcores);
+ bitObj.put("pool", task.groupName);
+ bitObj.put("state", task.state);
+ bitObj.put("trackingState", task.trackingState);
+ bitObj.put("endpoint",
+ ZKClusterCoordinatorDriver.asString(task.endpoint));
+ bitObj.put("link", task.getLink());
+ bitObj.put("startTime", task.getStartTime());
+ bits.add(bitObj);
+ }
+ root.put("drillbits", bits);
+
+ return root;
+ }
+ }
+
+ /**
+ * Stop the cluster. Uses a key to validate the request. The value of the key is
+ * set in the Drill-on-YARN configuration file. The purpose is simply to prevent
+ * accidental cluster shutdown when experimenting with the REST API; this is
+ * not meant to be a security mechanism.
+ *
+ * @param key
+ * @return
+ */
+
+ @Path("/stop")
+ @PermitAll
+ public static class StopResource
+ {
+ @DefaultValue( "" )
+ @QueryParam( "key" )
+ String key;
+
+ @POST
+ @Produces(MediaType.APPLICATION_JSON)
+ public Map<String,String> postStop( )
+ {
+ Map<String, String> error = checkKey(key);
+ if (error != null) {
+ return error;
+ }
+
+ dispatcher.getController().shutDown();
+ return successResponse("Shutting down");
+ }
+ }
+
+ @Path("/resize/{quantity}")
+ @PermitAll
+ public static class ResizeResource
+ {
+ @PathParam(value = "quantity")
+ String quantity;
+ @DefaultValue( "" )
+ @QueryParam( "key" )
+ String key;
+
+ @POST
+ @Produces(MediaType.APPLICATION_JSON)
+ public Map<String,String> postResize( )
+ {
+ ResizeRequest request = new ResizeRequest(key, quantity);
+ if (request.error != null) {
+ return request.error;
+ }
+
+ int curSize = dispatcher.getController().getTargetCount();
+ dispatcher.getController().resizeTo(request.n);
+ return successResponse("Resizing from " + curSize + " to " + request.n);
+ }
+ }
+
+ protected static class ResizeRequest
+ {
+ Map<String,String> error;
+ int n;
+
+ public ResizeRequest( String key, String quantity ) {
+ error = checkKey(key);
+ if (error != null) {
+ return;
+ }
+ try {
+ n = Integer.parseInt(quantity);
+ } catch (NumberFormatException e) {
+ error = errorResponse("Invalid argument: " + quantity);
+ }
+ if (n < 0) {
+ error = errorResponse("Invalid argument: " + quantity);
+ }
+ }
+ }
+
+ @Path("/grow/{quantity}")
+ @PermitAll
+ public static class GrowResource
+ {
+ @PathParam(value = "quantity")
+ @DefaultValue( "1" )
+ String quantity;
+ @DefaultValue( "" )
+ @QueryParam( "key" )
+ String key;
+
+ @POST
+ @Produces(MediaType.APPLICATION_JSON)
+ public Map<String,String> postResize( )
+ {
+ ResizeRequest request = new ResizeRequest(key, quantity);
+ if (request.error != null) {
+ return request.error;
+ }
+
+ int curSize = dispatcher.getController().getTargetCount();
+ int newSize = curSize + request.n;
+ dispatcher.getController().resizeTo(newSize);
+ return successResponse("Growing by " + request.n + " to " + newSize);
+ }
+ }
+
+ @Path("/shrink/{quantity}")
+ @PermitAll
+ public static class ShrinkResource
+ {
+ @PathParam(value = "quantity")
+ @DefaultValue( "1" )
+ String quantity;
+ @DefaultValue( "" )
+ @QueryParam( "key" )
+ String key;
+
+ @POST
+ @Produces(MediaType.APPLICATION_JSON)
+ public Map<String,String> postResize( )
+ {
+ ResizeRequest request = new ResizeRequest(key, quantity);
+ if (request.error != null) {
+ return request.error;
+ }
+ int curSize = dispatcher.getController().getTargetCount();
+ int newSize = Math.max(curSize - request.n, 0);
+ dispatcher.getController().resizeTo(newSize);
+ return successResponse("Shrinking by " + request.n + " to " + newSize);
+ }
+ }
+
+ private static Map<String, String> checkKey(String key) {
+ String masterKey = DrillOnYarnConfig.config()
+ .getString(DrillOnYarnConfig.HTTP_REST_KEY);
+ if (!DoYUtil.isBlank(masterKey) && !masterKey.equals(key)) {
+ return errorResponse("Invalid Key");
+ }
+ return null;
+ }
+
+ private static Map<String, String> errorResponse(String msg) {
+ Map<String, String> resp = new HashMap<>();
+ resp.put("status", "error");
+ resp.put("message", msg);
+ return resp;
+ }
+
+ private static Map<String, String> successResponse(String msg) {
+ Map<String, String> resp = new HashMap<>();
+ resp.put("status", "ok");
+ resp.put("message", msg);
+ return resp;
+ }
+
+ public AmRestApi(Dispatcher dispatcher) {
+ super(dispatcher);
+
+ register(ConfigResource.class);
+ register(StatusResource.class);
+ register(StopResource.class);
+ register(ResizeResource.class);
+ register(GrowResource.class);
+ register(ShrinkResource.class);
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AuthDynamicFeature.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AuthDynamicFeature.java
new file mode 100644
index 000000000..55cd59aaa
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AuthDynamicFeature.java
@@ -0,0 +1,114 @@
+/**
+ * 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.drill.yarn.appMaster.http;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.yarn.appMaster.http.WebUiPageTree.LogInLogOutPages;
+import org.glassfish.jersey.server.model.AnnotatedMethod;
+
+import javax.annotation.Priority;
+import javax.annotation.security.PermitAll;
+import javax.annotation.security.RolesAllowed;
+import javax.ws.rs.Priorities;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerRequestFilter;
+import javax.ws.rs.container.DynamicFeature;
+import javax.ws.rs.container.ResourceInfo;
+import javax.ws.rs.core.FeatureContext;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.SecurityContext;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URLEncoder;
+
+/**
+ * Implementation of {@link DynamicFeature}. As part of the setup it adds the
+ * auth check filter {@link AuthCheckFilter} for resources that need to have
+ * user authenticated. If authentication is not done, request is forwarded to
+ * login page.
+ * <p>
+ * Shameless copy of
+ * {@link org.apache.drill.exec.server.rest.auth.DynamicFeature}; the two
+ * implementations should be merged at some point. The difference is only the
+ * log in/log out constant references.
+ */
+
+public class AuthDynamicFeature implements DynamicFeature {
+ private static final Log LOG = LogFactory.getLog(AuthDynamicFeature.class);
+
+ @Override
+ public void configure(final ResourceInfo resourceInfo,
+ final FeatureContext configuration) {
+ AnnotatedMethod am = new AnnotatedMethod(resourceInfo.getResourceMethod());
+
+ // RolesAllowed on the method takes precedence over PermitAll
+ RolesAllowed ra = am.getAnnotation(RolesAllowed.class);
+ if (ra != null) {
+ configuration.register(AuthCheckFilter.INSTANCE);
+ return;
+ }
+
+ // PermitAll takes precedence over RolesAllowed on the class
+ if (am.isAnnotationPresent(PermitAll.class)) {
+ // Do nothing.
+ return;
+ }
+
+ // RolesAllowed on the class takes precedence over PermitAll
+ ra = resourceInfo.getResourceClass().getAnnotation(RolesAllowed.class);
+ if (ra != null) {
+ configuration.register(AuthCheckFilter.INSTANCE);
+ }
+ }
+
+ @Priority(Priorities.AUTHENTICATION) // authentication filter - should go
+ // first before all other filters.
+ private static class AuthCheckFilter implements ContainerRequestFilter {
+ private static AuthCheckFilter INSTANCE = new AuthCheckFilter();
+
+ @Override
+ public void filter(ContainerRequestContext requestContext)
+ throws IOException {
+ final SecurityContext sc = requestContext.getSecurityContext();
+ if (!isUserLoggedIn(sc)) {
+ try {
+ final String destResource = URLEncoder.encode(
+ requestContext.getUriInfo().getRequestUri().toString(), "UTF-8");
+ final URI loginURI = requestContext.getUriInfo().getBaseUriBuilder()
+ .path(LogInLogOutPages.LOGIN_RESOURCE)
+ .queryParam(LogInLogOutPages.REDIRECT_QUERY_PARM, destResource)
+ .build();
+ requestContext
+ .abortWith(Response.temporaryRedirect(loginURI).build());
+ } catch (final Exception ex) {
+ final String errMsg = String.format(
+ "Failed to forward the request to login page: %s",
+ ex.getMessage());
+ LOG.error(errMsg, ex);
+ requestContext
+ .abortWith(Response.serverError().entity(errMsg).build());
+ }
+ }
+ }
+ }
+
+ public static boolean isUserLoggedIn(final SecurityContext sc) {
+ return sc != null && sc.getUserPrincipal() != null;
+ }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/ControllerModel.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/ControllerModel.java
new file mode 100644
index 000000000..8947df57f
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/ControllerModel.java
@@ -0,0 +1,208 @@
+/*
+ * 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.drill.yarn.appMaster.http;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.drill.yarn.appMaster.AMYarnFacade.YarnAppHostReport;
+import org.apache.drill.yarn.appMaster.ClusterController;
+import org.apache.drill.yarn.appMaster.ClusterControllerImpl;
+import org.apache.drill.yarn.appMaster.ClusterControllerImpl.State;
+import org.apache.drill.yarn.appMaster.ControllerVisitor;
+import org.apache.drill.yarn.appMaster.Scheduler;
+import org.apache.drill.yarn.appMaster.SchedulerStateActions;
+import org.apache.drill.yarn.core.ContainerRequestSpec;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.drill.yarn.zk.ZKRegistry;
+
+import com.typesafe.config.Config;
+
+@XmlRootElement
+public class ControllerModel implements ControllerVisitor {
+ public static class ClusterGroupModel {
+ protected String name;
+ protected String type;
+ protected int targetCount;
+ protected int taskCount;
+ protected int liveCount;
+ protected int memory;
+ protected int vcores;
+ protected double disks;
+
+ public String getName( ) { return name; }
+ public String getType( ) { return type; }
+ public int getTargetCount( ) { return targetCount; }
+ public int getTaskCount( ) { return taskCount; }
+ public int getLiveCount( ) { return liveCount; }
+ public int getMemory( ) { return memory; }
+ public int getVcores( ) { return vcores; }
+ public String getDisks( ) {
+ return String.format( "%.02f", disks );
+ }
+ }
+
+ protected String zkConnectStr;
+ protected String zkRoot;
+ protected String zkClusterId;
+ protected ClusterControllerImpl.State state;
+ protected String stateHint;
+ protected boolean supportsDisks;
+ protected int yarnMemory;
+ protected int yarnVcores;
+ protected int yarnNodeCount;
+ protected int taskCount;
+ protected int liveCount;
+ protected int unmanagedCount;
+ protected int targetCount;
+ protected int totalDrillMemory;
+ protected int totalDrillVcores;
+ protected double totalDrillDisks;
+ protected int blacklistCount;
+ protected int freeNodeCount;
+ protected YarnAppHostReport appRpt;
+ protected int refreshSecs;
+ protected List<ClusterGroupModel> groups = new ArrayList<>( );
+
+ public boolean supportsDiskResource( ) { return supportsDisks; }
+ public int getRefreshSecs( ) { return refreshSecs; }
+ public String getZkConnectionStr( ) { return zkConnectStr; }
+ public String getZkRoot( ) { return zkRoot; }
+ public String getZkClusterId( ) { return zkClusterId; }
+ public String getAppId( ) { return appRpt.appId; }
+ public String getRmHost( ) { return appRpt.rmHost; }
+ public String getRmLink( ) { return appRpt.rmUrl; }
+ public String getNmHost( ) { return appRpt.nmHost; }
+ public String getNmLink( ) { return appRpt.nmUrl; }
+ public String getRmAppLink( ) { return appRpt.rmAppUrl; }
+ public String getNmAppLink( ) { return appRpt.nmAppUrl; }
+ public String getState( ) { return state.toString( ); }
+ public String getStateHint( ) { return stateHint; }
+ public int getYarnMemory( ) { return yarnMemory; }
+ public int getYarnVcores( ) { return yarnVcores; }
+ public int getDrillTotalMemory( ) { return totalDrillMemory; }
+ public int getDrillTotalVcores( ) { return totalDrillVcores; }
+ public String getDrillTotalDisks( ) {
+ return String.format( "%.2f", totalDrillDisks );
+ }
+ public int getYarnNodeCount( ) { return yarnNodeCount; }
+ public int getTaskCount( ) { return taskCount; }
+ public int getLiveCount( ) { return liveCount; }
+ public int getUnmanagedCount( ) { return unmanagedCount; }
+ public int getTargetCount( ) { return targetCount; }
+ public List<ClusterGroupModel> getGroups( ) { return groups; }
+ public int getBlacklistCount( ) { return blacklistCount; }
+ public int getFreeNodeCount( ) { return freeNodeCount; }
+
+ private static Map<ClusterControllerImpl.State,String> stateHints = makeStateHints( );
+
+ @Override
+ public void visit(ClusterController controller) {
+ Config config = DrillOnYarnConfig.config();
+ refreshSecs = config.getInt( DrillOnYarnConfig.HTTP_REFRESH_SECS );
+ zkConnectStr = config.getString( DrillOnYarnConfig.ZK_CONNECT );
+ zkRoot = config.getString( DrillOnYarnConfig.ZK_ROOT );
+ zkClusterId = config.getString( DrillOnYarnConfig.CLUSTER_ID );
+
+ ClusterControllerImpl impl = (ClusterControllerImpl) controller;
+ appRpt = impl.getYarn().getAppHostReport();
+
+ state = impl.getState( );
+ stateHint = stateHints.get( state );
+
+ // Removed based on feedback. Users should check the
+ // YARN RM UI instead.
+
+// if ( state == State.LIVE ) {
+// RegisterApplicationMasterResponse resp = impl.getYarn( ).getRegistrationResponse();
+// yarnVcores = resp.getMaximumResourceCapability().getVirtualCores();
+// yarnMemory = resp.getMaximumResourceCapability().getMemory();
+// yarnNodeCount = impl.getYarn( ).getNodeCount();
+// }
+ capturePools( impl );
+ supportsDisks = impl.supportsDiskResource();
+
+ blacklistCount = impl.getNodeInventory( ).getBlacklist( ).size( );
+ freeNodeCount = impl.getFreeNodeCount();
+ }
+
+ private void capturePools(ClusterControllerImpl impl) {
+ for ( SchedulerStateActions pool : impl.getPools( ) ) {
+ ControllerModel.ClusterGroupModel poolModel = new ControllerModel.ClusterGroupModel( );
+ Scheduler sched = pool.getScheduler();
+ ContainerRequestSpec containerSpec = sched.getResource( );
+ poolModel.name = sched.getName();
+ poolModel.type = sched.getType( );
+ poolModel.targetCount = sched.getTarget();
+ poolModel.memory = containerSpec.memoryMb;
+ poolModel.vcores = containerSpec.vCores;
+ poolModel.disks = containerSpec.disks;
+ poolModel.taskCount = pool.getTaskCount();
+ poolModel.liveCount = pool.getLiveCount( );
+ targetCount += poolModel.targetCount;
+ taskCount += poolModel.taskCount;
+ liveCount += poolModel.liveCount;
+ totalDrillMemory += poolModel.liveCount * poolModel.memory;
+ totalDrillVcores += poolModel.liveCount * poolModel.vcores;
+ totalDrillDisks += poolModel.liveCount * poolModel.disks;
+ groups.add( poolModel );
+ }
+ if ( state != State.LIVE ) {
+ targetCount = 0;
+ }
+ }
+
+ /**
+ * Count the unmanaged drillbits. Do this as a separate call, not via the
+ * {@link #visit(ClusterController) visit} method, to avoid locking both
+ * the cluster controller and ZK registry.
+ *
+ * @param controller
+ */
+
+ public void countStrayDrillbits(ClusterController controller) {
+ ZKRegistry zkRegistry = (ZKRegistry) controller.getProperty( ZKRegistry.CONTROLLER_PROPERTY );
+ if ( zkRegistry != null ) {
+ unmanagedCount = zkRegistry.listUnmanagedDrillits().size();
+ }
+ }
+
+ /**
+ * Create a table of user-visible descriptions for each controller state.
+ *
+ * @return
+ */
+
+ private static Map<State, String> makeStateHints() {
+ Map<ClusterControllerImpl.State,String> hints = new HashMap<>( );
+ // UI likely will never display the FAILED state.
+ hints.put( ClusterControllerImpl.State.START, "AM is starting up." );
+ hints.put( ClusterControllerImpl.State.LIVE, "AM is operating normally." );
+ hints.put( ClusterControllerImpl.State.ENDING, "AM is shutting down." );
+ // UI will never display the ENDED state.
+ hints.put( ClusterControllerImpl.State.ENDED, "AM is about to exit." );
+ // UI will never display the FAILED state.
+ hints.put( ClusterControllerImpl.State.FAILED, "AM failed to start and is about to exit." );
+ return hints;
+ }
+
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/PageTree.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/PageTree.java
new file mode 100644
index 000000000..e4d5dc1b1
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/PageTree.java
@@ -0,0 +1,80 @@
+/*
+ * 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.drill.yarn.appMaster.http;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.ws.rs.core.SecurityContext;
+
+import org.apache.drill.exec.server.rest.auth.DrillUserPrincipal;
+import org.apache.drill.yarn.appMaster.Dispatcher;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.glassfish.jersey.server.ResourceConfig;
+
+import com.typesafe.config.Config;
+
+/**
+ * Base class for a tree of web pages (or REST resources) represented
+ * as POJOs. Since the AM web UI is simple, this is the most convenient,
+ * compact way to implement the UI.
+ */
+
+public class PageTree extends ResourceConfig {
+ // These items are a bit clumsy. We need them, but we can't make them
+ // instance variables without a bunch of messiness in the page classes.
+ // So, we let them be static. No harm in setting them multiple times.
+
+ static Dispatcher dispatcher;
+ static Config config;
+
+ public PageTree(Dispatcher dispatcher) {
+ PageTree.dispatcher = dispatcher;
+ config = DrillOnYarnConfig.config();
+ }
+
+ /**
+ * Creates a FreeMarker model that contains two top-level items:
+ * the model itself (as in the default implementation) and the
+ * cluster name (used as a title on each UI page.)
+ *
+ * @param base
+ * @return
+ */
+
+ public static Map<String, Object> toModel(SecurityContext sc, Object base) {
+ Map<String, Object> model = new HashMap<>();
+ model.put("model", base);
+ return toMapModel(sc, model);
+ }
+
+ public static Map<String, Object> toMapModel(SecurityContext sc,
+ Map<String, Object> model) {
+ model.put("clusterName", config.getString(DrillOnYarnConfig.APP_NAME));
+ boolean useAuth = AMSecurityManagerImpl.isEnabled();
+ final boolean isUserLoggedIn = (useAuth)
+ ? AuthDynamicFeature.isUserLoggedIn(sc) : false;
+ model.put("showLogin", useAuth && !isUserLoggedIn);
+ model.put("showLogout", isUserLoggedIn);
+ model.put("docsLink", config.getString(DrillOnYarnConfig.HTTP_DOCS_LINK));
+ String userName = isUserLoggedIn ? sc.getUserPrincipal().getName()
+ : DrillUserPrincipal.ANONYMOUS_USER;
+ model.put("loggedInUserName", userName);
+ return model;
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/WebServer.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/WebServer.java
new file mode 100644
index 000000000..aeeafdefc
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/WebServer.java
@@ -0,0 +1,467 @@
+/*
+ * 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.drill.yarn.appMaster.http;
+
+import static org.apache.drill.exec.server.rest.auth.DrillUserPrincipal.ADMIN_ROLE;
+
+import java.math.BigInteger;
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.KeyStore;
+import java.security.Principal;
+import java.security.SecureRandom;
+import java.security.cert.X509Certificate;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Set;
+
+import javax.servlet.http.HttpSession;
+import javax.servlet.http.HttpSessionEvent;
+import javax.servlet.http.HttpSessionListener;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.yarn.appMaster.Dispatcher;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.bouncycastle.asn1.x500.X500NameBuilder;
+import org.bouncycastle.asn1.x500.style.BCStyle;
+import org.bouncycastle.cert.X509v3CertificateBuilder;
+import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter;
+import org.bouncycastle.cert.jcajce.JcaX509v3CertificateBuilder;
+import org.bouncycastle.operator.ContentSigner;
+import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder;
+import org.eclipse.jetty.http.HttpVersion;
+import org.eclipse.jetty.security.ConstraintMapping;
+import org.eclipse.jetty.security.ConstraintSecurityHandler;
+import org.eclipse.jetty.security.DefaultIdentityService;
+import org.eclipse.jetty.security.DefaultUserIdentity;
+import org.eclipse.jetty.security.IdentityService;
+import org.eclipse.jetty.security.LoginService;
+import org.eclipse.jetty.security.SecurityHandler;
+import org.eclipse.jetty.security.authentication.FormAuthenticator;
+import org.eclipse.jetty.security.authentication.SessionAuthentication;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.SecureRequestCustomizer;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.SessionManager;
+import org.eclipse.jetty.server.SslConnectionFactory;
+import org.eclipse.jetty.server.UserIdentity;
+import org.eclipse.jetty.server.handler.ErrorHandler;
+import org.eclipse.jetty.server.session.HashSessionManager;
+import org.eclipse.jetty.server.session.SessionHandler;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.util.resource.Resource;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.glassfish.jersey.servlet.ServletContainer;
+import org.joda.time.DateTime;
+
+import com.google.common.collect.ImmutableSet;
+import com.typesafe.config.Config;
+
+/**
+ * Wrapper around the Jetty web server.
+ * <p>
+ * Adapted from Drill's drill.exec.WebServer class. Would be good to create a
+ * common base class later, but the goal for the initial project is to avoid
+ * Drill code changes.
+ *
+ * @see <a href=
+ * "http://www.eclipse.org/jetty/documentation/current/embedding-jetty.html">
+ * Jetty Embedding documentation</a>
+ */
+
+public class WebServer implements AutoCloseable {
+ private static final Log LOG = LogFactory.getLog(WebServer.class);
+ private final Server jettyServer;
+ private Dispatcher dispatcher;
+
+ public WebServer(Dispatcher dispatcher) {
+ this.dispatcher = dispatcher;
+ Config config = DrillOnYarnConfig.config();
+ if (config.getBoolean(DrillOnYarnConfig.HTTP_ENABLED)) {
+ jettyServer = new Server();
+ } else {
+ jettyServer = null;
+ }
+ }
+
+ /**
+ * Start the web server including setup.
+ *
+ * @throws Exception
+ */
+ public void start() throws Exception {
+ if (jettyServer == null) {
+ return;
+ }
+
+ build();
+ jettyServer.start();
+ }
+
+ private void build() throws Exception {
+ Config config = DrillOnYarnConfig.config();
+ buildConnector(config);
+ buildServlets(config);
+ }
+
+ private void buildConnector(Config config) throws Exception {
+ final ServerConnector serverConnector;
+ if (config.getBoolean(DrillOnYarnConfig.HTTP_ENABLE_SSL)) {
+ serverConnector = createHttpsConnector(config);
+ } else {
+ serverConnector = createHttpConnector(config);
+ }
+ jettyServer.addConnector(serverConnector);
+ }
+
+ /**
+ * Build the web app with embedded servlets.
+ * <p>
+ * <b>ServletContextHandler</b>: is a Jetty-provided handler that add the
+ * extra bits needed to set up the context that servlets expect. Think of it
+ * as an adapter between the (simple) Jetty handler and the (more complex)
+ * servlet API.
+ *
+ */
+ private void buildServlets(Config config) {
+
+ final ServletContextHandler servletContextHandler = new ServletContextHandler(
+ null, "/");
+ servletContextHandler.setErrorHandler(createErrorHandler());
+ jettyServer.setHandler(servletContextHandler);
+
+ // Servlet holder for the pages of the Drill AM web app. The web app is a
+ // javax.ws application driven from annotations. The servlet holder "does
+ // the right thing" to drive the application, which is rooted at "/".
+ // The servlet container comes from Jersey, and manages the servlet
+ // lifecycle.
+
+ final ServletHolder servletHolder = new ServletHolder(
+ new ServletContainer(new WebUiPageTree(dispatcher)));
+ servletHolder.setInitOrder(1);
+ servletContextHandler.addServlet(servletHolder, "/*");
+
+ final ServletHolder restHolder = new ServletHolder(
+ new ServletContainer(new AmRestApi(dispatcher)));
+ restHolder.setInitOrder(2);
+ servletContextHandler.addServlet(restHolder, "/rest/*");
+
+ // Static resources (CSS, images, etc.)
+
+ setupStaticResources(servletContextHandler);
+
+ // Security, if requested.
+
+ if (AMSecurityManagerImpl.isEnabled()) {
+ servletContextHandler.setSecurityHandler(createSecurityHandler(config));
+ servletContextHandler.setSessionHandler(createSessionHandler(config,
+ servletContextHandler.getSecurityHandler()));
+ }
+ }
+
+ private ErrorHandler createErrorHandler() {
+ // Error handler to show detailed errors.
+ // Should probably be turned off in production.
+ final ErrorHandler errorHandler = new ErrorHandler();
+ errorHandler.setShowStacks(true);
+ errorHandler.setShowMessageInTitle(true);
+ return errorHandler;
+ }
+
+ private void setupStaticResources(
+ ServletContextHandler servletContextHandler) {
+
+ // Access to static resources (JS pages, images, etc.)
+ // The static resources themselves come from Drill exec sub-project
+ // and the Drill-on-YARN project.
+ //
+ // We handle static content this way because we want to do it
+ // in the context of a servlet app, so we need the Jetty "default servlet"
+ // that handles static content. That servlet is designed to take its
+ // properties
+ // from the web.xml, file; but can also take them programmatically as done
+ // here. (The Jetty manual suggests a simpler handler, but that is a
+ // non-Servlet
+ // version.)
+
+ final ServletHolder staticHolder = new ServletHolder("static",
+ DefaultServlet.class);
+ staticHolder.setInitParameter("resourceBase",
+ Resource.newClassPathResource("/rest/static").toString());
+ staticHolder.setInitParameter("dirAllowed", "false");
+ staticHolder.setInitParameter("pathInfoOnly", "true");
+ servletContextHandler.addServlet(staticHolder, "/static/*");
+
+ final ServletHolder amStaticHolder = new ServletHolder("am-static",
+ DefaultServlet.class);
+ amStaticHolder.setInitParameter("resourceBase",
+ Resource.newClassPathResource("/drill-am/static").toString());
+ amStaticHolder.setInitParameter("dirAllowed", "false");
+ amStaticHolder.setInitParameter("pathInfoOnly", "true");
+ servletContextHandler.addServlet(amStaticHolder, "/drill-am/static/*");
+ }
+
+ public static class AMUserPrincipal implements Principal {
+ public final String userName;
+
+ public AMUserPrincipal(String userName) {
+ this.userName = userName;
+ }
+
+ @Override
+ public String getName() {
+ return userName;
+ }
+ }
+
+ public static class AmLoginService implements LoginService {
+ private AMSecurityManager securityMgr;
+ protected IdentityService identityService = new DefaultIdentityService();
+
+ public AmLoginService(AMSecurityManager securityMgr) {
+ this.securityMgr = securityMgr;
+ }
+
+ @Override
+ public String getName() {
+ return "drill-am";
+ }
+
+ @Override
+ public UserIdentity login(String username, Object credentials) {
+ if (!securityMgr.login(username, (String) credentials)) {
+ return null;
+ }
+ return new DefaultUserIdentity(null, new AMUserPrincipal(username),
+ new String[] { ADMIN_ROLE });
+ }
+
+ @Override
+ public boolean validate(UserIdentity user) {
+ return true;
+ }
+
+ @Override
+ public IdentityService getIdentityService() {
+ return identityService;
+ }
+
+ @Override
+ public void setIdentityService(IdentityService service) {
+ this.identityService = service;
+ }
+
+ @Override
+ public void logout(UserIdentity user) {
+ }
+
+ // @Override
+ // protected UserIdentity loadUser(String username) {
+ // // TODO Auto-generated method stub
+ // return null;
+ // }
+ //
+ // @Override
+ // protected void loadUsers() throws IOException {
+ // putUser( "fred", new Password( "wilma" ), new String[] { ADMIN_ROLE } );
+ // }
+
+ }
+
+ /**
+ * @return
+ * @return
+ * @see http://www.eclipse.org/jetty/documentation/current/embedded-examples.html
+ */
+
+ private ConstraintSecurityHandler createSecurityHandler(Config config) {
+ ConstraintSecurityHandler security = new ConstraintSecurityHandler();
+
+ Set<String> knownRoles = ImmutableSet.of(ADMIN_ROLE);
+ security.setConstraintMappings(Collections.<ConstraintMapping> emptyList(),
+ knownRoles);
+
+ security.setAuthenticator(new FormAuthenticator("/login", "/login", true));
+ security
+ .setLoginService(new AmLoginService(AMSecurityManagerImpl.instance()));
+
+ return security;
+ }
+
+ /**
+ * @return A {@link SessionHandler} which contains a
+ * {@link HashSessionManager}
+ */
+ private SessionHandler createSessionHandler(Config config,
+ final SecurityHandler securityHandler) {
+ SessionManager sessionManager = new HashSessionManager();
+ sessionManager.setMaxInactiveInterval(
+ config.getInt(DrillOnYarnConfig.HTTP_SESSION_MAX_IDLE_SECS));
+ sessionManager.addEventListener(new HttpSessionListener() {
+ @Override
+ public void sessionCreated(HttpSessionEvent se) {
+ // No-op
+ }
+
+ @Override
+ public void sessionDestroyed(HttpSessionEvent se) {
+ final HttpSession session = se.getSession();
+ if (session == null) {
+ return;
+ }
+
+ final Object authCreds = session
+ .getAttribute(SessionAuthentication.__J_AUTHENTICATED);
+ if (authCreds != null) {
+ final SessionAuthentication sessionAuth = (SessionAuthentication) authCreds;
+ securityHandler.logout(sessionAuth);
+ session.removeAttribute(SessionAuthentication.__J_AUTHENTICATED);
+ }
+ }
+ });
+
+ return new SessionHandler(sessionManager);
+ }
+
+ /**
+ * Create HTTP connector.
+ *
+ * @return Initialized {@link ServerConnector} instance for HTTP connections.
+ * @throws Exception
+ */
+ private ServerConnector createHttpConnector(Config config) throws Exception {
+ LOG.info("Setting up HTTP connector for web server");
+ final HttpConfiguration httpConfig = new HttpConfiguration();
+ final ServerConnector httpConnector = new ServerConnector(jettyServer,
+ new HttpConnectionFactory(httpConfig));
+ httpConnector.setPort(config.getInt(DrillOnYarnConfig.HTTP_PORT));
+
+ return httpConnector;
+ }
+
+ /**
+ * Create an HTTPS connector for given jetty server instance. If the admin has
+ * specified keystore/truststore settings they will be used else a self-signed
+ * certificate is generated and used.
+ * <p>
+ * This is a shameless copy of
+ * {@link org.apache.drill.exec.server.rest.Webserver#createHttpsConnector( )}.
+ * The two should be merged at some point. The primary issue is that the Drill
+ * version is tightly coupled to Drillbit configuration.
+ *
+ * @return Initialized {@link ServerConnector} for HTTPS connections.
+ * @throws Exception
+ */
+
+ private ServerConnector createHttpsConnector(Config config) throws Exception {
+ LOG.info("Setting up HTTPS connector for web server");
+
+ final SslContextFactory sslContextFactory = new SslContextFactory();
+
+ // if (config.hasPath(ExecConstants.HTTP_KEYSTORE_PATH) &&
+ // !Strings.isNullOrEmpty(config.getString(ExecConstants.HTTP_KEYSTORE_PATH)))
+ // {
+ // LOG.info("Using configured SSL settings for web server");
+ // sslContextFactory.setKeyStorePath(config.getString(ExecConstants.HTTP_KEYSTORE_PATH));
+ // sslContextFactory.setKeyStorePassword(config.getString(ExecConstants.HTTP_KEYSTORE_PASSWORD));
+ //
+ // // TrustStore and TrustStore password are optional
+ // if (config.hasPath(ExecConstants.HTTP_TRUSTSTORE_PATH)) {
+ // sslContextFactory.setTrustStorePath(config.getString(ExecConstants.HTTP_TRUSTSTORE_PATH));
+ // if (config.hasPath(ExecConstants.HTTP_TRUSTSTORE_PASSWORD)) {
+ // sslContextFactory.setTrustStorePassword(config.getString(ExecConstants.HTTP_TRUSTSTORE_PASSWORD));
+ // }
+ // }
+ // } else {
+ LOG.info("Using generated self-signed SSL settings for web server");
+ final SecureRandom random = new SecureRandom();
+
+ // Generate a private-public key pair
+ final KeyPairGenerator keyPairGenerator = KeyPairGenerator
+ .getInstance("RSA");
+ keyPairGenerator.initialize(1024, random);
+ final KeyPair keyPair = keyPairGenerator.generateKeyPair();
+
+ final DateTime now = DateTime.now();
+
+ // Create builder for certificate attributes
+ final X500NameBuilder nameBuilder = new X500NameBuilder(BCStyle.INSTANCE)
+ .addRDN(BCStyle.OU, "Apache Drill (auth-generated)")
+ .addRDN(BCStyle.O, "Apache Software Foundation (auto-generated)")
+ .addRDN(BCStyle.CN, "Drill AM");
+
+ final Date notBefore = now.minusMinutes(1).toDate();
+ final Date notAfter = now.plusYears(5).toDate();
+ final BigInteger serialNumber = new BigInteger(128, random);
+
+ // Create a certificate valid for 5years from now.
+ final X509v3CertificateBuilder certificateBuilder = new JcaX509v3CertificateBuilder(
+ nameBuilder.build(), // attributes
+ serialNumber, notBefore, notAfter, nameBuilder.build(),
+ keyPair.getPublic());
+
+ // Sign the certificate using the private key
+ final ContentSigner contentSigner = new JcaContentSignerBuilder(
+ "SHA256WithRSAEncryption").build(keyPair.getPrivate());
+ final X509Certificate certificate = new JcaX509CertificateConverter()
+ .getCertificate(certificateBuilder.build(contentSigner));
+
+ // Check the validity
+ certificate.checkValidity(now.toDate());
+
+ // Make sure the certificate is self-signed.
+ certificate.verify(certificate.getPublicKey());
+
+ // Generate a random password for keystore protection
+ final String keyStorePasswd = RandomStringUtils.random(20);
+ final KeyStore keyStore = KeyStore.getInstance("JKS");
+ keyStore.load(null, null);
+ keyStore.setKeyEntry("DrillAutoGeneratedCert", keyPair.getPrivate(),
+ keyStorePasswd.toCharArray(),
+ new java.security.cert.Certificate[] { certificate });
+
+ sslContextFactory.setKeyStore(keyStore);
+ sslContextFactory.setKeyStorePassword(keyStorePasswd);
+ // }
+
+ final HttpConfiguration httpsConfig = new HttpConfiguration();
+ httpsConfig.addCustomizer(new SecureRequestCustomizer());
+
+ // SSL Connector
+ final ServerConnector sslConnector = new ServerConnector(jettyServer,
+ new SslConnectionFactory(sslContextFactory,
+ HttpVersion.HTTP_1_1.asString()),
+ new HttpConnectionFactory(httpsConfig));
+ sslConnector.setPort(config.getInt(DrillOnYarnConfig.HTTP_PORT));
+
+ return sslConnector;
+ }
+
+ @Override
+ public void close() throws Exception {
+ if (jettyServer != null) {
+ jettyServer.stop();
+ }
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/WebUiPageTree.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/WebUiPageTree.java
new file mode 100644
index 000000000..fc44e4577
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/WebUiPageTree.java
@@ -0,0 +1,527 @@
+/*
+ * 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.drill.yarn.appMaster.http;
+
+import static org.apache.drill.exec.server.rest.auth.DrillUserPrincipal.ADMIN_ROLE;
+
+import java.net.URI;
+import java.net.URLDecoder;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.annotation.security.PermitAll;
+import javax.annotation.security.RolesAllowed;
+import javax.inject.Inject;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.HttpSession;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.FormParam;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.SecurityContext;
+import javax.ws.rs.core.UriBuilder;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.drill.yarn.appMaster.Dispatcher;
+import org.apache.drill.yarn.core.DoYUtil;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.eclipse.jetty.security.authentication.FormAuthenticator;
+import org.glassfish.jersey.server.filter.RolesAllowedDynamicFeature;
+import org.glassfish.jersey.server.mvc.Viewable;
+import org.glassfish.jersey.server.mvc.freemarker.FreemarkerMvcFeature;
+
+/**
+ * The Drill AM web UI. The format is highly compact. We use javax.ws.rs to mark
+ * up a Pojo with page path, permissions and HTTP methods. The ADMIN_ROLE is
+ * reused from Drill's web UI.
+ * <p>
+ * In general, all pages require admin role, except for two: the login page and
+ * the redirect page which the YARN web UI follows to start the AM UI.
+ */
+
+public class WebUiPageTree extends PageTree {
+
+ /**
+ * Main DoY page that displays cluster status, and the status of
+ * the resource groups. Available only to the admin user when
+ * DoY is secured.
+ */
+
+ @Path("/")
+ @RolesAllowed(ADMIN_ROLE)
+ public static class RootPage {
+ @Inject
+ SecurityContext sc;
+
+ @GET
+ @Produces(MediaType.TEXT_HTML)
+ public Viewable getRoot() {
+ ControllerModel model = new ControllerModel();
+ dispatcher.getController().visit(model);
+ model.countStrayDrillbits(dispatcher.getController());
+ return new Viewable("/drill-am/index.ftl", toModel(sc, model));
+ }
+ }
+
+ /**
+ * Pages, adapted from Drill, that display the login and logout pages.
+ * Login uses the security mechanism, again borrowed from Drill, to
+ * validate the user against either the simple user/password
+ * configured in DoY, or the user who launched DoY using the
+ * Drill security mechanism.
+ */
+
+ @Path("/")
+ @PermitAll
+ public static class LogInLogOutPages {
+ @Inject
+ SecurityContext sc;
+
+ public static final String REDIRECT_QUERY_PARM = "redirect";
+ public static final String LOGIN_RESOURCE = "login";
+
+ @GET
+ @Path("/login")
+ @Produces(MediaType.TEXT_HTML)
+ public Viewable getLoginPage(@Context HttpServletRequest request,
+ @Context HttpServletResponse response, @Context SecurityContext sc,
+ @Context UriInfo uriInfo,
+ @QueryParam(REDIRECT_QUERY_PARM) String redirect) throws Exception {
+
+ if (!StringUtils.isEmpty(redirect)) {
+ // If the URL has redirect in it, set the redirect URI in session, so
+ // that after the login is successful, request
+ // is forwarded to the redirect page.
+ final HttpSession session = request.getSession(true);
+ final URI destURI = UriBuilder
+ .fromUri(URLDecoder.decode(redirect, "UTF-8")).build();
+ session.setAttribute(FormAuthenticator.__J_URI, destURI.toString());
+ }
+
+ return new Viewable("/drill-am/login.ftl", toModel(sc, (Object) null));
+ }
+
+ // Request type is POST because POST request which contains the login
+ // credentials are invalid and the request is
+ // dispatched here directly.
+ @POST
+ @Path("/login")
+ @Produces(MediaType.TEXT_HTML)
+ public Viewable getLoginPageAfterValidationError() {
+ return new Viewable("/drill-am/login.ftl",
+ toModel(sc, "Invalid user name or password."));
+ }
+
+ @GET
+ @Path("/logout")
+ public Viewable logout(@Context HttpServletRequest req,
+ @Context HttpServletResponse resp) throws Exception {
+ final HttpSession session = req.getSession();
+ if (session != null) {
+ session.invalidate();
+ }
+
+ req.getRequestDispatcher("/login").forward(req, resp);
+ return null;
+ }
+ }
+
+ /**
+ * DoY provides a link to YARN to display the AM UI. YARN wants to display the
+ * linked page in a frame, which does not play well with the DoY UI. To avoid
+ * this, we give YARN a link to this redirect page which does nothing other
+ * than to redirect the browser to the (full) DoY main UI.
+ */
+
+ @Path("/redirect")
+ @PermitAll
+ public static class RedirectPage {
+ @GET
+ @Produces(MediaType.TEXT_HTML)
+ public Viewable getRoot() {
+ Map<String, String> map = new HashMap<>();
+ String baseUrl = DoYUtil.unwrapAmUrl(dispatcher.getTrackingUrl());
+ map.put("amLink", baseUrl);
+ map.put("clusterName", config.getString(DrillOnYarnConfig.APP_NAME));
+ return new Viewable("/drill-am/redirect.ftl", map);
+ }
+ }
+
+ /**
+ * Display the configuration page which displays the contents of
+ * DoY and selected Drill config as name/value pairs. Visible only
+ * to the admin when DoY is secure.
+ */
+
+ @Path("/config")
+ @RolesAllowed(ADMIN_ROLE)
+ public static class ConfigPage {
+ @Inject
+ private SecurityContext sc;
+
+ @GET
+ @Produces(MediaType.TEXT_HTML)
+ public Viewable getRoot() {
+ return new Viewable("/drill-am/config.ftl",
+ toModel(sc, DrillOnYarnConfig.instance().getPairs()));
+ }
+ }
+
+ /**
+ * Displays the list of Drillbits showing details for each Drillbit.
+ * (DoY uses the generic term "task", but, at present, the only
+ * task that DoY runs is a Drillbit.
+ */
+
+ @Path("/drillbits")
+ @RolesAllowed(ADMIN_ROLE)
+ public static class DrillbitsPage {
+ @Inject
+ private SecurityContext sc;
+
+ @GET
+ @Produces(MediaType.TEXT_HTML)
+ public Viewable getRoot() {
+ AbstractTasksModel.TasksModel model = new AbstractTasksModel.TasksModel();
+ dispatcher.getController().visitTasks(model);
+ model.listAnomalies(dispatcher.getController());
+ model.sortTasks();
+
+ // Done this funky way because FreeMarker only understands lists if they
+ // are members of a hash (grumble, grumble...)
+
+ Map<String, Object> map = new HashMap<>();
+ map.put("model", model);
+ map.put("tasks", model.getTasks());
+ if (model.hasUnmanagedDrillbits()) {
+ map.put("strays", model.getUnnamaged());
+ }
+ if (model.hasBlacklist()) {
+ map.put("blacklist", model.getBlacklist());
+ }
+ map.put("showDisks", dispatcher.getController().supportsDiskResource());
+ map.put("refreshSecs", DrillOnYarnConfig.config()
+ .getInt(DrillOnYarnConfig.HTTP_REFRESH_SECS));
+ return new Viewable("/drill-am/tasks.ftl", toMapModel(sc, map));
+ }
+ }
+
+ /**
+ * Displays a warning page to ask the user if they want to cancel
+ * a Drillbit. This is a bit old-school; we display this as a
+ * separate page. A good future enhancement is to do this as
+ * a pop-up in Javascript. The GET request display the confirmation
+ * page, the PUT request confirms cancellation and does the deed.
+ * The task to be cancelled appears as a query parameter:
+ * <pre>.../cancel?id=&lt;task id></pre>
+ */
+
+ @Path("/cancel/")
+ @RolesAllowed(ADMIN_ROLE)
+ public static class CancelDrillbitPage {
+ @Inject
+ private SecurityContext sc;
+
+ @QueryParam("id")
+ private int id;
+
+ @GET
+ @Produces(MediaType.TEXT_HTML)
+ public Viewable getPage() {
+ ConfirmShrink confirm;
+ if (dispatcher.getController().isTaskLive(id)) {
+ confirm = new ConfirmShrink(ConfirmShrink.Mode.KILL);
+ } else {
+ confirm = new ConfirmShrink(ConfirmShrink.Mode.CANCEL);
+ }
+ confirm.id = id;
+ return new Viewable("/drill-am/shrink-warning.ftl", toModel(sc, confirm));
+ }
+
+ @POST
+ @Produces(MediaType.TEXT_HTML)
+ public Viewable postPage() {
+ Acknowledge ack;
+ if (dispatcher.getController().cancelTask(id)) {
+ ack = new Acknowledge(Acknowledge.Mode.CANCELLED);
+ } else {
+ ack = new Acknowledge(Acknowledge.Mode.INVALID_TASK);
+ }
+ ack.value = id;
+ return new Viewable("/drill-am/confirm.ftl", toModel(sc, ack));
+ }
+ }
+
+ /**
+ * Displays a history of completed tasks which indicates failed or cancelled
+ * Drillbits. Helps the admin to understand what has been happening on the
+ * cluster if Drillbits have died.
+ */
+
+ @Path("/history")
+ @RolesAllowed(ADMIN_ROLE)
+ public static class HistoryPage {
+ @Inject
+ SecurityContext sc;
+
+ @GET
+ @Produces(MediaType.TEXT_HTML)
+ public Viewable getRoot() {
+ AbstractTasksModel.HistoryModel model = new AbstractTasksModel.HistoryModel();
+ dispatcher.getController().visit(model);
+ Map<String, Object> map = new HashMap<>();
+ map.put("model", model.results);
+ map.put("refreshSecs", DrillOnYarnConfig.config()
+ .getInt(DrillOnYarnConfig.HTTP_REFRESH_SECS));
+ return new Viewable("/drill-am/history.ftl", toMapModel(sc, map));
+ }
+ }
+
+ /**
+ * Page that lets the admin change the cluster size or shut down the cluster.
+ */
+
+ @Path("/manage")
+ @RolesAllowed(ADMIN_ROLE)
+ public static class ManagePage {
+ @Inject
+ SecurityContext sc;
+
+ @GET
+ @Produces(MediaType.TEXT_HTML)
+ public Viewable getRoot() {
+ ControllerModel model = new ControllerModel();
+ dispatcher.getController().visit(model);
+ return new Viewable("/drill-am/manage.ftl", toModel(sc, model));
+ }
+ }
+
+ /**
+ * Passes information to the acknowledgement page.
+ */
+
+ public static class Acknowledge {
+ public enum Mode {
+ STOPPED, INVALID_RESIZE, INVALID_ACTION, NULL_RESIZE, RESIZED, CANCELLED, INVALID_TASK
+ };
+
+ Mode mode;
+ Object value;
+
+ public Acknowledge(Mode mode) {
+ this.mode = mode;
+ }
+
+ public String getType() {
+ return mode.toString();
+ }
+
+ public Object getValue() {
+ return value;
+ }
+ }
+
+ /**
+ * Passes information to the confirmation page.
+ */
+
+ public static class ConfirmShrink {
+ public enum Mode {
+ SHRINK, STOP, CANCEL, KILL
+ };
+
+ Mode mode;
+ int value;
+ int id;
+
+ public ConfirmShrink(Mode mode) {
+ this.mode = mode;
+ }
+
+ public boolean isStop() {
+ return mode == Mode.STOP;
+ }
+
+ public boolean isCancel() {
+ return mode == Mode.CANCEL;
+ }
+
+ public boolean isKill() {
+ return mode == Mode.KILL;
+ }
+
+ public boolean isShrink() {
+ return mode == Mode.SHRINK;
+ }
+
+ public int getCount() {
+ return value;
+ }
+
+ public int getId() {
+ return id;
+ }
+ }
+
+ /**
+ * Confirm that the user wants to resize the cluster. Displays a warning if
+ * the user wants to shrink the cluster, since, at present, doing so will
+ * kill any in-flight queries. The GET request display the warning,
+ * the POST request confirms the action. The action itself is provided
+ * as query parameters:
+ * <pre>.../resize?type=&lt;type>&n=&lt;quantity></pre>
+ * Where the type is one of "resize", "grow", "shrink" or
+ * "force-shrink" and n is the associated quantity.
+ * <p>
+ * Note that the manage page only provides the "resize" option; the
+ * grow and shrink options were removed from the Web UI and are only
+ * visible through the REST API.
+ */
+
+ @Path("/resize")
+ @RolesAllowed(ADMIN_ROLE)
+ public static class ResizePage {
+ @Inject
+ SecurityContext sc;
+
+ @FormParam("n")
+ int n;
+ @FormParam("type")
+ String type;
+
+ @POST
+ @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
+ @Produces(MediaType.TEXT_HTML)
+ public Viewable resize() {
+ int curSize = dispatcher.getController().getTargetCount();
+ if (n <= 0) {
+ Acknowledge confirm = new Acknowledge(Acknowledge.Mode.INVALID_RESIZE);
+ confirm.value = n;
+ return new Viewable("/drill-am/confirm.ftl", toModel(sc, confirm));
+ }
+ if (type == null) {
+ type = "null";
+ }
+ int newSize;
+ boolean confirmed = false;
+ if (type.equalsIgnoreCase("resize")) {
+ newSize = n;
+ } else if (type.equalsIgnoreCase("grow")) {
+ newSize = curSize + n;
+ } else if (type.equalsIgnoreCase("shrink")) {
+ newSize = curSize - n;
+ } else if (type.equalsIgnoreCase("force-shrink")) {
+ newSize = curSize - n;
+ confirmed = true;
+ } else {
+ Acknowledge confirm = new Acknowledge(Acknowledge.Mode.INVALID_ACTION);
+ confirm.value = type;
+ return new Viewable("/drill-am/confirm.ftl", toModel(sc, confirm));
+ }
+
+ if (curSize == newSize) {
+ Acknowledge confirm = new Acknowledge(Acknowledge.Mode.NULL_RESIZE);
+ confirm.value = newSize;
+ return new Viewable("/drill-am/confirm.ftl", toModel(sc, confirm));
+ } else if (confirmed || curSize < newSize) {
+ Acknowledge confirm = new Acknowledge(Acknowledge.Mode.RESIZED);
+ confirm.value = dispatcher.getController().resizeTo(newSize);
+ return new Viewable("/drill-am/confirm.ftl", toModel(sc, confirm));
+ } else {
+ ConfirmShrink confirm = new ConfirmShrink(ConfirmShrink.Mode.SHRINK);
+ confirm.value = curSize - newSize;
+ return new Viewable("/drill-am/shrink-warning.ftl",
+ toModel(sc, confirm));
+ }
+ }
+ }
+
+ /**
+ * Confirmation page when the admin asks to stop the cluster.
+ * The GET request displays the confirmation, the POST does
+ * the deed. As for other confirmation pages, this is an old-style,
+ * quick & dirty solution. A more modern solution would be to use JavaScript
+ * to pop up a confirmation dialog.
+ */
+
+ @Path("/stop/")
+ @RolesAllowed(ADMIN_ROLE)
+ public static class StopPage {
+ @Inject
+ SecurityContext sc;
+
+ @GET
+ @Produces(MediaType.TEXT_HTML)
+ public Viewable requestStop() {
+ ConfirmShrink confirm = new ConfirmShrink(ConfirmShrink.Mode.STOP);
+ return new Viewable("/drill-am/shrink-warning.ftl", toModel(sc, confirm));
+ }
+
+ @POST
+ @Produces(MediaType.TEXT_HTML)
+ public Viewable doStop() {
+ dispatcher.getController().shutDown();
+ Acknowledge confirm = new Acknowledge(Acknowledge.Mode.STOPPED);
+ return new Viewable("/drill-am/confirm.ftl", toModel(sc, confirm));
+ }
+ }
+
+ /**
+ * Build the pages for the Web UI using Freemarker to implement the
+ * MVC mechanism. This class builds on a rather complex mechanism; understand
+ * that to understand what the lines of code below are doing.
+ *
+ * @param dispatcher the DoY AM dispatcher that receives requests for
+ * information about, or requests to change the state of, the Drill clutser
+ */
+
+ public WebUiPageTree(Dispatcher dispatcher) {
+ super(dispatcher);
+
+ // Markup engine
+ register(FreemarkerMvcFeature.class);
+
+ // Web UI Pages
+ register(RootPage.class);
+ register(RedirectPage.class);
+ register(ConfigPage.class);
+ register(DrillbitsPage.class);
+ register(CancelDrillbitPage.class);
+ register(HistoryPage.class);
+ register(ManagePage.class);
+ register(ResizePage.class);
+ register(StopPage.class);
+
+ // Authorization
+ // See: https://jersey.java.net/documentation/latest/security.html
+
+ if (AMSecurityManagerImpl.isEnabled()) {
+ register(LogInLogOutPages.class);
+ register(AuthDynamicFeature.class);
+ register(RolesAllowedDynamicFeature.class);
+ }
+ }
+
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/package-info.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/package-info.java
new file mode 100644
index 000000000..13f1bd831
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/package-info.java
@@ -0,0 +1,22 @@
+/**
+ * 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.drill.yarn.appMaster.http; \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/package-info.java b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/package-info.java
new file mode 100644
index 000000000..0ff835d8b
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/package-info.java
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+/**
+ * Implements the Drill Application Master for YARN.
+ * <p>
+ * Note that AM implementation classes use org.apache.commons.logging
+ * to be consistent with the logging used within YARN itself. However,
+ * the AM uses Drill's class path which uses logback logging. To enable
+ * logging, modify
+ * <code>$DRILL_HOME/conf/logback.xml</code> and add a section something
+ * like this:
+ * <pre><code>
+ * &lt;logger name="org.apache.drill.yarn" additivity="false">
+ * &lt;level value="trace" />
+ * &lt;appender-ref ref="STDOUT" />
+ * &lt;/logger>
+ * </code></pre>
+ */
+
+package org.apache.drill.yarn.appMaster; \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/AMRunner.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/AMRunner.java
new file mode 100644
index 000000000..5252b8822
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/AMRunner.java
@@ -0,0 +1,368 @@
+/*
+ * 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.drill.yarn.client;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.util.Map;
+
+import org.apache.drill.yarn.core.AppSpec;
+import org.apache.drill.yarn.core.DoYUtil;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.drill.yarn.core.LaunchSpec;
+import org.apache.drill.yarn.core.YarnClientException;
+import org.apache.drill.yarn.core.YarnRMClient;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+
+import com.typesafe.config.Config;
+
+/**
+ * Launch the AM through YARN. Builds the launch description, then tracks
+ * the launch operation itself. Finally, provides the user with links to
+ * track the AM both through YARN and via the AM's own web UI.
+ */
+
+public class AMRunner {
+ private Config config;
+ private boolean verbose;
+ private ApplicationId appId;
+ public Map<String, LocalResource> resources;
+ public String drillArchivePath;
+ public String siteArchivePath;
+ public String remoteDrillHome;
+ public String remoteSiteDir;
+ private YarnRMClient client;
+ private GetNewApplicationResponse appResponse;
+ private boolean dryRun;
+
+ public AMRunner(Config config, boolean verbose, boolean dryRun) {
+ this.config = config;
+ this.verbose = verbose;
+ this.dryRun = dryRun;
+ }
+
+ public void run() throws ClientException {
+ connectToYarn();
+ if (dryRun) {
+ doDryRun();
+ } else {
+ doLaunch();
+ }
+ }
+
+ private void connectToYarn() {
+ System.out.print("Loading YARN Config...");
+ client = new YarnRMClient();
+ System.out.println(" Loaded.");
+ }
+
+ private void doDryRun() throws ClientException {
+ AppSpec master = buildSpec();
+ dump(master, System.out);
+ }
+
+ private void doLaunch() throws ClientException {
+ createApp();
+ AppSpec master = buildSpec();
+ if (verbose) {
+ dump(master, System.out);
+ }
+ validateResources(master);
+ launchApp(master);
+ waitForStartAndReport(master.appName);
+ writeAppIdFile();
+ }
+
+ private void dump(AppSpec master, PrintStream out) {
+ out.println("----------------------------------------------");
+ out.println("Application Master Launch Spec");
+ master.dump(out);
+ out.println("----------------------------------------------");
+ }
+
+ private AppSpec buildSpec() throws ClientException {
+ AppSpec master = new AppSpec();
+
+ // Heap memory
+
+ String heapMem = config.getString( DrillOnYarnConfig.AM_HEAP );
+ master.env.put( DrillOnYarnConfig.AM_HEAP_ENV_VAR, heapMem );
+
+ // Any additional VM arguments from the config file.
+
+ addIfSet( master, DrillOnYarnConfig.AM_VM_ARGS, DrillOnYarnConfig.AM_JAVA_OPTS_ENV_VAR );
+
+ // Any user specified override jars
+ // Not really needed by the AM.
+
+ addIfSet( master, DrillOnYarnConfig.AM_PREFIX_CLASSPATH, DrillOnYarnConfig.DRILL_CLASSPATH_PREFIX_ENV_VAR );
+
+ // Any user specified classpath.
+
+ addIfSet( master, DrillOnYarnConfig.AM_CLASSPATH, DrillOnYarnConfig.DRILL_CLASSPATH_ENV_VAR );
+
+ // Any user-specified library path
+
+ addIfSet( master, DrillOnYarnConfig.JAVA_LIB_PATH, DrillOnYarnConfig.DOY_LIBPATH_ENV_VAR );
+
+ // AM logs (of which there are none.
+ // Relies on the LOG_DIR_EXPANSION_VAR marker which is replaced by
+ // the container log directory.
+ // Must be set for the AM to prevent drill-config.sh from trying to create
+ // the log directory in $DRILL_HOME (which won't be writable under YARN.)
+
+ if (!config.getBoolean(DrillOnYarnConfig.DISABLE_YARN_LOGS)) {
+ master.env.put("DRILL_YARN_LOG_DIR",
+ ApplicationConstants.LOG_DIR_EXPANSION_VAR);
+ }
+
+ // AM launch script
+ // The drill home location is either a non-localized location,
+ // or, more typically, the expanded Drill directory under the
+ // container's working directory. When the localized directory,
+ // we rely on the fact that the current working directory is
+ // set to the container directory, so we just need the name
+ // of the Drill folder under the cwd.
+
+ master.command = remoteDrillHome + "/bin/drill-am.sh";
+
+ // If site dir, add that as an argument.
+
+ if ( remoteSiteDir != null ) {
+ master.cmdArgs.add( "--site" );
+ master.cmdArgs.add( remoteSiteDir );
+ }
+
+ // Strangely, YARN has no way to tell an AM what its app ID
+ // is. So, we pass it along here.
+
+ String appIdStr = dryRun ? "Unknown" : appId.toString();
+ master.env.put( DrillOnYarnConfig.APP_ID_ENV_VAR, appIdStr );
+
+ // Debug launch: dumps environment variables and other information
+ // in the launch script.
+
+ if ( config.getBoolean( DrillOnYarnConfig.AM_DEBUG_LAUNCH ) ) {
+ master.env.put( DrillOnYarnConfig.DRILL_DEBUG_ENV_VAR, "1" );
+ }
+
+ // If localized, add the drill and optionally site archive.
+
+ if ( config.getBoolean( DrillOnYarnConfig.LOCALIZE_DRILL) ) {
+
+ // Also, YARN has no way to tell an AM what localized resources are
+ // available, so we pass them along as environment variables.
+
+ master.env.put( DrillOnYarnConfig.DRILL_ARCHIVE_ENV_VAR, drillArchivePath );
+ if ( siteArchivePath != null ) {
+ master.env.put( DrillOnYarnConfig.SITE_ARCHIVE_ENV_VAR, siteArchivePath );
+ }
+ }
+
+ // Localized resources
+
+ master.resources.putAll( resources );
+
+ // Container specification.
+
+ master.memoryMb = config.getInt( DrillOnYarnConfig.AM_MEMORY );
+ master.vCores = config.getInt( DrillOnYarnConfig.AM_VCORES );
+ master.disks = config.getDouble( DrillOnYarnConfig.AM_DISKS );
+ master.appName = config.getString( DrillOnYarnConfig.APP_NAME );
+ master.queueName = config.getString( DrillOnYarnConfig.YARN_QUEUE );
+ master.priority = config.getInt( DrillOnYarnConfig.YARN_PRIORITY );
+ master.nodeLabelExpr = config.getString( DrillOnYarnConfig.AM_NODE_LABEL_EXPR );
+ return master;
+ }
+
+ private void addIfSet(LaunchSpec spec, String configParam, String envVar) {
+ String value = config.getString(configParam);
+ if (!DoYUtil.isBlank(value)) {
+ spec.env.put(envVar, value);
+ }
+ }
+
+ private void createApp() throws ClientException {
+ try {
+ appResponse = client.createAppMaster();
+ } catch (YarnClientException e) {
+ throw new ClientException("Failed to allocate Drill application master",
+ e);
+ }
+ appId = appResponse.getApplicationId();
+ System.out.println("Application ID: " + appId.toString());
+ }
+
+ private void validateResources( AppSpec master ) throws ClientException {
+
+ // Memory and core checks per YARN app specs.
+
+ int maxMemory = appResponse.getMaximumResourceCapability().getMemory();
+ int maxCores = appResponse.getMaximumResourceCapability().getVirtualCores();
+ if (verbose) {
+ System.out.println("Max Memory: " + maxMemory);
+ System.out.println("Max Cores: " + maxCores);
+ }
+
+ // YARN behaves very badly if we request a container larger than the
+ // maximum.
+
+ if (master.memoryMb > maxMemory) {
+ throw new ClientException( "YARN maximum memory is " + maxMemory
+ + " but the application master requests " + master.memoryMb );
+ }
+ if (master.vCores > maxCores) {
+ throw new ClientException("YARN maximum vcores is " + maxCores
+ + " but the application master requests " + master.vCores);
+ }
+
+ // Verify the limits for the Drillbit as well.
+
+ if (config.getInt(DrillOnYarnConfig.DRILLBIT_MEMORY) > maxMemory) {
+ throw new ClientException(
+ "YARN maximum memory is " + maxMemory + " but the Drillbit requests "
+ + config.getInt(DrillOnYarnConfig.DRILLBIT_MEMORY));
+ }
+ if (config.getInt(DrillOnYarnConfig.DRILLBIT_VCORES) > maxCores) {
+ throw new ClientException("YARN maximum vcores is " + maxCores
+ + " but the Drillbit requests "
+ + config.getInt(DrillOnYarnConfig.DRILLBIT_VCORES));
+ }
+ }
+
+ private void launchApp(AppSpec master) throws ClientException {
+ try {
+ client.submitAppMaster(master);
+ } catch (YarnClientException e) {
+ throw new ClientException("Failed to start Drill application master", e);
+ }
+ }
+
+ /**
+ * Write the app id file needed for subsequent commands. The app id file is
+ * the only way we know the YARN application associated with our Drill-on-YARN
+ * session. This file is ready by subsequent status, resize and stop commands
+ * so we can find our Drill AM on the YARN cluster.
+ *
+ * @throws ClientException
+ */
+
+ private void writeAppIdFile() throws ClientException {
+ // Write the appid file that lets us work with the app later
+ // (Analogous to a pid file.)
+ // File goes into the directory above Drill Home (which should be the
+ // folder that contains the localized archive) and is named for the
+ // ZK cluster (to ensure that the name is a valid file name.)
+
+ File appIdFile = ClientCommand.getAppIdFile();
+ try {
+ PrintWriter writer = new PrintWriter(new FileWriter(appIdFile));
+ writer.println(appId);
+ writer.close();
+ } catch (IOException e) {
+ throw new ClientException(
+ "Failed to write appid file: " + appIdFile.getAbsolutePath());
+ }
+ }
+
+ /**
+ * Poll YARN to track the launch process of the application so that we can
+ * wait until the AM is live before pointing the user to the AM's web UI.
+ */
+
+ private class StartMonitor {
+ StatusCommand.Reporter reporter;
+ private YarnApplicationState state;
+ private int pollWaitSec;
+ private int startupWaitSec;
+
+ public StartMonitor() {
+ pollWaitSec = config.getInt(DrillOnYarnConfig.CLIENT_POLL_SEC);
+ if (pollWaitSec < 1) {
+ pollWaitSec = 1;
+ }
+ startupWaitSec = config.getInt(DrillOnYarnConfig.CLIENT_START_WAIT_SEC);
+ }
+
+ void run(String appName) throws ClientException {
+ System.out.print("Launching " + appName + "...");
+ reporter = new StatusCommand.Reporter(client);
+ reporter.getReport();
+ if (!reporter.isStarting()) {
+ return;
+ }
+ updateState(reporter.getState());
+ try {
+ int attemptCount = startupWaitSec / pollWaitSec;
+ for (int attempt = 0; attempt < attemptCount; attempt++) {
+ if (!poll()) {
+ break;
+ }
+ }
+ } finally {
+ System.out.println();
+ }
+ reporter.display(verbose, true);
+ if (reporter.isStarting()) {
+ System.out.println(
+ "Application Master is slow to start, use the 'status' command later to check status.");
+ }
+ }
+
+ private boolean poll() throws ClientException {
+ try {
+ Thread.sleep(pollWaitSec * 1000);
+ } catch (InterruptedException e) {
+ return false;
+ }
+ reporter.getReport();
+ if (!reporter.isStarting()) {
+ return false;
+ }
+ YarnApplicationState newState = reporter.getState();
+ if (newState == state) {
+ System.out.print(".");
+ return true;
+ }
+ System.out.println();
+ updateState(newState);
+ return true;
+ }
+
+ private void updateState(YarnApplicationState newState) {
+ state = newState;
+ if (verbose) {
+ System.out.print("Application State: ");
+ System.out.println(state.toString());
+ System.out.print("Starting...");
+ }
+ }
+ }
+
+ private void waitForStartAndReport(String appName) throws ClientException {
+ StartMonitor monitor = new StartMonitor();
+ monitor.run(appName);
+ }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/CleanCommand.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/CleanCommand.java
new file mode 100644
index 000000000..1fcba2d47
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/CleanCommand.java
@@ -0,0 +1,89 @@
+/*
+ * 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.drill.yarn.client;
+
+import java.io.File;
+
+import org.apache.drill.yarn.core.DfsFacade;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+
+import com.typesafe.config.Config;
+
+import org.apache.drill.yarn.core.DfsFacade.DfsFacadeException;
+
+public class CleanCommand extends ClientCommand {
+ private Config config;
+ private DfsFacade dfs;
+
+ @Override
+ public void run() throws ClientException {
+ config = DrillOnYarnConfig.config();
+ if (!isLocalized()) {
+ System.out.println("Not using localized files; nothing to clean.");
+ return;
+ }
+ connectToDfs();
+ removeDrillArchive();
+ removeSiteArchive();
+ }
+
+ public boolean isLocalized() {
+ return config.getBoolean(DrillOnYarnConfig.LOCALIZE_DRILL);
+ }
+
+ protected void connectToDfs() throws ClientException {
+ try {
+ System.out.print("Connecting to DFS...");
+ dfs = new DfsFacade(config);
+ dfs.connect();
+ System.out.println(" Connected.");
+ } catch (DfsFacadeException e) {
+ System.out.println("Failed.");
+ throw new ClientException("Failed to connect to DFS", e);
+ }
+ }
+
+ private void removeDrillArchive() {
+ String localArchivePath = config
+ .getString(DrillOnYarnConfig.DRILL_ARCHIVE_PATH);
+ String archiveName = new File(localArchivePath).getName();
+ removeArchive(archiveName);
+ }
+
+ private void removeArchive(String archiveName) {
+ System.out.print("Removing " + archiveName + " ...");
+ try {
+ dfs.removeDrillFile(archiveName);
+ System.out.println(" Removed");
+ ;
+ } catch (DfsFacadeException e) {
+ System.out.println();
+ System.err.println(e.getMessage());
+ }
+ }
+
+ private void removeSiteArchive() {
+ DrillOnYarnConfig doyConfig = DrillOnYarnConfig.instance();
+ if (!doyConfig.hasSiteDir()) {
+ return;
+ }
+ String archiveName = DrillOnYarnConfig.SITE_ARCHIVE_NAME;
+ removeArchive(archiveName);
+ }
+
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientCommand.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientCommand.java
new file mode 100644
index 000000000..469d04cab
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientCommand.java
@@ -0,0 +1,100 @@
+/*
+ * 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.drill.yarn.client;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.IOException;
+
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.drill.yarn.core.YarnRMClient;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+
+public abstract class ClientCommand {
+ protected CommandLineOptions opts;
+
+ public void setOpts(CommandLineOptions opts) {
+ this.opts = opts;
+ }
+
+ public abstract void run() throws ClientException;
+
+ /**
+ * Return the path to the app id file. The file goes into the directory above
+ * Drill Home (which should be the folder that contains the localized archive)
+ * and is named for the ZK cluster (to ensure that the name is a valid file
+ * name.)
+ *
+ * @return
+ */
+ protected static File getAppIdFile() {
+ return DrillOnYarnConfig.instance().getLocalAppIdFile();
+ }
+
+ protected ApplicationId checkAppId() throws ClientException {
+ String appIdStr;
+ if (opts.appId != null) {
+ appIdStr = opts.appId;
+ } else {
+ File appIdFile = getAppIdFile();
+ appIdStr = loadAppId(appIdFile);
+ if (appIdStr == null) {
+ throw new ClientException(
+ "No Drill cluster is running (did not find file appid file: "
+ + appIdFile.toString() + ")");
+ }
+ }
+ return ConverterUtils.toApplicationId(appIdStr);
+ }
+
+ protected YarnRMClient getClient() throws ClientException {
+ return new YarnRMClient(checkAppId());
+ }
+
+ protected String loadAppId(File appIdFile) {
+ BufferedReader reader = null;
+ String appIdStr;
+ try {
+ reader = new BufferedReader(new FileReader(appIdFile));
+ appIdStr = reader.readLine();
+ if (appIdStr != null) {
+ appIdStr = appIdStr.trim();
+ }
+ } catch (FileNotFoundException e) {
+ return null;
+ } catch (IOException e) {
+ return null;
+ } finally {
+ try {
+ if (reader != null) {
+ reader.close();
+ }
+ } catch (IOException e) {
+ // Ignore
+ }
+ }
+ return appIdStr;
+ }
+
+ protected void removeAppIdFile() {
+ getAppIdFile().delete();
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientContext.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientContext.java
new file mode 100644
index 000000000..377b3b32c
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientContext.java
@@ -0,0 +1,48 @@
+/*
+ * 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.drill.yarn.client;
+
+import java.io.PrintStream;
+
+/**
+ * Provides a static set of contextual operations that can be configured one way
+ * for production, a separate way for unit tests.
+ */
+
+public class ClientContext {
+
+ private static ClientContext instance;
+ public static PrintStream err = System.err;
+ public static PrintStream out = System.out;
+
+ public static void init() {
+ init(new ClientContext());
+ }
+
+ protected static void init(ClientContext instance) {
+ ClientContext.instance = instance;
+ }
+
+ public static ClientContext instance() {
+ return instance;
+ }
+
+ public void exit(int exitCode) {
+ System.exit(exitCode);
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientException.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientException.java
new file mode 100644
index 000000000..24c062bfe
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientException.java
@@ -0,0 +1,34 @@
+/*
+ * 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.drill.yarn.client;
+
+public class ClientException extends Exception {
+ private static final long serialVersionUID = 1L;
+
+ public ClientException(String msg) {
+ super(msg);
+ }
+
+ public ClientException(String msg, Exception e) {
+ super(msg, e);
+ }
+
+ public ClientException(Exception e) {
+ super(e.getMessage(), e);
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/CommandLineOptions.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/CommandLineOptions.java
new file mode 100644
index 000000000..174265dd3
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/CommandLineOptions.java
@@ -0,0 +1,230 @@
+/*
+ * 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.drill.yarn.client;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Drill YARN client command line options.
+ * <p><pre>
+ * DrillYarnClient -h|--help |
+ * start |
+ * stop |
+ * status |
+ * resize [+|-]n
+ * </pre></p>
+ * <ul>
+ * <li>help: Prints command line usage</li>
+ * <li>start: starts the defined cluster</li>
+ * <li>stop: stops the defined cluster</li>
+ * <li>resize: adds (+n), removes (-n) or resizes (n) the cluster</li>
+ * <li>status: prints status about the cluster</li>
+ * </ul>
+ * <p>
+ * This is a do-it-yourself parser because the command line parser
+ * used by Drill does not accept parameters (arguments) without a dash,
+ * and does not accept arguments (such as resize -3) with a dash.
+ */
+
+public class CommandLineOptions {
+
+ public enum Command {
+ HELP( "help", "Provide description of usage."),
+
+ /**
+ * Primary command to upload the application archive and start the Drill cluster.
+ */
+
+ START( "start", "Start the cluster."),
+
+ // Removed at QA request. QA wants a "real" restart. Also, upload of the
+ // archive is fast enough that a "start without upload" option is not really
+ // needed.
+// /**
+// * Convenience method when debugging, testing. Restarts the cluster without the
+// * archive upload; assumes the upload was already done.
+// */
+//
+// RESTART( "restart", "Restart the cluster (without archive upload)."),
+
+ /**
+ * Primary command to stop a running cluster.
+ */
+
+ STOP( "stop", "Stop the cluster."),
+
+ /**
+ * Primary command to get the status of a running cluster.
+ */
+
+ STATUS( "status", "Provide the status of the cluster."),
+
+ RESIZE( "resize", "Resize the cluster +n: add nodes, -n: remove nodes, n resize to given size."),
+
+ TEST( null, null ),
+
+ /**
+ * Convenience command to display the effective configuration settings to
+ * diagnose problems.
+ */
+
+ DESCRIBE( "describe", "Display and validate configuration." ),
+
+ /**
+ * Convenience command to upload the application archive to test the DFS
+ * settings without launching the Drill cluster.
+ */
+
+ UPLOAD( "upload", "Upload archives to validate DFS." ),
+
+ /**
+ * Convenience command to remove the Drill-on-YARN archive(s) from DFS.
+ * Note: doing this while a Drill cluster is running will cause subsequent
+ * Drillbit launches to fail.
+ */
+
+ CLEAN( "clean", "Remove archives stored in DFS." );
+
+ private String cmd;
+ private String descrip;
+
+ private Command(String cmd, String descrip) {
+ this.cmd = cmd;
+ this.descrip = descrip;
+ }
+
+ public boolean isMatch(String arg) {
+ String key = (cmd == null) ? toString() : cmd;
+ return key.equalsIgnoreCase(arg);
+ }
+
+ public boolean isHidden() {
+ return descrip == null;
+ }
+
+ public String getCommand( ) { return cmd; }
+ public String getDescription( ) { return descrip; }
+ }
+
+ Command command;
+ public String appId;
+ public boolean dryRun;
+ public String resizePrefix;
+ public int resizeValue;
+ public boolean verbose = false;
+ public boolean force = false;
+
+ /**
+ * Parse the command line. Invalid option combinations result in the
+ * error option being set.
+ */
+ public boolean parse(String args[]) {
+ for (int i = 0; i < args.length; i++) {
+ String arg = args[i];
+ if (arg.equals("-h") || arg.equals("-?")) {
+ command = Command.HELP;
+ break;
+ }
+ if (arg.equals("-v") || arg.equals("--verbose")) {
+ verbose = true;
+ continue;
+ }
+ if (arg.equals("-f") || arg.equals("--force")) {
+ force = true;
+ continue;
+ }
+ if (arg.equals("-d") || arg.equals("--dryrun")) {
+ dryRun = true;
+ continue;
+ }
+ if (arg.equals("-a") || arg.equals("--appid")) {
+ if (i + 1 == args.length) {
+ return false;
+ }
+ appId = args[++i];
+ continue;
+ }
+ if (command != null) {
+ command = null;
+ return false;
+ }
+
+ // Check if a command line word matches this command. Be nice,
+ // allow -foo and --foo in addition to the "proper" foo.
+
+ String cmdStr = arg;
+ if (cmdStr.startsWith("--")) {
+ cmdStr = arg.substring(2);
+ } else if (cmdStr.startsWith("-")) {
+ cmdStr = cmdStr.substring(1);
+ }
+ for (Command cmd : Command.values()) {
+ if (cmd.isMatch(cmdStr)) {
+ command = cmd;
+ if (command == Command.RESIZE) {
+ if (i + 1 == args.length) {
+ command = null;
+ break;
+ }
+ parseResizeOption(args[++i]);
+ }
+ break;
+ }
+ }
+ }
+ return true;
+ }
+
+ private void parseResizeOption(String resize) {
+ Pattern p = Pattern.compile("([+-]?)(\\d+)");
+ Matcher m = p.matcher(resize);
+ if (m.matches()) {
+ resizePrefix = m.group(1);
+ resizeValue = Integer.parseInt(m.group(2));
+ } else {
+ command = null;
+ return;
+ }
+ }
+
+ public Command getCommand() {
+ return command;
+ }
+
+ public String getResizePrefix() {
+ return resizePrefix;
+ }
+
+ public int getResizeValue() {
+ return resizeValue;
+ }
+
+ public void usage() {
+ ClientContext.out.println(
+ "Usage: drill-on-yarn.sh [--site site-dir] command [-v|--verbose][-a app-id]");
+ ClientContext.out.println("Where command is one of:");
+ for (Command cmd : Command.values()) {
+ if (cmd.isHidden()) {
+ continue;
+ }
+ ClientContext.out
+ .println(" " + cmd.getCommand() + " - " + cmd.getDescription());
+ }
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/DrillOnYarn.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/DrillOnYarn.java
new file mode 100644
index 000000000..587766ad7
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/DrillOnYarn.java
@@ -0,0 +1,176 @@
+/*
+ * 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.drill.yarn.client;
+
+
+import org.apache.drill.yarn.core.DoyConfigException;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.log4j.BasicConfigurator;
+
+/**
+ * Client for the Drill-on-YARN integration. See YARN documentation
+ * for the role of a YARN client.
+ * <p>
+ * The client needs configuration information from drill-on-yarn.conf,
+ * the directory of which must be in the class path. It is put there
+ * by the drill-on-yarn.sh script.
+ * <p>
+ * The client also requires a debugging configuration file to be given
+ * explicitly as follows:<br>
+ * -Dlogback.configurationFile=/path/to/yarn-client-log.xml<br>
+ * The drillbit itself uses the default logging config file name of
+ * logback.xml; which contains references to system properties that are
+ * not defined in this client. The result of not including the log
+ * configuration file is that you'll see various var.name_IS_UNDEFINED
+ * files in the directory from which you launched the client.
+ * <p>
+ * The client accepts a command, creates a command object for that
+ * command, and executes it. There are a few main commands (start, stop),
+ * along with some management commands (status, resize), and a few commands
+ * mostly used for debugging and diagnosis (upload,etc.) Some commands
+ * are very similar, so a single command object may handle multiple user
+ * commands.
+ * <p>
+ * The client requires a working distributed file system (DFS), the
+ * configuration of which is given either implicitly, or in the Hadoop
+ * configuration files. Similarly, the client requires a working YARN
+ * deployment, again with either implicit configuration or configuration
+ * given in the Hadoop configuration. The Hadoop configuration must be
+ * on the class path when launching this client.
+ *
+ * <h3>Debugging</h3>
+ * <p>
+ * To debug this class, add two or three directories to your class path:
+ * <ul>
+ * <li>$DRILL_CONF_DIR (if using a separate site directory)</li>
+ * <li>$HADOOP_HOME/etc/hadoop</li>
+ * <li>$DRILL_HOME/conf</li>
+ * </ul>
+ * Note that these MUST be in the order listed since $DRILL_HOME/conf
+ * contains, by default, a version of core-site.xml that probably is
+ * NOT the one you want to use for YARN. For YARN, you want the one
+ * in $HADOOP_HOME/etc/hadoop.
+ * <p>
+ * Also, set the following VM argument:<br>
+ * -Dlogback.configurationFile=/path/to/drill/conf/yarn-client-log.xml<br>
+ * or<br>
+ * -Dlogback.configurationFile=/path/to/drill-site/yarn-client-log.xml<br>
+ */
+
+public class DrillOnYarn {
+ public static void main(String argv[]) {
+ BasicConfigurator.configure();
+ ClientContext.init();
+ run(argv);
+ }
+
+ public static void run(String argv[]) {
+ ClientContext context = ClientContext.instance();
+
+ // Parse command-line options.
+
+ CommandLineOptions opts = new CommandLineOptions();
+ if (!opts.parse(argv)) {
+ opts.usage();
+ context.exit(-1);
+ }
+ if (opts.getCommand() == null) {
+ opts.usage();
+ context.exit(-1);
+ }
+
+ // Load configuration.
+
+ try {
+ DrillOnYarnConfig.load().setClientPaths();
+ } catch (DoyConfigException e) {
+ ClientContext.err.println(e.getMessage());
+ context.exit(-1);
+ }
+
+ // Create the required command object.
+
+ ClientCommand cmd;
+ switch (opts.getCommand()) {
+ case UPLOAD:
+ cmd = new StartCommand(true, false);
+ break;
+ case START:
+ cmd = new StartCommand(true, true);
+ break;
+ // Removed at QA request. QA wants a "real" restart. Also, upload of the
+ // archive is fast enough that a "start without upload" option is not really
+ // needed.
+// case RESTART:
+// cmd = new StartCommand(false, true);
+// break;
+ case DESCRIBE:
+ cmd = new PrintConfigCommand();
+ break;
+ case STATUS:
+ cmd = new StatusCommand();
+ break;
+ case STOP:
+ cmd = new StopCommand();
+ break;
+ case CLEAN:
+ cmd = new CleanCommand();
+ break;
+ case RESIZE:
+ cmd = new ResizeCommand();
+ break;
+ default:
+ cmd = new HelpCommand();
+ }
+
+ // Run the command.
+
+ cmd.setOpts(opts);
+ try {
+ cmd.run();
+ } catch (ClientException e) {
+ displayError(opts, e);
+ context.exit(1);
+ }
+ }
+
+ private static void displayError(CommandLineOptions opts, ClientException e) {
+
+ // Show the Drill-provided explanation of the error.
+
+ ClientContext.err.println(e.getMessage());
+
+ // Add the underlying exception information, if any.
+
+ Throwable parent = e;
+ Throwable cause = e.getCause();
+ while (cause != null && cause != parent) {
+ ClientContext.err.print(" Caused by: ");
+ ClientContext.err.println(cause.getMessage());
+ parent = cause;
+ cause = cause.getCause();
+ }
+
+ // Include the full stack trace if requested.
+
+ if (opts.verbose) {
+ ClientContext.err.println("Full stack trace:");
+ e.printStackTrace(ClientContext.err);
+ }
+ }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/FileUploader.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/FileUploader.java
new file mode 100644
index 000000000..ace2d036a
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/FileUploader.java
@@ -0,0 +1,551 @@
+/*
+ * 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.drill.yarn.client;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintStream;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.yarn.core.DfsFacade;
+import org.apache.drill.yarn.core.DoYUtil;
+import org.apache.drill.yarn.core.DoyConfigException;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.drill.yarn.core.DfsFacade.DfsFacadeException;
+import org.apache.drill.yarn.core.DfsFacade.Localizer;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+
+import com.typesafe.config.Config;
+
+/**
+ * Performs the file upload portion of the operation by uploading an archive to
+ * the target DFS system and directory. Records the uploaded archive so it may
+ * be used for localizing Drill in the launch step.
+ * <p>
+ * Some of the code is a bit of a dance so we can get information early to
+ * display in status messages.
+ * <p>
+ * This class handles x cases:
+ * <ol>
+ * <li>Non-localized, config in $DRILL_HOME/conf.</li>
+ * <li>Non-localized, config in a site directory.</li>
+ * <li>Localized, config in $DRILL_HOME.</li>
+ * <li>Localized, config in a site directory.</li>
+ * </ol>
+ * <p>
+ * The non-localized case adds complexity, but is very handy when doing
+ * development as it avoids the wait for the archives to up- and down-load. The
+ * non-localized mode is not advertised to users as it defeats one of the main
+ * benefits of YARN.
+ * <p>
+ * In the localized case, YARN is incomplete; there is no API to inform the AM
+ * of the set of localized files, so we pass the information along in
+ * environment variables. Also, tar is a bit annoying because it includes the
+ * root directory name when unpacking, so that the drill.tar.gz archive unpacks
+ * to, say, apache-drill.x.y.z. So, we must pass along the directory name as
+ * well.
+ * <p>
+ * All of this is further complicated by the way YARN needs detailed information
+ * to localize resources, and that YARN uses a "key" to identify localized
+ * resources, which becomes the directory name in the task's working folder.
+ * Thus, Drill becomes, say<br>
+ * $PWD/drill/apache-drill.x.y.z/bin, conf, ...<br>
+ * YARN provides PWD. The Drillbit launch script needs to know the next two
+ * directory names.
+ * <p>
+ * For efficiency, we omit uploading the Drill archive if one already exists in
+ * dfs and is the same size as the one on the client. We always upload the
+ * config archive (if needed) because config changes are likely to be one reason
+ * that someone (re)starts the Drill cluster.
+ */
+
+public abstract class FileUploader {
+ protected DrillOnYarnConfig doyConfig;
+ protected Config config;
+ protected DfsFacade dfs;
+ protected boolean dryRun;
+ protected boolean verbose;
+ protected File localDrillHome;
+ protected File localSiteDir;
+ protected File localDrillArchivePath;
+
+ public Map<String, LocalResource> resources = new HashMap<>();
+ public String drillArchivePath;
+ public String siteArchivePath;
+ public String remoteDrillHome;
+ public String remoteSiteDir;
+
+ public static class NonLocalized extends FileUploader {
+ public NonLocalized(boolean dryRun, boolean verbose) {
+ super(dryRun, verbose);
+ }
+
+ @Override
+ public void run() throws ClientException {
+ setup();
+ prepareDrillHome();
+ if (hasSiteDir()) {
+ prepareSiteDir();
+ }
+ if (verbose || dryRun) {
+ dump(System.out);
+ }
+ }
+
+ private void prepareDrillHome() throws ClientException {
+ // We need the drill home property. The client can figure out the
+ // Drill home, but the AM must be told.
+
+ String drillHomeProp = config.getString(DrillOnYarnConfig.DRILL_HOME);
+ if (DoYUtil.isBlank(drillHomeProp)) {
+ System.out.println("Warning: non-localized run "
+ + DrillOnYarnConfig.DRILL_HOME + " is not set.");
+ System.out.println(
+ "Assuming remote Drill home is the same as the local location: "
+ + localDrillHome.getAbsolutePath());
+ }
+ }
+
+ private void prepareSiteDir() throws ClientException {
+ String siteDirProp = config.getString(DrillOnYarnConfig.SITE_DIR);
+ if (DoYUtil.isBlank(siteDirProp)) {
+ System.out.println("Warning: non-localized run "
+ + DrillOnYarnConfig.SITE_DIR + " is not set.");
+ System.out.println(
+ "Assuming remote Drill site is the same as the local location: "
+ + localSiteDir.getAbsolutePath());
+ }
+ }
+ }
+
+ public static class ReuseFiles extends FileUploader {
+ public ReuseFiles(boolean dryRun, boolean verbose) {
+ super(dryRun, verbose);
+ }
+
+ @Override
+ public void run() throws ClientException {
+ setup();
+ checkDrillArchive();
+ if (hasSiteDir()) {
+ checkSiteArchive();
+ }
+ if (verbose || dryRun) {
+ dump(System.out);
+ }
+ }
+
+ /**
+ * Upload the Drill archive if desired. Skip the upload if the file already
+ * exists in dfs and is the same size as the local file. However using the
+ * force option can force an upload even if the sizes match.
+ * <p>
+ * Prepares the information needed to tell YARN and the AM about the
+ * localized archive.
+ * <p>
+ * Note that the Drill archive is not created by this client; it must
+ * already exist on disk. Typically, it is just the archive downloaded from
+ * Apache or some other distribution. The uploaded archive retains the name
+ * of the archive in the client, which may be useful to check the version of
+ * the uploaded code based on the file name.
+ *
+ * @throws ClientException
+ */
+
+ private void checkDrillArchive() throws ClientException {
+ // Print the progress message here because doing the connect takes
+ // a while and the message makes it look like we're doing something.
+
+ DfsFacade.Localizer localizer = makeDrillLocalizer();
+ connectToDfs();
+ try {
+ if (!localizer.destExists()) {
+ throw new ClientException(
+ "Drill archive not found in DFS: " + drillArchivePath);
+ }
+ } catch (IOException e) {
+ throw new ClientException(
+ "Failed to check existence of " + drillArchivePath, e);
+ }
+ if (!localDrillArchivePath.exists()) {
+ return;
+ }
+ if (!localizer.filesMatch()) {
+ System.out.println(
+ "Warning: Drill archive on DFS does not match the local version.");
+ }
+ defineResources(localizer, DrillOnYarnConfig.DRILL_ARCHIVE_KEY);
+ }
+
+ private void checkSiteArchive() throws ClientException {
+ // Print the progress message here because doing the connect takes
+ // a while and the message makes it look like we're doing something.
+
+ DfsFacade.Localizer localizer = makeSiteLocalizer(null);
+ try {
+ if (!localizer.destExists()) {
+ throw new ClientException(
+ "Drill archive not found in DFS: " + drillArchivePath);
+ }
+ } catch (IOException e) {
+ throw new ClientException(
+ "Failed to check existence of " + drillArchivePath, e);
+ }
+ defineResources(localizer, DrillOnYarnConfig.SITE_ARCHIVE_KEY);
+ }
+ }
+
+ public static class UploadFiles extends FileUploader {
+ private boolean force;
+
+ public UploadFiles(boolean force, boolean dryRun, boolean verbose) {
+ super(dryRun, verbose);
+ this.force = force;
+ }
+
+ @Override
+ public void run() throws ClientException {
+ setup();
+ uploadDrillArchive();
+ if (hasSiteDir()) {
+ uploadSite();
+ }
+ if (verbose || dryRun) {
+ dump(System.out);
+ }
+ }
+
+ /**
+ * Create a temporary archive of the site directory and upload it to DFS. We
+ * always upload the site; we never reuse an existing one.
+ *
+ * @throws ClientException
+ */
+
+ private void uploadSite() throws ClientException {
+ File siteArchive = createSiteArchive();
+ try {
+ uploadSiteArchive(siteArchive);
+ } finally {
+ siteArchive.delete();
+ }
+ }
+
+ /**
+ * Upload the Drill archive if desired. Skip the upload if the file already
+ * exists in dfs and is the same size as the local file. However using the
+ * force option can force an upload even if the sizes match.
+ * <p>
+ * Prepares the information needed to tell YARN and the AM about the
+ * localized archive.
+ * <p>
+ * Note that the Drill archive is not created by this client; it must
+ * already exist on disk. Typically, it is just the archive downloaded from
+ * Apache or some other distribution. The uploaded archive retains the name
+ * of the archive in the client, which may be useful to check the version of
+ * the uploaded code based on the file name.
+ *
+ * @throws ClientException
+ */
+
+ private void uploadDrillArchive() throws ClientException {
+ // Print the progress message here because doing the connect takes
+ // a while and the message makes it look like we're doing something.
+
+ connectToDfs();
+ DfsFacade.Localizer localizer = makeDrillLocalizer();
+ boolean needsUpload = force || !localizer.filesMatch();
+
+ if (needsUpload) {
+ // Thoroughly check the Drill archive. Errors with the archive seem a
+ // likely source of confusion, so provide detailed error messages for
+ // common cases. Don't bother with these checks if no upload is needed.
+
+ if (!localDrillArchivePath.exists()) {
+ throw new ClientException(
+ "Drill archive not found: " + localDrillArchivePath.getAbsolutePath());
+ }
+ if (!localDrillArchivePath.canRead()) {
+ throw new ClientException(
+ "Drill archive is not readable: " + localDrillArchivePath.getAbsolutePath());
+ }
+ if (localDrillArchivePath.isDirectory()) {
+ throw new ClientException(
+ "Drill archive cannot be a directory: " + localDrillArchivePath.getAbsolutePath());
+ }
+ }
+
+ drillArchivePath = localizer.getDestPath();
+ if (needsUpload) {
+ if (dryRun) {
+ System.out.print(
+ "Upload " + localDrillArchivePath.getAbsolutePath() + " to " + drillArchivePath);
+ } else {
+ System.out.print("Uploading " + localDrillArchivePath.getAbsolutePath() + " to "
+ + drillArchivePath + " ... ");
+ upload(localizer);
+ }
+ } else {
+ System.out.println(
+ "Using existing Drill archive in DFS: " + drillArchivePath);
+ }
+
+ defineResources(localizer, DrillOnYarnConfig.DRILL_ARCHIVE_KEY);
+ }
+
+ /**
+ * Run the tar command to archive the site directory into a temporary
+ * archive which is then uploaded to DFS using a standardized name. The site
+ * directory is always uploaded since configuration is subject to frequent
+ * changes.
+ *
+ * @return
+ * @throws ClientException
+ */
+
+ private File createSiteArchive() throws ClientException {
+ File siteArchiveFile;
+ try {
+ siteArchiveFile = File.createTempFile("drill-site-", ".tar.gz");
+ } catch (IOException e) {
+ throw new ClientException("Failed to create site archive temp file", e);
+ }
+ String cmd[] = new String[] { "tar", "-C", localSiteDir.getAbsolutePath(),
+ "-czf", siteArchiveFile.getAbsolutePath(), "." };
+ List<String> cmdList = Arrays.asList(cmd);
+ String cmdLine = DoYUtil.join(" ", cmdList);
+ if (dryRun) {
+ System.out.print("Site archive command: ");
+ System.out.println(cmdLine);
+ return siteArchiveFile;
+ }
+
+ ProcessBuilder builder = new ProcessBuilder(cmdList);
+ builder.redirectErrorStream(true);
+ Process proc;
+ try {
+ proc = builder.start();
+ } catch (IOException e) {
+ throw new ClientException("Failed to launch tar process: " + cmdLine,
+ e);
+ }
+
+ // Should not be much output. But, we have to read it anyway to avoid
+ // blocking. We'll use the output if we encounter an error.
+
+ BufferedReader br = new BufferedReader(
+ new InputStreamReader(proc.getInputStream()));
+ StringBuilder buf = new StringBuilder();
+ try {
+ String line;
+ while ((line = br.readLine()) != null) {
+ buf.append(line);
+ buf.append("\n");
+ }
+ br.close();
+ } catch (IOException e) {
+ throw new ClientException("Failed to read output from tar command", e);
+ }
+ try {
+ proc.waitFor();
+ } catch (InterruptedException e) {
+ // Won't occur.
+ }
+ if (proc.exitValue() != 0) {
+ String msg = buf.toString().trim();
+ throw new ClientException("Tar of site directory failed: " + msg);
+ }
+ return siteArchiveFile;
+ }
+
+ /**
+ * Upload the site archive. For debugging, the client provides the option to
+ * use existing files, which users should not do in production.
+ *
+ * @param siteArchive
+ * @throws ClientException
+ */
+
+ private void uploadSiteArchive(File siteArchive) throws ClientException {
+ DfsFacade.Localizer localizer = makeSiteLocalizer(siteArchive);
+
+ if (dryRun) {
+ System.out.println("Upload site archive to " + siteArchivePath);
+ } else {
+ System.out
+ .print("Uploading site directory " + localSiteDir.getAbsolutePath() +
+ " to " + siteArchivePath + " ... ");
+ upload(localizer);
+ }
+ defineResources(localizer, DrillOnYarnConfig.SITE_ARCHIVE_KEY);
+ }
+ }
+
+ public FileUploader(boolean dryRun, boolean verbose) {
+ doyConfig = DrillOnYarnConfig.instance();
+ this.config = doyConfig.getConfig();
+ this.dryRun = dryRun;
+ this.verbose = verbose;
+ }
+
+ public abstract void run() throws ClientException;
+
+ /**
+ * Common setup of the Drill and site directories.
+ *
+ * @throws ClientException
+ */
+
+ protected void setup() throws ClientException {
+
+ // Local and remote Drill home locations.
+
+ localDrillHome = doyConfig.getLocalDrillHome();
+ try {
+ remoteDrillHome = doyConfig.getRemoteDrillHome();
+ } catch (DoyConfigException e) {
+ throw new ClientException(e);
+ }
+
+ // Site directory is optional. Local and remote locations, if provided.
+ // Check that the site directory is an existing directory.
+
+ localSiteDir = doyConfig.getLocalSiteDir();
+ if (hasSiteDir()) {
+ if (!localSiteDir.isDirectory()) {
+ throw new ClientException(
+ "Drill site dir not a directory: " + localSiteDir);
+ }
+ remoteSiteDir = doyConfig.getRemoteSiteDir();
+ }
+
+ // Disclaimer that this is just a dry run when that option is selected.
+
+ if (dryRun) {
+ System.out.println("Dry run only.");
+ }
+ }
+
+ public boolean hasSiteDir() {
+ return localSiteDir != null;
+ }
+
+ /**
+ * Report whether the user wants to localize (upload) Drill files, or just use
+ * files already on the worker nodes.
+ *
+ * @return
+ */
+
+ public boolean isLocalized() {
+ return config.getBoolean(DrillOnYarnConfig.LOCALIZE_DRILL);
+ }
+
+ protected void connectToDfs() throws ClientException {
+ try {
+ System.out.print("Connecting to DFS...");
+ dfs = new DfsFacade(config);
+ dfs.connect();
+ System.out.println(" Connected.");
+ } catch (DfsFacadeException e) {
+ System.out.println("Failed.");
+ throw new ClientException("Failed to connect to DFS", e);
+ }
+ }
+
+ protected Localizer makeDrillLocalizer() throws ClientException {
+ String localArchivePath = config
+ .getString(DrillOnYarnConfig.DRILL_ARCHIVE_PATH);
+ if (DoYUtil.isBlank(localArchivePath)) {
+ throw new ClientException("Drill archive path ("
+ + DrillOnYarnConfig.DRILL_ARCHIVE_PATH + ") is not set.");
+ }
+
+ // Archive is either absolute, or relative to $DRILL_HOME.
+
+ localDrillArchivePath = new File(localArchivePath);
+ if (!localDrillArchivePath.isAbsolute()) {
+ localDrillArchivePath = new File(
+ DrillOnYarnConfig.instance().getLocalDrillHome(), localArchivePath);
+ }
+ DfsFacade.Localizer localizer = new DfsFacade.Localizer(dfs,
+ localDrillArchivePath, "Drill");
+ drillArchivePath = localizer.getDestPath();
+ return localizer;
+ }
+
+ protected Localizer makeSiteLocalizer(File siteArchive) {
+ DfsFacade.Localizer localizer = new DfsFacade.Localizer(dfs, siteArchive,
+ DrillOnYarnConfig.SITE_ARCHIVE_NAME, "Site");
+ siteArchivePath = localizer.getDestPath();
+ return localizer;
+ }
+
+ protected void upload(Localizer localizer) throws ClientException {
+ try {
+ localizer.upload();
+ } catch (DfsFacadeException e) {
+ System.out.println("Failed.");
+ throw new ClientException(
+ "Failed to upload " + localizer.getLabel() + " archive", e);
+ }
+ System.out.println("Uploaded.");
+ }
+
+ protected void defineResources(Localizer localizer, String keyProp)
+ throws ClientException {
+ String key = config.getString(keyProp);
+ try {
+ localizer.defineResources(resources, key);
+ } catch (DfsFacadeException e) {
+ throw new ClientException(
+ "Failed to get DFS status for " + localizer.getLabel() + " archive",
+ e);
+ }
+ }
+
+ protected void dump(PrintStream out) {
+ out.print("Localized: ");
+ out.println((isLocalized()) ? "Yes" : "No");
+ out.print("Has Site Dir: ");
+ out.println((hasSiteDir()) ? "Yes" : "No");
+ out.print("Local Drill home: ");
+ out.println(localDrillHome.getAbsolutePath());
+ out.print("Remote Drill home: ");
+ out.println(remoteDrillHome);
+ if (hasSiteDir()) {
+ out.print("Local Site dir: ");
+ out.println(localSiteDir.getAbsolutePath());
+ out.print("Remote Site dir: ");
+ out.println(remoteSiteDir);
+ }
+ if (isLocalized()) {
+ out.print("Drill archive DFS path: ");
+ out.println(drillArchivePath);
+ if (hasSiteDir()) {
+ out.print("Site archive DFS path: ");
+ out.println(siteArchivePath);
+ }
+ }
+ }
+} \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/HelpCommand.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/HelpCommand.java
new file mode 100644
index 000000000..3e7e5d90a
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/HelpCommand.java
@@ -0,0 +1,26 @@
+/*
+ * 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.drill.yarn.client;
+
+public class HelpCommand extends ClientCommand {
+ @Override
+ public void run() {
+ opts.usage();
+ }
+
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/KillCommand.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/KillCommand.java
new file mode 100644
index 000000000..8b7914c11
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/KillCommand.java
@@ -0,0 +1,48 @@
+/*
+ * 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.drill.yarn.client;
+
+import org.apache.drill.yarn.core.YarnClientException;
+import org.apache.drill.yarn.core.YarnRMClient;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+
+public class KillCommand extends ClientCommand {
+
+ @Override
+ public void run() throws ClientException {
+ ApplicationId appId = checkAppId();
+ if (appId == null) {
+ System.exit(-1);
+ }
+ YarnRMClient client = new YarnRMClient(appId);
+ try {
+ client.killApplication();
+ } catch (YarnClientException e) {
+ throw new ClientException(e);
+ }
+ System.out.println("Kill request sent, waiting for shut-down.");
+ try {
+ client.waitForCompletion();
+ } catch (YarnClientException e) {
+ throw new ClientException(
+ "Wait for completion failed for app id: " + appId.toString(), e);
+ }
+ System.out.println("Application completed: " + appId.toString());
+ }
+
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/PrintConfigCommand.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/PrintConfigCommand.java
new file mode 100644
index 000000000..69cdf55fc
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/PrintConfigCommand.java
@@ -0,0 +1,49 @@
+/*
+ * 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.drill.yarn.client;
+
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+public class PrintConfigCommand extends ClientCommand {
+ @Override
+ public void run() {
+ // Dump configuration if requested for diagnostic use.
+
+ System.out.println("----------------------------------------------");
+ System.out.println("Effective Drill-on-YARN Configuration");
+ DrillOnYarnConfig.instance().dump();
+ System.out.println("----------------------------------------------");
+
+ // Dump YARN configuration.
+
+ System.out.println("YARN, DFS and Hadoop Configuration");
+ YarnConfiguration conf = new YarnConfiguration();
+ try {
+ YarnConfiguration.dumpConfiguration(conf,
+ new OutputStreamWriter(System.out));
+ System.out.println();
+ } catch (IOException e) {
+ // Ignore;
+ }
+ System.out.println("----------------------------------------------");
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/ResizeCommand.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/ResizeCommand.java
new file mode 100644
index 000000000..43ae02cf4
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/ResizeCommand.java
@@ -0,0 +1,115 @@
+/*
+ * 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.drill.yarn.client;
+
+import org.apache.drill.yarn.core.DoYUtil;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.drill.yarn.core.YarnRMClient;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+
+import com.typesafe.config.Config;
+
+public class ResizeCommand extends ClientCommand {
+ private Config config;
+ private YarnRMClient client;
+
+ @Override
+ public void run() throws ClientException {
+ config = DrillOnYarnConfig.config();
+ client = getClient();
+ System.out.println(
+ "Resizing cluster for Application ID: " + client.getAppId().toString());
+
+ // First get an application report to ensure that the AM is,
+ // in fact, running, and to get the HTTP endpoint.
+
+ StatusCommand.Reporter reporter = new StatusCommand.Reporter(client);
+ try {
+ reporter.getReport();
+ } catch (ClientException e) {
+ reporter = null;
+ }
+ String prefix = opts.resizePrefix;
+ int quantity = opts.resizeValue;
+ String cmd;
+ if (prefix.equals("+")) {
+ cmd = "grow";
+ if (opts.verbose) {
+ System.out.println("Growing cluster by " + quantity + " nodes.");
+ }
+ } else if (prefix.equals("-")) {
+ cmd = "shrink";
+ if (opts.verbose) {
+ System.out.println("Shrinking cluster by " + quantity + " nodes.");
+ }
+ } else {
+ cmd = "resize";
+ if (opts.verbose) {
+ System.out.println("Resizing cluster to " + quantity + " nodes.");
+ }
+ }
+ if (sendResize(reporter.getAmUrl(), cmd, quantity)) {
+ System.out.println("Use web UI or status command to check progress.");
+ }
+ }
+
+ private boolean sendResize(String baseUrl, String cmd, int quantity) {
+ try {
+ if (DoYUtil.isBlank(baseUrl)) {
+ return false;
+ }
+ SimpleRestClient restClient = new SimpleRestClient();
+ String tail = "rest/" + cmd + "/" + quantity;
+ String masterKey = config.getString(DrillOnYarnConfig.HTTP_REST_KEY);
+ if (!DoYUtil.isBlank(masterKey)) {
+ tail += "?key=" + masterKey;
+ }
+ if (opts.verbose) {
+ System.out.println("Resizing with POST " + baseUrl + "/" + tail);
+ }
+ String result = restClient.send(baseUrl, tail, true);
+
+ JSONParser parser = new JSONParser();
+ Object response;
+ try {
+ response = parser.parse(result);
+ } catch (ParseException e) {
+ System.err.println("Invalid response received from AM");
+ if (opts.verbose) {
+ System.out.println(result);
+ System.out.println(e.getMessage());
+ }
+ return false;
+ }
+ JSONObject root = (JSONObject) response;
+
+ System.out.println("AM responded: " + root.get("message"));
+ if ("ok".equals(root.get("status"))) {
+ return true;
+ }
+ System.err.println("Failed to resize the application master.");
+ return false;
+ } catch (ClientException e) {
+ System.err.println("Resize failed: " + e.getMessage());
+ return false;
+ }
+ }
+
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/SimpleRestClient.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/SimpleRestClient.java
new file mode 100644
index 000000000..e47fb581e
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/SimpleRestClient.java
@@ -0,0 +1,66 @@
+/*
+ * 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.drill.yarn.client;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.client.ClientProtocolException;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.methods.HttpRequestBase;
+import org.apache.http.impl.client.DefaultHttpClient;
+
+public class SimpleRestClient {
+ public String send(String baseUrl, String resource, boolean isPost)
+ throws ClientException {
+ String url = baseUrl;
+ if (!url.endsWith("/")) {
+ url += "/";
+ }
+ url += resource;
+ try {
+ HttpClient client = new DefaultHttpClient();
+ HttpRequestBase request;
+ if (isPost) {
+ request = new HttpPost(url);
+ } else {
+ request = new HttpGet(url);
+ }
+
+ HttpResponse response = client.execute(request);
+ BufferedReader rd = new BufferedReader(
+ new InputStreamReader(response.getEntity().getContent()));
+ StringBuilder buf = new StringBuilder();
+ String line = null;
+ while ((line = rd.readLine()) != null) {
+ buf.append(line);
+ }
+ return buf.toString().trim();
+ } catch (ClientProtocolException e) {
+ throw new ClientException("Internal REST error", e);
+ } catch (IllegalStateException e) {
+ throw new ClientException("Internal REST error", e);
+ } catch (IOException e) {
+ throw new ClientException("REST request failed: " + url, e);
+ }
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/StartCommand.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/StartCommand.java
new file mode 100644
index 000000000..fe505a248
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/StartCommand.java
@@ -0,0 +1,145 @@
+/*
+ * 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.drill.yarn.client;
+
+import java.io.File;
+
+import org.apache.drill.yarn.client.StatusCommand.Reporter;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.drill.yarn.core.YarnRMClient;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+
+import com.typesafe.config.Config;
+
+/**
+ * Launches a drill cluster by uploading the Drill archive then launching the
+ * Drill Application Master (AM). For testing, can also do just the upload or
+ * just the launch. Handles both a localized Drill and a non-localized launch
+ * (which uses a pre-installed Drill.)
+ * <p>
+ * This single operation combines upload and launch because the upload
+ * Information is needed by the launch.
+ * <p>
+ * On the surface, it would seem that uploading a file and launching an app
+ * should be simple operations. However, under YARN, we must handle a large
+ * number of details that must be gotten exactly right. Plus, both the upload
+ * and launch can be slow operations, so we provide feedback to the user that
+ * something is, indeed, happening.
+ */
+
+public class StartCommand extends ClientCommand {
+ private Config config;
+ private boolean upload;
+ private boolean launch;
+ private boolean dryRun;
+
+ public StartCommand(boolean upload, boolean launch) {
+ this.upload = upload;
+ this.launch = launch;
+ }
+
+ @Override
+ public void run() throws ClientException {
+ checkExistingApp();
+
+ dryRun = opts.dryRun;
+ config = DrillOnYarnConfig.config();
+ FileUploader uploader = upload();
+ if (launch) {
+ launch(uploader);
+ }
+ }
+
+ /**
+ * Check if an application ID file exists. If it does, check if an application
+ * is running. If an app is running, then we can't start a new one. If the app
+ * is not running, then clean up the "orphan" app id file.
+ *
+ * @throws ClientException
+ */
+
+ private void checkExistingApp() throws ClientException {
+ File appIdFile = getAppIdFile();
+ if (!appIdFile.exists()) {
+ return;
+ }
+
+ // File exists. Ask YARN about status.
+
+ Reporter reporter;
+ ApplicationId appId;
+ try {
+ System.out.println("Found app ID file: " + appIdFile.getAbsolutePath());
+ appId = checkAppId();
+ System.out.print("Checking application ID: " + appId.toString() + "...");
+ YarnRMClient client = new YarnRMClient(appId);
+ reporter = new Reporter(client);
+ reporter.getReport();
+ } catch (ClientException e) {
+ // This exception occurs when we ask for a report about an application
+ // that
+ // YARN does not know about. (YARN has likely been restarted.)
+
+ System.out.println(" Not running.");
+ appIdFile.delete();
+ return;
+ }
+
+ // YARN knows about the application. But, was it stopped, perhaps from the
+ // web UI?
+
+ if (reporter.isStopped()) {
+ System.out.println(" Completed with state " + reporter.getState());
+ appIdFile.delete();
+ return;
+ }
+
+ // The app (or another one with the same App ID) is running.
+
+ System.out.println(" Still running!");
+ throw new ClientException(
+ "Error: AM already running as Application ID: " + appId);
+ }
+
+ private FileUploader upload() throws ClientException {
+ FileUploader uploader;
+ if (!config.getBoolean(DrillOnYarnConfig.LOCALIZE_DRILL)) {
+ uploader = new FileUploader.NonLocalized(dryRun, opts.verbose);
+ } else if (upload) {
+ uploader = new FileUploader.UploadFiles(opts.force, dryRun, opts.verbose);
+ } else {
+ uploader = new FileUploader.ReuseFiles(dryRun, opts.verbose);
+ }
+ uploader.run();
+ return uploader;
+ }
+
+ private void launch(FileUploader uploader) throws ClientException {
+ AMRunner runner = new AMRunner(config, opts.verbose, dryRun);
+ runner.resources = uploader.resources;
+ runner.remoteDrillHome = uploader.remoteDrillHome;
+ runner.remoteSiteDir = uploader.remoteSiteDir;
+ if (uploader.isLocalized()) {
+ runner.drillArchivePath = uploader.drillArchivePath.toString();
+ if (uploader.hasSiteDir()) {
+ runner.siteArchivePath = uploader.siteArchivePath.toString();
+ }
+ }
+ runner.run();
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/StatusCommand.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/StatusCommand.java
new file mode 100644
index 000000000..863b70069
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/StatusCommand.java
@@ -0,0 +1,189 @@
+/*
+ * 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.drill.yarn.client;
+
+import org.apache.drill.yarn.core.DoYUtil;
+import org.apache.drill.yarn.core.YarnClientException;
+import org.apache.drill.yarn.core.YarnRMClient;
+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.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+
+public class StatusCommand extends ClientCommand {
+ public static class Reporter {
+ private YarnRMClient client;
+ ApplicationReport report;
+
+ public Reporter(YarnRMClient client) {
+ this.client = client;
+ }
+
+ public void getReport() throws ClientException {
+ try {
+ report = client.getAppReport();
+ } catch (YarnClientException e) {
+ throw new ClientException(
+ "Failed to get report for Drill application master", e);
+ }
+ }
+
+ public void display(boolean verbose, boolean isNew) {
+ YarnApplicationState state = report.getYarnApplicationState();
+ if (verbose || !isNew) {
+ System.out.println("Application State: " + state.toString());
+ System.out.println("Host: " + report.getHost());
+ }
+ if (verbose || !isNew) {
+ System.out.println("Queue: " + report.getQueue());
+ System.out.println("User: " + report.getUser());
+ long startTime = report.getStartTime();
+ System.out.println("Start Time: " + DoYUtil.toIsoTime(startTime));
+ System.out.println("Application Name: " + report.getName());
+ }
+ System.out.println("Tracking URL: " + report.getTrackingUrl());
+ if (isNew) {
+ System.out.println("Application Master URL: " + getAmUrl());
+ }
+ showFinalStatus();
+ }
+
+ public String getAmUrl() {
+ return StatusCommand.getAmUrl(report);
+ }
+
+ public void showFinalStatus() {
+ YarnApplicationState state = report.getYarnApplicationState();
+ if (state == YarnApplicationState.FAILED
+ || state == YarnApplicationState.FINISHED) {
+ FinalApplicationStatus status = report.getFinalApplicationStatus();
+ System.out.println("Final status: " + status.toString());
+ if (status != FinalApplicationStatus.SUCCEEDED) {
+ String diag = report.getDiagnostics();
+ if (!DoYUtil.isBlank(diag)) {
+ System.out.println("Diagnostics: " + diag);
+ }
+ }
+ }
+ }
+
+ public YarnApplicationState getState() {
+ return report.getYarnApplicationState();
+ }
+
+ public boolean isStarting() {
+ YarnApplicationState state = getState();
+ return state == YarnApplicationState.ACCEPTED
+ || state == YarnApplicationState.NEW
+ || state == YarnApplicationState.NEW_SAVING
+ || state == YarnApplicationState.SUBMITTED;
+ }
+
+ public boolean isStopped() {
+ YarnApplicationState state = getState();
+ return state == YarnApplicationState.FAILED
+ || state == YarnApplicationState.FINISHED
+ || state == YarnApplicationState.KILLED;
+ }
+
+ public boolean isRunning() {
+ YarnApplicationState state = getState();
+ return state == YarnApplicationState.RUNNING;
+ }
+ }
+
+ public static String getAmUrl(ApplicationReport report) {
+ return DoYUtil.unwrapAmUrl(report.getOriginalTrackingUrl());
+ }
+
+ @Override
+ public void run() throws ClientException {
+ YarnRMClient client = getClient();
+ System.out.println("Application ID: " + client.getAppId().toString());
+ Reporter reporter = new Reporter(client);
+ try {
+ reporter.getReport();
+ } catch (Exception e) {
+ removeAppIdFile();
+ System.out.println("Application is not running.");
+ return;
+ }
+ reporter.display(opts.verbose, false);
+ if (reporter.isRunning()) {
+ showAmStatus(reporter.report);
+ }
+ }
+
+ private void showAmStatus(ApplicationReport report) {
+ try {
+ String baseUrl = getAmUrl(report);
+ if (DoYUtil.isBlank(baseUrl)) {
+ return;
+ }
+ SimpleRestClient restClient = new SimpleRestClient();
+ String tail = "rest/status";
+ if (opts.verbose) {
+ System.out.println("Getting status with " + baseUrl + "/" + tail);
+ }
+ String result = restClient.send(baseUrl, tail, false);
+ formatResponse(result);
+ System.out.println("For more information, visit: " + baseUrl);
+ } catch (ClientException e) {
+ System.out.println("Failed to get AM status");
+ System.err.println(e.getMessage());
+ }
+ }
+
+ private void formatResponse(String result) {
+ JSONParser parser = new JSONParser();
+ Object status;
+ try {
+ status = parser.parse(result);
+ } catch (ParseException e) {
+ System.err.println("Invalid response received from AM");
+ if (opts.verbose) {
+ System.out.println(result);
+ System.out.println(e.getMessage());
+ }
+ return;
+ }
+ JSONObject root = (JSONObject) status;
+ showMetric("AM State", root, "state");
+ showMetric("Target Drillbit Count", root.get("summary"), "targetBitCount");
+ showMetric("Live Drillbit Count", root.get("summary"), "liveBitCount");
+ showMetric("Unmanaged Drillbit Count", root.get("summary"), "unmanagedCount");
+ showMetric("Blacklisted Node Count", root.get("summary"), "blackListCount");
+ showMetric("Free Node Count", root.get("summary"), "freeNodeCount");
+ }
+
+ private void showMetric(String label, Object object, String key) {
+ if (object == null) {
+ return;
+ }
+ if (!(object instanceof JSONObject)) {
+ return;
+ }
+ object = ((JSONObject) object).get(key);
+ if (object == null) {
+ return;
+ }
+ System.out.println(label + ": " + object.toString());
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/StopCommand.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/StopCommand.java
new file mode 100644
index 000000000..95f7bf341
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/StopCommand.java
@@ -0,0 +1,223 @@
+/*
+ * 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.drill.yarn.client;
+
+import org.apache.drill.yarn.core.DoYUtil;
+import org.apache.drill.yarn.core.DrillOnYarnConfig;
+import org.apache.drill.yarn.core.YarnClientException;
+import org.apache.drill.yarn.core.YarnRMClient;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+
+import com.typesafe.config.Config;
+
+/**
+ * Perform a semi-graceful shutdown of the Drill-on-YARN AM. We send a message
+ * to the AM to request shutdown because the YARN-provided message just kills
+ * the AM. (There seems to be no way to get YARN to call its own
+ * AMRMClientAsync.CallbackHandler.onShutdownRequest message.) The AM, however,
+ * cannot gracefully shut down the drill-bits because Drill itself has no
+ * graceful shutdown. But, at least this technique gives the AM a fighting
+ * chance to do graceful shutdown in the future.
+ */
+
+public class StopCommand extends ClientCommand {
+ /**
+ * Poll the YARN RM to check the stop status of the AM. Periodically poll,
+ * waiting to get an app state that indicates app completion.
+ */
+
+ private static class StopMonitor {
+ StatusCommand.Reporter reporter;
+ private YarnApplicationState state;
+ private int pollWaitSec;
+ private int shutdownWaitSec;
+
+ StopMonitor(Config config, StatusCommand.Reporter reporter) {
+ this.reporter = reporter;
+ pollWaitSec = config.getInt(DrillOnYarnConfig.CLIENT_POLL_SEC);
+ if (pollWaitSec < 1) {
+ pollWaitSec = 1;
+ }
+ shutdownWaitSec = config.getInt(DrillOnYarnConfig.CLIENT_STOP_WAIT_SEC);
+ }
+
+ boolean run(boolean verbose) throws ClientException {
+ System.out.print("Stopping...");
+ try {
+ int attemptCount = shutdownWaitSec / pollWaitSec;
+ for (int attempt = 0; attempt < attemptCount; attempt++) {
+ if (!poll()) {
+ break;
+ }
+ }
+ } catch (ClientException e) {
+ System.out.println();
+ throw e;
+ }
+ if (reporter.isStopped()) {
+ System.out.println(" Stopped.");
+ reporter.showFinalStatus();
+ return true;
+ } else {
+ System.out.println();
+ System.out.println(
+ "Application Master is slow to stop, use YARN to check status.");
+ return false;
+ }
+ }
+
+ private boolean poll() throws ClientException {
+ try {
+ Thread.sleep(pollWaitSec * 1000);
+ } catch (InterruptedException e) {
+ return false;
+ }
+ reporter.getReport();
+ if (reporter.isStopped()) {
+ return false;
+ }
+ YarnApplicationState newState = reporter.getState();
+ if (newState == state) {
+ System.out.print(".");
+ return true;
+ }
+ updateState(newState);
+ return true;
+ }
+
+ private void updateState(YarnApplicationState newState) {
+ YarnApplicationState oldState = state;
+ state = newState;
+ if (oldState == null) {
+ return;
+ }
+ System.out.println();
+ System.out.print("Application State: ");
+ System.out.println(state.toString());
+ System.out.print("Stopping...");
+ }
+ }
+
+ private Config config;
+ private YarnRMClient client;
+
+ @Override
+ public void run() throws ClientException {
+ config = DrillOnYarnConfig.config();
+ client = getClient();
+ System.out
+ .println("Stopping Application ID: " + client.getAppId().toString());
+
+ // First get an application report to ensure that the AM is,
+ // in fact, running, and to get the HTTP endpoint.
+
+ StatusCommand.Reporter reporter = new StatusCommand.Reporter(client);
+ try {
+ reporter.getReport();
+ } catch (ClientException e) {
+ reporter = null;
+ }
+
+ // Handle the case of an already stopped app.
+
+ boolean stopped = true;
+ if (reporter == null || reporter.isStopped()) {
+ System.out.println("Application is not running.");
+ } else {
+ // Try to stop the server by sending a STOP REST request.
+
+ if (opts.force) {
+ System.out.println("Forcing shutdown");
+ } else {
+ stopped = gracefulStop(reporter.getAmUrl());
+ }
+
+ // If that did not work, then forcibly kill the AM.
+ // YARN will forcibly kill the AM's containers.
+ // Not pretty, but it works.
+
+ if (opts.force || !stopped) {
+ forcefulStop();
+ }
+
+ // Wait for the AM to stop. The AM may refuse to stop in
+ // the time allowed to wait.
+
+ stopped = new StopMonitor(config, reporter).run(opts.verbose);
+ }
+
+ // If the AM is gone because it started out dead or
+ // we killed it, then forget its App Id.
+
+ if (stopped) {
+ removeAppIdFile();
+ }
+ }
+
+ /**
+ * Do a graceful shutdown by using the AM's REST API call to request stop.
+ * Include the master key with the request to differentiate this request from
+ * accidental uses of the stop REST API.
+ *
+ * @param report
+ * @return
+ */
+
+ private boolean gracefulStop(String baseUrl) {
+ try {
+ if (DoYUtil.isBlank(baseUrl)) {
+ return false;
+ }
+ SimpleRestClient restClient = new SimpleRestClient();
+ String tail = "rest/stop";
+ String masterKey = config.getString(DrillOnYarnConfig.HTTP_REST_KEY);
+ if (!DoYUtil.isBlank(masterKey)) {
+ tail += "?key=" + masterKey;
+ }
+ if (opts.verbose) {
+ System.out.println("Stopping with POST " + baseUrl + "/" + tail);
+ }
+ String result = restClient.send(baseUrl, tail, true);
+ if (result.contains("\"ok\"")) {
+ return true;
+ }
+ System.err.println(
+ "Failed to stop the application master. Response = " + result);
+ return false;
+ } catch (ClientException e) {
+ System.err.println(e.getMessage());
+ System.out.println("Resorting to forced kill");
+ return false;
+ }
+ }
+
+ /**
+ * If the graceful approach did not work, resort to a forceful request. This
+ * asks the AM's NM to kill the AM process.
+ *
+ * @throws ClientException
+ */
+
+ private void forcefulStop() throws ClientException {
+ try {
+ client.killApplication();
+ } catch (YarnClientException e) {
+ throw new ClientException("Failed to stop application master", e);
+ }
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/client/package-info.java b/drill-yarn/src/main/java/org/apache/drill/yarn/client/package-info.java
new file mode 100644
index 000000000..c03c2fa33
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/client/package-info.java
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/**
+ * Implements a "YARN client" for Drill-on-YARN. The client uploads files to
+ * DFS, then requests that YARN start the Application Master. Much fiddling
+ * about is required to support this, such as zipping up the user's configuration,
+ * creating a local file with the app id so we can get app status and shut down
+ * the app, etc.
+ * <p>
+ * Divided into a main program ({@link DrillOnYarn}) and a series of commands.
+ * Some commands are further divided into tasks. Builds on the
+ * YARN and DFS facades defined in the core module.
+ */
+
+package org.apache.drill.yarn.client; \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/core/AppSpec.java b/drill-yarn/src/main/java/org/apache/drill/yarn/core/AppSpec.java
new file mode 100644
index 000000000..34d4ad193
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/core/AppSpec.java
@@ -0,0 +1,169 @@
+/*
+ * 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.drill.yarn.core;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.YarnClientApplication;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Abstract description of a remote process launch that describes the many
+ * details needed to launch a process on a remote node. The YARN launch
+ * specification is a mess to work with; this class provides a simpler facade to
+ * gather the information, then turns around and builds the required YARN
+ * object.
+ * <p>
+ * Based on <a href="https://github.com/hortonworks/simple-yarn-app">Simple YARN
+ * App</a>.
+ */
+
+public class AppSpec extends LaunchSpec {
+
+ static final private Log LOG = LogFactory.getLog(LaunchSpec.class);
+
+ /**
+ * The memory required in the allocated container, in MB.
+ */
+
+ public int memoryMb;
+
+ /**
+ * The number of YARN "vcores" (roughly equivalent to CPUs) to allocate to the
+ * process.
+ */
+
+ public int vCores = 1;
+
+ /**
+ * The number of disk resources (that is, disk channels) used by the process.
+ * Available only on some YARN distributions. Fractional values allowed.
+ */
+
+ public double disks;
+
+ /**
+ * The name of the application given to YARN. Appears in the YARN admin UI.
+ */
+
+ public String appName;
+
+ /**
+ * The YARN queue in which to place the application launch request.
+ */
+
+ public String queueName = "default";
+
+ public int priority = 1;
+
+ /**
+ * Whether to run the AM in unmanaged mode. Leave this false for production
+ * code.
+ */
+
+ public boolean unmanaged;
+
+ /**
+ * Optional node label expression for the launch. Selects the nodes on which
+ * the task can run.
+ */
+
+ public String nodeLabelExpr;
+
+ /**
+ * Given this generic description of an application, create the detailed YARN
+ * application submission context required to launch the application.
+ *
+ * @param conf
+ * the YARN configuration obtained by reading the Hadoop
+ * configuration files
+ * @param app
+ * the YARN definition of the client application to be populated from
+ * this generic description
+ * @return the completed application launch context for the given application
+ * @throws IOException
+ * if localized resources are not found in the distributed file
+ * system (such as HDFS)
+ */
+
+ public ApplicationSubmissionContext createAppLaunchContext(
+ YarnConfiguration conf, YarnClientApplication app) throws IOException {
+ ContainerLaunchContext amContainer = createLaunchContext(conf);
+
+ // Finally, set-up ApplicationSubmissionContext for the application
+ ApplicationSubmissionContext appContext = app
+ .getApplicationSubmissionContext();
+ appContext.setApplicationName(appName); // application name
+ appContext.setAMContainerSpec(amContainer);
+ appContext.setResource(getCapability());
+ appContext.setQueue(queueName); // queue
+ appContext.setPriority(Priority.newInstance(priority));
+ if (!DoYUtil.isBlank(nodeLabelExpr)) {
+ LOG.info(
+ "Requesting to run the AM using node expression: " + nodeLabelExpr);
+ appContext.setNodeLabelExpression(nodeLabelExpr);
+ }
+
+ appContext.setUnmanagedAM(unmanaged);
+
+ // Only try the AM once. It will fail if things are misconfigured. Retrying
+ // is unlikely
+ // to fix the configuration problem.
+
+ appContext.setMaxAppAttempts(1);
+
+ // TODO: Security tokens
+
+ return appContext;
+ }
+
+ public Resource getCapability() {
+
+ // Set up resource type requirements for ApplicationMaster
+ Resource capability = Records.newRecord(Resource.class);
+ capability.setMemory(memoryMb);
+ capability.setVirtualCores(vCores);
+ DoYUtil.callSetDiskIfExists(capability, disks);
+ return capability;
+ }
+
+ @Override
+ public void dump(PrintStream out) {
+ out.print("Memory (MB): ");
+ out.println(memoryMb);
+ out.print("Vcores: ");
+ out.println(vCores);
+ out.print("Disks: ");
+ out.println(disks);
+ out.print("Application Name: ");
+ out.println(appName);
+ out.print("Queue: ");
+ out.println(queueName);
+ out.print("Priority: ");
+ out.println(priority);
+ super.dump(out);
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/core/ClusterDef.java b/drill-yarn/src/main/java/org/apache/drill/yarn/core/ClusterDef.java
new file mode 100644
index 000000000..223b6068d
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/core/ClusterDef.java
@@ -0,0 +1,212 @@
+/*
+ * 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.drill.yarn.core;
+
+import java.io.PrintStream;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.yarn.appMaster.TaskSpec;
+import org.mortbay.log.Log;
+
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigList;
+import com.typesafe.config.ConfigValue;
+
+public class ClusterDef {
+ // The following keys are relative to the cluster group definition
+
+ public static final String GROUP_NAME = "name";
+ public static final String GROUP_TYPE = "type";
+ public static final String GROUP_SIZE = "count";
+
+ // For the labeled pool
+
+ public static final String DRILLBIT_LABEL = "drillbit-label-expr";
+ public static final String AM_LABEL = "am-label-expr";
+
+ /**
+ * Defined cluster tier types. The value of the type appears as the value of
+ * the {@link $CLUSTER_TYPE} parameter in the config file.
+ */
+
+ public enum GroupType {
+ BASIC,
+ LABELED;
+
+ public static GroupType toEnum(String value) {
+ return GroupType.valueOf( value.toUpperCase() );
+ }
+
+ public String toValue() {
+ return name().toLowerCase();
+ }
+ }
+
+ public static class ClusterGroup {
+ private final String name;
+ private final int count;
+ private final GroupType type;
+
+ public ClusterGroup( Map<String, Object> group, int index, GroupType type ) {
+ this.type = type;
+
+ // Config system has already parsed the value. We insist that the value,
+ // when parsed, was interpreted as an integer. That is, the value had
+ // to be, say 10. Not "10", not 10.0, but just a plain integer.
+
+ try {
+ count = (Integer) group.get(GROUP_SIZE);
+ } catch (ClassCastException e) {
+ throw new IllegalArgumentException(
+ "Expected an integer for " + GROUP_SIZE + " for tier " + index);
+ }
+ Object nameValue = group.get(GROUP_NAME);
+ String theName = null;
+ if (nameValue != null) {
+ theName = nameValue.toString();
+ }
+ if (DoYUtil.isBlank(theName)) {
+ theName = "tier-" + Integer.toString(index);
+ }
+ name = theName;
+ }
+
+
+ public String getName( ) { return name; }
+ public int getCount( ) { return count; }
+ public GroupType getType( ) { return type; }
+
+ public void getPairs(int index, List<NameValuePair> pairs) {
+ String key = DrillOnYarnConfig.append(DrillOnYarnConfig.CLUSTERS,
+ Integer.toString(index));
+ addPairs(pairs, key);
+ }
+
+ protected void addPairs(List<NameValuePair> pairs, String key) {
+ pairs.add(
+ new NameValuePair(DrillOnYarnConfig.append(key, GROUP_NAME), name));
+ pairs.add(
+ new NameValuePair(DrillOnYarnConfig.append(key, GROUP_TYPE), type));
+ pairs.add(
+ new NameValuePair(DrillOnYarnConfig.append(key, GROUP_SIZE), count));
+ }
+
+ public void dump(String prefix, PrintStream out) {
+ out.print(prefix);
+ out.print("name = ");
+ out.println(name);
+ out.print(prefix);
+ out.print("type = ");
+ out.println(type.toValue());
+ out.print(prefix);
+ out.print("count = ");
+ out.println(count);
+ }
+
+ public void modifyTaskSpec(TaskSpec taskSpec) {
+ }
+ }
+
+ public static class BasicGroup extends ClusterGroup {
+
+ public BasicGroup(Map<String, Object> pool, int index) {
+ super(pool, index, GroupType.BASIC);
+ }
+
+ }
+
+ public static class LabeledGroup extends ClusterGroup {
+
+ private final String drillbitLabelExpr;
+
+ public LabeledGroup(Map<String, Object> pool, int index) {
+ super(pool, index, GroupType.LABELED);
+ drillbitLabelExpr = (String) pool.get(DRILLBIT_LABEL);
+ if (drillbitLabelExpr == null) {
+ Log.warn("Labeled pool is missing the drillbit label expression ("
+ + DRILLBIT_LABEL + "), will treat pool as basic.");
+ }
+ }
+
+ public String getLabelExpr( ) { return drillbitLabelExpr; }
+
+ @Override
+ public void dump(String prefix, PrintStream out) {
+ out.print(prefix);
+ out.print("Drillbit label expr = ");
+ out.println((drillbitLabelExpr == null) ? "<none>" : drillbitLabelExpr);
+ }
+
+ @Override
+ protected void addPairs(List<NameValuePair> pairs, String key) {
+ super.addPairs(pairs, key);
+ pairs.add(new NameValuePair(DrillOnYarnConfig.append(key, DRILLBIT_LABEL),
+ drillbitLabelExpr));
+ }
+
+ @Override
+ public void modifyTaskSpec(TaskSpec taskSpec) {
+ taskSpec.containerSpec.nodeLabelExpr = drillbitLabelExpr;
+ }
+ }
+
+ /**
+ * Deserialize a node tier from the configuration file.
+ *
+ * @param n
+ * @return
+ */
+
+ public static ClusterGroup getCluster(Config config, int n) {
+ int index = n + 1;
+ ConfigList tiers = config.getList(DrillOnYarnConfig.CLUSTERS);
+ ConfigValue value = tiers.get(n);
+ if ( value == null ) {
+ throw new IllegalArgumentException( "If cluster group is provided, it cannot be null: group " + index );
+ }
+ @SuppressWarnings("unchecked")
+ Map<String, Object> tier = (Map<String, Object>) value.unwrapped();
+ String type;
+ try {
+ type = tier.get(GROUP_TYPE).toString();
+ } catch (NullPointerException e) {
+ throw new IllegalArgumentException(
+ "Pool type is required for cluster group " + index);
+ }
+ GroupType groupType = GroupType.toEnum(type);
+ if (groupType == null) {
+ throw new IllegalArgumentException(
+ "Undefined type for cluster group " + index + ": " + type);
+ }
+ ClusterGroup tierDef;
+ switch (groupType) {
+ case BASIC:
+ tierDef = new BasicGroup( tier, index );
+ break;
+ case LABELED:
+ tierDef = new LabeledGroup( tier, index );
+ break;
+ default:
+ assert false;
+ throw new IllegalStateException(
+ "Undefined cluster group type: " + groupType);
+ }
+ return tierDef;
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/core/ContainerRequestSpec.java b/drill-yarn/src/main/java/org/apache/drill/yarn/core/ContainerRequestSpec.java
new file mode 100644
index 000000000..99a22d7e5
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/core/ContainerRequestSpec.java
@@ -0,0 +1,125 @@
+/*
+ * 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.drill.yarn.core;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.yarn.appMaster.Scheduler;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Describes a container request in terms of priority, memory, cores and
+ * placement preference. This is a simplified version of the YARN
+ * ContainerRequest structure. This structure is easier to use within the app,
+ * then is translated to the YARN structure when needed.
+ */
+
+public class ContainerRequestSpec {
+ static final Log LOG = LogFactory.getLog(ContainerRequestSpec.class);
+
+ /**
+ * Application-specific priority. Drill-on-Yarn uses the priority to associate
+ * YARN requests with a {@link Scheduler}. When the resource allocation
+ * arrives, we use the priority to trace back to the scheduler that requested
+ * it, and from there to the task to be run in the allocation.
+ * <p>
+ * For this reason, the priority is set by the Drill-on-YARN application; it
+ * is not a user-adjustable value.
+ */
+
+ public int priority = 0;
+
+ /**
+ * Memory, in MB, required by the container.
+ */
+
+ public int memoryMb;
+
+ /**
+ * Number of "virtual cores" required by the task. YARN allocates whole CPU
+ * cores and does not support fractional allocations.
+ */
+
+ public int vCores = 1;
+
+ /**
+ * Number of virtual disks (channels, spindles) to request. Not supported in
+ * Apache YARN, is supported in selected distributions.
+ */
+
+ public double disks;
+
+ /**
+ * Node label expression to apply to this request.
+ */
+
+ public String nodeLabelExpr;
+
+ public List<String> racks = new ArrayList<>();
+ public List<String> hosts = new ArrayList<>();
+
+ /**
+ * Create a YARN ContainerRequest object from the information in this object.
+ *
+ * @return
+ */
+ public ContainerRequest makeRequest() {
+ assert memoryMb != 0;
+
+ Priority priorityRec = Records.newRecord(Priority.class);
+ priorityRec.setPriority(priority);
+
+ Resource capability = Records.newRecord(Resource.class);
+ capability.setMemory(memoryMb);
+ capability.setVirtualCores(vCores);
+ DoYUtil.callSetDiskIfExists(capability, disks);
+
+ boolean relaxLocality = true;
+ String nodeArr[] = null;
+ if (!hosts.isEmpty()) {
+ nodeArr = new String[hosts.size()];
+ hosts.toArray(nodeArr);
+ relaxLocality = false;
+ }
+ String rackArr[] = null;
+ if (!racks.isEmpty()) {
+ nodeArr = new String[racks.size()];
+ racks.toArray(rackArr);
+ relaxLocality = false;
+ }
+ String nodeExpr = null;
+ if (!DoYUtil.isBlank(nodeLabelExpr)) {
+ nodeExpr = nodeLabelExpr;
+ LOG.info("Requesting a container using node expression: " + nodeExpr);
+ }
+
+ // YARN is fragile. To (potentially) pass a node expression, we must use the
+ // 5-argument constructor. The fourth argument (relax locality) MUST be set
+ // to true if we omit the rack and node specs. (Else we get a runtime
+ // error.
+
+ return new ContainerRequest(capability, nodeArr, rackArr, priorityRec,
+ relaxLocality, nodeExpr);
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/core/DfsFacade.java b/drill-yarn/src/main/java/org/apache/drill/yarn/core/DfsFacade.java
new file mode 100644
index 000000000..09e88ae61
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/core/DfsFacade.java
@@ -0,0 +1,345 @@
+/*
+ * 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.drill.yarn.core;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+
+import com.typesafe.config.Config;
+
+/**
+ * Facade to the distributed file system (DFS) system that implements
+ * Drill-on-YARN related operations. Some operations are used by both the client
+ * and AM applications.
+ */
+
+public class DfsFacade {
+ public static class DfsFacadeException extends Exception {
+ private static final long serialVersionUID = 1L;
+
+ public DfsFacadeException(String msg) {
+ super(msg);
+ }
+
+ public DfsFacadeException(String msg, Exception e) {
+ super(msg, e);
+ }
+ }
+
+ private FileSystem fs;
+ private Configuration yarnConf;
+ private Config config;
+ private boolean localize;
+
+ public DfsFacade(Config config) {
+ this.config = config;
+ localize = config.getBoolean(DrillOnYarnConfig.LOCALIZE_DRILL);
+ }
+
+ public boolean isLocalized() {
+ return localize;
+ }
+
+ public void connect() throws DfsFacadeException {
+ loadYarnConfig();
+ String dfsConnection = config.getString(DrillOnYarnConfig.DFS_CONNECTION);
+ try {
+ if (DoYUtil.isBlank(dfsConnection)) {
+ fs = FileSystem.get(yarnConf);
+ } else {
+ URI uri;
+ try {
+ uri = new URI(dfsConnection);
+ } catch (URISyntaxException e) {
+ throw new DfsFacadeException(
+ "Illformed DFS connection: " + dfsConnection, e);
+ }
+ fs = FileSystem.get(uri, yarnConf);
+ }
+ } catch (IOException e) {
+ throw new DfsFacadeException("Failed to create the DFS", e);
+ }
+ }
+
+ /**
+ * Lazy loading of YARN configuration since it takes a long time to load.
+ * (YARN provides no caching, sadly.)
+ */
+
+ private void loadYarnConfig() {
+ if (yarnConf == null) {
+ yarnConf = new YarnConfiguration();
+ // On some distributions, lack of proper configuration causes
+ // DFS to default to the local file system. So, a local file
+ // system generally means that the config is wrong, or running
+ // the wrong build of Drill for the user's environment.
+ URI fsUri = FileSystem.getDefaultUri( yarnConf );
+ if(fsUri.toString().startsWith("file:/")) {
+ System.err.println("Warning: Default DFS URI is for a local file system: " + fsUri);
+ }
+ }
+ }
+
+ public static class Localizer {
+ private final DfsFacade dfs;
+ protected File localArchivePath;
+ protected Path dfsArchivePath;
+ FileStatus fileStatus;
+ private String label;
+
+ /**
+ * Resources to be localized (downloaded) to each AM or drillbit node.
+ */
+
+ public Localizer(DfsFacade dfs, File archivePath, String label) {
+ this(dfs, archivePath, dfs.getUploadPath(archivePath), label);
+ }
+
+ public Localizer(DfsFacade dfs, File archivePath, String destName,
+ String label) {
+ this(dfs, archivePath, dfs.getUploadPath(destName), label);
+ }
+
+ public Localizer(DfsFacade dfs, String destPath) {
+ this( dfs, null, new Path(destPath), null );
+ }
+
+ public Localizer(DfsFacade dfs, File archivePath, Path destPath, String label) {
+ this.dfs = dfs;
+ dfsArchivePath = destPath;
+ this.label = label;
+ localArchivePath = archivePath;
+ }
+
+ public String getBaseName() {
+ return localArchivePath.getName();
+ }
+
+ public String getDestPath() {
+ return dfsArchivePath.toString();
+ }
+
+ public void upload() throws DfsFacadeException {
+ dfs.uploadArchive(localArchivePath, dfsArchivePath, label);
+ fileStatus = null;
+ }
+
+ /**
+ * The client may check file status multiple times. Cache it here so we
+ * only retrieve the status once. Cache it here so that the client
+ * doen't have to do the caching.
+ *
+ * @return
+ * @throws DfsFacadeException
+ */
+
+ private FileStatus getStatus() throws DfsFacadeException {
+ if (fileStatus == null) {
+ fileStatus = dfs.getFileStatus(dfsArchivePath);
+ }
+ return fileStatus;
+ }
+
+ public void defineResources(Map<String, LocalResource> resources,
+ String key) throws DfsFacadeException {
+ // Put the application archive, visible to only the application.
+ // Because it is an archive, it will be expanded by YARN prior to launch
+ // of the AM.
+
+ LocalResource drillResource = dfs.makeResource(dfsArchivePath,
+ getStatus(), LocalResourceType.ARCHIVE,
+ LocalResourceVisibility.APPLICATION);
+ resources.put(key, drillResource);
+ }
+
+ public boolean filesMatch() {
+ FileStatus status;
+ try {
+ status = getStatus();
+ } catch (DfsFacadeException e) {
+
+ // An exception is DFS's way of tell us the file does
+ // not exist.
+
+ return false;
+ }
+ return status.getLen() == localArchivePath.length();
+ }
+
+ public String getLabel() {
+ return label;
+ }
+
+ public boolean destExists() throws IOException {
+ return dfs.exists(dfsArchivePath);
+ }
+ }
+
+ public boolean exists(Path path) throws IOException {
+ return fs.exists(path);
+ }
+
+ public Path getUploadPath(File localArchiveFile) {
+ return getUploadPath(localArchiveFile.getName());
+ }
+
+ public Path getUploadPath(String baseName) {
+ String dfsDirStr = config.getString(DrillOnYarnConfig.DFS_APP_DIR);
+
+ Path appDir;
+ if (dfsDirStr.startsWith("/")) {
+ appDir = new Path(dfsDirStr);
+ } else {
+ Path home = fs.getHomeDirectory();
+ appDir = new Path(home, dfsDirStr);
+ }
+ return new Path(appDir, baseName);
+ }
+
+ public void uploadArchive(File localArchiveFile, Path destPath, String label)
+ throws DfsFacadeException {
+ // Create the application upload directory if it does not yet exist.
+
+ String dfsDirStr = config.getString(DrillOnYarnConfig.DFS_APP_DIR);
+ Path appDir = new Path(dfsDirStr);
+ try {
+ // If the directory does not exist, create it, giving this user
+ // (only) read and write access.
+
+ if (!fs.isDirectory(appDir)) {
+ fs.mkdirs(appDir, new FsPermission(FsAction.READ_WRITE, FsAction.NONE, FsAction.NONE));
+ }
+ } catch (IOException e) {
+ throw new DfsFacadeException(
+ "Failed to create DFS directory: " + dfsDirStr, e);
+ }
+
+ // The file must be an archive type so YARN knows to extract its contents.
+
+ String baseName = localArchiveFile.getName();
+ if (DrillOnYarnConfig.findSuffix(baseName) == null) {
+ throw new DfsFacadeException(
+ label + " archive must be .tar.gz, .tgz or .zip: " + baseName);
+ }
+
+ Path srcPath = new Path(localArchiveFile.getAbsolutePath());
+
+ // Do the upload, replacing the old archive.
+
+ try {
+ // TODO: Specify file permissions and owner.
+
+ fs.copyFromLocalFile(false, true, srcPath, destPath);
+ } catch (IOException e) {
+ throw new DfsFacadeException(
+ "Failed to upload " + label + " archive to DFS: "
+ + localArchiveFile.getAbsolutePath() + " --> " + destPath,
+ e);
+ }
+ }
+
+ private FileStatus getFileStatus(Path dfsPath) throws DfsFacadeException {
+ try {
+ return fs.getFileStatus(dfsPath);
+ } catch (IOException e) {
+ throw new DfsFacadeException(
+ "Failed to get DFS status for file: " + dfsPath, e);
+ }
+ }
+
+ /**
+ * Create a local resource definition for YARN. A local resource is one that
+ * must be localized onto the remote node prior to running a command on that
+ * node.
+ * <p>
+ * YARN uses the size and timestamp are used to check if the file has changed
+ * on HDFS to check if YARN can use an existing copy, if any.
+ * <p>
+ * Resources are made public.
+ *
+ * @param conf
+ * Configuration created from the Hadoop config files, in this case,
+ * identifies the target file system.
+ * @param resourcePath
+ * the path (relative or absolute) to the file on the configured file
+ * system (usually HDFS).
+ * @return a YARN local resource records that contains information about path,
+ * size, type, resource and so on that YARN requires.
+ * @throws IOException
+ * if the resource does not exist on the configured file system
+ */
+
+ public LocalResource makeResource(Path dfsPath, FileStatus dfsFileStatus,
+ LocalResourceType type, LocalResourceVisibility visibility)
+ throws DfsFacadeException {
+ URL destUrl;
+ try {
+ destUrl = ConverterUtils.getYarnUrlFromPath(
+ FileContext.getFileContext().makeQualified(dfsPath));
+ } catch (UnsupportedFileSystemException e) {
+ throw new DfsFacadeException(
+ "Unable to convert dfs file to a URL: " + dfsPath.toString(), e);
+ }
+ LocalResource resource = LocalResource.newInstance(destUrl, type,
+ visibility, dfsFileStatus.getLen(),
+ dfsFileStatus.getModificationTime());
+ return resource;
+ }
+
+ public void removeDrillFile(String fileName) throws DfsFacadeException {
+ Path destPath = getUploadPath(fileName);
+ try {
+ fs.delete(destPath, false);
+ } catch (IOException e) {
+ throw new DfsFacadeException(
+ "Failed to delete file: " + destPath.toString(), e);
+ }
+
+ // Remove the Drill directory, but only if it is now empty.
+
+ Path dir = destPath.getParent();
+ try {
+ RemoteIterator<FileStatus> iter = fs.listStatusIterator(dir);
+ if (!iter.hasNext()) {
+ fs.delete(dir, false);
+ }
+ } catch (IOException e) {
+ throw new DfsFacadeException(
+ "Failed to delete directory: " + dir.toString(), e);
+ }
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/core/DoYUtil.java b/drill-yarn/src/main/java/org/apache/drill/yarn/core/DoYUtil.java
new file mode 100644
index 000000000..3c1d17d08
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/core/DoYUtil.java
@@ -0,0 +1,189 @@
+/*
+ * 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.drill.yarn.core;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.security.CodeSource;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+import java.util.TimeZone;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.Container;
+
+public class DoYUtil {
+ static final private Log LOG = LogFactory.getLog(DoYUtil.class);
+
+ private DoYUtil() {
+ }
+
+ public static String join(String separator, List<String> list) {
+ StringBuilder buf = new StringBuilder();
+ String sep = "";
+ for (String item : list) {
+ buf.append(sep);
+ buf.append(item);
+ sep = separator;
+ }
+ return buf.toString();
+ }
+
+ public static void addNonEmpty(List<String> list, String value) {
+ if ( ! isBlank( value ) ) {
+ list.add(value.trim( ));
+ }
+ }
+
+ public static boolean isBlank(String str) {
+ return str == null || str.trim().isEmpty();
+ }
+
+ public static String toIsoTime(long timestamp) {
+
+ // Uses old-style dates rather than java.time because
+ // the code still must compile for JDK 7.
+
+ DateFormat fmt = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+ fmt.setTimeZone(TimeZone.getDefault());
+ return fmt.format(new Date(timestamp));
+ }
+
+ public static String labelContainer(Container container) {
+ StringBuilder buf = new StringBuilder()
+ .append("[id: ")
+ .append(container.getId())
+ .append(", host: ")
+ .append(container.getNodeId().getHost())
+ .append(", priority: ")
+ .append(container.getPriority())
+ .append("]");
+ return buf.toString();
+ }
+
+ /**
+ * Utility method to display YARN container information in a useful way for
+ * log messages.
+ *
+ * @param container
+ * @return
+ */
+
+ public static String describeContainer(Container container) {
+ StringBuilder buf = new StringBuilder()
+ .append("[id: ")
+ .append(container.getId())
+ .append(", host: ")
+ .append(container.getNodeId().getHost())
+ .append(", priority: ")
+ .append(container.getPriority())
+ .append(", memory: ")
+ .append(container.getResource().getMemory())
+ .append(" MB, vcores: ")
+ .append(container.getResource().getVirtualCores())
+ .append("]");
+ return buf.toString();
+ }
+
+ /**
+ * The tracking URL given to YARN is a redirect URL. When giving the URL to
+ * the user, "unwrap" that redirect URL to get the actual site URL.
+ *
+ * @param trackingUrl
+ * @return
+ */
+
+ public static String unwrapAmUrl(String trackingUrl) {
+ return trackingUrl.replace("/redirect", "/");
+ }
+
+ public static Object dynamicCall(Object target, String fnName, Object args[],
+ Class<?> types[]) {
+
+ // First, look for the method using the names and types provided.
+
+ final String methodLabel = target.getClass().getName() + "." + fnName;
+ Method m;
+ try {
+ m = target.getClass().getMethod(fnName, types);
+ } catch (NoSuchMethodException e) {
+
+ // Ignore, but log: the method does not exist in this distribution.
+
+ StringBuilder buf = new StringBuilder();
+ if (types != null) {
+ String sep = "";
+ for (Class<?> type : types) {
+ buf.append(sep);
+ buf.append(type.getName());
+ sep = ",";
+ }
+ }
+ LOG.trace("Not supported in this YARN distribution: " + methodLabel + "("
+ + buf.toString() + ")");
+ CodeSource src = target.getClass().getProtectionDomain().getCodeSource();
+ if (src != null) {
+ java.net.URL jar = src.getLocation();
+ LOG.trace("Class found in URL: " + jar.toString());
+ }
+ return null;
+ } catch (SecurityException e) {
+ LOG.error("Security prevents dynamic method calls", e);
+ return null;
+ }
+
+ // Next, call the method with the arguments provided.
+
+ Object ret = null;
+ try {
+ ret = m.invoke(target, args);
+ } catch (IllegalAccessException | IllegalArgumentException
+ | InvocationTargetException e) {
+ LOG.error("Failed to dynamically call " + methodLabel, e);
+ return null;
+ }
+ StringBuilder buf = new StringBuilder();
+ if (args != null) {
+ String sep = "";
+ for (Object arg : args) {
+ buf.append(sep);
+ buf.append(arg == null ? "null" : arg.toString());
+ sep = ",";
+ }
+ }
+ LOG.trace(
+ "Successfully called " + methodLabel + "( " + buf.toString() + ")");
+
+ // Return any return value. Will be null if the method is returns void.
+
+ return ret;
+ }
+
+ public static void callSetDiskIfExists(Object target, double arg) {
+ dynamicCall(target, "setDisks", new Object[] { arg },
+ new Class<?>[] { Double.TYPE });
+ }
+
+ public static double callGetDiskIfExists(Object target) {
+ Object ret = dynamicCall(target, "getDisks", null, null);
+ return (ret == null) ? 0.0 : (Double) ret;
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/core/DoyConfigException.java b/drill-yarn/src/main/java/org/apache/drill/yarn/core/DoyConfigException.java
new file mode 100644
index 000000000..422b89b22
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/core/DoyConfigException.java
@@ -0,0 +1,30 @@
+/*
+ * 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.drill.yarn.core;
+
+public class DoyConfigException extends Exception {
+ private static final long serialVersionUID = 1L;
+
+ public DoyConfigException(String msg) {
+ super(msg);
+ }
+
+ public DoyConfigException(String msg, Exception e) {
+ super(msg, e);
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/core/DrillOnYarnConfig.java b/drill-yarn/src/main/java/org/apache/drill/yarn/core/DrillOnYarnConfig.java
new file mode 100644
index 000000000..38ecd1c0e
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/core/DrillOnYarnConfig.java
@@ -0,0 +1,841 @@
+/*
+ * 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.drill.yarn.core;
+
+import java.io.File;
+import java.io.PrintStream;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.config.CommonConstants;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.scanner.ClassPathScanner;
+import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.ExecConstants;
+
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigException;
+import com.typesafe.config.ConfigFactory;
+
+/**
+ * Configuration used within the Drill-on-YARN code. Configuration comes from four
+ * sources (in order of precedence):
+ * <ol>
+ * <li>System properties</li>
+ * <li>$SITE_DIR/drill-on-yarn.conf</li>
+ * <li>Distribution-specific properties in $SITE_HOME/conf/doy-distrib.conf</li>
+ * <li>Drill-on-YARN defaults in drill-on-yarn-defaults.conf. (Which should be
+ * disjoint from the Drill settings.)</li>
+ * <li>Drill properties (via the Drill override system)</li>
+ * </ol>
+ * <p>
+ * Defines constants for each property, including some defined in Drill. This provides
+ * a uniform property access interface even if some properties migrate between DoY and
+ * Drill proper.
+ */
+
+public class DrillOnYarnConfig {
+ public static final String DEFAULTS_FILE_NAME = "drill-on-yarn-defaults.conf";
+ public static final String DISTRIB_FILE_NAME = "doy-distrib.conf";
+ public static final String CONFIG_FILE_NAME = "drill-on-yarn.conf";
+
+ public static final String DRILL_ON_YARN_PARENT = "drill.yarn";
+ public static final String DOY_CLIENT_PARENT = append(DRILL_ON_YARN_PARENT, "client");
+ public static final String DOY_AM_PARENT = append(DRILL_ON_YARN_PARENT, "am");
+ public static final String DOY_DRILLBIT_PARENT = append(DRILL_ON_YARN_PARENT, "drillbit");
+ public static final String FILES_PARENT = append(DRILL_ON_YARN_PARENT, "drill-install");
+ public static final String DFS_PARENT = append(DRILL_ON_YARN_PARENT, "dfs");
+ public static final String HTTP_PARENT = append(DRILL_ON_YARN_PARENT, "http");
+ public static final String YARN_PARENT = append(DRILL_ON_YARN_PARENT, "yarn");
+ public static final String HADOOP_PARENT = append(DRILL_ON_YARN_PARENT, "hadoop");
+ public static final String CLIENT_PARENT = append(DRILL_ON_YARN_PARENT, "client");
+
+ public static final String APP_NAME = append(DRILL_ON_YARN_PARENT, "app-name");
+ public static final String CLUSTER_ID = ExecConstants.SERVICE_NAME;
+
+ public static final String DFS_CONNECTION = append(DFS_PARENT, "connection");
+ public static final String DFS_APP_DIR = append(DFS_PARENT, "app-dir");
+
+ public static final String YARN_QUEUE = append(YARN_PARENT, "queue");
+ public static final String YARN_PRIORITY = append(YARN_PARENT, "priority");
+
+ public static final String DRILL_ARCHIVE_PATH = append(FILES_PARENT, "client-path");
+ public static final String DRILL_DIR_NAME = append(FILES_PARENT, "dir-name");
+
+ /**
+ * Key used for the Drill archive file in the AM launch config.
+ */
+
+ public static final String DRILL_ARCHIVE_KEY = append(FILES_PARENT, "drill-key");
+ public static final String SITE_ARCHIVE_KEY = append(FILES_PARENT, "site-key");
+ public static final String LOCALIZE_DRILL = append(FILES_PARENT, "localize");
+ public static final String CONF_AS_SITE = append(FILES_PARENT, "conf-as-site");
+ public static final String DRILL_HOME = append(FILES_PARENT, "drill-home");
+ public static final String SITE_DIR = append(FILES_PARENT, "site-dir");
+ public static final String JAVA_LIB_PATH = append(FILES_PARENT, "library-path");
+
+ public static final String HADOOP_HOME = append(HADOOP_PARENT, "home");
+ public static final String HADOOP_CLASSPATH = append(HADOOP_PARENT, "class-path");
+ public static final String HBASE_CLASSPATH = append(HADOOP_PARENT, "hbase-class-path");
+
+ public static final String MEMORY_KEY = "memory-mb";
+ public static final String VCORES_KEY = "vcores";
+ public static final String DISKS_KEY = "disks";
+ public static final String VM_ARGS_KEY = "vm-args";
+ public static final String HEAP_KEY = "heap";
+
+ public static final String AM_MEMORY = append(DOY_AM_PARENT, MEMORY_KEY);
+ public static final String AM_VCORES = append(DOY_AM_PARENT, VCORES_KEY);
+ public static final String AM_DISKS = append(DOY_AM_PARENT, DISKS_KEY);
+ public static final String AM_NODE_LABEL_EXPR = append(DOY_AM_PARENT, "node-label-expr");
+ public static final String AM_HEAP = append(DOY_AM_PARENT, HEAP_KEY);
+ public static final String AM_VM_ARGS = append(DOY_AM_PARENT, VM_ARGS_KEY);
+ public static final String AM_POLL_PERIOD_MS = append(DOY_AM_PARENT, "poll-ms");
+ public static final String AM_TICK_PERIOD_MS = append(DOY_AM_PARENT, "tick-ms");
+ public static final String AM_PREFIX_CLASSPATH = append(DOY_AM_PARENT, "prefix-class-path");
+ public static final String AM_CLASSPATH = append(DOY_AM_PARENT, "class-path");
+ public static final String AM_DEBUG_LAUNCH = append(DOY_AM_PARENT, "debug-launch");
+ public static final String AM_ENABLE_AUTO_SHUTDOWN = append(DOY_AM_PARENT, "auto-shutdown");
+
+ public static final String DRILLBIT_MEMORY = append(DOY_DRILLBIT_PARENT, MEMORY_KEY);
+ public static final String DRILLBIT_VCORES = append(DOY_DRILLBIT_PARENT, VCORES_KEY);
+ public static final String DRILLBIT_DISKS = append(DOY_DRILLBIT_PARENT, DISKS_KEY);
+ public static final String DRILLBIT_VM_ARGS = append(DOY_DRILLBIT_PARENT, VM_ARGS_KEY);
+ public static final String DRILLBIT_HEAP = append(DOY_DRILLBIT_PARENT, HEAP_KEY);
+ public static final String DRILLBIT_DIRECT_MEM = append(DOY_DRILLBIT_PARENT, "max-direct-memory");
+ public static final String DRILLBIT_CODE_CACHE = append(DOY_DRILLBIT_PARENT, "code-cache");
+ public static final String DRILLBIT_LOG_GC = append(DOY_DRILLBIT_PARENT, "log-gc");
+ public static final String DRILLBIT_PREFIX_CLASSPATH = append( DOY_DRILLBIT_PARENT, "prefix-class-path");
+ public static final String DRILLBIT_EXTN_CLASSPATH = append( DOY_DRILLBIT_PARENT, "extn-class-path");
+ public static final String DRILLBIT_CLASSPATH = append(DOY_DRILLBIT_PARENT, "class-path");
+ public static final String DRILLBIT_MAX_RETRIES = append(DOY_DRILLBIT_PARENT, "max-retries");
+ public static final String DRILLBIT_DEBUG_LAUNCH = append(DOY_DRILLBIT_PARENT, "debug-launch");
+ public static final String DRILLBIT_HTTP_PORT = ExecConstants.HTTP_PORT;
+ public static final String DISABLE_YARN_LOGS = append(DOY_DRILLBIT_PARENT, "disable-yarn-logs");
+ public static final String DRILLBIT_USER_PORT = ExecConstants.INITIAL_USER_PORT;
+ public static final String DRILLBIT_BIT_PORT = ExecConstants.INITIAL_BIT_PORT;
+ public static final String DRILLBIT_USE_HTTPS = ExecConstants.HTTP_ENABLE_SSL;
+ public static final String DRILLBIT_MAX_EXTRA_NODES = append(DOY_DRILLBIT_PARENT, "max-extra-nodes");
+ public static final String DRILLBIT_REQUEST_TIMEOUT_SEC = append(DOY_DRILLBIT_PARENT, "request-timeout-secs");
+
+ public static final String ZK_CONNECT = ExecConstants.ZK_CONNECTION;
+ public static final String ZK_ROOT = ExecConstants.ZK_ROOT;
+ public static final String ZK_FAILURE_TIMEOUT_MS = ExecConstants.ZK_TIMEOUT;
+ public static final String ZK_RETRY_COUNT = ExecConstants.ZK_RETRY_TIMES;
+ public static final String ZK_RETRY_DELAY_MS = ExecConstants.ZK_RETRY_DELAY;
+
+ // Names selected to be parallel to Drillbit HTTP config.
+
+ public static final String HTTP_ENABLED = append(HTTP_PARENT, "enabled");
+ public static final String HTTP_ENABLE_SSL = append(HTTP_PARENT, "ssl-enabled");
+ public static final String HTTP_PORT = append(HTTP_PARENT, "port");
+ public static final String HTTP_AUTH_TYPE = append(HTTP_PARENT, "auth-type");
+ public static final String HTTP_REST_KEY = append(HTTP_PARENT, "rest-key");
+ public static final String HTTP_SESSION_MAX_IDLE_SECS = append(HTTP_PARENT, "session-max-idle-secs");
+ public static final String HTTP_DOCS_LINK = append(HTTP_PARENT, "docs-link");
+ public static final String HTTP_REFRESH_SECS = append(HTTP_PARENT, "refresh-secs");
+ public static final String HTTP_USER_NAME = append(HTTP_PARENT, "user-name");
+ public static final String HTTP_PASSWORD = append(HTTP_PARENT, "password");
+
+ public static final String AUTH_TYPE_NONE = "none";
+ public static final String AUTH_TYPE_DRILL = "drill";
+ public static final String AUTH_TYPE_SIMPLE = "simple";
+
+ public static final String CLIENT_POLL_SEC = append(CLIENT_PARENT, "poll-sec");
+ public static final String CLIENT_START_WAIT_SEC = append(CLIENT_PARENT, "start-wait-sec");
+ public static final String CLIENT_STOP_WAIT_SEC = append(CLIENT_PARENT, "stop-wait-sec");
+
+ public static final String CLUSTERS = append(DRILL_ON_YARN_PARENT, "cluster");
+
+ /**
+ * Name of the subdirectory of the container directory that will hold
+ * localized Drill distribution files. This name must be consistent between AM
+ * launch request and AM launch, and between Drillbit launch request and
+ * Drillbit launch. This name is fixed; there is no reason for the user to
+ * change it as it is visible only in the YARN container environment.
+ */
+
+ public static String LOCAL_DIR_NAME = "drill";
+
+ // Environment variables used to pass information from the Drill-on-YARN
+ // Client to the AM, or from the AM to the Drillbit launch script.
+
+ public static final String APP_ID_ENV_VAR = "DRILL_AM_APP_ID";
+ public static final String DRILL_ARCHIVE_ENV_VAR = "DRILL_ARCHIVE";
+ public static final String SITE_ARCHIVE_ENV_VAR = "SITE_ARCHIVE";
+ public static final String DRILL_HOME_ENV_VAR = "DRILL_HOME";
+ public static final String DRILL_SITE_ENV_VAR = "DRILL_CONF_DIR";
+ public static final String AM_HEAP_ENV_VAR = "DRILL_AM_HEAP";
+ public static final String AM_JAVA_OPTS_ENV_VAR = "DRILL_AM_JAVA_OPTS";
+ public static final String DRILL_CLASSPATH_ENV_VAR = "DRILL_CLASSPATH";
+ public static final String DRILL_CLASSPATH_PREFIX_ENV_VAR = "DRILL_CLASSPATH_PREFIX";
+ public static final String DOY_LIBPATH_ENV_VAR = "DOY_JAVA_LIB_PATH";
+ public static final String DRILL_DEBUG_ENV_VAR = "DRILL_DEBUG";
+
+ /**
+ * Special value for the DRILL_DIR_NAME parameter to indicate to use the base
+ * name of the archive as the Drill home path.
+ */
+
+ private static final Object BASE_NAME_MARKER = "<base>";
+
+ /**
+ * The name of the Drill site archive stored in dfs. Since the archive is
+ * created by the client as a temp file, it's local name has no meaning; we
+ * use this standard name on dfs.
+ */
+
+ public static final String SITE_ARCHIVE_NAME = "site.tar.gz";
+
+ protected static DrillOnYarnConfig instance;
+ private File drillSite;
+ private File drillHome;
+ private static DrillConfig drillConfig;
+ private Config config;
+ private ScanResult classPathScan;
+
+ public static String append(String parent, String key) {
+ return parent + "." + key;
+ }
+
+ // Protected only to allow creating a test version of this class.
+
+ protected DrillOnYarnConfig( ) {
+ }
+
+ public static DrillOnYarnConfig load() throws DoyConfigException {
+ instance = new DrillOnYarnConfig();
+ instance.doLoad(Thread.currentThread().getContextClassLoader());
+ return instance;
+ }
+
+ /**
+ * Load the config.
+ * @param cl class loader to use for resource searches (except defaults).
+ * Allows test to specify a specialized version.
+ * <p>
+ * Implemented in a way that allows unit testing. The parseUrl( ) methods
+ * let us mock the files; the load( ) methods seem to not actually use the
+ * provided class loader.
+ *
+ * @throws DoyConfigException
+ */
+ protected void doLoad(ClassLoader cl) throws DoyConfigException {
+ Config drillConfig = loadDrillConfig();
+
+ // Resolution order, larger numbers take precedence.
+ // 1. Drill-on-YARN defaults.
+ // File is at root of the package tree.
+
+ URL url = DrillOnYarnConfig.class.getResource(DEFAULTS_FILE_NAME);
+ if (url == null) {
+ throw new IllegalStateException(
+ "Drill-on-YARN defaults file is required: " + DEFAULTS_FILE_NAME);
+ }
+ config = ConfigFactory.parseURL(url).withFallback(drillConfig);
+
+ // 2. Optional distribution-specific configuration-file.
+ // (Lets a vendor, for example, specify the default DFS upload location
+ // without tinkering with the user's own settings.
+
+ url = cl.getResource(DISTRIB_FILE_NAME);
+ if (url != null) {
+ config = ConfigFactory.parseURL(url).withFallback(config);
+ }
+
+ // 3. User's Drill-on-YARN configuration.
+ // Optional since defaults are fine & ZK comes from drill-override.conf.
+
+ url = cl.getResource(CONFIG_FILE_NAME);
+ if (url != null) {
+ config = ConfigFactory.parseURL(url).withFallback(config);
+ }
+
+ // 4. System properties
+ // Allows -Dfoo=bar on the command line.
+ // But, note that substitutions are NOT allowed in system properties!
+
+ config = ConfigFactory.systemProperties().withFallback(config);
+
+ // Resolution allows ${foo.bar} syntax in values, but only for values
+ // from config files, not from system properties.
+
+ config = config.resolve();
+ }
+
+ private static Config loadDrillConfig() {
+ drillConfig = DrillConfig
+ .create(CommonConstants.CONFIG_OVERRIDE_RESOURCE_PATHNAME);
+ return drillConfig.resolve();
+ }
+
+ public DrillConfig getDrillConfig() {
+ return drillConfig;
+ }
+
+ /**
+ * Return Drill's class path scan. This is used only in the main thread during
+ * initialization. Not needed by the client, so done in an unsynchronized,
+ * lazy fashion.
+ *
+ * @return
+ */
+
+ public ScanResult getClassPathScan() {
+ if (classPathScan == null) {
+ classPathScan = ClassPathScanner.fromPrescan(drillConfig);
+ }
+ return classPathScan;
+ }
+
+ /**
+ * Obtain Drill home from the DRILL_HOME environment variable set by
+ * drill-config.sh, which is called from drill-on-yarn.sh. When debugging,
+ * DRILL_HOME must be set in the environment.
+ * <p>
+ * This information is required only by the client to prepare for uploads to
+ * DFS.
+ *
+ * @throws DoyConfigException
+ */
+
+ public void setClientPaths() throws DoyConfigException {
+ setClientDrillHome();
+ setSiteDir();
+ }
+
+ private void setClientDrillHome() throws DoyConfigException {
+ // Try the environment variable that should have been
+ // set in drill-on-yarn.sh (for the client) or in the
+ // launch environment (for the AM.)
+
+ String homeDir = getEnv(DRILL_HOME_ENV_VAR);
+
+ // For ease in debugging, allow setting the Drill home in
+ // drill-on-yarn.conf.
+ // This setting is also used for a non-localized run.
+
+ if (DoYUtil.isBlank(homeDir)) {
+ homeDir = config.getString(DRILL_HOME);
+ }
+ if (DoYUtil.isBlank(homeDir)) {
+ throw new DoyConfigException(
+ "The DRILL_HOME environment variable must point to your Drill install.");
+ }
+ drillHome = new File(homeDir);
+ }
+
+ /**
+ * All environment variable access goes through this function to allow unit
+ * tests to replace this function to set test values. (The Java environment is
+ * immutable, so it is not possible for unit tests to change the actual
+ * environment.)
+ *
+ * @param key
+ * @return
+ */
+
+ protected String getEnv(String key) {
+ return System.getenv(key);
+ }
+
+ /**
+ * On both the client and the AM, the site directory is optional. If provided,
+ * it was set with the --config (or --site) option to the script that launched
+ * the client or AM. In both cases, the script sets the drill.yarn.siteDir
+ * system property (and leaks the DRILL_HOME environment variable.)
+ * <p>
+ * For ease of debugging, if neither of those are set, this method uses the
+ * location of the drill-on-yarn configuration file to infer the site
+ * directory.
+ * <p>
+ * On the client, the site directory will be the "original" directory that
+ * contains the user's "master" files. On the AM, the site directory is a
+ * localized version of the client directory. Because of the way tar works,
+ * both the client and AM site directories have the same name; though the path
+ * to that name obviously differs.
+ *
+ * @throws DoyConfigException
+ */
+
+ private void setSiteDir() throws DoyConfigException {
+ // The site directory is the one where the config file lives.
+ // This should have been set in an environment variable by the launch
+ // script.
+
+ String sitePath = getEnv("DRILL_CONF_DIR");
+ if (!DoYUtil.isBlank(sitePath)) {
+ drillSite = new File(sitePath);
+ } else {
+
+ // Otherwise, let's guess it from the config file. This version assists
+ // in debugging as it reduces setup steps.
+
+ ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
+ if (classLoader == null) {
+ classLoader = DrillOnYarnConfig.class.getClassLoader();
+ }
+
+ URL url = classLoader.getResource(CommonConstants.CONFIG_OVERRIDE_RESOURCE_PATHNAME);
+ if (url == null) {
+ throw new DoyConfigException(
+ "Drill configuration file is missing: " + CommonConstants.CONFIG_OVERRIDE_RESOURCE_PATHNAME);
+ }
+ File confFile;
+ try {
+ java.nio.file.Path confPath = Paths.get(url.toURI());
+ confFile = confPath.toFile();
+ } catch (URISyntaxException e) {
+ throw new DoyConfigException(
+ "Invalid path to Drill-on-YARN configuration file: "
+ + url.toString(),
+ e);
+ }
+ drillSite = confFile.getParentFile();
+ }
+
+ // Verify that the site directory is not just $DRILL_HOME/conf.
+ // Since the calling script does not differentiate between the two cases.
+ // But, treat $DRILL_HOME/conf as the site directory if:
+ // 1. The conf-as-site property is true, or
+ // 2. The Drill archive resides within $DRILL_HOME.
+ //
+ // The above situations occur in certain distributions that
+ // ship the archive inside the site directory and don't use a
+ // site directory.
+
+ if (drillHome.equals(drillSite.getParentFile())
+ && !config.getBoolean(CONF_AS_SITE)) {
+ String drillArchivePath = config
+ .getString(DrillOnYarnConfig.DRILL_ARCHIVE_PATH);
+ if (!DoYUtil.isBlank(drillArchivePath)) {
+ File archiveFile = new File(drillArchivePath);
+ if (!archiveFile.isAbsolute() && !archiveFile.getAbsolutePath()
+ .startsWith(drillHome.getAbsolutePath())) {
+ drillSite = null;
+ }
+ }
+ }
+ }
+
+ /**
+ * Retrieve the AM Drill home location from the DRILL_HOME variable set in the
+ * drill-am.sh launch script.
+ *
+ * @throws DoyConfigException
+ */
+
+ public void setAmDrillHome() throws DoyConfigException {
+ String drillHomeStr = getEnv(DRILL_HOME_ENV_VAR);
+ drillHome = new File(drillHomeStr);
+ setSiteDir();
+ }
+
+ public Config getConfig() {
+ return instance.config;
+ }
+
+ public static DrillOnYarnConfig instance() {
+ assert instance != null;
+ return instance;
+ }
+
+ public static Config config() {
+ return instance().getConfig();
+ }
+
+ /**
+ * Return the Drill home on this machine as inferred from the config file
+ * contents or location.
+ *
+ * @return
+ */
+
+ public File getLocalDrillHome() {
+ return drillHome;
+ }
+
+ public void dump() {
+ dump(System.out);
+ }
+
+ private static final String keys[] = {
+ // drill.yarn
+
+ APP_NAME,
+ CLUSTER_ID,
+
+ // drill.yarn.dfs
+
+ DFS_CONNECTION,
+ DFS_APP_DIR,
+
+ // drill.yarn.hadoop
+
+ HADOOP_HOME,
+ HADOOP_CLASSPATH,
+ HBASE_CLASSPATH,
+
+ // drill.yarn.yarn
+
+ YARN_QUEUE,
+ YARN_PRIORITY,
+
+ // drill.yarn.drill-install
+
+ DRILL_ARCHIVE_PATH,
+ DRILL_DIR_NAME,
+ LOCALIZE_DRILL,
+ CONF_AS_SITE,
+ DRILL_HOME,
+ DRILL_ARCHIVE_KEY,
+ SITE_ARCHIVE_KEY,
+ JAVA_LIB_PATH,
+
+ // drill.yarn.client
+
+ CLIENT_POLL_SEC,
+ CLIENT_START_WAIT_SEC,
+ CLIENT_STOP_WAIT_SEC,
+
+ // drill.yarn.am
+
+ AM_MEMORY,
+ AM_VCORES,
+ AM_DISKS,
+ AM_NODE_LABEL_EXPR,
+ AM_VM_ARGS,
+ AM_HEAP,
+ AM_POLL_PERIOD_MS,
+ AM_TICK_PERIOD_MS,
+ AM_PREFIX_CLASSPATH,
+ AM_CLASSPATH,
+ AM_DEBUG_LAUNCH,
+ AM_ENABLE_AUTO_SHUTDOWN,
+
+ // drill.yarn.zk
+
+ ZK_CONNECT,
+ ZK_ROOT,
+ ZK_RETRY_COUNT,
+ ZK_RETRY_DELAY_MS,
+ ZK_FAILURE_TIMEOUT_MS,
+
+ // drill.yarn.drillbit
+
+ DRILLBIT_MEMORY,
+ DRILLBIT_VCORES,
+ DRILLBIT_DISKS,
+ DRILLBIT_VM_ARGS,
+ DRILLBIT_HEAP,
+ DRILLBIT_DIRECT_MEM,
+ DRILLBIT_CODE_CACHE,
+ DRILLBIT_PREFIX_CLASSPATH,
+ DRILLBIT_EXTN_CLASSPATH,
+ DRILLBIT_CLASSPATH,
+ DRILLBIT_MAX_RETRIES,
+ DRILLBIT_DEBUG_LAUNCH,
+ DRILLBIT_MAX_EXTRA_NODES,
+ DRILLBIT_REQUEST_TIMEOUT_SEC,
+ DISABLE_YARN_LOGS,
+ DRILLBIT_HTTP_PORT,
+ DRILLBIT_USER_PORT,
+ DRILLBIT_BIT_PORT,
+ DRILLBIT_USE_HTTPS,
+
+ // drill.yarn.http
+
+ HTTP_ENABLED,
+ HTTP_ENABLE_SSL,
+ HTTP_PORT,
+ HTTP_AUTH_TYPE,
+ HTTP_SESSION_MAX_IDLE_SECS,
+ HTTP_DOCS_LINK,
+ HTTP_REFRESH_SECS,
+ // Do not include AM_REST_KEY: it is supposed to be secret.
+ // Same is true of HTTP_USER_NAME and HTTP_PASSWORD
+ };
+
+ private static String envVars[] = {
+ APP_ID_ENV_VAR,
+ DRILL_HOME_ENV_VAR,
+ DRILL_SITE_ENV_VAR,
+ AM_HEAP_ENV_VAR,
+ AM_JAVA_OPTS_ENV_VAR,
+ DRILL_CLASSPATH_PREFIX_ENV_VAR,
+ DRILL_CLASSPATH_ENV_VAR,
+ DRILL_ARCHIVE_ENV_VAR,
+ SITE_ARCHIVE_ENV_VAR,
+ DRILL_DEBUG_ENV_VAR
+ };
+
+ private void dump(PrintStream out) {
+ for (String key : keys) {
+ out.print(key);
+ out.print(" = ");
+ try {
+ out.println(config.getString(key));
+ } catch (ConfigException.Missing e) {
+ out.println("<missing>");
+ }
+ }
+ out.print(CLUSTERS);
+ out.println("[");
+ for (int i = 0; i < clusterGroupCount(); i++) {
+ ClusterDef.ClusterGroup cluster = ClusterDef.getCluster(config, i);
+ out.print(i);
+ out.println(" = {");
+ cluster.dump(" ", out);
+ out.println(" }");
+ }
+ out.println("]");
+ }
+
+ public void dumpEnv(PrintStream out) {
+ out.print("environment");
+ out.println("[");
+ for (String envVar : envVars) {
+ String value = getEnv(envVar);
+ out.print(envVar);
+ out.print(" = ");
+ if (value == null) {
+ out.print("<unset>");
+ } else {
+ out.print("\"");
+ out.print(value);
+ out.print("\"");
+ }
+ out.println();
+ }
+ out.println("]");
+ }
+
+ public List<NameValuePair> getPairs() {
+ List<NameValuePair> pairs = new ArrayList<>();
+ for (String key : keys) {
+ pairs.add(new NameValuePair(key, config.getString(key)));
+ }
+ for (int i = 0; i < clusterGroupCount(); i++) {
+ ClusterDef.ClusterGroup pool = ClusterDef.getCluster(config, i);
+ pool.getPairs(i, pairs);
+ }
+
+ // Add environment variables as "pseudo" properties,
+ // prefixed with "envt.".
+
+ for (String envVar : envVars) {
+ pairs.add(new NameValuePair("envt." + envVar, getEnv(envVar)));
+ }
+ return pairs;
+ }
+
+ public static String clusterGroupKey(int index, String key) {
+ return CLUSTERS + "." + index + "." + key;
+ }
+
+ public int clusterGroupCount() {
+ return config.getList(CLUSTERS).size();
+ }
+
+ private static String suffixes[] = { ".tar.gz", ".tgz", ".zip" };
+
+ public static String findSuffix(String baseName) {
+ baseName = baseName.toLowerCase();
+ for (String extn : suffixes) {
+ if (baseName.endsWith(extn)) {
+ return extn;
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Get the location of Drill home on a remote machine, relative to the
+ * container working directory. Used when constructing a launch context.
+ * Assumes either the absolute path from the config file, or a constructed
+ * path to the localized Drill on the remote node. YARN examples use "./foo"
+ * to refer to container resources. But, since we cannot be sure when such a
+ * path is evaluated, we explicitly use YARN's PWD environment variable to get
+ * the absolute path.
+ *
+ * @return the remote path, with the "$PWD" environment variable.
+ * @throws DoyConfigException
+ */
+
+ public String getRemoteDrillHome() throws DoyConfigException {
+ // If the application is not localized, then the user can tell us the remote
+ // path in the config file. Otherwise, we assume that the remote path is the
+ // same as the local path.
+
+ if (!config.getBoolean(LOCALIZE_DRILL)) {
+ String drillHomePath = config.getString(DRILL_HOME);
+ if (DoYUtil.isBlank(drillHomePath)) {
+ drillHomePath = drillHome.getAbsolutePath();
+ }
+ return drillHomePath;
+ }
+
+ // The application is localized. Work out the location within the container
+ // directory. The path starts with the "key" we specify when uploading the
+ // Drill archive; YARN expands the archive into a folder of that name.
+
+ String drillHome = "$PWD/" + config.getString(DRILL_ARCHIVE_KEY);
+
+ String home = config.getString(DRILL_DIR_NAME);
+ if (DoYUtil.isBlank(home)) {
+ // Assume the archive expands without a subdirectory.
+ }
+
+ // If the special "<base>" marker is used, assume that the path depends
+ // on the name of the archive, which we know from the config file.
+
+ else if (home.equals(BASE_NAME_MARKER)) {
+
+ // Otherwise, assume that the archive expands to a directory with the
+ // same name as the archive itself (minus the archive suffix.)
+
+ String drillArchivePath = config
+ .getString(DrillOnYarnConfig.DRILL_ARCHIVE_PATH);
+ if (DoYUtil.isBlank(drillArchivePath)) {
+ throw new DoyConfigException("Required config property not set: "
+ + DrillOnYarnConfig.DRILL_ARCHIVE_PATH);
+ }
+ File localArchiveFile = new File(drillArchivePath);
+ home = localArchiveFile.getName();
+ String suffix = findSuffix(home);
+ if (suffix == null) {
+ throw new DoyConfigException(DrillOnYarnConfig.DRILL_ARCHIVE_PATH
+ + " does not name a valid archive: " + drillArchivePath);
+ }
+ drillHome += "/" + home.substring(0, home.length() - suffix.length());
+ } else {
+ // If the user told us the name of the directory within the archive,
+ // use it.
+
+ drillHome += "/" + home;
+ }
+ return drillHome;
+ }
+
+ /**
+ * Get the optional remote site directory name. This name will include the
+ * absolute path for a non-localized application. It will return the path
+ * relative to the container for a localized application. In the localized
+ * case, the site archive is tar'ed relative to the site directory so that its
+ * contents are unarchived directly into the YARN-provided folder (with the
+ * name of the archive) key. That is, if the site directory on the client is
+ * /var/drill/my-site, the contents of the tar file will be
+ * "./drill-override.conf", etc., and the remote location is
+ * $PWD/site-key/drill-override.conf, where site-key is the key name used to
+ * localize the site archive.
+ *
+ * @return
+ */
+
+ public String getRemoteSiteDir() {
+ // If the application does not use a site directory, then return null.
+
+ if (!hasSiteDir()) {
+ return null;
+ }
+
+ // If the application is not localized, then use the remote site path
+ // provided in the config file. Otherwise, assume that the remote path
+ // is the same as the local path.
+
+ if (!config.getBoolean(LOCALIZE_DRILL)) {
+ String drillSitePath = config.getString(SITE_DIR);
+ if (DoYUtil.isBlank(drillSitePath)) {
+ drillSitePath = drillSite.getAbsolutePath();
+ }
+ return drillSitePath;
+ }
+
+ // Work out the site directory name as above for the Drill directory.
+ // The caller must include a archive subdirectory name if required.
+
+ return "$PWD/" + config.getString(SITE_ARCHIVE_KEY);
+ }
+
+ /**
+ * Return the app ID file to use for this client run. The file is in the
+ * directory that holds the site dir (if a site dir is used), else the
+ * directory that holds Drill home (otherwise.) Not that the file does NOT go
+ * into the site dir or Drill home as we upload these directories (via
+ * archives) to DFS so we don't want to change them by adding a file.
+ * <p>
+ * It turns out that Drill allows two distinct clusters to share the same ZK
+ * root and/or cluster ID (just not the same combination), so the file name
+ * contains both parts.
+ *
+ * @param clusterId
+ * @return
+ */
+
+ public File getLocalAppIdFile() {
+ String rootDir = config.getString(DrillOnYarnConfig.ZK_ROOT);
+ String clusterId = config.getString(DrillOnYarnConfig.CLUSTER_ID);
+ String key = rootDir + "-" + clusterId;
+ String appIdFileName = key + ".appid";
+ File appIdDir;
+ if (hasSiteDir()) {
+ appIdDir = drillSite.getParentFile();
+ } else {
+ appIdDir = drillHome.getParentFile();
+ }
+ return new File(appIdDir, appIdFileName);
+ }
+
+ public boolean hasSiteDir() {
+ return drillSite != null;
+ }
+
+ public File getLocalSiteDir() {
+ return drillSite;
+ }
+
+ /**
+ * Returns the DFS path to the localized Drill archive. This is an AM-only
+ * method as it relies on an environment variable set by the client. It is set
+ * only if the application is localized, it is not set for a non-localized
+ * run.
+ *
+ * @return
+ */
+
+ public String getDrillArchiveDfsPath() {
+ return getEnv(DrillOnYarnConfig.DRILL_ARCHIVE_ENV_VAR);
+ }
+
+ /**
+ * Returns the DFS path to the localized site archive. This is an AM-only
+ * method as it relies on an environment variable set by the client. This
+ * variable is optional; if not set then the AM can infer that the application
+ * does not use a site archive (configuration files reside in
+ * $DRILL_HOME/conf), or the application is not localized.
+ *
+ * @return
+ */
+
+ public String getSiteArchiveDfsPath() {
+ return getEnv(DrillOnYarnConfig.SITE_ARCHIVE_ENV_VAR);
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/core/LaunchSpec.java b/drill-yarn/src/main/java/org/apache/drill/yarn/core/LaunchSpec.java
new file mode 100644
index 000000000..6c2287478
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/core/LaunchSpec.java
@@ -0,0 +1,248 @@
+/*
+ * 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.drill.yarn.core;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Abstract description of a remote process launch that describes the many
+ * details needed to launch a process on a remote node.
+ * <p>
+ * Based on <a href="https://github.com/hortonworks/simple-yarn-app">Simple YARN
+ * App</a>.
+ */
+
+public class LaunchSpec {
+ /**
+ * List of (key, file) pairs to be localized to the node before running the
+ * command. The file must exist in a distributed file system (such as HDFS)
+ * visible to both the client and remote node. Typically, the path is relative
+ * or absolute within the file system defined by the fs.defaultFS parameter in
+ * core-site.xml.
+ * <p>
+ * TODO: Can the value also be a URL such as
+ * <p>
+ * <code>hdfs://somehost:1234//path/to/file
+ * <p>
+ * The key is used as (what?).
+ */
+
+ public Map<String, LocalResource> resources = new HashMap<>();
+
+ /**
+ * Defines environment variables to be set on the remote host before launching
+ * the remote app. Note: do not set CLASSPATH here; use {@link #classPath}
+ * instead.
+ */
+
+ public Map<String, String> env = new HashMap<>();
+
+ /**
+ * Set to the name of the OS command to run when we wish to run a non-Java
+ * command.
+ */
+
+ public String command;
+
+ /**
+ * Set to the name of the Java main class (the one with the main method) when
+ * we wish to run a Java command.
+ */
+
+ public String mainClass;
+
+ /**
+ * Set to the application-specific class path for the Java application. These
+ * values are added to the Hadoop-provided values. These items are relative to
+ * (what?), use (what variables) to refer to the localized application
+ * directory.
+ */
+
+ public List<String> classPath = new ArrayList<>();
+
+ /**
+ * Optional VM arguments to pass to the JVM when running a Java class; ignored
+ * when running an OS command.
+ */
+
+ public List<String> vmArgs = new ArrayList<>();
+
+ /**
+ * Arguments to the remote command.
+ */
+
+ public List<String> cmdArgs = new ArrayList<>();
+
+ public LaunchSpec() {
+ }
+
+ /**
+ * Create the command line to run on the remote node. The command can either
+ * be a simple OS command (if the {@link #command} member is set) or can be a
+ * Java class (if the {@link #mainClass} member is set. If the command is
+ * Java, then we pass along optional Java VM arguments.
+ * <p>
+ * In all cases we append arguments to the command itself, and redirect stdout
+ * and stderr to log files.
+ *
+ * @return the complete command string
+ */
+
+ public String getCommand() {
+ List<String> cmd = new ArrayList<>();
+ if (command != null) {
+ cmd.add(command);
+ } else {
+ assert mainClass != null;
+
+ // JAVA_HOME is provided by YARN.
+
+ cmd.add(Environment.JAVA_HOME.$$() + "/bin/java");
+ cmd.addAll(vmArgs);
+ if (!classPath.isEmpty()) {
+ cmd.add("-cp");
+ cmd.add(DoYUtil.join(":", classPath));
+ }
+ cmd.add(mainClass);
+ }
+ cmd.addAll(cmdArgs);
+ cmd.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
+ cmd.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
+
+ // Java 8
+ // return String.join( " ", cmd );
+ return DoYUtil.join(" ", cmd);
+ }
+
+ /**
+ * Given this generic description of an application, create the detailed YARN
+ * application submission context required to launch the application.
+ *
+ * @param conf
+ * the YARN configuration obtained by reading the Hadoop
+ * configuration files
+ * @return the completed application launch context for the given application
+ * @throws IOException
+ * if localized resources are not found in the distributed file
+ * system (such as HDFS)
+ */
+
+ public ContainerLaunchContext createLaunchContext(YarnConfiguration conf)
+ throws IOException {
+ // Set up the container launch context
+ ContainerLaunchContext container = Records
+ .newRecord(ContainerLaunchContext.class);
+
+ // Set up the list of commands to run. Here, we assume that we run only
+ // one command.
+
+ container.setCommands(Collections.singletonList(getCommand()));
+
+ // Add localized resources
+
+ container.setLocalResources(resources);
+
+ // Environment.
+
+ container.setEnvironment(env);
+
+ return container;
+ }
+
+ public void dump(PrintStream out) {
+ if (command != null) {
+ out.print("Command: ");
+ out.println(command);
+ }
+ if (mainClass != null) {
+ out.print("Main Class: ");
+ out.println(mainClass);
+ out.println("VM Args:");
+ if (vmArgs.isEmpty()) {
+ out.println(" None");
+ } else {
+ for (String item : vmArgs) {
+ out.print(" ");
+ out.println(item);
+ }
+ }
+ out.println("Class Path:");
+ if (classPath.isEmpty()) {
+ out.println(" None");
+ } else {
+ for (String item : classPath) {
+ out.print(" ");
+ out.println(item);
+ }
+ }
+ }
+ out.println("Program Args:");
+ if (cmdArgs.isEmpty()) {
+ out.println(" None");
+ } else {
+ for (String item : cmdArgs) {
+ out.print(" ");
+ out.println(item);
+ }
+ }
+ out.println("Environment:");
+ if (env.isEmpty()) {
+ out.println(" None");
+ } else {
+ for (String key : env.keySet()) {
+ out.print(" ");
+ out.print(key);
+ out.print("=");
+ out.println(env.get(key));
+ }
+ }
+ out.println("Resources: ");
+ if (resources.isEmpty()) {
+ out.println(" None");
+ } else {
+ for (String key : resources.keySet()) {
+ out.print(" Key: ");
+ out.println(key);
+ LocalResource resource = resources.get(key);
+ out.print(" URL: ");
+ out.println(resource.getResource().toString());
+ out.print(" Size: ");
+ out.println(resource.getSize());
+ out.print(" Timestamp: ");
+ out.println(DoYUtil.toIsoTime(resource.getTimestamp()));
+ out.print(" Type: ");
+ out.println(resource.getType().toString());
+ out.print(" Visiblity: ");
+ out.println(resource.getVisibility().toString());
+ }
+ }
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/core/NameValuePair.java b/drill-yarn/src/main/java/org/apache/drill/yarn/core/NameValuePair.java
new file mode 100644
index 000000000..5872ab901
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/core/NameValuePair.java
@@ -0,0 +1,46 @@
+/*
+ * 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.drill.yarn.core;
+
+public class NameValuePair {
+ private String name;
+ private Object value;
+
+ public NameValuePair(String name, Object value) {
+ this.name = name;
+ this.value = value;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public Object getValue() {
+ return value;
+ }
+
+ public String getQuotedValue() {
+ if (value == null) {
+ return "<unset>";
+ }
+ if (value instanceof String) {
+ return "\"" + value + "\"";
+ }
+ return value.toString();
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/core/YarnClientException.java b/drill-yarn/src/main/java/org/apache/drill/yarn/core/YarnClientException.java
new file mode 100644
index 000000000..62dd46868
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/core/YarnClientException.java
@@ -0,0 +1,30 @@
+/*
+ * 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.drill.yarn.core;
+
+public class YarnClientException extends Exception {
+ private static final long serialVersionUID = -1411110715738266578L;
+
+ public YarnClientException(String msg) {
+ super(msg);
+ }
+
+ public YarnClientException(String msg, Exception e) {
+ super(msg, e);
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/core/YarnRMClient.java b/drill-yarn/src/main/java/org/apache/drill/yarn/core/YarnRMClient.java
new file mode 100644
index 000000000..8905ce32e
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/core/YarnRMClient.java
@@ -0,0 +1,207 @@
+/*
+ * 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.drill.yarn.core;
+
+import java.io.IOException;
+
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.client.api.YarnClientApplication;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+
+/**
+ * YARN resource manager client implementation for Drill. Provides a wrapper
+ * around the YARN client interface to the Resource Manager. Used by the client
+ * app to start the Drill application master.
+ * <p>
+ * Based on
+ * <a href="https://github.com/hortonworks/simple-yarn-app">simple-yarn-app</a>
+ */
+
+public class YarnRMClient {
+ private YarnConfiguration conf;
+ private YarnClient yarnClient;
+
+ /**
+ * Application ID. Semantics are such that each session of Drill-on-YARN works
+ * with no more than one application ID.
+ */
+
+ private ApplicationId appId;
+ private YarnClientApplication app;
+
+ public YarnRMClient() {
+ this(new YarnConfiguration());
+ }
+
+ public YarnRMClient(ApplicationId appId) {
+ this();
+ this.appId = appId;
+ }
+
+ public YarnRMClient(YarnConfiguration conf) {
+ this.conf = conf;
+ yarnClient = YarnClient.createYarnClient();
+ yarnClient.init(conf);
+ yarnClient.start();
+ }
+
+ public GetNewApplicationResponse createAppMaster()
+ throws YarnClientException {
+ // Create application via yarnClient
+ // Response is a new application ID along with cluster capacity info
+
+ try {
+ app = yarnClient.createApplication();
+ } catch (YarnException | IOException e) {
+ throw new YarnClientException("Create application failed", e);
+ }
+ GetNewApplicationResponse response = app.getNewApplicationResponse();
+ appId = response.getApplicationId();
+ return response;
+ }
+
+ public void submitAppMaster(AppSpec spec) throws YarnClientException {
+ if (app == null) {
+ throw new IllegalStateException("call createAppMaster( ) first");
+ }
+
+ ApplicationSubmissionContext appContext;
+ try {
+ appContext = spec.createAppLaunchContext(conf, app);
+ } catch (IOException e) {
+ throw new YarnClientException("Create app launch context failed", e);
+ }
+
+ // Submit application
+ try {
+ yarnClient.submitApplication(appContext);
+ } catch (YarnException | IOException e) {
+ throw new YarnClientException("Submit application failed", e);
+ }
+ }
+
+ public ApplicationId getAppId() {
+ return appId;
+ }
+
+ public ApplicationReport getAppReport() throws YarnClientException {
+ try {
+ return yarnClient.getApplicationReport(appId);
+ } catch (YarnException | IOException e) {
+ throw new YarnClientException("Get application report failed", e);
+ }
+ }
+
+ /**
+ * Waits for the application to start. This version is somewhat informal, the
+ * intended use is when debugging unmanaged applications.
+ *
+ * @throws YarnClientException
+ */
+ public ApplicationAttemptId waitForStart() throws YarnClientException {
+ ApplicationReport appReport;
+ YarnApplicationState appState;
+ ApplicationAttemptId attemptId;
+ for (;;) {
+ appReport = getAppReport();
+ appState = appReport.getYarnApplicationState();
+ attemptId = appReport.getCurrentApplicationAttemptId();
+ if (appState != YarnApplicationState.NEW
+ && appState != YarnApplicationState.NEW_SAVING
+ && appState != YarnApplicationState.SUBMITTED) {
+ break;
+ }
+ System.out.println("App State: " + appState);
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ // Should never occur.
+ }
+ }
+ if (appState != YarnApplicationState.ACCEPTED) {
+ throw new YarnClientException(
+ "Application start failed with status " + appState);
+ }
+
+ return attemptId;
+ }
+
+ /**
+ * Wait for the application to enter one of the completion states. This is an
+ * informal implementation useful for testing.
+ *
+ * @throws YarnClientException
+ */
+
+ public void waitForCompletion() throws YarnClientException {
+ ApplicationReport appReport;
+ YarnApplicationState appState;
+ for (;;) {
+ appReport = getAppReport();
+ appState = appReport.getYarnApplicationState();
+ if (appState == YarnApplicationState.FINISHED
+ || appState == YarnApplicationState.KILLED
+ || appState == YarnApplicationState.FAILED) {
+ break;
+ }
+ try {
+ Thread.sleep(100);
+ } catch (InterruptedException e) {
+ // Should never occur.
+ }
+ }
+
+ System.out.println("Application " + appId + " finished with" + " state "
+ + appState + " at " + appReport.getFinishTime());
+ }
+
+ public Token<AMRMTokenIdentifier> getAMRMToken() throws YarnClientException {
+ try {
+ return yarnClient.getAMRMToken(appId);
+ } catch (YarnException | IOException e) {
+ throw new YarnClientException("Get AM/RM token failed", e);
+ }
+ }
+
+ /**
+ * Return standard class path entries from the YARN application class path.
+ */
+
+ public String[] getYarnAppClassPath() {
+ return conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH,
+ YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH);
+ }
+
+ public void killApplication() throws YarnClientException {
+ try {
+ yarnClient.killApplication(appId);
+ } catch (YarnException | IOException e) {
+ throw new YarnClientException(
+ "Kill failed for application: " + appId.toString());
+ }
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/core/package-info.java b/drill-yarn/src/main/java/org/apache/drill/yarn/core/package-info.java
new file mode 100644
index 000000000..aaa0ffffe
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/core/package-info.java
@@ -0,0 +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.
+ */
+
+/**
+ * Holds functionality common to the Drill-on-YARN client and Application Master (AM).
+ * Includes configuration, utilities, and wrappers around various YARN data classes.
+ */
+
+package org.apache.drill.yarn.core; \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/package-info.java b/drill-yarn/src/main/java/org/apache/drill/yarn/package-info.java
new file mode 100644
index 000000000..170dfa8d5
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/package-info.java
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/**
+ * Hosts Apache Drill under Apache Hadoop YARN. Consists of two main
+ * components as required by YARN: a client application which uses YARN to
+ * start the Drill cluster, and an Application Master (AM) which manages
+ * the cluster. The AM in turn starts, manages and stops drillbits.
+ * <p>
+ * Much of the functionality is simply plumbing to get YARN to do what is
+ * needed. The core of the AM is a "cluster controller" which starts,
+ * monitors and stops Drillbits, tracking their state transitions though
+ * the several lifecycle stages that result.
+ * <p>
+ * Note about logs here: Drill-on-YARN is a YARN application and so it
+ * uses the same logging system used by the YARN code. This is different
+ * than that used by Drill.
+ */
+
+package org.apache.drill.yarn; \ No newline at end of file
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/AMRegistry.java b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/AMRegistry.java
new file mode 100644
index 000000000..9cc95e58a
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/AMRegistry.java
@@ -0,0 +1,145 @@
+/*
+ * 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.drill.yarn.zk;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException.NodeExistsException;
+
+/**
+ * Register this App Master in ZK to prevent duplicates.
+ * <p>
+ * Possible enhancement is to put the registry in some well-known location, such
+ * as /drill-am,
+ */
+public class AMRegistry {
+ private static final String AM_REGISTRY = "/drill-on-yarn";
+
+ private ZKClusterCoordinator zkCoord;
+ @SuppressWarnings("unused")
+ private String amHost;
+ @SuppressWarnings("unused")
+ private int amPort;
+ @SuppressWarnings("unused")
+ private String amAppId;
+
+ private String zkRoot;
+
+ private String clusterId;
+
+ public AMRegistry(ZKClusterCoordinator zkCoord) {
+ this.zkCoord = zkCoord;
+ }
+
+ public void useLocalRegistry(String zkRoot, String clusterId) {
+ this.zkRoot = zkRoot;
+ this.clusterId = clusterId;
+ }
+
+ /**
+ * Register this AM as an ephemeral znode in ZK. The structure of ZK is as
+ * follows:
+ *
+ * <pre>
+ * /drill
+ * . &lt;cluster-id>
+ * . . &lt;Drillbit GUID> (Value is Proto-encoded drillbit info)
+ * . drill-on-yarn
+ * . . &lt;cluster-id> (value: amHost:port)
+ * </pre>
+ * <p>
+ * The structure acknowledges that the cluster-id znode may be renamed, and
+ * there may be multiple cluster IDs for a single drill root node. (Odd, but
+ * supported.) To address this, we put the AM registrations in their own
+ * (persistent) znode: drill-on-yarn. Each is keyed by the cluster ID (so we
+ * can find it), and holds the host name, HTTP port and Application ID of the
+ * AM.
+ * <p>
+ * When the AM starts, it atomically checks and sets the AM registration. If
+ * another AM already is running, then this AM will fail, displaying a log
+ * error message with the host, port and (most importantly) app ID so the user
+ * can locate the problem.
+ *
+ * @throws ZKRuntimeException
+ */
+
+ public void register(String amHost, int amPort, String amAppId)
+ throws ZKRuntimeException {
+ this.amHost = amHost;
+ this.amPort = amPort;
+ this.amAppId = amAppId;
+ try {
+
+ // The znode to hold AMs may or may not exist. Create it if missing.
+
+ try {
+ zkCoord.getCurator().create().withMode(CreateMode.PERSISTENT)
+ .forPath(AM_REGISTRY, new byte[0]);
+ } catch (NodeExistsException e) {
+ // OK
+ }
+
+ // Try to create the AM registration.
+
+ String amPath = AM_REGISTRY + "/" + clusterId;
+ String content = amHost + ":" + Integer.toString(amPort) + ":" + amAppId;
+ try {
+ zkCoord.getCurator().create().withMode(CreateMode.EPHEMERAL)
+ .forPath(amPath, content.getBytes("UTF-8"));
+ } catch (NodeExistsException e) {
+
+ // ZK says that a node exists, which means that another AM is already
+ // running.
+ // Display an error, handling the case where the AM just disappeared,
+ // the
+ // registration is badly formatted, etc.
+
+ byte data[] = zkCoord.getCurator().getData().forPath(amPath);
+ String existing;
+ if (data == null) {
+ existing = "Unknown";
+ } else {
+ String packed = new String(data, "UTF-8");
+ String unpacked[] = packed.split(":");
+ if (unpacked.length < 3) {
+ existing = packed;
+ } else {
+ existing = unpacked[0] + ", port: " + unpacked[1]
+ + ", Application ID: " + unpacked[2];
+ }
+ }
+
+ // Die with a clear (we hope!) error message.
+
+ throw new ZKRuntimeException(
+ "FAILED! An Application Master already exists for " + zkRoot + "/"
+ + clusterId + " on host: " + existing);
+ }
+ } catch (ZKRuntimeException e) {
+
+ // Something bad happened with ZK.
+
+ throw e;
+ } catch (Exception e) {
+
+ // Something bad happened with ZK.
+
+ throw new ZKRuntimeException("Failed to create AM registration node", e);
+ }
+ }
+
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinator.java b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinator.java
new file mode 100644
index 000000000..7c5f5f359
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinator.java
@@ -0,0 +1,318 @@
+/*
+ * 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.drill.yarn.zk;
+
+import static com.google.common.base.Throwables.propagate;
+import static com.google.common.collect.Collections2.transform;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.curator.RetryPolicy;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.curator.retry.RetryNTimes;
+import org.apache.curator.x.discovery.ServiceCache;
+import org.apache.curator.x.discovery.ServiceDiscovery;
+import org.apache.curator.x.discovery.ServiceDiscoveryBuilder;
+import org.apache.curator.x.discovery.ServiceInstance;
+import org.apache.curator.x.discovery.details.ServiceCacheListener;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.coord.DistributedSemaphore;
+import org.apache.drill.exec.coord.DrillServiceInstanceHelper;
+import org.apache.drill.exec.coord.store.CachingTransientStoreFactory;
+import org.apache.drill.exec.coord.store.TransientStore;
+import org.apache.drill.exec.coord.store.TransientStoreConfig;
+import org.apache.drill.exec.coord.store.TransientStoreFactory;
+import org.apache.drill.exec.coord.zk.ZKRegistrationHandle;
+import org.apache.drill.exec.coord.zk.ZkDistributedSemaphore;
+import org.apache.drill.exec.coord.zk.ZkEphemeralStore;
+import org.apache.drill.exec.coord.zk.ZkTransientStoreFactory;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.State;
+
+import com.google.common.base.Function;
+
+/**
+ * Manages cluster coordination utilizing zookeeper.
+ * <p>
+ * This is a clone of the Drill class
+ * org.apache.drill.exec.coord.zk.ZKClusterCoordinator with a number of
+ * modifications:
+ * <ul>
+ * <li>Removed dependency on the Drill config system. That system uses Google's
+ * Guava library version 18, which conflicts with the earlier versions used by
+ * YARN and Hadoop, which resulted in runtime undefined method exceptions.</li>
+ * <li>Instead of getting config information out of the Drill config, the
+ * parameters are instead passed directly.</li>
+ * <li>Adds support for the drillbits registered event which was neither needed
+ * nor implemented by Drill.</li>
+ * <li>Use the YARN logging system instead of Drill's.</li>
+ * </ul>
+ * <p>
+ * This class should be replaced by the Drill version if/when the Guava
+ * conflicts can be resolved (and when registered Drillbit notifications are
+ * added to the Drill version.)
+ */
+
+public class ZKClusterCoordinator extends ClusterCoordinator {
+
+ protected static final Log logger = LogFactory
+ .getLog(ZKClusterCoordinator.class);
+
+ private CuratorFramework curator;
+ private ServiceDiscovery<DrillbitEndpoint> discovery;
+ private volatile Collection<DrillbitEndpoint> endpoints = Collections
+ .emptyList();
+ private final String serviceName;
+ private final CountDownLatch initialConnection = new CountDownLatch(1);
+ private final TransientStoreFactory factory;
+ private ServiceCache<DrillbitEndpoint> serviceCache;
+
+ public ZKClusterCoordinator(String connect, String zkRoot, String clusterId,
+ int retryCount, int retryDelayMs, int connectTimeoutMs)
+ throws IOException {
+ logger.debug("ZK connect: " + connect + ", zkRoot: " + zkRoot
+ + ", clusterId: " + clusterId);
+
+ this.serviceName = clusterId;
+ RetryPolicy rp = new RetryNTimes(retryCount, retryDelayMs);
+ curator = CuratorFrameworkFactory.builder().namespace(zkRoot)
+ .connectionTimeoutMs(connectTimeoutMs).retryPolicy(rp)
+ .connectString(connect).build();
+ curator.getConnectionStateListenable()
+ .addListener(new InitialConnectionListener());
+ curator.start();
+ discovery = newDiscovery();
+ factory = CachingTransientStoreFactory
+ .of(new ZkTransientStoreFactory(curator));
+ }
+
+ public CuratorFramework getCurator() {
+ return curator;
+ }
+
+ @Override
+ public void start(long millisToWait) throws Exception {
+ logger.debug("Starting ZKClusterCoordination.");
+ discovery.start();
+
+ if (millisToWait != 0) {
+ boolean success = this.initialConnection.await(millisToWait,
+ TimeUnit.MILLISECONDS);
+ if (!success) {
+ throw new IOException(String.format(
+ "Failure to connect to the zookeeper cluster service within the allotted time of %d milliseconds.",
+ millisToWait));
+ }
+ } else {
+ this.initialConnection.await();
+ }
+
+ serviceCache = discovery.serviceCacheBuilder().name(serviceName).build();
+ serviceCache.addListener(new EndpointListener());
+ serviceCache.start();
+ updateEndpoints();
+ }
+
+ private class InitialConnectionListener implements ConnectionStateListener {
+
+ @Override
+ public void stateChanged(CuratorFramework client,
+ ConnectionState newState) {
+ if (newState == ConnectionState.CONNECTED) {
+ initialConnection.countDown();
+ client.getConnectionStateListenable().removeListener(this);
+ }
+ }
+
+ }
+
+ private class EndpointListener implements ServiceCacheListener {
+ @Override
+ public void stateChanged(CuratorFramework client,
+ ConnectionState newState) {
+ }
+
+ @Override
+ public void cacheChanged() {
+ logger.debug("Got cache changed --> updating endpoints");
+ updateEndpoints();
+ }
+ }
+
+ @Override
+ public void close() throws Exception {
+ // discovery attempts to close its caches(ie serviceCache) already. however,
+ // being good citizens we make sure to
+ // explicitly close serviceCache. Not only that we make sure to close
+ // serviceCache before discovery to prevent
+ // double releasing and disallowing jvm to spit bothering warnings. simply
+ // put, we are great!
+ AutoCloseables.close(serviceCache, discovery, curator, factory);
+ }
+
+ @Override
+ public RegistrationHandle register(DrillbitEndpoint data) {
+ try {
+ ServiceInstance<DrillbitEndpoint> serviceInstance = newServiceInstance(
+ data);
+ discovery.registerService(serviceInstance);
+ return new ZKRegistrationHandle(serviceInstance.getId(), data);
+ } catch (Exception e) {
+ throw propagate(e);
+ }
+ }
+
+ @Override
+ public void unregister(RegistrationHandle handle) {
+ if (!(handle instanceof ZKRegistrationHandle)) {
+ throw new UnsupportedOperationException(
+ "Unknown handle type: " + handle.getClass().getName());
+ }
+
+ // when Drillbit is unregistered, clean all the listeners registered in CC.
+ this.listeners.clear();
+
+ ZKRegistrationHandle h = (ZKRegistrationHandle) handle;
+ try {
+ ServiceInstance<DrillbitEndpoint> serviceInstance = ServiceInstance
+ .<DrillbitEndpoint> builder().address("").port(0).id(h.id)
+ .name(serviceName).build();
+ discovery.unregisterService(serviceInstance);
+ } catch (Exception e) {
+ propagate(e);
+ }
+ }
+
+ @Override
+ public Collection<DrillbitEndpoint> getAvailableEndpoints() {
+ return this.endpoints;
+ }
+
+ @Override
+ public DistributedSemaphore getSemaphore(String name, int maximumLeases) {
+ return new ZkDistributedSemaphore(curator, "/semaphore/" + name,
+ maximumLeases);
+ }
+
+ @Override
+ public <V> TransientStore<V> getOrCreateTransientStore(
+ final TransientStoreConfig<V> config) {
+ final ZkEphemeralStore<V> store = (ZkEphemeralStore<V>) factory
+ .getOrCreateStore(config);
+ return store;
+ }
+
+ private synchronized void updateEndpoints() {
+ try {
+ Collection<DrillbitEndpoint> newDrillbitSet = transform(
+ discovery.queryForInstances(serviceName),
+ new Function<ServiceInstance<DrillbitEndpoint>, DrillbitEndpoint>() {
+ @Override
+ public DrillbitEndpoint apply(
+ ServiceInstance<DrillbitEndpoint> input) {
+ return input.getPayload();
+ }
+ });
+
+ // set of newly dead bits : original bits - new set of active bits.
+ Set<DrillbitEndpoint> unregisteredBits = new HashSet<>(endpoints);
+ unregisteredBits.removeAll(newDrillbitSet);
+
+ // Set of newly live bits : new set of active bits - original bits.
+ Set<DrillbitEndpoint> registeredBits = new HashSet<>(newDrillbitSet);
+ registeredBits.removeAll(endpoints);
+
+ endpoints = newDrillbitSet;
+
+ if (logger.isDebugEnabled()) {
+ StringBuilder builder = new StringBuilder();
+ builder.append("Active drillbit set changed. Now includes ");
+ builder.append(newDrillbitSet.size());
+ builder.append(" total bits.");
+ if (!newDrillbitSet.isEmpty()) {
+ builder.append(" New active drillbits: \n");
+ }
+ for (DrillbitEndpoint bit : newDrillbitSet) {
+ builder.append('\t');
+ builder.append(bit.getAddress());
+ builder.append(':');
+ builder.append(bit.getUserPort());
+ builder.append(':');
+ builder.append(bit.getControlPort());
+ builder.append(':');
+ builder.append(bit.getDataPort());
+ builder.append('\n');
+ }
+ logger.debug(builder.toString());
+ }
+
+ // Notify the drillbit listener for newly unregistered bits.
+ if (!(unregisteredBits.isEmpty())) {
+ drillbitUnregistered(unregisteredBits);
+ }
+ // Notify the drillbit listener for newly registered bits.
+ if (!(registeredBits.isEmpty())) {
+ drillbitRegistered(registeredBits);
+ }
+
+ } catch (Exception e) {
+ logger.error("Failure while update Drillbit service location cache.", e);
+ }
+ }
+
+ protected ServiceInstance<DrillbitEndpoint> newServiceInstance(
+ DrillbitEndpoint endpoint) throws Exception {
+ return ServiceInstance.<DrillbitEndpoint> builder().name(serviceName)
+ .payload(endpoint).build();
+ }
+
+ protected ServiceDiscovery<DrillbitEndpoint> newDiscovery() {
+ return ServiceDiscoveryBuilder.builder(DrillbitEndpoint.class).basePath("/")
+ .client(curator).serializer(DrillServiceInstanceHelper.SERIALIZER)
+ .build();
+ }
+
+ @Override
+ public Collection<DrillbitEndpoint> getOnlineEndPoints() {
+
+ // Not used in DoY
+
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public RegistrationHandle update(RegistrationHandle handle, State state) {
+
+ // Not used in DoY
+
+ throw new UnsupportedOperationException();
+ }
+
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinatorDriver.java b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinatorDriver.java
new file mode 100644
index 000000000..3f83ff2bc
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinatorDriver.java
@@ -0,0 +1,315 @@
+/*
+ * 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.drill.yarn.zk;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.work.foreman.DrillbitStatusListener;
+import org.apache.drill.yarn.appMaster.AMRegistrar;
+
+/**
+ * Driver class for the ZooKeeper cluster coordinator. Provides defaults for
+ * most options, but allows customizing each. Provides a {@link #build()} method
+ * to create <i>and start</i> the ZK service. Obtains the initial set of
+ * Drillbits (which should be empty for a YARN-defined cluster) which can be
+ * retrieved after building.
+ * <p>
+ * Maintains the ZK connection and monitors for disconnect. This class simply
+ * detects a disconnect timeout, it does not send a disconnect event itself to
+ * avoid creating a timer thread just for this purpose. Instead, the caller can
+ * poll {@link #hasFailed()}.
+ * <p>
+ * Defaults match those in Drill. (Actual Drill defaults are not yet used due to
+ * code incompatibility issues.)
+ */
+
+public class ZKClusterCoordinatorDriver implements AMRegistrar {
+ private static final Pattern ZK_COMPLEX_STRING = Pattern
+ .compile("(^.*?)/(.*)/([^/]*)$");
+
+ // Defaults are taken from java-exec's drill-module.conf
+
+ private String connect = "localhost:2181";
+ private String clusterId = "drillbits1";
+ private String zkRoot = "drill";
+ private int retryCount = 7200;
+ private int connectTimeoutMs = 5_000;
+ private int retryDelayMs = 500;
+
+ // Default timeout before we declare that ZK is down: 2 minutes.
+
+ private int failureTimeoutMs = 120_000;
+
+ // Maximum ZK startup wait defaults to 30 seconds. It is only 10 seconds
+ // in the Drill implementation.
+
+ private int maxStartWaitMs = 30_000;
+
+ // Expected ports used to match ZK registries with
+ // containers. ZK lists the ports as part of its key, we have to anticipate
+ // these values in order to match.
+
+ private int userPort = 31010;
+ private int controlPort = 31011;
+ private int dataPort = 31012;
+
+ private List<DrillbitEndpoint> initialEndpoints;
+ private ConnectionStateListener stateListener = new ConnectionStateListener() {
+
+ @Override
+ public void stateChanged(CuratorFramework client,
+ ConnectionState newState) {
+ ZKClusterCoordinatorDriver.this.stateChanged(newState);
+ }
+ };
+
+ private ZKClusterCoordinator zkCoord;
+
+ private long connectionLostTime;
+
+ private AMRegistry amRegistry;
+
+ public ZKClusterCoordinatorDriver() {
+ }
+
+ /**
+ * Specify connect string in the form: host:/zkRoot/clusterId
+ *
+ * @param connect
+ * @return
+ * @throws ZKConfigException
+ */
+ public ZKClusterCoordinatorDriver setConnect(String connect)
+ throws ZKConfigException {
+
+ // check if this is a complex zk string. If so, parse into components.
+ Matcher m = ZK_COMPLEX_STRING.matcher(connect);
+ if (!m.matches()) {
+ throw new ZKConfigException("Bad connect string: " + connect);
+ }
+ this.connect = m.group(1);
+ zkRoot = m.group(2);
+ clusterId = m.group(3);
+ return this;
+ }
+
+ public ZKClusterCoordinatorDriver setConnect(String connect, String zkRoot,
+ String clusterId) {
+ this.connect = connect;
+ this.zkRoot = zkRoot;
+ this.clusterId = clusterId;
+ return this;
+ }
+
+ public ZKClusterCoordinatorDriver setRetryCount(int n) {
+ retryCount = n;
+ return this;
+ }
+
+ public ZKClusterCoordinatorDriver setConnectTimeoutMs(int ms) {
+ connectTimeoutMs = ms;
+ return this;
+ }
+
+ public ZKClusterCoordinatorDriver setRetryDelayMs(int ms) {
+ retryDelayMs = ms;
+ return this;
+ }
+
+ public ZKClusterCoordinatorDriver setMaxStartWaitMs(int ms) {
+ maxStartWaitMs = ms;
+ return this;
+ }
+
+ public ZKClusterCoordinatorDriver setFailureTimoutMs(int ms) {
+ failureTimeoutMs = ms;
+ return this;
+ }
+
+ public ZKClusterCoordinatorDriver setPorts(int userPort, int controlPort,
+ int dataPort) {
+ this.userPort = userPort;
+ this.controlPort = controlPort;
+ this.dataPort = dataPort;
+ return this;
+ }
+
+ /**
+ * Builds and starts the ZooKeeper cluster coordinator, translating any errors
+ * that occur. After this call, the listener will start receiving messages.
+ *
+ * @return
+ * @throws ZKRuntimeException
+ * if ZK startup fails
+ */
+ public ZKClusterCoordinatorDriver build() throws ZKRuntimeException {
+ try {
+ zkCoord = new ZKClusterCoordinator(connect, zkRoot, clusterId, retryCount,
+ retryDelayMs, connectTimeoutMs);
+ } catch (IOException e) {
+ throw new ZKRuntimeException(
+ "Failed to initialize the ZooKeeper cluster coordination", e);
+ }
+ try {
+ zkCoord.start(maxStartWaitMs);
+ } catch (Exception e) {
+ throw new ZKRuntimeException(
+ "Failed to start the ZooKeeper cluster coordination after "
+ + maxStartWaitMs + " ms.",
+ e);
+ }
+ initialEndpoints = new ArrayList<>(zkCoord.getAvailableEndpoints());
+ zkCoord.getCurator().getConnectionStateListenable()
+ .addListener(stateListener);
+ amRegistry = new AMRegistry(zkCoord);
+ amRegistry.useLocalRegistry(zkRoot, clusterId);
+ return this;
+ }
+
+ public void addDrillbitListener(DrillbitStatusListener listener) {
+ zkCoord.addDrillbitStatusListener(listener);
+ }
+
+ public void removeDrillbitListener(DrillbitStatusListener listener) {
+ zkCoord.removeDrillbitStatusListener(listener);
+ }
+
+ /**
+ * Returns the set of Drillbits registered at the time of the {@link #build()}
+ * call. Should be empty for a cluster managed by YARN.
+ *
+ * @return
+ */
+
+ public List<DrillbitEndpoint> getInitialEndpoints() {
+ return initialEndpoints;
+ }
+
+ /**
+ * Convenience method to convert a Drillbit to a string. Note that ZK does not
+ * advertise the HTTP port, so it does not appear in the generated string.
+ *
+ * @param bit
+ * @return
+ */
+
+ public static String asString(DrillbitEndpoint bit) {
+ return formatKey(bit.getAddress(), bit.getUserPort(), bit.getControlPort(),
+ bit.getDataPort());
+ }
+
+ public String toKey(String host) {
+ return formatKey(host, userPort, controlPort, dataPort);
+ }
+
+ public static String formatKey(String host, int userPort, int controlPort,
+ int dataPort) {
+ StringBuilder buf = new StringBuilder();
+ buf.append(host).append(":").append(userPort).append(':')
+ .append(controlPort).append(':').append(dataPort);
+ return buf.toString();
+ }
+
+ /**
+ * Translate ZK connection events into a connected/disconnected state along
+ * with the time of the first disconnect not followed by a connect.
+ *
+ * @param newState
+ */
+
+ protected void stateChanged(ConnectionState newState) {
+ switch (newState) {
+ case CONNECTED:
+ case READ_ONLY:
+ case RECONNECTED:
+ if (connectionLostTime != 0) {
+ ZKClusterCoordinator.logger.info("ZK connection regained");
+ }
+ connectionLostTime = 0;
+ break;
+ case LOST:
+ case SUSPENDED:
+ if (connectionLostTime == 0) {
+ ZKClusterCoordinator.logger.info("ZK connection lost");
+ connectionLostTime = System.currentTimeMillis();
+ }
+ break;
+ }
+ }
+
+ /**
+ * Reports our best guess as to whether ZK has failed. We assume ZK has failed
+ * if we received a connection lost notification without a subsequent connect
+ * notification, and we received the disconnect notification log enough ago
+ * that we assume that a timeout has occurred.
+ *
+ * @return
+ */
+
+ public boolean hasFailed() {
+ if (connectionLostTime == 0) {
+ return false;
+ }
+ return System.currentTimeMillis() - connectionLostTime > failureTimeoutMs;
+ }
+
+ public long getLostConnectionDurationMs() {
+ if (connectionLostTime == 0) {
+ return 0;
+ }
+ return System.currentTimeMillis() - connectionLostTime;
+ }
+
+ public void close() {
+ if (zkCoord == null) {
+ return;
+ }
+ zkCoord.getCurator().getConnectionStateListenable()
+ .removeListener(stateListener);
+ try {
+ zkCoord.close();
+ } catch (Exception e) {
+ ZKClusterCoordinator.logger.error("Error occurred on ZK close, ignored",
+ e);
+ }
+ zkCoord = null;
+ }
+
+ @Override
+ public void register(String amHost, int amPort, String appId)
+ throws AMRegistrationException {
+ try {
+ amRegistry.register(amHost, amPort, appId);
+ } catch (ZKRuntimeException e) {
+ throw new AMRegistrationException(e);
+ }
+ }
+
+ @Override
+ public void deregister() {
+ // Nothing to do: ZK does it for us.
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKConfigException.java b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKConfigException.java
new file mode 100644
index 000000000..700d84b66
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKConfigException.java
@@ -0,0 +1,26 @@
+/*
+ * 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.drill.yarn.zk;
+
+public class ZKConfigException extends Exception {
+ private static final long serialVersionUID = 1L;
+
+ public ZKConfigException(String msg) {
+ super(msg);
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRegistry.java b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRegistry.java
new file mode 100644
index 000000000..04265787c
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRegistry.java
@@ -0,0 +1,582 @@
+/*
+ * 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.drill.yarn.zk;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.work.foreman.DrillbitStatusListener;
+import org.apache.drill.yarn.appMaster.AMWrapperException;
+import org.apache.drill.yarn.appMaster.EventContext;
+import org.apache.drill.yarn.appMaster.Pollable;
+import org.apache.drill.yarn.appMaster.RegistryHandler;
+import org.apache.drill.yarn.appMaster.Task;
+import org.apache.drill.yarn.appMaster.TaskLifecycleListener;
+
+/**
+ * AM-specific implementation of a Drillbit registry backed by ZooKeeper.
+ * Listens to ZK events for registering a Drillbit and deregistering. Alerts the
+ * Cluster Controller of these events.
+ * <p>
+ * Locking strategy: Receives events from both ZK and the cluster controller,
+ * both of which must be synchronized. To prevent deadlocks, this class NEVER
+ * calls into the cluster controller while holding a lock. This prevents the
+ * following:
+ * <p>
+ * ClusterController --> ZKRegistry (OK) <br>
+ * ZK --> ZKRegistry (OK) <br>
+ * ZK --> ZKRegistry --> Cluster Controller (bad)
+ * <p>
+ * In the case of registration, ZK calls the registry which must alert the
+ * cluster controller. Cluster controller alerting is handled outside the ZK
+ * update critical section.
+ * <p>
+ * Because ZK events are occur relatively infrequently, any deadlock will occur
+ * once in a blue moon, which will make it very hard to reproduce. So, extreme
+ * caution is needed at design time to prevent the problem.
+ */
+
+public class ZKRegistry
+ implements TaskLifecycleListener, DrillbitStatusListener, Pollable {
+ /**
+ * State of each Drillbit that we've discovered through ZK or launched via the
+ * AM. The tracker is where we combine the ZK information with AM to correlate
+ * overall Drillbit health.
+ */
+
+ protected static class DrillbitTracker {
+ /**
+ * A Drillbit can be in one of four states.
+ */
+
+ public enum State {
+
+ /**
+ * An unmanaged Drillbit is one that has announced itself via ZK, but
+ * which the AM didn't launch (or has not yet received confirmation from
+ * YARN that it was launched.) In the normal state, this state either does
+ * not occur (YARN reports the task launch before the Drillbit registers
+ * in ZK) or is transient (if the Drillbit registers in ZK before YARN
+ * gets around to telling the AM that the Drillbit was launched.) A
+ * Drillbit that stays in the unregistered state is likely one launched
+ * outside the AM: either launched manually or (possibly), one left from a
+ * previous, failed AM run (though YARN is supposed to kill left-over
+ * child processes in that case.)
+ */
+
+ UNMANAGED,
+
+ /**
+ * A new Drillbit is one that the AM has launched, but that has not yet
+ * registered itself with ZK. This is normally a transient state that
+ * occurs as ZK registration catches up with the YARN launch notification.
+ * If a Drillbit says in this state, then something is seriously wrong
+ * (perhaps a mis-configuration). The cluster controller will patiently
+ * wait a while, then decide bad things are happening and will ask YARN to
+ * kill the Drillbit, then will retry a time or two, after which it will
+ * throw up its hands, blacklist the node, and wait for the admin to sort
+ * things out.
+ */
+
+ NEW,
+
+ /**
+ * Normal operating state: the AM launched the Drillbit, which then
+ * dutifully registered itself in ZK. Nothing to see here, move along.
+ */
+
+ REGISTERED,
+
+ /**
+ * The Drillbit was working just fine, but its registration has dropped
+ * out of ZK for a reason best left to the cluster controller to
+ * determine. Perhaps the controller has decided to kill the Drillbit.
+ * Perhaps the Drillbit became unresponsive (in which case the controller
+ * will kill it and retry) or has died (in which case YARN will alert the
+ * AM that the process exited.)
+ */
+
+ DEREGISTERED
+ }
+
+ /**
+ * The common key used between tasks and ZK registrations. The key is of the
+ * form:<br>
+ *
+ * <pre>
+ * host:port:port:port
+ * </pre>
+ */
+
+ protected final String key;
+
+ /**
+ * ZK tracking state.
+ *
+ * @see {@link State}
+ */
+
+ protected State state;
+
+ /**
+ * For Drillbits started by the AM, the task object for this Drillbit.
+ */
+
+ protected Task task;
+
+ /**
+ * For Drillbits discovered through ZK, the Drill endpoint for the Drillbit.
+ */
+
+ protected DrillbitEndpoint endpoint;
+
+ public DrillbitTracker(String key, DrillbitEndpoint endpoint) {
+ this.key = key;
+ this.state = DrillbitTracker.State.UNMANAGED;
+ this.endpoint = endpoint;
+ }
+
+ public DrillbitTracker(String key, Task task) {
+ this.key = key;
+ this.task = task;
+ state = DrillbitTracker.State.NEW;
+ }
+
+ /**
+ * Mark that a YARN-managed task has become registered in ZK. This indicates
+ * that the task has come online. Tell the task to update its state to
+ * record that the task is, in fact, registered in ZK. This indicates a
+ * normal, healthy task.
+ *
+ * @param tracker
+ */
+
+ private void becomeRegistered() {
+ state = DrillbitTracker.State.REGISTERED;
+ }
+
+ /**
+ * Mark that a YARN-managed Drillbit has dropped out of ZK.
+ *
+ * @param registryHandler
+ */
+
+ public void becomeUnregistered() {
+ assert state == DrillbitTracker.State.REGISTERED;
+ state = DrillbitTracker.State.DEREGISTERED;
+ endpoint = null;
+ }
+ }
+
+ public static final String CONTROLLER_PROPERTY = "zk";
+
+ public static final int UPDATE_PERIOD_MS = 20_000;
+
+ public static final String ENDPOINT_PROPERTY = "endpoint";
+
+ private static final Log LOG = LogFactory.getLog(ZKRegistry.class);
+
+ /**
+ * Map of host:port:port:port keys to tracking objects. Identifies the
+ * Drillbits discovered from ZK, started by the controller, or (ideally) both.
+ */
+
+ private Map<String, DrillbitTracker> registry = new HashMap<>();
+
+ /**
+ * Interface to Drill's cluster coordinator.
+ */
+
+ private ZKClusterCoordinatorDriver zkDriver;
+
+ /**
+ * Drill's cluster coordinator (or, at least, Drill-on-YARN's version of it.
+ */
+
+ private RegistryHandler registryHandler;
+
+ /**
+ * Last check of ZK status.
+ */
+
+ private long lastUpdateTime;
+
+ public ZKRegistry(ZKClusterCoordinatorDriver zkDriver) {
+ this.zkDriver = zkDriver;
+ }
+
+ /**
+ * Called during AM startup to initialize ZK. Checks if any Drillbits are
+ * already running. These are "unmanaged" because the AM could not have
+ * started them (since they predate the AM.)
+ */
+
+ public void start(RegistryHandler controller) {
+ this.registryHandler = controller;
+ try {
+ zkDriver.build();
+ } catch (ZKRuntimeException e) {
+ LOG.error("Failed to start ZK monitoring", e);
+ throw new AMWrapperException("Failed to start ZK monitoring", e);
+ }
+ for (DrillbitEndpoint dbe : zkDriver.getInitialEndpoints()) {
+ String key = toKey(dbe);
+ registry.put(key, new DrillbitTracker(key, dbe));
+
+ // Blacklist the host for each unmanaged drillbit.
+
+ controller.reserveHost(dbe.getAddress());
+
+ LOG.warn("Host " + dbe.getAddress()
+ + " already running a Drillbit outside of YARN.");
+ }
+ zkDriver.addDrillbitListener(this);
+ }
+
+ /**
+ * Convert a Drillbit endpoint to a string key used in the (zk-->task) map.
+ * Note that the string format here must match the one used in
+ * {@link #toKey(Task)} to map a task to string key.
+ *
+ * @param dbe
+ * the Drillbit endpoint from ZK
+ * @return a string key for this object
+ */
+
+ private String toKey(DrillbitEndpoint dbe) {
+ return ZKClusterCoordinatorDriver.asString(dbe);
+ }
+
+ /**
+ * Convert a task to a string key used in the (zk-->task) map. Note that the
+ * string format here must match the one used in
+ * {@link #toKey(DrillbitEndpoint)} to map a drillbit endpoint to string key.
+ *
+ * @param task
+ * the task tracked by the cluster controller
+ * @return a string key for this object
+ */
+
+ private String toKey(Task task) {
+ return zkDriver.toKey(task.getHostName());
+ }
+
+ // private String toKey(Container container) {
+ // return zkDriver.toKey(container.getNodeId().getHost());
+ // }
+
+ public static class AckEvent {
+ Task task;
+ DrillbitEndpoint endpoint;
+
+ public AckEvent(Task task, DrillbitEndpoint endpoint) {
+ this.task = task;
+ this.endpoint = endpoint;
+ }
+ }
+
+ /**
+ * Callback from ZK to indicate that one or more drillbits have become
+ * registered. We handle registrations in a critical section, then alert the
+ * cluster controller outside the critical section.
+ */
+
+ @Override
+ public void drillbitRegistered(Set<DrillbitEndpoint> registeredDrillbits) {
+ List<AckEvent> updates = registerDrillbits(registeredDrillbits);
+ for (AckEvent event : updates) {
+ if (event.task == null) {
+ registryHandler.reserveHost(event.endpoint.getAddress());
+ } else {
+ registryHandler.startAck(event.task, ENDPOINT_PROPERTY, event.endpoint);
+ }
+ }
+ }
+
+ private synchronized List<AckEvent> registerDrillbits(
+ Set<DrillbitEndpoint> registeredDrillbits) {
+ List<AckEvent> events = new ArrayList<>();
+ for (DrillbitEndpoint dbe : registeredDrillbits) {
+ AckEvent event = drillbitRegistered(dbe);
+ if (event != null) {
+ events.add(event);
+ }
+ }
+ return events;
+ }
+
+ /**
+ * Called when a drillbit has become registered. There are two cases. Either
+ * this is a normal registration of a previously-started task, or this is an
+ * unmanaged drillbit for which we have no matching task.
+ */
+
+ private AckEvent drillbitRegistered(DrillbitEndpoint dbe) {
+ String key = toKey(dbe);
+ DrillbitTracker tracker = registry.get(key);
+ if (tracker == null) {
+ // Unmanaged drillbit case
+
+ LOG.info("Registration of unmanaged drillbit: " + key);
+ tracker = new DrillbitTracker(key, dbe);
+ registry.put(key, tracker);
+ return new AckEvent(null, dbe);
+ }
+
+ // Managed drillbit case. Might be we lost, then regained
+ // ZK connection.
+
+ if (tracker.state == DrillbitTracker.State.REGISTERED) {
+ LOG.info("Re-registration of known drillbit: " + key);
+ return null;
+ }
+
+ // Otherwise, the Drillbit has just registered with ZK.
+ // Or, if the ZK connection was lost and regained, the
+ // state changes from DEREGISTERED --> REGISTERED
+
+ LOG.info("Drillbit registered: " + key + ", task: " + tracker.task.toString() );
+ tracker.endpoint = dbe;
+ tracker.becomeRegistered();
+ return new AckEvent(tracker.task, dbe);
+ }
+
+ /**
+ * Callback from ZK to indicate that one or more drillbits have become
+ * deregistered from ZK. We handle the deregistrations in a critical section,
+ * but updates to the cluster controller outside of a critical section.
+ */
+
+ @Override
+ public void drillbitUnregistered(
+ Set<DrillbitEndpoint> unregisteredDrillbits) {
+ List<AckEvent> updates = unregisterDrillbits(unregisteredDrillbits);
+ for (AckEvent event : updates) {
+ registryHandler.completionAck(event.task, ENDPOINT_PROPERTY);
+ }
+ }
+
+ private synchronized List<AckEvent> unregisterDrillbits(
+ Set<DrillbitEndpoint> unregisteredDrillbits) {
+ List<AckEvent> events = new ArrayList<>();
+ for (DrillbitEndpoint dbe : unregisteredDrillbits) {
+ AckEvent event = drillbitUnregistered(dbe);
+ if (event != null) {
+ events.add(event);
+ }
+ }
+ return events;
+ }
+
+ /**
+ * Handle the case that a drillbit becomes unregistered. There are three
+ * cases.
+ * <ol>
+ * <li>The deregistration is for a drillbit that is not in the registry table.
+ * Indicates a code error.</li>
+ * <li>The drillbit is unmanaged. This occurs for drillbits started and
+ * stopped outside of YARN.</li>
+ * <li>Normal case of deregistration of a YARN-managed drillbit. Inform the
+ * controller of this event.</li>
+ * </ol>
+ *
+ * @param dbe
+ */
+
+ private AckEvent drillbitUnregistered(DrillbitEndpoint dbe) {
+ String key = toKey(dbe);
+ DrillbitTracker tracker = registry.get(key);
+ assert tracker != null;
+ if (tracker == null) {
+ // Something is terribly wrong.
+ // Have seen this when a user kills the Drillbit just after it starts. Evidently, the
+ // Drillbit registers with ZK just before it is killed, but before DoY hears about
+ // the registration.
+
+ LOG.error("Internal error - Unexpected drillbit unregistration: " + key);
+ return null;
+ }
+ if (tracker.state == DrillbitTracker.State.UNMANAGED) {
+ // Unmanaged drillbit
+
+ assert tracker.task == null;
+ LOG.info("Unmanaged drillbit unregistered: " + key);
+ registry.remove(key);
+ registryHandler.releaseHost(dbe.getAddress());
+ return null;
+ }
+ LOG.info("Drillbit unregistered: " + key + ", task: " + tracker.task.toString() );
+ tracker.becomeUnregistered();
+ return new AckEvent(tracker.task, dbe);
+ }
+
+ /**
+ * Listen for selected YARN task state changes. Called from within the cluster
+ * controller's critical section.
+ */
+
+ @Override
+ public synchronized void stateChange(Event event, EventContext context) {
+ switch (event) {
+ case ALLOCATED:
+ taskCreated(context.task);
+ break;
+ case ENDED:
+ taskEnded(context.task);
+ break;
+ default:
+ break;
+ }
+ }
+
+ /**
+ * Indicates that the cluster controller has created a task that we expect to
+ * be monitored by ZK. We handle two cases: the normal case in which we later
+ * receive a ZK notification. And, the unusual case in which we've already
+ * received the ZK notification and we now match that notification with this
+ * task. (The second case could occur if latency causes us to receive the ZK
+ * notification before we learn from the NM that the task is alive.)
+ *
+ * @param task
+ */
+
+ private void taskCreated(Task task) {
+ String key = toKey(task);
+ DrillbitTracker tracker = registry.get(key);
+ if (tracker == null) {
+ // Normal case: no ZK registration yet.
+
+ registry.put(key, new DrillbitTracker(key, task));
+ } else if (tracker.state == DrillbitTracker.State.UNMANAGED) {
+ // Unusual case: ZK registration came first.
+
+ LOG.info("Unmanaged drillbit became managed: " + key);
+ tracker.task = task;
+ tracker.becomeRegistered();
+
+ // Note: safe to call this here as we are already in the controller
+ // critical section.
+
+ registryHandler.startAck(task, ENDPOINT_PROPERTY, tracker.endpoint);
+ } else {
+ LOG.error(task.getLabel() + " - Drillbit registry in wrong state "
+ + tracker.state + " for new task: " + key);
+ }
+ }
+
+ /**
+ * Report whether the given task is still registered in ZK. Called while
+ * waiting for a deregistration event to catch possible cases where the
+ * messages is lost. The message should never be lost, but we've seen
+ * cases where tasks hang in this state. This is a potential work-around.
+ *
+ * @param task
+ * @return
+ */
+
+ public synchronized boolean isRegistered(Task task) {
+ String key = toKey(task);
+ DrillbitTracker tracker = registry.get(key);
+ if (tracker==null) {
+ return false;
+ }
+ return tracker.state == DrillbitTracker.State.REGISTERED;
+ }
+
+ /**
+ * Mark that a task (YARN container) has ended. Updates the (zk --> task)
+ * registry by removing the task. The cluster controller state machine
+ * monitors ZK and does not end the task until the ZK registration for that
+ * task drops. As a result, the entry here should be in the deregistered state
+ * or something is seriously wrong.
+ *
+ * @param task
+ */
+
+ private void taskEnded(Task task) {
+
+ // If the task has no host name then the task is being cancelled before
+ // a YARN container was allocated. Just ignore such a case.
+
+ if (task.getHostName() == null) {
+ return;
+ }
+ String key = toKey(task);
+ DrillbitTracker tracker = registry.get(key);
+ assert tracker != null;
+ assert tracker.state == DrillbitTracker.State.DEREGISTERED;
+ registry.remove(key);
+ }
+
+ /**
+ * Periodically check ZK status. If the ZK connection has timed out, something
+ * is very seriously wrong. Shut the whole Drill cluster down since Drill
+ * cannot operate without ZooKeeper.
+ * <p>
+ * This method should not be synchronized. It checks only the ZK state, not
+ * internal state. Further, if we do reconnect to ZK, then a ZK thread may
+ * attempt to update this registry, which will acquire a synchronization lock.
+ */
+
+ @Override
+ public void tick(long curTime) {
+ if (lastUpdateTime + UPDATE_PERIOD_MS < curTime) {
+ return;
+ }
+
+ lastUpdateTime = curTime;
+ if (zkDriver.hasFailed()) {
+ int secs = (int) ((zkDriver.getLostConnectionDurationMs() + 500) / 1000);
+ LOG.error(
+ "ZooKeeper connection lost, failing after " + secs + " seconds.");
+ registryHandler.registryDown();
+ }
+ }
+
+ public void finish(RegistryHandler handler) {
+ zkDriver.removeDrillbitListener(this);
+ zkDriver.close();
+ }
+
+ public synchronized List<String> listUnmanagedDrillits() {
+ List<String> drillbits = new ArrayList<>();
+ for (DrillbitTracker item : registry.values()) {
+ if (item.state == DrillbitTracker.State.UNMANAGED) {
+ drillbits.add(item.key);
+ }
+ }
+ return drillbits;
+ }
+
+ /**
+ * Get the current registry for testing. Why for testing? Because this is
+ * unsynchronized. In production code, the map may change out from under you.
+ *
+ * @return
+ */
+
+ protected Map<String, DrillbitTracker> getRegistryForTesting() {
+ return registry;
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRuntimeException.java b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRuntimeException.java
new file mode 100644
index 000000000..4e1b115d5
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRuntimeException.java
@@ -0,0 +1,30 @@
+/*
+ * 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.drill.yarn.zk;
+
+public class ZKRuntimeException extends Exception {
+ private static final long serialVersionUID = 1L;
+
+ public ZKRuntimeException(String msg, Exception e) {
+ super(msg, e);
+ }
+
+ public ZKRuntimeException(String msg) {
+ super(msg);
+ }
+}
diff --git a/drill-yarn/src/main/java/org/apache/drill/yarn/zk/package-info.java b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/package-info.java
new file mode 100644
index 000000000..14bb427e3
--- /dev/null
+++ b/drill-yarn/src/main/java/org/apache/drill/yarn/zk/package-info.java
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/**
+ * Interface between the Application Master and ZooKeeper. Classes here manage two
+ * registrations: Drillbits and the AM itself.
+ * <p>
+ * Drillbit registration is used to confirm that Drillbits have indeed come online.
+ * If Drillbits fail to come online, then the AM concludes that somethign went wrong.
+ * If Drilbits drop offline unexpectedly, the AM concludes that the Drillbit is sick
+ * and restarts it.
+ * <p>
+ * The AM registry prevents two AMs from attempting to manage the same cluster.
+ */
+
+package org.apache.drill.yarn.zk; \ No newline at end of file