You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ar...@apache.org on 2018/03/04 17:13:41 UTC

[01/12] drill git commit: DRILL-1170: YARN integration for Drill

Repository: drill
Updated Branches:
  refs/heads/master cf2478f7a -> 36159e228


http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/TestScripts.java
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/TestScripts.java b/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/TestScripts.java
new file mode 100644
index 0000000..e8f30a6
--- /dev/null
+++ b/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/TestScripts.java
@@ -0,0 +1,1288 @@
+/*
+ * 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.scripts;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.drill.yarn.scripts.ScriptUtils.DrillbitRun;
+import org.apache.drill.yarn.scripts.ScriptUtils.RunResult;
+import org.apache.drill.yarn.scripts.ScriptUtils.ScriptRunner;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * Unit tests to test the many ways that the Drill shell scripts can run.
+ * Since it would be difficult to test options using the actual Drillbit, the
+ * scripts make use of a special test fixture in runbit: the ability to pass
+ * a "wrapper" script to run in place of the Drillit. That script probes stderr,
+ * stdout and log files, and writes its arguments (which is the Drillbit launch
+ * command) to a file. As a result, we can capture this output and analyze it
+ * to ensure we are passing the right arguments to the Drillbit, and that output
+ * is going to the right destinations.
+ */
+
+// Turned of by default: works only in a developer setup
+@Ignore
+public class TestScripts {
+  static ScriptUtils context;
+
+  @BeforeClass
+  public static void initialSetup() throws IOException {
+    context = ScriptUtils.instance();
+    context.initialSetup();
+  }
+
+  /**
+   * Test the simplest case: use the $DRILL_HOME/conf directory and default log
+   * location. Non-existent drill-env.sh and drill-config.sh files. Everything
+   * is at its Drill-provided defaults. Then, try overriding each user-settable
+   * environment variable in the environment (which simulates what YARN might
+   * do.)
+   */
+
+  @Test
+  public void testStockCombined() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    // No drill-env.sh, no distrib-env.sh
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .run();
+      assertEquals(0, result.returnCode);
+      result.validateJava();
+      result.validateStockArgs();
+      result.validateClassPath(ScriptUtils.stdCp);
+      result.validateStdOut();
+      result.validateStdErr();
+      result.validateDrillLog();
+    }
+
+    // As above, but pass an argument.
+
+    {
+      String propArg = "-Dproperty=value";
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withArg(propArg).run();
+      assertEquals(0, result.returnCode);
+      result.validateStdOut();
+      result.validateArg(propArg);
+    }
+
+    // Custom Java opts to achieve the same result
+
+    {
+      String propArg = "-Dproperty=value";
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILL_JAVA_OPTS", propArg).run();
+      assertEquals(0, result.returnCode);
+      result.validateStockArgs(); // Should not lose standard JVM args
+      result.validateStdOut();
+      result.validateArg(propArg);
+    }
+
+    // Custom Drillbit Java Opts to achieve the same result
+
+    {
+      String propArg = "-Dproperty2=value2";
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILLBIT_JAVA_OPTS", propArg).run();
+      assertEquals(0, result.returnCode);
+      result.validateStockArgs(); // Should not lose standard JVM args
+      result.validateStdOut();
+      result.validateArg(propArg);
+    }
+
+    // Both sets of options
+
+    {
+      String propArg = "-Dproperty=value";
+      String propArg2 = "-Dproperty2=value2";
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILL_JAVA_OPTS", propArg)
+          .addEnv("DRILLBIT_JAVA_OPTS", propArg2).run();
+      assertEquals(0, result.returnCode);
+      result.validateArgs(new String[] { propArg, propArg2 });
+    }
+
+    // Custom heap memory
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILL_HEAP", "5G").run();
+      result.validateArgs(new String[] { "-Xms5G", "-Xmx5G" });
+    }
+
+    // Custom direct memory
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILL_MAX_DIRECT_MEMORY", "7G").run();
+      result.validateArg("-XX:MaxDirectMemorySize=7G");
+    }
+
+    // Enable GC logging
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("SERVER_LOG_GC", "1").run();
+      String logTail = context.testLogDir.getName() + "/drillbit.gc";
+      result.validateArgRegex("-Xloggc:.*/" + logTail);
+    }
+
+    // Max Perm Size
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILLBIT_MAX_PERM", "600M").run();
+      result.validateArg("-XX:MaxPermSize=600M");
+    }
+
+    // Code cache size
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILLBIT_CODE_CACHE_SIZE", "2G").run();
+      result.validateArg("-XX:ReservedCodeCacheSize=2G");
+    }
+  }
+
+  /**
+   * Use the "stock" setup, but add each custom bit of the class path to ensure
+   * it is passed to the Drillbit.
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testClassPath() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    File extrasDir = context.createDir(new File(context.testDir, "extras"));
+    File hadoopJar = context.makeDummyJar(extrasDir, "hadoop");
+    File hbaseJar = context.makeDummyJar(extrasDir, "hbase");
+    File prefixJar = context.makeDummyJar(extrasDir, "prefix");
+    File cpJar = context.makeDummyJar(extrasDir, "cp");
+    File extnJar = context.makeDummyJar(extrasDir, "extn");
+    File toolsJar = context.makeDummyJar(extrasDir, "tools");
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILL_CLASSPATH_PREFIX", prefixJar.getAbsolutePath()).run();
+      result.validateClassPath(prefixJar.getAbsolutePath());
+    }
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILL_TOOL_CP", toolsJar.getAbsolutePath()).run();
+      result.validateClassPath(toolsJar.getAbsolutePath());
+    }
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("HADOOP_CLASSPATH", hadoopJar.getAbsolutePath()).run();
+      result.validateClassPath(hadoopJar.getAbsolutePath());
+    }
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("HBASE_CLASSPATH", hbaseJar.getAbsolutePath()).run();
+      result.validateClassPath(hbaseJar.getAbsolutePath());
+    }
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("EXTN_CLASSPATH", extnJar.getAbsolutePath()).run();
+      result.validateClassPath(extnJar.getAbsolutePath());
+    }
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILL_CLASSPATH", cpJar.getAbsolutePath()).run();
+      result.validateClassPath(cpJar.getAbsolutePath());
+    }
+
+    // Site jars not on path if not created
+
+    File siteJars = new File(siteDir, "jars");
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN).run();
+      assertFalse(result.classPathContains(siteJars.getAbsolutePath()));
+    }
+
+    // Site/jars on path if exists
+
+    context.createDir(siteJars);
+    context.makeDummyJar(siteJars, "site");
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN).run();
+      result.validateClassPath(siteJars.getAbsolutePath() + "/*");
+    }
+  }
+
+  /**
+   * Create a custom log folder location.
+   */
+
+  @Test
+  public void testLogDir() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+    File logsDir = context.createDir(new File(context.testDir, "logs"));
+    context.removeDir(new File(context.testDrillHome, "log"));
+
+    {
+      String logPath = logsDir.getAbsolutePath();
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILL_LOG_DIR", logPath).withLogDir(logsDir).run();
+      assertEquals(0, result.returnCode);
+      result.validateArgs(
+          new String[] { "-Dlog.path=" + logPath + "/drillbit.log",
+              "-Dlog.query.path=" + logPath + "/drillbit_queries.json", });
+      result.validateStdOut();
+      result.validateStdErr();
+      result.validateDrillLog();
+    }
+
+  }
+
+  /**
+   * Create a custom Java lib path. This uses the new DRILL_JAVA_LIB_PATH
+   * variable.
+   */
+
+  @Test
+  public void testLibPath() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+    File logsDir = context.createDir(new File(context.testDir, "logs"));
+    context.removeDir(new File(context.testDrillHome, "log"));
+
+    {
+      String logPath = logsDir.getAbsolutePath();
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILL_LOG_DIR", logPath).withLogDir(logsDir).run();
+      assertEquals(0, result.returnCode);
+      result.validateArgs(
+          new String[] { "-Dlog.path=" + logPath + "/drillbit.log",
+              "-Dlog.query.path=" + logPath + "/drillbit_queries.json", });
+      result.validateStdOut();
+      result.validateStdErr();
+      result.validateDrillLog();
+    }
+
+  }
+
+  /**
+   * Try setting custom environment variable values in drill-env.sh in the
+   * $DRILL_HOME/conf location.
+   */
+
+  @Test
+  public void testDrillEnv() throws IOException {
+    doEnvFileTest("drill-env.sh");
+  }
+
+  /**
+   * Repeat the above test using distrib-env.sh in the $DRILL_HOME/conf
+   * location.
+   */
+
+  @Test
+  public void testDistribEnv() throws IOException {
+    doEnvFileTest("distrib-env.sh");
+  }
+
+  /**
+   * Implementation of the drill-env.sh and distrib-env.sh tests.
+   */
+
+  private void doEnvFileTest(String fileName) throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    /**
+     * Set all properties in the env file.
+     */
+
+    Map<String, String> drillEnv = new HashMap<>();
+    String propArg = "-Dproperty=value";
+    drillEnv.put("DRILL_JAVA_OPTS", propArg);
+    drillEnv.put("DRILL_HEAP", "5G");
+    drillEnv.put("DRILL_MAX_DIRECT_MEMORY", "7G");
+    drillEnv.put("SERVER_LOG_GC", "1");
+    drillEnv.put("DRILLBIT_MAX_PERM", "600M");
+    drillEnv.put("DRILLBIT_CODE_CACHE_SIZE", "2G");
+    context.createEnvFile(new File(siteDir, fileName), drillEnv);
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN).run();
+      assertEquals(0, result.returnCode);
+
+      String expectedArgs[] = {
+          propArg,
+          "-Xms5G", "-Xmx5G",
+          "-XX:MaxDirectMemorySize=7G",
+          "-XX:ReservedCodeCacheSize=2G",
+          "-XX:MaxPermSize=600M"
+      };
+
+      result.validateArgs(expectedArgs);
+      String logTail = context.testLogDir.getName() + "/drillbit.gc";
+      result.validateArgRegex("-Xloggc:.*/" + logTail);
+    }
+
+    // Change some drill-env.sh options in the environment.
+    // The generated drill-env.sh should allow overrides.
+    // (The generated form is the form that customers should use.)
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("SERVER_LOG_GC", "0")
+          .addEnv("DRILL_MAX_DIRECT_MEMORY", "9G")
+          .run();
+      assertEquals(0, result.returnCode);
+      result.validateArg("-XX:MaxDirectMemorySize=9G");
+      result.validateArg("-XX:MaxPermSize=600M");
+      String logTail = context.testDrillHome.getName() + "/log/drillbit.gc";
+      assertFalse(result.containsArgRegex("-Xloggc:.*/" + logTail));
+    }
+  }
+
+  /**
+   * Test that drill-env.sh overrides distrib-env.sh, and that the environment
+   * overrides both. Assumes the basics were tested above.
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testDrillAndDistribEnv() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    Map<String, String> distribEnv = new HashMap<>();
+    distribEnv.put("DRILL_HEAP", "5G");
+    distribEnv.put("DRILL_MAX_DIRECT_MEMORY", "7G");
+    distribEnv.put("DRILLBIT_MAX_PERM", "600M");
+    context.createEnvFile(new File(siteDir, "distrib-env.sh"), distribEnv);
+
+    Map<String, String> drillEnv = new HashMap<>();
+    drillEnv.put("DRILL_HEAP", "6G");
+    drillEnv.put("DRILL_MAX_DIRECT_MEMORY", "9G");
+    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv);
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN).run();
+      assertEquals(0, result.returnCode);
+      String expectedArgs[] = {
+          "-Xms6G", "-Xmx6G",
+          "-XX:MaxDirectMemorySize=9G",
+          "-XX:MaxPermSize=600M",
+          "-XX:ReservedCodeCacheSize=1G" // Default
+      };
+
+      result.validateArgs(expectedArgs);
+    }
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILL_MAX_DIRECT_MEMORY", "5G").run();
+      assertEquals(0, result.returnCode);
+      String expectedArgs[] = {
+          "-Xms6G", "-Xmx6G",
+          "-XX:MaxDirectMemorySize=5G",
+          "-XX:MaxPermSize=600M",
+          "-XX:ReservedCodeCacheSize=1G" // Default
+      };
+
+      result.validateArgs(expectedArgs);
+    }
+  }
+
+  @Test
+  public void testBadSiteDir() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.removeDir(siteDir);
+
+    // Directory does not exist.
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withSite(siteDir).run();
+      assertEquals(1, result.returnCode);
+      assertTrue(result.stderr.contains("Config dir does not exist"));
+    }
+
+    // Not a directory
+
+    context.writeFile(siteDir, "dummy");
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withSite(siteDir).run();
+      assertEquals(1, result.returnCode);
+      assertTrue(result.stderr.contains("Config dir does not exist"));
+    }
+
+    // Directory exists, but drill-override.conf does not
+
+    siteDir.delete();
+    context.createDir(siteDir);
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withSite(siteDir).run();
+      assertEquals(1, result.returnCode);
+      assertTrue(result.stderr.contains("Drill config file missing"));
+    }
+  }
+
+  /**
+   * Move configuration to a site folder out of $DRILL_HOME/conf. The site
+   * folder can contain code (which is why we call it "site" and not "config".)
+   * The site directory can be passed to the Drillbit in several ways.
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testSiteDir() throws IOException {
+    context.createMockDistrib();
+    File confDir = new File(context.testDrillHome, "conf");
+    context.createDir(confDir);
+    File siteDir = new File(context.testDir, "site");
+    context.createMockConf(siteDir);
+
+    // Dummy drill-env.sh to simulate the shipped "example" file.
+
+    context.writeFile(new File(confDir, "drill-env.sh"),
+        "#!/usr/bin/env bash\n" + "# Example file");
+    File siteJars = new File(siteDir, "jars");
+
+    Map<String, String> distribEnv = new HashMap<>();
+    distribEnv.put("DRILL_HEAP", "5G");
+    distribEnv.put("DRILL_MAX_DIRECT_MEMORY", "7G");
+    distribEnv.put("DRILLBIT_MAX_PERM", "600M");
+    context.createEnvFile(new File(confDir, "distrib-env.sh"), distribEnv);
+
+    Map<String, String> drillEnv = new HashMap<>();
+    drillEnv.put("DRILL_HEAP", "6G");
+    drillEnv.put("DRILL_MAX_DIRECT_MEMORY", "9G");
+    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv);
+
+    String expectedArgs[] = {
+        "-Xms6G", "-Xmx6G",
+        "-XX:MaxDirectMemorySize=9G",
+        "-XX:MaxPermSize=600M",
+        "-XX:ReservedCodeCacheSize=1G" // Default
+    };
+
+    // Site set using argument
+
+    {
+      // Use --config explicitly
+
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withArg("--config")
+          .withArg(siteDir.getAbsolutePath())
+          .run();
+      assertEquals(0, result.returnCode);
+      result.validateArgs(expectedArgs);
+      result.validateClassPath(siteDir.getAbsolutePath());
+    }
+
+    {
+      RunResult result = new DrillbitRun()
+          .withArg("--config")
+          .withArg(siteDir.getAbsolutePath())
+          .withArg(DrillbitRun.DRILLBIT_RUN)
+          .run();
+      assertEquals(0, result.returnCode);
+      result.validateArgs(expectedArgs);
+    }
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withSite(siteDir)
+          .run();
+      assertEquals(0, result.returnCode);
+      result.validateArgs(expectedArgs);
+    }
+
+    // Site argument and argument to Drillbit
+
+    {
+      String propArg = "-Dproperty=value";
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withSite(siteDir)
+          .withArg(propArg)
+          .run( );
+      assertEquals(0, result.returnCode);
+      result.validateArgs(expectedArgs);
+      result.validateArg(propArg);
+    }
+
+    // Set as an environment variable
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .addEnv("DRILL_CONF_DIR",siteDir.getAbsolutePath())
+          .run();
+      assertEquals(0, result.returnCode);
+      result.validateArgs(expectedArgs);
+    }
+
+    // Site jars not on path if not created
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withSite(siteDir)
+          .run();
+      assertFalse(result.classPathContains(siteJars.getAbsolutePath()));
+    }
+
+    // Site/jars on path if exists
+
+    context.createDir(siteJars);
+    context.makeDummyJar(siteJars, "site");
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withSite(siteDir)
+          .run();
+      assertTrue(result.classPathContains(siteJars.getAbsolutePath() + "/*"));
+    }
+  }
+
+  /**
+   * Test the Java library path. Three sources:
+   * <ol>
+   * <li>DRILL_JAVA_LIB_PATH Set in drill-env.sh</li>
+   * <li>DOY_JAVA_LIB_PATH passed in from an env. var.</li>
+   * <li>$DRILL_SITE/lib, if it exists.</li>
+   * </ol>
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testJavaLibDir() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    // Stock run: no lib dir.
+
+    String prefix = "-Djava.library.path=";
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .run();
+      assertFalse(result.containsArgRegex(prefix + ".*"));
+      assertNull(result.libPath);
+    }
+
+    // Old-style argument in DRILL_JAVA_OPTS
+
+    {
+      Map<String, String> env = new HashMap<>();
+      env.put("DRILL_JAVA_OPTS", prefix + "/foo/bar:/foo/mumble");
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withEnvironment(env)
+          .run();
+      assertTrue(result.containsArgRegex(prefix + ".*"));
+      assertNotNull(result.libPath);
+      assertEquals(2, result.libPath.length);
+      assertEquals("/foo/bar", result.libPath[0]);
+      assertEquals("/foo/mumble", result.libPath[1]);
+    }
+
+    // New-style argument in DRILL_JAVA_LIB_PATH
+
+    {
+      Map<String, String> env = new HashMap<>();
+      env.put("DRILL_JAVA_LIB_PATH", "/foo/bar:/foo/mumble");
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withEnvironment(env)
+          .run();
+      assertTrue(result.containsArgRegex(prefix + ".*"));
+      assertNotNull(result.libPath);
+      assertEquals(2, result.libPath.length);
+      assertEquals("/foo/bar", result.libPath[0]);
+      assertEquals("/foo/mumble", result.libPath[1]);
+    }
+
+    // YARN argument in DOY_JAVA_LIB_PATH
+
+    {
+      Map<String, String> env = new HashMap<>();
+      env.put("DOY_JAVA_LIB_PATH", "/foo/bar:/foo/mumble");
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withEnvironment(env)
+          .run();
+      assertTrue(result.containsArgRegex(prefix + ".*"));
+      assertNotNull(result.libPath);
+      assertEquals(2, result.libPath.length);
+      assertEquals("/foo/bar", result.libPath[0]);
+      assertEquals("/foo/mumble", result.libPath[1]);
+    }
+
+    // Both DRILL_JAVA_LIB_PATH and DOY_JAVA_LIB_PATH
+
+    {
+      Map<String, String> env = new HashMap<>();
+      env.put("DRILL_JAVA_LIB_PATH", "/foo/bar:/foo/mumble");
+      env.put("DOY_JAVA_LIB_PATH", "/doy/bar:/doy/mumble");
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withEnvironment(env)
+          .run();
+      assertTrue(result.containsArgRegex(prefix + ".*"));
+      assertNotNull(result.libPath);
+      assertEquals(4, result.libPath.length);
+      assertEquals("/doy/bar", result.libPath[0]);
+      assertEquals("/doy/mumble", result.libPath[1]);
+      assertEquals("/foo/bar", result.libPath[2]);
+      assertEquals("/foo/mumble", result.libPath[3]);
+    }
+
+    // Site directory with a lib folder
+
+    siteDir = new File(context.testDir, "site");
+    context.createMockConf(siteDir);
+    File libDir = new File(siteDir, "lib");
+    context.createDir(libDir);
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withSite(siteDir)
+          .run();
+      assertTrue(result.containsArgRegex(prefix + ".*"));
+      assertNotNull(result.libPath);
+      assertEquals(1, result.libPath.length);
+      assertEquals(libDir.getAbsolutePath(), result.libPath[0]);
+    }
+
+    // The whole enchilada: all three settings.
+
+    {
+      Map<String, String> env = new HashMap<>();
+      env.put("DRILL_JAVA_LIB_PATH", "/foo/bar:/foo/mumble");
+      env.put("DOY_JAVA_LIB_PATH", "/doy/bar:/doy/mumble");
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RUN)
+          .withSite(siteDir)
+          .withEnvironment(env)
+          .run();
+      assertTrue(result.containsArgRegex(prefix + ".*"));
+      assertNotNull(result.libPath);
+      assertEquals(5, result.libPath.length);
+      assertEquals(libDir.getAbsolutePath(), result.libPath[0]);
+      assertEquals("/doy/bar", result.libPath[1]);
+      assertEquals("/doy/mumble", result.libPath[2]);
+      assertEquals("/foo/bar", result.libPath[3]);
+      assertEquals("/foo/mumble", result.libPath[4]);
+    }
+  }
+
+  /**
+   * Test running a (simulated) Drillbit as a daemon with start, status, stop.
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testStockDaemon() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    // No drill-env.sh, no distrib-env.sh
+
+    File pidFile;
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_START).start();
+      assertEquals(0, result.returnCode);
+      result.validateJava();
+      result.validateStockArgs();
+      result.validateClassPath(ScriptUtils.stdCp);
+      assertTrue(result.stdout.contains("Starting drillbit, logging"));
+      assertTrue(result.log.contains("Starting drillbit on"));
+      assertTrue(result.log.contains("Drill Log Message"));
+      assertTrue(result.out.contains("Drill Stdout Message"));
+      assertTrue(result.out.contains("Stderr Message"));
+      pidFile = result.pidFile;
+    }
+
+    // Save the pid file for reuse.
+
+    assertTrue(pidFile.exists());
+    File saveDir = new File(context.testDir, "save");
+    context.createDir(saveDir);
+    File savedPidFile = new File(saveDir, pidFile.getName());
+    context.copyFile(pidFile, savedPidFile);
+
+    // Status should be running
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_STATUS).run( );
+      assertEquals(0, result.returnCode);
+      assertTrue(result.stdout.contains("drillbit is running"));
+    }
+
+    // Start should refuse to start a second Drillbit.
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_START).start();
+      assertEquals(1, result.returnCode);
+      assertTrue(
+          result.stdout.contains("drillbit is already running as process"));
+    }
+
+    // Normal start, allow normal shutdown
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_STOP).run();
+      assertEquals(0, result.returnCode);
+      assertTrue(result.log.contains("Terminating drillbit pid"));
+      assertTrue(result.stdout.contains("Stopping drillbit"));
+    }
+
+    // Status should report no drillbit (no pid file)
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_STATUS).run();
+      assertEquals(1, result.returnCode);
+      assertTrue(result.stdout.contains("drillbit is not running"));
+    }
+
+    // Stop should report no pid file
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_STOP).run();
+      assertEquals(1, result.returnCode);
+      assertTrue(
+          result.stdout.contains("No drillbit to stop because no pid file"));
+    }
+
+    // Get nasty. Put the pid file back. But, there is no process with that pid.
+
+    context.copyFile(savedPidFile, pidFile);
+
+    // Status should now complain.
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_STATUS).run();
+      assertEquals(1, result.returnCode);
+      assertTrue(result.stdout
+          .contains("file is present but drillbit is not running"));
+    }
+
+    // As should stop.
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_STOP).run();
+      assertEquals(1, result.returnCode);
+      assertTrue(
+          result.stdout.contains("No drillbit to stop because kill -0 of pid"));
+    }
+  }
+
+  @Test
+  public void testStockDaemonWithArg() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    // As above, but pass an argument.
+
+    {
+      String propArg = "-Dproperty=value";
+      DrillbitRun runner = new DrillbitRun(DrillbitRun.DRILLBIT_START);
+      runner.withArg(propArg);
+      RunResult result = runner.start();
+      assertEquals(0, result.returnCode);
+      result.validateArg(propArg);
+    }
+
+    validateAndCloseDaemon(null);
+  }
+
+  private void validateAndCloseDaemon(File siteDir) throws IOException {
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_STATUS)
+          .withSite(siteDir)
+          .run();
+      assertEquals(0, result.returnCode);
+      assertTrue(result.stdout.contains("drillbit is running"));
+    }
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_STOP)
+          .withSite(siteDir)
+          .run();
+      assertEquals(0, result.returnCode);
+    }
+  }
+
+  /**
+   * The Daemon process creates a pid file. Verify that the DRILL_PID_DIR can be
+   * set to put the pid file in a custom location. The test is done with the
+   * site (conf) dir in the default location.
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testPidDir() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+    File pidDir = context.createDir(new File(context.testDir, "pid"));
+    Map<String, String> drillEnv = new HashMap<>();
+    drillEnv.put("DRILL_PID_DIR", pidDir.getAbsolutePath());
+    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv);
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_START)
+          .withPidDir(pidDir)
+          .start();
+      assertEquals(0, result.returnCode);
+      assertTrue(result.pidFile.getParentFile().equals(pidDir));
+      assertTrue(result.pidFile.exists());
+    }
+
+    validateAndCloseDaemon(null);
+  }
+
+  /**
+   * Test a custom site directory with the Drill daemon process. The custom
+   * directory contains a drill-env.sh with a custom option. Verify that that
+   * option is picked up when starting Drill.
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testSiteDirWithDaemon() throws IOException {
+    context.createMockDistrib();
+
+    File siteDir = new File(context.testDir, "site");
+    context.createMockConf(siteDir);
+
+    Map<String, String> drillEnv = new HashMap<>();
+    drillEnv.put("DRILL_MAX_DIRECT_MEMORY", "9G");
+    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv);
+
+    // Use the -site (--config) option.
+
+    {
+      DrillbitRun runner = new DrillbitRun(DrillbitRun.DRILLBIT_START);
+      runner.withSite(siteDir);
+      RunResult result = runner.start();
+      assertEquals(0, result.returnCode);
+      result.validateArg("-XX:MaxDirectMemorySize=9G");
+    }
+
+    validateAndCloseDaemon(siteDir);
+
+    // Set an env var.
+
+    {
+      DrillbitRun runner = new DrillbitRun(DrillbitRun.DRILLBIT_START);
+      runner.addEnv("DRILL_CONF_DIR", siteDir.getAbsolutePath());
+      RunResult result = runner.start();
+      assertEquals(0, result.returnCode);
+      result.validateArg("-XX:MaxDirectMemorySize=9G");
+    }
+
+    validateAndCloseDaemon(siteDir);
+  }
+
+  /**
+   * Launch the Drill daemon using a custom log file location. The config is in
+   * the default location.
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testLogDirWithDaemon() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+    File logsDir = context.createDir(new File(context.testDir, "logs"));
+    context.removeDir(new File(context.testDrillHome, "log"));
+    Map<String, String> drillEnv = new HashMap<>();
+    drillEnv.put("DRILL_LOG_DIR", logsDir.getAbsolutePath());
+    context.createEnvFile(new File(siteDir, "drill-env.sh"), drillEnv);
+
+    {
+      DrillbitRun runner = new DrillbitRun(DrillbitRun.DRILLBIT_START);
+      runner.withLogDir(logsDir);
+      RunResult result = runner.start();
+      assertEquals(0, result.returnCode);
+      assertNotNull(result.logFile);
+      assertTrue(result.logFile.getParentFile().equals(logsDir));
+      assertTrue(result.logFile.exists());
+      assertNotNull(result.outFile);
+      assertTrue(result.outFile.getParentFile().equals(logsDir));
+      assertTrue(result.outFile.exists());
+    }
+
+    validateAndCloseDaemon(null);
+  }
+
+  /**
+   * Some distributions create symlinks to drillbit.sh in standard locations
+   * such as /usr/bin. Because drillbit.sh uses its own location to compute
+   * DRILL_HOME, it must handle symlinks. This test verifies that process.
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testDrillbitSymlink() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    File drillbitFile = new File(context.testDrillHome, "bin/drillbit.sh");
+    File linksDir = context.createDir(new File(context.testDir, "links"));
+    File link = new File(linksDir, drillbitFile.getName());
+    try {
+      Files.createSymbolicLink(link.toPath(), drillbitFile.toPath());
+    } catch (UnsupportedOperationException e) {
+      // Well. This is a system without symlinks, so we won't be testing
+      // syminks here...
+
+      return;
+    }
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_START).start();
+      assertEquals(0, result.returnCode);
+      assertEquals(result.pidFile.getParentFile(), context.testDrillHome);
+    }
+    validateAndCloseDaemon(null);
+  }
+
+  /**
+   * Test the restart command of drillbit.sh
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testRestart() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    int firstPid;
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_START).start();
+      assertEquals(0, result.returnCode);
+      firstPid = result.getPid();
+    }
+
+    // Make sure it is running.
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_STATUS)
+          .withSite(siteDir)
+          .run();
+      assertEquals(0, result.returnCode);
+      assertTrue(result.stdout.contains("drillbit is running"));
+    }
+
+    // Restart. Should get new pid.
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_RESTART).start();
+      assertEquals(0, result.returnCode);
+      int secondPid = result.getPid();
+      assertNotEquals(firstPid, secondPid);
+    }
+
+    validateAndCloseDaemon(null);
+  }
+
+  /**
+   * Simulate a Drillbit that refuses to die. The stop script wait a while, then
+   * forces killing.
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testForcedKill() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    {
+      DrillbitRun runner = new DrillbitRun(DrillbitRun.DRILLBIT_START);
+      runner.addEnv("PRETEND_HUNG", "1");
+      RunResult result = runner.start();
+      assertEquals(0, result.returnCode);
+    }
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_STATUS)
+          .preserveLogs()
+          .run();
+      assertEquals(0, result.returnCode);
+      assertTrue(result.stdout.contains("drillbit is running"));
+    }
+
+    {
+      RunResult result = new DrillbitRun(DrillbitRun.DRILLBIT_STOP)
+          .addEnv("DRILL_STOP_TIMEOUT","5")
+          .preserveLogs()
+          .run();
+      assertEquals(0, result.returnCode);
+      assertTrue(result.stdout.contains("drillbit did not complete after"));
+    }
+  }
+
+  /**
+   * Verify the basics of the sqlline script, including the env vars that can be
+   * customized. Also validate running in embedded mode (using drillbit memory
+   * and other options.)
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testSqlline() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    int stockArgCount;
+    {
+      // Out-of-the-box sqlline
+
+      RunResult result = new ScriptRunner("sqlline").run();
+      assertEquals(0, result.returnCode);
+      result.validateJava();
+      result.validateClassPath(ScriptUtils.stdCp);
+      assertTrue(result.containsArgsRegex(ScriptUtils.sqlLineArgs));
+      stockArgCount = result.echoArgs.size();
+    }
+
+    {
+      RunResult result = new ScriptRunner("sqlline")
+          .withArg("arg1")
+          .withArg("arg2")
+          .run( );
+      assertTrue(result.containsArg("arg1"));
+      assertTrue(result.containsArg("arg2"));
+    }
+    {
+      // Change drill memory and other drill-specific
+      // settings: should not affect sqlline
+
+      Map<String, String> drillEnv = new HashMap<>();
+      drillEnv.put("DRILL_JAVA_OPTS", "-Dprop=value");
+      drillEnv.put("DRILL_HEAP", "5G");
+      drillEnv.put("DRILL_MAX_DIRECT_MEMORY", "7G");
+      drillEnv.put("SERVER_LOG_GC", "1");
+      drillEnv.put("DRILLBIT_MAX_PERM", "600M");
+      drillEnv.put("DRILLBIT_CODE_CACHE_SIZE", "2G");
+      RunResult result = new ScriptRunner("sqlline")
+          .withEnvironment(drillEnv)
+          .run();
+      assertTrue(result.containsArgsRegex(ScriptUtils.sqlLineArgs));
+
+      // Nothing new should have been added
+
+      assertEquals(stockArgCount, result.echoArgs.size());
+    }
+    {
+      // Change client memory: should affect sqlline
+
+      Map<String, String> shellEnv = new HashMap<>();
+      shellEnv.put("CLIENT_GC_OPTS", "-XX:+UseG1GC");
+      shellEnv.put("SQLLINE_JAVA_OPTS", "-XX:MaxPermSize=256M");
+      RunResult result = new ScriptRunner("sqlline")
+          .withEnvironment(shellEnv)
+          .run();
+      assertTrue(result.containsArg("-XX:MaxPermSize=256M"));
+      assertTrue(result.containsArg("-XX:+UseG1GC"));
+    }
+    {
+      // Change drill memory and other drill-specific
+      // settings: then set the "magic" variable that says
+      // that Drill is embedded. The scripts should now use
+      // the Drillbit options.
+
+      Map<String, String> drillEnv = new HashMap<>();
+      drillEnv.put("DRILL_JAVA_OPTS", "-Dprop=value");
+      drillEnv.put("DRILL_HEAP", "5G");
+      drillEnv.put("DRILL_MAX_DIRECT_MEMORY", "7G");
+      drillEnv.put("SERVER_LOG_GC", "1");
+      drillEnv.put("DRILLBIT_MAX_PERM", "600M");
+      drillEnv.put("DRILLBIT_CODE_CACHE_SIZE", "2G");
+      drillEnv.put("DRILL_EMBEDDED", "1");
+      RunResult result = new ScriptRunner("sqlline")
+          .withEnvironment(drillEnv)
+          .run();
+
+      String expectedArgs[] = {
+          "-Dprop=value",
+          "-Xms5G", "-Xmx5G",
+          "-XX:MaxDirectMemorySize=7G",
+          "-XX:ReservedCodeCacheSize=2G",
+          "-XX:MaxPermSize=600M"
+      };
+
+      result.validateArgs(expectedArgs);
+      assertTrue(result.containsArg("sqlline.SqlLine"));
+    }
+  }
+
+  /**
+   * Verify that the sqlline client works with the --site option by customizing
+   * items in the site directory.
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testSqllineSiteDir() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDir, "site");
+    context.createMockConf(siteDir);
+
+    // Dummy drill-env.sh to simulate the shipped "example" file
+    // with some client-specific changes.
+
+    context.writeFile( new File( siteDir, "drill-env.sh" ),
+        "#!/usr/bin/env bash\n" +
+        "# Example file\n" +
+        "export SQLLINE_JAVA_OPTS=\"-XX:MaxPermSize=256M\"\n"
+        );
+    File siteJars = new File(siteDir, "jars");
+    context.createDir(siteJars);
+    context.makeDummyJar(siteJars, "site");
+    {
+      RunResult result = new ScriptRunner("sqlline").withSite(siteDir).run();
+      assertEquals(0, result.returnCode);
+      assertTrue(result.containsArg("-XX:MaxPermSize=256M"));
+      result.validateClassPath(siteJars.getAbsolutePath() + "/*");
+    }
+  }
+
+  /**
+   * Tests the three scripts that wrap sqlline for specific purposes:
+   * <ul>
+   * <li>drill-conf — Wrapper for sqlline, uses drill config to find Drill.
+   * Seems this one needs fixing to use a config other than the hard-coded
+   * $DRILL_HOME/conf location.</li>
+   * <li>drill-embedded — Starts a drill “embedded” in SqlLine, using a local
+   * ZK.</li>
+   * <li>drill-localhost — Wrapper for sqlline, uses a local ZK.</li>
+   * </ul>
+   *
+   * Of these, drill-embedded runs an embedded Drillbit and so should use the
+   * Drillbit memory options. The other two are clients, but with simple default
+   * options for finding the Drillbit.
+   * <p>
+   * Because the scripts are simple wrappers, all we do is verify that the right
+   * "extra" options are set, not the fundamentals (which were already covered
+   * in the sqlline tests.)
+   *
+   * @throws IOException
+   */
+
+  @Test
+  public void testSqllineWrappers() throws IOException {
+    context.createMockDistrib();
+    File siteDir = new File(context.testDrillHome, "conf");
+    context.createMockConf(siteDir);
+
+    {
+      // drill-conf: just adds a stub JDBC connect string.
+
+      RunResult result = new ScriptRunner("drill-conf")
+          .withArg("arg1")
+          .run();
+      assertEquals(0, result.returnCode);
+      result.validateJava();
+      result.validateClassPath(ScriptUtils.stdCp);
+      assertTrue(result.containsArgsRegex(ScriptUtils.sqlLineArgs));
+      assertTrue(result.containsArg("-u"));
+      assertTrue(result.containsArg("jdbc:drill:"));
+      assertTrue(result.containsArg("arg1"));
+    }
+
+    {
+      // drill-localhost: Adds a JDBC connect string to a drillbit
+      // on the localhost
+
+      RunResult result = new ScriptRunner("drill-localhost")
+          .withArg("arg1")
+          .run();
+      assertEquals(0, result.returnCode);
+      result.validateJava();
+      result.validateClassPath(ScriptUtils.stdCp);
+      assertTrue(result.containsArgsRegex(ScriptUtils.sqlLineArgs));
+      assertTrue(result.containsArg("-u"));
+      assertTrue(result.containsArg("jdbc:drill:drillbit=localhost"));
+      assertTrue(result.containsArg("arg1"));
+    }
+
+    {
+      // drill-embedded: Uses drillbit startup options and
+      // connects to the embedded drillbit.
+
+      RunResult result = new ScriptRunner("drill-embedded")
+          .withArg("arg1")
+          .run();
+      assertEquals(0, result.returnCode);
+      result.validateJava();
+      result.validateClassPath(ScriptUtils.stdCp);
+      assertTrue(result.containsArgsRegex(ScriptUtils.sqlLineArgs));
+      assertTrue(result.containsArg("-u"));
+      assertTrue(result.containsArg("jdbc:drill:zk=local"));
+      assertTrue(result.containsArg("-Xms4G"));
+      assertTrue(result.containsArg("-XX:MaxDirectMemorySize=8G"));
+      assertTrue(result.containsArg("arg1"));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/java/org/apache/drill/yarn/zk/TestAmRegistration.java
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/java/org/apache/drill/yarn/zk/TestAmRegistration.java b/drill-yarn/src/test/java/org/apache/drill/yarn/zk/TestAmRegistration.java
new file mode 100644
index 0000000..31c6349
--- /dev/null
+++ b/drill-yarn/src/test/java/org/apache/drill/yarn/zk/TestAmRegistration.java
@@ -0,0 +1,129 @@
+/*
+ * 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 org.junit.Assert.*;
+
+import org.apache.curator.test.TestingServer;
+import org.apache.drill.yarn.appMaster.AMRegistrar.AMRegistrationException;
+import org.junit.Test;
+
+public class TestAmRegistration {
+  private static final String TEST_CLUSTER_ID = "drillbits";
+  private static final String TEST_ZK_ROOT = "drill";
+  private static final String TEST_AM_HOST = "localhost";
+  private static final int TEST_AM_PORT = 8048;
+  private static final String TEST_APP_ID = "Application_001";
+
+  private static final String PROBE_AM_HOST = "somehost";
+  private static final String PROBE_APP_ID = "Application_002";
+  private static final int PROBE_AM_PORT = 8049;
+  private static final String PROBE_CLUSTER_ID = "second";
+  private static final String PROBE_ZK_ROOT = "myRoot";
+
+  @Test
+  public void testAMRegistry() throws Exception {
+    try (TestingServer server = new TestingServer()) {
+      server.start();
+      String connStr = server.getConnectString();
+      ZKClusterCoordinatorDriver driver = new ZKClusterCoordinatorDriver()
+          .setConnect(connStr, TEST_ZK_ROOT, TEST_CLUSTER_ID).build();
+
+      // Register an AM using the above.
+
+      driver.register(TEST_AM_HOST, TEST_AM_PORT, TEST_APP_ID);
+
+      // Simulate a second AM for the same cluster.
+
+      {
+        ZKClusterCoordinatorDriver driver2 = new ZKClusterCoordinatorDriver()
+            .setConnect(connStr, TEST_ZK_ROOT, TEST_CLUSTER_ID).build();
+
+        // Register an AM on the same (root, cluster id).
+
+        try {
+          driver.register(PROBE_AM_HOST, PROBE_AM_PORT, PROBE_APP_ID);
+          fail();
+        } catch (AMRegistrationException e) {
+          String msg = e.getMessage();
+          assertTrue(msg.contains("Application Master already exists"));
+          assertTrue(
+              msg.contains(" " + TEST_ZK_ROOT + "/" + TEST_CLUSTER_ID + " "));
+          assertTrue(msg.contains(" host: " + TEST_AM_HOST));
+          assertTrue(msg.contains(" Application ID: " + TEST_APP_ID));
+        }
+
+        driver2.close();
+      }
+
+      {
+        ZKClusterCoordinatorDriver driver2 = new ZKClusterCoordinatorDriver()
+            .setConnect(connStr, TEST_ZK_ROOT, PROBE_CLUSTER_ID).build();
+
+        // Register an AM on a different cluster id, same root.
+
+        try {
+          driver2.register(PROBE_AM_HOST, PROBE_AM_PORT, PROBE_APP_ID);
+        } catch (AMRegistrationException e) {
+          fail("Registration should be OK");
+        }
+
+        driver2.close();
+      }
+
+      {
+        ZKClusterCoordinatorDriver driver2 = new ZKClusterCoordinatorDriver()
+            .setConnect(connStr, PROBE_ZK_ROOT, TEST_CLUSTER_ID).build();
+
+        // Register an AM on a different root.
+
+        try {
+          driver2.register(PROBE_AM_HOST, PROBE_AM_PORT, PROBE_APP_ID);
+        } catch (AMRegistrationException e) {
+          fail("Registration should be OK");
+        }
+
+        driver2.close();
+      }
+
+      // First AM exits.
+
+      driver.close();
+
+      {
+        // Should be able to register an AM for the same cluster.
+
+        ZKClusterCoordinatorDriver driver2 = new ZKClusterCoordinatorDriver()
+            .setConnect(connStr, TEST_ZK_ROOT, TEST_CLUSTER_ID).build();
+
+        // Register an AM on the same (root, cluster id).
+
+        try {
+          driver2.register(PROBE_AM_HOST, PROBE_AM_PORT, PROBE_APP_ID);
+        } catch (AMRegistrationException e) {
+          fail("Registration should be OK");
+        }
+
+        driver2.close();
+      }
+
+      server.stop();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/java/org/apache/drill/yarn/zk/TestZkRegistry.java
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/java/org/apache/drill/yarn/zk/TestZkRegistry.java b/drill-yarn/src/test/java/org/apache/drill/yarn/zk/TestZkRegistry.java
new file mode 100644
index 0000000..b6f6540
--- /dev/null
+++ b/drill-yarn/src/test/java/org/apache/drill/yarn/zk/TestZkRegistry.java
@@ -0,0 +1,459 @@
+/*
+ * 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 org.junit.Assert.*;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryNTimes;
+import org.apache.curator.test.TestingServer;
+import org.apache.curator.x.discovery.ServiceInstance;
+import org.apache.drill.exec.coord.DrillServiceInstanceHelper;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.work.foreman.DrillbitStatusListener;
+import org.apache.drill.yarn.appMaster.EventContext;
+import org.apache.drill.yarn.appMaster.RegistryHandler;
+import org.apache.drill.yarn.appMaster.Task;
+import org.apache.drill.yarn.appMaster.TaskLifecycleListener.Event;
+import org.apache.drill.yarn.zk.ZKRegistry.DrillbitTracker;
+import org.junit.Test;
+
+/**
+ * Tests for the AM version of the cluster coordinator. The AM version has no
+ * dependencies on the DoY config system or other systems, making it easy to
+ * test in isolation using the Curator-provided test server.
+ */
+
+public class TestZkRegistry {
+  private static final String BARNEY_HOST = "barney";
+  private static final String WILMA_HOST = "wilma";
+  private static final String TEST_HOST = "host";
+  private static final String FRED_HOST = "fred";
+  public static final int TEST_USER_PORT = 123;
+  public static final int TEST_CONTROL_PORT = 456;
+  public static final int TEST_DATA_PORT = 789;
+  public static final String ZK_ROOT = "test-root";
+  public static final String CLUSTER_ID = "test-cluster";
+
+  /**
+   * Validate that the key format used for endpoint is the same as that
+   * generated for hosts coming from YARN.
+   */
+
+  @Test
+  public void testFormat() {
+    DrillbitEndpoint dbe = makeEndpoint(TEST_HOST);
+    assertEquals(makeKey(TEST_HOST), ZKClusterCoordinatorDriver.asString(dbe));
+
+    ZKClusterCoordinatorDriver driver = new ZKClusterCoordinatorDriver()
+        .setPorts(123, 456, 789);
+    assertEquals(makeKey(TEST_HOST), driver.toKey(TEST_HOST));
+
+    // Internal default ports (used mostly for testing.)
+
+    driver = new ZKClusterCoordinatorDriver();
+    assertEquals("fred:31010:31011:31012", driver.toKey(FRED_HOST));
+  }
+
+  public static String makeKey(String host) {
+    return host + ":" + TEST_USER_PORT + ":" + TEST_CONTROL_PORT + ":"
+        + TEST_DATA_PORT;
+  }
+
+  /**
+   * Basic setup: start a ZK and verify that the initial endpoint list is empty.
+   * Also validates the basics of the test setup (mock server, etc.)
+   *
+   * @throws Exception
+   */
+
+  @Test
+  public void testBasics() throws Exception {
+    try (TestingServer server = new TestingServer()) {
+      server.start();
+      String connStr = server.getConnectString();
+      ZKClusterCoordinatorDriver driver = new ZKClusterCoordinatorDriver()
+          .setConnect(connStr, "drill", "drillbits").build();
+      assertTrue(driver.getInitialEndpoints().isEmpty());
+      driver.close();
+      server.stop();
+    }
+  }
+
+  private class TestDrillbitStatusListener implements DrillbitStatusListener {
+    protected Set<DrillbitEndpoint> added;
+    protected Set<DrillbitEndpoint> removed;
+
+    @Override
+    public void drillbitUnregistered(
+        Set<DrillbitEndpoint> unregisteredDrillbits) {
+      removed = unregisteredDrillbits;
+    }
+
+    @Override
+    public void drillbitRegistered(Set<DrillbitEndpoint> registeredDrillbits) {
+      added = registeredDrillbits;
+    }
+
+    public void clear() {
+      added = null;
+      removed = null;
+    }
+  }
+
+  /**
+   * Test a typical life cycle: existing Drillbit on AM start, add a Drilbit
+   * (simulates a drillbit starting), and remove a drillbit (simulates a
+   * Drillbit ending).
+   *
+   * @throws Exception
+   */
+
+  @Test
+  public void testCycle() throws Exception {
+    TestingServer server = new TestingServer();
+    server.start();
+    String connStr = server.getConnectString();
+
+    CuratorFramework probeZk = connectToZk(connStr);
+    addDrillbit(probeZk, FRED_HOST);
+
+    ZKClusterCoordinatorDriver driver = new ZKClusterCoordinatorDriver()
+        .setConnect(connStr, ZK_ROOT, CLUSTER_ID).build();
+    List<DrillbitEndpoint> bits = driver.getInitialEndpoints();
+    assertEquals(1, bits.size());
+    assertEquals(makeKey(FRED_HOST),
+        ZKClusterCoordinatorDriver.asString(bits.get(0)));
+
+    TestDrillbitStatusListener listener = new TestDrillbitStatusListener();
+    driver.addDrillbitListener(listener);
+
+    addDrillbit(probeZk, WILMA_HOST);
+    Thread.sleep(50);
+    assertNull(listener.removed);
+    assertNotNull(listener.added);
+    assertEquals(1, listener.added.size());
+    for (DrillbitEndpoint dbe : listener.added) {
+      assertEquals(makeKey(WILMA_HOST),
+          ZKClusterCoordinatorDriver.asString(dbe));
+    }
+
+    listener.clear();
+    removeDrillbit(probeZk, FRED_HOST);
+    Thread.sleep(50);
+    assertNull(listener.added);
+    assertNotNull(listener.removed);
+    assertEquals(1, listener.removed.size());
+    for (DrillbitEndpoint dbe : listener.removed) {
+      assertEquals(makeKey(FRED_HOST),
+          ZKClusterCoordinatorDriver.asString(dbe));
+    }
+
+    probeZk.close();
+    driver.close();
+    server.stop();
+    server.close();
+  }
+
+  /**
+   * Make a Drill endpoint using the hard-coded test ports and the given host
+   * name.
+   *
+   * @param host
+   * @return
+   */
+
+  private DrillbitEndpoint makeEndpoint(String host) {
+    return DrillbitEndpoint.newBuilder().setAddress(host)
+        .setControlPort(TEST_CONTROL_PORT).setDataPort(TEST_DATA_PORT)
+        .setUserPort(TEST_USER_PORT).build();
+  }
+
+  /**
+   * Pretend to be a Drillbit creating its ZK entry. Real Drillbits use a GUID
+   * as the key, but we just use the host name, which is good enough for our
+   * purposes here.
+   *
+   * @param zk
+   * @param host
+   * @throws Exception
+   */
+
+  private void addDrillbit(CuratorFramework zk, String host) throws Exception {
+    DrillbitEndpoint dbe = makeEndpoint(host);
+    ServiceInstance<DrillbitEndpoint> si = ServiceInstance
+        .<DrillbitEndpoint> builder().name(CLUSTER_ID).payload(dbe).build();
+    byte data[] = DrillServiceInstanceHelper.SERIALIZER.serialize(si);
+    zk.create().forPath("/" + host, data);
+  }
+
+  private void removeDrillbit(CuratorFramework zk, String host)
+      throws Exception {
+    zk.delete().forPath("/" + host);
+  }
+
+  /**
+   * Connect to the test ZK for the simulated Drillbit side of the test. (The AM
+   * side of the test uses the actual AM code, which is what we're testing
+   * here...)
+   *
+   * @param connectString
+   * @return
+   */
+
+  public static CuratorFramework connectToZk(String connectString) {
+    CuratorFramework client = CuratorFrameworkFactory.builder()
+        .namespace(ZK_ROOT + "/" + CLUSTER_ID).connectString(connectString)
+        .retryPolicy(new RetryNTimes(3, 1000)).build();
+    client.start();
+    return client;
+  }
+
+  private static class TestRegistryHandler implements RegistryHandler {
+    String reserved;
+    String released;
+    Task start;
+    Task end;
+
+    public void clear() {
+      reserved = null;
+      released = null;
+      start = null;
+      end = null;
+    }
+
+    @Override
+    public void reserveHost(String hostName) {
+      assertNull(reserved);
+      reserved = hostName;
+    }
+
+    @Override
+    public void releaseHost(String hostName) {
+      assertNull(released);
+      released = hostName;
+    }
+
+    @Override
+    public void startAck(Task task, String propertyKey, Object value) {
+      start = task;
+    }
+
+    @Override
+    public void completionAck(Task task, String endpointProperty) {
+      end = task;
+    }
+
+    @Override
+    public void registryDown() {
+      // TODO Auto-generated method stub
+
+    }
+  }
+
+  public static class TestTask extends Task {
+    private String host;
+
+    public TestTask(String host) {
+      super(null, null);
+      this.host = host;
+    }
+
+    @Override
+    public String getHostName() {
+      return host;
+    }
+
+    @Override
+    public void resetTrackingState() {
+      trackingState = TrackingState.NEW;
+    }
+  }
+
+  @Test
+  public void testZKRegistry() throws Exception {
+    TestingServer server = new TestingServer();
+    server.start();
+    String connStr = server.getConnectString();
+
+    CuratorFramework probeZk = connectToZk(connStr);
+    addDrillbit(probeZk, FRED_HOST);
+
+    ZKClusterCoordinatorDriver driver = new ZKClusterCoordinatorDriver()
+        .setConnect(connStr, ZK_ROOT, CLUSTER_ID)
+        .setPorts(TEST_USER_PORT, TEST_CONTROL_PORT, TEST_DATA_PORT).build();
+    ZKRegistry registry = new ZKRegistry(driver);
+    TestRegistryHandler handler = new TestRegistryHandler();
+    registry.start(handler);
+
+    // We started with one "stray" drillbit that will be reported as unmanaged.
+
+    assertEquals(FRED_HOST, handler.reserved);
+    List<String> unmanaged = registry.listUnmanagedDrillits();
+    assertEquals(1, unmanaged.size());
+    String fredsKey = makeKey(FRED_HOST);
+    assertEquals(fredsKey, unmanaged.get(0));
+    Map<String, DrillbitTracker> trackers = registry.getRegistryForTesting();
+    assertEquals(1, trackers.size());
+    assertTrue(trackers.containsKey(fredsKey));
+    DrillbitTracker fredsTracker = trackers.get(fredsKey);
+    assertEquals(fredsKey, fredsTracker.key);
+    assertEquals(DrillbitTracker.State.UNMANAGED, fredsTracker.state);
+    assertNull(fredsTracker.task);
+    assertEquals(fredsKey,
+        ZKClusterCoordinatorDriver.asString(fredsTracker.endpoint));
+
+    // The handler should have been told about the initial stray.
+
+    assertEquals(FRED_HOST, handler.reserved);
+
+    // Pretend to start a new Drillbit.
+
+    Task wilmasTask = new TestTask(WILMA_HOST);
+    EventContext context = new EventContext(wilmasTask);
+
+    // Registry ignores the created event.
+
+    registry.stateChange(Event.CREATED, context);
+    assertEquals(1, registry.getRegistryForTesting().size());
+
+    // But, does care about the allocated event.
+
+    registry.stateChange(Event.ALLOCATED, context);
+    assertEquals(2, registry.getRegistryForTesting().size());
+    String wilmasKey = makeKey(WILMA_HOST);
+    DrillbitTracker wilmasTracker = registry.getRegistryForTesting()
+        .get(wilmasKey);
+    assertNotNull(wilmasTracker);
+    assertEquals(wilmasTask, wilmasTracker.task);
+    assertNull(wilmasTracker.endpoint);
+    assertEquals(wilmasKey, wilmasTracker.key);
+    assertEquals(DrillbitTracker.State.NEW, wilmasTracker.state);
+    handler.clear();
+
+    // Time goes on. The Drillbit starts and registers itself.
+
+    addDrillbit(probeZk, WILMA_HOST);
+    Thread.sleep(100);
+    assertEquals(wilmasTask, handler.start);
+    assertEquals(DrillbitTracker.State.REGISTERED, wilmasTracker.state);
+    assertEquals(handler.start, wilmasTask);
+
+    // Create another task: Barney
+
+    Task barneysTask = new TestTask(BARNEY_HOST);
+    context = new EventContext(barneysTask);
+    registry.stateChange(Event.CREATED, context);
+
+    // Start Barney, but assume a latency in Yarn, but not ZK.
+    // We get the ZK registration before the YARN launch confirmation.
+
+    handler.clear();
+    addDrillbit(probeZk, BARNEY_HOST);
+    Thread.sleep(100);
+    assertEquals(BARNEY_HOST, handler.reserved);
+    String barneysKey = makeKey(BARNEY_HOST);
+    DrillbitTracker barneysTracker = registry.getRegistryForTesting()
+        .get(barneysKey);
+    assertNotNull(barneysTracker);
+    assertEquals(DrillbitTracker.State.UNMANAGED, barneysTracker.state);
+    assertNull(barneysTracker.task);
+    assertEquals(2, registry.listUnmanagedDrillits().size());
+
+    handler.clear();
+    registry.stateChange(Event.ALLOCATED, context);
+    assertEquals(DrillbitTracker.State.REGISTERED, barneysTracker.state);
+    assertEquals(handler.start, barneysTask);
+    assertEquals(barneysTask, barneysTracker.task);
+    assertEquals(1, registry.listUnmanagedDrillits().size());
+
+    // Barney is having problems, it it drops out of ZK.
+
+    handler.clear();
+    removeDrillbit(probeZk, BARNEY_HOST);
+    Thread.sleep(100);
+    assertEquals(barneysTask, handler.end);
+    assertEquals(DrillbitTracker.State.DEREGISTERED, barneysTracker.state);
+
+    // Barney comes alive (presumably before the controller gives up and kills
+    // the Drillbit.)
+
+    handler.clear();
+    addDrillbit(probeZk, BARNEY_HOST);
+    Thread.sleep(100);
+    assertEquals(barneysTask, handler.start);
+    assertEquals(DrillbitTracker.State.REGISTERED, barneysTracker.state);
+
+    // Barney is killed by the controller.
+    // ZK entry drops. Tracker is removed, controller is notified.
+
+    handler.clear();
+    removeDrillbit(probeZk, BARNEY_HOST);
+    Thread.sleep(100);
+    assertNotNull(registry.getRegistryForTesting().get(barneysKey));
+    assertEquals(barneysTask, handler.end);
+
+    // The controller tells the registry to stop tracking the Drillbit.
+
+    handler.clear();
+    registry.stateChange(Event.ENDED, context);
+    assertNull(handler.end);
+    assertNull(registry.getRegistryForTesting().get(barneysKey));
+
+    // The stray drillbit deregisters from ZK. The tracker is removed.
+
+    handler.clear();
+    removeDrillbit(probeZk, FRED_HOST);
+    Thread.sleep(100);
+    assertNull(registry.getRegistryForTesting().get(fredsKey));
+    assertNull(handler.end);
+    assertEquals(FRED_HOST, handler.released);
+
+    // Wilma is killed by the controller.
+
+    handler.clear();
+    removeDrillbit(probeZk, WILMA_HOST);
+    Thread.sleep(100);
+    assertEquals(wilmasTask, handler.end);
+    assertNull(handler.released);
+    assertEquals(DrillbitTracker.State.DEREGISTERED, wilmasTracker.state);
+    assertNotNull(registry.getRegistryForTesting().get(wilmasKey));
+
+    handler.clear();
+    context = new EventContext(wilmasTask);
+    registry.stateChange(Event.ENDED, context);
+    assertNull(registry.getRegistryForTesting().get(wilmasKey));
+    assertNull(handler.released);
+    assertNull(handler.end);
+
+    // All drillbits should be gone.
+
+    assertTrue(registry.getRegistryForTesting().isEmpty());
+
+    probeZk.close();
+    driver.close();
+    server.stop();
+    server.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/resources/doy-test-logback.xml
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/resources/doy-test-logback.xml b/drill-yarn/src/test/resources/doy-test-logback.xml
new file mode 100644
index 0000000..b00ff5f
--- /dev/null
+++ b/drill-yarn/src/test/resources/doy-test-logback.xml
@@ -0,0 +1,56 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!-- 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. -->
+<configuration>
+
+<!-- 
+  <appender name="SOCKET" class="de.huxhorn.lilith.logback.appender.ClassicMultiplexSocketAppender">
+    <Compressing>true</Compressing>
+    <ReconnectionDelay>10000</ReconnectionDelay>
+    <IncludeCallerData>true</IncludeCallerData>
+    <RemoteHosts>${LILITH_HOSTNAME:-localhost}</RemoteHosts>
+  </appender> -->
+
+ <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+    <!-- encoders are assigned the type
+         ch.qos.logback.classic.encoder.PatternLayoutEncoder by default -->
+    <encoder>
+      <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n</pattern>
+    </encoder>
+  </appender>
+
+<!--
+  <appender name="FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+    <file>/logs/test-common.log</file>
+    <encoder>
+      <pattern>%date %level [%thread] %logger{10} [%file:%line] %msg%n</pattern>
+    </encoder>
+    <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+      <fileNamePattern>/logs/test-common.%d{yyyy-MM-dd}.log</fileNamePattern>
+      <maxHistory>30</maxHistory>
+    </rollingPolicy>
+  </appender>
+  -->
+  <logger name="org.apache.drill" additivity="false">
+    <level value="error" />
+    <!--   <appender-ref ref="SOCKET" /> -->
+    <appender-ref ref="STDOUT" />
+<!--     <appender-ref ref="FILE" /> -->
+  </logger>
+
+  <root>
+    <level value="error" />
+    <!-- <appender-ref ref="SOCKET" /> -->
+    <appender-ref ref="STDOUT" />
+<!--     <appender-ref ref="FILE" /> -->
+  </root>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/resources/second-test-config.conf
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/resources/second-test-config.conf b/drill-yarn/src/test/resources/second-test-config.conf
new file mode 100644
index 0000000..11c180f
--- /dev/null
+++ b/drill-yarn/src/test/resources/second-test-config.conf
@@ -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.
+
+# This file contains a test set of configuration settings used by the
+# TestConfig unit test. It has a benign name to prevent any accidental
+# inclusion into production code. The test class makes a copy of this
+# file with the desired name for the life of the test run only.
+
+drill.yarn: {
+  app-name: "My-App"
+  
+  drill-install: {
+    localize: false
+    drill-home: "/config/drill/home"
+    site-dir: "/config/drill/site"
+    client-path: "/foo/bar/drill-archive.tar.gz"
+  }
+  
+  yarn: {
+    queue: "my-queue"
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/resources/test-doy-config.conf
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/resources/test-doy-config.conf b/drill-yarn/src/test/resources/test-doy-config.conf
new file mode 100644
index 0000000..b588530
--- /dev/null
+++ b/drill-yarn/src/test/resources/test-doy-config.conf
@@ -0,0 +1,32 @@
+# 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.
+
+# This file contains a test set of configuration settings used by the
+# TestConfig unit test. It has a benign name to prevent any accidental
+# inclusion into production code. The test class makes a copy of this
+# file with the desired name for the life of the test run only.
+
+drill.yarn: {
+  app-name: "My-App"
+  
+  drill-install: {
+    drill-home: "/config/drill/home"
+    client-path: "/foo/bar/drill-archive.tar.gz"
+  }
+  
+  yarn: {
+    queue: "my-queue"
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/resources/test-doy-distrib.conf
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/resources/test-doy-distrib.conf b/drill-yarn/src/test/resources/test-doy-distrib.conf
new file mode 100644
index 0000000..ff73ef3
--- /dev/null
+++ b/drill-yarn/src/test/resources/test-doy-distrib.conf
@@ -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.
+
+# This file contains a test set of configuration settings used by the
+# TestConfig unit test. It has a benign name to prevent any accidental
+# inclusion into production code. The test class makes a copy of this
+# file with the desired name for the life of the test run only.
+
+drill.yarn: {
+  app-name: "config-app-name"
+  
+  dfs: {
+    app-dir: "/opt/drill"
+  }
+  yarn: {
+    queue: "distrib-queue"
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/resources/third-test-config.conf
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/resources/third-test-config.conf b/drill-yarn/src/test/resources/third-test-config.conf
new file mode 100644
index 0000000..f585e22
--- /dev/null
+++ b/drill-yarn/src/test/resources/third-test-config.conf
@@ -0,0 +1,32 @@
+# 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.
+
+# This file contains a test set of configuration settings used by the
+# TestConfig unit test. It has a benign name to prevent any accidental
+# inclusion into production code. The test class makes a copy of this
+# file with the desired name for the life of the test run only.
+
+drill.yarn: {
+  app-name: "My-App"
+  
+  drill-install: {
+    localize: false
+    drill-home: "/config/drill/home"
+  }
+  
+  yarn: {
+    queue: "my-queue"
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/resources/wrapper.sh
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/resources/wrapper.sh b/drill-yarn/src/test/resources/wrapper.sh
new file mode 100644
index 0000000..f436eb5
--- /dev/null
+++ b/drill-yarn/src/test/resources/wrapper.sh
@@ -0,0 +1,53 @@
+#!/bin/bash
+#
+# 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.
+
+echo "Drill Stdout Message"
+echo "Stderr Message" 1>&2
+
+output="$DRILL_HOME/../output.txt"
+while [[ $# > 0 ]]
+do
+  if [[ $1 =~ ^-Dlog.path= ]]; then
+    thelog=${1/-Dlog.path=//}
+    echo "Drill Log Message" >> $thelog
+  fi
+  echo $1 >> $output
+  shift
+done
+
+function clean_up {
+  echo "Received SIGTERM"
+  if [ "$PRETEND_HUNG" == "1" ]; then
+    echo "Pretending to be hung."
+  else
+    echo "Exiting"
+    if [ "$PRETEND_FAIL" == "1" ]; then
+      exit 55
+    else
+      exit 0
+    fi
+  fi
+}
+
+trap clean_up SIGTERM
+
+if [ "$KEEP_RUNNING" == "1" ]; then
+  while [[ 1 > 0 ]]
+  do
+    sleep 1
+  done
+fi

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2c1f5fc..a163270 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1555,6 +1555,62 @@
             </exclusions>
           </dependency>
           <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-client</artifactId>
+            <version>${hadoop.version}</version>
+            <scope>compile</scope>
+            <exclusions>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-core</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-server</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-json</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-client</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-core-asl</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-mapper-asl</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-xc</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-jaxrs</artifactId>
+              </exclusion>
+
+              <!-- Exclude the following to prevent enforcer failures. YARN uses them,
+                   but Drill forbids them. -->
+
+              <exclusion>
+                <groupId>log4j</groupId>
+                <artifactId>log4j</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>javax.servlet</groupId>
+                <artifactId>servlet-api</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>commons-logging</groupId>
+                <artifactId>commons-logging</artifactId>
+              </exclusion>
+            </exclusions>
+          </dependency>
+          <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-client</artifactId>
             <version>${hbase.version}</version>
@@ -2143,6 +2199,62 @@
             </exclusions>
           </dependency>
           <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-client</artifactId>
+            <version>${hadoop.version}</version>
+            <scope>compile</scope>
+            <exclusions>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-core</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-server</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-json</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-client</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-core-asl</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-mapper-asl</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-xc</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-jaxrs</artifactId>
+              </exclusion>
+
+              <!-- Exclude the following to prevent enforcer failures. YARN uses them,
+                   but Drill forbids them. -->
+
+              <exclusion>
+                <groupId>log4j</groupId>
+                <artifactId>log4j</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>javax.servlet</groupId>
+                <artifactId>servlet-api</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>commons-logging</groupId>
+                <artifactId>commons-logging</artifactId>
+              </exclusion>
+            </exclusions>
+          </dependency>
+          <dependency>
             <groupId>org.antlr</groupId>
             <artifactId>antlr4-runtime</artifactId>
             <version>4.5</version>
@@ -2506,6 +2618,62 @@
               </exclusion>
             </exclusions>
           </dependency>
+          <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-client</artifactId>
+            <version>2.3.0-cdh5.0.3</version>
+            <scope>compile</scope>
+            <exclusions>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-core</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-server</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-json</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-client</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-core-asl</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-mapper-asl</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-xc</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-jaxrs</artifactId>
+              </exclusion>
+
+              <!-- Exclude the following to prevent enforcer failures. YARN uses them,
+                   but Drill forbids them. -->
+
+              <exclusion>
+                <groupId>log4j</groupId>
+                <artifactId>log4j</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>javax.servlet</groupId>
+                <artifactId>servlet-api</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>commons-logging</groupId>
+                <artifactId>commons-logging</artifactId>
+              </exclusion>
+            </exclusions>
+          </dependency>
         </dependencies>
       </dependencyManagement>
       <repositories>
@@ -2580,6 +2748,62 @@
               </exclusion>
             </exclusions>
           </dependency>
+          <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-yarn-client</artifactId>
+            <version>2.4.0.2.1.3.0-563</version>
+            <scope>compile</scope>
+            <exclusions>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-core</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-server</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-json</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>com.sun.jersey</groupId>
+                <artifactId>jersey-client</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-core-asl</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-mapper-asl</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-xc</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>org.codehaus.jackson</groupId>
+                <artifactId>jackson-jaxrs</artifactId>
+              </exclusion>
+
+              <!-- Exclude the following to prevent enforcer failures. YARN uses them,
+                   but Drill forbids them. -->
+
+              <exclusion>
+                <groupId>log4j</groupId>
+                <artifactId>log4j</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>javax.servlet</groupId>
+                <artifactId>servlet-api</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>commons-logging</groupId>
+                <artifactId>commons-logging</artifactId>
+              </exclusion>
+            </exclusions>
+          </dependency>
         </dependencies>
       </dependencyManagement>
       <repositories>
@@ -2723,6 +2947,7 @@
     <module>common</module>
     <module>logical</module>
     <module>exec</module>
+    <module>drill-yarn</module>
     <module>distribution</module>
   </modules>
 </project>


[12/12] drill git commit: DRILL-6208: Fix FunctionInitializerTest#testConcurrentFunctionBodyLoad to use Mockito instead of JMockit

Posted by ar...@apache.org.
DRILL-6208: Fix FunctionInitializerTest#testConcurrentFunctionBodyLoad to use Mockito instead of JMockit

closes #1149


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/36159e22
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/36159e22
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/36159e22

Branch: refs/heads/master
Commit: 36159e2282153dbce52b8345d19e9bd95d67d377
Parents: 863ff0b
Author: Arina Ielchiieva <ar...@gmail.com>
Authored: Sat Mar 3 21:00:50 2018 +0200
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sun Mar 4 17:46:55 2018 +0200

----------------------------------------------------------------------
 .../drill/exec/expr/fn/FunctionInitializer.java |  5 +--
 .../exec/expr/fn/FunctionInitializerTest.java   | 33 ++++++++++----------
 2 files changed, 20 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/36159e22/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionInitializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionInitializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionInitializer.java
index 20b1d12..e9d98f2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionInitializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionInitializer.java
@@ -17,13 +17,13 @@
  */
 package org.apache.drill.exec.expr.fn;
 
-import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.StringReader;
 import java.util.List;
 import java.util.Map;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.util.DrillFileUtils;
 import org.codehaus.commons.compiler.CompileException;
@@ -126,7 +126,8 @@ public class FunctionInitializer {
    * @return compilation unit
    * @throws IOException if did not find class or could not load it
    */
-  private CompilationUnit convertToCompilationUnit(Class<?> clazz) throws IOException {
+  @VisibleForTesting
+  CompilationUnit convertToCompilationUnit(Class<?> clazz) throws IOException {
     String path = clazz.getName();
     path = path.replaceFirst("\\$.*", "");
     path = path.replace(".", DrillFileUtils.SEPARATOR);

http://git-wip-us.apache.org/repos/asf/drill/blob/36159e22/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/FunctionInitializerTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/FunctionInitializerTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/FunctionInitializerTest.java
index c2d4bdb..edbd341 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/FunctionInitializerTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/FunctionInitializerTest.java
@@ -17,18 +17,17 @@
 package org.apache.drill.exec.expr.fn;
 
 import com.google.common.collect.Lists;
-import mockit.Invocation;
-import mockit.Mock;
-import mockit.MockUp;
-import mockit.integration.junit4.JMockit;
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.test.TestTools;
 import org.apache.drill.exec.util.JarUtil;
-import org.codehaus.janino.Java;
+import org.codehaus.janino.Java.CompilationUnit;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.runners.MockitoJUnitRunner;
+import org.mockito.stubbing.Answer;
 
 import java.net.URL;
 import java.net.URLClassLoader;
@@ -45,9 +44,11 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
 
-
-@RunWith(JMockit.class)
+@RunWith(MockitoJUnitRunner.class)
 @Category(SqlFunctionTest.class)
 public class FunctionInitializerTest {
 
@@ -92,16 +93,16 @@ public class FunctionInitializerTest {
 
   @Test
   public void testConcurrentFunctionBodyLoad() throws Exception {
-    final FunctionInitializer functionInitializer = new FunctionInitializer(CLASS_NAME, classLoader);
-
+    final FunctionInitializer spyFunctionInitializer = spy(new FunctionInitializer(CLASS_NAME, classLoader));
     final AtomicInteger counter = new AtomicInteger();
-    new MockUp<FunctionInitializer>() {
-      @Mock
-      Java.CompilationUnit convertToCompilationUnit(Invocation inv, Class<?> clazz) {
+
+    doAnswer(new Answer<CompilationUnit>() {
+      @Override
+      public CompilationUnit answer(InvocationOnMock invocation) throws Throwable {
         counter.incrementAndGet();
-        return inv.proceed();
+        return (CompilationUnit) invocation.callRealMethod();
       }
-    };
+    }).when(spyFunctionInitializer).convertToCompilationUnit(any(Class.class));
 
     int threadsNumber = 5;
     ExecutorService executor = Executors.newFixedThreadPool(threadsNumber);
@@ -109,8 +110,8 @@ public class FunctionInitializerTest {
     try {
       List<Future<String>> results = executor.invokeAll(Collections.nCopies(threadsNumber, new Callable<String>() {
         @Override
-        public String call() throws Exception {
-          return functionInitializer.getMethod("eval");
+        public String call() {
+          return spyFunctionInitializer.getMethod("eval");
         }
       }));
 


[08/12] drill git commit: DRILL-1170: YARN integration for Drill

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractScheduler.java
----------------------------------------------------------------------
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 0000000..01b89ae
--- /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; }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractTaskManager.java
----------------------------------------------------------------------
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 0000000..7acd402
--- /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; }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/BatchScheduler.java
----------------------------------------------------------------------
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 0000000..8f3aaab
--- /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.
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterController.java
----------------------------------------------------------------------
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 0000000..6aaa18b
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ClusterControllerImpl.java
----------------------------------------------------------------------
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 0000000..3c011ec
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ControllerFactory.java
----------------------------------------------------------------------
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 0000000..b8d6e06
--- /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;
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/ControllerVisitor.java
----------------------------------------------------------------------
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 0000000..5774d7d
--- /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);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Dispatcher.java
----------------------------------------------------------------------
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 0000000..f5257e6
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DispatcherAddOn.java
----------------------------------------------------------------------
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 0000000..5c7100b
--- /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);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillApplicationMaster.java
----------------------------------------------------------------------
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 0000000..c0db9a1
--- /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
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillControllerFactory.java
----------------------------------------------------------------------
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 0000000..013fdba
--- /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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/DrillbitScheduler.java
----------------------------------------------------------------------
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 0000000..76936b5
--- /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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/EventContext.java
----------------------------------------------------------------------
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 0000000..bec8cf9
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/NodeInventory.java
----------------------------------------------------------------------
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 0000000..ec20307
--- /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);
+  }
+}


[11/12] drill git commit: DRILL-6189: Security: passwords logging and file permisions

Posted by ar...@apache.org.
DRILL-6189: Security: passwords logging and file permisions

1. Overrided serialization methods for instances with passwords
2. Changed file permissions for configuration files

closes #1139


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/863ff0bc
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/863ff0bc
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/863ff0bc

Branch: refs/heads/master
Commit: 863ff0bcff1ee01df292277194e0fee25dbe3460
Parents: f2ac874
Author: Vladimir Tkach <vo...@gmail.com>
Authored: Wed Feb 28 19:13:51 2018 +0200
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sun Mar 4 17:45:40 2018 +0200

----------------------------------------------------------------------
 .../exec/store/jdbc/JdbcStorageConfig.java      |  2 +
 distribution/src/assemble/bin.xml               | 12 ++++--
 distribution/src/resources/distrib-env.sh       |  0
 distribution/src/resources/drill-env.sh         |  0
 .../planner/sql/handlers/DefaultSqlHandler.java |  8 +++-
 .../apache/drill/exec/rpc/user/UserServer.java  | 40 +++++++++++++++++++-
 .../common/config/LogicalPlanPersistence.java   |  2 +
 7 files changed, 57 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/863ff0bc/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStorageConfig.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStorageConfig.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStorageConfig.java
index 5a921d4..15eb675 100755
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStorageConfig.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStorageConfig.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.store.jdbc;
 
+import com.fasterxml.jackson.annotation.JsonFilter;
 import org.apache.drill.common.logical.StoragePluginConfig;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
@@ -24,6 +25,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 
 @JsonTypeName(JdbcStorageConfig.NAME)
+@JsonFilter("passwordFilter")
 public class JdbcStorageConfig extends StoragePluginConfig {
 
   public static final String NAME = "jdbc";

http://git-wip-us.apache.org/repos/asf/drill/blob/863ff0bc/distribution/src/assemble/bin.xml
----------------------------------------------------------------------
diff --git a/distribution/src/assemble/bin.xml b/distribution/src/assemble/bin.xml
index 7ca1140..82c4d90 100644
--- a/distribution/src/assemble/bin.xml
+++ b/distribution/src/assemble/bin.xml
@@ -356,10 +356,12 @@
     <file>
       <source>src/resources/drill-override.conf</source>
       <outputDirectory>conf</outputDirectory>
+      <fileMode>0640</fileMode>
     </file>
     <file>
       <source>src/resources/logback.xml</source>
       <outputDirectory>conf</outputDirectory>
+      <fileMode>0640</fileMode>
     </file>
     <file>
       <source>src/resources/yarn-client-log.xml</source>
@@ -373,12 +375,12 @@
     </file>
     <file>
       <source>src/resources/drill-env.sh</source>
-      <fileMode>0755</fileMode>
+      <fileMode>0750</fileMode>
       <outputDirectory>conf</outputDirectory>
     </file>
     <file>
       <source>src/resources/distrib-env.sh</source>
-      <fileMode>0755</fileMode>
+      <fileMode>0750</fileMode>
       <outputDirectory>conf</outputDirectory>
     </file>
     <file>
@@ -388,21 +390,23 @@
     </file>
     <file>
       <source>src/resources/drill-setup.sh</source>
-      <fileMode>0755</fileMode>
+      <fileMode>0750</fileMode>
       <outputDirectory>conf</outputDirectory>
     </file>
     <file>
       <source>src/resources/distrib-setup.sh</source>
-      <fileMode>0755</fileMode>
+      <fileMode>0750</fileMode>
       <outputDirectory>conf</outputDirectory>
     </file>
     <file>
       <source>src/resources/drill-override-example.conf</source>
       <outputDirectory>conf</outputDirectory>
+      <fileMode>0640</fileMode>
     </file>
     <file>
       <source>src/resources/core-site-example.xml</source>
       <outputDirectory>conf</outputDirectory>
+      <fileMode>0640</fileMode>
     </file>
     <file>
       <source>src/resources/saffron.properties</source>

http://git-wip-us.apache.org/repos/asf/drill/blob/863ff0bc/distribution/src/resources/distrib-env.sh
----------------------------------------------------------------------
diff --git a/distribution/src/resources/distrib-env.sh b/distribution/src/resources/distrib-env.sh
old mode 100644
new mode 100755

http://git-wip-us.apache.org/repos/asf/drill/blob/863ff0bc/distribution/src/resources/drill-env.sh
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-env.sh b/distribution/src/resources/drill-env.sh
old mode 100644
new mode 100755

http://git-wip-us.apache.org/repos/asf/drill/blob/863ff0bc/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index 9b75fb7..58fac66 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -23,7 +23,11 @@ import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import com.fasterxml.jackson.databind.ser.PropertyFilter;
+import com.fasterxml.jackson.databind.ser.impl.SimpleBeanPropertyFilter;
+import com.fasterxml.jackson.databind.ser.impl.SimpleFilterProvider;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Sets;
 import org.apache.calcite.plan.RelOptCostImpl;
 import org.apache.calcite.plan.RelOptLattice;
 import org.apache.calcite.plan.RelOptMaterialization;
@@ -158,7 +162,9 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
 
   protected void log(final String name, final PhysicalPlan plan, final Logger logger) throws JsonProcessingException {
     if (logger.isDebugEnabled()) {
-      String planText = plan.unparse(context.getLpPersistence().getMapper().writer());
+      PropertyFilter filter = new SimpleBeanPropertyFilter.SerializeExceptFilter(Sets.newHashSet("password"));
+      String planText = plan.unparse(context.getLpPersistence().getMapper()
+              .writer(new SimpleFilterProvider().addFilter("passwordFilter", filter)));
       logger.debug(name + " : \n" + planText);
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/863ff0bc/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
index 58d9df0..df73b9e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.rpc.user;
 
 import java.io.IOException;
 import java.net.SocketAddress;
+import java.util.List;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.UUID;
@@ -92,6 +93,40 @@ public class UserServer extends BasicServer<RpcType, BitToUserConnection> {
     userConnectionMap = new ConcurrentHashMap<>();
   }
 
+  /**
+   * Serialize {@link org.apache.drill.exec.proto.UserProtos.BitToUserHandshake} instance without password
+   * @param inbound handshake instance for serialization
+   * @return String of serialized object
+   */
+  private String serializeUserToBitHandshakeWithoutPassword(UserToBitHandshake inbound) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("rpc_version: ");
+    sb.append(inbound.getRpcVersion());
+    sb.append("\ncredentials:\n\t");
+    sb.append(inbound.getCredentials());
+    sb.append("properties:");
+    List<Property> props = inbound.getProperties().getPropertiesList();
+    for (Property p: props) {
+      if (!p.getKey().equalsIgnoreCase("password")) {
+        sb.append("\n\tproperty:\n\t\t");
+        sb.append("key: \"");
+        sb.append(p.getKey());
+        sb.append("\"\n\t\tvalue: \"");
+        sb.append(p.getValue());
+        sb.append("\"");
+      }
+    }
+    sb.append("\nsupport_complex_types: ");
+    sb.append(inbound.getSupportComplexTypes());
+    sb.append("\nsupport_timeout: ");
+    sb.append(inbound.getSupportTimeout());
+    sb.append("sasl_support: ");
+    sb.append(inbound.getSaslSupport());
+    sb.append("\nclient_infos:\n\t");
+    sb.append(inbound.getClientInfos().toString().replace("\n", "\n\t"));
+    return sb.toString();
+  }
+
   public UserServer(BootStrapContext context, BufferAllocator allocator, EventLoopGroup eventLoopGroup,
                     UserWorker worker) throws DrillbitStartupException {
     super(UserRpcConfig.getMapping(context.getConfig(), context.getExecutor()),
@@ -320,8 +355,9 @@ public class UserServer extends BasicServer<RpcType, BitToUserConnection> {
 
       @Override
       public BitToUserHandshake getHandshakeResponse(UserToBitHandshake inbound) throws Exception {
-        logger.trace("Handling handshake from user to bit. {}", inbound);
-
+        if (logger.isTraceEnabled()) {
+          logger.trace("Handling handshake from user to bit. {}", serializeUserToBitHandshakeWithoutPassword(inbound));
+        }
         // if timeout is unsupported or is set to false, disable timeout.
         if (!inbound.hasSupportTimeout() || !inbound.getSupportTimeout()) {
           connection.disableReadTimeout();

http://git-wip-us.apache.org/repos/asf/drill/blob/863ff0bc/logical/src/main/java/org/apache/drill/common/config/LogicalPlanPersistence.java
----------------------------------------------------------------------
diff --git a/logical/src/main/java/org/apache/drill/common/config/LogicalPlanPersistence.java b/logical/src/main/java/org/apache/drill/common/config/LogicalPlanPersistence.java
index cd7a8d0..ccc4c5a 100644
--- a/logical/src/main/java/org/apache/drill/common/config/LogicalPlanPersistence.java
+++ b/logical/src/main/java/org/apache/drill/common/config/LogicalPlanPersistence.java
@@ -19,6 +19,7 @@ package org.apache.drill.common.config;
 
 import java.util.Set;
 
+import com.fasterxml.jackson.databind.ser.impl.SimpleFilterProvider;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.FormatPluginConfigBase;
@@ -52,6 +53,7 @@ public class LogicalPlanPersistence {
     mapper.configure(Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
     mapper.configure(JsonGenerator.Feature.QUOTE_FIELD_NAMES, true);
     mapper.configure(Feature.ALLOW_COMMENTS, true);
+    mapper.setFilterProvider(new SimpleFilterProvider().setFailOnUnknownId(false));
     registerSubtypes(LogicalOperatorBase.getSubTypes(scanResult));
     registerSubtypes(StoragePluginConfigBase.getSubTypes(scanResult));
     registerSubtypes(FormatPluginConfigBase.getSubTypes(scanResult));


[05/12] drill git commit: DRILL-1170: YARN integration for Drill

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/AMRunner.java
----------------------------------------------------------------------
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 0000000..5252b88
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/CleanCommand.java
----------------------------------------------------------------------
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 0000000..1fcba2d
--- /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);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientCommand.java
----------------------------------------------------------------------
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 0000000..469d04c
--- /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();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientContext.java
----------------------------------------------------------------------
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 0000000..377b3b3
--- /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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/ClientException.java
----------------------------------------------------------------------
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 0000000..24c062b
--- /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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/CommandLineOptions.java
----------------------------------------------------------------------
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 0000000..174265d
--- /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());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/DrillOnYarn.java
----------------------------------------------------------------------
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 0000000..587766a
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/FileUploader.java
----------------------------------------------------------------------
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 0000000..ace2d03
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/HelpCommand.java
----------------------------------------------------------------------
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 0000000..3e7e5d9
--- /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();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/KillCommand.java
----------------------------------------------------------------------
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 0000000..8b7914c
--- /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());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/PrintConfigCommand.java
----------------------------------------------------------------------
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 0000000..69cdf55
--- /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("----------------------------------------------");
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/ResizeCommand.java
----------------------------------------------------------------------
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 0000000..43ae02c
--- /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;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/SimpleRestClient.java
----------------------------------------------------------------------
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 0000000..e47fb58
--- /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);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/StartCommand.java
----------------------------------------------------------------------
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 0000000..fe505a2
--- /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();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/StatusCommand.java
----------------------------------------------------------------------
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 0000000..863b700
--- /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());
+  }
+}


[07/12] drill git commit: DRILL-1170: YARN integration for Drill

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PersistentTaskScheduler.java
----------------------------------------------------------------------
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 0000000..73a045f
--- /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);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Pollable.java
----------------------------------------------------------------------
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 0000000..7e1c9a3
--- /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);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/PulseRunnable.java
----------------------------------------------------------------------
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 0000000..81d5a5d
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/RegistryHandler.java
----------------------------------------------------------------------
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 0000000..ff29bdf
--- /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();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Scheduler.java
----------------------------------------------------------------------
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 0000000..7f8be0c
--- /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();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerState.java
----------------------------------------------------------------------
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 0000000..7a1f8bd
--- /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();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateActions.java
----------------------------------------------------------------------
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 0000000..65e8f2a
--- /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);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/SchedulerStateImpl.java
----------------------------------------------------------------------
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 0000000..4c85cf3
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/Task.java
----------------------------------------------------------------------
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 0000000..147f5f7
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskLifecycleListener.java
----------------------------------------------------------------------
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 0000000..218cd9b
--- /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);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskSpec.java
----------------------------------------------------------------------
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 0000000..4399a86
--- /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;
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskState.java
----------------------------------------------------------------------
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 0000000..3d52105
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/TaskVisitor.java
----------------------------------------------------------------------
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 0000000..c90d4f8
--- /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);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/YarnFacadeException.java
----------------------------------------------------------------------
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 0000000..8ac0a5d
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AMSecurityManager.java
----------------------------------------------------------------------
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 0000000..fbca171
--- /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();
+}


[04/12] drill git commit: DRILL-1170: YARN integration for Drill

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/StopCommand.java
----------------------------------------------------------------------
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 0000000..95f7bf3
--- /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);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/client/package-info.java
----------------------------------------------------------------------
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 0000000..c03c2fa
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/core/AppSpec.java
----------------------------------------------------------------------
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 0000000..34d4ad1
--- /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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/core/ClusterDef.java
----------------------------------------------------------------------
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 0000000..223b606
--- /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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/core/ContainerRequestSpec.java
----------------------------------------------------------------------
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 0000000..99a22d7
--- /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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/core/DfsFacade.java
----------------------------------------------------------------------
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 0000000..09e88ae
--- /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);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/core/DoYUtil.java
----------------------------------------------------------------------
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 0000000..3c1d17d
--- /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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/core/DoyConfigException.java
----------------------------------------------------------------------
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 0000000..422b89b
--- /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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/core/DrillOnYarnConfig.java
----------------------------------------------------------------------
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 0000000..38ecd1c
--- /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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/core/LaunchSpec.java
----------------------------------------------------------------------
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 0000000..6c22874
--- /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());
+      }
+    }
+  }
+}


[02/12] drill git commit: DRILL-1170: YARN integration for Drill

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/drill-am/manage.ftl
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/drill-am/manage.ftl b/drill-yarn/src/main/resources/drill-am/manage.ftl
new file mode 100644
index 0000000..682530c
--- /dev/null
+++ b/drill-yarn/src/main/resources/drill-am/manage.ftl
@@ -0,0 +1,78 @@
+<#-- 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. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+</#macro>
+
+<#macro page_body>
+  <h4>Manage Drill Cluster</h4>
+
+  Current Status: ${model.getLiveCount( )}
+  <#if model.getLiveCount( ) == 1 >Drillbit is
+  <#else>Drillbits are
+  </#if>running.
+  <p>
+  Free YARN nodes: Approximately ${model.getFreeNodeCount( )} 
+  <p><p>
+  
+  <table class="table table-hover" style="width: auto;">
+    <#-- Removed per user feedback. (Kept in REST API as client needs them.
+    <tr><td style="vertical-align: middle;">
+      <form action="/resize" method="POST" class="form-inline" role="form">
+        <div class="form-group">
+          <input hidden name="type" value="grow">
+          <label for="add">Add</label>
+          <input type="text" name="n" size="6" id="add" class="form-control"
+                  placeholder="+n" style="padding: 0 1em; margin: 0 1em;"/>
+          drillbits.
+          <button type="submit" class="btn btn-primary" style="margin: 0 1em;">Go</button>
+        </div>
+      </form>
+    </td></tr>
+    <tr><td>
+      <form action="/resize" method="POST" class="form-inline" role="form">
+        <div class="form-group">
+          <input hidden name="type" value="shrink">
+          <label for="shrink">Remove</label>
+          <input type="text" name="n" size="6" id="shrink" class="form-control"
+                  placeholder="-n" style="padding: 0 1em; margin: 0 1em;"/>
+          drillbits.
+          <button type="submit" class="btn btn-primary" style="margin: 0 1em;">Go</button>
+        </div>
+      </form>
+    </td></tr>
+    -->
+    <tr><td>
+      <form action="/resize" method="POST" class="form-inline" role="form">
+        <div class="form-group">
+          <input hidden name="type" value="resize">
+          <label for="resize">Resize to</label>
+          <input type="text" name="n" id="resize" size="6"
+                  placeholder="Size" class="form-control" style="padding: 0 1em; margin: 0 1em;"/>
+          drillbits.
+          <button type="submit" class="btn btn-primary" style="margin: 0 1em;">Go</button>
+        </div>
+      </form>
+    </td></tr>
+    <tr><td>
+      <form action="/stop" method="GET" class="form-inline" role="form">
+        <div class="form-group">
+          <label for="stop">Stop</label> the Drill cluster.
+          <button type="submit" id="stop" class="btn btn-primary" style="margin: 0 1em;">Go</button>
+        </div>
+      </form>
+    </td></tr>
+  </table>
+
+</#macro>
+
+<@page_html/>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/drill-am/redirect.ftl
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/drill-am/redirect.ftl b/drill-yarn/src/main/resources/drill-am/redirect.ftl
new file mode 100644
index 0000000..b38d10c
--- /dev/null
+++ b/drill-yarn/src/main/resources/drill-am/redirect.ftl
@@ -0,0 +1,33 @@
+<#-- 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. -->
+
+<!DOCTYPE html>
+<html lang="en">
+  <head>
+    <meta charset="utf-8">
+    <title>Apache Drill - Application Master</title>
+    <META http-equiv="refresh" content="0;URL=${amLink}">
+    <style>
+      body { font-family: sans-serif;
+             text-align: center; }
+    </style> 
+  </head>
+  <body>
+    <h3>YARN Application Master &ndash; ${clusterName}</h3>   
+    <h4>Redirect</h4>
+    The Drill Application Master UI does not work correctly inside the proxy page
+    provided by YARN.
+    <p>Click
+    <a href="${amLink}">here</a> to go to the Application Master directly
+    if this page does not automatically redirect you.
+  </body>
+</html>
+  
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/drill-am/shrink-warning.ftl
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/drill-am/shrink-warning.ftl b/drill-yarn/src/main/resources/drill-am/shrink-warning.ftl
new file mode 100644
index 0000000..dded46a
--- /dev/null
+++ b/drill-yarn/src/main/resources/drill-am/shrink-warning.ftl
@@ -0,0 +1,58 @@
+<#-- 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. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+</#macro>
+
+<#macro page_body>
+  <h4><#if model.isStop( )>
+  Confirm Cluster Shutdown
+  <#else>
+  Confirm Stopping of Drillbits
+  </#if></h4>
+
+  <div class="alert alert-danger">
+    <strong>Warning!</strong> You have requested to
+    <#if model.isStop()>
+    stop the Drill cluster.
+    <#elseif model.isCancel()>
+    cancel Drillbit ${model.getId()}.
+    <#elseif model.isKill()>
+    kill Drillbit ${model.getId()}.
+    <#else>
+    remove ${model.getCount( )}
+    <#if model.getCount() == 1>Drillbit<#else>Drillbits</#if>.
+    </#if>
+    <#if ! model.isCancel()>
+    In this version of Drill, stopping Drillbits will
+    cause in-flight queries to fail.
+    </#if>
+  </div>
+  <#if model.isStop( )>
+    <form method="POST" action="/stop">
+  <#elseif model.isCancel( )>
+    <form method="POST" action="/cancel?id=${model.getId( )}">
+  <#elseif model.isKill( )>
+    <form method="POST" action="/cancel?id=${model.getId( )}">
+  <#else>
+    <form method="POST" action="/resize">
+  </#if>
+  <#if model.isShrink( )>
+    <input type="hidden" name="n" value="${model.getCount( )}">
+    <input type="hidden" name="type" value="force-shrink">
+  </#if>
+  <input type="submit" value="Confirm"> or
+  <a href="/">Cancel</a>.
+  </form>
+</#macro>
+
+<@page_html/>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/drill-am/static/css/drill-am.css
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/drill-am/static/css/drill-am.css b/drill-yarn/src/main/resources/drill-am/static/css/drill-am.css
new file mode 100644
index 0000000..a58fad1
--- /dev/null
+++ b/drill-yarn/src/main/resources/drill-am/static/css/drill-am.css
@@ -0,0 +1,20 @@
+/* Licensed to the Apache Software Foundation (ASF) under one or more contributor
+  license agreements. See the NOTICE file distributed with this work for additional
+  information regarding copyright ownership. The ASF licenses this file to
+  You under the Apache License, Version 2.0 (the "License"); you may not use
+  this file except in compliance with the License. You may obtain a copy of
+  the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required
+  by applicable law or agreed to in writing, software distributed under the
+  License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS
+  OF ANY KIND, either express or implied. See the License for the specific
+  language governing permissions and limitations under the License. */
+  
+body {
+  padding-top: 50px;
+}
+.drill-am {
+  padding: 0 15px 20px 15px;
+}
+h3 {
+  text-align: center;
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/drill-am/static/img/apache-drill-logo.png
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/drill-am/static/img/apache-drill-logo.png b/drill-yarn/src/main/resources/drill-am/static/img/apache-drill-logo.png
new file mode 100644
index 0000000..bce39c0
Binary files /dev/null and b/drill-yarn/src/main/resources/drill-am/static/img/apache-drill-logo.png differ

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/drill-am/static/img/drill.ico
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/drill-am/static/img/drill.ico b/drill-yarn/src/main/resources/drill-am/static/img/drill.ico
new file mode 100644
index 0000000..0f9654e
Binary files /dev/null and b/drill-yarn/src/main/resources/drill-am/static/img/drill.ico differ

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/drill-am/tasks.ftl
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/drill-am/tasks.ftl b/drill-yarn/src/main/resources/drill-am/tasks.ftl
new file mode 100644
index 0000000..c6725b1
--- /dev/null
+++ b/drill-yarn/src/main/resources/drill-am/tasks.ftl
@@ -0,0 +1,113 @@
+<#-- 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. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+  <meta http-equiv="refresh" content="${refreshSecs}" >
+</#macro>
+
+<#macro page_body>
+  <h4>Drillbit Status</h4>
+  <p>&nbsp;
+
+  <#if model.hasTasks( ) >
+    <div class="table-responsive">
+      <table class="table table-hover">
+        <tr>
+          <th><span data-toggle="tooltip" title="Internal AM ID for the Drillbit.">ID</span></th>
+          <th><span data-toggle="tooltip" title="Cluster group from config file">Group</span></th>
+          <th><span data-toggle="tooltip"
+                    title="Host name or IP running the Drillbit and ink to Drillbit web UI.">
+              Host</span></th>
+          <th><span data-toggle="tooltip" title="State of the Drillbit process, hover for details.">State</span></th>
+          <th><span data-toggle="tooltip" title="ZooKeeper tracking state for the Drillbit, hover for details.">ZK State</span></th>
+          <th><span data-toggle="tooltip"
+                    title="YARN Container allocated to the Drillbit and link to the YARN Node Manager container UI.">
+              Container</span></th>
+          <th><span data-toggle="tooltip" title="Memory granted by YARN to the Drillbit.">Memory (MB)</span></th>
+          <th><span data-toggle="tooltip" title="Virtual cores granted by YARN to the Drillbit.">Virtual Cores</span></th>
+          <#if showDisks >
+            <th><span data-toggle="tooltip" title="Disk resources granted by YARN to the Drillbit.">Disks</span></th>
+          </#if>
+          <th><span data-toggle="tooltip" title="Start time in the AM server time zone.">Start Time</span></th>
+        </th>
+        <#list tasks as task>
+          <tr>
+            <td><b>${task.getTaskId( )}</b></td>
+            <td>${task.getGroupName( )}</td>
+            <td>
+            <#if task.isLive( )>
+              <a href="${task.getLink( )}" data-toggle="tooltip" title="Link to the Drillbit Web UI"></#if>
+            ${task.getHost( )}
+            <#if task.isLive( )></a></#if>
+            </td>
+            <td><span data-toggle="tooltip" title="${task.getStateHint( )}">${task.getState( )}</span>
+            <#if task.isCancelled( )><br/>(Cancelled)</#if>
+            <#if task.isCancellable( )>
+              <a href="/cancel?id=${task.getTaskId( )}" data-toggle="tooltip" title="Kill this Drillbit">[x]</a>
+            </#if>
+            </td>
+            <td><span data-toggle="tooltip" title="${task.getTrackingStateHint( )}">${task.getTrackingState( )}</span></td>
+            <td><#if task.hasContainer( )>
+              <a href="${task.getNmLink( )}" data-toggle="tooltip" title="Node Manager UI for Drillbit container">${task.getContainerId()}</a>
+            <#else>&nbsp;</#if></td>
+            <td>${task.getMemory( )}</td>
+            <td>${task.getVcores( )}</td>
+            <#if showDisks >
+              <td>${task.getDisks( )}</td>
+            </#if>
+            <td>${task.getStartTime( )}</td>
+          </tr>
+        </#list>
+      </table>
+    <#else>
+      <div class="alert alert-danger">
+        No drillbits are running.
+      </div>
+    </#if>
+    <#if model.hasUnmanagedDrillbits( ) >
+      <hr>
+      <div class="alert alert-danger">
+        <strong>Warning:</strong> ZooKeeper reports that
+        ${model.getUnmanagedDrillbitCount( )} Drillbit(s) are running that were not
+        started by the YARN Application Master. Perhaps they were started manually.
+      </div>
+      <table class="table table-hover" style="width: auto;">
+        <tr><th>Drillbit Host</th><th>Ports</th></tr>
+        <#list strays as stray >
+          <tr>
+            <td>${stray.getHost( )}</td>
+            <td>${stray.getPorts( )}</td>
+          </tr>
+        </#list>
+      </table>
+    </#if>
+    <#if model.hasBlacklist( ) >
+      <hr>
+      <div class="alert alert-danger">
+        <strong>Warning:</strong> 
+        ${model.getBlacklistCount( )} nodes have been black-listed due to
+        repeated Drillbit launch failures. Perhaps the nodes or Drill are
+        improperly configured.
+      </div>
+      <table class="table table-hover" style="width: auto;">
+        <tr><th>Blacklisted Host</th></tr>
+        <#list blacklist as node >
+          <tr>
+            <td>${node}</td>
+          </tr>
+        </#list>
+      </table>
+    </#if>
+  </div>
+</#macro>
+
+<@page_html/>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/org/apache/drill/yarn/core/drill-on-yarn-defaults.conf
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/org/apache/drill/yarn/core/drill-on-yarn-defaults.conf b/drill-yarn/src/main/resources/org/apache/drill/yarn/core/drill-on-yarn-defaults.conf
new file mode 100644
index 0000000..bda8152
--- /dev/null
+++ b/drill-yarn/src/main/resources/org/apache/drill/yarn/core/drill-on-yarn-defaults.conf
@@ -0,0 +1,275 @@
+# 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.
+# ---------------------------------------------------------------------------
+# Configuration for the Drill-on-YARN feature.
+# See drill-on-yarn-example.conf for details.
+# Some properties are private; the are meant for use by the Drill
+# developers. Those are marked as private here, and are explained here
+# rather than in the example config.
+
+drill.yarn: {
+  app-name: "Drill-on-YARN"
+
+  # Settings here support a default single-node cluster on the local host,
+  # using the default HDFS connection obtained from the Hadoop config files.
+
+  dfs: {
+    connection: ""
+    app-dir: "/user/drill"
+  }
+
+  drill-install: {
+    site-dir: ""
+    client-path: ""
+    dir-name: "<base>"
+
+    # Drill-on-YARN uses localization by default; it is the simplest and
+    # most reliable for production systems. However, localization can be
+    # disabled for development, testing or other special needs.
+
+    localize: true
+
+    # Localized runs typically provide a site directory. For various reasons,
+    # some users may wish to avoid using a site directory and instead just
+    # use their own $DRILL_HOME/conf directory as the site directory. In
+    # this case, $DRILL_HOME/conf must contain all site-specific content,
+    # including jars. This setting says to treat $DRILL_HOME/conf as the
+    # site directory. Disable this option if the Drill archive is rebuilt
+    # to contain the $DRILL_HOME/conf files. This flag is also automatically
+    # assumed if the Drill archive is *inside* $DRILL_HOME.
+
+    conf-as-site: true
+
+    # When localization is disabled, the path to Drill software defaults
+    # to the same location as on the client node. However, if the path
+    # on worker nodes is different, set the actual path here.
+
+    drill-home: ""
+
+    # Drill provides the option to store site-specific configuration outside
+    # of the Drill home directory. The site directory contains configuration
+    # files and custom Java code (in the jars subdirectory.) The site dir
+    # normally is localized by the client. However, in a non-localized
+    # run, use this setting to identify the location of the site directory
+    # on each worker node if the location is different than the node
+    # that runs the client.
+
+    site-dir: ""
+
+    # Key used for the Drill archive file in the AM launch config.
+    # Not usually changed by users.
+
+    drill-key: "drill"
+    site-key: "site"
+
+    # Set the Java java.library.path option to files that pre-exist on
+    # each Drillbit node. (This is not for libraries that are distributed
+    # by YARN.)
+
+    library-path: ""
+  }
+
+  yarn: {
+    queue: "default"
+    priority: 1
+    user: ""
+  }
+
+  hadoop: {
+    home: ""
+    class-path: ""
+    hbase-class-path: ""
+  }
+
+  client: {
+
+    # Time (in secs) between poll attempts on start and stop.
+
+    poll-sec: 1
+
+    # Maximum time to wait when starting the AM before asking the
+    # user to check status later.
+
+    start-wait-sec: 60
+
+    # Maximum time to wait when stopping the AM before skipping
+    # the confirmation step.
+
+    stop-wait-sec: 60
+  }
+
+  am: {
+    memory-mb: 512
+
+    # The AM is CPU-light, a single core will do.
+
+    vcores: 1
+
+    # The AM uses no disk. Available on selected YARN distributions.
+
+    disks: 0
+
+    # Heap memory for the AM. Estimate based on observation.
+
+    heap: "450M"
+
+    # Arguments passed to the AM JVM. (Private)
+
+    vm-args: ""
+
+    # AM-to-RM period in ms. (Private)
+
+    poll-ms: 1000
+
+    # AM clock tick period in ms. (Private)
+
+    tick-ms: 2000
+
+    # Set to true to dump launch variables and command to log.
+
+    debug-launch: false
+
+    # Extra class path for AM. The path must be valid on all nodes.
+    # Not normally used; provided only for special cases to avoid editing
+    # the launch script. This is a class path prefix to allow overrides
+    # of Drill classes.
+    # Analogous to DRILL_CLASSPATH_PREFIX in drill-config.sh
+    # Multiple entries should be separated by colons.
+
+    prefix-class-path: ""
+
+    # Extra class path for AM. The path must be valid on all nodes on which
+    # the AM may run.
+    # Not normally used; provided only for special cases to avoid editing
+    # the launch script.
+
+    class-path: ""
+
+    # Enables/disables auto shutdown when no Drillbits can run. Disable
+    # this when required for single-node testing.
+
+    auto-shutdown: true
+
+    # Specify nodes that can run the AM via a YARN node label expression.
+    # Blank means that node labele expression will be applied.
+
+    node-label-expr: ""
+  }
+
+  drillbit: {
+
+    # Default memory: 4096 (heap) + 8192 (direct) +  1024 (code cache) + 1024 overhead.
+
+    memory-mb: 14336
+    vcores: 4
+    disks: 1
+
+    heap: "4G"
+    max-direct-memory: "8G"
+    code-cache: "1G"
+
+    # Additional JVM arguments for the drillbit (Private)
+    # The value here is appended to any set in the launch script.
+    # Separate multiple arguments by spaces.
+
+    vm-args: ""
+
+    # Set to true to turn on garbage collection logging.
+    # Similar to SERVER_GC_OPTS in drillbit.sh. However under YARN
+    # all logging goes to YARN's log directory and the GC log is
+    # always called "gc.log".
+
+    log-gc: false
+
+    # Extra class path for Drill-bit. The path must be valid on all nodes.
+    # Not normally used; provided only for special cases to avoid editing
+    # the launch script. This is a class path prefix to allow overrides
+    # of Drill classes.
+    # Equivalent to DRILL_CLASSPATH_PREFIX in drill-config.sh
+    # Multiple entries should be separated by colons.
+
+    prefix-class-path: ""
+
+    # Extra class path for Drill extensions such as Hadoop or HBase.
+    # The path must be valid on all nodes. Inserted into Drill's
+    # class path before Drill's own third-party extensions; allows
+    # overriding of Drill's own included jars.
+    # Equivalent to HADOOP_CLASSPATH and HBASE_CLASSPATH
+    # in drill-config.sh
+
+    extn-class-path: ""
+
+    # Normal class path specification for jars added to Drill's own.
+    # Appears in the class path after Drill's jars. Useful for dependencies
+    # on custom data source implementations.
+    # The path must be valid on all nodes.
+    # Equivalent to DRILL_CLASSPATH in drill-config.sh
+
+    class-path: ""
+
+    # Maximum number of retries of each Drillbit before blacklisting the node.
+
+    max-retries: 3
+
+    # Set to true to dump launch variables and command to log.
+
+    debug-launch: false
+
+    # Disable normal YARN log location. Allows Drill to write to
+    # it's own log location as when running outside of YARN. However,
+    # under YARN, log aggregation is generally considered a "good thing."
+
+    disable-yarn-logs: false
+
+    # DoY maintains an approximate count of free nodes available under YARN.
+    # A common error is to ask for far more Drillbits than can be run given
+    # the available nodes. This setting limits the number of "extra" requests
+    # beyond the known free nodes. The reason this number is not zero is that
+    # the DoY count is approximate, so we want to provide so "wiggle room."
+
+    max-extra-nodes: 2
+
+    # Maximum time that the AM will wait when requesting a container to run
+    # the Drillbit. After this time, the request will timeout and the AM will
+    # decrease the desired cluster size to match the fact that no additional
+    # node appears to be available. The timeout is in seconds. 0 means no
+    # timeout
+
+    request-timeout-secs: 600
+  }
+
+  http: {
+    enabled: true
+    port: 8048
+    ssl-enabled: false
+    auth-type: "none"
+    # Default of one hour, as in the Drillbit
+    session-max-idle-secs: 3600
+    rest-key="secret"
+    docs-link: "http://drill.apache.org/docs/"
+    refresh-secs: 5
+  }
+
+  cluster: [
+
+    # Defined to support a demo single-node cluster.
+
+    {
+      name: "default"
+      type: "basic"
+      count: 1
+    }
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/java/org/apache/drill/yarn/client/TestClient.java
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/java/org/apache/drill/yarn/client/TestClient.java b/drill-yarn/src/test/java/org/apache/drill/yarn/client/TestClient.java
new file mode 100644
index 0000000..7912bd5
--- /dev/null
+++ b/drill-yarn/src/test/java/org/apache/drill/yarn/client/TestClient.java
@@ -0,0 +1,137 @@
+/*
+ * 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 static org.junit.Assert.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.io.UnsupportedEncodingException;
+
+import org.apache.drill.yarn.client.ClientContext;
+import org.apache.drill.yarn.client.DrillOnYarn;
+import org.junit.Test;
+
+public class TestClient {
+
+  /**
+   * Unchecked exception to allow capturing "exit" events without actually
+   * exiting.
+   */
+
+  public static class SimulatedExitException extends RuntimeException {
+    private static final long serialVersionUID = 1L;
+    public int exitCode;
+
+    public SimulatedExitException(int exitCode) {
+      this.exitCode = exitCode;
+    }
+  }
+
+  public static class TestContext extends ClientContext {
+    public static ByteArrayOutputStream captureOut = new ByteArrayOutputStream();
+    public static ByteArrayOutputStream captureErr = new ByteArrayOutputStream();
+
+    public static void testInit() {
+      init(new TestContext());
+      resetOutput();
+    }
+
+    @Override
+    public void exit(int exitCode) {
+      throw new SimulatedExitException(exitCode);
+    }
+
+    public static void resetOutput() {
+      try {
+        out.flush();
+        captureOut.reset();
+        out = new PrintStream(captureOut, true, "UTF-8");
+        err.flush();
+        captureErr.reset();
+        err = new PrintStream(captureErr, true, "UTF-8");
+      } catch (UnsupportedEncodingException e) {
+        throw new IllegalStateException(e);
+      }
+    }
+
+    public static String getOut() {
+      out.flush();
+      try {
+        return captureOut.toString("UTF-8");
+      } catch (UnsupportedEncodingException e) {
+        throw new IllegalStateException(e);
+      }
+    }
+
+    public static String getErr() {
+      out.flush();
+      try {
+        return captureErr.toString("UTF-8");
+      } catch (UnsupportedEncodingException e) {
+        throw new IllegalStateException(e);
+      }
+    }
+  }
+
+  /**
+   * Test the basics of the DrillOnYarn app. Does not try any real commands, but
+   * does check for the basic error conditions.
+   */
+
+  @Test
+  public void testBasics() {
+    TestContext.testInit();
+
+    // No arguments provided.
+
+    try {
+      DrillOnYarn.run(new String[] {});
+      fail();
+    } catch (SimulatedExitException e) {
+      assert (e.exitCode == -1);
+      assertTrue(TestContext.getOut().contains("Usage: "));
+    }
+
+    // Bogus command
+
+    try {
+      DrillOnYarn.run(new String[] { "bogus" });
+      fail();
+    } catch (SimulatedExitException e) {
+      assert (e.exitCode == -1);
+      assertTrue(TestContext.getOut().contains("Usage: "));
+    }
+
+    // Help command
+
+    try {
+      DrillOnYarn.run(new String[] { "help" });
+      fail();
+    } catch (SimulatedExitException e) {
+      assert (e.exitCode == -1);
+      assertTrue(TestContext.getOut().contains("Usage: "));
+    }
+  }
+
+  // The idea here is to set up a simulated client environment, then
+  // test each command. This is a big project.
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/java/org/apache/drill/yarn/client/TestCommandLineOptions.java
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/java/org/apache/drill/yarn/client/TestCommandLineOptions.java b/drill-yarn/src/test/java/org/apache/drill/yarn/client/TestCommandLineOptions.java
new file mode 100644
index 0000000..f996c5b
--- /dev/null
+++ b/drill-yarn/src/test/java/org/apache/drill/yarn/client/TestCommandLineOptions.java
@@ -0,0 +1,84 @@
+/*
+ * 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.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import org.apache.drill.yarn.client.CommandLineOptions;
+
+public class TestCommandLineOptions {
+  @Test
+  public void testOptions() {
+    CommandLineOptions opts = new CommandLineOptions();
+    opts.parse(new String[] {});
+    assertNull(opts.getCommand());
+
+    opts = new CommandLineOptions();
+    opts.parse(new String[] { "-h" });
+    assertEquals(CommandLineOptions.Command.HELP, opts.getCommand());
+
+    opts = new CommandLineOptions();
+    opts.parse(new String[] { "-?" });
+    assertEquals(CommandLineOptions.Command.HELP, opts.getCommand());
+
+    opts = new CommandLineOptions();
+    opts.parse(new String[] { "help" });
+    assertEquals(CommandLineOptions.Command.HELP, opts.getCommand());
+
+    opts = new CommandLineOptions();
+    opts.parse(new String[] { "start" });
+    assertEquals(CommandLineOptions.Command.START, opts.getCommand());
+
+    opts = new CommandLineOptions();
+    opts.parse(new String[] { "stop" });
+    assertEquals(CommandLineOptions.Command.STOP, opts.getCommand());
+
+    opts = new CommandLineOptions();
+    opts.parse(new String[] { "status" });
+    assertEquals(CommandLineOptions.Command.STATUS, opts.getCommand());
+
+    opts = new CommandLineOptions();
+    opts.parse(new String[] { "resize" });
+    assertNull(opts.getCommand());
+
+    opts = new CommandLineOptions();
+    opts.parse(new String[] { "resize", "10" });
+    assertEquals(CommandLineOptions.Command.RESIZE, opts.getCommand());
+    assertEquals("", opts.getResizePrefix());
+    assertEquals(10, opts.getResizeValue());
+
+    opts = new CommandLineOptions();
+    opts.parse(new String[] { "resize", "+2" });
+    assertEquals(CommandLineOptions.Command.RESIZE, opts.getCommand());
+    assertEquals("+", opts.getResizePrefix());
+    assertEquals(2, opts.getResizeValue());
+
+    opts = new CommandLineOptions();
+    opts.parse(new String[] { "resize", "-3" });
+    assertEquals(CommandLineOptions.Command.RESIZE, opts.getCommand());
+    assertEquals("-", opts.getResizePrefix());
+    assertEquals(3, opts.getResizeValue());
+
+    opts = new CommandLineOptions();
+    opts.parse(new String[] { "myDrill" });
+    assertNull(opts.getCommand());
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/java/org/apache/drill/yarn/core/TestConfig.java
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/java/org/apache/drill/yarn/core/TestConfig.java b/drill-yarn/src/test/java/org/apache/drill/yarn/core/TestConfig.java
new file mode 100644
index 0000000..0519484
--- /dev/null
+++ b/drill-yarn/src/test/java/org/apache/drill/yarn/core/TestConfig.java
@@ -0,0 +1,267 @@
+/*
+ * 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 static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.Test;
+
+import com.typesafe.config.Config;
+
+public class TestConfig {
+
+  /**
+   * Mock config that lets us tinker with loading and environment access for
+   * testing.
+   */
+
+  private static class DoYTestConfig extends DrillOnYarnConfig {
+    protected Map<String, String> mockEnv = new HashMap<>();
+    protected File configDir;
+
+    public DoYTestConfig(TestClassLoader cl, File configDir)
+        throws DoyConfigException {
+      doLoad(cl);
+      instance = this;
+      this.configDir = configDir;
+    }
+
+    @Override
+    protected String getEnv(String key) {
+      return mockEnv.get(key);
+    }
+  }
+
+  /**
+   * Mock class loader to let us add config files after the JVM starts. (In
+   * production code, the config file directories are added to the class path.)
+   */
+
+  private static class TestClassLoader extends ClassLoader {
+    private File configDir;
+
+    public TestClassLoader(ClassLoader parent, File configDir) {
+      super(parent);
+      this.configDir = configDir;
+    }
+
+    @Override
+    protected URL findResource(String name) {
+      File file = new File(configDir, name);
+      if (file.exists()) {
+        try {
+          return file.toURI().toURL();
+        } catch (MalformedURLException e) {
+          ;
+        }
+      }
+      return null;
+    }
+  }
+
+  /**
+   * Creates a stack of settings to test overrides.
+   * <table>
+   * <tr>
+   * <th>property</th>
+   * <th>default</th>
+   * <th>distrib</th>
+   * <th>user</th>
+   * <th>system</th>
+   * </tr>
+   * <tr>
+   * <td>drill-key</td>
+   * <td>"drill"</td>
+   * <td></td>
+   * <td></td>
+   * <td></td>
+   * </tr>
+   * <tr>
+   * <td>app-dir</td>
+   * <td>"/user/drill"</td>
+   * <td>"/opt/drill"</td>
+   * <td></td>
+   * <td></td>
+   * </tr>
+   * <tr>
+   * <td>app-name</td>
+   * <td>"Drill-on-YARN"</td>
+   * <td>"config-app-name"</td>
+   * <td>"My-App"</td>
+   * <td></td>
+   * </tr>
+   * <tr>
+   * <td>queue</td>
+   * <td>"default"</td>
+   * <td>"distrib-queue"</td>
+   * <td>"my-queue"</td>
+   * <td>"sys-queue"</td>
+   * </table>
+   * <p>
+   * Full property names:
+   * <ul>
+   * <li>drill.yarn.drill-install.drill-key</li>
+   * <li>drill.yarn.dfs.app-dir</li>
+   * <li>drill.yarn.app-name</li>
+   * <li>drill.yarn.zk.connect</li>
+   * </ul>
+   *
+   * @throws IOException
+   * @throws DoyConfigException
+   */
+  @Test
+  public void testLoad() throws IOException, DoyConfigException {
+
+    DoYTestConfig doyConfig = initConfig("test-doy-config.conf");
+    Config config = DrillOnYarnConfig.config();
+
+    assertEquals("drill",
+        config.getString(DrillOnYarnConfig.DRILL_ARCHIVE_KEY));
+    assertEquals("/opt/drill", config.getString(DrillOnYarnConfig.DFS_APP_DIR));
+    assertEquals("My-App", config.getString(DrillOnYarnConfig.APP_NAME));
+    // Commenting out for now, fails on VM.
+    //assertEquals("sys-queue", config.getString(DrillOnYarnConfig.YARN_QUEUE));
+
+    // Should also have access to Drill options.
+    // Does not test Drill's override mechanism because have not found a good
+    // way to add drill-override.conf to the class path in this test.
+
+    // assertEquals( "org.apache.drill.exec.opt.IdentityOptimizer",
+    // config.getString( "drill.exec.optimizer" ) );
+    assertEquals("drillbits1", config.getString(DrillOnYarnConfig.CLUSTER_ID));
+
+    // Drill home: with and without an env var.
+    // Must set the site env var. Class path testing can't be done here.
+    // No DRILL_HOME: will only occur during testing. In that case, we use
+    // the setting from the config file. Explicit site dir.
+
+    assertNull(doyConfig.mockEnv.get(DrillOnYarnConfig.DRILL_HOME_ENV_VAR));
+    doyConfig.mockEnv.put(DrillOnYarnConfig.DRILL_SITE_ENV_VAR, "/drill/site");
+    doyConfig.setClientPaths();
+    assertEquals("/config/drill/home",
+        doyConfig.getLocalDrillHome().getAbsolutePath());
+    assertTrue(doyConfig.hasSiteDir());
+    assertEquals("/drill/site", doyConfig.getLocalSiteDir().getAbsolutePath());
+
+    // Home set in an env var
+
+    doyConfig.mockEnv.put(DrillOnYarnConfig.DRILL_HOME_ENV_VAR, "/drill/home");
+    doyConfig.setClientPaths();
+    assertEquals("/drill/home",
+        doyConfig.getLocalDrillHome().getAbsolutePath());
+
+    // Remote site: localized case
+
+    assertTrue(config.getBoolean(DrillOnYarnConfig.LOCALIZE_DRILL));
+    assertEquals("/foo/bar/drill-archive.tar.gz",
+        config.getString(DrillOnYarnConfig.DRILL_ARCHIVE_PATH));
+    assertEquals("$PWD/drill/drill-archive", doyConfig.getRemoteDrillHome());
+    assertEquals("site", config.getString(DrillOnYarnConfig.SITE_ARCHIVE_KEY));
+    assertEquals("$PWD/site", doyConfig.getRemoteSiteDir());
+
+    // Localized, but no separate site directory
+
+    doyConfig.mockEnv.put(DrillOnYarnConfig.DRILL_SITE_ENV_VAR,
+        "/drill/home/conf");
+    doyConfig.setClientPaths();
+    // If $DRILL_HOME/conf is used, we still treat id as a site dir.
+//    assertFalse(doyConfig.hasSiteDir());
+//    assertNull(doyConfig.getRemoteSiteDir());
+
+    // Local app id file: composed from Drill home, ZK root and cluster id.
+    // (Turns out that there can be two different clusters sharing the same
+    // root...)
+    // With no site dir, app id is in parent of the drill directory.
+
+    assertEquals("/drill/home",
+        doyConfig.getLocalDrillHome().getAbsolutePath());
+    assertEquals("drill", config.getString(DrillOnYarnConfig.ZK_ROOT));
+    assertEquals("drillbits1", config.getString(DrillOnYarnConfig.CLUSTER_ID));
+    assertEquals("/drill/home/drill-drillbits1.appid",
+        doyConfig.getLocalAppIdFile().getAbsolutePath());
+
+    // Again, but with a site directory. App id is in parent of the site
+    // directory.
+
+    doyConfig.mockEnv.put(DrillOnYarnConfig.DRILL_SITE_ENV_VAR,
+        "/var/drill/site");
+    doyConfig.setClientPaths();
+    assertEquals("/var/drill/drill-drillbits1.appid",
+        doyConfig.getLocalAppIdFile().getAbsolutePath());
+  }
+
+  private DoYTestConfig initConfig(String configName)
+      throws IOException, DoyConfigException {
+    File tempDir = new File(System.getProperty("java.io.tmpdir"));
+    File configDir = new File(tempDir, "config");
+    if (configDir.exists()) {
+      FileUtils.forceDelete(configDir);
+    }
+    configDir.mkdirs();
+    configDir.deleteOnExit();
+
+    InputStream in = getClass().getResourceAsStream("/" + configName);
+    File dest = new File(configDir, "drill-on-yarn.conf");
+    Files.copy(in, dest.toPath(), StandardCopyOption.REPLACE_EXISTING);
+    in = getClass().getResourceAsStream("/test-doy-distrib.conf");
+    dest = new File(configDir, "doy-distrib.conf");
+    Files.copy(in, dest.toPath(), StandardCopyOption.REPLACE_EXISTING);
+
+    System.setProperty(DrillOnYarnConfig.YARN_QUEUE, "sys-queue");
+
+    TestClassLoader cl = new TestClassLoader(this.getClass().getClassLoader(),
+        configDir);
+
+    assertNotNull(cl.getResource(DrillOnYarnConfig.DISTRIB_FILE_NAME));
+
+    return new DoYTestConfig(cl, configDir);
+  }
+
+  @Test
+  public void testNonLocalized() throws IOException, DoyConfigException {
+    DoYTestConfig doyConfig = initConfig("second-test-config.conf");
+
+    // Test the non-localized case
+
+    doyConfig.mockEnv.put(DrillOnYarnConfig.DRILL_SITE_ENV_VAR, "/drill/site");
+    doyConfig.setClientPaths();
+    assertEquals("/config/drill/home", doyConfig.getRemoteDrillHome());
+    assertEquals("/config/drill/site", doyConfig.getRemoteSiteDir());
+  }
+
+  @Test
+  public void testNonLocalizedNonSite() throws IOException, DoyConfigException {
+    DoYTestConfig doyConfig = initConfig("third-test-config.conf");
+
+    // Test the non-localized case
+
+    assertEquals("/config/drill/home", doyConfig.getRemoteDrillHome());
+    assertNull(doyConfig.getRemoteSiteDir());
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/README.md
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/README.md b/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/README.md
new file mode 100644
index 0000000..3e4017b
--- /dev/null
+++ b/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/README.md
@@ -0,0 +1,65 @@
+# Script Test Overview
+
+The tests here exercise the Drill shell scripts with a wide variety of options.
+The Drill scripts are designed to be run from the command line or from YARN.
+The scripts allow passing in values from a vendor-specific configuration file
+($DRILL_HOME/conf/distrib-env.sh), a user-specific configuration file
+($DRILL_SITE/drill-env.sh) or from environment variables set by YARN.
+
+Testing scripts is normally tedious because the scripts are designed to start
+a process, perhaps a Drillbit or Sqlline. To make automated tests possible,
+the scripts incorporate a "shim": an environment variable that, if set, is
+used to put a "wrapper" script around the Java execution line. The wrapper
+captures the environment and the command line, and generates stderr and
+stdout output. The test progams use this captured output to determine if
+the Java command line has the options we expect. (We boldly assume that
+if we give Java the right options, it will do the right thing with them.)
+
+Why are the script tests in the drill-yarn project? Because YARN is the most
+sensitive to any changes: YARN provides several levels of indirection between
+the user and the scripts; the scripts must work exactly as the Drill-on-YARN
+code expects (or visa-versa) or things tend to break.
+
+## Inputs
+
+The test program needs the following inputs:
+
+- The scripts, which are copied from the source tree. (Need details)
+- /src/test/resources/wrapper.sh which is the "magic" wrapper script
+for capturing the command line, etc.
+- A temporary directory where the program can build its mock Drill
+and site directories.
+
+## Running the Tests
+
+Simply run the tests. Each test sets up its distribution and
+optional site directory and required environment variables. Each
+test uses a builder to build up the required script launch and
+to analyze the results. Each test function usually does a single
+setup, then does a bunch of test runs against that environment.
+
+Each test uses "gobbler" threads to read the stdout and stderr
+from the test run. If you run the test in a debugger, you'll
+see a steady stream of threads come and go. This is a test, so
+we don't bother with a thread pool; we just brute-force create
+new threads as needed.
+
+## Extending the Tests
+
+You should extend the tests if you:
+
+- Add new environment variables to any script.
+- Change the logic of any script.
+- Find a bug that these tests somehow did not catch.
+
+Note that it is very important to ensure that each new enviornment
+variable or other option works when set in distrib-env.sh,
+drill-env.sh or in the environment. These tests are the only (sane)
+way to test the many combinations, and to do that on each
+subsequent change.
+
+## About the Code
+
+TestScripts.java are the tests, organized by functional area. ScriptUtils.java
+is a large number of (mostly ad-hoc) utilities needed to set up, run, tear down
+and analyze the results of each run.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/ScriptUtils.java
----------------------------------------------------------------------
diff --git a/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/ScriptUtils.java b/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/ScriptUtils.java
new file mode 100644
index 0000000..3517cf8
--- /dev/null
+++ b/drill-yarn/src/test/java/org/apache/drill/yarn/scripts/ScriptUtils.java
@@ -0,0 +1,847 @@
+/*
+ * 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.scripts;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import org.apache.commons.io.FileUtils;
+
+public class ScriptUtils {
+
+  private static ScriptUtils instance = new ScriptUtils();
+  public File distribDir;
+  public File javaHome;
+  public File testDir;
+  public File testDrillHome;
+  public File testSiteDir;
+  public File testLogDir;
+  public boolean externalLogDir;
+
+  /**
+   * Out-of-the-box command-line arguments when launching sqlline.
+   * Order is not important here (though it is to Java.)
+   */
+
+  public static String sqlLineArgs[] = makeSqlLineArgs( );
+
+  private static String[] makeSqlLineArgs( ) {
+    String args[] = {
+      "-Dlog.path=/.*/drill/log/sqlline\\.log",
+      "-Dlog.query.path=/.*/drill/log/sqlline_queries\\.json",
+      "-XX:MaxPermSize=512M",
+      "sqlline\\.SqlLine",
+      "-d",
+      "org\\.apache\\.drill\\.jdbc\\.Driver",
+      "--maxWidth=10000",
+      "--color=true"
+    };
+
+    // Special handling if this machine happens to have the default
+    // /var/log/drill log location.
+
+    if ( new File( "/var/log/drill" ).exists() ) {
+      args[ 0 ] = "-Dlog\\.path=/var/log/drill/sqlline\\.log";
+      args[ 1 ] = "-Dlog\\.query\\.path=/var/log/drill/sqlline_queries\\.json";
+    }
+    return args;
+  }
+
+  public static final boolean USE_SOURCE = true;
+  public static final String TEMP_DIR = "/tmp";
+  public static boolean useSource = USE_SOURCE;
+
+  private ScriptUtils() {
+    String drillScriptsDir = System.getProperty("drillScriptDir");
+    assertNotNull(drillScriptsDir);
+    distribDir = new File(drillScriptsDir);
+    javaHome = new File(System.getProperty("java.home"));
+  }
+
+  public static ScriptUtils instance() {
+    return instance;
+  }
+
+  public ScriptUtils fromSource(String sourceDir) {
+    useSource = true;
+    return this;
+  }
+
+  public ScriptUtils fromDistrib(String distrib) {
+    distribDir = new File(distrib);
+    useSource = false;
+    return this;
+  }
+
+  /**
+   * Out-of-the-box command-line arguments when launching Drill.
+   * Order is not important here (though it is to Java.)
+   */
+
+  public static String stdArgs[] = buildStdArgs( );
+
+  private static String[] buildStdArgs( )
+  {
+    String args[] = {
+      "-Xms4G",
+      "-Xmx4G",
+      "-XX:MaxDirectMemorySize=8G",
+      "-XX:MaxPermSize=512M",
+      "-XX:ReservedCodeCacheSize=1G",
+      // Removed in Drill 1.8
+//      "-Ddrill\\.exec\\.enable-epoll=true",
+      "-XX:\\+CMSClassUnloadingEnabled",
+      "-XX:\\+UseG1GC",
+      "org\\.apache\\.drill\\.exec\\.server\\.Drillbit",
+      "-Dlog\\.path=/.*/script-test/drill/log/drillbit\\.log",
+      "-Dlog\\.query\\.path=/.*/script-test/drill/log/drillbit_queries\\.json",
+    };
+
+    // Special handling if this machine happens to have the default
+    // /var/log/drill log location.
+
+    if ( new File( "/var/log/drill" ).exists() ) {
+      args[ args.length-2 ] = "-Dlog\\.path=/var/log/drill/drillbit\\.log";
+      args[ args.length-1 ] = "-Dlog\\.query\\.path=/var/log/drill/drillbit_queries\\.json";
+    }
+    return args;
+  };
+
+  /**
+   * Out-of-the-box class-path before any custom additions.
+   */
+
+  static String stdCp[] =
+  {
+    "conf",
+    "jars/*",
+    "jars/ext/*",
+    "jars/3rdparty/*",
+    "jars/classb/*"
+  };
+
+  /**
+   * Directories to create to simulate a Drill distribution.
+   */
+
+  static String distribDirs[] = {
+      "bin",
+      "jars",
+      "jars/3rdparty",
+      "jars/ext",
+      "conf"
+  };
+
+  /**
+   * Out-of-the-box Jar directories.
+   */
+
+  static String jarDirs[] = {
+      "jars",
+      "jars/3rdparty",
+      "jars/ext",
+  };
+
+  /**
+   * Scripts we must copy from the source tree to create a simulated
+   * Drill bin directory.
+   */
+
+  public static String scripts[] = {
+      "drill-config.sh",
+      "drill-embedded",
+      "drill-localhost",
+      "drill-on-yarn.sh",
+      "drillbit.sh",
+      "drill-conf",
+      //dumpcat
+      //hadoop-excludes.txt
+      "runbit",
+      "sqlline",
+      //sqlline.bat
+      //submit_plan
+      "yarn-drillbit.sh"
+  };
+
+  /**
+   * Create the basic test directory. Tests add or remove details.
+   */
+
+  public void initialSetup() throws IOException {
+    File tempDir = new File(TEMP_DIR);
+    testDir = new File(tempDir, "script-test");
+    testDrillHome = new File(testDir, "drill");
+    testSiteDir = new File(testDir, "site");
+    File varLogDrill = new File( "/var/log/drill" );
+    if ( varLogDrill.exists() ) {
+      testLogDir = varLogDrill;
+      externalLogDir = true;
+    } else {
+      testLogDir = new File(testDrillHome, "log");
+    }
+    if (testDir.exists()) {
+      FileUtils.forceDelete(testDir);
+    }
+    testDir.mkdirs();
+    testSiteDir.mkdir();
+    testLogDir.mkdir();
+  }
+
+  public void createMockDistrib() throws IOException {
+    if (ScriptUtils.useSource) {
+      buildFromSource();
+    } else {
+      buildFromDistrib();
+    }
+  }
+
+  /**
+   * Build the Drill distribution directory directly from sources.
+   */
+
+  private void buildFromSource() throws IOException {
+    createMockDirs();
+    copyScripts(ScriptUtils.instance().distribDir);
+  }
+
+  /**
+   * Build the shell of a Drill distribution directory by creating the required
+   * directory structure.
+   */
+
+  private void createMockDirs() throws IOException {
+    if (testDrillHome.exists()) {
+      FileUtils.forceDelete(testDrillHome);
+    }
+    testDrillHome.mkdir();
+    for (String path : ScriptUtils.distribDirs) {
+      File subDir = new File(testDrillHome, path);
+      subDir.mkdirs();
+    }
+    for (String path : ScriptUtils.jarDirs) {
+      makeDummyJar(new File(testDrillHome, path), "dist");
+    }
+  }
+
+  /**
+   * The tests should not require jar files, but we simulate them to be a bit
+   * more realistic. Since we dont' run Java, the jar files can be simulated.
+   */
+
+  public File makeDummyJar(File dir, String prefix) throws IOException {
+    String jarName = "";
+    if (prefix != null) {
+      jarName += prefix + "-";
+    }
+    jarName += dir.getName() + ".jar";
+    File jarFile = new File(dir, jarName);
+    writeFile(jarFile, "Dummy jar");
+    return jarFile;
+  }
+
+  /**
+   * Create a simple text file with the given contents.
+   */
+
+  public void writeFile(File file, String contents) throws IOException {
+    try (PrintWriter out = new PrintWriter(new FileWriter(file))) {
+      out.println(contents);
+    }
+  }
+
+  /**
+   * Create a drill-env.sh or distrib-env.sh file with the given environment in
+   * the recommended format.
+   */
+
+  public void createEnvFile(File file, Map<String, String> env)
+      throws IOException {
+    try (PrintWriter out = new PrintWriter(new FileWriter(file))) {
+      out.println("#!/usr/bin/env bash");
+      for (String key : env.keySet()) {
+        String value = env.get(key);
+        out.print("export ");
+        out.print(key);
+        out.print("=${");
+        out.print(key);
+        out.print(":-\"");
+        out.print(value);
+        out.println("\"}");
+      }
+    }
+  }
+
+  /**
+   * Copy the standard scripts from source location to the mock distribution
+   * directory.
+   */
+
+  private void copyScripts(File sourceDir) throws IOException {
+    File binDir = new File(testDrillHome, "bin");
+    for (String script : ScriptUtils.scripts) {
+      File source = new File(sourceDir, script);
+      File dest = new File(binDir, script);
+      copyFile(source, dest);
+      dest.setExecutable(true);
+    }
+
+    // Create the "magic" wrapper script that simulates the Drillbit and
+    // captures the output we need for testing.
+
+    String wrapper = "wrapper.sh";
+    File dest = new File(binDir, wrapper);
+    try (InputStream is = getClass().getResourceAsStream("/" + wrapper)) {
+      Files.copy(is, dest.toPath(), StandardCopyOption.REPLACE_EXISTING);
+    }
+    dest.setExecutable(true);
+  }
+
+  private void buildFromDistrib() {
+    // TODO Auto-generated method stub
+
+  }
+
+  /**
+   * Consume the input from a stream, specifically the stderr or stdout stream
+   * from a process.
+   *
+   * @see http://stackoverflow.com/questions/14165517/processbuilder-forwarding-stdout-and-stderr-of-started-processes-without-blocki
+   */
+
+  private static class StreamGobbler extends Thread {
+    InputStream is;
+    public StringBuilder buf = new StringBuilder();
+
+    private StreamGobbler(InputStream is) {
+      this.is = is;
+    }
+
+    @Override
+    public void run() {
+      try {
+        InputStreamReader isr = new InputStreamReader(is);
+        BufferedReader br = new BufferedReader(isr);
+        String line = null;
+        while ((line = br.readLine()) != null) {
+          buf.append(line);
+          buf.append("\n");
+        }
+      } catch (IOException ioe) {
+        ioe.printStackTrace();
+      }
+    }
+  }
+
+  /**
+   * Handy run result class to capture the information we need for testing and
+   * to do various kinds of validation on it.
+   */
+
+  public static class RunResult {
+    File logDir;
+    File logFile;
+    String stdout;
+    String stderr;
+    List<String> echoArgs;
+    int returnCode;
+    String classPath[];
+    String libPath[];
+    String log;
+    public File pidFile;
+    public File outFile;
+    String out;
+
+    /**
+     * Split the class path into strings for easier validation.
+     */
+
+    public void analyze() {
+      if (echoArgs == null) {
+        return;
+      }
+      for (int i = 0; i < echoArgs.size(); i++) {
+        String arg = echoArgs.get(i);
+        if (arg.equals("-cp")) {
+          classPath = Pattern.compile(":").split((echoArgs.get(i + 1)));
+          break;
+        }
+      }
+      String probe = "-Djava.library.path=";
+      for (int i = 0; i < echoArgs.size(); i++) {
+        String arg = echoArgs.get(i);
+        if (arg.startsWith(probe)) {
+          assertNull(libPath);
+          libPath = Pattern.compile(":").split((arg.substring(probe.length())));
+          break;
+        }
+      }
+    }
+
+    /**
+     * Read the log file, if any, generated by the process.
+     */
+
+    public void loadLog() throws IOException {
+      log = loadFile(logFile);
+    }
+
+    private String loadFile(File file) throws IOException {
+      StringBuilder buf = new StringBuilder();
+      try ( BufferedReader reader = new BufferedReader(new FileReader(file)) ) {
+        String line;
+        while ((line = reader.readLine()) != null) {
+          buf.append(line);
+          buf.append("\n");
+        }
+        return buf.toString();
+      } catch (FileNotFoundException e) {
+        return null;
+      }
+    }
+
+    /**
+     * Validate that the first argument invokes Java correctly.
+     */
+
+    public void validateJava() {
+      assertNotNull(echoArgs);
+      String java = instance.javaHome + "/bin/java";
+      List<String> actual = echoArgs;
+      assertEquals(java, actual.get(0));
+    }
+
+    public boolean containsArg(String arg) {
+      for (String actual : echoArgs) {
+        if (actual.equals(arg)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    public void validateStockArgs() {
+      for (String arg : ScriptUtils.stdArgs) {
+        assertTrue("Argument not found: " + arg + " in " + echoArgs, containsArgRegex(arg));
+      }
+    }
+
+    public void validateArg(String arg) {
+      validateArgs(Collections.singletonList(arg));
+    }
+
+    public void validateArgs(String args[]) {
+      validateArgs(Arrays.asList(args));
+    }
+
+    public void validateArgs(List<String> args) {
+      validateJava();
+      for (String arg : args) {
+        assertTrue(containsArg(arg));
+      }
+    }
+
+    public void validateArgRegex(String arg) {
+      assertTrue(containsArgRegex(arg));
+    }
+
+    public void validateArgsRegex(List<String> args) {
+      assertTrue(containsArgsRegex(args));
+    }
+
+    public boolean containsArgsRegex(List<String> args) {
+      for (String arg : args) {
+        if (!containsArgRegex(arg)) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    public boolean containsArgsRegex(String args[]) {
+      for (String arg : args) {
+        if (!containsArgRegex(arg)) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    public boolean containsArgRegex(String arg) {
+      for (String actual : echoArgs) {
+        if (actual.matches(arg)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    public void validateClassPath(String expectedCP) {
+      assertTrue(classPathContains(expectedCP));
+    }
+
+    public void validateClassPath(String expectedCP[]) {
+      assertTrue(classPathContains(expectedCP));
+    }
+
+    public boolean classPathContains(String expectedCP[]) {
+      for (String entry : expectedCP) {
+        if (!classPathContains(entry)) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    public boolean classPathContains(String expectedCP) {
+      if (classPath == null) {
+        fail("No classpath returned");
+      }
+      String tail = "/" + instance.testDir.getName() + "/"
+          + instance.testDrillHome.getName() + "/";
+      String expectedPath;
+      if (expectedCP.startsWith("/")) {
+        expectedPath = expectedCP;
+      } else {
+        expectedPath = tail + expectedCP;
+      }
+      for (String entry : classPath) {
+        if (entry.endsWith(expectedPath)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    public void loadOut() throws IOException {
+      out = loadFile(outFile);
+    }
+
+    /**
+     * Ensure that the Drill log file contains at least the sample message
+     * written by the wrapper.
+     */
+
+    public void validateDrillLog() {
+      assertNotNull(log);
+      assertTrue(log.contains("Drill Log Message"));
+    }
+
+    /**
+     * Validate that the stdout contained the expected message.
+     */
+
+    public void validateStdOut() {
+      assertTrue(stdout.contains("Starting drillbit on"));
+    }
+
+    /**
+     * Validate that the stderr contained the sample error message from the
+     * wrapper.
+     */
+
+    public void validateStdErr() {
+      assertTrue(stderr.contains("Stderr Message"));
+    }
+
+    public int getPid() throws IOException {
+      try (BufferedReader reader = new BufferedReader(new FileReader(pidFile))) {
+        return Integer.parseInt(reader.readLine());
+      }
+      finally { }
+    }
+
+  }
+
+  /**
+   * The "business end" of the tests: runs drillbit.sh and captures results.
+   */
+
+  public static class ScriptRunner {
+    // Drillbit commands
+
+    public static String DRILLBIT_RUN = "run";
+    public static String DRILLBIT_START = "start";
+    public static String DRILLBIT_STATUS = "status";
+    public static String DRILLBIT_STOP = "stop";
+    public static String DRILLBIT_RESTART = "restart";
+
+    public File cwd = instance.testDir;
+    public File drillHome = instance.testDrillHome;
+    public String script;
+    public List<String> args = new ArrayList<>();
+    public Map<String, String> env = new HashMap<>();
+    public File logDir;
+    public File pidFile;
+    public File outputFile;
+    public boolean preserveLogs;
+
+    public ScriptRunner(String script) {
+      this.script = script;
+    }
+
+    public ScriptRunner(String script, String cmd) {
+      this(script);
+      args.add(cmd);
+    }
+
+    public ScriptRunner(String script, String cmdArgs[]) {
+      this(script);
+      for (String arg : cmdArgs) {
+        args.add(arg);
+      }
+    }
+
+    public ScriptRunner withArg(String arg) {
+      args.add(arg);
+      return this;
+    }
+
+    public ScriptRunner withSite(File siteDir) {
+      if (siteDir != null) {
+        args.add("--site");
+        args.add(siteDir.getAbsolutePath());
+      }
+      return this;
+    }
+
+    public ScriptRunner withEnvironment(Map<String, String> env) {
+      if (env != null) {
+        this.env.putAll(env);
+      }
+      return this;
+    }
+
+    public ScriptRunner addEnv(String key, String value) {
+      env.put(key, value);
+      return this;
+    }
+
+    public ScriptRunner withLogDir(File logDir) {
+      this.logDir = logDir;
+      return this;
+    }
+
+    public ScriptRunner preserveLogs() {
+      preserveLogs = true;
+      return this;
+    }
+
+    public RunResult run() throws IOException {
+      File binDir = new File(drillHome, "bin");
+      File scriptFile = new File(binDir, script);
+      assertTrue(scriptFile.exists());
+      outputFile = new File(instance.testDir, "output.txt");
+      outputFile.delete();
+      if (logDir == null) {
+        logDir = instance.testLogDir;
+      }
+      if (!preserveLogs) {
+        cleanLogs(logDir);
+      }
+
+      Process proc = startProcess(scriptFile);
+      RunResult result = runProcess(proc);
+      if (result.returnCode == 0) {
+        captureOutput(result);
+        captureLog(result);
+      }
+      return result;
+    }
+
+    private void cleanLogs(File logDir) throws IOException {
+      if ( logDir == instance.testLogDir  &&  instance.externalLogDir ) {
+        return;
+      }
+      if (logDir.exists()) {
+        FileUtils.forceDelete(logDir);
+      }
+    }
+
+    private Process startProcess(File scriptFile) throws IOException {
+      outputFile.delete();
+      List<String> cmd = new ArrayList<>();
+      cmd.add(scriptFile.getAbsolutePath());
+      cmd.addAll(args);
+      ProcessBuilder pb = new ProcessBuilder().command(cmd).directory(cwd);
+      Map<String, String> pbEnv = pb.environment();
+      pbEnv.clear();
+      pbEnv.putAll(env);
+      File binDir = new File(drillHome, "bin");
+      File wrapperCmd = new File(binDir, "wrapper.sh");
+
+      // Set the magic wrapper to capture output.
+
+      pbEnv.put("_DRILL_WRAPPER_", wrapperCmd.getAbsolutePath());
+      pbEnv.put("JAVA_HOME", instance.javaHome.getAbsolutePath());
+      return pb.start();
+    }
+
+    private RunResult runProcess(Process proc) {
+      StreamGobbler errorGobbler = new StreamGobbler(proc.getErrorStream());
+      StreamGobbler outputGobbler = new StreamGobbler(proc.getInputStream());
+      outputGobbler.start();
+      errorGobbler.start();
+
+      try {
+        proc.waitFor();
+      } catch (InterruptedException e) {
+        // Won't occur.
+      }
+
+      RunResult result = new RunResult();
+      result.stderr = errorGobbler.buf.toString();
+      result.stdout = outputGobbler.buf.toString();
+      result.returnCode = proc.exitValue();
+      return result;
+    }
+
+    private void captureOutput(RunResult result) throws IOException {
+      // Capture the Java arguments which the wrapper script wrote to a file.
+
+      try (BufferedReader reader = new BufferedReader(new FileReader(outputFile))) {
+         result.echoArgs = new ArrayList<>();
+        String line;
+        while ((line = reader.readLine()) != null) {
+          result.echoArgs.add(line);
+        }
+        result.analyze();
+      } catch (FileNotFoundException e) {
+        ;
+      }
+    }
+
+    private void captureLog(RunResult result) throws IOException {
+      result.logDir = logDir;
+      result.logFile = new File(logDir, "drillbit.log");
+      if (result.logFile.exists()) {
+        result.loadLog();
+      } else {
+        result.logFile = null;
+      }
+    }
+  }
+
+  public static class DrillbitRun extends ScriptRunner {
+    public File pidDir;
+
+    public DrillbitRun() {
+      super("drillbit.sh");
+    }
+
+    public DrillbitRun(String cmd) {
+      super("drillbit.sh", cmd);
+    }
+
+    public DrillbitRun withPidDir(File pidDir) {
+      this.pidDir = pidDir;
+      return this;
+    }
+
+    public DrillbitRun asDaemon() {
+      addEnv("KEEP_RUNNING", "1");
+      return this;
+    }
+
+    public RunResult start() throws IOException {
+      if (pidDir == null) {
+        pidDir = drillHome;
+      }
+      pidFile = new File(pidDir, "drillbit.pid");
+      // pidFile.delete();
+      asDaemon();
+      RunResult result = run();
+      if (result.returnCode == 0) {
+        capturePidFile(result);
+        captureDrillOut(result);
+      }
+      return result;
+    }
+
+    private void capturePidFile(RunResult result) {
+      assertTrue(pidFile.exists());
+      result.pidFile = pidFile;
+    }
+
+    private void captureDrillOut(RunResult result) throws IOException {
+      // Drillbit.out
+
+      result.outFile = new File(result.logDir, "drillbit.out");
+      if (result.outFile.exists()) {
+        result.loadOut();
+      } else {
+        result.outFile = null;
+      }
+    }
+
+  }
+
+  /**
+   * Build a "starter" conf or site directory by creating a mock
+   * drill-override.conf file.
+   */
+
+  public void createMockConf(File siteDir) throws IOException {
+    createDir(siteDir);
+    File override = new File(siteDir, "drill-override.conf");
+    writeFile(override, "# Dummy override");
+  }
+
+  public void removeDir(File dir) throws IOException {
+    if (dir.exists()) {
+      FileUtils.forceDelete(dir);
+    }
+  }
+
+  /**
+   * Remove, then create a directory.
+   */
+
+  public File createDir(File dir) throws IOException {
+    removeDir(dir);
+    dir.mkdirs();
+    return dir;
+  }
+
+  public void copyFile(File source, File dest) throws IOException {
+    Files.copy(source.toPath(), dest.toPath(),
+        StandardCopyOption.REPLACE_EXISTING);
+  }
+
+}


[03/12] drill git commit: DRILL-1170: YARN integration for Drill

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/core/NameValuePair.java
----------------------------------------------------------------------
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 0000000..5872ab9
--- /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();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/core/YarnClientException.java
----------------------------------------------------------------------
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 0000000..62dd468
--- /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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/core/YarnRMClient.java
----------------------------------------------------------------------
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 0000000..8905ce3
--- /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());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/core/package-info.java
----------------------------------------------------------------------
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 0000000..aaa0fff
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/package-info.java
----------------------------------------------------------------------
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 0000000..170dfa8
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/zk/AMRegistry.java
----------------------------------------------------------------------
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 0000000..9cc95e5
--- /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);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinator.java
----------------------------------------------------------------------
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 0000000..7c5f5f3
--- /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();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKClusterCoordinatorDriver.java
----------------------------------------------------------------------
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 0000000..3f83ff2
--- /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.
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKConfigException.java
----------------------------------------------------------------------
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 0000000..700d84b
--- /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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRegistry.java
----------------------------------------------------------------------
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 0000000..0426578
--- /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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/zk/ZKRuntimeException.java
----------------------------------------------------------------------
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 0000000..4e1b115
--- /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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/zk/package-info.java
----------------------------------------------------------------------
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 0000000..14bb427
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/drill-am/config.ftl
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/drill-am/config.ftl b/drill-yarn/src/main/resources/drill-am/config.ftl
new file mode 100644
index 0000000..8405c1f
--- /dev/null
+++ b/drill-yarn/src/main/resources/drill-am/config.ftl
@@ -0,0 +1,41 @@
+<#-- 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. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+</#macro>
+
+<#macro page_body>
+  <h4>Fully Resolved Configuration Settings</h4>
+  <p>&nbsp;
+
+  <table class="table table-hover" style="width: auto;">
+    <tr>
+      <th>Configuration Key</td>
+      <th>Value</td>
+    </tr>
+    <#list model as pair>
+      <tr>
+        <td>${pair.getName()}</td>
+        <td>${pair.getQuotedValue()}</td>
+      </tr>
+    </#list>
+  </table>
+  <p>
+  To modify these values:
+  <ol>
+  <li>Edit <code>$DRILL_SITE/drill-on-yarn.conf</code> (for the drill.yarn settings),</li>
+  <li>Edit <code>$DRILL_SITE/drill-override.conf</code> (for the drill.exec settings).</li>
+  <li>Restart your Drill cluster using the Drill-on-YARN client.</li>
+  </ol>
+</#macro>
+
+<@page_html/>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/drill-am/confirm.ftl
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/drill-am/confirm.ftl b/drill-yarn/src/main/resources/drill-am/confirm.ftl
new file mode 100644
index 0000000..515293d
--- /dev/null
+++ b/drill-yarn/src/main/resources/drill-am/confirm.ftl
@@ -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. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+</#macro>
+
+<#macro page_body>
+  <h4><#if model.getType( ) == "STOPPED">
+  Stop Drill Cluster
+  <#else>
+  Resize Drill Cluster
+  </#if></h4>
+  <p>&nbsp;
+
+  <#if model.getType( ) == "RESIZED">
+    <div class="alert alert-success">
+      <strong>Success!</strong> Cluster resizing to ${model.getValue( )} nodes.
+    </div>
+  <#elseif model.getType( ) == "CANCELLED">
+    <div class="alert alert-info">
+       <strong>Success!</strong> Drillbit ${model.getValue( )} was cancelled.
+    </div>
+  <#elseif model.getType( ) == "NULL_RESIZE">
+    <div class="alert alert-info">
+      <strong>Note:</strong> The new size of ${model.getValue( )} is the
+      same as the current cluster size.
+    </div>
+  <#elseif model.getType( ) == "INVALID_RESIZE">
+    <div class="alert alert-danger">
+      <strong>Error!</strong> Invalid cluster resize level: ${model.getValue( )}.
+      Please <a href="/manage">try again</a>.
+    </div>
+  <#elseif model.getType( ) == "INVALID_GROW">
+    <div class="alert alert-danger">
+      <strong>Error!</strong> Invalid cluster grow amount: ${model.getValue( )}.
+      Please <a href="/manage">try again</a>.
+    </div>
+  <#elseif model.getType( ) == "INVALID_SHRINK">
+    <div class="alert alert-danger">
+      <strong>Error!</strong> Invalid cluster shrink amount: ${model.getValue( )}.
+      Please <a href="/manage">try again</a>.
+    </div>
+  <#elseif model.getType( ) == "INVALID_TASK">
+    <div class="alert alert-danger">
+      <strong>Error!</strong> Invalid Drillbit ID: ${model.getValue( )}.
+      Perhaps the Drillbit has already stopped.
+    </div>
+  <#elseif model.getType( ) == "STOPPED">
+    <div class="alert alert alert-success">
+      <strong>Success!</strong> Cluster is shutting down.
+    </div>
+    Pages on this site will be unavailable until the cluster restarts.
+  </#if>
+  <#if model.getType( ) == "CANCELLED">
+    Return to the <a href="/drillbits">Drillbits page</a>.
+  <#elseif model.getType( ) != "STOPPED">
+    Return to the <a href="/manage">Management page</a>.
+  </#if>
+</#macro>
+
+<@page_html/>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/drill-am/generic.ftl
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/drill-am/generic.ftl b/drill-yarn/src/main/resources/drill-am/generic.ftl
new file mode 100644
index 0000000..b76a917
--- /dev/null
+++ b/drill-yarn/src/main/resources/drill-am/generic.ftl
@@ -0,0 +1,78 @@
+<#-- 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. -->
+
+<#-- Adapted from the Drill generic.ftl, adjusted for use in the AM. -->
+
+<#macro page_head>
+</#macro>
+
+<#macro page_body>
+</#macro>
+
+<#macro page_html>
+      <meta http-equiv="X-UA-Compatible" content="IE=edge">
+
+      <title>Apache Drill - Application Master</title>
+      <link rel="shortcut icon" href="/static/img/drill.ico">
+
+      <link href="/static/css/bootstrap.min.css" rel="stylesheet">
+      <link href="/drill-am/static/css/drill-am.css" rel="stylesheet">
+
+      <script src="/static/js/jquery.min.js"></script>
+      <script src="/static/js/bootstrap.min.js"></script>
+
+      <!-- HTML5 shim and Respond.js IE8 support of HTML5 elements and media queries -->
+      <!--[if lt IE 9]>
+        <script src="/static/js/html5shiv.js"></script>
+        <script src="/static/js/1.4.2/respond.min.js"></script>
+      <![endif]-->
+
+      <@page_head/>
+    </head>
+    <body role="document">
+      <div class="navbar navbar-inverse navbar-fixed-top" role="navigation">
+        <div class="container-fluid">
+          <div class="navbar-header">
+            <button type="button" class="navbar-toggle" data-toggle="collapse" data-target=".navbar-collapse">
+              <span class="sr-only">Toggle navigation</span>
+              <span class="icon-bar"></span>
+              <span class="icon-bar"></span>
+              <span class="icon-bar"></span>
+            </button>
+            <a class="navbar-brand" href="/">Apache Drill</a>
+          </div>
+          <div class="navbar-collapse collapse">
+            <ul class="nav navbar-nav">
+              <li><a href="/config">Configuration</a></li>
+              <li><a href="/drillbits">Drillbits</a></li>
+              <li><a href="/manage">Manage</a></li>
+              <li><a href="/history">History</a></li>
+            </ul>
+            <ul class="nav navbar-nav navbar-right">
+              <li><a href="${docsLink}">Documentation</a>
+              <#if showLogin == true >
+              <li><a href="/login">Log In</a>
+              </#if>
+              <#if showLogout == true >
+              <li><a href="/logout">Log Out (${loggedInUserName})</a>
+             </#if>
+            </ul>
+          </div>
+        </div>
+      </div>
+
+      <div class="container-fluid drill-am" role="main">
+        <h3>YARN Application Master &ndash; ${clusterName}</h3>
+        <@page_body/>
+      </div>
+    </body>
+  </html>
+</#macro>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/drill-am/history.ftl
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/drill-am/history.ftl b/drill-yarn/src/main/resources/drill-am/history.ftl
new file mode 100644
index 0000000..c588d06
--- /dev/null
+++ b/drill-yarn/src/main/resources/drill-am/history.ftl
@@ -0,0 +1,59 @@
+<#-- 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. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+  <meta http-equiv="refresh" content="${refreshSecs}" >
+</#macro>
+
+<#macro page_body>
+  <h4>Drillbit History</h4>
+  <p>&nbsp;
+
+  <div class="table-responsive">
+    <table class="table table-hover">
+      <tr>
+        <th>ID</th>
+        <th>Try</th>
+        <th>Pool</th>
+        <th>Host</th>
+        <th>Container</th>
+        <th>Memory (MB)</th>
+        <th>Virtual Cores</th>
+        <th>Start Time</th>
+        <th>End Time</th>
+        <th>Disposition</th>
+      </th>
+       <#assign count=0>
+       <#list model as task>
+        <#assign count=count+1>
+        <tr>
+          <td><b>${task.getTaskId( )}</b></td>
+          <td>${task.getTryCount( )}</td>
+          <td>${task.getGroupName( )}</td>
+          <td><#if task.hasContainer( )><a href="${task.getNmLink( )}">${task.getHost( )}</a>
+          <#else>&nbsp;</#if></td>
+          <td>${task.getContainerId()}</td>
+          <td>${task.getMemory( )}</td>
+          <td>${task.getVcores( )}</td>
+          <td>${task.getStartTime( )}</td>
+          <td>${task.getEndTime( )}</td>
+          <td>${task.getDisposition( )}</td>
+        </tr>
+      </#list>
+    </table>
+    <#if count == 0>
+    No drillbits have completed.
+    </#if>
+  </div>
+</#macro>
+
+<@page_html/>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/drill-am/index.ftl
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/drill-am/index.ftl b/drill-yarn/src/main/resources/drill-am/index.ftl
new file mode 100644
index 0000000..18d6ab5
--- /dev/null
+++ b/drill-yarn/src/main/resources/drill-am/index.ftl
@@ -0,0 +1,128 @@
+<#-- 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. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+  <meta http-equiv="refresh" content="${model.getRefreshSecs( )}" >
+</#macro>
+
+<#macro page_body>
+  <h4>Drill Cluster Status</h4>
+
+  <table class="table table-hover" style="width: auto;">
+    <tr>
+      <td>YARN Application ID:</td>
+      <td><a href="${model.getRmAppLink( )}" data-toggle="tooltip" title="YARN Resource Manager page for this application">
+      ${model.getAppId( )}</a></td>
+    </tr>
+    <tr>
+      <td>YARN Resource Manager:</td>
+      <td><#if model.getRmLink()?? > <#-- Occurs early in startup before app is fully registered. -->
+      <a href="${model.getRmLink( )}" data-toggle="tooltip" title="YARN Resource Manager page for this container">
+      ${model.getRmHost( )}</a>
+      <#else>Unavailable
+      </#if></td>
+    </tr>
+    <tr>
+      <td>YARN Node Manager for AM:</td>
+      <td><#if model.getNmLink()?? > <#-- Occurs early in startup before app is fully registered. -->
+      <a href="${model.getNmLink( )}" data-toggle="tooltip" title="YARN Node Manager">
+      ${model.getNmHost( )}</a> |
+          <a href="${model.getNmAppLink( )}" data-toggle="tooltip" title="YARN Node Manager page for this application">App info</a>
+      <#else>Unavailable
+      </#if></td>
+    </tr>
+    <tr>
+      <td>ZooKeeper Hosts:</td>
+      <td><span data-toggle="tooltip" title="ZooKeeper connection string.">
+      ${model.getZkConnectionStr( )}</span></td>
+    </tr>
+    <tr>
+      <td>ZooKeeper Root:</td>
+      <td><span data-toggle="tooltip" title="ZooKeeper Drill root.">
+      ${model.getZkRoot( )}</span></td>
+    </tr>
+    <tr>
+      <td>ZooKeeper Cluster ID:</td>
+      <td><span data-toggle="tooltip" title="ZooKeeper Drill cluster-id.">
+      ${model.getZkClusterId( )}</span></td>
+    </tr>
+    <tr>
+      <td>State:</td>
+      <td><span data-toggle="tooltip" title="${model.getStateHint( )}">
+      ${model.getState( )}</span></td>
+    </tr>
+    <tr>
+      <td>Target Drillbit Count:</td>
+      <td>${model.getTargetCount( )}</td>
+    </tr>
+    <tr>
+      <td>Live Drillbit Count:</td>
+      <td>${model.getLiveCount( )}</td>
+    </tr>
+    <#if model.getUnmanagedCount( ) gt 0 >
+      <tr>
+        <td style="color: red">Unmanaged Drillbit Count:</td>
+        <td>${model.getUnmanagedCount( )}</td>
+      </tr>
+    </#if>
+    <#if model.getBlacklistCount( ) gt 0 >
+      <tr>
+        <td style="color: red">Blacklisted Node Count:</td>
+        <td>${model.getBlacklistCount( )}</td>
+      </tr>
+    </#if>
+    <tr>
+      <td>Total Drill Virtual Cores:</td>
+      <td>${model.getDrillTotalVcores( )}</td>
+    </tr>
+    <tr>
+      <td>Total Drill Memory (MB):</td>
+      <td>${model.getDrillTotalMemory( )}</td>
+    </tr>
+    <#if model.supportsDiskResource( ) >
+      <tr>
+        <td>Total Drill Disks:</td>
+        <td>${model.getDrillTotalDisks( )}</td>
+      </tr>
+    </#if>
+  </table>
+  <table class="table table-hover" style="width: auto;">
+    <tr>
+      <th>Group</th>
+      <th>Type</th>
+      <th>Target Drillbit Count</th>
+      <th>Total Drillbits</th>
+      <th>Live Drillbits</th>
+      <th>Memory per Drillbit (MB)</th>
+      <th>VCores per Drillbit</th>
+      <#if model.supportsDiskResource( ) >
+        <th>Disks per Drillbit</th>
+      </#if>
+    </tr>
+    <#list model.getGroups( ) as group>
+      <tr>
+        <td>${group.getName( )}</td>
+        <td>${group.getType( )}</td>
+        <td>${group.getTargetCount( )}</td>
+        <td>${group.getTaskCount( )}</td>
+        <td>${group.getLiveCount( )}</td>
+        <td>${group.getMemory( )}</td>
+        <td>${group.getVcores( )}</td>
+        <#if model.supportsDiskResource( ) >
+          <td>${group.getDisks( )}</td>
+        </#if>
+      </tr>
+    </#list>
+  </table>
+</#macro>
+
+<@page_html/>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/resources/drill-am/login.ftl
----------------------------------------------------------------------
diff --git a/drill-yarn/src/main/resources/drill-am/login.ftl b/drill-yarn/src/main/resources/drill-am/login.ftl
new file mode 100644
index 0000000..036229e
--- /dev/null
+++ b/drill-yarn/src/main/resources/drill-am/login.ftl
@@ -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. -->
+
+<#include "*/generic.ftl">
+<#macro page_head>
+</#macro>
+
+<#macro page_body>
+  <div align="center" class="table-responsive">
+    <form role="form" name="input" action="/j_security_check" method="POST">
+      <fieldset>
+        <div class="form-group">
+          <img src="/drill-am/static/img/apache-drill-logo.png" alt="Apache Drill Logo">
+          <#if model??>
+          <div class="alert alert-danger">
+            <strong>Error</strong> ${model}
+          </div>
+          </#if>
+          <p><input type="text" size="30" name="j_username" placeholder="Username"></p>
+          <p><input type="password" size="30" name="j_password" placeholder="Password"></p>
+          <p><button type="submit" class="btn btn-primary">Log In</button></p>
+        </div>
+      </fieldset>
+    </form>
+  </div>
+</#macro>
+<@page_html/>
\ No newline at end of file


[10/12] drill git commit: DRILL-1170: YARN integration for Drill

Posted by ar...@apache.org.
DRILL-1170: YARN integration for Drill

closes #1011


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/f2ac8749
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/f2ac8749
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/f2ac8749

Branch: refs/heads/master
Commit: f2ac8749b42539ca6301024becbf3e7092e9511e
Parents: cf2478f
Author: Paul Rogers <pr...@maprtech.com>
Authored: Thu Oct 26 00:24:00 2017 -0700
Committer: Arina Ielchiieva <ar...@gmail.com>
Committed: Sun Mar 4 17:43:22 2018 +0200

----------------------------------------------------------------------
 distribution/pom.xml                            |    5 +
 distribution/src/assemble/bin.xml               |   40 +
 distribution/src/resources/drill-am-log.xml     |   54 +
 distribution/src/resources/drill-am.sh          |  137 ++
 .../src/resources/drill-on-yarn-example.conf    |  204 +++
 distribution/src/resources/drill-on-yarn.sh     |   74 +
 distribution/src/resources/yarn-client-log.xml  |   44 +
 distribution/src/resources/yarn-drillbit.sh     |  178 +++
 drill-yarn/README.md                            |  190 +++
 drill-yarn/USAGE.md                             |  941 +++++++++++++
 drill-yarn/img/am-overview.png                  |  Bin 0 -> 111982 bytes
 drill-yarn/img/client-classes.png               |  Bin 0 -> 138409 bytes
 drill-yarn/img/controller-classes.png           |  Bin 0 -> 157668 bytes
 drill-yarn/img/overview.png                     |  Bin 0 -> 65994 bytes
 drill-yarn/pom.xml                              |  144 ++
 .../drill/yarn/appMaster/AMException.java       |   30 +
 .../drill/yarn/appMaster/AMRegistrar.java       |   43 +
 .../yarn/appMaster/AMWrapperException.java      |   25 +
 .../drill/yarn/appMaster/AMYarnFacade.java      |   91 ++
 .../drill/yarn/appMaster/AMYarnFacadeImpl.java  |  288 ++++
 .../appMaster/AbstractDrillbitScheduler.java    |  166 +++
 .../drill/yarn/appMaster/AbstractScheduler.java |  112 ++
 .../yarn/appMaster/AbstractTaskManager.java     |   50 +
 .../drill/yarn/appMaster/BatchScheduler.java    |   88 ++
 .../drill/yarn/appMaster/ClusterController.java |  206 +++
 .../yarn/appMaster/ClusterControllerImpl.java   |  785 +++++++++++
 .../drill/yarn/appMaster/ControllerFactory.java |   30 +
 .../drill/yarn/appMaster/ControllerVisitor.java |   22 +
 .../apache/drill/yarn/appMaster/Dispatcher.java |  345 +++++
 .../drill/yarn/appMaster/DispatcherAddOn.java   |   30 +
 .../yarn/appMaster/DrillApplicationMaster.java  |  119 ++
 .../yarn/appMaster/DrillControllerFactory.java  |  398 ++++++
 .../drill/yarn/appMaster/DrillbitScheduler.java |   51 +
 .../drill/yarn/appMaster/EventContext.java      |   70 +
 .../drill/yarn/appMaster/NodeInventory.java     |  195 +++
 .../yarn/appMaster/PersistentTaskScheduler.java |  174 +++
 .../apache/drill/yarn/appMaster/Pollable.java   |   28 +
 .../drill/yarn/appMaster/PulseRunnable.java     |   79 ++
 .../drill/yarn/appMaster/RegistryHandler.java   |   35 +
 .../apache/drill/yarn/appMaster/Scheduler.java  |  162 +++
 .../drill/yarn/appMaster/SchedulerState.java    |   79 ++
 .../yarn/appMaster/SchedulerStateActions.java   |  101 ++
 .../yarn/appMaster/SchedulerStateImpl.java      |  467 +++++++
 .../org/apache/drill/yarn/appMaster/Task.java   |  323 +++++
 .../yarn/appMaster/TaskLifecycleListener.java   |   26 +
 .../apache/drill/yarn/appMaster/TaskSpec.java   |   40 +
 .../apache/drill/yarn/appMaster/TaskState.java  |  895 ++++++++++++
 .../drill/yarn/appMaster/TaskVisitor.java       |   22 +
 .../yarn/appMaster/YarnFacadeException.java     |   30 +
 .../yarn/appMaster/http/AMSecurityManager.java  |   34 +
 .../appMaster/http/AMSecurityManagerImpl.java   |  221 +++
 .../yarn/appMaster/http/AbstractTasksModel.java |  380 ++++++
 .../drill/yarn/appMaster/http/AmRestApi.java    |  296 ++++
 .../yarn/appMaster/http/AuthDynamicFeature.java |  114 ++
 .../yarn/appMaster/http/ControllerModel.java    |  208 +++
 .../drill/yarn/appMaster/http/PageTree.java     |   80 ++
 .../drill/yarn/appMaster/http/WebServer.java    |  467 +++++++
 .../yarn/appMaster/http/WebUiPageTree.java      |  527 +++++++
 .../drill/yarn/appMaster/http/package-info.java |   22 +
 .../drill/yarn/appMaster/package-info.java      |   36 +
 .../org/apache/drill/yarn/client/AMRunner.java  |  368 +++++
 .../apache/drill/yarn/client/CleanCommand.java  |   89 ++
 .../apache/drill/yarn/client/ClientCommand.java |  100 ++
 .../apache/drill/yarn/client/ClientContext.java |   48 +
 .../drill/yarn/client/ClientException.java      |   34 +
 .../drill/yarn/client/CommandLineOptions.java   |  230 ++++
 .../apache/drill/yarn/client/DrillOnYarn.java   |  176 +++
 .../apache/drill/yarn/client/FileUploader.java  |  551 ++++++++
 .../apache/drill/yarn/client/HelpCommand.java   |   26 +
 .../apache/drill/yarn/client/KillCommand.java   |   48 +
 .../drill/yarn/client/PrintConfigCommand.java   |   49 +
 .../apache/drill/yarn/client/ResizeCommand.java |  115 ++
 .../drill/yarn/client/SimpleRestClient.java     |   66 +
 .../apache/drill/yarn/client/StartCommand.java  |  145 ++
 .../apache/drill/yarn/client/StatusCommand.java |  189 +++
 .../apache/drill/yarn/client/StopCommand.java   |  223 +++
 .../apache/drill/yarn/client/package-info.java  |   31 +
 .../org/apache/drill/yarn/core/AppSpec.java     |  169 +++
 .../org/apache/drill/yarn/core/ClusterDef.java  |  212 +++
 .../drill/yarn/core/ContainerRequestSpec.java   |  125 ++
 .../org/apache/drill/yarn/core/DfsFacade.java   |  345 +++++
 .../org/apache/drill/yarn/core/DoYUtil.java     |  189 +++
 .../drill/yarn/core/DoyConfigException.java     |   30 +
 .../drill/yarn/core/DrillOnYarnConfig.java      |  841 ++++++++++++
 .../org/apache/drill/yarn/core/LaunchSpec.java  |  248 ++++
 .../apache/drill/yarn/core/NameValuePair.java   |   46 +
 .../drill/yarn/core/YarnClientException.java    |   30 +
 .../apache/drill/yarn/core/YarnRMClient.java    |  207 +++
 .../apache/drill/yarn/core/package-info.java    |   24 +
 .../org/apache/drill/yarn/package-info.java     |   35 +
 .../org/apache/drill/yarn/zk/AMRegistry.java    |  145 ++
 .../drill/yarn/zk/ZKClusterCoordinator.java     |  318 +++++
 .../yarn/zk/ZKClusterCoordinatorDriver.java     |  315 +++++
 .../apache/drill/yarn/zk/ZKConfigException.java |   26 +
 .../org/apache/drill/yarn/zk/ZKRegistry.java    |  582 ++++++++
 .../drill/yarn/zk/ZKRuntimeException.java       |   30 +
 .../org/apache/drill/yarn/zk/package-info.java  |   31 +
 .../src/main/resources/drill-am/config.ftl      |   41 +
 .../src/main/resources/drill-am/confirm.ftl     |   70 +
 .../src/main/resources/drill-am/generic.ftl     |   78 ++
 .../src/main/resources/drill-am/history.ftl     |   59 +
 .../src/main/resources/drill-am/index.ftl       |  128 ++
 .../src/main/resources/drill-am/login.ftl       |   35 +
 .../src/main/resources/drill-am/manage.ftl      |   78 ++
 .../src/main/resources/drill-am/redirect.ftl    |   33 +
 .../main/resources/drill-am/shrink-warning.ftl  |   58 +
 .../resources/drill-am/static/css/drill-am.css  |   20 +
 .../drill-am/static/img/apache-drill-logo.png   |  Bin 0 -> 29802 bytes
 .../resources/drill-am/static/img/drill.ico     |  Bin 0 -> 580 bytes
 .../src/main/resources/drill-am/tasks.ftl       |  113 ++
 .../drill/yarn/core/drill-on-yarn-defaults.conf |  275 ++++
 .../apache/drill/yarn/client/TestClient.java    |  137 ++
 .../yarn/client/TestCommandLineOptions.java     |   84 ++
 .../org/apache/drill/yarn/core/TestConfig.java  |  267 ++++
 .../org/apache/drill/yarn/scripts/README.md     |   65 +
 .../apache/drill/yarn/scripts/ScriptUtils.java  |  847 ++++++++++++
 .../apache/drill/yarn/scripts/TestScripts.java  | 1288 ++++++++++++++++++
 .../drill/yarn/zk/TestAmRegistration.java       |  129 ++
 .../apache/drill/yarn/zk/TestZkRegistry.java    |  459 +++++++
 .../src/test/resources/doy-test-logback.xml     |   56 +
 .../src/test/resources/second-test-config.conf  |   34 +
 .../src/test/resources/test-doy-config.conf     |   32 +
 .../src/test/resources/test-doy-distrib.conf    |   30 +
 .../src/test/resources/third-test-config.conf   |   32 +
 drill-yarn/src/test/resources/wrapper.sh        |   53 +
 pom.xml                                         |  225 +++
 126 files changed, 21163 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/distribution/pom.xml
----------------------------------------------------------------------
diff --git a/distribution/pom.xml b/distribution/pom.xml
index d2f65e4..c374939 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -59,6 +59,11 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>drill-yarn</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.drill.exec</groupId>
       <artifactId>drill-jdbc</artifactId>
       <version>${project.version}</version>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/distribution/src/assemble/bin.xml
----------------------------------------------------------------------
diff --git a/distribution/src/assemble/bin.xml b/distribution/src/assemble/bin.xml
index bed34ed..7ca1140 100644
--- a/distribution/src/assemble/bin.xml
+++ b/distribution/src/assemble/bin.xml
@@ -110,6 +110,16 @@
       <useProjectArtifact>false</useProjectArtifact>
     </dependencySet>
 
+    <dependencySet>
+      <!-- Drill-on-YARN goes into its own directory so it does not end up
+           on the Drillbit class path. Define the jars/tools folder as any jar
+           we ship that is not used by drillbits. -->
+      <includes>
+        <include>org.apache.drill:drill-yarn:jar</include>
+      </includes>
+      <outputDirectory>jars/tools</outputDirectory>
+      <useProjectArtifact>false</useProjectArtifact>
+    </dependencySet>
 
 
     <dependencySet>
@@ -324,6 +334,21 @@
       <outputDirectory>bin</outputDirectory>
     </file>
     <file>
+      <source>src/resources/drill-on-yarn.sh</source>
+      <fileMode>0750</fileMode>
+      <outputDirectory>bin</outputDirectory>
+    </file>
+    <file>
+      <source>src/resources/drill-am.sh</source>
+      <fileMode>0750</fileMode>
+      <outputDirectory>bin</outputDirectory>
+    </file>
+    <file>
+      <source>src/resources/yarn-drillbit.sh</source>
+      <fileMode>0750</fileMode>
+      <outputDirectory>bin</outputDirectory>
+    </file>
+    <file>
       <source>src/resources/submit_plan</source>
       <fileMode>0755</fileMode>
       <outputDirectory>bin</outputDirectory>
@@ -337,6 +362,16 @@
       <outputDirectory>conf</outputDirectory>
     </file>
     <file>
+      <source>src/resources/yarn-client-log.xml</source>
+      <outputDirectory>conf</outputDirectory>
+      <fileMode>0640</fileMode>
+   </file>
+    <file>
+      <source>src/resources/drill-am-log.xml</source>
+      <outputDirectory>conf</outputDirectory>
+     <fileMode>0640</fileMode>
+    </file>
+    <file>
       <source>src/resources/drill-env.sh</source>
       <fileMode>0755</fileMode>
       <outputDirectory>conf</outputDirectory>
@@ -373,5 +408,10 @@
       <source>src/resources/saffron.properties</source>
       <outputDirectory>conf</outputDirectory>
     </file>
+    <file>
+      <source>src/resources/drill-on-yarn-example.conf</source>
+      <outputDirectory>conf</outputDirectory>
+      <fileMode>0640</fileMode>
+   </file>
   </files>
 </assembly>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/distribution/src/resources/drill-am-log.xml
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-am-log.xml b/distribution/src/resources/drill-am-log.xml
new file mode 100644
index 0000000..77fc37c
--- /dev/null
+++ b/distribution/src/resources/drill-am-log.xml
@@ -0,0 +1,54 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+<!--
+ Logging settings for the Drill-on-YARN application master.
+ See http://logback.qos.ch/manual/index.html for more information.
+
+ YARN already captures stdout and stderr, and the AM produces
+ moderate amount of logging, so the logging simply goes to stdout
+ and from there into the YARN-provided output log directory.
+-->
+<configuration>
+   
+  <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+    <encoder>
+      <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n
+      </pattern>
+    </encoder>
+  </appender>
+    
+  <logger name="org.apache.drill" additivity="false">
+    <level value="info" />
+    <appender-ref ref="STDOUT" />
+  </logger>
+
+  <!-- All Drill-on-YARN code is under the org.apache.drill.yarn package.
+       Level defaults to info, which provides an overview of AM activities.
+       Set to "error" if the information is not needed.
+  -->
+  <logger name="org.apache.drill.yarn" additivity="false">
+    <level value="info" />
+    <appender-ref ref="STDOUT" />
+  </logger>
+
+  <root>
+    <level value="error" />
+    <appender-ref ref="STDOUT" />
+  </root>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/distribution/src/resources/drill-am.sh
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-am.sh b/distribution/src/resources/drill-am.sh
new file mode 100644
index 0000000..3a8bdce
--- /dev/null
+++ b/distribution/src/resources/drill-am.sh
@@ -0,0 +1,137 @@
+#!/bin/bash
+#
+# 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.
+
+# Launch script for the Drill Application Master (AM).
+# This script runs under YARN and assumes the environment that YARN provides to an AM.
+# This script likely will not work from the command line.
+#
+# Environment variables set by the client:
+#
+# DRILL_DEBUG       Set to 1 to print environment and other information to
+#                   diagnose problems.
+# DRILL_AM_HEAP     AM heap memory. (The AM uses no direct memory.)
+# DRILL_AM_JAVA_OPT Optional additional JVM options for the AM, such as
+#                   options to enable debugging.
+#
+# The following environment variables are set in the AM launch context,
+# not used by this script, but used the the AM itself.
+#
+# DRILL_AM_APP_ID   Informs the AM of its YARN application ID.
+#                   (Strangely, YARN provides no way for an AM to learn this
+#                   from YARN itself.)
+# YARN_RM_WEBAPP    Informs the AM of the URL to the YARN RM web app.
+#                   Again, YARN informs the Client of this information, but
+#                   not the AM.
+# DRILL_ARCHIVE     The DFS path to the Drill archive used to localize Drillbit
+#                   code.
+# SITE_ARCHIVE      The DFS path to the optional site archive used to localize
+#                   Drillbit configuration.
+#
+# Further, this script infers DRILL_HOME from the location
+# of the script itself. The site directory (if used) is provided
+# via the --config command-line option.
+
+# YARN requires that the AM run as a child process until completion; so this script
+# does not launch the AM in the background.
+
+# This script is run from $DRILL_HOME/bin, wherever the user has configured it.
+
+bin=`dirname "${BASH_SOURCE-$0}"`
+bin=`cd "$bin">/dev/null; pwd`
+DRILL_HOME=`cd "$bin/..">/dev/null; pwd`
+
+if [ -n "$DRILL_DEBUG" ]; then
+  echo
+  echo "Drill AM launch script"
+  echo "DRILL_HOME: $DRILL_HOME"
+fi
+
+# AM-specific options for drill-config.sh. The AM
+# code is in the tools folder which is not loaded by
+# the Drillbit, only by the AM and client.
+#
+# Add the Hadoop config directory which we need to gain access to
+# YARN and HDFS. This is an odd location to add the config dir,
+# but if we add it sooner, Jersey complains with many class not
+# found errors for reasons not yet known. Note that, to add the
+# Hadoop config here, the Drill 1.6 $DRILL_HOME/conf/core-site.xml file
+# MUST have been removed or renamed else Hadoop will pick up
+# our dummy file instead of the real Hadoop file.
+
+DRILL_TOOL_CP="$DRILL_HOME/jars/tools/*:$HADOOP_CONF_DIR"
+
+# Use Drill's standard configuration, including drill-env.sh.
+# The AM discards most of the information, but does use JAVA
+# and a few others.
+
+. "$DRILL_HOME/bin/drill-config.sh"
+
+# DRILL_AM_HEAP and DRILL_AM_JAVA_OPTS are set by the
+# Drill client via YARN. To set these, use the following
+# configuration options:
+#
+# DRILL_AM_HEAP: drill.yarn.am.heap
+# DRILL_AM_JAVA_OPTS: drill.yarn.am.vm-args
+
+DRILL_AM_HEAP="${DRILL_AM_HEAP:-512M}"
+
+# AM logging setup. Note: the Drillbit log file uses the default name
+# of logback.xml.
+# The AM uses a non-default log configuration file name.
+# So, we must tell the AM to use an AM-specific file
+# else we'll get warnings about the log.query.path system property
+# not being set (and we won't pick up the AM logging settings.)
+# See http://logback.qos.ch/manual/configuration.html
+# The name provided must be on the class path. By adding
+# the site dir before $DRILL_HOME/conf, the user can
+# provide a custom config without editing the default one.
+# If this is wrong, you will see files such as
+# log.path_IS_UNDEFINED in the launch directory.
+
+AM_LOG_CONF="-Dlogback.configurationFile=drill-am-log.xml"
+#SITE_OPT="-Ddrill.yarn.siteDir=$DRILL_CONF_DIR"
+
+AM_JAVA_OPTS="-Xms$DRILL_AM_HEAP -Xmx$DRILL_AM_HEAP -XX:MaxPermSize=512M"
+AM_JAVA_OPTS="$AM_JAVA_OPTS $SITE_OPT $DRILL_AM_JAVA_OPTS $AM_LOG_CONF"
+if [ -n "$DRILL_JAVA_LIB_PATH" ]; then
+  AM_JAVA_OPTS="$AM_JAVA_OPTS -Djava.library.path=$DRILL_JAVA_LIB_PATH"
+fi
+
+# drill-config.sh built the class path.
+# Note that the class path uses the Hadoop, YARN and DFS jars
+# packaged with Drill; not those from the YARN-provided
+# environment variables in the launch context.
+
+AMCMD="$JAVA $AM_JAVA_OPTS ${args[@]} -cp $CP org.apache.drill.yarn.appMaster.DrillApplicationMaster"
+
+if [ -n "$DRILL_DEBUG" ]; then
+  echo "AM launch environment:"
+  echo "-----------------------------------"
+  env
+  echo "-----------------------------------"
+  echo "Command:"
+  echo "$AMCMD"
+fi
+
+# Note: no need to capture output, YARN does that for us.
+# AM is launched as a child process of caller, replacing this script.
+
+# Replace this script process with the AM. Needed so that
+# the YARN node manager can kill the AM if necessary by killing
+# the PID for this script.
+
+exec $AMCMD

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/distribution/src/resources/drill-on-yarn-example.conf
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-on-yarn-example.conf b/distribution/src/resources/drill-on-yarn-example.conf
new file mode 100644
index 0000000..1076676
--- /dev/null
+++ b/distribution/src/resources/drill-on-yarn-example.conf
@@ -0,0 +1,204 @@
+# 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.
+# ---------------------------------------------------------------------------
+# Configuration for the Drill-on-YARN feature. Provides information needed to
+# launch the Drill Application Master under YARN, and to configure the
+# Drill cluster.
+#
+# This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md
+# for more information.
+#
+# To create your own configuration file, copy only those lines that you want
+# to change. All values have reasonable defaults.
+
+drill.yarn: {
+
+  # Name displayed in the YARN management UI for your Drill cluster.
+
+  app-name: "Drill-on-YARN"
+
+  dfs: {
+    # Connection to the distributed file system. Defaults to work with
+    # a single-node Drill on the local machine.
+    # Omit this if you want to get the configuration either from the
+    # Hadoop config (set with config-dir above) or from the
+    # $DRILL_HOME/core-site.xml.
+
+    connection: "hdfs://localhost/"
+
+    # The distributed file system (DFS such as HDFS) directory in which to store the Drill
+    # archive file. Change this if you launch multiple Drill clusters to give
+    # each cluster it's own upload location.
+
+    app-dir: "/users/drill"
+  }
+
+  yarn: {
+
+    # Specify the queue you want to use when launching Drill-on-YARN.
+
+    queue: "default"
+  }
+
+  drill-install: {
+
+    # The archive (in .tar.gz format) that contains your Drill software, your custom jar files,
+    # and your Drill configuration. This is the path the file on your client machine.
+    # The base name is used when uploading the file to DFS.
+
+    client-path: "/etc/drill/drill.tar.gz"
+
+    # Set this if the name of the directory inside your archive file differs from
+    # the name of the file. That is, by default, my-drill.tar.gz should contain
+    # my-drill as its directory name.
+
+    # dir-name: "drill"
+
+    # Note that there are no parameters for the site directory. You specify the
+    # site directory location on client launch; Drill-on-YARN automatically
+    # creates an archive of that directory.
+
+    # Set the Java java.library.path option to files that pre-exist on
+    # each Drillbit node. (This is not for libraries that are distributed
+    # by YARN.) (Place Drill-specific libraries in $DRILL_SITE/lib.)
+
+    library-path: "/opt/libs"
+  }
+
+  # Application Master settings. You should not have to change these.
+
+  am: {
+
+    # Heap memory for the AM. Change this only if you run a large cluster and
+    # encounter memory errors.
+
+    heap: "450M"
+
+    # Amount of memory to allocate for the YARN container. Adjust this only if
+    # you adjust the heap amount above.
+
+    memory-mb: 512
+
+    # Optional label to apply to the AM request. Restricts the AM to run on
+    # only nodes with the label. Leave blank unless you need to restrict AM
+    # location.
+
+    node-label-expr: "drill-am"
+  }
+
+  # Settings for the Application Master's internal web server.
+
+  http: {
+
+    # Listen port. Change this if you run multiple Drill clusters as
+    # YARN may allocate two or more AMs on the same YARN worker node.
+
+    port: 8048
+
+    # Turn this on to enable HTTPS for the AM web UI.
+
+    ssl-enabled: true
+
+    # The type of authorization to use for the AM web UI. One of
+    # "none" (no authorization, the default), "drill" (use Drill's
+    # configured authentication system), or "simple" (use the user name
+    # and password defined here.) With Drill authentication enabled, only
+    # the user that launched Drill-on-YARN can access the Drill-on-YARN
+    # web UI. With simple authentication, then the defined user name and
+    # password is the only valid user.
+
+    auth-type: "drill"
+
+    # Set only for the simple auth type.
+
+    user_name: "fred"
+    password: "wilma"
+
+    # Set this to some value to protect the management REST calls.
+    # When the key is set, each rest call must include the key=value
+    # parameter. This is not high security, just a speed-bump to prevent
+    # accidental cluster changes.
+
+    rest-key=""
+  }
+
+  # Configuration for each Drillbit. Drill requires that all Drillbits use the
+  # same configuration.
+
+  drillbit: {
+
+    # The amount of heap memory to allocate to Drill. This is the equivalent of the
+    # DRILL_HEAP environment variable in drill-env.sh when launching Drill outside
+    # of YARN.
+
+    heap: "4G"
+
+    # The maximum amount of direct memory to allocate to Drill. This is the equivalent of the
+    # DRILL_MAX_DIRECT_MEMORY environment variable in drill-env.sh when launching Drill outside
+    # of YARN.
+
+    max-direct-memory: "8G"
+
+    # The amount of memory to alocate to the JVM code cache which holds the classes
+    # that Drill creates dynamically for each query execution. The default setting is
+    # fine for most applications.
+
+    code-cache: "1G"
+
+    # The amount of memory to allocate to each Drill-bit. This is the amount of memory requested
+    # from YARN for each drill-bit. This must be sufficient for
+    # heap + max-direct-memory + code-cache + JVM overhead (assume 1G)
+
+    memory-mb: 14336
+
+    # The number of YARN virtual cores to allocate to each drill-bit.
+
+    vcores: 4
+
+    # The number of disk channels (or spindles) that Drill typically uses.
+    # This value is supported only only selected YARN distributions.
+
+    disks: 3
+
+    # Add to the class path. Use for things such as HBase, etc. which must be installed
+    # on every node. (Drill automatically adds its own class path as well as the one
+    # provided by YARN.)
+
+    classpath: ""
+  }
+
+  # The groups of hosts on which to launch Drill. Each group can be one of several types.
+  #
+  # Note: This version supports a single group. The example below has multiple groups
+  # only to illustrate the various group types.
+
+  cluster: [
+    {
+      # Arbitrary name, just used for error and statistics reporting.
+      # Defaults to "group-" group index, starting at 1.
+
+      name: "group1"
+
+      # Basic group: let YARN pick any available node.
+
+      type: "basic"
+
+      # Number of Drill-bits to run. (Drill-on-YARN runs each on a separate host,
+      # so this is also the number of hosts that will run Drill.)
+
+      count: 2
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/distribution/src/resources/drill-on-yarn.sh
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-on-yarn.sh b/distribution/src/resources/drill-on-yarn.sh
new file mode 100644
index 0000000..a84b0a8
--- /dev/null
+++ b/distribution/src/resources/drill-on-yarn.sh
@@ -0,0 +1,74 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Run the Drill-on-YARN client to launch Drill on a YARN cluster.
+# Uses the $DRILL_HOME/conf/drill-on-yarn.conf file for client options.
+# Uses the $DRILL_HOME/conf/drill-cluster.conf file for cluster options.
+#
+# The config files (along with the Drill config files) MUST be in the
+# $DRILL_HOME directory so that they are properly localized. Drill-on-YARN does not permit
+# placing configuration files outside the $DRILL_HOME directory.
+#
+# Requires the location of Hadoop home. Maybe passed using the --hadoop option,
+# set in the environment, or set in $DRILL_HOME/conf/yarn-env.sh.
+
+usage="Usage: drill-on-yarn.sh start|stop|status"
+
+bin=`dirname "${BASH_SOURCE-$0}"`
+bin=`cd "$bin">/dev/null; pwd`
+export DRILL_HOME=`cd "$bin/..">/dev/null; pwd`
+
+# Use Drillbit's config script. We throw away most of the information, we really just
+# need JAVA_HOME and HADOOP_CONF_DIR or HADOOP_HOME.
+
+DRILL_TOOL_CP="$DRILL_HOME/jars/tools/*"
+. "$DRILL_HOME/bin/drill-config.sh"
+
+# Hadoop config or home is required
+if [ -z "$HADOOP_CONF_DIR" ]; then
+  if [ -z "$HADOOP_HOME" ]; then
+    echo "Hadoop home undefined: set HADOOP_CONF_DIR, HADOOP_HOME" >&2
+    exit 1
+  fi
+  HADOOP_CONF_DIR="$HADOOP_HOME/etc/hadoop"
+fi
+
+DRILL_CLIENT_HEAP=${DRILL_CLIENT_HEAP:-512M}
+VM_OPTS="-Xms$DRILL_CLIENT_HEAP -Xmx$DRILL_CLIENT_HEAP $DRILL_CLIENT_VM_OPTS"
+VM_OPTS="$VM_OPTS -Dlogback.configurationFile=yarn-client-log.xml"
+#VM_OPTS="$VM_OPTS -Ddrill.yarn.siteDir=$DRILL_CONF_DIR"
+
+# Add Hadoop configuration at the end of the class path. This will
+# fail if the 1.6-and earlier core-site.xml file resides in the conf
+# directory.
+
+CP="$CP:$HADOOP_CONF_DIR"
+
+if [ ${#args[@]} = 0 ]; then
+  echo $usage
+  exit 1
+fi
+
+CLIENT_CMD="$JAVA $VM_OPTS -cp $CP org.apache.drill.yarn.client.DrillOnYarn ${args[@]}"
+
+case ${args[0]} in
+debug)
+  env
+  echo "Command: $CLIENT_CMD"
+  ;;
+*)
+  exec $CLIENT_CMD
+  ;;
+esac

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/distribution/src/resources/yarn-client-log.xml
----------------------------------------------------------------------
diff --git a/distribution/src/resources/yarn-client-log.xml b/distribution/src/resources/yarn-client-log.xml
new file mode 100644
index 0000000..3f84726
--- /dev/null
+++ b/distribution/src/resources/yarn-client-log.xml
@@ -0,0 +1,44 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+<!--
+ Logging settings for the Drill-on-YARN client application. Not that the
+ client does not do much logging, it relies on the -v (verbose) option
+ instead.
+ See http://logback.qos.ch/manual/index.html for more information.
+-->
+<configuration>
+   
+  <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+    <encoder>
+      <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n
+      </pattern>
+    </encoder>
+  </appender>
+
+  <!-- All Drill-on-YARN code is under the org.apache.drill.yarn package. -->
+  <logger name="org.apache.drill.yarn" additivity="false">
+    <level value="error" />
+    <appender-ref ref="STDOUT" />
+  </logger>
+
+  <root>
+    <level value="error" />
+    <appender-ref ref="STDOUT" />
+  </root>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/distribution/src/resources/yarn-drillbit.sh
----------------------------------------------------------------------
diff --git a/distribution/src/resources/yarn-drillbit.sh b/distribution/src/resources/yarn-drillbit.sh
new file mode 100644
index 0000000..72b7f6e
--- /dev/null
+++ b/distribution/src/resources/yarn-drillbit.sh
@@ -0,0 +1,178 @@
+# 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.
+#
+# Internal script to launch a Drillbit under YARN. Not for use outside
+# of YARN.
+
+# This is a heavily modified version of drillbit.sh, drill-config.sh and
+# runbit, modified for use in YARN and performing a single operation:
+# launching a Drillbit and waiting for Drillbit exit.
+#
+# Under YARN, the typical way to launch a Java app is to do all the
+# setup in Java code in the launch context. However, Drill depends on
+# drill-env.sh to set site-specific options before launch. This script
+# performs Drill launch, integrating options from drill-env.sh.
+#
+# Input environment variables:
+#
+# DRILL_DEBUG
+#     Used to debug this script. Dumps debugging information.
+#     Set from the drill.yarn.drillbit.debug-launch config parameter.
+# DRILL_HOME
+#     Identifies the Drill software to use: either at a fixed
+#     location, or localized. Set from the drill.yarn.drill-install
+#     localize and drill-home config parameters. Note that this
+#     variable tells this script where to find the drill-env.sh
+#     file (in $DRILL_HOME/conf), and so DRILL_HOME cannot be
+#     overridden in drill-env.sh.
+# DRILL_LOG_DIR
+#     The location to which to write log files. Often set in drill-env.sh.
+#     Non-YARN launch provides a number of default locations. This variable
+#     is ignored in a YARN launch if the drill.yarn.drillbit.yarn-logs is
+#     true (the default.) Set the config variable to false to use this
+#     value for the long directory under YARN.
+# DRILL_YARN_LOG_DIR
+#     If using the YARN log directory, this variable points to that location.
+#     If drill.yarn.drillbit.disable-yarn-logs is true, then this variable is not
+#     set and the DRILL_LOG_DIR is used instead.
+# DRILL_MAX_DIRECT_MEMORY:
+#     The amount of direct memory set in the
+#     drill.yarn.drillbit.max-direct-memory config parameter.
+#     When Drill is run outside of YARN, this value is set in drill-env.sh.
+#     But doing so under YARN decouples the Drill memory settings from the
+#     YARN settings. If you do not set the config parameter, Drill will
+#     default to the value set in drill-env.sh.
+# DRILL_HEAP
+#     The amount of Java heap memory set in the
+#     drill.yarn.drillbit.heap setting. Same override rules as
+#     DRILL_MAX_DIRECT_MEMORY.
+# DRILL_JAVA_OPTS
+#     The standard JVM options needed to launch Drill. Must be set in
+#     drill-env.sh.
+# DRILL_JVM_OPTS
+#     Additional YARN-specific JVM options set in the
+#     drill.yarn.drillbit.vm-args config parameter. Note that the YARN-specific
+#     options are in addition to (not an override of) the DRILL_JAVA_OPTS
+#     values.
+# SERVER_GC_OPTS
+#     Garbage collection (GC) related JVM options set in drill-env.sh. Not
+#     overridden in YARN.
+# HADOOP_HOME
+#     Location of the Hadoop software and configuration. Can be
+#     set with the drill.yarn.hadoop.home or in drill-env.sh. If both are set, the
+#     latter value overrides the former.
+# JAVA_HOME
+#     Set by YARN, bypassing Drill's usual process for searching for JAVA_HOME.
+# DRILL_CONF_DIR
+#     Location of Drill's configuration files. Non-YARN launch provides a set of
+#     defaults. Under YARN, this value must either be set explicitly using the
+#     drill.yarn.drillbit.conf-dir parameter, or will default to
+#     the (typically localized) $DRILL_HOME/conf directory.
+# DRILL_CLASSPATH_PREFIX
+#     Optional extra classpath added before Drill's own jars. Set from the
+#     drill.yarn.drillbit.prefix-class-path config parameter, or in
+#     drill-env.sh. As with all class path settings, if both values are set,
+#     drill-env.sh takes precedence.
+# EXTN_CLASSPATH
+#     Optional extra classpath added after Drill's own jars but before
+#     3rd party jars. Allows overriding Drill's bundled version of Hadoop
+#     and so on. Allows adding jars needed by plug-ins. Config parameter
+#     is drill.yarn.drillbit.extn-class-path.
+# HADOOP_CLASSPATH
+#     As above, but for the Hadoop class path. Config parameter is
+#     drill.yarn.drillbit.hadoop-class-path. This is a legacy setting. Use
+#     drill.yarn.drillbit.extn-class-path for new deployments.
+# HBASE_CLASSPATH
+#     As above, but for the Hbase class path. Config parameter is
+#     drill.yarn.drillbit.hbase-class-path. This is a legacy setting. Use
+#     drill.yarn.drillbit.extn-class-path for new deployments.
+# DRILL_CLASSPATH
+#     Optional extra classpath after all Drill-provided jars. This is the
+#     typical place to add jars needed by plugins, etc. (Note, no need to set
+#     this if the jars reside in the $DRILL_HOME/jars/3rdparty directory.)
+#     Config parameter is drill.yarn.drillbit.drill-classpath.
+# DRILL_JVM_OPTS
+#     Additional JVM options passed via YARN from the
+#     drill.yarn.drillbit.vm-args parameter.
+# ENABLE_GC_LOG
+#     Enables Java GC logging. Passed from the drill.yarn.drillbit.log-gc
+#     garbage collection option.
+
+if [ -n "$DRILL_DEBUG" ]; then
+  echo
+  echo "Drillbit Environment from YARN:"
+  echo "-----------------------------------"
+  env
+  echo "-----------------------------------"
+fi
+
+# DRILL_HOME is set by the AM to point to the Drill distribution.
+
+# In YARN, configuration defaults to the the standard location.
+
+DRILL_CONF_DIR=${DRILL_CONF_DIR:-$DRILL_HOME/conf}
+
+# On some distributions, YARN sets HADOOP_CLASSPATH, but for a different
+# purpose than the one to which Drill puts that variable. Unset the YARN
+# version and only use the variable if drill-env.sh sets it.
+
+unset HADOOP_CLASSPATH
+unset HBASE_CLASSPATH
+
+# Replace the values with the values from the Drill-on-YARN config
+# file, if given.
+
+if [ -n "$DRILL_HADOOP_CLASSPATH" ]; then
+  export HADOOP_CLASSPATH="$DRILL_HADOOP_CLASSPATH"
+fi
+if [ -n "$DRILL_HBASE_CLASSPATH" ]; then
+  export HBASE_CLASSPATH="$DRILL_HBASE_CLASSPATH"
+fi
+
+# HADOOP_HOME can be set by Drill-on-YARN, or in drill-env.sh. The
+# latter takes precedence. It seems that YARN does not use the
+# HADOOP_HOME variable; instead it provides HADOOP_COMMON_HOME,
+# HADOOP_HDFS_HOME, and HADOOP_YARN_HOME.
+#
+# Default HADOOP_COMMON_HOME if HADOOP_HOME is not otherwise set in a YARN
+# environment.
+
+if [ -n "$HADOOP_HOME" ]; then
+  export HADOOP_HOME="$HADOOP_COMMON_HOME"
+fi
+
+# Use Drill's standard configuration, including drill-env.sh.
+
+. "$DRILL_HOME/bin/drill-config.sh"
+
+# Debugging information
+
+if [ -n "$DRILL_DEBUG" ]; then
+  echo "Command:"
+  "$DRILL_HOME/bin/runbit" debug
+  echo
+  echo "Local Environment:"
+  echo "-----------------------------------"
+  set
+  echo "-----------------------------------"
+fi
+
+# Launch Drill itself.
+# Passes along Drill's exit code as our own.
+
+echo "`date` Starting drillbit on `hostname` under YARN, logging to $DRILLBIT_LOG_PATH"
+echo "`ulimit -a`" >> "$DRILLBIT_LOG_PATH" 2>&1
+
+"$DRILL_HOME/bin/runbit" exec

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/README.md
----------------------------------------------------------------------
diff --git a/drill-yarn/README.md b/drill-yarn/README.md
new file mode 100644
index 0000000..1794afb
--- /dev/null
+++ b/drill-yarn/README.md
@@ -0,0 +1,190 @@
+# Drill-on-YARN: YARN Integration for Apache Drill
+
+Drill-on-YARN (DoY) runs Apache Drill as a long-running process under Apache YARN. Key components
+include:
+
+1. The Drill-on-YARN client: starts, stops, resizes and checks the status of your Drill cluster.
+2. Drill-on-YARN Application Master (AM): a long-running process under YARN that uses YARN
+to manage your Drill cluster.
+3. Drillbits: The Drill server process, now started by YARN rather than by hand or other
+ad-hoc scripts.
+4. Configuration: Cluster configuration now resides in a drill-on-yarn.conf.
+
+Extensive user documentation is to be available on the Apache Drill site. Until then,
+a user guide is attached to [DRILL-1170](https://issues.apache.org/jira/browse/DRILL-1170).
+
+## Building
+
+Drill-on-YARN builds as part of Apache Drill. The build produces a new DoY jar
+which resides in a new `$DRILL_HOME/jars/tools` folder so that the DoY classes are not 
+visible to Drill itself.
+
+## Understanding the Code
+
+The DoY code divides into three main modules:
+
+1. The DoY command-line client application.
+2. The DoY AM server application.
+3. Scripts around the client, AM and Drillbit applications.
+
+Scripts reside in the distribution project under `src/resources`.
+
+All DoY code resides in this module in the `org.apache.drill.yarn` package.
+
+- `client`: The command-line client application.
+- `appMaster`: The DoY AM.
+- `core`: Code shared between the client and AM.
+- `zk`: Integration with ZooKeeper to monitor Drillbits.
+
+DoY depends on Drill modules, but only the `distribution` project depends on
+DoY.
+
+Because DoY is a YARN application, we recommend that you become familiar with
+YARN itself before diving into the DoY code. YARN has a very particular model
+for how to run distributed applications and that model drove the design of
+the DoY modules and classes.
+
+### Major Components
+
+The following diagram shows the major DoY components and how they relate to
+the YARN components:
+
+![System Overview](./img/overview.png)
+
+The client communicates with the YARN Resource Manager (RM) to request the start
+of the DoY AM. The RM locates a node to run the AM's container and asks the
+Node Manager (NM) on that node to start the AM. The AM starts and registers
+itself with ZooKeeper to prevent multiple AMs for the same Drill cluster.
+
+The AM then requests containers from the RM in which to run Drillbits. Next, the
+AM asks the assigned NMs to start each Drillbit. The Drillbit starts and 
+registers itself with ZooKeeper (ZK). The AM monitors
+ZK to confirm that the Drillbit did, in fact, start.
+
+To shut down, the client contacts the AM directly using the AM REST API
+and requests shutdown.
+The AM sends a kill request to each NM, which kills the Drillbit processes.
+The AM monitors ZK to confirm that the Drillbit has dropped its registration.
+Once the last Drillbit has completed, the AM itself exits. The client will
+wait (up to a limit) for the AM to shut down so that the client can report
+as successful shutdown.
+
+### Client
+
+The client application consists of a main class, `DrillOnYarn` and a set of
+command classes. Each command performs one operation, such as start, stop,
+resize, and so on. The client is designed to start, perform one operation,
+and exit. That is, while the AM is a persistent process, the client is not.
+
+A user will start their Drill cluster, then later will want to stop it. The
+Drill cluster is a YARN application, represented by YARN with
+an "application id" (app id). To stop a Drill cluster, the client needs the
+app id assigned to the application at start time. While the user can use the
+`-a` option to provide the app id explicitly, it is more convenient for
+the client to "remember" the
+app id. DoY uses an "app id file" for this purpose. This convenience works
+if the user starts, manages and stops the cluster from a single host.
+
+The following diagram shows the major classes in the DoY client:
+
+![Client Overview](./img/client-classes.png)
+
+
+The client uses a "facade" to communicate with YARN. The facade,
+`YarnRMClient`, interfaces to YARN to perform the required YARN operations.
+Similarly, another facade, `DfsFacade`, provides a layer on top of the HDFS
+API. The facades simplify code and provide an abstraction handy for mocking
+these systems during unit testing.
+
+YARN simplifies the task of running Drill (or any other application) by
+"localizing" the required files onto each worker node. The localization process
+starts with the client uploading the files to the distributed file system (DFS),
+typically HDFS. DoY localizes two separate files. The first is the Drill software
+itself, typically using the original Drill archive from Apache or your distribution.
+Drill requires site-specific configuration, optionally including custom code
+for user-defined functions (UDFs), etc. Site files reside in a Drill 
+site directory. For YARN, the site
+directory must be outside of the drill software distribution (see the user
+documentation for details.) DoY archives the site directory and uploads it to
+DFS along with the Drill archive. The code that does that work resides in the
+`FileUploader` class.
+
+To start a Drill cluster, the client asks YARN to launch the AM by specifying
+a large number of detailed options: environment variables, files, command
+line to run, and so on. This work is done in the `AMRunner` class.
+
+## Application Master
+
+The AM must perform several tasks, including:
+
+* Maintain the desired number of Drillbits.
+* Work with YARN to request a container for each Drillbit, and to launch
+the Drillbit.
+* Ensure that YARN allocates only one Drillbit container per cluster host.
+(Required because all Drillbits within a cluster share the same set of ports.)
+* Monitor Zookeeper to watch Drillbits. Drillbits perform a heartbeat with
+ZK, which the AM can monitor. The AM will restart any Drillbit that drops out
+of ZK, since such a Drillbit is likely in a bad state.
+* Provide a Web UI to monitor and manage the cluster.
+* Provide a REST API that the client uses to communicate directly with the AM.
+
+The AM is composed of a number of components. The following diagram shows the
+major classses involved in setting up the AM:
+
+![AM Overview](./img/am-overview.png)
+
+he `DrillApplicationMaster` class is the main AM program. It has to key
+tasks: 1) create the `DrillControllerFactory` that assembles the required
+parts of the AM, and 2) runs the `Dispatcher`, which is the actual AM server.
+
+The AM is designed to be generic; Drill-specific bits are abstracted out into
+helpers. This design simplifies testing and also anticipates that Drill may
+eventually include other, specialized, servers. The `DrillControllerFactory`
+is the class that pulls together all the Drill-specific pieces to assemble
+the server. During testing, different factories are used to assemble a test
+server.
+
+The `Dispatcher` receives events from YARN, from the REST API and from a timer
+and routes them to the `ClusterController` which takes actions based on the
+events. This structure separates the API aspects of working with YARN (in the
+`Dispatcher`) from the logic of running a cluster (in the `ClusterController`.)
+
+The `ClusterController` attempts to keep the cluster in the desired state. Today
+this means running a specified number of Drillbits. In the future, DoY may
+support multiple Drillbit groups (one set that runs all the time, say, and another
+that runs only during the day when needed for interactive users.)
+
+A large amount of detailed fiddling is needed to propertly request a container
+for a Drillbit, launch the Drillbit, monitor it and shut it down. The `Task`
+class monitors the lifecycle of each task (here, a Drillbit). Behavior of the
+task differs depending on the task's state. The `TaskState` class, and its
+subclasses, provide the task-specific behavior. For example, handling of a
+task cancellation is different depending on whether the task is in the
+`RequestingState` or in the `RunningState`.
+
+The following diagram illustrates some of the details of the cluster controller
+system.
+
+![Controller Detail](./img/controller-classes.png)
+
+Some events are time based. For example, a Drillbit is given a certain amount
+of time to register itself in ZK before DoY assumes that the Drillbit is
+unhealthy and is restarted. The `PulseRunnable` is the thread that implements
+the timer; `Pollable` is the listener for each "tick" event.
+
+The `Scheduler` and its subclasses (such as `DrillbitScheduler`) maintain the
+desired number of Drillbits, asking the `ClusterController` to start and stop
+tasks as needed. The `Scheduler` also handles task-specific tasks. At present,
+Drill has no means to perform a graceful shutdown. However, when Drill does,
+the `DrillbitScheduler` will be responsible for sending the required message.
+
+The `appMaster.http` package contains the implementation for the web UI and
+REST API using an embedded Jetty server. If Drill security is enabled, the
+web UI will prompt the user to log in. The only recognized user is the one
+that launched DoY.
+
+The `NodeRegistry` tracks the set of nodes running Drillbits so we can avoid
+starting a second on any of them. Drillbits are started though YARN, of course,
+but can also be "stray": Drillbits started outside of DoY and discovered
+though ZK. Even stray Drillbits are registered to avoid nasty surprises if
+DoY where to try to launch a Drillbit on that same node.


[06/12] drill git commit: DRILL-1170: YARN integration for Drill

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AMSecurityManagerImpl.java
----------------------------------------------------------------------
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 0000000..d31690e
--- /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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AbstractTasksModel.java
----------------------------------------------------------------------
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 0000000..bc4e8d6
--- /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 ) );
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AmRestApi.java
----------------------------------------------------------------------
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 0000000..21ddc4b
--- /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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/AuthDynamicFeature.java
----------------------------------------------------------------------
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 0000000..55cd59a
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/ControllerModel.java
----------------------------------------------------------------------
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 0000000..8947df5
--- /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;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/PageTree.java
----------------------------------------------------------------------
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 0000000..e4d5dc1
--- /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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/WebServer.java
----------------------------------------------------------------------
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 0000000..aeeafde
--- /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();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/WebUiPageTree.java
----------------------------------------------------------------------
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 0000000..fc44e45
--- /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);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/http/package-info.java
----------------------------------------------------------------------
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 0000000..13f1bd8
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/package-info.java
----------------------------------------------------------------------
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 0000000..0ff835d
--- /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


[09/12] drill git commit: DRILL-1170: YARN integration for Drill

Posted by ar...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/USAGE.md
----------------------------------------------------------------------
diff --git a/drill-yarn/USAGE.md b/drill-yarn/USAGE.md
new file mode 100644
index 0000000..db7fb4b
--- /dev/null
+++ b/drill-yarn/USAGE.md
@@ -0,0 +1,941 @@
+# Drill-on-YARN User Guide
+
+## Introduction
+
+Drill's YARN integration launches your Drill cluster through YARN. Drill becomes a long-running application with YARN. When you launch Drill, YARN automatically deploys (“localizes”) Drill software onto each node, avoiding the need to pre-install Drill on each node. Resource management is simplified because YARN is now aware of resources dedicated to Drill.
+
+The discussion below assumes that you are familiar with the main YARN concepts: the Resource Manager (RM), Node Manager (NM) and so on.
+
+The discussion also assumes that you already have a working Drill cluster that you wish to launch under YARN. Drill configuration is best tested by launching Drill directly; launch Drill under YARN when the configuration becomes stable.
+
+### YARN Resource Settings
+
+Drill, by design, aggressively uses all available resources to run queries at optimal speed. When running Drill under YARN, you inform YARN of the resources that Drill will consume. These settings are descriptive, not proscriptive. That is, Drill does not limit itself to the YARN settings; instead the YARN settings inform YARN of the resources that Drill will consume so that YARN does not over-allocate those same resources to other tasks.
+
+All YARN distributions provide settings for memory and CPU (called “vcores” by YARN). Some distributions also provide disk settings.
+
+For memory, you first configure Drill’s memory as described below, then you inform YARN of the Drill configuration.
+
+Drill will use all available disk I/Os. Drill will also use all available CPU. Consider enabling Linux cgroups to limit Drill's CPU usage to match the YARN vcores allocation.
+
+## Components
+
+Major components include:
+
+* Drill distribution archive: the original .tar.gz file for your Drill distribution. Drill-on-YARN uploads this archive to your distributed file system (DFS). YARN downloads it (localized it) to each worker node.
+* Drill site directory: a directory that contains your Drill configuration and custom jar files. Drill-on-YARN copies this directory to each worker node.
+Configuration: A configuration file which tells Drill-on-YARN how to manage your Drill cluster. This file is separate from your configuration files for Drill itself.
+* Drill-on-YARN client: a command-line program to start, stop and monitor your YARN-managed Drill cluster.
+* Drill Application Master (AM): The software that works with YARN to request resources, launch Drill-bits, and so on. The AM provides a web UI to manage your Drill cluster.
+* Drill-Bit: The Drill daemon software which YARN runs on each node.
+​
+###​ Overview
+
+The following are the key steps to launch Drill under YARN. Each step is explained in detail in this documentation.
+
+* Create a Drill site directory with your site-specific files.
+* Configure Drill-on-YARN using the the drill-on-yarn.conf configuration file.
+* Use the Drill-on-YARN client tool to launch your Drill cluster.
+* Use Drill-on-YARN client or web UI to monitor and shut down the Drill cluster.
+
+## Quick-Start Guide
+
+This section walks you through the steps needed to create a basic Drill cluster. Later sections discuss additional options and details needed in special situations.
+​
+### The Client Machine
+
+YARN works by launching an application using a “client” application. For Drill, this is the Drill-on-YARN client. The client can run on any machine that has both the Drill and Hadoop software. The client machine need not be part of your YARN cluster; any host from which you currently launch YARN jobs can be the client. All the steps that follow are done on the client machine. When running Drill outside of YARN, you must install Drill on every node of your cluster. With YARN, you need install Drill only on the client machine; Drill-on-YARN automatically deploys  (“localizes”) Drill to the worker nodes.
+
+When running Drill without YARN, many users place their configuration files and custom code within the Drill distribution directory. When running under YARN, all your configuration and custom code resides in the site directory; you should not change anything in the Drill install. (This allows Drill-on-YARN to upload your original Drill install archive without rebuilding it.)
+​
+### Create a Master Directory
+
+To localize Drill files, the client tool requires a copy of the original Drill distribution archive and the location of your site directory.  For ease of discussion, we assume all these components reside in a single “master directory” described as `$MASTER_DIR`. On the client machine, do the following to create the master directory:
+```
+export MASTER_DIR=/path/to/master/dir
+mkdir $MASTER_DIR
+cd $MASTER_DIR
+```
+The following is a summary of the steps needed to build your master directory. Each step is explained in detail in the following sections.
+
+* Download the Drill archive to `$MASTER_DRILL`.
+* Unpack the archive to create `$DRILL_HOME`.
+* Create the site directory with the required configuration files.
+
+### Install Drill
+
+These instructions assume you are installing Drill as part of the Drill-on-YARN setup. You can use your existing Drill 1.8 or later install as long as it meets the criteria spelled out here.
+
+Follow the Drill install directions to install Drill on your client host. The install steps are different for YARN than for the Embedded or Cluster install.
+
+1. Select a Drill version. The name is used in multiple places below. For convenience, define an environment variable for the name:
+```
+export DRILL_NAME=apache-drill-x.y.z
+```
+Replace x.y.z with the selected version.
+
+2. Download your Drill version.
+```
+wget \ http://apache.mesi.com.ar/drill/drill-x.y.z/$DRILL_NAME.tar.gz
+```
+or
+```
+curl -o $DRILL_NAME.tar.gz \ http://apache.mesi.com.ar/drill/drill-x.y.z/$DRILL_NAME.tar.gz
+```
+Again, replace x.y.z with the selected version.
+
+3. Expand the Drill distribution into this folder to create the master directory
+```
+tar -xzf $DRILL_NAME.tar.gz
+```
+4. For ease of following the remaining steps, call your expanded Drill folder `$DRILL_HOME`:
+```
+export DRILL_HOME=$MASTER_DIR/$DRILL_NAME
+```
+Your master directory should now contain the original Drill archive along with an expanded copy of that archive.
+​
+### Create the Site Directory
+
+The site directory contains your site-specific files for Drill. If you are converting an existing Drill install, see the “Site Directory” section later in this guide.
+
+1. Create the site directory within your master directory:
+```
+export DRILL_SITE=$MASTER_DIR/site
+mkdir $DRILL_SITE
+```
+When you install Drill fresh, Drill includes a conf directory under `$DRILL_HOME`. Use the files there to create your site directory.
+```
+cp $DRILL_HOME/conf/drill-override-example.conf $DRILL_SITE/drill-override.conf
+cp $DRILL_HOME/conf/drill-on-yarn-example.conf $DRILL_SITE/drill-on-yarn.conf
+cp $DRILL_HOME/conf/drill-env.sh $DRILL_SITE
+cp $DRILL_HOME/conf/distrib-env.sh $DRILL_SITE
+```
+Then edit the above configuration files as per the Drill install instructions, and the Drill-on-YARN instructions below. (Note that, under YARN, you set the Drill memory limits in `drill-on-yarn.sh` instead of `drill-env.sh`.)
+
+The instructions above have you copy the distribution-specific `distrib-env.sh` file to your site directory. This is done because the file often contains values set during Drill installation. When you upgrade Drill, be sure to replace the file with the latest version from `$DRILL_HOME/conf`.
+
+If you develop custom code (data sources or user-defined functions (UDFs)), place the Java jar files in `$DRILL_SITE/jars`.
+
+Your master directory should now contain the Drill software and your site directory with default files.
+
+You will use the site directory each time you start Drill by using the `--site` (or `--config`) option. The following are examples, don’t run these yet:
+```
+drillbit.sh --site $DRILL_SITE
+drill-on-yarn.sh --site $DRILL_SITE
+```
+Once you’ve created your site directory, upgrades are trivial. Simply delete the old Drill distribution and install the new one. Your files remain unchanged in the site directory.
+​
+### Configure Drill-on-YARN using Existing Settings
+
+The next step is to configure Drill. If you have not yet used Drill, then you should start with Drill in distributed mode to learn which configuration options you need. YARN is an awkward environment in which to learn Drill configuration. Here, we assume that you have already worked out the required configuration on a separate Drill install. Let's call that location `$PROD_DRILL_HOME`.
+
+From `$PROD_DRILL_HOME` copy the following to corresponding locations in `$DRILL_SITE`:
+```
+cp $PROD_DRILL_HOME/conf/drill-override.conf $DRILL_SITE
+cp $PROD_DRILL_HOME/conf/drill-env.sh $DRILL_SITE
+cp $PROD_DRILL_HOME/jars/3rdparty/yourJarName.jar $DRILL_SITE/jars
+```
+
+See the Preparation section above for changes you should make to your existing `drill-env.sh` file.
+
+### Create Your Cluster Configuration File
+
+The next step is to specify additional configuration which Drill-on-YARN requires to launch your Drill cluster.
+
+Start by editing `$DRILL_SITE/drill-on-yarn.conf` using your favorite editor. This file is in the same [HOCON](https://github.com/typesafehub/config/blob/master/HOCON.md) format used by `drill-override.conf`.
+
+Consult `$DRILL_HOME/conf/drill-on-yarn-example.conf` as an example. However, don’t just copy the example file; instead, copy only the specific configuration settings that you need; the others will automatically take the Drill-defined default values.
+
+The following sections discuss each configuration option that you must set.
+
+### Drill Resource Configuration
+
+The two key Drill memory parameters are Java heap size and Direct memory. In a non-YARN cluster, you set these in `$DRILL_HOME/conf/drill-env.sh` as follows, shown with the default values:
+```
+DRILL_MAX_DIRECT_MEMORY="8G"
+DRILL_HEAP="4G"
+```
+Drill-on-YARN uses a different mechanism to set these values. You set the values in `drill-on-yarn.conf`, then Drill-on-YARN copies the values into the environment variables when launching each Drill-bit.
+```
+  drillbit: {
+    heap: "4G"
+    max-direct-memory: "8G"
+  }
+```
+To create the Drill-on-YARN setup, simply copy the values directly from your pre-YARN `drill-env.sh` file into the above configuration. (Drill-on-YARN will copy the values back into the environment variables when launching Drill.)
+
+Next, you must determine the container size needed to run Drill under YARN. Typically this size is simply the sum of the heap and direct memory. However, if you are using custom libraries that perform their own memory allocation, or launch sub-processes, you must account for that memory usage as well. The YARN memory is expressed in MB. For example, for the default settings above, we need 12G of memory or 12288MB:
+```
+  drillbit: {
+    memory-mb: 12288
+  }
+```
+Finally, you must determine how much CPU to grant to Drill. Drill is a CPU-intensive operation and greatly benefits from each additional core. However, you can limit Drill’s CPU usage under YARN by specifying the number of YARN virtual cores (vcores) to allocate to Drill:
+```
+  drillbit: {
+    vcores: 4
+  }
+```
+Note that in the above, each configuration setting was shown separately. In your actual file, however, they appear within a single group as follows:
+```
+  drillbit: {
+    heap: "4G"
+    max-direct-memory: "8G"
+    memory-mb: 6144
+    vcores: 4
+  }
+```
+
+### Drillbit Cluster Configuration
+
+Drill-on-YARN uses the concept of a “cluster group” of drill-bits to describe the set of drill-bits to launch. A group can be one of three kinds:
+
+* Basic: launches drill-bits anywhere in the YARN cluster where a container is available.
+* Labeled: Uses YARN labels to identify the set of nodes that should run Drill.
+
+This section describes how to create a basic group suitable for testing. See later sections for the other two types.
+
+For a basic group, you need only specify the group type and the number of drill-bits to launch:
+```
+  cluster: [
+    {
+      name: "drillbits"
+      type: "basic"
+      count: 1
+    }
+  ]
+```
+The above syntax says that cluster is a list that contains a series of cluster group objects contained in braces. In this release, however, Drill supports just one cluster group.
+​
+### ZooKeeper Configuration
+
+Drill uses ZooKeeper to coordinate between Drillbits. When run under YARN, the Drill Application Master uses ZooKeeper to monitor Drillbit health. Drill-on-YARN reads your `$DRILL_SITE/drill-override.conf` file for ZK settings.
+
+### Configure the Drill Distribution Archive
+
+Next configure the name of the Drill distribution archive that you downloaded earlier.
+```
+  drill-install: {
+    client-path: "archive-path"
+   }
+```
+Where archive-path is the location of your archive. In our example, this is `$MASTER_DIR/apache-drill.x.y.z.tar.gz` Use the full name of the master directory, not the environment variable. (Substitute your actual version number for x.y.z.)
+
+### Select the Distributed File System Location
+
+Drill copies your archive onto your distributed file system (such as HDFS) in a location you provide. Set the DFS options as follows:
+```
+  dfs: {
+    connection: "hdfs://localhost/"
+    dir: "/user/drill"
+  }
+```
+Drill can read the connection information from your Hadoop configuration files (`$HADOOP_HOME/etc/hadoop/core-site.xml`). Or you can specify a connection directly in the Drill cluster configuration file using the connection attribute.
+
+Then, choose a DFS file system location. Drill uses “`/user/drill`” by default.
+
+### Hadoop Location
+
+Apache Drill users must tell Drill-on-YARN the location of your Hadoop install. Set the `HADOOP_HOME` environment variable in `$DRILL_SITE/drill-env.sh` to point to your Hadoop installation:
+```
+export HADOOP_HOME=/path/to/hadoop-home
+```
+This assumes that Hadoop configuration is in the default location: `$HADOOP_HOME/etc/hadoop`. If your configuration is elsewhere, set `HADOOP_CONF_DIR` instead:
+```
+export HADOOP_CONF_DIR=/path/to/hadoop-config
+```
+
+## Launch Drill under YARN
+
+Finally, use the client tool to launch your new Drill cluster:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh --site $DRILL_SITE start
+```
+You will see a number of lines describing the startup process. The tool automatically archives and uploads your site directory, which YARN copies (along with Drill) onto each node. If all goes well, the tool prints a URL for the Drill Application Master process that you can use to monitor the cluster. Your Drillbits should now be up and running. (If not, see the Troubleshooting section below.)
+To check the status of your Drill cluster:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh --site $DRILL_SITE status
+```
+To stop your cluster:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh --site $DRILL_SITE stop
+```
+Note, to avoid typing the site argument each time, you can set an environment variable:
+```
+export DRILL_CONF_DIR=$DRILL_SITE
+$DRILL_HOME/bin/drill-on-yarn.sh start
+```
+
+## Configuration Reference
+
+The Getting Started section presented the minimum configuration needed to launch Drill under YARN. Additional configuration options are available for specialized cases. See drill-on-yarn-example.conf for information about the other options.
+​
+### Application Name
+
+The application name appears when starting or stopping the Drill cluster and in the Drill-on-YARN web UI. Choose a name helpful to you:
+```
+app-name: "My Drill Cluster"
+```
+
+### Drill Distribution Archive
+
+The Drill distribution archive is assumed to expand to create a folder that has the same name as the archive itself (minus the .tar.gz suffix). That is, the archive `apache-drill-x.y.z.tar.gz` is assumed to expand to a directory named apache-drill-x.y.z. Apache Drill archives follow this pattern. In specialized cases, you may have to create your own archive. If you do, it is most convenient if you follow the same pattern. However, if cannot follow the pattern, you can configure Drill-on-YARN to follow a custom pattern using the drill-install.dir-name option:
+```
+  drill-install: {
+    client-path: "/path/to/your-custom-archive.tar.gz"
+    dir-name: "your-drill-directory"
+  }
+```
+Where:
+
+* */path/to/your-custom-archive.tar.gz* is the location of your archive.
+* *your-drill-directory* is the name of your Drill directory within the archive.
+
+​### Customize Web UI Port
+
+If you run multiple Drill clusters per YARN cluster, then YARN may choose to place two Drill AM processes on the same node. To avoid port conflicts, change the HTTP port for one or both of the Drill clusters:
+```
+drill.yarn:
+  http: {
+    port: 12345
+  }
+}
+```
+
+### Customize Application Master Settings
+
+The following settings apply to the Application Master. All are prefixed with `drill.yarn.am`.
+
+| Name | Description | Default |
+| ---- | ----------- | ------- |
+| memory-mb | Memory, in MB, to allocate to the AM. | 512 |
+| vcores | Number of CPUS to allocate to the AM.| 1 |
+| heap | Java heap for the AM. | 450M |
+| node-label-expr | YARN node label expression to use to select nodes to run the AM. | None |
+
+​5.5.​ Drillbit Customization
+The following Drill-on-YARN configuration options control the Drillbit processes. All properties start with drill.yarn.drillbit.
+
+| Name | Description | Default |
+| ---- | ----------- | ------- |
+| memory-mb | Memory, in MB, to allocate to the Drillbit. | 13000 |
+| vcores | Number of CPUS to allocate to the AM. | 4 |
+| disks | Number of disk equivalents consumed by Drill (on versions of YARN that support disk resources.) | 1 |
+| heap | Java heap memory. | 4G |
+| max-direct-memory | Direct (off-heap) memory for the Drillbit. | 8G |
+| log-gc | Enables Java garbage collector logging | false |
+| class-path | Additional class-path entries. | blank |
+
+Note that the Drillbit node expression is set in the labeled pool below.
+
+### Cluster Groups
+
+YARN was originally designed for Map-Reduce jobs that can run on any node, and that often can be combined onto a single node. Compared to the traditional Map-Reduce jobs, Drill has additional constraints:
+
+* Only one Drillbit (per Drill cluster) can run per host (to avoid port conflict.)
+* Drillbits work best when launched on the same host as the data that the Drillbit is to scan.
+
+Drill provides two ways to launch your Drill cluster: one for convenience, the other for production.
+
+#### Basic Cluster
+
+A basic cluster launches n drill-bits on distinct nodes anywhere in your YARN cluster. The basic cluster is great for testing and other informal tasks: just configure the desired vcores and memory, along with a number of nodes, then launch Drill. YARN will locate a set of suitable hosts anywhere on the YARN cluster.
+
+#### Using YARN Labels
+
+More typically you will decide the set of hosts that are to run Drill. Perhaps you want to run Drill on a few hosts per rack for data locality. Perhaps certain hosts are allocated to certain departments or groups within your organization. There are two ways to do this: using a queue label or using node labels. In either case, you start by Identify the hosts on which Drill should run using YARN labels as described in the YARN documentation.
+
+(Note: To be tested; this works on MapR's version of YARN, needs testing on the Apache version.)
+
+#### Labeled Queue
+
+(TODO: Need information on queue labels.)
+
+#### Labeled Hosts
+
+Drill-on-YARN can handle node placement directly without the use of labeled queues. You use the “labeled” pool type. Then, set the drillbit-label-expr property to a YARN label expression that matches the nodes on which Drill should run. You will most often care only about Drillbit placement. Finally, indicate the number of Drillbits to run on the selected nodes.
+
+(Need example)
+
+## Drill-on-YARN Command-line Tool
+
+Drill provides the drill-on-yarn command line tool to start, stop, resize and check the status of your Drill cluster. The tool is located in:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh --site $DRILL_SITE command
+```
+Where command is one of those described below.
+
+### Start the Drill Cluster
+
+Start your drill cluster with the start command:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh start
+```
+The command shows the startup status followed by a summary of the application:
+```
+Launching Drill-on-YARN...
+Application ID: application_1462842354064_0001
+Application State: ACCEPTED
+Starting......
+Application State: RUNNING
+Tracking URL: http://10.250.50.31:8088/proxy/application_1462842354064_0001/
+Application Master URL: http://10.250.50.31:8048/
+```
+The first line confirms which cluster is starting by displaying the cluster name from your configuration file. The next line shows YARN’s application ID and tracks the job status from Accepted to Running. Once the job starts, you’ll see YARN’s job tracking URL along with Drill-on-YARN’s web UI url. Use this URL to visit the web UI described below.
+
+Once the application starts, the Drill-on-YARN writes an “appid” file into your master directory:
+```
+ls $MASTER_DIR
+…
+drillbits1.appid
+```
+The file name is the same as your Drill cluster ID. The file contains the id if the Drill-on-YARN application for use by the other commands described below.
+
+You can run only one Drill AM at a time. If you attempt to start as second one from the same client machine on which you started the first, the client command will complain that the appid file already exists. If you attempt to start the cluster from a different node, then the second AM will detect the conflict and will shut down again.
+​
+### Drill Cluster Status
+
+You can retrieve basic information about the Drill cluster as follows:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh status
+```
+You will see output something like the following:
+```
+Application ID: application_1462842354064_0001
+Application State: RUNNING
+Host: yosemite/10.250.50.31
+Tracking URL: http://10.250.50.31:8088/proxy/application_1462842354064_0001/
+Queue: default
+User: drilluser
+Start Time: 2016-05-09 16:56:40
+Application Name: Drill-on-YARN
+AM State: LIVE
+Target Drillbit Count: 1
+Live Drillbit Count: 1
+For more information, visit: http://10.250.50.31:8048/
+```
+The first two several lines give you information about YARN’s state: the application ID, the application state and YARN’s tracking URL for the application. Next is the host on which the Drill AM is running, the queue on which the application was placed and the user who submitted the application. The start time tells you when YARN started the application.
+
+The next few lines are specific to Drill: the name of the application (which you configured in the Drill-on-YARN configuration file), the Drill application master URL, the number of drillbits you requested to run and the number actually running.
+
+Finally, the last line gives you the URL to use to access the Drill-on-YARN web UI described below.
+
+### Stop the Drill Cluster
+
+You can stop the Drill cluster from the command line:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh stop
+```
+Note that this command is “forceful”, it kills any in-flight queries. The output tracks the shutdown and displays the final YARN application status:
+```
+Stopping Application ID: application_1462842354064_0001
+Stopping...
+Stopped.
+Final status: SUCCEEDED
+```
+
+### Resize the Drill Cluster
+
+You can add or remove nodes to your Drill cluster while the cluster runs using the resize command. You can specify the change either by giving the number of nodes you want to run:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh resize 10
+```
+Or by specifying the change in node count: + for increase, - for decrease. To add two nodes:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh resize +2
+```
+To remove three nodes:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh resize -3
+```
+Drill will add nodes only if additional nodes are available from YARN. If you request to stop more nodes than are running, Drill stops all the running nodes.
+
+Note that in the current version of Drill, stopping nodes is a forceful operation: any in-flight queries will fail.
+
+### Clean the DFS Files
+
+If you run Drill-on-YARN for a temporary cluster, Drill will leave the Drill software archive in your designated DFS directory. You can remove those files with the following:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh clean
+```
+Specifically, the first start uploads your Drill archive to DFS. Stop leaves the archive in DFS. Subsequent start commands reuse the cached archive if it is the same size as the version on the local disk. Clean removes the cached file, forcing Drill to upload a fresh copy if you again restart the Drill cluster.
+​
+## Using the Web UI
+
+Applications that run under YARN provide an Application Master (AM) process to manage the application’s task. Drill provides the Drill-on-YARN Application Master for this purpose. When you launch Drill using the command line tool, the tool asks YARN to launch Drill’s AM, which in turn launches your Drillbits.
+
+The Drill application master provides a web UI to monitor cluster status and to perform simple operations such as increasing or decreasing cluster size, or stop the cluster.
+
+You can reach the UI using the URL provided when the application starts. You can also follow the link from the YARN Resource Manager UI. Find the page for the Drill application. Click on the Tracking URL link.
+
+The UI provides five pages:
+
+* A main page that provides overall cluster status.
+* A configuration page that lets you view the complete set of configuration variables which the Drill AM uses.
+* Detailed list of the running Drillbits.
+* Management page with a number of simple operations to resize or stop the cluster.
+* A history of stopped, killed or failed Drillbits. Use this to diagnose problems.
+
+### Main Page
+
+The main page shows the state of the Drill cluster.
+
+Drill Cluster Status: the state of the Drill cluster, one of:
+
+* LIVE: Normal state: shows that your Drill cluster is running.
+* ENDING: The cluster is in the process of shutting down.
+
+There is no “ENDED” state: once the cluster is shut down, the AM itself exists and the web UI is no longer available.
+
+**Target Drillbit Count**: The number of Drillbits to run in the cluster. The actual number may be less if Drillbits have not yet started, or if YARN cannot allocate enough containers.
+
+**Live Drillbit Count**: Number of drillbits that are ready for use. These have successfully started, have registered with ZooKeeper, and are ready for use. You can see the detail of all Drillbits (including those in the process of starting or stopping) using the Drillbits page. Each Drillbit must run on a separate node, so this is also the number of nodes in the cluster running Drill.
+
+**Total Drillbit Memory and Virtual Cores**: The total number of YARN resources currently allocated to running Drillbits.
+
+**YARN Node Count, Memory and Virtual Cores**: Reports general information about YARN itself including the number of nodes, the total cluster memory and total number of virtual cores.
+
+**Groups**: Lists the cluster groups defined in the configuration file (of which only one is currently supported), along with the target and actual number of Drillbits in that group.
+​
+### Configuration Page
+
+The configuration page shows the complete set of configuration values used for the current run. The values come from your own configuration along with Drill-provided defaults. Use this page to diagnose configuration-related issues. Names are shown in fully-expanded form. That is the name “drill.yarn.http.port” refers to the parameter defined as follows in your configuration file:
+```
+drill.yarn:
+  http: {
+    port: 8048
+  }
+}
+```
+
+### Drillbits Page
+
+The Drillbits page lists all drillbits in all states.
+
+**ID**: A sequential number assigned to each new Drillbit. Numbers may not start with 1 if you’ve previously shut down some drillbits.
+
+**Group**: The cluster group that started the Drillbit. (Cluster groups are from the configuration file.)
+
+**Host**: The host name or IP address on which the Drillbit runs. If the Drillbit is in normal operating state, this field is also a hyperlink to the Web UI for the drillbit.
+
+**State**: The operating state of the Drillbit. The normal state is “Running.” The drillbit passes through a number of states as YARN allocates a container and launches a process, as the AM waits for the Drillbit to become registered in ZooKeeper, and so on. Similarly, the Drillbit passes through a different set of states during shutdown. Use this value to diagnose problems.
+
+If the Drillbit is in a live state, then this field shows an “[X]” link that you can use to kill this particular Drillbit. Use this if the Drillbit has startup problems or seems unresponsive. During the shut-down process, the kill link disappears and is replaced with a “(Cancelled)” note.
+
+**ZK State**: The ZooKeeper handshake state. Normal state is “START_ACK”, meaning that the Drillbit has registered with ZooKeeper. This state is useful when diagnosing problems.
+
+**Container ID**: The YARN-assigned container ID for the Drillbit task. The ID is a link, it takes you to the YARN Node Manager UI for the Drillbit task.
+
+**Memory and Virtual Cores**: The amount of resources actually allocated to the Drillbit by YARN.
+
+**Start Time**: The date and time (in your local time-zone, displayed in ISO format) when the Drillbit launch started.
+
+This page will also display unmanaged Drillbits, if present. An unmanage Drillbit is one that is running, has registered with ZooKeeper, but was not started by the Drill Application Master. Likely the Drillbit was launched using the drillbit.sh script directly. Use the host name to locate the machine running the Drillbit if you want to convert that Drillbit to run under YARN.
+​
+### Manage Page
+
+The Manage page allows you to resize or stop the cluster. You can resize the cluster by adding Drillbits, removing Drillbits or setting the cluster to a desired size.
+
+Drill is a long-running application. In normal practice, you leave Drill running indefinitely. You would shut down your Drill cluster only to, say, perform an upgrade of the Drill software or to change configuration options. When you terminate your Drill cluster, any in-progress queries will fail. Therefore, a good practice is to perform the shut down with users so that Drill is not processing any queries at the time of the shut-down.
+
+When removing or shutting-down the cluster, you will receive a confirmation page asking if you really do want to stop Drillbit processes. Click Confirm to continue.
+
+### History Page
+
+The History page lists all Drillbits which have failed, been killed, or been restarted. The History page allows you to detect failures and diagnose problems. Use the YARN container ID listed on this page to locate the log files for the Drillbit.
+
+## Multiple Drill Clusters
+
+Drill-on-YARN allows you to easily define multiple Drill clusters on a single YARN cluster. Each Drill cluster is a collection of Drillbits that work as an independent unit. For example, you might define one test cluster of a few machines on the same physical cluster that runs larger clusters for, say, Development and Marketing.
+
+Drill clusters coordinate using ZooKeeper, so you must assign each cluster a distinct ZooKeeper entry. YARN may launch Drillbits from different clusters on the same physical node, so each Drill cluster must use a distinct set of ports. Since each cluster requires its own setup, you must create a separate site directory for each. The instructions below explain the required setup.
+
+### Create a New Site Directory
+
+Create a new site directory for your new cluster. Let’s say that your new cluster has the name “second”. Using the same structure as above, create a new site directory under your master directory:
+```
+export SECOND_SITE=$MASTER_DIR/second
+mkdir $SECOND_SITE
+```
+Copy files into this new site directory as you did to create the first one. You can copy and modify an existing set of files, or create the site from scratch.
+
+### Configure Drill
+
+At a minimum, you must set the following configuration options in drill-override.sh:
+```
+drill.exec: {
+  cluster-id: "drillbits",
+  zk: {
+    root: "second"
+    connect: "zk-host:2181"
+  }
+
+  rpc {
+    user.server.port: 41010
+    bit.server.port: 41011
+  }
+  http.port: 9047
+}
+```
+You have two options for how your new cluster interacts with the existing cluster. The normal case is a shared-nothing scenario in which the two clusters are entirely independent at the configuration level. For this case, ensure that the zk.root name is distinct from any existing cluster.
+
+In the more advanced case, if both clusters share the same zk.root value, then they will share settings such as storage plugins. If the clusters share the same root, then they must have distinct cluster-id values.
+
+In addition, the three ports must have values distinct from all other clusters. In the example above, we’ve added a 1 to the first digit of the default port numbers; you can choose any available ports.
+
+### Drill-on-YARN Configuration
+
+Create the drill-on-yarn.conf file as described in an earlier section. The following must be distinct for your cluster:
+```
+drill.yarn: {
+  app-name: "Second Cluster"
+
+  dfs: {
+    app-dir: "/user/drill2"
+  }
+
+  http : {
+    port: 9048
+  }
+}
+```
+That is, give your cluster a distinct name, a distinct upload directory in DFS, and a distinct port number.
+
+### Start the Cluster
+
+Use the site directory for the second cluster to start the cluster:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh --site $SECOND_SITE start
+```
+
+## Enabling Web UI Security
+
+Drill-on-YARN provides a web UI as described earlier. By default, the UI is open to everyone. You can secure the UI using either a simple predefined user name and password, or using Drill’s user authentication.
+​
+### Simple Security
+
+Simple security is enabled using three configuration settings:
+```
+drill.yarn.http: {
+    auth-type: "simple"
+    user-name: "bob"
+    password: "secret"
+}
+```
+Restart the Drill-on-YARN application master. When you visit the web UI, a login page should appear, prompting you to log in. Only the above user and password are valid.
+
+Obviously, simple security is not highly secure; but it is handy for testing, prototypes and the like.
+
+### Using Drill’s User Authentication
+
+Drill-on-YARN can use Drill’s authentication system. In this mode, the user name and password must match that of the user that started the Drill-on-YARN application. To enable Drill security:
+```
+drill.yarn.http: {
+    auth-type: "drill"
+}
+```
+You must have previously enabled Drill user authentication as described in the [Drill documentation](http://drill.apache.org/docs/configuring-user-authentication/):
+```
+  rpc {
+    user.server.port: 41010
+    bit.server.port: 41011
+  }
+  http.port: 9047
+```
+
+## Release Notes
+
+Drill-on-YARN creates a tighter coupling between Drill and Hadoop than did previous Drill versions. You should be aware of a number of compatibility issues.
+
+### Migrating the `$DRILL_HOME/conf/drill-env.sh` Script
+
+Prior to Drill 1.8, the `drill-env.sh` script contained Drill defaults, distribution-specific settings, and configuration specific to your application (“site”.) In Drill 1.8, the Drill and distribution settings are moved to other locations. The site-specific settings change in format to allow YARN to override them. The following section details the changes you must make if you reuse a `drill-env.sh` file from a prior release. (If you create a new file, you can skip this section.)
+
+At the end of this process, your file should contain just two lines for memory settings, plus any additional custom settings you may have added.
+​
+#### Memory Settings
+
+Most Drill configuration is done via the Drill configuration file and the configuration registry. However, certain options must be set at the time that Drill starts; such options are configured in the `$DRILL_HOME/conf/drill-env.sh` file. Under YARN, these settings are set in the YARN configuration. To ensure that the YARN configuration options are used, you must modify your existing `drill-env.sh` file as follows. (If you are installing Drill fresh, and don’t have an existing file, you can skip these steps. The Drill 1.8 and later files already have the correct format.)
+
+Find the two lines that look like this:
+```
+DRILL_MAX_DIRECT_MEMORY="8G"
+DRILL_HEAP="4G"
+```
+Replace them with the following two lines.
+```
+export DRILL_MAX_DIRECT_MEMORY=${DRILL_MAX_DIRECT_MEMORY:-"8G"}
+export DRILL_HEAP=${DRILL_HEAP:-"4G"}
+```
+Copy the actual values from the old lines to the new ones (e.g. the “8G” and “4G” values.) Those are the values that Drill when use if you launch it outside of YARN. The new lines ensure that these values are replaced by those set by Drill-on-YARN when running under YARN.
+
+If you omit this change, then Drill will ignore your memory settings in Drill-on-YARN, resulting in a potential mismatch between the Drill memory settings and the amount of memory requested from YARN.
+
+#### Remove General Drill Settings
+
+If you are reusing the drill-env.sh from a prior release, find lines similar to the following:
+```
+export DRILL_JAVA_OPTS="-Xms$DRILL_HEAP -Xmx$DRILL_HEAP -XX:MaxDirectMemorySize=$DRILL_MAX_DIRECT_MEMORY \ -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=1G \ -Ddrill.exec.enable-epoll=true"
+```
+Compare these lines to the original `drill-env.sh` to determine if you modified the lines. These general Drill defaults now reside in other Drill scripts and can be remove from your site-specific version of `drill-env.sh`.
+
+#### Remove Distribution-specific Settings
+
+Some Drill distributions added distribution-specific settings to the `drill-env.sh script`. Drill 1.8 moves these settings to a new `$DRILL_HOME/conf/distrib-env.sh` file. Compare drill-env.sh and distrib-env.sh. Lines that occur in both files should be removed from drill-env.sh. If you later find you need to change the settings in `distrib-env.sh`, simply copy the line to `drill-env.sh` and modify the line. Drill reads `drill-env.sh` after `distrib-env.sh`, so your site-specific settings will replace the default distribution settings.
+
+### Hadoop Jar Files
+
+Drill depends on certain Hadoop Java jar files which the Drill distribution includes in the `$DRILL_HOME/jars/3rdparty` directory. Although YARN offers Drill a Java class-path with the Hadoop jars, Drill uses its own copies instead to ensure Drill runs under the same configuration with which it was tested. Drill distributions that are part of a complete Hadoop distribution (such as the MapR distribution) have already verified version compatibility for you. If you are assembling your own Hadoop and Drill combination, you should verify that the Hadoop version packaged with Drill is compatible with the version running our YARN cluster.
+
+### `$DRILL_HOME/conf/core-site.xml` Issue
+
+Prior versions of Drill included a file in the `$DRILL_HOME/conf` directory called `core-site.xml`. As it turns out, YARN relies on a file of the same name in the Hadoop configuration directory. The Drill copy hides the YARN copy, preventing YARN from operating correctly. For this reason, version 1.8 of Drill renames the example file to `core-site-example.xml`.
+
+When upgrading an existing Drill installation, do not copy the file from your current version of Drill to the new version.
+
+If you modified core-site.xml, we recommend you merge your changes with Hadoop’s `core-site.xml` file.
+​
+### Mac OS setsid Issue
+
+YARN has a bug which prevents Drillbits from properly shutting down when run under YARN on Mac OS.
+
+[YARN-3066](https://issues.apache.org/jira/browse/YARN-3066): Hadoop leaves orphaned tasks running after job is killed
+
+You may encounter this problem if you use a Mac to try out the YARN integration for Drill. The symptom is that you:
+
+* Start Drill as described below
+* Attempt to stop the Drill cluster as described below
+* Afterwards use `jps` to list Java processes and find that Drillbit is still running.
+
+The problem is that the setsid command is not available under MacOS. The workaround is to use the open source equivalent:
+
+* Install the [XCode command line tools](https://developer.apple.com/library/ios/technotes/tn2339/_index.html).
+* Using git, clone ersatz-ssid from [`https://github.com/jerrykuch/ersatz-setsid`](https://github.com/jerrykuch/ersatz-setsid)
+* Cd into the ersatz-ssid directory and type: `make`
+* Copy the resulting executable into `/usr/bin`: `sudo cp setsid /usr/bin`
+
+### Apache YARN Node Labels and Labeled Queues
+
+The Drill-on-YARN feature should work with Apache YARN node labels, but such support is currently not tested. Early indications are that the Apache YARN label documentation does not quite match the implementation, and that labels are very tricky. The Drill team is looking forward to community assistance to better support Apache YARN node labels.
+
+### Apache YARN RM Failure and Recovery
+
+Drill-on-YARN currently does not gracefully handle Apache YARN Resource Manager failure and recovery. According to the Apache YARN documentation, a failed RM may restart any in-flight Application Masters, then alert the AM of any in-flight tasks. Drill-on-YARN is not currently aware of this restart capability. Existing Drillbits will continue to run, at least for a time. They may be reported in the Drill-on-YARN web UI as unmanaged. Presumably, eventually YARN will kill the old Drillbits at which time Drill-on-YARN should start replacements. This is an area for future improvement based on community experience.
+
+### Configuring User Authentication
+
+The [Drill Documentation](http://drill.apache.org/docs/configuring-user-authentication/) describes how to configure user authentication using PAM. Two revisions are needed for Drill-on-YARN:
+
+* Configure user authentication for Drill using a site directory
+* Configure Drill-on-YARN authentication
+
+The existing instructions explain how to configure PAM authentication by changing Drill config files and adding libraries to the Drill distribution directory. If you use that approach, you must rebuild the Drill software archive as described elsewhere in this document. It is easier, however, to configure security using the site directory as explained below.
+
+#### Configuring User Authentication for the Drillbit
+
+Existing instructions:
+
+Untar the file, and copy the `libjpam.so` file into a directory that does not contain other Hadoop components.
+
+Example: `/opt/pam/`
+
+Revised instructions: You have the option of deploying the library to each node (as described above), or allowing YARN to distribute the library. To have YARN do the distribution:
+
+Create the following directory:
+`
+$DRILL_SITE/lib
+`
+Untar the file and copy `libjpam.so` into `$DRILL_SITE/lib`.
+
+Existing instructions:
+
+> Add the following line to `<DRILL_HOME>/conf/drill-env.sh`, including the directory where the `libjpam.so` file is located:
+>
+> `export DRILLBIT_JAVA_OPTS="-Djava.library.path=<directory>"`
+>
+> Example: `export DRILLBIT_JAVA_OPTS="-Djava.library.path=/opt/pam/"`
+
+Revised instructions:
+
+If you are not using Drill-on-YARN, set a new environment variable in `drill-env.sh`:
+```
+export DRILL_JAVA_LIB_PATH=”<directory>”
+```
+If you install the library yourself, either set `DRILL_JAVA_LIB_PATH` as above, or set the following in `drill-on-yarn.conf`:
+```
+drill.yarn.files: {
+   library-path: "<directory>"
+}
+```
+**Note**: do not explicitly set `DRILLBIT_JAVA_OPTS` as you may have done in previous releases; Drill won’t know how to add your `$DRILL_SITE/lib` directory or how to interpret the library-path item above.
+
+If you put the library in the `$DRILL_SITE/lib` directory, then Drill-on-YARN will automatically do the needed configuration; there is nothing more for you to do.
+
+### Implementing and Configuring a Custom Authenticator
+
+Most of the existing steps are fine, except for step 3. Current text:
+
+> Add the JAR file that you built to the following directory on each Drill node:
+
+> `<DRILLINSTALL_HOME>/jars`
+
+Revised text: Add the JAR file that you built to the following directory on each Drill node:
+```
+$DRILL_SITE/jars
+```
+If running under YARN, you only need to add the jar to the site directory on the node from which you start Drill-on-YARN (which we’ve referred to as $MASTER_DIR.)
+
+Also, step 5:
+
+Restart the Drillbit process on each Drill node.
+```
+<DRILLINSTALL_HOME>/bin/drillbit.sh restart
+```
+Under YARN, restart the YARN cluster:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh --site $DRILL_SITE restart
+```
+
+#### Configuring User Authentication for the Application Master
+
+If you configure user authentication for Drill, then user authentication is automatically configured in the Application Master also. Only users with admin privileges can use the AM web UI.
+
+## Testing User Authentication on the Mac
+
+The [Drill Documentation](http://drill.apache.org/docs/configuring-user-authentication/) describes how to configure user authentication using PAM, including instructions for downloading a required native library. However, if you are testing security on the Mac, the referenced library does not work on modern Macs. Instead, see the work-around in [DRILL-4756](https://issues.apache.org/jira/browse/DRILL-4756).
+​
+​## `drill-env.sh` Settings
+
+When running Drill outside of YARN, you can set many startup options in `drill-env.sh`. Most users accept the defaults. However, some users require specialized settings.
+
+Under YARN, Drill still reads your `$DRILL_SITE/drill-env.sh` file to pick up configuration. However, for most options, Drill-on-YARN provides configuration options in `drill-on-yarn.conf` to set options that were formerly set in `drill-env.sh`. The following provides a mapping:
+
+
+| `drill-env.sh` Environment Variable | `drill-on-yarn.conf` Configuration Parameter |
+| ----------------------------------- | -------------------------------------------- |
+| DRILL_MAX_DIRECT_MEMORY * | drill.yarn.drillbit.max-direct-memory |
+| DRILL_HEAP * | drill.yarn.drillbit.heap |
+| DRILL_JAVA_OPTS | drill.yarn.drillbit.vm-args (Added to those in `drill-env.sh`.) |
+| SERVER_GC_OPTS (to add GC logging) | `drill.yarn.drillbit.log-gc` (To enable GC logging) |
+| DRILL_HOME | Set automatically when files are localized (`drill.yarn.drill-install.localize` is true), else `drill.yarn.drill-install.drill-home`.
+| DRILL_CONF_DIR | Set automatically when files are localized, else uses the normal defaults. |
+| DRILL_LOG_DIR | Set automatically to point to YARN’s log directory unless disabled by setting `drill.yarn.drillbit.disable-yarn-logs` to false. If disabled, uses the normal Drill defaults. |
+| DRILL_CLASSPATH_PREFIX * | `drill.yarn.drillbit.prefix-class-path` |
+| HADOOP_CLASSPATH * | `drill.yarn.hadoop.class-path` (or, better `drill.yarn.drillbit.extn-class-path`.) |
+| HBASE_CLASSPATH * | `drill.yarn.hadoop.hbase-class-path` (or, better `drill.yarn.drillbit.extn-class-path`.) |
+| EXTN_CLASSPATH * (New in Drill 1.8.) | `drill.yarn.drillbit.extn-class-path` |
+| DRILL_CLASSPATH * | `drill.yarn.drillbit.class-path` |
+
+\* If  you set these options in both places, then the value in `drill-env.sh` takes precedence.
+
+Note that `EXTN_CLASSPATH` (and `drill.yarn.drillbit.extn-class-path`) are a newer, more general way to add extensions. Rather than setting specific Hadoop or HBase variables, you can combine any number of extensions into the single extension classpath.
+
+## Troubleshooting
+
+Drill-on-YARN starts a complex chain of events: the client starts the AM and the AM starts Drillbits, both using YARN. Many opportunities exist for configuration issues to derail the process. Below are a number of items to check if things go wrong.
+
+### Client Start
+
+The Drill-on-YARN client prints all messages to the console. Some common errors are:
+
+#### Missing `HADOOP_HOME`
+
+Drill-on-YARN requires access to your Hadoop configuration as described above. The client will display an error and fail if it is unable to load the DFS or YARN configuration, typically because HADOOP_HOME is not set.
+​
+#### Missing/wrong Drill Archive
+
+Drill-on-YARN uploads your Drill archive to DFS. The client will fail if the archive configuration is missing, if the archive does not exist, is not readable, or is not in the correct format. Check that the drill.yarn.drill-install.client-path provides the full path name to the Drill archive.
+
+#### DFS Configuration or Connection Problems
+
+The Drill-on-YARN client uploads the Drill archive and your site directory to your DFS. Possible problems here include:
+
+* Missing DFS configuration in the Hadoop configuration folder
+* Incorrect DFS configuration (wrong host or port)
+* Missing permissions to write to the folder identified by the `drill.yarn.dfs.app-dir` configuration property (`/user/drill` by default.)
+​
+#### Wrong Version of the Drill Archive
+
+Drill-on-YARN uploads a new copy of your site archive each time you start your Drill cluster. However, the Drill software archive is large, so Drill-on-YARN uploads the archive only when it changes. Drill detects changes by comparing the size of the DFS copy with your local copy. Most of the time this works fine. However, if you suspect that Drill has not uploaded the most recent copy, you can force the client to perform an upload by either manually deleting the Drill archive from DFS, or using the -f option:
+```
+$DRILL_HOME/bin/drill-on-yarn.sh --site $DRILL_SITE start -f
+```
+
+#### Site Directory Problems
+
+Drill creates a tar archive of your site directory using the following command:
+```
+tar -C $DRILL_SITE -czf /some/tmp/dir/temp-name.tar.gz
+```
+For some temporary directory selected by Java. This command can fail if your version of tar does not recognize the above arguments, if the site directory is not readable, or the temporary file cannot be created.
+
+#### YARN Application Launch Failure
+
+YARN may fail to launch the Drill AM for a number of reasons. The user running the Drill-on-YARN client may not have permission to launch YARN applications. YARN itself may encounter errors. Check the YARN log files for possible causes.
+
+#### Diagnosing Post-Launch Problems
+
+If the Drill AM starts, but does not launch Drillbits, your next step is to check the Drill AM web UI using the link provided when the application starts. If the AM exits quickly, then the URL may not be valid.
+
+Instead, you can use YARN’s own Resource Manager UI to check the status of the application, using the Application ID provided when the application starts. Look at the application's log files for possible problems.
+
+#### Application Master Port Conflict
+
+The Drill-on-YARN Application Master provides a web UI on port 8048 (one greater than the Drill web UI port) by default. However, if another application is already bound to that port, the AM will fail to launch. Select a different port as follows:
+```
+drill.yarn.http.port: 12345
+```
+
+#### Preserve the YARN Launch Context
+
+Some problems are easiest to diagnose if you set the YARN option to preserve the application’s launch context:
+
+((Need option))
+
+Then, locate the Node Manager host that launched the AM, and locate the nmPrivate subfolder for the application. Several Drill-on-YARN settings are passed to the Drill AM as environment variables. Review the launch_container.sh script to look for incorrect settings.
+
+Next, look for the PWD variable and visit that directory to ensure that the drill and site directories are properly created and contain the expected files. As noted above, ensure that the subdirectory of the drill directory either has the same name as your archive file, or is identified using the app-dir property as defined above.
+
+Finally, look at the directory used to create the stdout and stderr files in the launch container. Visit that directory and review the various log file to look for possible problems.
+
+### AM Failure
+
+#### AM Resource Limits
+
+YARN is very fragile, if configuration is not exactly right, our tasks will silently fail or will hang indefinitely. The Drill AM is a small process, but must be within the limits assigned to all YARN tasks, to AM’s on the assigned queue, to AM limits on the assigned queue, and AM limits for the user. Submitting an AM that exceeds any of these limits will lead to silent failure.
+
+#### Multiple AMs
+
+It is easy to accidentally start multiple AMs for the same Drill cluster. Two lines of defense protect against this fault:
+The Drill-on-YARN client look for an existing appid file and refuses to start a new AM when the file is present. (Use the -f file if the file is suprious.)
+
+The AM registers with ZK and will automatically shut down if another AM is already registered.
+
+## Recreate the Drill Archive
+
+The above instructions assume that you make no changes to your Drill installation; that all your site-specific files reside in a separate site directory. Prior Drill versions put all configuration within the Drill directory. If you chose to continue that pattern, or if you change the Drill installation, you must rebuild the Drill archive and configure Drill-on-YARN to upload your custom archive in place of the standard archive. The steps below explain the process.
+
+To change the contents of the Drill archive, you must perform two steps:
+
+* Create an archive of the Drill install directory.
+* Configure Drill-on-YARN to use that archive.
+
+### Create the Drill Archive
+
+The first step is to create the master archive of your Drill files. Do the following with the master directory as the current directory.
+```
+cd $MASTER_DIR
+tar -czf archive-name.tar.gz $DRILL_HOME
+```
+Replace “archive-name” with the name you chose above.
+
+### Configure Drill-on-YARN to Use the Archive
+
+Modify your drill-on-yarn.conf file to identify the archive you must created:
+```
+drill.yarn.drill-install.client-path: "/path/to/archive-name.tar.gz"
+```
+YARN expects that, when extracting the master directory, that it creates a directory called archive-name that contains the Drill directories conf, jars, and so on. However, if archive-name is different than the name of the $DRILL_HOME directory, simply configure the correct name of the expanded folder:
+```
+drill.yarn.drill-install.dir-name: “your-dir-name”
+```

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/img/am-overview.png
----------------------------------------------------------------------
diff --git a/drill-yarn/img/am-overview.png b/drill-yarn/img/am-overview.png
new file mode 100644
index 0000000..991a69d
Binary files /dev/null and b/drill-yarn/img/am-overview.png differ

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/img/client-classes.png
----------------------------------------------------------------------
diff --git a/drill-yarn/img/client-classes.png b/drill-yarn/img/client-classes.png
new file mode 100644
index 0000000..fa6e999
Binary files /dev/null and b/drill-yarn/img/client-classes.png differ

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/img/controller-classes.png
----------------------------------------------------------------------
diff --git a/drill-yarn/img/controller-classes.png b/drill-yarn/img/controller-classes.png
new file mode 100644
index 0000000..75086db
Binary files /dev/null and b/drill-yarn/img/controller-classes.png differ

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/img/overview.png
----------------------------------------------------------------------
diff --git a/drill-yarn/img/overview.png b/drill-yarn/img/overview.png
new file mode 100644
index 0000000..3e7b1ed
Binary files /dev/null and b/drill-yarn/img/overview.png differ

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/drill-yarn/pom.xml b/drill-yarn/pom.xml
new file mode 100644
index 0000000..7f33210
--- /dev/null
+++ b/drill-yarn/pom.xml
@@ -0,0 +1,144 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>drill-root</artifactId>
+    <groupId>org.apache.drill</groupId>
+    <version>1.13.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>drill-yarn</artifactId>
+  <packaging>jar</packaging>
+  <name>Drill-on-YARN</name>
+
+  <build>
+    <plugins>
+      <plugin>
+         <groupId>org.apache.maven.plugins</groupId>
+         <artifactId>maven-surefire-plugin</artifactId>
+         <configuration>
+           <systemPropertyVariables>
+             <drillScriptDir>${project.basedir}/../distribution/src/resources</drillScriptDir>
+             <logback.configurationFile>doy-test-logback.xml</logback.configurationFile>
+           </systemPropertyVariables>
+         </configuration>
+      </plugin>
+        <!-- See http://stackoverflow.com/questions/31173467/maven-cannot-find-git-dotgitdirectory -->
+  <!-- 
+      <plugin>
+        <groupId>pl.project13.maven</groupId>
+        <artifactId>git-commit-id-plugin</artifactId>
+        <version>2.1.9</version>
+        <executions>
+          <execution>
+            <id>for-jars</id>
+            <inherited>true</inherited>
+            <goals>
+              <goal>revision</goal>
+            </goals>
+            <configuration>
+              <dotGitDirectory>${project.basedir}/../.git</dotGitDirectory>
+              <generateGitPropertiesFilename>target/classes/git.properties</generateGitPropertiesFilename>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+     -->
+    </plugins>
+  </build>
+  
+  <dependencies>
+
+    <!-- For YARN client. -->
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    
+    <!--  Included as a reference because this seems to be the only
+          way to pick up profile-specific jars. -->
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-client</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <!--  For ZK monitoring -->
+    <dependency>
+      <groupId>org.apache.drill.exec</groupId>
+      <artifactId>drill-java-exec</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>drill-protocol</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-test</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- For configuration -->
+    <dependency>
+      <groupId>com.typesafe</groupId>
+      <artifactId>config</artifactId>
+      <version>1.0.0</version>
+    </dependency>
+
+    <!-- Logging -->
+    <dependency>
+      <groupId>ch.qos.logback</groupId>
+      <artifactId>logback-classic</artifactId>
+      <version>1.0.13</version>
+      <scope>test</scope>
+    </dependency>
+    
+    <!--  Testing -->
+    <dependency>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>drill-common</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <dependencyManagement>
+   <dependencies>
+	   <dependency>
+	      <groupId>org.apache.curator</groupId>
+	      <artifactId>curator-test</artifactId>
+	      <version>2.7.1</version>
+	      <scope>test</scope>
+	      <exclusions>
+          <exclusion>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+          </exclusion>
+	      </exclusions>
+	    </dependency>
+    </dependencies>
+  </dependencyManagement>
+</project>

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMException.java
----------------------------------------------------------------------
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 0000000..f4e7400
--- /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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMRegistrar.java
----------------------------------------------------------------------
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 0000000..fdce55b
--- /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();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMWrapperException.java
----------------------------------------------------------------------
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 0000000..d445ec7
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMYarnFacade.java
----------------------------------------------------------------------
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 0000000..62ebdfc
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AMYarnFacadeImpl.java
----------------------------------------------------------------------
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 0000000..ac25d0e
--- /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

http://git-wip-us.apache.org/repos/asf/drill/blob/f2ac8749/drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/AbstractDrillbitScheduler.java
----------------------------------------------------------------------
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 0000000..cf99eb3
--- /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());
+  }
+}