You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by xk...@apache.org on 2019/06/25 15:09:28 UTC

[hadoop] branch trunk updated: HDFS-12345 Add Dynamometer to hadoop-tools, a tool for scale testing the HDFS NameNode with real metadata and workloads. Contributed by Erik Krogen.

This is an automated email from the ASF dual-hosted git repository.

xkrogen pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new ab0b180  HDFS-12345 Add Dynamometer to hadoop-tools, a tool for scale testing the HDFS NameNode with real metadata and workloads. Contributed by Erik Krogen.
ab0b180 is described below

commit ab0b180ddb5d0775a2452d5eeb7badd252aadb91
Author: Erik Krogen <xk...@apache.org>
AuthorDate: Wed Mar 20 14:06:13 2019 -0700

    HDFS-12345 Add Dynamometer to hadoop-tools, a tool for scale testing the HDFS NameNode with real metadata and workloads. Contributed by Erik Krogen.
---
 dev-support/bin/dist-layout-stitching              |    1 +
 .../assemblies/hadoop-dynamometer-blockgen.xml     |   35 +
 .../assemblies/hadoop-dynamometer-infra.xml        |   35 +
 .../assemblies/hadoop-dynamometer-workload.xml     |   35 +
 .../resources/assemblies/hadoop-dynamometer.xml    |   73 ++
 .../src/main/resources/assemblies/hadoop-tools.xml |   24 +-
 hadoop-dist/pom.xml                                |    1 +
 .../hadoop-dynamometer-blockgen/pom.xml            |  131 +++
 .../src/main/bash/generate-block-lists.sh          |   38 +
 .../dynamometer/blockgenerator/BlockInfo.java      |  136 +++
 .../blockgenerator/GenerateBlockImagesDriver.java  |  140 +++
 .../GenerateDNBlockInfosReducer.java               |   99 ++
 .../dynamometer/blockgenerator/XMLParser.java      |  154 +++
 .../blockgenerator/XMLParserMapper.java            |   81 ++
 .../dynamometer/blockgenerator/package-info.java   |   26 +
 .../dynamometer/blockgenerator/TestBlockGen.java   |   89 ++
 .../dynamometer/blockgenerator/TestXMLParser.java  |   70 ++
 .../test/resources/fsimage_0000000000000061740.xml |  553 ++++++++++
 .../hadoop-dynamometer-dist/pom.xml                |  128 +++
 .../hadoop-dynamometer-infra/pom.xml               |  201 ++++
 .../src/main/bash/create-slim-hadoop-tar.sh        |   53 +
 .../src/main/bash/parse-metrics.sh                 |   84 ++
 .../src/main/bash/start-dynamometer-cluster.sh     |   41 +
 .../src/main/bash/upload-fsimage.sh                |   84 ++
 .../apache/hadoop/tools/dynamometer/AMOptions.java |  316 ++++++
 .../dynamometer/AllowAllImpersonationProvider.java |   39 +
 .../tools/dynamometer/ApplicationMaster.java       |  885 +++++++++++++++
 .../BlockPlacementPolicyAlwaysSatisfied.java       |   53 +
 .../apache/hadoop/tools/dynamometer/Client.java    | 1132 ++++++++++++++++++++
 .../hadoop/tools/dynamometer/DynoConstants.java    |  130 +++
 .../hadoop/tools/dynamometer/DynoInfraUtils.java   |  584 ++++++++++
 .../hadoop/tools/dynamometer/DynoResource.java     |   72 ++
 .../tools/dynamometer/SimulatedDataNodes.java      |  182 ++++
 .../hadoop/tools/dynamometer/package-info.java     |   29 +
 .../src/main/resources/start-component.sh          |  344 ++++++
 .../tools/dynamometer/TestDynamometerInfra.java    |  542 ++++++++++
 .../tools/dynamometer/TestDynoInfraUtils.java      |   66 ++
 .../src/test/resources/blocks/dn0-a-0-r-00000      |   34 +
 .../src/test/resources/blocks/dn1-a-0-r-00001      |   33 +
 .../src/test/resources/blocks/dn2-a-0-r-00002      |   33 +
 .../test/resources/conf/etc/hadoop/core-site.xml   |   20 +
 .../test/resources/conf/etc/hadoop/hdfs-site.xml   |   20 +
 .../resources/conf/etc/hadoop/log4j.properties     |   26 +
 .../src/test/resources/hadoop_3_1/VERSION          |    7 +
 .../hadoop_3_1/fsimage_0000000000000061740         |  Bin 0 -> 13795 bytes
 .../hadoop_3_1/fsimage_0000000000000061740.md5     |    1 +
 .../src/test/resources/yarn-site.xml               |   19 +
 .../hadoop-dynamometer-workload/pom.xml            |  146 +++
 .../src/main/bash/parse-start-timestamp.sh         |   52 +
 .../src/main/bash/start-workload.sh                |   42 +
 .../workloadgenerator/CreateFileMapper.java        |  146 +++
 .../workloadgenerator/VirtualInputFormat.java      |   57 +
 .../workloadgenerator/VirtualInputSplit.java       |   52 +
 .../workloadgenerator/VirtualRecordReader.java     |   78 ++
 .../workloadgenerator/WorkloadDriver.java          |  208 ++++
 .../workloadgenerator/WorkloadMapper.java          |   59 +
 .../audit/AuditCommandParser.java                  |   57 +
 .../audit/AuditLogDirectParser.java                |  145 +++
 .../audit/AuditLogHiveTableParser.java             |   70 ++
 .../audit/AuditReplayCommand.java                  |  163 +++
 .../workloadgenerator/audit/AuditReplayMapper.java |  290 +++++
 .../workloadgenerator/audit/AuditReplayThread.java |  310 ++++++
 .../audit/NoSplitTextInputFormat.java              |   47 +
 .../workloadgenerator/audit/package-info.java      |   27 +
 .../workloadgenerator/package-info.java            |   28 +
 .../workloadgenerator/TestWorkloadGenerator.java   |  136 +++
 .../audit/TestAuditLogDirectParser.java            |  143 +++
 .../src/test/resources/audit_trace_direct/audit0   |    6 +
 .../src/test/resources/audit_trace_hive/audit0     |    6 +
 hadoop-tools/{ => hadoop-dynamometer}/pom.xml      |   33 +-
 hadoop-tools/pom.xml                               |    1 +
 71 files changed, 9156 insertions(+), 25 deletions(-)

diff --git a/dev-support/bin/dist-layout-stitching b/dev-support/bin/dist-layout-stitching
index 119ee60..6579e0c 100755
--- a/dev-support/bin/dist-layout-stitching
+++ b/dev-support/bin/dist-layout-stitching
@@ -148,6 +148,7 @@ run cp -p "${ROOT}/hadoop-client-modules/hadoop-client-runtime/target/hadoop-cli
 run cp -p "${ROOT}/hadoop-client-modules/hadoop-client-minicluster/target/hadoop-client-minicluster-${VERSION}.jar" share/hadoop/client/
 
 run copy "${ROOT}/hadoop-tools/hadoop-tools-dist/target/hadoop-tools-dist-${VERSION}" .
+run copy "${ROOT}/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-dist/target/hadoop-dynamometer-dist-${VERSION}" .
 
 
 echo
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-dynamometer-blockgen.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dynamometer-blockgen.xml
new file mode 100644
index 0000000..8d35141
--- /dev/null
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dynamometer-blockgen.xml
@@ -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.
+-->
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3 http://maven.apache.org/xsd/assembly-1.1.3.xsd">
+  <id>hadoop-dynamometer-blockgen</id>
+  <formats>
+    <format>dir</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+
+  <fileSets>
+    <fileSet>
+      <directory>${basedir}/src/main/bash</directory>
+      <outputDirectory>dynamometer-blockgen/bin</outputDirectory>
+      <fileMode>0755</fileMode>
+    </fileSet>
+  </fileSets>
+
+</assembly>
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-dynamometer-infra.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dynamometer-infra.xml
new file mode 100644
index 0000000..a8d4c09
--- /dev/null
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dynamometer-infra.xml
@@ -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.
+-->
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3 http://maven.apache.org/xsd/assembly-1.1.3.xsd">
+  <id>hadoop-dynamometer-infra</id>
+  <formats>
+    <format>dir</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+
+  <fileSets>
+    <fileSet>
+      <directory>${basedir}/src/main/bash</directory>
+      <outputDirectory>dynamometer-infra/bin</outputDirectory>
+      <fileMode>0755</fileMode>
+    </fileSet>
+  </fileSets>
+
+</assembly>
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-dynamometer-workload.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dynamometer-workload.xml
new file mode 100644
index 0000000..80cf304
--- /dev/null
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dynamometer-workload.xml
@@ -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.
+-->
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3 http://maven.apache.org/xsd/assembly-1.1.3.xsd">
+  <id>hadoop-dynamometer-workload</id>
+  <formats>
+    <format>dir</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+
+  <fileSets>
+    <fileSet>
+      <directory>${basedir}/src/main/bash</directory>
+      <outputDirectory>dynamometer-workload/bin</outputDirectory>
+      <fileMode>0755</fileMode>
+    </fileSet>
+  </fileSets>
+
+</assembly>
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-dynamometer.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dynamometer.xml
new file mode 100644
index 0000000..4480352
--- /dev/null
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dynamometer.xml
@@ -0,0 +1,73 @@
+<!--
+   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 Li2cense 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.
+-->
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3 http://maven.apache.org/xsd/assembly-1.1.3.xsd">
+  <id>hadoop-dynamometer</id>
+  <formats>
+    <format>dir</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <fileSets>
+    <fileSet>
+      <directory>../hadoop-dynamometer-blockgen/target</directory>
+      <outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>
+      <includes>
+        <include>*-sources.jar</include>
+      </includes>
+    </fileSet>
+    <fileSet>
+      <directory>../hadoop-dynamometer-blockgen/target/hadoop-dynamometer-blockgen-${project.version}/dynamometer-blockgen</directory>
+      <outputDirectory>/share/hadoop/${hadoop.component}/dynamometer/dynamometer-blockgen</outputDirectory>
+    </fileSet>
+    <fileSet>
+      <directory>../hadoop-dynamometer-workload/target</directory>
+      <outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>
+      <includes>
+        <include>*-sources.jar</include>
+      </includes>
+    </fileSet>
+    <fileSet>
+      <directory>../hadoop-dynamometer-workload/target/hadoop-dynamometer-workload-${project.version}/dynamometer-workload</directory>
+      <outputDirectory>/share/hadoop/${hadoop.component}/dynamometer/dynamometer-workload</outputDirectory>
+    </fileSet>
+    <fileSet>
+      <directory>../hadoop-dynamometer-infra/target</directory>
+      <outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>
+      <includes>
+        <include>*-sources.jar</include>
+      </includes>
+    </fileSet>
+    <fileSet>
+      <directory>../hadoop-dynamometer-infra/target/hadoop-dynamometer-infra-${project.version}/dynamometer-infra</directory>
+      <outputDirectory>/share/hadoop/${hadoop.component}/dynamometer/dynamometer-infra</outputDirectory>
+    </fileSet>
+  </fileSets>
+  <dependencySets>
+    <dependencySet>
+      <outputDirectory>/share/hadoop/${hadoop.component}/lib</outputDirectory>
+      <unpack>false</unpack>
+      <scope>runtime</scope>
+      <useProjectArtifact>false</useProjectArtifact>
+      <excludes>
+        <!-- use slf4j from common to avoid multiple binding warnings -->
+        <exclude>org.slf4j:slf4j-api</exclude>
+        <exclude>org.slf4j:slf4j-log4j12</exclude>
+      </excludes>
+    </dependencySet>
+  </dependencySets>
+</assembly>
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml
index 7d34154..00f3b66 100644
--- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-tools.xml
@@ -175,6 +175,28 @@
       <outputDirectory>/share/hadoop/${hadoop.component}/sls</outputDirectory>
     </fileSet>
     <fileSet>
+    <directory>../hadoop-dynamometer/hadoop-dynamometer-blockgen/target</directory>
+    <outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>
+    <includes>
+    <include>*-sources.jar</include>
+    </includes>
+    </fileSet>
+    <fileSet>
+    <directory>../hadoop-dynamometer/hadoop-dynamometer-blockgen/target/hadoop-dynamometer-blockgen-${project.version}/dynamometer-blockgen</directory>
+    <outputDirectory>/share/hadoop/${hadoop.component}/dynamometer-blockgen</outputDirectory>
+    </fileSet>
+    <!--<fileSet>-->
+      <!--<directory>../hadoop-dynamometer/hadoop-dynamometer-blockgen/target</directory>-->
+      <!--<outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>-->
+      <!--<includes>-->
+        <!--<include>*-sources.jar</include>-->
+      <!--</includes>-->
+    <!--</fileSet>-->
+    <!--<fileSet>-->
+      <!--<directory>../hadoop-dynamometer/hadoop-dynamometer-blockgen/target/hadoop-dynamometer-blockgen-${project.version}/dynamometer-blockgen</directory>-->
+      <!--<outputDirectory>/share/hadoop/${hadoop.component}/dynamometer-blockgen</outputDirectory>-->
+    <!--</fileSet>-->
+    <fileSet>
       <directory>../hadoop-resourceestimator/target</directory>
       <outputDirectory>/share/hadoop/${hadoop.component}/sources</outputDirectory>
       <includes>
@@ -209,7 +231,7 @@
       <excludes>
         <exclude>org.apache.hadoop:hadoop-common</exclude>
         <exclude>org.apache.hadoop:hadoop-hdfs</exclude>
-        <exclude>org.apache.hadoop:hadoop-mapreduce</exclude>
+        <exclude>org.apache.hadoop:hadoop-client</exclude>
         <!-- pipes is native stuff, this just keeps pom from being package-->
         <exclude>org.apache.hadoop:hadoop-pipes</exclude>
         <!-- use slf4j from common to avoid multiple binding warnings -->
diff --git a/hadoop-dist/pom.xml b/hadoop-dist/pom.xml
index 07aa7b1..a56b105 100644
--- a/hadoop-dist/pom.xml
+++ b/hadoop-dist/pom.xml
@@ -196,6 +196,7 @@
                     <argument>${project.version}</argument>
                     <argument>${project.build.directory}</argument>
                     <argument>${basedir}/../hadoop-tools</argument>
+                    <!--<argument>${basedir}/../hadoop-tools/hadoop-dynamometer</argument>-->
                   </arguments>
                 </configuration>
               </execution>
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/pom.xml b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/pom.xml
new file mode 100644
index 0000000..bf53ffa
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/pom.xml
@@ -0,0 +1,131 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<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>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.3.0-SNAPSHOT</version>
+    <relativePath>../../../hadoop-project</relativePath>
+  </parent>
+  <artifactId>hadoop-dynamometer-blockgen</artifactId>
+  <version>3.3.0-SNAPSHOT</version>
+  <description>Apache Hadoop Dynamometer Block Listing Generator</description>
+  <name>Apache Hadoop Dynamometer Block Listing Generator</name>
+  <packaging>jar</packaging>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>1.10.19</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+        <configuration>
+          <attach>true</attach>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <profiles>
+    <profile>
+      <id>dist</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-assembly-plugin</artifactId>
+            <dependencies>
+              <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-assemblies</artifactId>
+                <version>${project.version}</version>
+              </dependency>
+            </dependencies>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>prepare-package</phase>
+                <goals>
+                  <goal>single</goal>
+                </goals>
+                <configuration>
+                  <appendAssemblyId>false</appendAssemblyId>
+                  <attach>false</attach>
+                  <finalName>${project.artifactId}-${project.version}</finalName>
+                  <descriptorRefs>
+                    <descriptorRef>hadoop-dynamometer-blockgen</descriptorRef>
+                  </descriptorRefs>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+       <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>deplist</id>
+            <phase>compile</phase>
+            <goals>
+              <goal>list</goal>
+            </goals>
+            <configuration>
+              <!-- referenced by a built-in command -->
+              <outputFile>${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt</outputFile>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/bash/generate-block-lists.sh b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/bash/generate-block-lists.sh
new file mode 100755
index 0000000..7fc6961
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/bash/generate-block-lists.sh
@@ -0,0 +1,38 @@
+#!/usr/bin/env 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.
+
+if type hadoop &> /dev/null; then
+  hadoop_cmd="hadoop"
+elif type "$HADOOP_COMMON_HOME/bin/hadoop" &> /dev/null; then
+  hadoop_cmd="$HADOOP_COMMON_HOME/bin/hadoop"
+elif type "$HADOOP_HOME/bin/hadoop" &> /dev/null; then
+  hadoop_cmd="$HADOOP_HOME/bin/hadoop"
+else
+  echo "Unable to find a valid hadoop command to execute; exiting."
+  exit 1
+fi
+
+script_pwd="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)/.."
+
+for f in ${script_pwd}/lib/*.jar; do
+  # Skip adding the blockgen JAR since it is added by the `hadoop jar` command
+  if [[ "$f" != *"dynamometer-blockgen-"* ]]; then
+    export HADOOP_CLASSPATH="$HADOOP_CLASSPATH:$f"
+  fi
+done
+"$hadoop_cmd" jar "${script_pwd}"/lib/dynamometer-blockgen-*.jar \
+  org.apache.hadoop.tools.dynamometer.blockgenerator.GenerateBlockImagesDriver "$@"
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/BlockInfo.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/BlockInfo.java
new file mode 100644
index 0000000..550329c
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/BlockInfo.java
@@ -0,0 +1,136 @@
+/**
+ * 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.hadoop.tools.dynamometer.blockgenerator;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * This is the MapOutputValue class. It has the blockId and the block generation
+ * stamp which is needed to generate the block images in the reducer.
+ *
+ * This also stores the replication of the block, but note that it does not
+ * serialize this value as part of its {@link Writable} interface, and does not
+ * consider the replication when doing equality / hash comparisons.
+ */
+
+public class BlockInfo implements Writable {
+
+  public LongWritable getBlockId() {
+    return blockId;
+  }
+
+  public void setBlockId(LongWritable blockId) {
+    this.blockId = blockId;
+  }
+
+  public LongWritable getBlockGenerationStamp() {
+    return blockGenerationStamp;
+  }
+
+  public void setBlockGenerationStamp(LongWritable blockGenerationStamp) {
+    this.blockGenerationStamp = blockGenerationStamp;
+  }
+
+  public LongWritable getSize() {
+    return size;
+  }
+
+  public void setSize(LongWritable size) {
+    this.size = size;
+  }
+
+  public short getReplication() {
+    return replication;
+  }
+
+  public void setReplication(short replication) {
+    this.replication = replication;
+  }
+
+  private LongWritable blockId;
+  private LongWritable blockGenerationStamp;
+  private LongWritable size;
+  private transient short replication;
+
+  public BlockInfo(BlockInfo blockInfo) {
+    this.blockId = blockInfo.getBlockId();
+    this.blockGenerationStamp = blockInfo.getBlockGenerationStamp();
+    this.size = blockInfo.getSize();
+    this.replication = replication;
+  }
+
+  public BlockInfo() {
+    this.blockId = new LongWritable();
+    this.blockGenerationStamp = new LongWritable();
+    this.size = new LongWritable(1);
+  }
+
+  public BlockInfo(long blockid, long blockgenerationstamp) {
+    this.blockId = new LongWritable(blockid);
+    this.blockGenerationStamp = new LongWritable(blockgenerationstamp);
+    this.size = new LongWritable(1);
+  }
+
+  public BlockInfo(long blockid, long blockgenerationstamp, long size) {
+    this.blockId = new LongWritable(blockid);
+    this.blockGenerationStamp = new LongWritable(blockgenerationstamp);
+    this.size = new LongWritable(size);
+  }
+
+  public BlockInfo(long blockid, long blockgenerationstamp, long size,
+      short replication) {
+    this.blockId = new LongWritable(blockid);
+    this.blockGenerationStamp = new LongWritable(blockgenerationstamp);
+    this.size = new LongWritable(size);
+    this.replication = replication;
+  }
+
+  public void write(DataOutput dataOutput) throws IOException {
+    blockId.write(dataOutput);
+    blockGenerationStamp.write(dataOutput);
+    size.write(dataOutput);
+  }
+
+  public void readFields(DataInput dataInput) throws IOException {
+    blockId.readFields(dataInput);
+    blockGenerationStamp.readFields(dataInput);
+    size.readFields(dataInput);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof BlockInfo)) {
+      return false;
+    }
+    BlockInfo blkInfo = (BlockInfo) o;
+    return blkInfo.getBlockId().equals(this.getBlockId())
+        && blkInfo.getBlockGenerationStamp()
+            .equals(this.getBlockGenerationStamp())
+        && blkInfo.getSize().equals(this.getSize());
+  }
+
+  @Override
+  public int hashCode() {
+    return blockId.hashCode() + 357 * blockGenerationStamp.hashCode()
+        + 9357 * size.hashCode();
+  }
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/GenerateBlockImagesDriver.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/GenerateBlockImagesDriver.java
new file mode 100644
index 0000000..613a60f
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/GenerateBlockImagesDriver.java
@@ -0,0 +1,140 @@
+/**
+ * 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.hadoop.tools.dynamometer.blockgenerator;
+
+import java.net.URI;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.PosixParser;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * This is the main driver class. It takes in the following arguments: -
+ * Required: input path of the fsImage from the HDFS cluster to be simulated -
+ * Required: output path for generated block image files for each Dynamometer
+ * DataNode - Required: Number of DataNodes to generate blocks for - Optional:
+ * Number of reducers to use for the job (defaults to number of DataNodes)
+ */
+
+public class GenerateBlockImagesDriver extends Configured implements Tool {
+
+  public static final String FSIMAGE_INPUT_PATH_ARG = "fsimage_input_path";
+  public static final String BLOCK_IMAGE_OUTPUT_ARG = "block_image_output_dir";
+  public static final String NUM_REDUCERS_ARG = "num_reducers";
+  public static final String NUM_DATANODES_ARG = "num_datanodes";
+
+  public static final String NUM_DATANODES_KEY = "dyno.blockgen.num.datanodes";
+
+  public GenerateBlockImagesDriver(Configuration conf) {
+    setConf(conf);
+  }
+
+  public int run(String[] args) throws Exception {
+    Options options = new Options();
+    options.addOption("h", "help", false, "Shows this message");
+    options.addOption(OptionBuilder.withArgName("Input path of the XML fsImage")
+        .hasArg().isRequired(true)
+        .withDescription("Input path to the Hadoop fsImage XML file (required)")
+        .create(FSIMAGE_INPUT_PATH_ARG));
+    options.addOption(OptionBuilder.withArgName("BlockImage output directory")
+        .hasArg().isRequired(true)
+        .withDescription("Directory where the generated files containing the "
+            + "block listing for each DataNode should be stored (required)")
+        .create(BLOCK_IMAGE_OUTPUT_ARG));
+    options.addOption(OptionBuilder.withArgName("Number of reducers").hasArg()
+        .isRequired(false)
+        .withDescription(
+            "Number of reducers for this job (defaults to number of datanodes)")
+        .create(NUM_REDUCERS_ARG));
+    options.addOption(OptionBuilder.withArgName("Number of datanodes").hasArg()
+        .isRequired(true)
+        .withDescription("Number of DataNodes to create blocks for (required)")
+        .create(NUM_DATANODES_ARG));
+
+    CommandLineParser parser = new PosixParser();
+    CommandLine cli = parser.parse(options, args);
+    if (cli.hasOption("h")) {
+      HelpFormatter formatter = new HelpFormatter();
+      formatter.printHelp(200, "hadoop jar dynamometer-*.jar "
+              + "org.apache.hadoop.tools.dynamometer.blockgenerator."
+              + "GenerateBlockImagesDriver [options]",
+          null, options, null);
+      return 0;
+    }
+
+    String fsImageInputPath = cli.getOptionValue(FSIMAGE_INPUT_PATH_ARG);
+    String blockImageOutputDir = cli.getOptionValue(BLOCK_IMAGE_OUTPUT_ARG);
+    int numDataNodes = Integer.parseInt(cli.getOptionValue(NUM_DATANODES_ARG));
+    int numReducers = Integer.parseInt(
+        cli.getOptionValue(NUM_REDUCERS_ARG, String.valueOf(numDataNodes)));
+
+    FileSystem fs = FileSystem.get(new URI(blockImageOutputDir), getConf());
+    Job job = Job.getInstance(getConf(), "Create blocksImages for Dynamometer");
+    FileInputFormat.setInputPaths(job, new Path(fsImageInputPath));
+    Path blockImagesDir = new Path(blockImageOutputDir);
+    fs.delete(blockImagesDir, true);
+    FileOutputFormat.setOutputPath(job, blockImagesDir);
+    job.getConfiguration().setInt(NUM_DATANODES_KEY, numDataNodes);
+
+    job.setJarByClass(GenerateBlockImagesDriver.class);
+    job.setInputFormatClass(NoSplitTextInputFormat.class);
+    job.setNumReduceTasks(numReducers);
+    LazyOutputFormat.setOutputFormatClass(job, TextOutputFormat.class);
+    job.setMapperClass(XMLParserMapper.class);
+    job.setReducerClass(GenerateDNBlockInfosReducer.class);
+    job.setMapOutputKeyClass(IntWritable.class);
+    job.setMapOutputValueClass(BlockInfo.class);
+    job.setOutputKeyClass(NullWritable.class);
+    job.setOutputValueClass(Text.class);
+
+    boolean success = job.waitForCompletion(true);
+    return success ? 0 : 1;
+  }
+
+  public static void main(String[] args) throws Exception {
+    GenerateBlockImagesDriver driver = new GenerateBlockImagesDriver(
+        new Configuration());
+    System.exit(ToolRunner.run(driver, args));
+  }
+
+  /** A simple text input format that doesn't allow splitting of files. */
+  public static class NoSplitTextInputFormat extends TextInputFormat {
+    @Override
+    public boolean isSplitable(JobContext context, Path file) {
+      return false;
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/GenerateDNBlockInfosReducer.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/GenerateDNBlockInfosReducer.java
new file mode 100644
index 0000000..3d32cc1
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/GenerateDNBlockInfosReducer.java
@@ -0,0 +1,99 @@
+/**
+ * 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.hadoop.tools.dynamometer.blockgenerator;
+
+import java.io.IOException;
+import java.util.Iterator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.MultipleOutputs;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This Reducer class generates a simple text file for each DN, listing the
+ * blocks to be generated.
+ *
+ * Input: {@link BlockInfo}'s from {@link XMLParserMapper}
+ *
+ * Output: A text file named as dni-XXX, where i is the ID of the DN and XXX is
+ * a reducer ID. Each line in the file is in format:
+ * blockID,blockGenStamp,blockSize
+ */
+public class GenerateDNBlockInfosReducer
+    extends Reducer<IntWritable, BlockInfo, NullWritable, Text> {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(GenerateDNBlockInfosReducer.class);
+
+  private MultipleOutputs<NullWritable, Text> multiOutputs;
+
+  @Override
+  public void setup(
+      Reducer<IntWritable, BlockInfo, NullWritable, Text>.Context context) {
+    multiOutputs = new MultipleOutputs<>(context);
+  }
+
+  @Override
+  public void cleanup(Context context)
+      throws IOException, InterruptedException {
+    multiOutputs.close();
+    multiOutputs = null;
+  }
+
+  @Override
+  public void reduce(IntWritable key, Iterable<BlockInfo> values,
+      Context context) throws IOException, InterruptedException {
+    long blockIndex = 0;
+    int datanodeId = key.get();
+    String dnFile = "dn" + datanodeId + "-a-"
+        + context.getTaskAttemptID().getId();
+    Iterator<BlockInfo> it = values.iterator();
+    long startTimestamp = System.currentTimeMillis();
+    long endTimestamp;
+
+    Path baseOutputPath = FileOutputFormat.getOutputPath(context);
+    String fullPath = new Path(baseOutputPath, dnFile).toString();
+
+    Text out = new Text();
+    while (it.hasNext()) {
+      BlockInfo blockInfo = new BlockInfo(it.next());
+      String blockLine = blockInfo.getBlockId() + ","
+          + blockInfo.getBlockGenerationStamp() + "," + blockInfo.getSize();
+      out.set(blockLine);
+      multiOutputs.write(NullWritable.get(), out, fullPath);
+
+      blockIndex++;
+
+      // Report progress for every 1000 blocks
+      if (blockIndex % 1000 == 0) {
+        context.progress();
+        endTimestamp = System.currentTimeMillis();
+        System.out.println("Time taken to process 1000 records in ms:"
+            + (endTimestamp - startTimestamp));
+        startTimestamp = endTimestamp;
+      }
+    }
+
+    LOG.info("Number of blocks processed:" + blockIndex);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/XMLParser.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/XMLParser.java
new file mode 100644
index 0000000..d7f7977
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/XMLParser.java
@@ -0,0 +1,154 @@
+/**
+ * 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.hadoop.tools.dynamometer.blockgenerator;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * This class parses an fsimage file in XML format. It accepts the file
+ * line-by-line and maintains an internal state machine to keep track of
+ * contextual information. A single parser must process the entire file with the
+ * lines in the order they appear in the original file.
+ *
+ * A file may be spread across multiple lines, so we need to track the
+ * replication of the file we are currently processing to be aware of what the
+ * replication factor is for each block we encounter. This is why we require a
+ * single mapper.
+ *
+ * The format is illustrated below (line breaks for readability):
+ * <pre>{@code
+ * <inode><id>inode_ID<id/> <type>inode_type</type>
+ * <replication>inode_replication</replication> [file attributes] <blocks>
+ * <block><id>XXX</id><genstamp>XXX</genstamp><numBytes>XXX</numBytes><block/>
+ * <blocks/> <inode/>
+ * }</pre>
+ *
+ * This is true in both Hadoop 2 and 3.
+ */
+class XMLParser {
+
+  private static final Pattern BLOCK_PATTERN = Pattern.compile("<block>"
+      + "<id>(\\d+)</id>"
+      + "<genstamp>(\\d+)</genstamp>"
+      + "<numBytes>(\\d+)</numBytes>"
+      + "</block>");
+
+  private State currentState = State.DEFAULT;
+  private short currentReplication;
+
+  /**
+   * Accept a single line of the XML file, and return a {@link BlockInfo} for
+   * any blocks contained within that line. Update internal state dependent on
+   * other XML values seen, e.g. the beginning of a file.
+   *
+   * @param line The XML line to parse.
+   * @return {@code BlockInfo}s for any blocks found.
+   */
+  List<BlockInfo> parseLine(String line) throws IOException {
+    if (line.contains("<inode>")) {
+      transitionTo(State.INODE);
+    }
+    if (line.contains("<type>FILE</type>")) {
+      transitionTo(State.FILE);
+    }
+    List<String> replicationStrings = valuesFromXMLString(line, "replication");
+    if (!replicationStrings.isEmpty()) {
+      if (replicationStrings.size() > 1) {
+        throw new IOException(String.format("Found %s replication strings",
+            replicationStrings.size()));
+      }
+      transitionTo(State.FILE_WITH_REPLICATION);
+      currentReplication = Short.parseShort(replicationStrings.get(0));
+    }
+    Matcher blockMatcher = BLOCK_PATTERN.matcher(line);
+    List<BlockInfo> blockInfos = new ArrayList<>();
+    while (blockMatcher.find()) {
+      if (currentState != State.FILE_WITH_REPLICATION) {
+        throw new IOException(
+            "Found a block string when in state: " + currentState);
+      }
+      long id = Long.parseLong(blockMatcher.group(1));
+      long gs = Long.parseLong(blockMatcher.group(2));
+      long size = Long.parseLong(blockMatcher.group(3));
+      blockInfos.add(new BlockInfo(id, gs, size, currentReplication));
+    }
+    if (line.contains("</inode>")) {
+      transitionTo(State.DEFAULT);
+    }
+    return blockInfos;
+  }
+
+  /**
+   * Attempt to transition to another state.
+   *
+   * @param nextState The new state to transition to.
+   * @throws IOException If the transition from the current state to
+   *                     {@code nextState} is not allowed.
+   */
+  private void transitionTo(State nextState) throws IOException {
+    if (currentState.transitionAllowed(nextState)) {
+      currentState = nextState;
+    } else {
+      throw new IOException("State transition not allowed; from " + currentState
+          + " to " + nextState);
+    }
+  }
+
+  /**
+   * @param xml An XML string
+   * @param field The field whose value(s) should be extracted
+   * @return List of the field's values.
+   */
+  private static List<String> valuesFromXMLString(String xml, String field) {
+    Matcher m = Pattern.compile("<" + field + ">(.+?)</" + field + ">")
+        .matcher(xml);
+    List<String> found = new ArrayList<>();
+    while (m.find()) {
+      found.add(m.group(1));
+    }
+    return found;
+  }
+
+  private enum State {
+    DEFAULT, INODE, FILE, FILE_WITH_REPLICATION;
+
+    private final Set<State> allowedTransitions = new HashSet<>();
+    static {
+      DEFAULT.addTransitions(DEFAULT, INODE);
+      INODE.addTransitions(DEFAULT, FILE);
+      FILE.addTransitions(DEFAULT, FILE_WITH_REPLICATION);
+      FILE_WITH_REPLICATION.addTransitions(DEFAULT);
+    }
+
+    private void addTransitions(State... nextState) {
+      allowedTransitions.addAll(Arrays.asList(nextState));
+    }
+
+    boolean transitionAllowed(State nextState) {
+      return allowedTransitions.contains(nextState);
+    }
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/XMLParserMapper.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/XMLParserMapper.java
new file mode 100644
index 0000000..29c9fb3
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/XMLParserMapper.java
@@ -0,0 +1,81 @@
+/**
+ * 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.hadoop.tools.dynamometer.blockgenerator;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This Mapper class generates a list of {@link BlockInfo}'s from a given
+ * fsimage.
+ *
+ * Input: fsimage in XML format. It should be generated using
+ * {@code org.apache.hadoop.hdfs.tools.offlineImageViewer.OfflineImageViewer}.
+ *
+ * Output: list of all {@link BlockInfo}'s
+ */
+public class XMLParserMapper
+    extends Mapper<LongWritable, Text, IntWritable, BlockInfo> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(XMLParserMapper.class);
+
+  @Override
+  public void setup(Mapper.Context context) {
+    Configuration conf = context.getConfiguration();
+    numDataNodes = conf.getInt(GenerateBlockImagesDriver.NUM_DATANODES_KEY, -1);
+    parser = new XMLParser();
+  }
+
+  // Blockindexes should be generated serially
+  private int blockIndex = 0;
+  private int numDataNodes;
+  private XMLParser parser;
+
+  /**
+   * Read the input XML file line by line, and generate list of blocks. The
+   * actual parsing logic is handled by {@link XMLParser}. This mapper just
+   * delegates to that class and then writes the blocks to the corresponding
+   * index to be processed by reducers.
+   */
+  @Override
+  public void map(LongWritable lineNum, Text line,
+      Mapper<LongWritable, Text, IntWritable, BlockInfo>.Context context)
+      throws IOException, InterruptedException {
+    List<BlockInfo> blockInfos = parser.parseLine(line.toString());
+    for (BlockInfo blockInfo : blockInfos) {
+      for (short i = 0; i < blockInfo.getReplication(); i++) {
+        context.write(new IntWritable((blockIndex + i) % numDataNodes),
+            blockInfo);
+      }
+
+      blockIndex++;
+      if (blockIndex % 1000000 == 0) {
+        LOG.info("Processed " + blockIndex + " blocks");
+      }
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/package-info.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/package-info.java
new file mode 100644
index 0000000..2974432
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/main/java/org/apache/hadoop/tools/dynamometer/blockgenerator/package-info.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.
+ */
+
+/**
+ * Tools to help generate block files as input to Dynamometer.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.tools.dynamometer.blockgenerator;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/test/java/org/apache/hadoop/tools/dynamometer/blockgenerator/TestBlockGen.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/test/java/org/apache/hadoop/tools/dynamometer/blockgenerator/TestBlockGen.java
new file mode 100644
index 0000000..61c6dce
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/test/java/org/apache/hadoop/tools/dynamometer/blockgenerator/TestBlockGen.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.hadoop.tools.dynamometer.blockgenerator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.assertEquals;
+
+
+/** Tests for block generation via {@link GenerateBlockImagesDriver}. */
+public class TestBlockGen {
+  private static final Logger LOG = LoggerFactory.getLogger(TestBlockGen.class);
+
+  private MiniDFSCluster dfsCluster;
+  private FileSystem fs;
+  private static final String FS_IMAGE_NAME = "fsimage_0000000000000061740.xml";
+  private static final String BLOCK_LIST_OUTPUT_DIR_NAME = "blockLists";
+  private Path tmpPath;
+
+  @Before
+  public void setup() throws Exception {
+    Configuration conf = new Configuration();
+    dfsCluster = new MiniDFSCluster.Builder(conf).build();
+    dfsCluster.waitActive();
+    LOG.info("Started MiniDFSCluster");
+    fs = dfsCluster.getFileSystem();
+    FileSystem.setDefaultUri(conf, fs.getUri());
+    tmpPath = fs.makeQualified(new Path("/tmp"));
+    fs.mkdirs(tmpPath);
+    String fsImageFile = this.getClass().getClassLoader()
+        .getResource(FS_IMAGE_NAME).getPath();
+
+    fs.copyFromLocalFile(new Path(fsImageFile),
+        new Path(tmpPath, FS_IMAGE_NAME));
+  }
+
+  @After
+  public void cleanUp() {
+    dfsCluster.shutdown();
+  }
+
+  @Test
+  public void testBlockGen() throws Exception {
+    LOG.info("Started test");
+
+    int datanodeCount = 40;
+
+    GenerateBlockImagesDriver driver = new GenerateBlockImagesDriver(
+        new Configuration());
+    driver.run(
+        new String[] {"-" + GenerateBlockImagesDriver.FSIMAGE_INPUT_PATH_ARG,
+            new Path(tmpPath, FS_IMAGE_NAME).toString(),
+            "-" + GenerateBlockImagesDriver.BLOCK_IMAGE_OUTPUT_ARG,
+            new Path(tmpPath, BLOCK_LIST_OUTPUT_DIR_NAME).toString(),
+            "-" + GenerateBlockImagesDriver.NUM_DATANODES_ARG,
+            String.valueOf(datanodeCount)});
+
+    for (int i = 0; i < datanodeCount; i++) {
+      final int idx = i;
+      assertEquals(1, fs.listStatus(
+          new Path(tmpPath, BLOCK_LIST_OUTPUT_DIR_NAME),
+          (path) -> path.getName().startsWith(String.format("dn%d-", idx))
+      ).length);
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/test/java/org/apache/hadoop/tools/dynamometer/blockgenerator/TestXMLParser.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/test/java/org/apache/hadoop/tools/dynamometer/blockgenerator/TestXMLParser.java
new file mode 100644
index 0000000..6f1dd7d
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/test/java/org/apache/hadoop/tools/dynamometer/blockgenerator/TestXMLParser.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.hadoop.tools.dynamometer.blockgenerator;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+
+/** Tests for {@link XMLParser}. */
+public class TestXMLParser {
+
+  /**
+   * Testing whether {@link XMLParser} correctly parses an XML fsimage file into
+   * {@link BlockInfo}'s. Note that some files have multiple lines.
+   */
+  @Test
+  public void testBlocksFromLine() throws Exception {
+    String[] lines = {
+        "<INodeSection><lastInodeId>1"
+            + "</lastInodeId><inode><id>2</id><type>FILE</type>"
+            + "<name>fake-file</name>"
+            + "<replication>3</replication><mtime>3</mtime>"
+            + "<atime>4</atime>" + "<perferredBlockSize>5</perferredBlockSize>"
+            + "<permission>hdfs:hdfs:rw-------</permission>"
+            + "<blocks><block><id>6</id><genstamp>7</genstamp>"
+            + "<numBytes>8</numBytes></block>"
+            + "<block><id>9</id><genstamp>10</genstamp>"
+            + "<numBytes>11</numBytes></block></inode>",
+        "<inode><type>DIRECTORY</type></inode>", "<inode><type>FILE</type>",
+        "<replication>12</replication>",
+        "<blocks><block><id>13</id><genstamp>14</genstamp>"
+            + "<numBytes>15</numBytes></block>",
+        "</inode>"
+    };
+
+    Map<BlockInfo, Short> expectedBlockCount = new HashMap<>();
+    expectedBlockCount.put(new BlockInfo(6, 7, 8), (short) 3);
+    expectedBlockCount.put(new BlockInfo(9, 10, 11), (short) 3);
+    expectedBlockCount.put(new BlockInfo(13, 14, 15), (short) 12);
+
+    final Map<BlockInfo, Short> actualBlockCount = new HashMap<>();
+    XMLParser parser = new XMLParser();
+    for (String line : lines) {
+      for (BlockInfo info : parser.parseLine(line)) {
+        actualBlockCount.put(info, info.getReplication());
+      }
+    }
+
+    for (Map.Entry<BlockInfo, Short> expect : expectedBlockCount.entrySet()) {
+      assertEquals(expect.getValue(), actualBlockCount.get(expect.getKey()));
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/test/resources/fsimage_0000000000000061740.xml b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/test/resources/fsimage_0000000000000061740.xml
new file mode 100644
index 0000000..fea4d12
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-blockgen/src/test/resources/fsimage_0000000000000061740.xml
@@ -0,0 +1,553 @@
+<?xml version="1.0"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<fsimage><NameSection>
+<genstampV1>1000</genstampV1><genstampV2>11100</genstampV2><genstampV1Limit>0</genstampV1Limit><lastAllocatedBlockId>1073751924</lastAllocatedBlockId><txid>61740</txid></NameSection>
+<INodeSection><lastInodeId>26700</lastInodeId><inode><id>16385</id><type>DIRECTORY</type><name></name><mtime>1494453394091</mtime><permission>hdfs:hdfs:rwxr-xr-x</permission><nsquota>9223372036854775807</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>16386</id><type>DIRECTORY</type><name>tmp</name><mtime>1494453775668</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>16387</id><type>DIRECTORY</type><name>data</name><mtime>1494453387057</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>16388</id><type>DIRECTORY</type><name>user</name><mtime>1494453399239</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>16389</id><type>DIRECTORY</type><name>hdfs</name><mtime>1494453399239</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>16390</id><type>DIRECTORY</type><name>files</name><mtime>1494614176953</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26491</id><type>DIRECTORY</type><name>0</name><mtime>1494614175397</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26492</id><type>DIRECTORY</type><name>0</name><mtime>1494614175198</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26493</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175183</mtime><atime>1494614175049</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751825</id><genstamp>11001</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26494</id><type>DIRECTORY</type><name>1</name><mtime>1494614175224</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26495</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175222</mtime><atime>1494614175212</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751826</id><genstamp>11002</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26496</id><type>DIRECTORY</type><name>2</name><mtime>1494614175259</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26497</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175257</mtime><atime>1494614175234</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751827</id><genstamp>11003</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26498</id><type>DIRECTORY</type><name>3</name><mtime>1494614175280</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26499</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175278</mtime><atime>1494614175269</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751828</id><genstamp>11004</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26500</id><type>DIRECTORY</type><name>4</name><mtime>1494614175302</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26501</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175299</mtime><atime>1494614175291</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751829</id><genstamp>11005</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26502</id><type>DIRECTORY</type><name>5</name><mtime>1494614175323</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26503</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175321</mtime><atime>1494614175312</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751830</id><genstamp>11006</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26504</id><type>DIRECTORY</type><name>6</name><mtime>1494614175345</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26505</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175343</mtime><atime>1494614175333</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751831</id><genstamp>11007</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26506</id><type>DIRECTORY</type><name>7</name><mtime>1494614175366</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26507</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175363</mtime><atime>1494614175355</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751832</id><genstamp>11008</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26508</id><type>DIRECTORY</type><name>8</name><mtime>1494614175387</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26509</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175385</mtime><atime>1494614175376</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751833</id><genstamp>11009</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26510</id><type>DIRECTORY</type><name>9</name><mtime>1494614175413</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26511</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175410</mtime><atime>1494614175402</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751834</id><genstamp>11010</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26512</id><type>DIRECTORY</type><name>1</name><mtime>1494614175626</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26513</id><type>DIRECTORY</type><name>0</name><mtime>1494614175447</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26514</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175445</mtime><atime>1494614175437</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751835</id><genstamp>11011</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26515</id><type>DIRECTORY</type><name>1</name><mtime>1494614175467</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26516</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175464</mtime><atime>1494614175456</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751836</id><genstamp>11012</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26517</id><type>DIRECTORY</type><name>2</name><mtime>1494614175486</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26518</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175484</mtime><atime>1494614175475</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751837</id><genstamp>11013</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26519</id><type>DIRECTORY</type><name>3</name><mtime>1494614175514</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26520</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175512</mtime><atime>1494614175496</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751838</id><genstamp>11014</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26521</id><type>DIRECTORY</type><name>4</name><mtime>1494614175533</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26522</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175531</mtime><atime>1494614175523</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751839</id><genstamp>11015</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26523</id><type>DIRECTORY</type><name>5</name><mtime>1494614175552</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26524</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175549</mtime><atime>1494614175540</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751840</id><genstamp>11016</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26525</id><type>DIRECTORY</type><name>6</name><mtime>1494614175574</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26526</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175572</mtime><atime>1494614175563</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751841</id><genstamp>11017</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26527</id><type>DIRECTORY</type><name>7</name><mtime>1494614175603</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26528</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175592</mtime><atime>1494614175583</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751842</id><genstamp>11018</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26529</id><type>DIRECTORY</type><name>8</name><mtime>1494614175622</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26530</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175620</mtime><atime>1494614175612</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751843</id><genstamp>11019</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26531</id><type>DIRECTORY</type><name>9</name><mtime>1494614175643</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26532</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175640</mtime><atime>1494614175631</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751844</id><genstamp>11020</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26533</id><type>DIRECTORY</type><name>2</name><mtime>1494614175829</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26534</id><type>DIRECTORY</type><name>0</name><mtime>1494614175669</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26535</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175666</mtime><atime>1494614175658</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751845</id><genstamp>11021</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26536</id><type>DIRECTORY</type><name>1</name><mtime>1494614175688</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26537</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175686</mtime><atime>1494614175678</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751846</id><genstamp>11022</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26538</id><type>DIRECTORY</type><name>2</name><mtime>1494614175708</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26539</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175705</mtime><atime>1494614175697</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751847</id><genstamp>11023</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26540</id><type>DIRECTORY</type><name>3</name><mtime>1494614175727</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26541</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175725</mtime><atime>1494614175716</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751848</id><genstamp>11024</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26542</id><type>DIRECTORY</type><name>4</name><mtime>1494614175746</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26543</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175744</mtime><atime>1494614175736</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751849</id><genstamp>11025</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26544</id><type>DIRECTORY</type><name>5</name><mtime>1494614175772</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26545</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175770</mtime><atime>1494614175755</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751850</id><genstamp>11026</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26546</id><type>DIRECTORY</type><name>6</name><mtime>1494614175790</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26547</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175788</mtime><atime>1494614175780</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751851</id><genstamp>11027</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26548</id><type>DIRECTORY</type><name>7</name><mtime>1494614175808</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26549</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175806</mtime><atime>1494614175798</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751852</id><genstamp>11028</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26550</id><type>DIRECTORY</type><name>8</name><mtime>1494614175826</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26551</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175823</mtime><atime>1494614175816</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751853</id><genstamp>11029</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26552</id><type>DIRECTORY</type><name>9</name><mtime>1494614175853</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26553</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175842</mtime><atime>1494614175834</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751854</id><genstamp>11030</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26554</id><type>DIRECTORY</type><name>3</name><mtime>1494614176026</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26555</id><type>DIRECTORY</type><name>0</name><mtime>1494614175875</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26556</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175873</mtime><atime>1494614175865</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751855</id><genstamp>11031</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26557</id><type>DIRECTORY</type><name>1</name><mtime>1494614175893</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26558</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175891</mtime><atime>1494614175883</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751856</id><genstamp>11032</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26559</id><type>DIRECTORY</type><name>2</name><mtime>1494614175911</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26560</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175908</mtime><atime>1494614175901</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751857</id><genstamp>11033</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26561</id><type>DIRECTORY</type><name>3</name><mtime>1494614175930</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26562</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175927</mtime><atime>1494614175919</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751858</id><genstamp>11034</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26563</id><type>DIRECTORY</type><name>4</name><mtime>1494614175947</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26564</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175946</mtime><atime>1494614175938</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751859</id><genstamp>11035</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26565</id><type>DIRECTORY</type><name>5</name><mtime>1494614175964</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26566</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175962</mtime><atime>1494614175955</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751860</id><genstamp>11036</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26567</id><type>DIRECTORY</type><name>6</name><mtime>1494614175980</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26568</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175978</mtime><atime>1494614175971</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751861</id><genstamp>11037</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26569</id><type>DIRECTORY</type><name>7</name><mtime>1494614176000</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26570</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614175997</mtime><atime>1494614175989</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751862</id><genstamp>11038</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26571</id><type>DIRECTORY</type><name>8</name><mtime>1494614176018</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26572</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176016</mtime><atime>1494614176009</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751863</id><genstamp>11039</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26573</id><type>DIRECTORY</type><name>9</name><mtime>1494614176040</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26574</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176038</mtime><atime>1494614176031</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751864</id><genstamp>11040</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26575</id><type>DIRECTORY</type><name>4</name><mtime>1494614176214</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26576</id><type>DIRECTORY</type><name>0</name><mtime>1494614176063</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26577</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176061</mtime><atime>1494614176053</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751865</id><genstamp>11041</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26578</id><type>DIRECTORY</type><name>1</name><mtime>1494614176081</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26579</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176080</mtime><atime>1494614176071</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751866</id><genstamp>11042</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26580</id><type>DIRECTORY</type><name>2</name><mtime>1494614176101</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26581</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176098</mtime><atime>1494614176089</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751867</id><genstamp>11043</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26582</id><type>DIRECTORY</type><name>3</name><mtime>1494614176118</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26583</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176115</mtime><atime>1494614176108</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751868</id><genstamp>11044</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26584</id><type>DIRECTORY</type><name>4</name><mtime>1494614176136</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26585</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176134</mtime><atime>1494614176127</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751869</id><genstamp>11045</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26586</id><type>DIRECTORY</type><name>5</name><mtime>1494614176153</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26587</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176151</mtime><atime>1494614176144</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751870</id><genstamp>11046</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26588</id><type>DIRECTORY</type><name>6</name><mtime>1494614176178</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26589</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176176</mtime><atime>1494614176160</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751871</id><genstamp>11047</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26590</id><type>DIRECTORY</type><name>7</name><mtime>1494614176194</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26591</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176193</mtime><atime>1494614176186</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751872</id><genstamp>11048</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26592</id><type>DIRECTORY</type><name>8</name><mtime>1494614176211</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26593</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176209</mtime><atime>1494614176201</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751873</id><genstamp>11049</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26594</id><type>DIRECTORY</type><name>9</name><mtime>1494614176228</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26595</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176226</mtime><atime>1494614176219</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751874</id><genstamp>11050</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26596</id><type>DIRECTORY</type><name>5</name><mtime>1494614176388</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26597</id><type>DIRECTORY</type><name>0</name><mtime>1494614176249</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26598</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176247</mtime><atime>1494614176240</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751875</id><genstamp>11051</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26599</id><type>DIRECTORY</type><name>1</name><mtime>1494614176266</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26600</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176264</mtime><atime>1494614176257</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751876</id><genstamp>11052</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26601</id><type>DIRECTORY</type><name>2</name><mtime>1494614176288</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26602</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176286</mtime><atime>1494614176274</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751877</id><genstamp>11053</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26603</id><type>DIRECTORY</type><name>3</name><mtime>1494614176304</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26604</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176303</mtime><atime>1494614176295</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751878</id><genstamp>11054</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26605</id><type>DIRECTORY</type><name>4</name><mtime>1494614176320</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26606</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176318</mtime><atime>1494614176311</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751879</id><genstamp>11055</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26607</id><type>DIRECTORY</type><name>5</name><mtime>1494614176336</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26608</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176334</mtime><atime>1494614176327</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751880</id><genstamp>11056</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26609</id><type>DIRECTORY</type><name>6</name><mtime>1494614176354</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26610</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176351</mtime><atime>1494614176343</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751881</id><genstamp>11057</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26611</id><type>DIRECTORY</type><name>7</name><mtime>1494614176369</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26612</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176368</mtime><atime>1494614176360</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751882</id><genstamp>11058</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26613</id><type>DIRECTORY</type><name>8</name><mtime>1494614176385</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26614</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176383</mtime><atime>1494614176376</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751883</id><genstamp>11059</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26615</id><type>DIRECTORY</type><name>9</name><mtime>1494614176401</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26616</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176399</mtime><atime>1494614176392</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751884</id><genstamp>11060</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26617</id><type>DIRECTORY</type><name>6</name><mtime>1494614176584</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26618</id><type>DIRECTORY</type><name>0</name><mtime>1494614176432</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26619</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176421</mtime><atime>1494614176413</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751885</id><genstamp>11061</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26620</id><type>DIRECTORY</type><name>1</name><mtime>1494614176450</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26621</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176448</mtime><atime>1494614176440</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751886</id><genstamp>11062</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26622</id><type>DIRECTORY</type><name>2</name><mtime>1494614176468</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26623</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176465</mtime><atime>1494614176457</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751887</id><genstamp>11063</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26624</id><type>DIRECTORY</type><name>3</name><mtime>1494614176486</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26625</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176483</mtime><atime>1494614176475</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751888</id><genstamp>11064</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26626</id><type>DIRECTORY</type><name>4</name><mtime>1494614176503</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26627</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176501</mtime><atime>1494614176493</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751889</id><genstamp>11065</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26628</id><type>DIRECTORY</type><name>5</name><mtime>1494614176519</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26629</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176517</mtime><atime>1494614176511</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751890</id><genstamp>11066</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26630</id><type>DIRECTORY</type><name>6</name><mtime>1494614176550</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26631</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176540</mtime><atime>1494614176526</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751891</id><genstamp>11067</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26632</id><type>DIRECTORY</type><name>7</name><mtime>1494614176566</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26633</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176564</mtime><atime>1494614176557</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751892</id><genstamp>11068</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26634</id><type>DIRECTORY</type><name>8</name><mtime>1494614176580</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26635</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176578</mtime><atime>1494614176572</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751893</id><genstamp>11069</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26636</id><type>DIRECTORY</type><name>9</name><mtime>1494614176596</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26637</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176594</mtime><atime>1494614176588</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751894</id><genstamp>11070</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26638</id><type>DIRECTORY</type><name>7</name><mtime>1494614176756</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26639</id><type>DIRECTORY</type><name>0</name><mtime>1494614176616</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26640</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176613</mtime><atime>1494614176606</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751895</id><genstamp>11071</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26641</id><type>DIRECTORY</type><name>1</name><mtime>1494614176631</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26642</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176629</mtime><atime>1494614176622</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751896</id><genstamp>11072</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26643</id><type>DIRECTORY</type><name>2</name><mtime>1494614176647</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26644</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176645</mtime><atime>1494614176638</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751897</id><genstamp>11073</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26645</id><type>DIRECTORY</type><name>3</name><mtime>1494614176662</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26646</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176660</mtime><atime>1494614176653</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751898</id><genstamp>11074</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26647</id><type>DIRECTORY</type><name>4</name><mtime>1494614176678</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26648</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176676</mtime><atime>1494614176669</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751899</id><genstamp>11075</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26649</id><type>DIRECTORY</type><name>5</name><mtime>1494614176703</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26650</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176701</mtime><atime>1494614176694</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751900</id><genstamp>11076</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26651</id><type>DIRECTORY</type><name>6</name><mtime>1494614176719</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26652</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176717</mtime><atime>1494614176710</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751901</id><genstamp>11077</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26653</id><type>DIRECTORY</type><name>7</name><mtime>1494614176736</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26654</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176734</mtime><atime>1494614176726</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751902</id><genstamp>11078</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26655</id><type>DIRECTORY</type><name>8</name><mtime>1494614176753</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26656</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176751</mtime><atime>1494614176743</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751903</id><genstamp>11079</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26657</id><type>DIRECTORY</type><name>9</name><mtime>1494614176770</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26658</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176768</mtime><atime>1494614176761</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751904</id><genstamp>11080</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26659</id><type>DIRECTORY</type><name>8</name><mtime>1494614176936</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26660</id><type>DIRECTORY</type><name>0</name><mtime>1494614176791</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26661</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176789</mtime><atime>1494614176782</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751905</id><genstamp>11081</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26662</id><type>DIRECTORY</type><name>1</name><mtime>1494614176822</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26663</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176820</mtime><atime>1494614176798</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751906</id><genstamp>11082</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26664</id><type>DIRECTORY</type><name>2</name><mtime>1494614176837</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26665</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176835</mtime><atime>1494614176828</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751907</id><genstamp>11083</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26666</id><type>DIRECTORY</type><name>3</name><mtime>1494614176853</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26667</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176851</mtime><atime>1494614176844</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751908</id><genstamp>11084</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26668</id><type>DIRECTORY</type><name>4</name><mtime>1494614176868</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26669</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176866</mtime><atime>1494614176859</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751909</id><genstamp>11085</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26670</id><type>DIRECTORY</type><name>5</name><mtime>1494614176886</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26671</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176883</mtime><atime>1494614176875</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751910</id><genstamp>11086</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26672</id><type>DIRECTORY</type><name>6</name><mtime>1494614176901</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26673</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176900</mtime><atime>1494614176892</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751911</id><genstamp>11087</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26674</id><type>DIRECTORY</type><name>7</name><mtime>1494614176917</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26675</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176915</mtime><atime>1494614176908</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751912</id><genstamp>11088</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26676</id><type>DIRECTORY</type><name>8</name><mtime>1494614176933</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26677</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176931</mtime><atime>1494614176924</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751913</id><genstamp>11089</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26678</id><type>DIRECTORY</type><name>9</name><mtime>1494614176949</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26679</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176947</mtime><atime>1494614176940</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751914</id><genstamp>11090</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26680</id><type>DIRECTORY</type><name>9</name><mtime>1494614177104</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26681</id><type>DIRECTORY</type><name>0</name><mtime>1494614176970</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26682</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176967</mtime><atime>1494614176961</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751915</id><genstamp>11091</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26683</id><type>DIRECTORY</type><name>1</name><mtime>1494614176984</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26684</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176982</mtime><atime>1494614176976</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751916</id><genstamp>11092</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26685</id><type>DIRECTORY</type><name>2</name><mtime>1494614176999</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26686</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614176998</mtime><atime>1494614176991</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751917</id><genstamp>11093</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26687</id><type>DIRECTORY</type><name>3</name><mtime>1494614177016</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26688</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614177014</mtime><atime>1494614177007</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751918</id><genstamp>11094</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26689</id><type>DIRECTORY</type><name>4</name><mtime>1494614177031</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26690</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614177029</mtime><atime>1494614177022</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751919</id><genstamp>11095</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26691</id><type>DIRECTORY</type><name>5</name><mtime>1494614177046</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26692</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614177044</mtime><atime>1494614177037</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751920</id><genstamp>11096</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26693</id><type>DIRECTORY</type><name>6</name><mtime>1494614177063</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26694</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614177061</mtime><atime>1494614177054</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751921</id><genstamp>11097</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26695</id><type>DIRECTORY</type><name>7</name><mtime>1494614177086</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26696</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614177084</mtime><atime>1494614177077</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751922</id><genstamp>11098</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26697</id><type>DIRECTORY</type><name>8</name><mtime>1494614177101</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26698</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614177099</mtime><atime>1494614177093</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751923</id><genstamp>11099</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+<inode><id>26699</id><type>DIRECTORY</type><name>9</name><mtime>1494614177117</mtime><permission>hdfs:hdfs:rwxr-x---</permission><nsquota>-1</nsquota><dsquota>-1</dsquota></inode>
+<inode><id>26700</id><type>FILE</type><name>foo</name><replication>1</replication><mtime>1494614177115</mtime><atime>1494614177108</atime><perferredBlockSize>536870912</perferredBlockSize><permission>hdfs:hdfs:rw-r-----</permission><blocks><block><id>1073751924</id><genstamp>11100</genstamp><numBytes>4</numBytes></block>
+</blocks>
+</inode>
+</INodeSection>
+<INodeReferenceSection></INodeReferenceSection><SnapshotSection><snapshotCounter>0</snapshotCounter></SnapshotSection>
+<INodeDirectorySection><directory><parent>16385</parent><inode>16387</inode><inode>16386</inode><inode>16388</inode></directory>
+<directory><parent>16386</parent><inode>16390</inode></directory>
+<directory><parent>16388</parent><inode>16389</inode></directory>
+<directory><parent>16390</parent><inode>26491</inode><inode>26512</inode><inode>26533</inode><inode>26554</inode><inode>26575</inode><inode>26596</inode><inode>26617</inode><inode>26638</inode><inode>26659</inode><inode>26680</inode></directory>
+<directory><parent>26491</parent><inode>26492</inode><inode>26494</inode><inode>26496</inode><inode>26498</inode><inode>26500</inode><inode>26502</inode><inode>26504</inode><inode>26506</inode><inode>26508</inode><inode>26510</inode></directory>
+<directory><parent>26492</parent><inode>26493</inode></directory>
+<directory><parent>26494</parent><inode>26495</inode></directory>
+<directory><parent>26496</parent><inode>26497</inode></directory>
+<directory><parent>26498</parent><inode>26499</inode></directory>
+<directory><parent>26500</parent><inode>26501</inode></directory>
+<directory><parent>26502</parent><inode>26503</inode></directory>
+<directory><parent>26504</parent><inode>26505</inode></directory>
+<directory><parent>26506</parent><inode>26507</inode></directory>
+<directory><parent>26508</parent><inode>26509</inode></directory>
+<directory><parent>26510</parent><inode>26511</inode></directory>
+<directory><parent>26512</parent><inode>26513</inode><inode>26515</inode><inode>26517</inode><inode>26519</inode><inode>26521</inode><inode>26523</inode><inode>26525</inode><inode>26527</inode><inode>26529</inode><inode>26531</inode></directory>
+<directory><parent>26513</parent><inode>26514</inode></directory>
+<directory><parent>26515</parent><inode>26516</inode></directory>
+<directory><parent>26517</parent><inode>26518</inode></directory>
+<directory><parent>26519</parent><inode>26520</inode></directory>
+<directory><parent>26521</parent><inode>26522</inode></directory>
+<directory><parent>26523</parent><inode>26524</inode></directory>
+<directory><parent>26525</parent><inode>26526</inode></directory>
+<directory><parent>26527</parent><inode>26528</inode></directory>
+<directory><parent>26529</parent><inode>26530</inode></directory>
+<directory><parent>26531</parent><inode>26532</inode></directory>
+<directory><parent>26533</parent><inode>26534</inode><inode>26536</inode><inode>26538</inode><inode>26540</inode><inode>26542</inode><inode>26544</inode><inode>26546</inode><inode>26548</inode><inode>26550</inode><inode>26552</inode></directory>
+<directory><parent>26534</parent><inode>26535</inode></directory>
+<directory><parent>26536</parent><inode>26537</inode></directory>
+<directory><parent>26538</parent><inode>26539</inode></directory>
+<directory><parent>26540</parent><inode>26541</inode></directory>
+<directory><parent>26542</parent><inode>26543</inode></directory>
+<directory><parent>26544</parent><inode>26545</inode></directory>
+<directory><parent>26546</parent><inode>26547</inode></directory>
+<directory><parent>26548</parent><inode>26549</inode></directory>
+<directory><parent>26550</parent><inode>26551</inode></directory>
+<directory><parent>26552</parent><inode>26553</inode></directory>
+<directory><parent>26554</parent><inode>26555</inode><inode>26557</inode><inode>26559</inode><inode>26561</inode><inode>26563</inode><inode>26565</inode><inode>26567</inode><inode>26569</inode><inode>26571</inode><inode>26573</inode></directory>
+<directory><parent>26555</parent><inode>26556</inode></directory>
+<directory><parent>26557</parent><inode>26558</inode></directory>
+<directory><parent>26559</parent><inode>26560</inode></directory>
+<directory><parent>26561</parent><inode>26562</inode></directory>
+<directory><parent>26563</parent><inode>26564</inode></directory>
+<directory><parent>26565</parent><inode>26566</inode></directory>
+<directory><parent>26567</parent><inode>26568</inode></directory>
+<directory><parent>26569</parent><inode>26570</inode></directory>
+<directory><parent>26571</parent><inode>26572</inode></directory>
+<directory><parent>26573</parent><inode>26574</inode></directory>
+<directory><parent>26575</parent><inode>26576</inode><inode>26578</inode><inode>26580</inode><inode>26582</inode><inode>26584</inode><inode>26586</inode><inode>26588</inode><inode>26590</inode><inode>26592</inode><inode>26594</inode></directory>
+<directory><parent>26576</parent><inode>26577</inode></directory>
+<directory><parent>26578</parent><inode>26579</inode></directory>
+<directory><parent>26580</parent><inode>26581</inode></directory>
+<directory><parent>26582</parent><inode>26583</inode></directory>
+<directory><parent>26584</parent><inode>26585</inode></directory>
+<directory><parent>26586</parent><inode>26587</inode></directory>
+<directory><parent>26588</parent><inode>26589</inode></directory>
+<directory><parent>26590</parent><inode>26591</inode></directory>
+<directory><parent>26592</parent><inode>26593</inode></directory>
+<directory><parent>26594</parent><inode>26595</inode></directory>
+<directory><parent>26596</parent><inode>26597</inode><inode>26599</inode><inode>26601</inode><inode>26603</inode><inode>26605</inode><inode>26607</inode><inode>26609</inode><inode>26611</inode><inode>26613</inode><inode>26615</inode></directory>
+<directory><parent>26597</parent><inode>26598</inode></directory>
+<directory><parent>26599</parent><inode>26600</inode></directory>
+<directory><parent>26601</parent><inode>26602</inode></directory>
+<directory><parent>26603</parent><inode>26604</inode></directory>
+<directory><parent>26605</parent><inode>26606</inode></directory>
+<directory><parent>26607</parent><inode>26608</inode></directory>
+<directory><parent>26609</parent><inode>26610</inode></directory>
+<directory><parent>26611</parent><inode>26612</inode></directory>
+<directory><parent>26613</parent><inode>26614</inode></directory>
+<directory><parent>26615</parent><inode>26616</inode></directory>
+<directory><parent>26617</parent><inode>26618</inode><inode>26620</inode><inode>26622</inode><inode>26624</inode><inode>26626</inode><inode>26628</inode><inode>26630</inode><inode>26632</inode><inode>26634</inode><inode>26636</inode></directory>
+<directory><parent>26618</parent><inode>26619</inode></directory>
+<directory><parent>26620</parent><inode>26621</inode></directory>
+<directory><parent>26622</parent><inode>26623</inode></directory>
+<directory><parent>26624</parent><inode>26625</inode></directory>
+<directory><parent>26626</parent><inode>26627</inode></directory>
+<directory><parent>26628</parent><inode>26629</inode></directory>
+<directory><parent>26630</parent><inode>26631</inode></directory>
+<directory><parent>26632</parent><inode>26633</inode></directory>
+<directory><parent>26634</parent><inode>26635</inode></directory>
+<directory><parent>26636</parent><inode>26637</inode></directory>
+<directory><parent>26638</parent><inode>26639</inode><inode>26641</inode><inode>26643</inode><inode>26645</inode><inode>26647</inode><inode>26649</inode><inode>26651</inode><inode>26653</inode><inode>26655</inode><inode>26657</inode></directory>
+<directory><parent>26639</parent><inode>26640</inode></directory>
+<directory><parent>26641</parent><inode>26642</inode></directory>
+<directory><parent>26643</parent><inode>26644</inode></directory>
+<directory><parent>26645</parent><inode>26646</inode></directory>
+<directory><parent>26647</parent><inode>26648</inode></directory>
+<directory><parent>26649</parent><inode>26650</inode></directory>
+<directory><parent>26651</parent><inode>26652</inode></directory>
+<directory><parent>26653</parent><inode>26654</inode></directory>
+<directory><parent>26655</parent><inode>26656</inode></directory>
+<directory><parent>26657</parent><inode>26658</inode></directory>
+<directory><parent>26659</parent><inode>26660</inode><inode>26662</inode><inode>26664</inode><inode>26666</inode><inode>26668</inode><inode>26670</inode><inode>26672</inode><inode>26674</inode><inode>26676</inode><inode>26678</inode></directory>
+<directory><parent>26660</parent><inode>26661</inode></directory>
+<directory><parent>26662</parent><inode>26663</inode></directory>
+<directory><parent>26664</parent><inode>26665</inode></directory>
+<directory><parent>26666</parent><inode>26667</inode></directory>
+<directory><parent>26668</parent><inode>26669</inode></directory>
+<directory><parent>26670</parent><inode>26671</inode></directory>
+<directory><parent>26672</parent><inode>26673</inode></directory>
+<directory><parent>26674</parent><inode>26675</inode></directory>
+<directory><parent>26676</parent><inode>26677</inode></directory>
+<directory><parent>26678</parent><inode>26679</inode></directory>
+<directory><parent>26680</parent><inode>26681</inode><inode>26683</inode><inode>26685</inode><inode>26687</inode><inode>26689</inode><inode>26691</inode><inode>26693</inode><inode>26695</inode><inode>26697</inode><inode>26699</inode></directory>
+<directory><parent>26681</parent><inode>26682</inode></directory>
+<directory><parent>26683</parent><inode>26684</inode></directory>
+<directory><parent>26685</parent><inode>26686</inode></directory>
+<directory><parent>26687</parent><inode>26688</inode></directory>
+<directory><parent>26689</parent><inode>26690</inode></directory>
+<directory><parent>26691</parent><inode>26692</inode></directory>
+<directory><parent>26693</parent><inode>26694</inode></directory>
+<directory><parent>26695</parent><inode>26696</inode></directory>
+<directory><parent>26697</parent><inode>26698</inode></directory>
+<directory><parent>26699</parent><inode>26700</inode></directory>
+</INodeDirectorySection>
+<FileUnderConstructionSection></FileUnderConstructionSection>
+<SnapshotDiffSection><diff><inodeid>16385</inodeid></diff></SnapshotDiffSection>
+<SecretManagerSection><currentId>0</currentId><tokenSequenceNumber>0</tokenSequenceNumber></SecretManagerSection><CacheManagerSection><nextDirectiveId>1</nextDirectiveId></CacheManagerSection>
+</fsimage>
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-dist/pom.xml b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-dist/pom.xml
new file mode 100644
index 0000000..dbd2395
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-dist/pom.xml
@@ -0,0 +1,128 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<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>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project-dist</artifactId>
+    <version>3.3.0-SNAPSHOT</version>
+    <relativePath>../../../hadoop-project-dist</relativePath>
+  </parent>
+  <artifactId>hadoop-dynamometer-dist</artifactId>
+  <version>3.3.0-SNAPSHOT</version>
+  <description>Apache Hadoop Dynamometer Dist</description>
+  <name>Apache Hadoop Dynamometer Dist</name>
+  <packaging>jar</packaging>
+
+  <properties>
+    <hadoop.component>tools</hadoop.component>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-dynamometer-blockgen</artifactId>
+      <scope>compile</scope>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-dynamometer-workload</artifactId>
+      <scope>compile</scope>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-dynamometer-infra</artifactId>
+      <scope>compile</scope>
+      <version>${project.version}</version>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <artifactId>maven-deploy-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+  <profiles>
+    <profile>
+      <id>dist</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+         <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-assembly-plugin</artifactId>
+            <dependencies>
+              <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-assemblies</artifactId>
+                <version>${project.version}</version>
+              </dependency>
+            </dependencies>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>prepare-package</phase>
+                <goals>
+                  <goal>single</goal>
+                </goals>
+                <configuration>
+                  <appendAssemblyId>false</appendAssemblyId>
+                  <attach>false</attach>
+                  <finalName>${project.artifactId}-${project.version}</finalName>
+                  <descriptorRefs>
+                    <descriptorRef>hadoop-dynamometer</descriptorRef>
+                  </descriptorRefs>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+          <artifactId>maven-dependency-plugin</artifactId>
+          <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <outputDirectory>${project.build.directory}/${project.artifactId}-${project.version}/share/hadoop/tools/lib</outputDirectory>
+            </configuration>
+          </execution>
+          </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/pom.xml b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/pom.xml
new file mode 100644
index 0000000..4aaf639
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/pom.xml
@@ -0,0 +1,201 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<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>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.3.0-SNAPSHOT</version>
+    <relativePath>../../../hadoop-project</relativePath>
+  </parent>
+  <artifactId>hadoop-dynamometer-infra</artifactId>
+  <version>3.3.0-SNAPSHOT</version>
+  <description>Apache Hadoop Dynamometer Cluster Simulator</description>
+  <name>Apache Hadoop Dynamometer Cluster Simulator</name>
+  <packaging>jar</packaging>
+
+  <dependencies>
+    <dependency>
+      <groupId>commons-cli</groupId>
+      <artifactId>commons-cli</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <!-- JUnit is required by MiniDFSCluster, which is used outside of tests -->
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <type>test-jar</type>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-dynamometer-workload</artifactId>
+      <scope>compile</scope>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>1.10.19</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-tests</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-dynamometer-workload</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+      <version>${project.version}</version>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+        <configuration>
+          <attach>true</attach>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <exclude>src/test/resources/blocks/dn1-a-0-r-00001</exclude>
+            <exclude>src/test/resources/blocks/dn0-a-0-r-00000</exclude>
+            <exclude>src/test/resources/blocks/dn2-a-0-r-00002</exclude>
+            <exclude>src/test/resources/hadoop_3_1/fsimage_0000000000000061740.md5</exclude>
+            <exclude>src/test/resources/hadoop_3_1/VERSION</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+  <profiles>
+    <profile>
+      <id>dist</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-assembly-plugin</artifactId>
+            <dependencies>
+              <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-assemblies</artifactId>
+                <version>${project.version}</version>
+              </dependency>
+            </dependencies>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>prepare-package</phase>
+                <goals>
+                  <goal>single</goal>
+                </goals>
+                <configuration>
+                  <appendAssemblyId>false</appendAssemblyId>
+                  <attach>false</attach>
+                  <finalName>${project.artifactId}-${project.version}</finalName>
+                  <descriptorRefs>
+                    <descriptorRef>hadoop-dynamometer-infra</descriptorRef>
+                  </descriptorRefs>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+       <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>deplist</id>
+            <phase>compile</phase>
+            <goals>
+              <goal>list</goal>
+            </goals>
+            <configuration>
+              <!-- referenced by a built-in command -->
+              <outputFile>${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt</outputFile>
+            </configuration>
+          </execution>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <outputDirectory>${project.build.directory}/lib</outputDirectory>
+              <includeScope>runtime</includeScope>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/bash/create-slim-hadoop-tar.sh b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/bash/create-slim-hadoop-tar.sh
new file mode 100755
index 0000000..4904a31
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/bash/create-slim-hadoop-tar.sh
@@ -0,0 +1,53 @@
+#!/usr/bin/env 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.
+
+if [[ "$#" != 1 ]] || [[ "$1" == "-h" ]] || [[ "$1" == "--help" ]]; then
+  echo "Usage: ./create-slim-hadoop-tar.sh path-to-hadoop-tar"
+  echo "  Takes path-to-hadoop-tar as a hadoop.tar.gz binary distribution"
+  echo "  and removes portions of it that are unnecessary for dynamometer"
+  echo "  (e.g. unrelated components like YARN)."
+  echo "  This overwrites the original file."
+  echo "  This is idempotent; you can safely rerun it on the same tar."
+  exit 1
+fi
+
+hadoopTar="$1"
+
+# ls output is intended for human consumption
+# shellcheck disable=SC2012
+echo "Slimming $hadoopTar; size before is $(ls -lh "$hadoopTar" | awk '{ print $5 }')"
+
+hadoopTarTmp="$hadoopTar.temporary"
+
+mkdir -p "$hadoopTarTmp"
+
+tar xzf "$hadoopTar" -C "$hadoopTarTmp"
+baseDir="$(find -H "$hadoopTarTmp" -depth 1 -type d | head -n 1)" # Should only be one subdir
+hadoopShare="$baseDir/share/hadoop"
+
+# Remove unnecessary files
+rm -rf "${baseDir}/share/doc" "${hadoopShare}/mapreduce ${hadoopShare}/yarn" \
+       "${hadoopShare}/kms" "${hadoopShare}/tools" "${hadoopShare}/httpfs" \
+       "${hadoopShare}"/*/sources "${hadoopShare}"/*/jdiff
+
+tar czf "$hadoopTarTmp.tar.gz" -C "$hadoopTarTmp" .
+rm -rf "$hadoopTarTmp"
+mv -f "$hadoopTarTmp.tar.gz" "$hadoopTar"
+
+# ls output is intended for human consumption
+# shellcheck disable=SC2012
+echo "Finished; size after is $(ls -lh "$hadoopTar" | awk '{ print $5 }')"
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/bash/parse-metrics.sh b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/bash/parse-metrics.sh
new file mode 100755
index 0000000..1ea94c2
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/bash/parse-metrics.sh
@@ -0,0 +1,84 @@
+#!/usr/bin/env 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.
+
+# This script is used for fetching the standard Hadoop metrics which the
+# Dynamometer NameNode generates during its execution (standard Hadoop metrics).
+# Those metrics are uploaded onto HDFS when the Dynamometer application completes.
+# This script will download them locally and parse out the specified metric for
+# the given time period. This is useful to, for example, isolate only the metrics
+# produced during the workload replay portion of a job. For this, specify startTimeMs
+# as the start time of the workload job (which it logs during execution) and
+# periodMinutes the period (in minutes) of the replay.
+
+if [ $# -lt 5 ]; then
+  echo "Usage:"
+  echo "./parse-metrics.sh applicationID outputFileName startTimeMs periodMinutes metricName [ context ] [ isCounter ]"
+  echo "If no file namenode_metrics_{applicationID} is present in the working directory,"
+  echo "attempts to download one from HDFS for applicationID. Filters values"
+  echo "for the specified metric, during the range"
+  echo "(startTimeMs, startTimeMs + periodMinutes) optionally filtering on the context as well"
+  echo "(which is just applied as a regex search across the metric line output)"
+  echo "and outputs CSV pairs of (seconds_since_start_time,value)."
+  echo "If isCounter is true, treats the metrics as a counter and outputs per-second rate values."
+  exit 1
+fi
+
+appId="$1"
+output="$2"
+start_ts="$3"
+period_minutes="$4"
+metric="$5"
+context="$6"
+is_counter="$7"
+
+localFile="namenode_metrics_$appId"
+if [ ! -f "$localFile" ]; then
+  remoteFile=".dynamometer/$appId/namenode_metrics"
+  echo "Downloading file from HDFS: $remoteFile"
+  if ! hdfs dfs -copyToLocal "$remoteFile" "$localFile"; then
+    exit 1
+  fi
+fi
+
+read -d '' -r awk_script <<'EOF'
+BEGIN {
+    metric_regex="[[:space:]]"metric"=([[:digit:].E]+)";
+    end_ts=start_ts+(period_minutes*60*1000)
+    last_val=0
+    last_ts=start_ts
+}
+"true" ~ is_counter && $0 ~ metric_regex && $0 ~ context && $1 < start_ts {
+    match($0, metric_regex, val_arr);
+    last_val=val_arr[1]
+    last_ts=$1
+}
+$0 ~ metric_regex && $0 ~ context && $1 >= start_ts && $1 <= end_ts {
+    match($0, metric_regex, val_arr);
+    val=val_arr[1]
+    if (is_counter == "true") {
+      tmp=val
+      val=val-last_val
+      val=val/(($1-last_ts)/1000)
+      last_ts=$1
+      last_val=tmp
+    }
+    printf("%.0f,%.6f\n", ($0-start_ts)/1000, val)
+}
+EOF
+
+gawk -v metric="$metric" -v context="$context" -v start_ts="$start_ts" \
+  -v period_minutes="$period_minutes" -v is_counter="$is_counter" -v OFS="," "$awk_script" "$localFile" > "$output"
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/bash/start-dynamometer-cluster.sh b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/bash/start-dynamometer-cluster.sh
new file mode 100755
index 0000000..2c27443
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/bash/start-dynamometer-cluster.sh
@@ -0,0 +1,41 @@
+#!/usr/bin/env 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.
+
+# This script simply passes its arguments along to the infrastructure / cluster
+# driver after finding a hadoop command in PATH/HADOOP_COMMON_HOME/HADOOP_HOME
+# (searching in that order).
+
+if type hadoop &> /dev/null; then
+  hadoop_cmd="hadoop"
+elif type "$HADOOP_COMMON_HOME/bin/hadoop" &> /dev/null; then
+  hadoop_cmd="$HADOOP_COMMON_HOME/bin/hadoop"
+elif type "$HADOOP_HOME/bin/hadoop" &> /dev/null; then
+  hadoop_cmd="$HADOOP_HOME/bin/hadoop"
+else
+  echo "Unable to find a valid hadoop command to execute; exiting."
+  exit 1
+fi
+
+script_pwd="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)/.."
+
+for f in ${script_pwd}/lib/*.jar; do
+  # Skip adding the infra JAR since it is added by the `hadoop jar` command
+  if [[ "$f" != *"dynamometer-infra-"* ]]; then
+    export HADOOP_CLASSPATH="$HADOOP_CLASSPATH:$f"
+  fi
+done
+"$hadoop_cmd" jar "${script_pwd}"/lib/dynamometer-infra-*.jar org.apache.hadoop.tools.dynamometer.Client "$@"
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/bash/upload-fsimage.sh b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/bash/upload-fsimage.sh
new file mode 100755
index 0000000..862dd5b
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/bash/upload-fsimage.sh
@@ -0,0 +1,84 @@
+#!/usr/bin/env 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.
+
+if [ $# -lt 2 ] || [ "$1" == "-h" ] || [ "$1" == "--help" ]; then
+  echo "Usage: ./upload-fsimage.sh image-txid remote-dir [ name-dir ]"
+  echo "       Takes the steps necessary to place all necessary components for"
+  echo "       the fsimage with the specified transaction ID (leading 0s not"
+  echo "       required) onto HDFS. This includes the fsimage file itself,"
+  echo "       the MD5 hash, the VERSION file, and the XML version of the"
+  echo "       fsimage file, which will be temporarily generated by this script"
+  echo "       (into \$TMPDIR if specified, else /tmp)."
+  echo "       These files will be uploaded to remote_dir (created if it does"
+  echo "       not yet exist). This is the same directory that should be passed"
+  echo "       to the Client as fs_image_dir."
+  echo "       If name-dir is specified, looks for fsimage files under"
+  echo "       \${name-dir}/current. Otherwise, looks in the current directory."
+  exit 1
+fi
+image_txid="$1"
+remote_dir="$2"
+if [[ $# -ge 3 ]]; then
+  name_dir="$3/current"
+else
+  name_dir="$(pwd)"
+fi
+
+image_file_count="$(find -H "${name_dir}" -depth 1 -name "fsimage_*$image_txid" -type f | wc -l)"
+if [[ "$image_file_count" != 1 ]]; then
+  echo "Error; found $image_file_count matching fsimage files."
+  exit 1
+fi
+image_file="$(find -H "${name_dir}" -depth 1 -name "fsimage_*$image_txid" -type f)"
+image_file_name="$(basename "${image_file}")"
+echo "Using fsimage: $image_file_name"
+image_file_md5="${image_file}.md5"
+version_file="${name_dir}/VERSION"
+
+if ! [[ -f "$image_file_md5" ]]; then
+  echo "Unable to find MD5 file for fsimage; looked at $image_file_md5"
+  exit 1
+fi
+if ! [[ -f "$version_file" ]]; then
+  echo "Unable to find VERSION file; looked at $version_file"
+  exit 1
+fi
+
+if ! tmp_dir="$(mktemp -d)"; then
+  echo "mktemp failed to make a temporary directory; exiting"
+  exit 1
+fi
+image_file_xml="${tmp_dir}/${image_file_name}.xml"
+echo "Creating temporary XML fsimage file at $image_file_xml"
+hdfs oiv -p XML -i "$image_file" -o "$image_file_xml"
+echo "Created temporary XML fsimage file"
+
+if ! hdfs dfs -mkdir -p "$remote_dir"; then
+  echo "Unable to create ${remote_dir}; exiting"
+  rm -rf "${tmp_dir}"
+  exit 1
+fi
+for file in ${image_file} ${image_file_xml} ${image_file_md5} ${version_file}; do
+  echo "Uploading ${file}..."
+  if ! hdfs dfs -copyFromLocal -f "$file" "${remote_dir}"; then
+    echo "Error while uploading ${file}; exiting"
+      rm -rf "${tmp_dir}"
+    exit 1
+  fi
+done
+rm -rf "${tmp_dir}"
+echo "Complete!"
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/AMOptions.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/AMOptions.java
new file mode 100644
index 0000000..24303a2
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/AMOptions.java
@@ -0,0 +1,316 @@
+/**
+ * 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.hadoop.tools.dynamometer;
+
+import com.google.common.base.Preconditions;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Options supplied to the Client which are then passed through to the
+ * ApplicationMaster.
+ */
+class AMOptions {
+
+  public static final String NAMENODE_MEMORY_MB_ARG = "namenode_memory_mb";
+  public static final String NAMENODE_MEMORY_MB_DEFAULT = "2048";
+  public static final String NAMENODE_VCORES_ARG = "namenode_vcores";
+  public static final String NAMENODE_VCORES_DEFAULT = "1";
+  public static final String NAMENODE_NODELABEL_ARG = "namenode_nodelabel";
+  public static final String NAMENODE_ARGS_ARG = "namenode_args";
+  public static final String DATANODE_MEMORY_MB_ARG = "datanode_memory_mb";
+  public static final String DATANODE_MEMORY_MB_DEFAULT = "2048";
+  public static final String DATANODE_VCORES_ARG = "datanode_vcores";
+  public static final String DATANODE_VCORES_DEFAULT = "1";
+  public static final String DATANODE_NODELABEL_ARG = "datanode_nodelabel";
+  public static final String DATANODE_ARGS_ARG = "datanode_args";
+  public static final String NAMENODE_METRICS_PERIOD_ARG =
+      "namenode_metrics_period";
+  public static final String NAMENODE_METRICS_PERIOD_DEFAULT = "60";
+  public static final String SHELL_ENV_ARG = "shell_env";
+  public static final String DATANODES_PER_CLUSTER_ARG =
+      "datanodes_per_cluster";
+  public static final String DATANODES_PER_CLUSTER_DEFAULT = "1";
+  public static final String DATANODE_LAUNCH_DELAY_ARG =
+      "datanode_launch_delay";
+  public static final String DATANODE_LAUNCH_DELAY_DEFAULT = "0s";
+  public static final String NAMENODE_NAME_DIR_ARG = "namenode_name_dir";
+  public static final String NAMENODE_EDITS_DIR_ARG = "namenode_edits_dir";
+
+  private final int datanodeMemoryMB;
+  private final int datanodeVirtualCores;
+  private final String datanodeArgs;
+  private final String datanodeNodeLabelExpression;
+  private final int datanodesPerCluster;
+  private final String datanodeLaunchDelay;
+  private final int namenodeMemoryMB;
+  private final int namenodeVirtualCores;
+  private final String namenodeArgs;
+  private final String namenodeNodeLabelExpression;
+  private final int namenodeMetricsPeriod;
+  private final String namenodeNameDir;
+  private final String namenodeEditsDir;
+  // Original shellEnv as passed in through arguments
+  private final Map<String, String> originalShellEnv;
+  // Extended shellEnv including custom environment variables
+  private final Map<String, String> shellEnv;
+
+  AMOptions(int datanodeMemoryMB, int datanodeVirtualCores, String datanodeArgs,
+      String datanodeNodeLabelExpression, int datanodesPerCluster,
+      String datanodeLaunchDelay, int namenodeMemoryMB,
+      int namenodeVirtualCores, String namenodeArgs,
+      String namenodeNodeLabelExpression, int namenodeMetricsPeriod,
+      String namenodeNameDir, String namenodeEditsDir,
+      Map<String, String> shellEnv) {
+    this.datanodeMemoryMB = datanodeMemoryMB;
+    this.datanodeVirtualCores = datanodeVirtualCores;
+    this.datanodeArgs = datanodeArgs;
+    this.datanodeNodeLabelExpression = datanodeNodeLabelExpression;
+    this.datanodesPerCluster = datanodesPerCluster;
+    this.datanodeLaunchDelay = datanodeLaunchDelay;
+    this.namenodeMemoryMB = namenodeMemoryMB;
+    this.namenodeVirtualCores = namenodeVirtualCores;
+    this.namenodeArgs = namenodeArgs;
+    this.namenodeNodeLabelExpression = namenodeNodeLabelExpression;
+    this.namenodeMetricsPeriod = namenodeMetricsPeriod;
+    this.namenodeNameDir = namenodeNameDir;
+    this.namenodeEditsDir = namenodeEditsDir;
+    this.originalShellEnv = shellEnv;
+    this.shellEnv = new HashMap<>(this.originalShellEnv);
+    this.shellEnv.put(DynoConstants.NN_ADDITIONAL_ARGS_ENV, this.namenodeArgs);
+    this.shellEnv.put(DynoConstants.DN_ADDITIONAL_ARGS_ENV, this.datanodeArgs);
+    this.shellEnv.put(DynoConstants.NN_FILE_METRIC_PERIOD_ENV,
+        String.valueOf(this.namenodeMetricsPeriod));
+    this.shellEnv.put(DynoConstants.NN_NAME_DIR_ENV, this.namenodeNameDir);
+    this.shellEnv.put(DynoConstants.NN_EDITS_DIR_ENV, this.namenodeEditsDir);
+  }
+
+  /**
+   * Verifies that arguments are valid; throws IllegalArgumentException if not.
+   */
+  void verify(long maxMemory, int maxVcores) throws IllegalArgumentException {
+    Preconditions.checkArgument(
+        datanodeMemoryMB > 0 && datanodeMemoryMB <= maxMemory,
+        "datanodeMemoryMB (%s) must be between 0 and %s", datanodeMemoryMB,
+        maxMemory);
+    Preconditions.checkArgument(
+        datanodeVirtualCores > 0 && datanodeVirtualCores <= maxVcores,
+        "datanodeVirtualCores (%s) must be between 0 and %s",
+        datanodeVirtualCores, maxVcores);
+    Preconditions.checkArgument(
+        namenodeMemoryMB > 0 && namenodeMemoryMB <= maxMemory,
+        "namenodeMemoryMB (%s) must be between 0 and %s", namenodeMemoryMB,
+        maxMemory);
+    Preconditions.checkArgument(
+        namenodeVirtualCores > 0 && namenodeVirtualCores <= maxVcores,
+        "namenodeVirtualCores (%s) must be between 0 and %s",
+        namenodeVirtualCores, maxVcores);
+    Preconditions.checkArgument(datanodesPerCluster > 0,
+        "datanodesPerCluster (%s) must be > 0", datanodesPerCluster);
+  }
+
+  /**
+   * Same as {@link #verify(long, int)} but does not set a max.
+   */
+  void verify() throws IllegalArgumentException {
+    verify(Integer.MAX_VALUE, Integer.MAX_VALUE);
+  }
+
+  void addToVargs(List<String> vargs) {
+    vargs.add("--" + DATANODE_MEMORY_MB_ARG + " " + datanodeMemoryMB);
+    vargs.add("--" + DATANODE_VCORES_ARG + " " + datanodeVirtualCores);
+    addStringValToVargs(vargs, DATANODE_ARGS_ARG, datanodeArgs);
+    addStringValToVargs(vargs, DATANODE_NODELABEL_ARG,
+        datanodeNodeLabelExpression);
+    vargs.add("--" + DATANODES_PER_CLUSTER_ARG + " " + datanodesPerCluster);
+    vargs.add("--" + DATANODE_LAUNCH_DELAY_ARG + " " + datanodeLaunchDelay);
+    vargs.add("--" + NAMENODE_MEMORY_MB_ARG + " " + namenodeMemoryMB);
+    vargs.add("--" + NAMENODE_VCORES_ARG + " " + namenodeVirtualCores);
+    addStringValToVargs(vargs, NAMENODE_ARGS_ARG, namenodeArgs);
+    addStringValToVargs(vargs, NAMENODE_NODELABEL_ARG,
+        namenodeNodeLabelExpression);
+    vargs.add("--" + NAMENODE_METRICS_PERIOD_ARG + " " + namenodeMetricsPeriod);
+    addStringValToVargs(vargs, NAMENODE_NAME_DIR_ARG, namenodeNameDir);
+    addStringValToVargs(vargs, NAMENODE_EDITS_DIR_ARG, namenodeEditsDir);
+    for (Map.Entry<String, String> entry : originalShellEnv.entrySet()) {
+      vargs.add(
+          "--" + SHELL_ENV_ARG + " " + entry.getKey() + "=" + entry.getValue());
+    }
+  }
+
+  private void addStringValToVargs(List<String> vargs, String optionName,
+      String val) {
+    if (!val.isEmpty()) {
+      vargs.add("--" + optionName + " \\\"" + val + "\\\"");
+    }
+  }
+
+  int getDataNodeMemoryMB() {
+    return datanodeMemoryMB;
+  }
+
+  int getDataNodeVirtualCores() {
+    return datanodeVirtualCores;
+  }
+
+  String getDataNodeNodeLabelExpression() {
+    return datanodeNodeLabelExpression;
+  }
+
+  int getDataNodesPerCluster() {
+    return datanodesPerCluster;
+  }
+
+  long getDataNodeLaunchDelaySec() {
+    // Leverage the human-readable time parsing capabilities of Configuration
+    String tmpConfKey = "___temp_config_property___";
+    Configuration tmpConf = new Configuration();
+    tmpConf.set(tmpConfKey, datanodeLaunchDelay);
+    return tmpConf.getTimeDuration(tmpConfKey, 0, TimeUnit.SECONDS);
+  }
+
+  int getNameNodeMemoryMB() {
+    return namenodeMemoryMB;
+  }
+
+  int getNameNodeVirtualCores() {
+    return namenodeVirtualCores;
+  }
+
+  String getNameNodeNodeLabelExpression() {
+    return namenodeNodeLabelExpression;
+  }
+
+  Map<String, String> getShellEnv() {
+    return shellEnv;
+  }
+
+  /**
+   * Set all of the command line options relevant to this class into the passed
+   * {@link Options}.
+   *
+   * @param opts
+   *          Where to set the command line options.
+   */
+  static void setOptions(Options opts) {
+    opts.addOption(SHELL_ENV_ARG, true,
+        "Environment for shell script. Specified as env_key=env_val pairs");
+    opts.addOption(NAMENODE_MEMORY_MB_ARG, true,
+        "Amount of memory in MB to be requested to run the NN (default "
+            + NAMENODE_MEMORY_MB_DEFAULT + "). "
+            + "Ignored unless the NameNode is run within YARN.");
+    opts.addOption(NAMENODE_VCORES_ARG, true,
+        "Amount of virtual cores to be requested to run the NN (default "
+            + NAMENODE_VCORES_DEFAULT + "). "
+            + "Ignored unless the NameNode is run within YARN.");
+    opts.addOption(NAMENODE_ARGS_ARG, true,
+        "Additional arguments to add when starting the NameNode. "
+            + "Ignored unless the NameNode is run within YARN.");
+    opts.addOption(NAMENODE_NODELABEL_ARG, true,
+        "The node label to specify for the container to use to "
+            + "run the NameNode.");
+    opts.addOption(NAMENODE_METRICS_PERIOD_ARG, true,
+        "The period in seconds for the NameNode's metrics to be emitted to "
+            + "file; if <=0, disables this functionality. Otherwise, a "
+            + "metrics file will be stored in the container logs for the "
+            + "NameNode (default " + NAMENODE_METRICS_PERIOD_DEFAULT + ").");
+    opts.addOption(NAMENODE_NAME_DIR_ARG, true,
+        "The directory to use for the NameNode's name data directory. "
+            + "If not specified, a location  within the container's working "
+            + "directory will be used.");
+    opts.addOption(NAMENODE_EDITS_DIR_ARG, true,
+        "The directory to use for the NameNode's edits directory. "
+            + "If not specified, a location  within the container's working "
+            + "directory will be used.");
+    opts.addOption(DATANODE_MEMORY_MB_ARG, true,
+        "Amount of memory in MB to be requested to run the DNs (default "
+            + DATANODE_MEMORY_MB_DEFAULT + ")");
+    opts.addOption(DATANODE_VCORES_ARG, true,
+        "Amount of virtual cores to be requested to run the DNs (default "
+            + DATANODE_VCORES_DEFAULT + ")");
+    opts.addOption(DATANODE_ARGS_ARG, true,
+        "Additional arguments to add when starting the DataNodes.");
+    opts.addOption(DATANODE_NODELABEL_ARG, true, "The node label to specify "
+        + "for the container to use to run the DataNode.");
+    opts.addOption(DATANODES_PER_CLUSTER_ARG, true,
+        "How many simulated DataNodes to run within each YARN container "
+            + "(default " + DATANODES_PER_CLUSTER_DEFAULT + ")");
+    opts.addOption(DATANODE_LAUNCH_DELAY_ARG, true,
+        "The period over which to launch the DataNodes; this will "
+            + "be used as the maximum delay and each DataNode container will "
+            + "be launched with some random delay less than  this value. "
+            + "Accepts human-readable time durations (e.g. 10s, 1m) (default "
+            + DATANODE_LAUNCH_DELAY_DEFAULT + ")");
+
+    opts.addOption("help", false, "Print usage");
+  }
+
+  /**
+   * Initialize an {@code AMOptions} from a command line parser.
+   *
+   * @param cliParser
+   *          Where to initialize from.
+   * @return A new {@code AMOptions} filled out with options from the parser.
+   */
+  static AMOptions initFromParser(CommandLine cliParser) {
+    Map<String, String> originalShellEnv = new HashMap<>();
+    if (cliParser.hasOption(SHELL_ENV_ARG)) {
+      for (String env : cliParser.getOptionValues(SHELL_ENV_ARG)) {
+        String trimmed = env.trim();
+        int index = trimmed.indexOf('=');
+        if (index == -1) {
+          originalShellEnv.put(trimmed, "");
+          continue;
+        }
+        String key = trimmed.substring(0, index);
+        String val = "";
+        if (index < (trimmed.length() - 1)) {
+          val = trimmed.substring(index + 1);
+        }
+        originalShellEnv.put(key, val);
+      }
+    }
+    return new AMOptions(
+        Integer.parseInt(cliParser.getOptionValue(DATANODE_MEMORY_MB_ARG,
+            DATANODE_MEMORY_MB_DEFAULT)),
+        Integer.parseInt(cliParser.getOptionValue(DATANODE_VCORES_ARG,
+            DATANODE_VCORES_DEFAULT)),
+        cliParser.getOptionValue(DATANODE_ARGS_ARG, ""),
+        cliParser.getOptionValue(DATANODE_NODELABEL_ARG, ""),
+        Integer.parseInt(cliParser.getOptionValue(DATANODES_PER_CLUSTER_ARG,
+            DATANODES_PER_CLUSTER_DEFAULT)),
+        cliParser.getOptionValue(DATANODE_LAUNCH_DELAY_ARG,
+            DATANODE_LAUNCH_DELAY_DEFAULT),
+        Integer.parseInt(cliParser.getOptionValue(NAMENODE_MEMORY_MB_ARG,
+            NAMENODE_MEMORY_MB_DEFAULT)),
+        Integer.parseInt(cliParser.getOptionValue(NAMENODE_VCORES_ARG,
+            NAMENODE_VCORES_DEFAULT)),
+        cliParser.getOptionValue(NAMENODE_ARGS_ARG, ""),
+        cliParser.getOptionValue(NAMENODE_NODELABEL_ARG, ""),
+        Integer.parseInt(cliParser.getOptionValue(NAMENODE_METRICS_PERIOD_ARG,
+            NAMENODE_METRICS_PERIOD_DEFAULT)),
+        cliParser.getOptionValue(NAMENODE_NAME_DIR_ARG, ""),
+        cliParser.getOptionValue(NAMENODE_EDITS_DIR_ARG, ""), originalShellEnv);
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/AllowAllImpersonationProvider.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/AllowAllImpersonationProvider.java
new file mode 100644
index 0000000..490453f
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/AllowAllImpersonationProvider.java
@@ -0,0 +1,39 @@
+/**
+ * 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.hadoop.tools.dynamometer;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.ImpersonationProvider;
+
+/**
+ * An {@link ImpersonationProvider} that indiscriminately allows all users to
+ * proxy as any other user.
+ */
+public class AllowAllImpersonationProvider extends Configured
+    implements ImpersonationProvider {
+
+  public void init(String configurationPrefix) {
+    // Do nothing
+  }
+
+  public void authorize(UserGroupInformation user, String remoteAddress) {
+    // Do nothing
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/ApplicationMaster.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/ApplicationMaster.java
new file mode 100644
index 0000000..2433e9f
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/ApplicationMaster.java
@@ -0,0 +1,885 @@
+/**
+ * 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.hadoop.tools.dynamometer;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Ints;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import java.util.function.Supplier;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+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.NodeReport;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
+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;
+import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.util.Records;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The ApplicationMaster for Dynamometer. This will launch DataNodes in YARN
+ * containers. If the RPC address of a NameNode is specified, it will configure
+ * the DataNodes to talk to that NameNode. Else, a NameNode will be launched as
+ * part of this YARN application. This does not implement any retry/failure
+ * handling.
+ * TODO: Add proper retry/failure handling
+ *
+ * <p/>The AM will persist until it has run for a period of time equal to the
+ * timeout specified or until the application is killed.
+ *
+ * <p/>If the NameNode is launched internally, it will upload some information
+ * onto the remote HDFS instance (i.e., the default FileSystem) about its
+ * hostname and ports. This is in the location determined by the
+ * {@link DynoConstants#DYNAMOMETER_STORAGE_DIR} and
+ * {@link DynoConstants#NN_INFO_FILE_NAME} constants and is in the
+ * {@link Properties} file format. This is consumed by this AM as well as the
+ * {@link Client} to determine how to contact the NameNode.
+ *
+ * <p/>Information about the location of the DataNodes is logged by the AM.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class ApplicationMaster {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ApplicationMaster.class);
+  private static final Random RAND = new Random();
+
+  // Configuration
+  private Configuration conf;
+
+  // Handle to communicate with the Resource Manager
+  private AMRMClientAsync<ContainerRequest> amRMClient;
+
+  // Handle to communicate with the Node Manager
+  private NMClientAsync nmClientAsync;
+  // Listen to process the response from the Node Manager
+  private NMCallbackHandler containerListener;
+  // The collection of options passed in via the Client
+  private AMOptions amOptions;
+
+  private List<LocalResource> blockListFiles;
+  private int numTotalDataNodes;
+  private int numTotalDataNodeContainers;
+
+  // Counter for completed datanodes (complete denotes successful or failed )
+  private AtomicInteger numCompletedDataNodeContainers = new AtomicInteger();
+  // Allocated datanode count so that we know how many datanodes has the RM
+  // allocated to us
+  private AtomicInteger numAllocatedDataNodeContainers = new AtomicInteger();
+  // Count of failed datanodes
+  private AtomicInteger numFailedDataNodeContainers = new AtomicInteger();
+
+  // True iff the application has completed and is ready for cleanup
+  // Once true, will never be false. This variable should not be accessed
+  // directly but rather through the isComplete, waitForCompletion, and
+  // markCompleted methods.
+  private boolean completed = false;
+  private final Object completionLock = new Object();
+
+  private ByteBuffer allTokens;
+
+  // Launch threads
+  private List<Thread> launchThreads = new ArrayList<>();
+
+  // True iff this AM should launch and manage a Namanode
+  private boolean launchNameNode;
+  // The service RPC address of a remote NameNode to be contacted by the
+  // launched DataNodes
+  private String namenodeServiceRpcAddress = "";
+  // Directory to use for remote storage (a location on the remote FS which
+  // can be accessed by all components)
+  private Path remoteStoragePath;
+  // The ACLs to view the launched containers
+  private Map<ApplicationAccessType, String> applicationAcls;
+  // The container the NameNode is running within
+  private volatile Container namenodeContainer;
+  // Map of the containers that the DataNodes are running within
+  private ConcurrentMap<ContainerId, Container> datanodeContainers =
+      new ConcurrentHashMap<>();
+
+  // Username of the user who launched this application.
+  private String launchingUser;
+
+  /**
+   * @param args Command line args
+   */
+  public static void main(String[] args) {
+    boolean result = false;
+    try {
+      ApplicationMaster appMaster = new ApplicationMaster();
+      LOG.info("Initializing ApplicationMaster");
+      boolean doRun = appMaster.init(args);
+      if (!doRun) {
+        System.exit(0);
+      }
+      result = appMaster.run();
+    } catch (Throwable t) {
+      LOG.error("Error running ApplicationMaster", t);
+      System.exit(1);
+    }
+    if (result) {
+      LOG.info("Application Master completed successfully. exiting");
+      System.exit(0);
+    } else {
+      LOG.info("Application Master failed. exiting");
+      System.exit(2);
+    }
+  }
+
+  public ApplicationMaster() {
+    // Set up the configuration
+    conf = new YarnConfiguration();
+  }
+
+  /**
+   * Parse command line options.
+   *
+   * @param args Command line args
+   * @return Whether init successful and run should be invoked
+   */
+  public boolean init(String[] args) throws ParseException {
+
+    Options opts = new Options();
+    AMOptions.setOptions(opts);
+    CommandLine cliParser = new GnuParser().parse(opts, args);
+
+    if (args.length == 0) {
+      printUsage(opts);
+      throw new IllegalArgumentException(
+          "No args specified for application master to initialize");
+    }
+
+    if (cliParser.hasOption("help")) {
+      printUsage(opts);
+      return false;
+    }
+
+    Map<String, String> envs = System.getenv();
+
+    remoteStoragePath = new Path(
+        envs.get(DynoConstants.REMOTE_STORAGE_PATH_ENV));
+    applicationAcls = new HashMap<>();
+    applicationAcls.put(ApplicationAccessType.VIEW_APP,
+        envs.get(DynoConstants.JOB_ACL_VIEW_ENV));
+    launchingUser = envs.get(Environment.USER.name());
+    if (envs.containsKey(DynoConstants.REMOTE_NN_RPC_ADDR_ENV)) {
+      launchNameNode = false;
+      namenodeServiceRpcAddress = envs
+          .get(DynoConstants.REMOTE_NN_RPC_ADDR_ENV);
+    } else {
+      launchNameNode = true;
+      // namenodeServiceRpcAddress will be set in run() once properties are
+      // available
+    }
+
+    ContainerId containerId =
+        ContainerId.fromString(envs.get(Environment.CONTAINER_ID.name()));
+    ApplicationAttemptId appAttemptID = containerId.getApplicationAttemptId();
+    LOG.info("Application master for app: appId={}, clusterTimestamp={}, "
+        + "attemptId={}", appAttemptID.getApplicationId().getId(),
+        appAttemptID.getApplicationId().getClusterTimestamp(),
+        appAttemptID.getAttemptId());
+
+    amOptions = AMOptions.initFromParser(cliParser);
+
+    return true;
+  }
+
+  /**
+   * Helper function to print usage.
+   *
+   * @param opts arsed command line options
+   */
+  private void printUsage(Options opts) {
+    new HelpFormatter().printHelp("ApplicationMaster", opts);
+  }
+
+  /**
+   * Main run function for the application master.
+   *
+   * @return True if the application completed successfully; false if if exited
+   *         unexpectedly, failed, was killed, etc.
+   */
+  public boolean run() throws YarnException, IOException, InterruptedException {
+    LOG.info("Starting ApplicationMaster");
+
+    Credentials credentials = UserGroupInformation.getCurrentUser()
+        .getCredentials();
+    DataOutputBuffer dob = new DataOutputBuffer();
+    credentials.writeTokenStorageToStream(dob);
+    // Now remove the AM->RM token so that containers cannot access it.
+    credentials.getAllTokens().removeIf((token) ->
+            token.getKind().equals(AMRMTokenIdentifier.KIND_NAME));
+    allTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+
+    AMRMClientAsync.AbstractCallbackHandler allocListener =
+        new RMCallbackHandler();
+    amRMClient = AMRMClientAsync.createAMRMClientAsync(1000, allocListener);
+    amRMClient.init(conf);
+    amRMClient.start();
+
+    containerListener = createNMCallbackHandler();
+    nmClientAsync = new NMClientAsyncImpl(containerListener);
+    nmClientAsync.init(conf);
+    nmClientAsync.start();
+
+    // Register self with ResourceManager
+    // This will start heartbeating to the RM
+    String appMasterHostname = NetUtils.getHostname();
+    amRMClient.registerApplicationMaster(appMasterHostname, -1, "");
+
+    // Supplier to use to indicate to wait-loops to stop waiting
+    Supplier<Boolean> exitCritera = this::isComplete;
+
+    Optional<Properties> namenodeProperties = Optional.empty();
+    if (launchNameNode) {
+      ContainerRequest nnContainerRequest = setupContainerAskForRM(
+          amOptions.getNameNodeMemoryMB(), amOptions.getNameNodeVirtualCores(),
+          0, amOptions.getNameNodeNodeLabelExpression());
+      LOG.info("Requested NameNode ask: " + nnContainerRequest.toString());
+      amRMClient.addContainerRequest(nnContainerRequest);
+
+      // Wait for the NN container to make its information available on the
+      // shared
+      // remote file storage
+      Path namenodeInfoPath = new Path(remoteStoragePath,
+          DynoConstants.NN_INFO_FILE_NAME);
+      LOG.info("Waiting on availability of NameNode information at "
+          + namenodeInfoPath);
+
+      namenodeProperties = DynoInfraUtils.waitForAndGetNameNodeProperties(
+          exitCritera, conf, namenodeInfoPath, LOG);
+      if (!namenodeProperties.isPresent()) {
+        cleanup();
+        return false;
+      }
+      namenodeServiceRpcAddress = DynoInfraUtils
+          .getNameNodeServiceRpcAddr(namenodeProperties.get()).toString();
+      LOG.info("NameNode information: " + namenodeProperties.get());
+      LOG.info("NameNode can be reached at: " + DynoInfraUtils
+          .getNameNodeHdfsUri(namenodeProperties.get()).toString());
+      DynoInfraUtils.waitForNameNodeStartup(namenodeProperties.get(),
+          exitCritera, LOG);
+    } else {
+      LOG.info("Using remote NameNode with RPC address: "
+          + namenodeServiceRpcAddress);
+    }
+
+    blockListFiles = Collections
+        .synchronizedList(getDataNodeBlockListingFiles());
+    numTotalDataNodes = blockListFiles.size();
+    if (numTotalDataNodes == 0) {
+      LOG.error(
+          "No block listing files were found! Cannot run with 0 DataNodes.");
+      markCompleted();
+      return false;
+    }
+    numTotalDataNodeContainers = (int) Math.ceil(((double) numTotalDataNodes)
+        / Math.max(1, amOptions.getDataNodesPerCluster()));
+
+    LOG.info("Requesting {} DataNode containers with {} MB memory, {} vcores",
+        numTotalDataNodeContainers, amOptions.getDataNodeMemoryMB(),
+        amOptions.getDataNodeVirtualCores());
+    for (int i = 0; i < numTotalDataNodeContainers; ++i) {
+      ContainerRequest datanodeAsk = setupContainerAskForRM(
+          amOptions.getDataNodeMemoryMB(), amOptions.getDataNodeVirtualCores(),
+          1, amOptions.getDataNodeNodeLabelExpression());
+      amRMClient.addContainerRequest(datanodeAsk);
+      LOG.debug("Requested datanode ask: " + datanodeAsk.toString());
+    }
+    LOG.info("Finished requesting datanode containers");
+
+    if (launchNameNode) {
+      DynoInfraUtils.waitForNameNodeReadiness(namenodeProperties.get(),
+          numTotalDataNodes, true, exitCritera, conf, LOG);
+    }
+
+    waitForCompletion();
+    return cleanup();
+  }
+
+  private NMCallbackHandler createNMCallbackHandler() {
+    return new NMCallbackHandler();
+  }
+
+  /**
+   * Wait until the application has finished and is ready for cleanup.
+   */
+  private void waitForCompletion() throws InterruptedException {
+    synchronized (completionLock) {
+      while (!completed) {
+        completionLock.wait();
+      }
+    }
+  }
+
+  /**
+   * Check completion status of the application.
+   *
+   * @return True iff it has completed.
+   */
+  private boolean isComplete() {
+    synchronized (completionLock) {
+      return completed;
+    }
+  }
+
+  /**
+   * Mark that this application should begin cleaning up and exit.
+   */
+  private void markCompleted() {
+    synchronized (completionLock) {
+      completed = true;
+      completionLock.notify();
+    }
+  }
+
+  /**
+   * @return True iff the application successfully completed
+   */
+  private boolean cleanup() {
+    // Join all launched threads
+    // needed for when we time out
+    // and we need to release containers
+    for (Thread launchThread : launchThreads) {
+      try {
+        launchThread.join(10000);
+      } catch (InterruptedException e) {
+        LOG.info("Exception thrown in thread join: " + e.getMessage());
+        e.printStackTrace();
+      }
+    }
+
+    // When the application completes, it should stop all running containers
+    LOG.info("Application completed. Stopping running containers");
+    nmClientAsync.stop();
+
+    // When the application completes, it should send a finish application
+    // signal to the RM
+    LOG.info("Application completed. Signalling finish to RM");
+
+    FinalApplicationStatus appStatus;
+    String appMessage = null;
+    boolean success;
+    if (numFailedDataNodeContainers.get() == 0
+        && numCompletedDataNodeContainers.get() == numTotalDataNodes) {
+      appStatus = FinalApplicationStatus.SUCCEEDED;
+      success = true;
+    } else {
+      appStatus = FinalApplicationStatus.FAILED;
+      appMessage = "Diagnostics: total=" + numTotalDataNodeContainers
+          + ", completed=" + numCompletedDataNodeContainers.get()
+          + ", allocated=" + numAllocatedDataNodeContainers.get()
+          + ", failed=" + numFailedDataNodeContainers.get();
+      success = false;
+    }
+    try {
+      amRMClient.unregisterApplicationMaster(appStatus, appMessage, null);
+    } catch (YarnException|IOException ex) {
+      LOG.error("Failed to unregister application", ex);
+    }
+
+    amRMClient.stop();
+    return success;
+  }
+
+  private class RMCallbackHandler
+      extends AMRMClientAsync.AbstractCallbackHandler {
+
+    @Override
+    public void onContainersCompleted(
+        List<ContainerStatus> completedContainers) {
+      LOG.info("Got response from RM for container ask, completedCnt="
+          + completedContainers.size());
+      for (ContainerStatus containerStatus : completedContainers) {
+        String containerInfo = "containerID=" + containerStatus.getContainerId()
+            + ", state=" + containerStatus.getState() + ", exitStatus="
+            + containerStatus.getExitStatus() + ", diagnostics="
+            + StringUtils.abbreviate(containerStatus.getDiagnostics(), 1000);
+        String component;
+        if (isNameNode(containerStatus.getContainerId())) {
+          component = "NAMENODE";
+        } else if (isDataNode(containerStatus.getContainerId())) {
+          component = "DATANODE";
+        } else {
+          LOG.error("Received container status for unknown container: "
+              + containerInfo);
+          continue;
+        }
+        LOG.info(
+            "Got container status for " + component + ": " + containerInfo);
+
+        // non complete containers should not be here
+        assert (containerStatus.getState() == ContainerState.COMPLETE);
+
+        if (component.equals("NAMENODE")) {
+          LOG.info("NameNode container completed; marking application as done");
+          markCompleted();
+        }
+
+        // increment counters for completed/failed containers
+        int exitStatus = containerStatus.getExitStatus();
+        int completedIdx = numCompletedDataNodeContainers.incrementAndGet();
+        if (0 != exitStatus) {
+          numFailedDataNodeContainers.incrementAndGet();
+        } else {
+          LOG.info("DataNode {} completed successfully, containerId={}",
+              completedIdx, containerStatus.getContainerId());
+        }
+      }
+
+      if (numCompletedDataNodeContainers.get() == numTotalDataNodeContainers) {
+        LOG.info(
+            "All datanode containers completed; marking application as done");
+        markCompleted();
+      }
+    }
+
+    @Override
+    public void onContainersAllocated(List<Container> allocatedContainers) {
+      LOG.info("Got response from RM for container ask, allocatedCnt="
+          + allocatedContainers.size());
+      for (Container container : allocatedContainers) {
+        LaunchContainerRunnable containerLauncher;
+        String componentType;
+        Resource rsrc = container.getResource();
+        if (launchNameNode
+            && rsrc.getMemorySize() >= amOptions.getNameNodeMemoryMB()
+            && rsrc.getVirtualCores() >= amOptions.getNameNodeVirtualCores()
+            && namenodeContainer == null) {
+          namenodeContainer = container;
+          componentType = "NAMENODE";
+          containerLauncher = new LaunchContainerRunnable(container, true);
+        } else if (rsrc.getMemorySize() >= amOptions.getDataNodeMemoryMB()
+            && rsrc.getVirtualCores() >= amOptions.getDataNodeVirtualCores()
+            && numAllocatedDataNodeContainers.get() < numTotalDataNodes) {
+          if (launchNameNode && namenodeContainer == null) {
+            LOG.error("Received a container with following resources suited "
+                + "for a DataNode but no NameNode container exists: "
+                + "containerMem=" + rsrc.getMemorySize() + ", containerVcores="
+                + rsrc.getVirtualCores());
+            continue;
+          }
+          numAllocatedDataNodeContainers.getAndIncrement();
+          datanodeContainers.put(container.getId(), container);
+          componentType = "DATANODE";
+          containerLauncher = new LaunchContainerRunnable(container, false);
+        } else {
+          LOG.warn("Received unwanted container allocation: " + container);
+          nmClientAsync.stopContainerAsync(container.getId(),
+              container.getNodeId());
+          continue;
+        }
+        LOG.info("Launching " + componentType + " on a new container."
+            + ", containerId=" + container.getId() + ", containerNode="
+            + container.getNodeId().getHost() + ":"
+            + container.getNodeId().getPort() + ", containerNodeURI="
+            + container.getNodeHttpAddress() + ", containerResourceMemory="
+            + rsrc.getMemorySize() + ", containerResourceVirtualCores="
+            + rsrc.getVirtualCores());
+        Thread launchThread = new Thread(containerLauncher);
+
+        // launch and start the container on a separate thread to keep
+        // the main thread unblocked
+        // as all containers may not be allocated at one go.
+        launchThreads.add(launchThread);
+        launchThread.start();
+      }
+    }
+
+    @Override
+    public void onShutdownRequest() {
+      markCompleted();
+    }
+
+    @Override
+    public void onNodesUpdated(List<NodeReport> updatedNodes) {
+      LOG.info("onNodesUpdated: " + Joiner.on(",").join(updatedNodes));
+    }
+
+    @Override
+    public float getProgress() {
+      return 0.0f;
+    }
+
+    @Override
+    public void onError(Throwable e) {
+      markCompleted();
+      amRMClient.stop();
+    }
+
+    @Override
+    public void onContainersUpdated(List<UpdatedContainer> containers) {
+      LOG.info("onContainersUpdated: " + Joiner.on(",").join(containers));
+    }
+  }
+
+  private class NMCallbackHandler
+      extends NMClientAsync.AbstractCallbackHandler {
+
+    @Override
+    public void onContainerStopped(ContainerId containerId) {
+      if (isNameNode(containerId)) {
+        LOG.info("NameNode container stopped: " + containerId);
+        namenodeContainer = null;
+        markCompleted();
+      } else if (isDataNode(containerId)) {
+        LOG.debug("DataNode container stopped: " + containerId);
+        datanodeContainers.remove(containerId);
+      } else {
+        LOG.error(
+            "onContainerStopped received unknown container ID: " + containerId);
+      }
+    }
+
+    @Override
+    public void onContainerStatusReceived(ContainerId containerId,
+        ContainerStatus containerStatus) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Container Status: id=" + containerId + ", status="
+            + containerStatus);
+      }
+    }
+
+    @Override
+    public void onContainerStarted(ContainerId containerId,
+        Map<String, ByteBuffer> allServiceResponse) {
+      if (isNameNode(containerId)) {
+        LOG.info("NameNode container started at ID " + containerId);
+      } else if (isDataNode(containerId)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Succeeded to start DataNode Container " + containerId);
+        }
+        nmClientAsync.getContainerStatusAsync(containerId,
+            datanodeContainers.get(containerId).getNodeId());
+      } else {
+        LOG.error(
+            "onContainerStarted received unknown container ID: " + containerId);
+      }
+    }
+
+    @Override
+    public void onStartContainerError(ContainerId containerId, Throwable t) {
+      if (isNameNode(containerId)) {
+        LOG.error("Failed to start namenode container ID " + containerId, t);
+        namenodeContainer = null;
+        markCompleted();
+      } else if (isDataNode(containerId)) {
+        LOG.error("Failed to start DataNode Container " + containerId);
+        datanodeContainers.remove(containerId);
+        numCompletedDataNodeContainers.incrementAndGet();
+        numFailedDataNodeContainers.incrementAndGet();
+      } else {
+        LOG.error("onStartContainerError received unknown container ID: "
+            + containerId);
+      }
+    }
+
+    @Override
+    public void onGetContainerStatusError(ContainerId containerId,
+        Throwable t) {
+      LOG.error("Failed to query the status of Container " + containerId);
+    }
+
+    @Override
+    public void onStopContainerError(ContainerId containerId, Throwable t) {
+      if (isNameNode(containerId)) {
+        LOG.error("Failed to stop NameNode container ID " + containerId);
+        namenodeContainer = null;
+      } else if (isDataNode(containerId)) {
+        LOG.error("Failed to stop DataNode Container " + containerId);
+        datanodeContainers.remove(containerId);
+      } else {
+        LOG.error("onStopContainerError received unknown containerID: "
+            + containerId);
+      }
+    }
+
+    @Override
+    @Deprecated
+    public void onContainerResourceIncreased(ContainerId containerId,
+        Resource resource) {
+      LOG.info("onContainerResourceIncreased: {}, {}", containerId, resource);
+    }
+
+    @Override
+    public void onContainerResourceUpdated(ContainerId containerId,
+        Resource resource) {
+      LOG.info("onContainerResourceUpdated: {}, {}", containerId, resource);
+    }
+
+    @Override
+    @Deprecated
+    public void onIncreaseContainerResourceError(ContainerId containerId,
+        Throwable t) {
+      LOG.info("onIncreaseContainerResourceError: {}", containerId, t);
+    }
+
+    @Override
+    public void onUpdateContainerResourceError(ContainerId containerId,
+        Throwable t) {
+      LOG.info("onUpdateContainerResourceError: {}", containerId, t);
+    }
+  }
+
+  /**
+   * Thread to connect to the {@link ContainerManagementProtocol} and launch the
+   * container that will execute the shell command.
+   */
+  private class LaunchContainerRunnable implements Runnable {
+
+    // Allocated container
+    private Container container;
+    private boolean isNameNodeLauncher;
+
+    /**
+     * @param lcontainer Allocated container
+     * @param isNameNode True iff this should launch a NameNode
+     */
+    LaunchContainerRunnable(Container lcontainer, boolean isNameNode) {
+      this.container = lcontainer;
+      this.isNameNodeLauncher = isNameNode;
+    }
+
+    /**
+     * Get the map of local resources to be used for launching this container.
+     */
+    private Map<String, LocalResource> getLocalResources() {
+      Map<String, LocalResource> localResources = new HashMap<>();
+
+      Map<String, String> envs = System.getenv();
+      addAsLocalResourceFromEnv(DynoConstants.CONF_ZIP, localResources, envs);
+      addAsLocalResourceFromEnv(DynoConstants.START_SCRIPT, localResources,
+          envs);
+      addAsLocalResourceFromEnv(DynoConstants.HADOOP_BINARY, localResources,
+          envs);
+      addAsLocalResourceFromEnv(DynoConstants.VERSION, localResources, envs);
+      addAsLocalResourceFromEnv(DynoConstants.DYNO_DEPENDENCIES, localResources,
+          envs);
+      if (isNameNodeLauncher) {
+        addAsLocalResourceFromEnv(DynoConstants.FS_IMAGE, localResources, envs);
+        addAsLocalResourceFromEnv(DynoConstants.FS_IMAGE_MD5, localResources,
+            envs);
+      } else {
+        int blockFilesToLocalize = Math.max(1,
+            amOptions.getDataNodesPerCluster());
+        for (int i = 0; i < blockFilesToLocalize; i++) {
+          try {
+            localResources.put(
+                DynoConstants.BLOCK_LIST_RESOURCE_PATH_PREFIX + i,
+                blockListFiles.remove(0));
+          } catch (IndexOutOfBoundsException e) {
+            break;
+          }
+        }
+      }
+      return localResources;
+    }
+
+    /**
+     * Connects to CM, sets up container launch context for shell command and
+     * eventually dispatches the container start request to the CM.
+     */
+    @Override
+    public void run() {
+      LOG.info("Setting up container launch context for containerid="
+          + container.getId() + ", isNameNode=" + isNameNodeLauncher);
+      ContainerLaunchContext ctx = Records
+          .newRecord(ContainerLaunchContext.class);
+
+      // Set the environment
+      ctx.setEnvironment(amOptions.getShellEnv());
+      ctx.setApplicationACLs(applicationAcls);
+
+      try {
+        ctx.setLocalResources(getLocalResources());
+
+        ctx.setCommands(getContainerStartCommand());
+      } catch (IOException e) {
+        LOG.error("Error while configuring container!", e);
+        return;
+      }
+
+      // Set up tokens for the container
+      ctx.setTokens(allTokens.duplicate());
+
+      nmClientAsync.startContainerAsync(container, ctx);
+      LOG.info("Starting {}; track at: http://{}/node/containerlogs/{}/{}/",
+          isNameNodeLauncher ? "NAMENODE" : "DATANODE",
+          container.getNodeHttpAddress(), container.getId(), launchingUser);
+    }
+
+    /**
+     * Return the command used to start this container.
+     */
+    private List<String> getContainerStartCommand() throws IOException {
+      // Set the necessary command to execute on the allocated container
+      List<String> vargs = new ArrayList<>();
+
+      // Set executable command
+      vargs.add("./" + DynoConstants.START_SCRIPT.getResourcePath());
+      String component = isNameNodeLauncher ? "namenode" : "datanode";
+      vargs.add(component);
+      if (isNameNodeLauncher) {
+        vargs.add(remoteStoragePath.getFileSystem(conf)
+            .makeQualified(remoteStoragePath).toString());
+      } else {
+        vargs.add(namenodeServiceRpcAddress);
+        vargs.add(String.valueOf(amOptions.getDataNodeLaunchDelaySec() < 1 ? 0
+            : RAND.nextInt(
+                Ints.checkedCast(amOptions.getDataNodeLaunchDelaySec()))));
+      }
+
+      // Add log redirect params
+      vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
+      vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
+
+      LOG.info("Completed setting up command for " + component + ": " + vargs);
+      return Lists.newArrayList(Joiner.on(" ").join(vargs));
+    }
+
+    /**
+     * Add the given resource into the map of resources, using information from
+     * the supplied environment variables.
+     *
+     * @param resource The resource to add.
+     * @param localResources Map of local resources to insert into.
+     * @param env Map of environment variables.
+     */
+    public void addAsLocalResourceFromEnv(DynoResource resource,
+        Map<String, LocalResource> localResources, Map<String, String> env) {
+      LOG.debug("Adding resource to localResources: " + resource);
+      String resourcePath = resource.getResourcePath();
+      if (resourcePath == null) {
+        // Default to using the file name in the path
+        resourcePath = resource.getPath(env).getName();
+      }
+      localResources.put(resourcePath,
+          LocalResource.newInstance(URL.fromPath(resource.getPath(env)),
+              resource.getType(), LocalResourceVisibility.APPLICATION,
+              resource.getLength(env), resource.getTimestamp(env)));
+    }
+  }
+
+  private List<LocalResource> getDataNodeBlockListingFiles()
+      throws IOException {
+    Path blockListDirPath = new Path(
+        System.getenv().get(DynoConstants.BLOCK_LIST_PATH_ENV));
+    LOG.info("Looking for block listing files in " + blockListDirPath);
+    FileSystem blockZipFS = blockListDirPath.getFileSystem(conf);
+    List<LocalResource> files = new LinkedList<>();
+    for (FileStatus stat : blockZipFS.listStatus(blockListDirPath,
+        DynoConstants.BLOCK_LIST_FILE_FILTER)) {
+      LocalResource blockListResource = LocalResource.newInstance(
+          URL.fromPath(stat.getPath()),
+          LocalResourceType.FILE, LocalResourceVisibility.APPLICATION,
+          stat.getLen(), stat.getModificationTime());
+      files.add(blockListResource);
+    }
+    return files;
+  }
+
+  /**
+   * Return true iff {@code containerId} represents the NameNode container.
+   */
+  private boolean isNameNode(ContainerId containerId) {
+    return namenodeContainer != null
+        && namenodeContainer.getId().equals(containerId);
+  }
+
+  /**
+   * Return true iff {@code containerId} represents a DataNode container.
+   */
+  private boolean isDataNode(ContainerId containerId) {
+    return datanodeContainers.containsKey(containerId);
+  }
+
+  /**
+   * Setup the request that will be sent to the RM for the container ask.
+   *
+   * @return the setup ResourceRequest to be sent to RM
+   */
+  private ContainerRequest setupContainerAskForRM(int memory, int vcores,
+      int priority, String nodeLabel) {
+    Priority pri = Records.newRecord(Priority.class);
+    pri.setPriority(priority);
+
+    // Set up resource type requirements
+    // For now, memory and CPU are supported so we set memory and cpu
+    // requirements
+    Resource capability = Records.newRecord(Resource.class);
+    capability.setMemorySize(memory);
+    capability.setVirtualCores(vcores);
+
+    return new ContainerRequest(capability, null, null, pri, true, nodeLabel);
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/BlockPlacementPolicyAlwaysSatisfied.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/BlockPlacementPolicyAlwaysSatisfied.java
new file mode 100644
index 0000000..34c1951
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/BlockPlacementPolicyAlwaysSatisfied.java
@@ -0,0 +1,53 @@
+/**
+ * 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.hadoop.tools.dynamometer;
+
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
+
+/**
+ * A BlockPlacementPolicy which always considered itself satisfied. This avoids
+ * the issue that the Dynamometer NameNode will complain about blocks being
+ * under-replicated because they're not being put on distinct racks.
+ */
+public class BlockPlacementPolicyAlwaysSatisfied
+    extends BlockPlacementPolicyDefault {
+
+  private static final BlockPlacementStatusSatisfied SATISFIED =
+      new BlockPlacementStatusSatisfied();
+
+  private static class BlockPlacementStatusSatisfied
+      implements BlockPlacementStatus {
+    @Override
+    public boolean isPlacementPolicySatisfied() {
+      return true;
+    }
+
+    public String getErrorDescription() {
+      return null;
+    }
+  }
+
+  @Override
+  public BlockPlacementStatus verifyBlockPlacement(DatanodeInfo[] locs,
+      int numberOfReplicas) {
+    return SATISFIED;
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/Client.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/Client.java
new file mode 100644
index 0000000..2cd6cbf
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/Client.java
@@ -0,0 +1,1132 @@
+/**
+ * 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.hadoop.tools.dynamometer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+import java.util.Optional;
+import java.util.function.Supplier;
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditReplayMapper;
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.WorkloadDriver;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionGroup;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.ClassUtil;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+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.ContainerLaunchContext;
+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.QueueInfo;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
+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.util.Records;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Client for submitting a Dynamometer YARN application, and optionally, a
+ * workload MapReduce job. This client uploads resources to HDFS as necessary
+ * for them to be accessed by the YARN app, then launches an
+ * {@link ApplicationMaster}, which is responsible for managing the lifetime of
+ * the application.
+ *
+ * <p/>The Dynamometer YARN application starts up the DataNodes of an HDFS
+ * cluster. If the namenode_servicerpc_addr option is specified, it should point
+ * to the service RPC address of an existing namenode, which the datanodes will
+ * talk to. Else, a namenode will be launched internal to this YARN application.
+ * The ApplicationMaster's logs contain links to the NN / DN containers to be
+ * able to access their logs. Some of this information is also printed by the
+ * client.
+ *
+ * <p/>The application will store files in the submitting user's home directory
+ * under a `.dynamometer/applicationID/` folder. This is mostly for uses
+ * internal to the application, but if the NameNode is launched through YARN,
+ * the NameNode's metrics will also be uploaded to a file `namenode_metrics`
+ * within this folder. This file is also accessible as part of the NameNode's
+ * logs, but this centralized location is easier to access for subsequent
+ * parsing.
+ *
+ * <p/>If the NameNode is launched internally, this Client will monitor the
+ * status of the NameNode, printing information about its availability as the
+ * DataNodes register (e.g., outstanding under replicated blocks as block
+ * reports arrive). If this is configured to launch the workload job, once the
+ * NameNode has gathered information from all of its DataNodes, the client will
+ * launch a workload job which is configured to act against the newly launched
+ * NameNode. Once the workload job completes, the infrastructure application
+ * will be shut down. At this time only the audit log replay
+ * ({@link AuditReplayMapper}) workload is supported.
+ *
+ * <p/>If there is no workload job configured, this application will, by
+ * default, persist indefinitely until killed by YARN. You can specify the
+ * timeout option to have it exit automatically after some time. This timeout
+ * will enforced if there is a workload job configured as well.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class Client extends Configured implements Tool {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Client.class);
+
+  public static final String APPNAME_ARG = "appname";
+  public static final String APPNAME_DEFAULT = "DynamometerTest";
+  public static final String QUEUE_ARG = "queue";
+  public static final String QUEUE_DEFAULT = "default";
+  public static final String TIMEOUT_ARG = "timeout";
+  public static final String TIMEOUT_DEFAULT = "-1";
+  public static final String HADOOP_VERSION_ARG = "hadoop_version";
+  public static final String HADOOP_BINARY_PATH_ARG = "hadoop_binary_path";
+  public static final String NAMENODE_SERVICERPC_ADDR_ARG =
+      "namenode_servicerpc_addr";
+  public static final String FS_IMAGE_DIR_ARG = "fs_image_dir";
+  public static final String BLOCK_LIST_PATH_ARG = "block_list_path";
+  public static final String CONF_PATH_ARG = "conf_path";
+  public static final String MASTER_VCORES_ARG = "master_vcores";
+  public static final String MASTER_VCORES_DEFAULT = "1";
+  public static final String MASTER_MEMORY_MB_ARG = "master_memory_mb";
+  public static final String MASTER_MEMORY_MB_DEFAULT = "2048";
+  public static final String TOKEN_FILE_LOCATION_ARG = "token_file_location";
+  public static final String WORKLOAD_REPLAY_ENABLE_ARG =
+      "workload_replay_enable";
+  public static final String WORKLOAD_INPUT_PATH_ARG = "workload_input_path";
+  public static final String WORKLOAD_THREADS_PER_MAPPER_ARG =
+      "workload_threads_per_mapper";
+  public static final String WORKLOAD_START_DELAY_ARG = "workload_start_delay";
+  public static final String WORKLOAD_RATE_FACTOR_ARG = "workload_rate_factor";
+  public static final String WORKLOAD_RATE_FACTOR_DEFAULT = "1.0";
+  public static final String WORKLOAD_CONFIG_ARG = "workload_config";
+
+  private static final String[] ARCHIVE_FILE_TYPES =
+      {".zip", ".tar", ".tgz", ".tar.gz"};
+
+  private static final String START_SCRIPT_LOCATION = Client.class
+      .getClassLoader()
+      .getResource(DynoConstants.START_SCRIPT.getResourcePath()).toString();
+
+  private YarnClient yarnClient;
+  // Application master specific info to register a new Application with RM/ASM
+  private String appName = "";
+  // Queue for App master
+  private String amQueue = "";
+  // Amt. of memory resource to request for to run the App Master
+  private int amMemory = 10;
+  // Amt. of virtual core resource to request for to run the App Master
+  private int amVCores = 1;
+
+  // Dependency JARs. Should include, at minimum, the JAR for the App Master
+  private final String[] dependencyJars;
+
+  private String hadoopBinary = "";
+  // Location of DN conf zip
+  private String confPath = "";
+  // Location of root dir for DN block image zips
+  private String blockListPath = "";
+  // Location of NN fs image
+  private String fsImagePath = "";
+  // Location of NN fs image md5 file
+  private String fsImageMD5Path = "";
+  // Location of NN VERSION file
+  private String versionFilePath = "";
+  // Service RPC address of the NameNode, if it is external
+  private String remoteNameNodeRpcAddress = "";
+  // True iff the NameNode should be launched within YARN
+  private boolean launchNameNode;
+  // The path to the file which contains the delegation tokens to be used for
+  // the launched
+  // containers (may be null)
+  private String tokenFileLocation;
+
+  // Holds all of the options which are passed to the AM
+  private AMOptions amOptions;
+
+  // The ApplicationId of the YARN infrastructure application.
+  private ApplicationId infraAppId;
+  // The current state of the YARN infrastructure application.
+  private volatile YarnApplicationState infraAppState =
+      YarnApplicationState.NEW;
+  private volatile JobStatus.State workloadAppState = JobStatus.State.PREP;
+  // Total number of DataNodes which will be launched.
+  private int numTotalDataNodes;
+
+  // Whether or not the workload job should be launched.
+  private boolean launchWorkloadJob = false;
+  // The workload job itself.
+  private volatile Job workloadJob;
+  // The input path for the workload job.
+  private String workloadInputPath = "";
+  // The number of threads to use per mapper for the workload job.
+  private int workloadThreadsPerMapper;
+  // The startup delay for the workload job.
+  private long workloadStartDelayMs;
+  private double workloadRateFactor = 0.0;
+  private Map<String, String> workloadExtraConfigs;
+
+  // Start time for client
+  private final long clientStartTime = System.currentTimeMillis();
+  // Timeout threshold for client. Kill app after time interval expires.
+  private long clientTimeout;
+
+  // Command line options
+  private Options opts;
+
+  /**
+   * @param args
+   *          Command line arguments
+   */
+  public static void main(String[] args) throws Exception {
+    Client client = new Client(
+        ClassUtil.findContainingJar(ApplicationMaster.class),
+        // JUnit is required by MiniDFSCluster at runtime, but is not included
+        // in standard Hadoop dependencies, so it must explicitly included here
+        ClassUtil.findContainingJar(Assert.class));
+    System.exit(ToolRunner.run(new YarnConfiguration(), client, args));
+  }
+
+  public int run(String[] args) {
+    boolean result;
+    try {
+      LOG.info("Initializing Client");
+      try {
+        boolean doRun = init(args);
+        if (!doRun) {
+          return 0;
+        }
+      } catch (IllegalArgumentException e) {
+        System.err.println(e.getLocalizedMessage());
+        printUsage();
+        return -1;
+      }
+      result = run();
+    } catch (Throwable t) {
+      LOG.error("Error running Client", t);
+      return 1;
+    }
+    if (result) {
+      LOG.info("Application completed successfully");
+      return 0;
+    }
+    LOG.error("Application failed to complete successfully");
+    return 2;
+  }
+
+  public Client(String... dependencyJars) {
+    Preconditions.checkArgument(
+        dependencyJars != null && dependencyJars.length > 0,
+        "Must specify at least one dependency JAR for the ApplicationMaster");
+    this.dependencyJars = dependencyJars;
+    opts = new Options();
+    opts.addOption(APPNAME_ARG, true,
+        "Application Name. (default '" + APPNAME_DEFAULT + "')");
+    opts.addOption(QUEUE_ARG, true, "RM Queue in which this application is "
+        + "to be submitted (default '" + QUEUE_DEFAULT + "')");
+    opts.addOption(TIMEOUT_ARG, true, "Application timeout in milliseconds "
+        + "(default " + TIMEOUT_DEFAULT + " = unlimited)");
+    opts.addOption(MASTER_MEMORY_MB_ARG, true, "Amount of memory in MB to be "
+        + "requested to run the application master (default "
+        + MASTER_MEMORY_MB_DEFAULT + ")");
+    opts.addOption(MASTER_VCORES_ARG, true, "Amount of virtual cores to be "
+        + "requested to run the application master (default "
+        + MASTER_VCORES_DEFAULT + ")");
+    // Dynamometer
+    opts.addOption(CONF_PATH_ARG, true, "Location of the directory or archive "
+        + "containing the Hadoop configuration. If this is already on a "
+        + "remote FS, will save the copy step, but must be an archive file. "
+        + "This must have the standard Hadoop conf layout containing e.g. "
+        + "etc/hadoop/*-site.xml");
+    opts.addOption(BLOCK_LIST_PATH_ARG, true,
+        "Location on HDFS of the files containing the DN block lists.");
+    opts.addOption(FS_IMAGE_DIR_ARG, true, "Location of the directory "
+        + "containing, at minimum, the VERSION file for the namenode. If "
+        + "running the namenode within YARN (namenode_info_path is not "
+        + "specified), this must also include the fsimage file and its md5 "
+        + "hash with names conforming to: `fsimage_XXXXXXXX[.md5]`.");
+    for (String option :
+        new String[] {CONF_PATH_ARG, BLOCK_LIST_PATH_ARG, FS_IMAGE_DIR_ARG}) {
+      opts.getOption(option).setRequired(true);
+    }
+    OptionGroup hadoopBinaryGroup = new OptionGroup();
+    hadoopBinaryGroup.addOption(new Option(HADOOP_BINARY_PATH_ARG, true,
+        "Location of Hadoop binary to be deployed (archive). One of this or "
+            + "hadoop_version is required."));
+    hadoopBinaryGroup.addOption(new Option(HADOOP_VERSION_ARG, true,
+        "Version of Hadoop (like '2.7.4' or '3.0.0-beta1') for which to "
+            + "download a binary. If this is specified, a Hadoop tarball "
+            + "will be downloaded from an Apache mirror. By default the "
+            + "Berkeley OCF mirror is used; specify "
+            + DynoInfraUtils.APACHE_DOWNLOAD_MIRROR_KEY
+            + " as a configuration or system property to change which mirror "
+            + "is used. The tarball will be downloaded to the working "
+            + "directory. One of this or hadoop_binary_path is required."));
+    hadoopBinaryGroup.setRequired(true);
+    opts.addOptionGroup(hadoopBinaryGroup);
+    opts.addOption(NAMENODE_SERVICERPC_ADDR_ARG, true, "Specify this option "
+        + "to run the NameNode external to YARN. This is the service RPC "
+        + "address of the NameNode, e.g. localhost:9020.");
+    opts.addOption(TOKEN_FILE_LOCATION_ARG, true, "If specified, this file "
+        + "will be used as the delegation token(s) for the launched "
+        + "containers. Otherwise, the delegation token(s) for the default "
+        + "FileSystem will be used.");
+    AMOptions.setOptions(opts);
+
+    opts.addOption(WORKLOAD_REPLAY_ENABLE_ARG, false, "If specified, this "
+        + "client will additionally launch the workload replay job to replay "
+        + "audit logs against the HDFS cluster which is started.");
+    opts.addOption(WORKLOAD_INPUT_PATH_ARG, true,
+        "Location of the audit traces to replay (Required for workload)");
+    opts.addOption(WORKLOAD_THREADS_PER_MAPPER_ARG, true, "Number of threads "
+        + "per mapper to use to replay the workload. (default "
+        + AuditReplayMapper.NUM_THREADS_DEFAULT + ")");
+    opts.addOption(WORKLOAD_START_DELAY_ARG, true, "Delay between launching "
+        + "the Workload MR job and starting the audit logic replay; this is "
+        + "used in an attempt to allow all mappers to be launched before any "
+        + "of them start replaying. Workloads with more mappers may need a "
+        + "longer delay to get all of the containers allocated. Human-readable "
+        + "units accepted (e.g. 30s, 10m). (default "
+        + WorkloadDriver.START_TIME_OFFSET_DEFAULT + ")");
+    opts.addOption(WORKLOAD_RATE_FACTOR_ARG, true, "Rate factor "
+        + "(multiplicative speed factor) to apply to workload replay (Default "
+        + WORKLOAD_RATE_FACTOR_DEFAULT + ")");
+    opts.addOption(WORKLOAD_CONFIG_ARG, true, "Additional configurations to "
+        + "pass only to the workload job. This can be used multiple times "
+        + "and should be specified as a key=value pair, e.g. '-"
+        + WORKLOAD_CONFIG_ARG + " conf.one=val1 -" + WORKLOAD_CONFIG_ARG
+        + " conf.two=val2'");
+  }
+
+  /**
+   * Helper function to print out usage.
+   */
+  private void printUsage() {
+    HelpFormatter formatter = new HelpFormatter();
+    // Option names are long so increasing the width is helpful
+    formatter.setWidth(100);
+    formatter.printHelp("Client", opts);
+  }
+
+  /**
+   * Parse command line options.
+   *
+   * @param args Parsed command line options
+   * @return Whether the init was successful to run the client
+   */
+  public boolean init(String[] args) throws ParseException, IOException {
+
+    CommandLineParser parser = new GnuParser();
+    if (parser.parse(
+        new Options().addOption("h", "help", false, "Shows this message."),
+        args, true).hasOption("h")) {
+      printUsage();
+      return false;
+    }
+
+    CommandLine cliParser = parser.parse(opts, args);
+
+    yarnClient = YarnClient.createYarnClient();
+    yarnClient.init(getConf());
+
+    LOG.info("Starting with arguments: [\"{}\"]",
+        Joiner.on("\" \"").join(args));
+
+    Path fsImageDir = new Path(cliParser.getOptionValue(FS_IMAGE_DIR_ARG, ""));
+    versionFilePath = new Path(fsImageDir, "VERSION").toString();
+    if (cliParser.hasOption(NAMENODE_SERVICERPC_ADDR_ARG)) {
+      launchNameNode = false;
+      remoteNameNodeRpcAddress =
+          cliParser.getOptionValue(NAMENODE_SERVICERPC_ADDR_ARG);
+    } else {
+      launchNameNode = true;
+      FileSystem localFS = FileSystem.getLocal(getConf());
+      fsImageDir = fsImageDir.makeQualified(localFS.getUri(),
+          localFS.getWorkingDirectory());
+      FileSystem fsImageFS = fsImageDir.getFileSystem(getConf());
+      FileStatus[] fsImageFiles = fsImageFS.listStatus(fsImageDir,
+          (path) -> path.getName().matches("^fsimage_(\\d)+$"));
+      if (fsImageFiles.length != 1) {
+        throw new IllegalArgumentException(
+            "Must be exactly one fsimage file present in fs_image_dir");
+      }
+      fsImagePath = fsImageFiles[0].getPath().toString();
+      fsImageMD5Path = fsImageFiles[0].getPath().suffix(".md5").toString();
+    }
+
+    if (amMemory < 0) {
+      throw new IllegalArgumentException("Invalid memory specified for "
+          + "application master, exiting. Specified memory=" + amMemory);
+    }
+    if (amVCores < 0) {
+      throw new IllegalArgumentException("Invalid virtual cores specified for "
+          + "application master, exiting. Specified virtual cores=" + amVCores);
+    }
+
+    this.appName = cliParser.getOptionValue(APPNAME_ARG, APPNAME_DEFAULT);
+    this.amQueue = cliParser.getOptionValue(QUEUE_ARG, QUEUE_DEFAULT);
+    this.amMemory = Integer.parseInt(cliParser
+        .getOptionValue(MASTER_MEMORY_MB_ARG, MASTER_MEMORY_MB_DEFAULT));
+    this.amVCores = Integer.parseInt(
+        cliParser.getOptionValue(MASTER_VCORES_ARG, MASTER_VCORES_DEFAULT));
+    this.confPath = cliParser.getOptionValue(CONF_PATH_ARG);
+    this.blockListPath = cliParser.getOptionValue(BLOCK_LIST_PATH_ARG);
+    if (cliParser.hasOption(HADOOP_BINARY_PATH_ARG)) {
+      this.hadoopBinary = cliParser.getOptionValue(HADOOP_BINARY_PATH_ARG);
+    } else {
+      this.hadoopBinary = DynoInfraUtils.fetchHadoopTarball(
+          new File(".").getAbsoluteFile(),
+          cliParser.getOptionValue(HADOOP_VERSION_ARG), getConf(), LOG)
+          .toString();
+    }
+    this.amOptions = AMOptions.initFromParser(cliParser);
+    this.clientTimeout = Integer
+        .parseInt(cliParser.getOptionValue(TIMEOUT_ARG, TIMEOUT_DEFAULT));
+    this.tokenFileLocation = cliParser.getOptionValue(TOKEN_FILE_LOCATION_ARG);
+
+    amOptions.verify();
+
+    Path blockPath = new Path(blockListPath);
+    FileSystem blockListFS = blockPath.getFileSystem(getConf());
+    if (blockListFS.getUri().equals(FileSystem.getLocal(getConf()).getUri())
+        || !blockListFS.exists(blockPath)) {
+      throw new IllegalArgumentException(
+          "block list path must already exist on remote fs!");
+    }
+    numTotalDataNodes = blockListFS.listStatus(blockPath,
+        DynoConstants.BLOCK_LIST_FILE_FILTER).length;
+
+    if (cliParser.hasOption(WORKLOAD_REPLAY_ENABLE_ARG)) {
+      if (!cliParser.hasOption(WORKLOAD_INPUT_PATH_ARG)
+          || !cliParser.hasOption(WORKLOAD_START_DELAY_ARG)) {
+        throw new IllegalArgumentException("workload_replay_enable was "
+            + "specified; must include all required workload_ parameters.");
+      }
+      launchWorkloadJob = true;
+      workloadInputPath = cliParser.getOptionValue(WORKLOAD_INPUT_PATH_ARG);
+      workloadThreadsPerMapper = Integer
+          .parseInt(cliParser.getOptionValue(WORKLOAD_THREADS_PER_MAPPER_ARG,
+              String.valueOf(AuditReplayMapper.NUM_THREADS_DEFAULT)));
+      workloadRateFactor = Double.parseDouble(cliParser.getOptionValue(
+          WORKLOAD_RATE_FACTOR_ARG, WORKLOAD_RATE_FACTOR_DEFAULT));
+      workloadExtraConfigs = new HashMap<>();
+      if (cliParser.getOptionValues(WORKLOAD_CONFIG_ARG) != null) {
+        for (String opt : cliParser.getOptionValues(WORKLOAD_CONFIG_ARG)) {
+          Iterator<String> kvPair =
+              Splitter.on("=").trimResults().split(opt).iterator();
+          workloadExtraConfigs.put(kvPair.next(), kvPair.next());
+        }
+      }
+      String delayString = cliParser.getOptionValue(WORKLOAD_START_DELAY_ARG,
+          WorkloadDriver.START_TIME_OFFSET_DEFAULT);
+      // Store a temporary config to leverage Configuration's time duration
+      // parsing.
+      getConf().set("___temp___", delayString);
+      workloadStartDelayMs = getConf().getTimeDuration("___temp___", 0,
+          TimeUnit.MILLISECONDS);
+    }
+
+    return true;
+  }
+
+  /**
+   * Main run function for the client.
+   *
+   * @return true if application completed successfully
+   */
+  public boolean run() throws IOException, YarnException {
+
+    LOG.info("Running Client");
+    yarnClient.start();
+
+    YarnClusterMetrics clusterMetrics = yarnClient.getYarnClusterMetrics();
+    LOG.info("Got Cluster metric info from ASM, numNodeManagers={}",
+        clusterMetrics.getNumNodeManagers());
+
+    QueueInfo queueInfo = yarnClient.getQueueInfo(this.amQueue);
+    LOG.info("Queue info: queueName={}, queueCurrentCapacity={}, "
+        + "queueMaxCapacity={}, queueApplicationCount={}, "
+        + "queueChildQueueCount={}", queueInfo.getQueueName(),
+        queueInfo.getCurrentCapacity(), queueInfo.getMaximumCapacity(),
+        queueInfo.getApplications().size(), queueInfo.getChildQueues().size());
+
+    // Get a new application id
+    YarnClientApplication app = yarnClient.createApplication();
+    GetNewApplicationResponse appResponse = app.getNewApplicationResponse();
+    long maxMem = appResponse.getMaximumResourceCapability().getMemorySize();
+    LOG.info("Max mem capabililty of resources in this cluster " + maxMem);
+    int maxVCores = appResponse.getMaximumResourceCapability()
+        .getVirtualCores();
+    LOG.info("Max virtual cores capabililty of resources in this cluster {}",
+        maxVCores);
+    if (amMemory > maxMem || amMemory < 0 || amVCores > maxVCores
+        || amVCores < 0) {
+      throw new IllegalArgumentException("Invalid AM memory or vcores: memory="
+          + amMemory + ", vcores=" + amVCores);
+    }
+    amOptions.verify(maxMem, maxVCores);
+
+    // set the application name
+    ApplicationSubmissionContext appContext =
+        app.getApplicationSubmissionContext();
+    infraAppId = appContext.getApplicationId();
+    appContext.setApplicationName(appName);
+
+    // Set up the container launch context for the application master
+    ContainerLaunchContext amContainer = Records
+        .newRecord(ContainerLaunchContext.class);
+    Map<ApplicationAccessType, String> acls = new HashMap<>();
+    acls.put(ApplicationAccessType.VIEW_APP, getConf().get(
+        MRJobConfig.JOB_ACL_VIEW_JOB, MRJobConfig.DEFAULT_JOB_ACL_VIEW_JOB));
+    amContainer.setApplicationACLs(acls);
+
+    FileSystem fs = FileSystem.get(getConf());
+    fs.mkdirs(getRemoteStoragePath(getConf(), infraAppId));
+
+    // Set the env variables to be setup in the env where the application master
+    // will be run
+    Map<String, String> env = setupRemoteResourcesGetEnv();
+
+    amContainer.setEnvironment(env);
+
+    // All of the resources for both AM and NN/DNs have been put on remote
+    // storage
+    // Only the application master JAR is needed as a local resource for the AM
+    // so
+    // we explicitly add it here
+    Map<String, LocalResource> localResources = new HashMap<>();
+    LocalResource scRsrc = LocalResource.newInstance(
+        org.apache.hadoop.yarn.api.records.URL
+            .fromPath(DynoConstants.DYNO_DEPENDENCIES.getPath(env)),
+        LocalResourceType.ARCHIVE, LocalResourceVisibility.APPLICATION,
+        DynoConstants.DYNO_DEPENDENCIES.getLength(env),
+        DynoConstants.DYNO_DEPENDENCIES.getTimestamp(env));
+    localResources.put(DynoConstants.DYNO_DEPENDENCIES.getResourcePath(),
+        scRsrc);
+    // Set local resource info into app master container launch context
+    amContainer.setLocalResources(localResources);
+
+    // Set the necessary command to execute the application master
+    amContainer.setCommands(getAMCommand());
+
+    Resource capability = Records.newRecord(Resource.class);
+    capability.setMemorySize(amMemory);
+    capability.setVirtualCores(amVCores);
+    appContext.setResource(capability);
+
+    // Setup security tokens
+    if (UserGroupInformation.isSecurityEnabled()) {
+      ByteBuffer fsTokens;
+      if (tokenFileLocation != null) {
+        fsTokens = ByteBuffer
+            .wrap(Files.readAllBytes(Paths.get(tokenFileLocation)));
+      } else {
+        Credentials credentials = new Credentials();
+        String tokenRenewer = getConf().get(YarnConfiguration.RM_PRINCIPAL);
+        if (tokenRenewer == null || tokenRenewer.length() == 0) {
+          throw new IOException("Can't get Master Kerberos principal for the "
+              + "RM to use as renewer");
+        }
+
+        // For now, only getting tokens for the default file-system.
+        final Token<?>[] tokens = fs.addDelegationTokens(tokenRenewer,
+            credentials);
+        if (tokens != null) {
+          for (Token<?> token : tokens) {
+            LOG.info("Got dt for " + fs.getUri() + "; " + token);
+          }
+        }
+        DataOutputBuffer dob = new DataOutputBuffer();
+        credentials.writeTokenStorageToStream(dob);
+        fsTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+      }
+      amContainer.setTokens(fsTokens);
+    }
+
+    appContext.setAMContainerSpec(amContainer);
+
+    // Set the queue to which this application is to be submitted in the RM
+    appContext.setQueue(amQueue);
+
+    LOG.info("Submitting application to RM");
+    yarnClient.submitApplication(appContext);
+
+    // Monitor the application
+    return monitorInfraApplication();
+
+  }
+
+  /**
+   * Set up the remote resources for the application. Upload them to remote
+   * storage as necessary, and set up the requisite environment variables. Does
+   * not set up any local resources.
+   *
+   * @return A Map representing the environment to be used for the
+   *         ApplicationMaster containing the information about all of the
+   *         remote resources.
+   */
+  private Map<String, String> setupRemoteResourcesGetEnv() throws IOException {
+    LOG.info("Set the environment for the application master");
+    Map<String, String> env = new HashMap<>();
+
+    // Copy local resources to a remote FS to prepare them for localization
+    // by containers. We do not need to set them as local resources here as
+    // the AM does not need them.
+    if (launchNameNode) {
+      setupRemoteResource(infraAppId, DynoConstants.FS_IMAGE, env, fsImagePath);
+      setupRemoteResource(infraAppId, DynoConstants.FS_IMAGE_MD5, env,
+          fsImageMD5Path);
+    } else {
+      env.put(DynoConstants.REMOTE_NN_RPC_ADDR_ENV, remoteNameNodeRpcAddress);
+    }
+    setupRemoteResource(infraAppId, DynoConstants.VERSION, env,
+        versionFilePath);
+    setupRemoteResource(infraAppId, DynoConstants.CONF_ZIP, env, confPath);
+    setupRemoteResource(infraAppId, DynoConstants.START_SCRIPT, env,
+        START_SCRIPT_LOCATION);
+    setupRemoteResource(infraAppId, DynoConstants.HADOOP_BINARY, env,
+        hadoopBinary);
+    setupRemoteResource(infraAppId, DynoConstants.DYNO_DEPENDENCIES, env,
+        dependencyJars);
+
+    env.put(DynoConstants.BLOCK_LIST_PATH_ENV, blockListPath);
+    env.put(DynoConstants.JOB_ACL_VIEW_ENV, getConf().get(
+        MRJobConfig.JOB_ACL_VIEW_JOB, MRJobConfig.DEFAULT_JOB_ACL_VIEW_JOB));
+
+    env.put(DynoConstants.REMOTE_STORAGE_PATH_ENV,
+        getRemoteStoragePath(getConf(), infraAppId).toString());
+
+    env.put(Environment.CLASSPATH.key(), getAMClassPathEnv());
+    return env;
+  }
+
+  private String getAMClassPathEnv() {
+    // Add AppMaster.jar location to classpath
+    // At some point we should not be required to add
+    // the hadoop specific classpaths to the env.
+    // It should be provided out of the box.
+    // For now setting all required classpaths including
+    // the classpath to "." for the application jar
+    StringBuilder classPathEnv = new StringBuilder(Environment.CLASSPATH.$())
+        .append(ApplicationConstants.CLASS_PATH_SEPARATOR).append("./")
+        .append(DynoConstants.DYNO_DEPENDENCIES.getResourcePath()).append("/*");
+    for (String c : getConf().getStrings(
+        YarnConfiguration.YARN_APPLICATION_CLASSPATH,
+        YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH)) {
+      classPathEnv.append(ApplicationConstants.CLASS_PATH_SEPARATOR);
+      classPathEnv.append(c.trim());
+    }
+    classPathEnv.append(ApplicationConstants.CLASS_PATH_SEPARATOR)
+        .append("./log4j.properties");
+
+    // add the runtime classpath needed for tests to work
+    if (getConf().getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, false)) {
+      classPathEnv.append(ApplicationConstants.CLASS_PATH_SEPARATOR);
+      classPathEnv.append(System.getProperty("java.class.path"));
+    }
+    return classPathEnv.toString();
+  }
+
+  private List<String> getAMCommand() {
+    List<String> vargs = new ArrayList<>();
+
+    // Set java executable command
+    vargs.add(Environment.JAVA_HOME.$() + "/bin/java");
+    // Set Xmx based on am memory size
+    long appMasterHeapSize = Math.round(amMemory * 0.85);
+    vargs.add("-Xmx" + appMasterHeapSize + "m");
+    // Set class name
+    vargs.add(ApplicationMaster.class.getCanonicalName());
+    // Set params for Application Master
+
+    amOptions.addToVargs(vargs);
+
+    vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
+    vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
+
+    LOG.info("Completed setting up app master command: " + vargs);
+    return Lists.newArrayList(Joiner.on(" ").join(vargs));
+  }
+
+  /**
+   * Upload a local resource to HDFS, or if it is nonlocal, just set environment
+   * appropriately. The location, length and timestamp information is added to
+   * AM container's environment, so it can launch containers later with the
+   * correct resource settings.
+   */
+  private void setupRemoteResource(ApplicationId appId, DynoResource resource,
+      Map<String, String> env, String... srcPaths) throws IOException {
+
+    FileStatus remoteFileStatus;
+    Path dstPath;
+
+    Preconditions.checkArgument(srcPaths.length > 0,
+        "Must supply at least one source path");
+    Preconditions.checkArgument(
+        resource.getType() == LocalResourceType.ARCHIVE || srcPaths.length == 1,
+        "Can only specify multiple source paths if using an ARCHIVE type");
+
+    List<URI> srcURIs = Arrays.stream(srcPaths).map(URI::create)
+        .collect(Collectors.toList());
+    Set<String> srcSchemes = srcURIs.stream().map(URI::getScheme)
+        .collect(Collectors.toSet());
+    Preconditions.checkArgument(srcSchemes.size() == 1,
+        "All source paths must have the same scheme");
+    String srcScheme = srcSchemes.iterator().next();
+
+    String srcPathString = "[" + Joiner.on(",").join(srcPaths) + "]";
+
+    if (srcScheme == null
+        || srcScheme.equals(FileSystem.getLocal(getConf()).getScheme())
+        || srcScheme.equals("jar")) {
+      // Need to upload this resource to remote storage
+      List<File> srcFiles = srcURIs.stream().map(URI::getSchemeSpecificPart)
+          .map(File::new).collect(Collectors.toList());
+      Path dstPathBase = getRemoteStoragePath(getConf(), appId);
+      boolean shouldArchive = srcFiles.size() > 1
+          || srcFiles.get(0).isDirectory()
+          || (resource.getType() == LocalResourceType.ARCHIVE
+              && Arrays.stream(ARCHIVE_FILE_TYPES).noneMatch(
+                  suffix -> srcFiles.get(0).getName().endsWith(suffix)));
+      if (shouldArchive) {
+        if ("jar".equals(srcScheme)) {
+          throw new IllegalArgumentException(String.format("Resources in JARs "
+                  + "can't be zipped; resource %s is ARCHIVE and src is: %s",
+              resource.getResourcePath(), srcPathString));
+        } else if (resource.getType() != LocalResourceType.ARCHIVE) {
+          throw new IllegalArgumentException(
+              String.format("Resource type is %s but srcPaths were: %s",
+                  resource.getType(), srcPathString));
+        }
+        dstPath = new Path(dstPathBase, resource.getResourcePath())
+            .suffix(".zip");
+      } else {
+        dstPath = new Path(dstPathBase, srcFiles.get(0).getName());
+      }
+      FileSystem remoteFS = dstPath.getFileSystem(getConf());
+      LOG.info("Uploading resource " + resource + " from " + srcPathString
+          + " to " + dstPath);
+      try (OutputStream outputStream = remoteFS.create(dstPath, true)) {
+        if ("jar".equals(srcScheme)) {
+          try (InputStream inputStream = new URL(srcPaths[0]).openStream()) {
+            IOUtils.copyBytes(inputStream, outputStream, getConf());
+          }
+        } else if (shouldArchive) {
+          List<File> filesToZip;
+          if (srcFiles.size() == 1 && srcFiles.get(0).isDirectory()) {
+            File[] childFiles = srcFiles.get(0).listFiles();
+            if (childFiles == null || childFiles.length == 0) {
+              throw new IllegalArgumentException(
+                  "Specified a directory to archive with no contents");
+            }
+            filesToZip = Lists.newArrayList(childFiles);
+          } else {
+            filesToZip = srcFiles;
+          }
+          ZipOutputStream zout = new ZipOutputStream(outputStream);
+          for (File fileToZip : filesToZip) {
+            addFileToZipRecursively(fileToZip.getParentFile(), fileToZip, zout);
+          }
+          zout.close();
+        } else {
+          try (InputStream inputStream = new FileInputStream(srcFiles.get(0))) {
+            IOUtils.copyBytes(inputStream, outputStream, getConf());
+          }
+        }
+      }
+      remoteFileStatus = remoteFS.getFileStatus(dstPath);
+    } else {
+      if (srcPaths.length > 1) {
+        throw new IllegalArgumentException("If resource is on remote, must be "
+            + "a single file: " + srcPathString);
+      }
+      LOG.info("Using resource {} directly from current location: {}",
+          resource, srcPaths[0]);
+      dstPath = new Path(srcPaths[0]);
+      // non-local file system; we can just use it directly from where it is
+      remoteFileStatus = FileSystem.get(dstPath.toUri(), getConf())
+          .getFileStatus(dstPath);
+      if (remoteFileStatus.isDirectory()) {
+        throw new IllegalArgumentException("If resource is on remote "
+            + "filesystem, must be a file: " + srcPaths[0]);
+      }
+    }
+    env.put(resource.getLocationEnvVar(), dstPath.toString());
+    env.put(resource.getTimestampEnvVar(),
+        String.valueOf(remoteFileStatus.getModificationTime()));
+    env.put(resource.getLengthEnvVar(),
+        String.valueOf(remoteFileStatus.getLen()));
+  }
+
+  /**
+   * Get the directory on the default FS which will be used for storing files
+   * relevant to this Dynamometer application. This is inside of the
+   * {@value DynoConstants#DYNAMOMETER_STORAGE_DIR} directory within the
+   * submitter's home directory.
+   *
+   * @param conf Configuration for this application.
+   * @param appId This application's ID.
+   * @return Fully qualified path on the default FS.
+   */
+  private static Path getRemoteStoragePath(Configuration conf,
+      ApplicationId appId) throws IOException {
+    FileSystem remoteFS = FileSystem.get(conf);
+    return remoteFS.makeQualified(new Path(remoteFS.getHomeDirectory(),
+        DynoConstants.DYNAMOMETER_STORAGE_DIR + "/" + appId));
+  }
+
+  private void addFileToZipRecursively(File root, File file,
+      ZipOutputStream out) throws IOException {
+
+    File[] files = file.listFiles();
+    if (files == null) { // Not a directory
+      String relativePath = file.getAbsolutePath()
+          .substring(root.getAbsolutePath().length() + 1);
+      try {
+        FileInputStream in = new FileInputStream(file.getAbsolutePath());
+        out.putNextEntry(new ZipEntry(relativePath));
+        IOUtils.copyBytes(in, out, getConf(), false);
+        out.closeEntry();
+        in.close();
+      } catch (FileNotFoundException fnfe) {
+        LOG.warn("Skipping file; it is a symlink with a nonexistent target: {}",
+            file);
+      }
+    } else {
+      for (File containedFile : files) {
+        addFileToZipRecursively(root, containedFile, out);
+      }
+    }
+  }
+
+  /**
+   * Monitor the submitted application for completion. Kill application if time
+   * expires.
+   *
+   * @return true if application completed successfully
+   */
+  private boolean monitorInfraApplication() throws YarnException, IOException {
+
+    boolean loggedApplicationInfo = false;
+    boolean success = false;
+
+    Thread namenodeMonitoringThread = new Thread(() -> {
+      Supplier<Boolean> exitCritera = () -> isCompleted(infraAppState);
+      Optional<Properties> namenodeProperties = Optional.empty();
+      while (!exitCritera.get()) {
+        try {
+          if (!namenodeProperties.isPresent()) {
+            namenodeProperties = DynoInfraUtils
+                .waitForAndGetNameNodeProperties(exitCritera, getConf(),
+                    getNameNodeInfoPath(), LOG);
+            if (namenodeProperties.isPresent()) {
+              Properties props = namenodeProperties.get();
+              LOG.info("NameNode can be reached via HDFS at: {}",
+                  DynoInfraUtils.getNameNodeHdfsUri(props));
+              LOG.info("NameNode web UI available at: {}",
+                  DynoInfraUtils.getNameNodeWebUri(props));
+              LOG.info("NameNode can be tracked at: {}",
+                  DynoInfraUtils.getNameNodeTrackingUri(props));
+            } else {
+              // Only happens if we should be shutting down
+              break;
+            }
+          }
+          DynoInfraUtils.waitForNameNodeStartup(namenodeProperties.get(),
+              exitCritera, LOG);
+          DynoInfraUtils.waitForNameNodeReadiness(namenodeProperties.get(),
+              numTotalDataNodes, false, exitCritera, getConf(), LOG);
+          break;
+        } catch (IOException ioe) {
+          LOG.error(
+              "Unexpected exception while waiting for NameNode readiness",
+              ioe);
+        } catch (InterruptedException ie) {
+          return;
+        }
+      }
+      if (!isCompleted(infraAppState) && launchWorkloadJob) {
+        launchAndMonitorWorkloadDriver(namenodeProperties.get());
+      }
+    });
+    if (launchNameNode) {
+      namenodeMonitoringThread.start();
+    }
+
+    while (true) {
+
+      // Check app status every 1 second.
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+        LOG.debug("Thread sleep in monitoring loop interrupted");
+      }
+
+      // Get application report for the appId we are interested in
+      ApplicationReport report = yarnClient.getApplicationReport(infraAppId);
+
+      if (report.getTrackingUrl() != null && !loggedApplicationInfo) {
+        loggedApplicationInfo = true;
+        LOG.info("Track the application at: " + report.getTrackingUrl());
+        LOG.info("Kill the application using: yarn application -kill "
+            + report.getApplicationId());
+      }
+
+      LOG.debug("Got application report from ASM for: appId={}, "
+          + "clientToAMToken={}, appDiagnostics={}, appMasterHost={}, "
+          + "appQueue={}, appMasterRpcPort={}, appStartTime={}, "
+          + "yarnAppState={}, distributedFinalState={}, appTrackingUrl={}, "
+          + "appUser={}",
+          infraAppId.getId(), report.getClientToAMToken(),
+          report.getDiagnostics(), report.getHost(), report.getQueue(),
+          report.getRpcPort(), report.getStartTime(),
+          report.getYarnApplicationState(), report.getFinalApplicationStatus(),
+          report.getTrackingUrl(), report.getUser());
+
+      infraAppState = report.getYarnApplicationState();
+      if (infraAppState == YarnApplicationState.KILLED) {
+        if (!launchWorkloadJob) {
+          success = true;
+        } else if (workloadJob == null) {
+          LOG.error("Infra app was killed before workload job was launched.");
+        } else if (!workloadJob.isComplete()) {
+          LOG.error("Infra app was killed before workload job completed.");
+        } else if (workloadJob.isSuccessful()) {
+          success = true;
+        }
+        LOG.info("Infra app was killed; exiting from client.");
+        break;
+      } else if (infraAppState == YarnApplicationState.FINISHED
+          || infraAppState == YarnApplicationState.FAILED) {
+        LOG.info("Infra app exited unexpectedly. YarnState="
+            + infraAppState.toString() + ". Exiting from client.");
+        break;
+      }
+
+      if ((clientTimeout != -1)
+          && (System.currentTimeMillis() > (clientStartTime + clientTimeout))) {
+        LOG.info("Reached client specified timeout of {} ms for application. "
+            + "Killing application", clientTimeout);
+        attemptCleanup();
+        break;
+      }
+
+      if (isCompleted(workloadAppState)) {
+        LOG.info("Killing infrastructure app");
+        try {
+          forceKillApplication(infraAppId);
+        } catch (YarnException | IOException e) {
+          LOG.error("Exception encountered while killing infra app", e);
+        }
+      }
+    }
+    if (launchNameNode) {
+      try {
+        namenodeMonitoringThread.interrupt();
+        namenodeMonitoringThread.join();
+      } catch (InterruptedException ie) {
+        LOG.warn("Interrupted while joining workload job thread; "
+            + "continuing to cleanup.");
+      }
+    }
+    attemptCleanup();
+    return success;
+  }
+
+  /**
+   * Return the path to the property file containing information about the
+   * launched NameNode.
+   */
+  @VisibleForTesting
+  Path getNameNodeInfoPath() throws IOException {
+    return new Path(getRemoteStoragePath(getConf(), infraAppId),
+        DynoConstants.NN_INFO_FILE_NAME);
+  }
+
+  /**
+   * Launch the workload driver ({@link WorkloadDriver}) and monitor the job.
+   * Waits for the launched job to complete.
+   *
+   * @param nameNodeProperties The set of properties with information about the
+   *                           launched NameNode.
+   */
+  private void launchAndMonitorWorkloadDriver(Properties nameNodeProperties) {
+    URI nameNodeURI = DynoInfraUtils.getNameNodeHdfsUri(nameNodeProperties);
+    LOG.info("Launching workload job using input path: " + workloadInputPath);
+    try {
+      long workloadStartTime = System.currentTimeMillis()
+          + workloadStartDelayMs;
+      Configuration workloadConf = new Configuration(getConf());
+      workloadConf.set(AuditReplayMapper.INPUT_PATH_KEY, workloadInputPath);
+      workloadConf.setInt(AuditReplayMapper.NUM_THREADS_KEY,
+          workloadThreadsPerMapper);
+      workloadConf.setDouble(AuditReplayMapper.RATE_FACTOR_KEY,
+          workloadRateFactor);
+      for (Map.Entry<String, String> configPair : workloadExtraConfigs
+          .entrySet()) {
+        workloadConf.set(configPair.getKey(), configPair.getValue());
+      }
+      workloadJob = WorkloadDriver.getJobForSubmission(workloadConf,
+          nameNodeURI.toString(), workloadStartTime, AuditReplayMapper.class);
+      workloadJob.submit();
+      while (!isCompleted(infraAppState) && !isCompleted(workloadAppState)) {
+        workloadJob.monitorAndPrintJob();
+        Thread.sleep(5000);
+        workloadAppState = workloadJob.getJobState();
+      }
+      if (isCompleted(workloadAppState)) {
+        LOG.info("Workload job completed successfully!");
+      } else {
+        LOG.warn("Workload job failed.");
+      }
+    } catch (Exception e) {
+      LOG.error("Exception encountered while running workload job", e);
+    }
+  }
+
+  /**
+   * Best-effort attempt to clean up any remaining applications (infrastructure
+   * or workload).
+   */
+  public void attemptCleanup() {
+    LOG.info("Attempting to clean up remaining running applications.");
+    if (workloadJob != null) {
+      try {
+        workloadAppState = workloadJob.getJobState();
+      } catch (IOException ioe) {
+        LOG.warn("Unable to fetch completion status of workload job. Will "
+                + "proceed to attempt to kill it.", ioe);
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+        return;
+      }
+      if (!isCompleted(workloadAppState)) {
+        try {
+          LOG.info("Attempting to kill workload app: {}",
+              workloadJob.getJobID());
+          workloadJob.killJob();
+          LOG.info("Killed workload app");
+        } catch (IOException ioe) {
+          LOG.error("Unable to kill workload app ({})", workloadJob.getJobID(),
+              ioe);
+        }
+      }
+    }
+    if (infraAppId != null && !isCompleted(infraAppState)) {
+      try {
+        LOG.info("Attempting to kill infrastructure app: " + infraAppId);
+        forceKillApplication(infraAppId);
+        LOG.info("Killed infrastructure app");
+      } catch (YarnException | IOException e) {
+        LOG.error("Unable to kill infrastructure app ({})", infraAppId, e);
+      }
+    }
+  }
+
+  /**
+   * Check if the input state represents completion.
+   */
+  private static boolean isCompleted(JobStatus.State state) {
+    return state == JobStatus.State.SUCCEEDED || state == JobStatus.State.FAILED
+        || state == JobStatus.State.KILLED;
+  }
+
+  /**
+   * Check if the input state represents completion.
+   */
+  private static boolean isCompleted(YarnApplicationState state) {
+    return state == YarnApplicationState.FINISHED
+        || state == YarnApplicationState.FAILED
+        || state == YarnApplicationState.KILLED;
+  }
+
+  /**
+   * Kill a submitted application by sending a call to the AM.
+   *
+   * @param appId Application Id to be killed.
+   */
+  private void forceKillApplication(ApplicationId appId)
+      throws YarnException, IOException {
+    // Response can be ignored as it is non-null on success or
+    // throws an exception in case of failures
+    yarnClient.killApplication(appId);
+  }
+
+  @VisibleForTesting
+  Job getWorkloadJob() {
+    return workloadJob;
+  }
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/DynoConstants.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/DynoConstants.java
new file mode 100644
index 0000000..c789f67
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/DynoConstants.java
@@ -0,0 +1,130 @@
+/**
+ * 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.hadoop.tools.dynamometer;
+
+import java.util.regex.Pattern;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.PathFilter;
+
+import static org.apache.hadoop.yarn.api.records.LocalResourceType.*;
+
+/**
+ * Constants used in both Client and Application Master.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class DynoConstants {
+
+  private DynoConstants() {}
+
+  // Directory to use for remote storage (a location on the remote FS which
+  // can be accessed by all components). This will be the name of the directory
+  // within the submitter's home directory.
+  public static final String DYNAMOMETER_STORAGE_DIR = ".dynamometer";
+
+  /* The following used for Client -> AM communication */
+
+  // Resource for the zip file of all of the configuration for the
+  // DataNodes/NameNode
+  public static final DynoResource CONF_ZIP =
+      new DynoResource("CONF_ZIP", ARCHIVE, "conf");
+  // Resource for the Hadoop binary archive (distribution tar)
+  public static final DynoResource HADOOP_BINARY =
+      new DynoResource("HADOOP_BINARY", ARCHIVE, "hadoopBinary");
+  // Resource for the script used to start the DataNodes/NameNode
+  public static final DynoResource START_SCRIPT =
+      new DynoResource("START_SCRIPT", FILE, "start-component.sh");
+  // Resource for the file system image file used by the NameNode
+  public static final DynoResource FS_IMAGE =
+      new DynoResource("FS_IMAGE", FILE, null);
+  // Resource for the md5 file accompanying the file system image for the
+  // NameNode
+  public static final DynoResource FS_IMAGE_MD5 =
+      new DynoResource("FS_IMAGE_MD5", FILE, null);
+  // Resource for the VERSION file accompanying the file system image
+  public static final DynoResource VERSION =
+      new DynoResource("VERSION", FILE, "VERSION");
+  // Resource for the archive containing all dependencies
+  public static final DynoResource DYNO_DEPENDENCIES =
+      new DynoResource("DYNO_DEPS", ARCHIVE, "dependencies");
+
+  // Environment variable which will contain the location of the directory
+  // which holds all of the block files for the DataNodes
+  public static final String BLOCK_LIST_PATH_ENV = "BLOCK_ZIP_PATH";
+  // The format of the name of a single block file
+  public static final Pattern BLOCK_LIST_FILE_PATTERN =
+      Pattern.compile("dn[0-9]+-a-[0-9]+-r-[0-9]+");
+  // The file name to use when localizing the block file on a DataNode; will be
+  // suffixed with an integer
+  public static final String BLOCK_LIST_RESOURCE_PATH_PREFIX = "blocks/block";
+  public static final PathFilter BLOCK_LIST_FILE_FILTER = (path) ->
+      DynoConstants.BLOCK_LIST_FILE_PATTERN.matcher(path.getName()).find();
+
+  // Environment variable which will contain the full path of the directory
+  // which should be used for remote (shared) storage
+  public static final String REMOTE_STORAGE_PATH_ENV = "REMOTE_STORAGE_PATH";
+  // Environment variable which will contain the RPC address of the NameNode
+  // which the DataNodes should contact, if the NameNode is not launched
+  // internally by this application
+  public static final String REMOTE_NN_RPC_ADDR_ENV = "REMOTE_NN_RPC_ADDR";
+
+  // Environment variable which will contain the view ACLs for the launched
+  // containers.
+  public static final String JOB_ACL_VIEW_ENV = "JOB_ACL_VIEW";
+
+  /* The following used for AM -> DN, NN communication */
+
+  // The name of the file which will store information about the NameNode
+  // (within the remote storage directory)
+  public static final String NN_INFO_FILE_NAME = "nn_info.prop";
+
+  // Environment variable which will contain additional arguments for the
+  // NameNode
+  public static final String NN_ADDITIONAL_ARGS_ENV = "NN_ADDITIONAL_ARGS";
+  // Environment variable which will contain additional arguments for the
+  // DataNode
+  public static final String DN_ADDITIONAL_ARGS_ENV = "DN_ADDITIONAL_ARGS";
+  // Environment variable which will contain the directory to use for the
+  // NameNode's name directory;
+  // if not specified a directory within the YARN container working directory
+  // will be used.
+  public static final String NN_NAME_DIR_ENV = "NN_NAME_DIR";
+  // Environment variable which will contain the directory to use for the
+  // NameNode's edits directory;
+  // if not specified a directory within the YARN container working directory
+  // will be used.
+  public static final String NN_EDITS_DIR_ENV = "NN_EDITS_DIR";
+
+  public static final String NN_FILE_METRIC_PERIOD_ENV =
+      "NN_FILE_METRIC_PERIOD";
+
+  /*
+   * These are used as the names of properties and as the environment variables
+   */
+
+  // The port to use on the NameNode host when contacting for client RPCs
+  public static final String NN_RPC_PORT = "NN_RPC_PORT";
+  // The hostname of the machine running the NameNode
+  public static final String NN_HOSTNAME = "NN_HOSTNAME";
+  // The port to use on the NameNode host when contacting for service RPCs
+  public static final String NN_SERVICERPC_PORT = "NN_SERVICERPC_PORT";
+  // The port to use on the NameNode host when contacting for HTTP access
+  public static final String NN_HTTP_PORT = "NN_HTTP_PORT";
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/DynoInfraUtils.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/DynoInfraUtils.java
new file mode 100644
index 0000000..e5cc705
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/DynoInfraUtils.java
@@ -0,0 +1,584 @@
+/**
+ * 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.hadoop.tools.dynamometer;
+
+import com.google.common.base.Joiner;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.HttpURLConnection;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Supplier;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.client.BlockReportOptions;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.JsonToken;
+import org.slf4j.Logger;
+
+
+/**
+ * A collection of utilities used by the Dynamometer infrastructure application.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public final class DynoInfraUtils {
+
+  private DynoInfraUtils() {}
+
+  public static final String DYNO_CONF_PREFIX = "dyno.";
+  public static final String DYNO_INFRA_PREFIX = DYNO_CONF_PREFIX + "infra.";
+
+  public static final String APACHE_DOWNLOAD_MIRROR_KEY = DYNO_CONF_PREFIX
+      + "apache-mirror";
+  // Set a generic mirror as the default.
+  public static final String APACHE_DOWNLOAD_MIRROR_DEFAULT =
+      "http://mirrors.ocf.berkeley.edu/apache/";
+  private static final String APACHE_DOWNLOAD_MIRROR_SUFFIX_FORMAT =
+      "hadoop/common/hadoop-%s/hadoop-%s.tar.gz";
+  public static final String HADOOP_TAR_FILENAME_FORMAT = "hadoop-%s.tar.gz";
+
+  public static final String DATANODE_LIVE_MIN_FRACTION_KEY =
+      DYNO_INFRA_PREFIX + "ready.datanode-min-fraction";
+  public static final float DATANODE_LIVE_MIN_FRACTION_DEFAULT = 0.99f;
+  public static final String MISSING_BLOCKS_MAX_FRACTION_KEY =
+      DYNO_INFRA_PREFIX + "ready.missing-blocks-max-fraction";
+  public static final float MISSING_BLOCKS_MAX_FRACTION_DEFAULT = 0.0001f;
+  public static final String UNDERREPLICATED_BLOCKS_MAX_FRACTION_KEY =
+      DYNO_INFRA_PREFIX + "ready.underreplicated-blocks-max-fraction";
+  public static final float UNDERREPLICATED_BLOCKS_MAX_FRACTION_DEFAULT = 0.01f;
+
+  // The JMX bean queries to execute for various beans.
+  public static final String NAMENODE_STARTUP_PROGRESS_JMX_QUERY =
+      "Hadoop:service=NameNode,name=StartupProgress";
+  public static final String FSNAMESYSTEM_JMX_QUERY =
+      "Hadoop:service=NameNode,name=FSNamesystem";
+  public static final String FSNAMESYSTEM_STATE_JMX_QUERY =
+      "Hadoop:service=NameNode,name=FSNamesystemState";
+  public static final String NAMENODE_INFO_JMX_QUERY =
+      "Hadoop:service=NameNode,name=NameNodeInfo";
+  // The JMX property names of various properties.
+  public static final String JMX_MISSING_BLOCKS = "MissingBlocks";
+  public static final String JMX_UNDER_REPLICATED_BLOCKS =
+      "UnderReplicatedBlocks";
+  public static final String JMX_BLOCKS_TOTAL = "BlocksTotal";
+  public static final String JMX_LIVE_NODE_COUNT = "NumLiveDataNodes";
+  public static final String JMX_LIVE_NODES_LIST = "LiveNodes";
+
+  /**
+   * If a file matching {@value HADOOP_TAR_FILENAME_FORMAT} and {@code version}
+   * is found in {@code destinationDir}, return its path. Otherwise, first
+   * download the tarball from an Apache mirror. If the
+   * {@value APACHE_DOWNLOAD_MIRROR_KEY} configuration or system property
+   * (checked in that order) is set, use that as the mirror; else use
+   * {@value APACHE_DOWNLOAD_MIRROR_DEFAULT}.
+   *
+   * @param version The version of Hadoop to download, like "2.7.4"
+   *                or "3.0.0-beta1"
+   * @return The path to the tarball.
+   */
+  public static File fetchHadoopTarball(File destinationDir, String version,
+      Configuration conf, Logger log) throws IOException {
+    log.info("Looking for Hadoop tarball for version: " + version);
+    File destinationFile = new File(destinationDir,
+        String.format(HADOOP_TAR_FILENAME_FORMAT, version));
+    if (destinationFile.exists()) {
+      log.info("Found tarball at: " + destinationFile.getAbsolutePath());
+      return destinationFile;
+    }
+    String apacheMirror = conf.get(APACHE_DOWNLOAD_MIRROR_KEY);
+    if (apacheMirror == null) {
+      apacheMirror = System.getProperty(APACHE_DOWNLOAD_MIRROR_KEY,
+          APACHE_DOWNLOAD_MIRROR_DEFAULT);
+    }
+
+    destinationDir.mkdirs();
+    URL downloadURL = new URL(apacheMirror + String
+        .format(APACHE_DOWNLOAD_MIRROR_SUFFIX_FORMAT, version, version));
+    log.info("Downloading tarball from: <{}> to <{}>", downloadURL,
+        destinationFile.getAbsolutePath());
+    FileUtils.copyURLToFile(downloadURL, destinationFile, 10000, 60000);
+    log.info("Completed downloading of Hadoop tarball");
+    return destinationFile;
+  }
+
+  /**
+   * Get the URI that can be used to access the launched NameNode for HDFS RPCs.
+   *
+   * @param nameNodeProperties The set of properties representing the
+   *                           information about the launched NameNode.
+   * @return The HDFS URI.
+   */
+  static URI getNameNodeHdfsUri(Properties nameNodeProperties) {
+    return URI.create(String.format("hdfs://%s:%s/",
+        nameNodeProperties.getProperty(DynoConstants.NN_HOSTNAME),
+        nameNodeProperties.getProperty(DynoConstants.NN_RPC_PORT)));
+  }
+
+  /**
+   * Get the URI that can be used to access the launched NameNode for HDFS
+   * Service RPCs (i.e. from DataNodes).
+   *
+   * @param nameNodeProperties The set of properties representing the
+   *                           information about the launched NameNode.
+   * @return The service RPC URI.
+   */
+  static URI getNameNodeServiceRpcAddr(Properties nameNodeProperties) {
+    return URI.create(String.format("hdfs://%s:%s/",
+        nameNodeProperties.getProperty(DynoConstants.NN_HOSTNAME),
+        nameNodeProperties.getProperty(DynoConstants.NN_SERVICERPC_PORT)));
+  }
+
+  /**
+   * Get the URI that can be used to access the launched NameNode's web UI, e.g.
+   * for JMX calls.
+   *
+   * @param nameNodeProperties The set of properties representing the
+   *                           information about the launched NameNode.
+   * @return The URI to the web UI.
+   */
+  static URI getNameNodeWebUri(Properties nameNodeProperties) {
+    return URI.create(String.format("http://%s:%s/",
+        nameNodeProperties.getProperty(DynoConstants.NN_HOSTNAME),
+        nameNodeProperties.getProperty(DynoConstants.NN_HTTP_PORT)));
+  }
+
+  /**
+   * Get the URI that can be used to access the tracking interface for the
+   * NameNode, i.e. the web UI of the NodeManager hosting the NameNode
+   * container.
+   *
+   * @param nameNodeProperties The set of properties representing the
+   *                           information about the launched NameNode.
+   * @return The tracking URI.
+   */
+  static URI getNameNodeTrackingUri(Properties nameNodeProperties)
+      throws IOException {
+    return URI.create(String.format("http://%s:%s/node/containerlogs/%s/%s/",
+        nameNodeProperties.getProperty(DynoConstants.NN_HOSTNAME),
+        nameNodeProperties.getProperty(Environment.NM_HTTP_PORT.name()),
+        nameNodeProperties.getProperty(Environment.CONTAINER_ID.name()),
+        UserGroupInformation.getCurrentUser().getShortUserName()));
+  }
+
+  /**
+   * Get the set of properties representing information about the launched
+   * NameNode. This method will wait for the information to be available until
+   * it is interrupted, or {@code shouldExit} returns true. It polls for a file
+   * present at {@code nameNodeInfoPath} once a second and uses that file to
+   * load the NameNode information.
+   *
+   * @param shouldExit Should return true iff this should stop waiting.
+   * @param conf The configuration.
+   * @param nameNodeInfoPath The path at which to expect the NameNode
+   *                         information file to be present.
+   * @param log Where to log information.
+   * @return Absent if this exited prematurely (i.e. due to {@code shouldExit}),
+   *         else returns a set of properties representing information about the
+   *         launched NameNode.
+   */
+  static Optional<Properties> waitForAndGetNameNodeProperties(
+      Supplier<Boolean> shouldExit, Configuration conf, Path nameNodeInfoPath,
+      Logger log) throws IOException, InterruptedException {
+    while (!shouldExit.get()) {
+      try (FSDataInputStream nnInfoInputStream = nameNodeInfoPath
+          .getFileSystem(conf).open(nameNodeInfoPath)) {
+        Properties nameNodeProperties = new Properties();
+        nameNodeProperties.load(nnInfoInputStream);
+        return Optional.of(nameNodeProperties);
+      } catch (FileNotFoundException fnfe) {
+        log.debug("NameNode host information not yet available");
+        Thread.sleep(1000);
+      } catch (IOException ioe) {
+        log.warn("Unable to fetch NameNode host information; retrying", ioe);
+        Thread.sleep(1000);
+      }
+    }
+    return Optional.empty();
+  }
+
+  /**
+   * Wait for the launched NameNode to finish starting up. Continues until
+   * {@code shouldExit} returns true.
+   *
+   * @param nameNodeProperties The set of properties containing information
+   *                           about the NameNode.
+   * @param shouldExit Should return true iff this should stop waiting.
+   * @param log Where to log information.
+   */
+  static void waitForNameNodeStartup(Properties nameNodeProperties,
+      Supplier<Boolean> shouldExit, Logger log)
+      throws IOException, InterruptedException {
+    if (shouldExit.get()) {
+      return;
+    }
+    log.info("Waiting for NameNode to finish starting up...");
+    waitForNameNodeJMXValue("Startup progress",
+        NAMENODE_STARTUP_PROGRESS_JMX_QUERY, "PercentComplete", 1.0, 0.01,
+        false, nameNodeProperties, shouldExit, log);
+    log.info("NameNode has started!");
+  }
+
+  /**
+   * Wait for the launched NameNode to be ready, i.e. to have at least 99% of
+   * its DataNodes register, have fewer than 0.01% of its blocks missing, and
+   * less than 1% of its blocks under replicated. Continues until the criteria
+   * have been met or {@code shouldExit} returns true.
+   *
+   * @param nameNodeProperties The set of properties containing information
+   *                           about the NameNode.
+   * @param numTotalDataNodes Total expected number of DataNodes to register.
+   * @param shouldExit Should return true iff this should stop waiting.
+   * @param log Where to log information.
+   */
+  static void waitForNameNodeReadiness(final Properties nameNodeProperties,
+      int numTotalDataNodes, boolean triggerBlockReports,
+      Supplier<Boolean> shouldExit, final Configuration conf, final Logger log)
+      throws IOException, InterruptedException {
+    if (shouldExit.get()) {
+      return;
+    }
+    int minDataNodes = (int) (conf.getFloat(DATANODE_LIVE_MIN_FRACTION_KEY,
+        DATANODE_LIVE_MIN_FRACTION_DEFAULT) * numTotalDataNodes);
+    log.info(String.format(
+        "Waiting for %d DataNodes to register with the NameNode...",
+        minDataNodes));
+    waitForNameNodeJMXValue("Number of live DataNodes",
+        FSNAMESYSTEM_STATE_JMX_QUERY, JMX_LIVE_NODE_COUNT, minDataNodes,
+        numTotalDataNodes * 0.001, false, nameNodeProperties, shouldExit, log);
+    final int totalBlocks = Integer.parseInt(fetchNameNodeJMXValue(
+        nameNodeProperties, FSNAMESYSTEM_STATE_JMX_QUERY, JMX_BLOCKS_TOTAL));
+    final AtomicBoolean doneWaiting = new AtomicBoolean(false);
+    if (triggerBlockReports) {
+      // This will be significantly lower than the actual expected number of
+      // blocks because it does not
+      // take into account replication factor. However the block reports are
+      // pretty binary; either a full
+      // report has been received or it hasn't. Thus we don't mind the large
+      // underestimate here.
+      final int blockThreshold = totalBlocks / numTotalDataNodes * 2;
+      // The Configuration object here is based on the host cluster, which may
+      // have security enabled; we need to disable it to talk to the Dyno NN
+      conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+          "simple");
+      conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
+          "false");
+      final DistributedFileSystem dfs = (DistributedFileSystem) FileSystem
+          .get(getNameNodeHdfsUri(nameNodeProperties), conf);
+      log.info("Launching thread to trigger block reports for Datanodes with <"
+          + blockThreshold + " blocks reported");
+      Thread blockReportThread = new Thread(() -> {
+        // Here we count both Missing and UnderReplicated within under
+        // replicated
+        long lastUnderRepBlocks = Long.MAX_VALUE;
+        try {
+          while (true) { // this will eventually exit via an interrupt
+            try {
+              Thread.sleep(TimeUnit.MINUTES.toMillis(1));
+              long underRepBlocks = Long
+                  .parseLong(fetchNameNodeJMXValue(nameNodeProperties,
+                      FSNAMESYSTEM_JMX_QUERY, JMX_MISSING_BLOCKS))
+                  + Long.parseLong(fetchNameNodeJMXValue(nameNodeProperties,
+                      FSNAMESYSTEM_STATE_JMX_QUERY,
+                      JMX_UNDER_REPLICATED_BLOCKS));
+              long blockDecrease = lastUnderRepBlocks - underRepBlocks;
+              lastUnderRepBlocks = underRepBlocks;
+              if (blockDecrease < 0
+                  || blockDecrease > (totalBlocks * 0.001)) {
+                continue;
+              }
+
+              String liveNodeListString = fetchNameNodeJMXValue(
+                  nameNodeProperties, NAMENODE_INFO_JMX_QUERY,
+                  JMX_LIVE_NODES_LIST);
+              Set<String> datanodesToReport = parseStaleDataNodeList(
+                  liveNodeListString, blockThreshold, log);
+              if (datanodesToReport.isEmpty() && doneWaiting.get()) {
+                log.info("BlockReportThread exiting; all DataNodes have "
+                    + "reported blocks");
+                break;
+              }
+              log.info("Queueing {} Datanodes for block report: {}",
+                      datanodesToReport.size(),
+                      Joiner.on(",").join(datanodesToReport));
+              DatanodeInfo[] datanodes = dfs.getDataNodeStats();
+              int cnt = 0;
+              for (DatanodeInfo datanode : datanodes) {
+                if (datanodesToReport.contains(datanode.getXferAddr(true))) {
+                  Thread.sleep(1); // to throw an interrupt if one is found
+                  triggerDataNodeBlockReport(conf, datanode.getIpcAddr(true));
+                  cnt++;
+                  Thread.sleep(1000);
+                }
+              }
+              if (cnt != datanodesToReport.size()) {
+                log.warn("Found {} Datanodes to queue block reports for but "
+                        + "was only able to trigger {}",
+                    datanodesToReport.size(), cnt);
+              }
+            } catch (IOException ioe) {
+              log.warn("Exception encountered in block report thread", ioe);
+            }
+          }
+        } catch (InterruptedException ie) {
+          // Do nothing; just exit
+        }
+        log.info("Block reporting thread exiting");
+      });
+      blockReportThread.setDaemon(true);
+      blockReportThread
+          .setUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
+      blockReportThread.start();
+    }
+    float maxMissingBlocks = totalBlocks * conf.getFloat(
+        MISSING_BLOCKS_MAX_FRACTION_KEY, MISSING_BLOCKS_MAX_FRACTION_DEFAULT);
+    log.info("Waiting for MissingBlocks to fall below {}...",
+        maxMissingBlocks);
+    waitForNameNodeJMXValue("Number of missing blocks", FSNAMESYSTEM_JMX_QUERY,
+        JMX_MISSING_BLOCKS, maxMissingBlocks, totalBlocks * 0.0001, true,
+        nameNodeProperties, shouldExit, log);
+    float maxUnderreplicatedBlocks = totalBlocks
+        * conf.getFloat(UNDERREPLICATED_BLOCKS_MAX_FRACTION_KEY,
+            UNDERREPLICATED_BLOCKS_MAX_FRACTION_DEFAULT);
+    log.info("Waiting for UnderReplicatedBlocks to fall below {}...",
+        maxUnderreplicatedBlocks);
+    waitForNameNodeJMXValue("Number of under replicated blocks",
+        FSNAMESYSTEM_STATE_JMX_QUERY, JMX_UNDER_REPLICATED_BLOCKS,
+        maxUnderreplicatedBlocks, totalBlocks * 0.001, true, nameNodeProperties,
+        shouldExit, log);
+    log.info("NameNode is ready for use!");
+    doneWaiting.set(true);
+  }
+
+  /**
+   * Trigger a block report on a given DataNode.
+   *
+   * @param conf Configuration
+   * @param dataNodeTarget The target; should be like {@code <host>:<port>}
+   */
+  private static void triggerDataNodeBlockReport(Configuration conf,
+      String dataNodeTarget) throws IOException {
+    InetSocketAddress datanodeAddr = NetUtils.createSocketAddr(dataNodeTarget);
+
+    ClientDatanodeProtocol dnProtocol = DFSUtilClient
+        .createClientDatanodeProtocolProxy(datanodeAddr,
+            UserGroupInformation.getCurrentUser(), conf,
+            NetUtils.getSocketFactory(conf, ClientDatanodeProtocol.class));
+
+    dnProtocol.triggerBlockReport(new BlockReportOptions.Factory().build());
+  }
+
+  /**
+   * Poll the launched NameNode's JMX for a specific value, waiting for it to
+   * cross some threshold. Continues until the threshold has been crossed or
+   * {@code shouldExit} returns true. Periodically logs the current value.
+   *
+   * @param valueName The human-readable name of the value which is being
+   *                  polled (for printing purposes only).
+   * @param jmxBeanQuery The JMX bean query to execute; should return a JMX
+   *                     property matching {@code jmxProperty}.
+   * @param jmxProperty The name of the JMX property whose value should be
+   *                    polled.
+   * @param threshold The threshold value to wait for the JMX property to be
+   *                  above/below.
+   * @param printThreshold The threshold between each log statement; controls
+   *                       how frequently the value is printed. For example,
+   *                       if this was 10, a statement would be logged every
+   *                       time the value has changed by more than 10.
+   * @param decreasing True iff the property's value is decreasing and this
+   *                   should wait until it is lower than threshold; else the
+   *                   value is treated as increasing and will wait until it
+   *                   is higher than threshold.
+   * @param nameNodeProperties The set of properties containing information
+   *                           about the NameNode.
+   * @param shouldExit Should return true iff this should stop waiting.
+   * @param log Where to log information.
+   */
+  private static void waitForNameNodeJMXValue(String valueName,
+      String jmxBeanQuery, String jmxProperty, double threshold,
+      double printThreshold, boolean decreasing, Properties nameNodeProperties,
+      Supplier<Boolean> shouldExit, Logger log) throws InterruptedException {
+    double lastPrintedValue = decreasing ? Double.MAX_VALUE : Double.MIN_VALUE;
+    double value;
+    int retryCount = 0;
+    long startTime = Time.monotonicNow();
+    while (!shouldExit.get()) {
+      try {
+        value = Double.parseDouble(fetchNameNodeJMXValue(nameNodeProperties,
+            jmxBeanQuery, jmxProperty));
+        if ((decreasing && value <= threshold)
+            || (!decreasing && value >= threshold)) {
+          log.info(String.format(
+              "%s = %.2f; %s threshold of %.2f; done waiting after %d ms.",
+              valueName, value, decreasing ? "below" : "above", threshold,
+              Time.monotonicNow() - startTime));
+          break;
+        } else if (Math.abs(value - lastPrintedValue) >= printThreshold) {
+          log.info(String.format("%s: %.2f", valueName, value));
+          lastPrintedValue = value;
+        }
+      } catch (IOException ioe) {
+        if (++retryCount % 20 == 0) {
+          log.warn("Unable to fetch {}; retried {} times / waited {} ms",
+              valueName, retryCount, Time.monotonicNow() - startTime, ioe);
+        }
+      }
+      Thread.sleep(3000);
+    }
+  }
+
+  static Set<String> parseStaleDataNodeList(String liveNodeJsonString,
+      final int blockThreshold, final Logger log) throws IOException {
+    final Set<String> dataNodesToReport = new HashSet<>();
+
+    JsonFactory fac = new JsonFactory();
+    JsonParser parser = fac.createJsonParser(IOUtils
+        .toInputStream(liveNodeJsonString, StandardCharsets.UTF_8.name()));
+
+    int objectDepth = 0;
+    String currentNodeAddr = null;
+    for (JsonToken tok = parser.nextToken(); tok != null; tok = parser
+        .nextToken()) {
+      if (tok == JsonToken.START_OBJECT) {
+        objectDepth++;
+      } else if (tok == JsonToken.END_OBJECT) {
+        objectDepth--;
+      } else if (tok == JsonToken.FIELD_NAME) {
+        if (objectDepth == 1) {
+          // This is where the Datanode identifiers are stored
+          currentNodeAddr = parser.getCurrentName();
+        } else if (objectDepth == 2) {
+          if (parser.getCurrentName().equals("numBlocks")) {
+            JsonToken valueToken = parser.nextToken();
+            if (valueToken != JsonToken.VALUE_NUMBER_INT
+                || currentNodeAddr == null) {
+              throw new IOException(String.format("Malformed LiveNodes JSON; "
+                      + "got token = %s; currentNodeAddr = %s: %s",
+                  valueToken, currentNodeAddr, liveNodeJsonString));
+            }
+            int numBlocks = parser.getIntValue();
+            if (numBlocks < blockThreshold) {
+              log.debug(String.format(
+                  "Queueing Datanode <%s> for block report; numBlocks = %d",
+                  currentNodeAddr, numBlocks));
+              dataNodesToReport.add(currentNodeAddr);
+            } else {
+              log.debug(String.format(
+                  "Not queueing Datanode <%s> for block report; numBlocks = %d",
+                  currentNodeAddr, numBlocks));
+            }
+          }
+        }
+      }
+    }
+    return dataNodesToReport;
+  }
+
+  /**
+   * Fetch a value from the launched NameNode's JMX.
+   *
+   * @param nameNodeProperties The set of properties containing information
+   *                           about the NameNode.
+   * @param jmxBeanQuery The JMX bean query to execute; should return a
+   *                     JMX property matching {@code jmxProperty}.
+   * @param property The name of the JMX property whose value should be polled.
+   * @return The value associated with the property.
+   */
+  static String fetchNameNodeJMXValue(Properties nameNodeProperties,
+      String jmxBeanQuery, String property) throws IOException {
+    URI nnWebUri = getNameNodeWebUri(nameNodeProperties);
+    URL queryURL;
+    try {
+      queryURL = new URL(nnWebUri.getScheme(), nnWebUri.getHost(),
+          nnWebUri.getPort(), "/jmx?qry=" + jmxBeanQuery);
+    } catch (MalformedURLException e) {
+      throw new IllegalArgumentException("Invalid JMX query: \"" + jmxBeanQuery
+          + "\" against " + "NameNode URI: " + nnWebUri);
+    }
+    HttpURLConnection conn = (HttpURLConnection) queryURL.openConnection();
+    if (conn.getResponseCode() != 200) {
+      throw new IOException(
+          "Unable to retrieve JMX: " + conn.getResponseMessage());
+    }
+    InputStream in = conn.getInputStream();
+    JsonFactory fac = new JsonFactory();
+    JsonParser parser = fac.createJsonParser(in);
+    if (parser.nextToken() != JsonToken.START_OBJECT
+        || parser.nextToken() != JsonToken.FIELD_NAME
+        || !parser.getCurrentName().equals("beans")
+        || parser.nextToken() != JsonToken.START_ARRAY
+        || parser.nextToken() != JsonToken.START_OBJECT) {
+      throw new IOException(
+          "Unexpected format of JMX JSON response for: " + jmxBeanQuery);
+    }
+    int objectDepth = 1;
+    String ret = null;
+    while (objectDepth > 0) {
+      JsonToken tok = parser.nextToken();
+      if (tok == JsonToken.START_OBJECT) {
+        objectDepth++;
+      } else if (tok == JsonToken.END_OBJECT) {
+        objectDepth--;
+      } else if (tok == JsonToken.FIELD_NAME) {
+        if (parser.getCurrentName().equals(property)) {
+          parser.nextToken();
+          ret = parser.getText();
+          break;
+        }
+      }
+    }
+    parser.close();
+    in.close();
+    conn.disconnect();
+    if (ret == null) {
+      throw new IOException(
+          "Property " + property + " not found within " + jmxBeanQuery);
+    } else {
+      return ret;
+    }
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/DynoResource.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/DynoResource.java
new file mode 100644
index 0000000..1a9253f
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/DynoResource.java
@@ -0,0 +1,72 @@
+/**
+ * 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.hadoop.tools.dynamometer;
+
+import java.util.Map;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+
+class DynoResource {
+
+  private final String name;
+  private final LocalResourceType type;
+  private final String resourcePath;
+
+  DynoResource(String name, LocalResourceType type, String resourcePath) {
+    this.name = name;
+    this.type = type;
+    this.resourcePath = resourcePath;
+  }
+
+  public Path getPath(Map<String, String> env) {
+    return new Path(env.get(getLocationEnvVar()));
+  }
+
+  public long getTimestamp(Map<String, String> env) {
+    return Long.parseLong(env.get(getTimestampEnvVar()));
+  }
+
+  public long getLength(Map<String, String> env) {
+    return Long.parseLong(env.get(getLengthEnvVar()));
+  }
+
+  public String getLocationEnvVar() {
+    return name + "_LOCATION";
+  }
+
+  public String getTimestampEnvVar() {
+    return name + "_TIMESTAMP";
+  }
+
+  public String getLengthEnvVar() {
+    return name + "_LENGTH";
+  }
+
+  public LocalResourceType getType() {
+    return type;
+  }
+
+  public String getResourcePath() {
+    return resourcePath;
+  }
+
+  public String toString() {
+    return name;
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/SimulatedDataNodes.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/SimulatedDataNodes.java
new file mode 100644
index 0000000..5c409b2
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/SimulatedDataNodes.java
@@ -0,0 +1,182 @@
+/**
+ * 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.hadoop.tools.dynamometer;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * Starts up a number of DataNodes within the same JVM. These DataNodes all use
+ * {@link org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset}, so they do
+ * not store any actual data, and do not persist anything to disk; they maintain
+ * all metadata in memory. This is useful for testing and simulation purposes.
+ * <p>
+ * The DataNodes will attempt to connect to a NameNode defined by the default
+ * FileSystem. There will be one DataNode started for each block list file
+ * passed as an argument. Each of these files should contain a list of blocks
+ * that the corresponding DataNode should contain, as specified by a triplet of
+ * block ID, block size, and generation stamp. Each line of the file is one
+ * block, in the format:
+ * <p>
+ * {@code blockID,blockGenStamp,blockSize}
+ * <p>
+ * This class is loosely based off of
+ * {@link org.apache.hadoop.hdfs.DataNodeCluster}.
+ */
+public class SimulatedDataNodes extends Configured implements Tool {
+
+  // Set this arbitrarily large (100TB) since we don't care about storage
+  // capacity
+  private static final long STORAGE_CAPACITY = 100 * 2L << 40;
+  private static final String USAGE = "Usage: "
+      + "org.apache.hadoop.tools.dynamometer.SimulatedDataNodes "
+      + "bpid blockListFile1 [ blockListFileN ... ]\n"
+      + "   bpid should be the ID of the block pool to which these DataNodes "
+      + "belong.\n"
+      + "   Each blockListFile specified should contain a list of blocks to "
+      + "be served by one DataNode.\n"
+      + "   See the Javadoc of this class for more detail.";
+
+  static void printUsageExit(String err) {
+    System.out.println(err);
+    System.out.println(USAGE);
+    System.exit(1);
+  }
+
+  public static void main(String[] args) throws Exception {
+    SimulatedDataNodes datanodes = new SimulatedDataNodes();
+    ToolRunner.run(new HdfsConfiguration(), datanodes, args);
+  }
+
+  public int run(String[] args) throws Exception {
+    if (args.length < 2) {
+      printUsageExit("Not enough arguments");
+    }
+    String bpid = args[0];
+    List<Path> blockListFiles = new ArrayList<>();
+    for (int i = 1; i < args.length; i++) {
+      blockListFiles.add(new Path(args[i]));
+    }
+
+    URI defaultFS = FileSystem.getDefaultUri(getConf());
+    if (!HdfsConstants.HDFS_URI_SCHEME.equals(defaultFS.getScheme())) {
+      printUsageExit(
+          "Must specify an HDFS-based default FS! Got <" + defaultFS + ">");
+    }
+    String nameNodeAdr = defaultFS.getAuthority();
+    if (nameNodeAdr == null) {
+      printUsageExit("No NameNode address and port in config");
+    }
+    System.out.println("DataNodes will connect to NameNode at " + nameNodeAdr);
+
+    System.setProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA,
+        DataNode.getStorageLocations(getConf()).get(0).getUri().getPath());
+    SimulatedFSDataset.setFactory(getConf());
+    getConf().setLong(SimulatedFSDataset.CONFIG_PROPERTY_CAPACITY,
+        STORAGE_CAPACITY);
+
+    UserGroupInformation.setConfiguration(getConf());
+    MiniDFSCluster mc = new MiniDFSCluster();
+    try {
+      mc.formatDataNodeDirs();
+    } catch (IOException e) {
+      System.out.println("Error formatting DataNode dirs: " + e);
+      System.exit(1);
+    }
+
+    try {
+      System.out.println("Found " + blockListFiles.size()
+          + " block listing files; launching DataNodes accordingly.");
+      mc.startDataNodes(getConf(), blockListFiles.size(), null, false,
+          StartupOption.REGULAR, null, null, null, null, false, true, true,
+          null);
+      long startTime = Time.monotonicNow();
+      System.out.println("Waiting for DataNodes to connect to NameNode and "
+          + "init storage directories.");
+      Set<DataNode> datanodesWithoutFSDataset = new HashSet<>(
+          mc.getDataNodes());
+      while (!datanodesWithoutFSDataset.isEmpty()) {
+        datanodesWithoutFSDataset
+            .removeIf((dn) -> DataNodeTestUtils.getFSDataset(dn) != null);
+        Thread.sleep(100);
+      }
+      System.out.println("Waited " + (Time.monotonicNow() - startTime)
+          + " ms for DataNode FSDatasets to be ready");
+
+      for (int dnIndex = 0; dnIndex < blockListFiles.size(); dnIndex++) {
+        Path blockListFile = blockListFiles.get(dnIndex);
+        try (FSDataInputStream fsdis = blockListFile.getFileSystem(getConf())
+            .open(blockListFile)) {
+          BufferedReader reader = new BufferedReader(
+              new InputStreamReader(fsdis));
+          List<Block> blockList = new ArrayList<>();
+          int cnt = 0;
+          for (String line = reader.readLine(); line != null; line = reader
+              .readLine()) {
+            // Format of the listing files is blockID,blockGenStamp,blockSize
+            String[] blockInfo = line.split(",");
+            blockList.add(new Block(Long.parseLong(blockInfo[0]),
+                Long.parseLong(blockInfo[2]), Long.parseLong(blockInfo[1])));
+            cnt++;
+          }
+          try {
+            mc.injectBlocks(dnIndex, blockList, bpid);
+          } catch (IOException ioe) {
+            System.out.printf("Error injecting blocks into DataNode %d for "
+                    + "block pool %s: %s%n", dnIndex, bpid,
+                ExceptionUtils.getStackTrace(ioe));
+          }
+          System.out.printf(
+              "Injected %d blocks into DataNode %d for block pool %s%n",
+              cnt, dnIndex, bpid);
+        }
+      }
+
+    } catch (IOException e) {
+      System.out.println(
+          "Error creating DataNodes: " + ExceptionUtils.getStackTrace(e));
+      return 1;
+    }
+    return 0;
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/package-info.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/package-info.java
new file mode 100644
index 0000000..b6a7630
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/package-info.java
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Dynamometer tool is used to stress test the HDFS NameNode by setting
+ * up a realistic environment consistent of fake DataNodes and a real NameNode
+ * with a real FS namesystem and near-real user workload (RPCs). Please see
+ * the documentation for more detail.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.tools.dynamometer;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/resources/start-component.sh b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/resources/start-component.sh
new file mode 100644
index 0000000..08b3c85
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/resources/start-component.sh
@@ -0,0 +1,344 @@
+#!/usr/bin/env 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.
+
+# Script for launching Dynamometer components within YARN containers.
+# USAGE:
+# ./start-component.sh namenode hdfs_storage
+# OR
+# ./start-component.sh datanode nn_servicerpc_address sleep_time_sec
+# First parameter should be component being launched, either `datanode` or `namenode`
+# If component is namenode, hdfs_storage is expected to point to a location to
+#   write out shared files such as the file containing the information about
+#   which ports the NN started on (at nn_info.prop) and the namenode's metrics
+#   (at namenode_metrics)
+# If component is datanode, nn_servicerpc_address is expected to point to the
+#   servicerpc address of the namenode. sleep_time_sec is the amount of time that
+#   should be allowed to elapse before launching anything. The
+#   `org.apache.hadoop.tools.dynamometer.SimulatedDataNodes` class will be used to start multiple
+#   DataNodes within the same JVM, and they will store their block files in memory.
+
+component="$1"
+if [[ "$component" != "datanode" && "$component" != "namenode" ]]; then
+  echo "Unknown component type: '${component}'"
+  exit 1
+fi
+if [[ "$component" = "namenode" ]]; then
+  if [[ $# -lt 2 ]]; then
+    echo "Not enough arguments for NameNode"
+    exit 1
+  fi
+  hdfsStoragePath="$2"
+else
+  if [[ $# -lt 3 ]]; then
+    echo "Not enough arguments for DataNode"
+    exit 1
+  fi
+  nnServiceRpcAddress="$2"
+  launchDelaySec="$3"
+fi
+containerID=${CONTAINER_ID##*_}
+
+echo "Starting ${component} with ID ${containerID}"
+echo "PWD is: $(pwd)"
+
+confDir="$(pwd)/conf/etc/hadoop"
+umask 022
+baseDir="$(pwd)/dyno-node"
+
+# Set Hadoop's log dir to that of the NodeManager,
+# then YARN will automatically help us handle the logs
+# May be a comma-separated list; just take the first one
+logDir=${LOG_DIRS%%,*}
+
+pidDir="$baseDir/pid"
+baseHttpPort=50075
+baseRpcPort=9000
+baseServiceRpcPort=9020
+
+rm -rf "$baseDir"
+mkdir -p "$pidDir"
+chmod 755 "$baseDir"
+chmod 700 "$pidDir"
+
+# Set Hadoop variables for component
+hadoopHome="$(find -H "$(pwd)/hadoopBinary" -depth 1 -type d | head -n 1)"
+# Save real environment for later
+hadoopConfOriginal=${HADOOP_CONF_DIR:-$confDir}
+hadoopHomeOriginal=${HADOOP_HOME:-$hadoopHome}
+echo "Saving original HADOOP_HOME as: $hadoopHomeOriginal"
+echo "Saving original HADOOP_CONF_DIR as: $hadoopConfOriginal"
+## @description  A function to perform an HDFS command under the system Hadoop
+##               instead of the Hadoop-under-test.
+## @audience     private
+## @stability    evolving
+function hdfs_original {
+  HADOOP_HOME="${hadoopHomeOriginal}" HADOOP_CONF_DIR="${hadoopConfOriginal}" \
+  HADOOP_HDFS_HOME="${hadoopHomeOriginal}" HADOOP_COMMON_HOME="${hadoopHomeOriginal}" \
+  "${hadoopHomeOriginal}/bin/hdfs" "$@"
+}
+
+extraClasspathDir="$(pwd)/additionalClasspath/"
+mkdir -p "${extraClasspathDir}"
+
+# Change environment variables for the Hadoop process
+export HADOOP_HOME="$hadoopHome"
+export HADOOP_PREFIX="$hadoopHome"
+export PATH="$HADOOP_HOME/bin:$PATH"
+export HADOOP_HDFS_HOME="$hadoopHome"
+export HADOOP_COMMON_HOME="$hadoopHome"
+export HADOOP_YARN_HOME="$hadoopHome"
+export LIBHDFS_OPTS="-Djava.library.path=$hadoopHome/lib/native"
+export HADOOP_MAPRED_HOME="$hadoopHome"
+export HADOOP_CONF_DIR="${confDir}"
+export YARN_CONF_DIR="${confDir}"
+export HADOOP_LOG_DIR="${logDir}"
+export HADOOP_PID_DIR="${pidDir}"
+HADOOP_CLASSPATH="$(pwd)/dependencies/*:$extraClasspathDir"
+export HADOOP_CLASSPATH
+echo "Environment variables are set as:"
+echo "(note that this doesn't include changes made by hadoop-env.sh)"
+printenv
+echo -e "\n\n"
+
+# Starting from base_port, add the last two digits of the containerID,
+# then keep searching upwards for a free port
+# find_available_port base_port
+find_available_port() {
+  basePort="$1"
+  currPort=$((basePort+((10#$containerID)%100)))
+  while netstat -nl | grep -q ":${currPort}[[:space:]]"; do
+    currPort=$((currPort+1))
+  done
+  echo "$currPort"
+}
+
+configOverrides=(
+  -D "hadoop.tmp.dir=${baseDir}"
+  -D "hadoop.security.authentication=simple"
+  -D "hadoop.security.authorization=false"
+  -D "dfs.http.policy=HTTP_ONLY"
+  -D "dfs.nameservices="
+  -D "dfs.web.authentication.kerberos.principal="
+  -D "dfs.web.authentication.kerberos.keytab="
+  -D "hadoop.http.filter.initializers="
+  -D "dfs.datanode.kerberos.principal="
+  -D "dfs.datanode.keytab.file="
+  -D "dfs.domain.socket.path="
+  -D "dfs.client.read.shortcircuit=false"
+)
+# NOTE: Must manually unset dfs.namenode.shared.edits.dir in configs
+#       because setting it to be empty is not enough (must be null)
+
+if [[ "$component" = "datanode" ]]; then
+
+  if ! dataDirsOrig="$(hdfs getconf "${configOverrides[@]}" -confKey dfs.datanode.data.dir)"; then
+    echo "Unable to fetch data directories from config; using default"
+    dataDirsOrig="/data-dir/1,/data-dir/2"
+  fi
+  dataDirsOrig=(${dataDirsOrig//,/ })
+  dataDirs=""
+  for dataDir in "${dataDirsOrig[@]}"; do
+    stripped="file://$baseDir/${dataDir#file://}"
+    dataDirs="$dataDirs,$stripped"
+  done
+  dataDirs=${dataDirs:1}
+
+  echo "Going to sleep for $launchDelaySec sec..."
+  for _ in $(seq 1 "${launchDelaySec}"); do
+    sleep 1
+    if ! kill -0 $PPID 2>/dev/null; then
+      echo "Parent process ($PPID) exited while waiting; now exiting"
+      exit 0
+     fi
+  done
+
+  versionFile="$(pwd)/VERSION"
+  bpId="$(grep "${versionFile}" -e blockpoolID | awk -F= '{print $2}')"
+  listingFiles=()
+  blockDir="$(pwd)/blocks"
+  for listingFile in "${blockDir}"/*; do
+    listingFiles+=("file://${listingFile}")
+  done
+
+  datanodeClusterConfigs=(
+    -D "fs.defaultFS=${nnServiceRpcAddress}"
+    -D "dfs.datanode.hostname=$(hostname)"
+    -D "dfs.datanode.data.dir=${dataDirs}"
+    -D "dfs.datanode.ipc.address=0.0.0.0:0"
+    -D "dfs.datanode.http.address=0.0.0.0:0"
+    -D "dfs.datanode.address=0.0.0.0:0"
+    -D "dfs.datanode.directoryscan.interval=-1"
+    -D "fs.du.interval=43200000"
+    -D "fs.getspaceused.jitterMillis=21600000"
+    ${configOverrides[@]}
+    "${bpId}"
+    ${listingFiles[@]}
+  )
+
+  echo "Executing the following:"
+  echo "${HADOOP_HOME}/bin/hadoop org.apache.hadoop.tools.dynamometer.SimulatedDataNodes \
+    $DN_ADDITIONAL_ARGS" "${datanodeClusterConfigs[@]}"
+  # The argument splitting of DN_ADDITIONAL_ARGS is desirable behavior here
+  # shellcheck disable=SC2086
+  "${HADOOP_HOME}/bin/hadoop" org.apache.hadoop.tools.dynamometer.SimulatedDataNodes \
+    $DN_ADDITIONAL_ARGS "${datanodeClusterConfigs[@]}" &
+  launchSuccess="$?"
+  componentPID="$!"
+  if [[ ${launchSuccess} -ne 0 ]]; then
+    echo "Unable to launch DataNode cluster; exiting."
+    exit 1
+  fi
+
+elif [[ "$component" = "namenode" ]]; then
+
+  nnHostname="${NM_HOST}"
+  nnRpcPort="$(find_available_port "$baseRpcPort")"
+  nnServiceRpcPort="$(find_available_port "$baseServiceRpcPort")"
+  nnHttpPort="$(find_available_port "$baseHttpPort")"
+
+  nnInfoLocalPath="$(pwd)/nn_info.prop"
+  rm -f "$nnInfoLocalPath"
+  # Port and host information to be uploaded to the non-Dynamometer HDFS
+  # to be consumed by the AM and Client
+  cat > "$nnInfoLocalPath" << EOF
+NN_HOSTNAME=${nnHostname}
+NN_RPC_PORT=${nnRpcPort}
+NN_SERVICERPC_PORT=${nnServiceRpcPort}
+NN_HTTP_PORT=${nnHttpPort}
+NM_HTTP_PORT=${NM_HTTP_PORT}
+CONTAINER_ID=${CONTAINER_ID}
+EOF
+  echo "Using the following ports for the namenode:"
+  cat "$nnInfoLocalPath"
+  nnInfoRemotePath="$hdfsStoragePath/nn_info.prop"
+  # We use the original conf dir since we are uploading to the non-dynamometer cluster
+  hdfs_original dfs -copyFromLocal -f "$nnInfoLocalPath" "$nnInfoRemotePath"
+  echo "Uploaded namenode port info to $nnInfoRemotePath"
+
+  if [[ "$NN_FILE_METRIC_PERIOD" -gt 0 ]]; then
+    nnMetricOutputFileLocal="$HADOOP_LOG_DIR/namenode_metrics"
+    nnMetricPropsFileLocal="$extraClasspathDir/hadoop-metrics2-namenode.properties"
+    if [[ -f "$confDir/hadoop-metrics2-namenode.properties" ]]; then
+      cp "$confDir/hadoop-metrics2-namenode.properties" "$nnMetricPropsFileLocal"
+      chmod u+w "$nnMetricPropsFileLocal"
+    elif [[ -f "$confDir/hadoop-metrics2.properties" ]]; then
+      cp "$confDir/hadoop-metrics2.properties" "$nnMetricPropsFileLocal"
+      chmod u+w "$nnMetricPropsFileLocal"
+    fi
+    cat >> "$nnMetricPropsFileLocal" << EOF
+namenode.sink.dyno-file.period=${NN_FILE_METRIC_PERIOD}
+namenode.sink.dyno-file.class=org.apache.hadoop.metrics2.sink.FileSink
+namenode.sink.dyno-file.filename=${nnMetricOutputFileLocal}
+EOF
+  fi
+
+  nameDir="${NN_NAME_DIR:-${baseDir}/name-data}"
+  editsDir="${NN_EDITS_DIR:-${baseDir}/name-data}"
+  checkpointDir="$baseDir/checkpoint"
+  rm -rf "$nameDir" "$editsDir" "$checkpointDir"
+  mkdir -p "$nameDir/current" "$editsDir/current" "$checkpointDir"
+  chmod -R 700 "$nameDir" "$editsDir" "$checkpointDir"
+  fsImageFile="$(find "$(pwd)" -depth 1 -name "fsimage_*" | tail -n 1)"
+  fsImageMD5File="$(find "$(pwd)" -depth 1 -name "fsimage_*.md5" | tail -n 1)"
+  ln -snf "$fsImageFile" "$nameDir/current/$(basename "$fsImageFile")"
+  ln -snf "$fsImageMD5File" "$nameDir/current/$(basename "$fsImageMD5File")"
+  ln -snf "$(pwd)/VERSION" "$nameDir/current/VERSION"
+  chmod 700 "$nameDir"/current/*
+
+  namenodeConfigs=(
+    -D "fs.defaultFS=hdfs://${nnHostname}:${nnRpcPort}"
+    -D "dfs.namenode.rpc-address=${nnHostname}:${nnRpcPort}"
+    -D "dfs.namenode.servicerpc-address=${nnHostname}:${nnServiceRpcPort}"
+    -D "dfs.namenode.http-address=${nnHostname}:${nnHttpPort}"
+    -D "dfs.namenode.https-address=${nnHostname}:0"
+    -D "dfs.namenode.name.dir=file://${nameDir}"
+    -D "dfs.namenode.edits.dir=file://${editsDir}"
+    -D "dfs.namenode.checkpoint.dir=file://${baseDir}/checkpoint"
+    -D "dfs.namenode.kerberos.internal.spnego.principal="
+    -D "dfs.hosts="
+    -D "dfs.hosts.exclude="
+    -D "dfs.namenode.legacy-oiv-image.dir="
+    -D "dfs.namenode.kerberos.principal="
+    -D "dfs.namenode.keytab.file="
+    -D "dfs.namenode.safemode.threshold-pct=0.0f"
+    -D "dfs.permissions.enabled=true"
+    -D "dfs.cluster.administrators=\"*\""
+    -D "dfs.block.replicator.classname=org.apache.hadoop.tools.dynamometer.BlockPlacementPolicyAlwaysSatisfied"
+    -D "hadoop.security.impersonation.provider.class=org.apache.hadoop.tools.dynamometer.AllowAllImpersonationProvider"
+    ${configOverrides[@]}
+  )
+
+  echo "Executing the following:"
+  echo "${HADOOP_HOME}/sbin/hadoop-daemon.sh start namenode" "${namenodeConfigs[@]}" "$NN_ADDITIONAL_ARGS"
+  # The argument splitting of NN_ADDITIONAL_ARGS is desirable behavior here
+  # shellcheck disable=SC2086
+  if ! "${HADOOP_HOME}/sbin/hadoop-daemon.sh" start namenode "${namenodeConfigs[@]}" $NN_ADDITIONAL_ARGS; then
+    echo "Unable to launch NameNode; exiting."
+    exit 1
+  fi
+  componentPIDFile="${pidDir}/hadoop-$(whoami)-${component}.pid"
+  while [[ ! -f "$componentPIDFile" ]]; do sleep 1; done
+  componentPID=$(cat "$componentPIDFile")
+
+  if [[ "$NN_FILE_METRIC_PERIOD" -gt 0 ]]; then
+    nnMetricOutputFileRemote="$hdfsStoragePath/namenode_metrics"
+    echo "Going to attempt to upload metrics to: $nnMetricOutputFileRemote"
+
+    touch "$nnMetricOutputFileLocal"
+    (tail -n 999999 -f "$nnMetricOutputFileLocal" & echo $! >&3) 3>metricsTailPIDFile | \
+      hdfs_original dfs -appendToFile - "$nnMetricOutputFileRemote" &
+    metricsTailPID="$(cat metricsTailPIDFile)"
+    if [[ "$metricsTailPID" = "" ]]; then
+      echo "Unable to upload metrics to HDFS"
+    else
+      echo "Metrics will be uploaded to HDFS by PID: $metricsTailPID"
+    fi
+  fi
+fi
+
+echo "Started $component at pid $componentPID"
+
+## @description  Perform cleanup, killing any outstanding processes and deleting files
+## @audience     private
+## @stability    evolving
+function cleanup {
+  echo "Cleaning up $component at pid $componentPID"
+  kill -9 "$componentPID"
+
+  if [[ "$metricsTailPID" != "" ]]; then
+    echo "Stopping metrics streaming at pid $metricsTailPID"
+    kill "$metricsTailPID"
+  fi
+
+  echo "Deleting any remaining files"
+  rm -rf "$baseDir"
+}
+
+trap cleanup EXIT
+
+echo "Waiting for parent process (PID: $PPID) OR $component process to exit"
+while kill -0 "${componentPID}" 2>/dev/null && kill -0 $PPID 2>/dev/null; do
+  sleep 1
+done
+
+if kill -0 $PPID 2>/dev/null; then
+  echo "$component process exited; continuing to finish"
+  exit 1
+else
+  echo "Parent process exited; continuing to finish"
+  exit 0
+fi
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/java/org/apache/hadoop/tools/dynamometer/TestDynamometerInfra.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/java/org/apache/hadoop/tools/dynamometer/TestDynamometerInfra.java
new file mode 100644
index 0000000..d93e044
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/java/org/apache/hadoop/tools/dynamometer/TestDynamometerInfra.java
@@ -0,0 +1,542 @@
+/**
+ * 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.hadoop.tools.dynamometer;
+
+import com.google.common.collect.Sets;
+import java.util.Optional;
+import java.util.function.Supplier;
+import org.apache.hadoop.test.PlatformAssumptions;
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditLogDirectParser;
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditReplayMapper;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.UUID;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.util.JarFinder;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.MiniYARNCluster;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.tools.dynamometer.DynoInfraUtils.fetchHadoopTarball;
+import static org.apache.hadoop.hdfs.MiniDFSCluster.PROP_TEST_BUILD_DATA;
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Start a Dynamometer cluster in a MiniYARNCluster. Ensure that the NameNode is
+ * able to start correctly, exit safemode, and run some commands. Subsequently
+ * the workload job is launched and it is verified that it completes
+ * successfully and is able to replay commands as expected.
+ *
+ * To run this test JAVA_HOME must be set correctly, and the {@code tar} utility
+ * must be available.
+ *
+ * You can optionally specify which version of HDFS should be started within the
+ * Dynamometer cluster; the default is {@value HADOOP_BIN_VERSION_DEFAULT}. This
+ * can be adjusted by setting the {@value HADOOP_BIN_VERSION_KEY} property. This
+ * will automatically download the correct Hadoop tarball for the specified
+ * version. It downloads from an Apache mirror (by default
+ * {@value DynoInfraUtils#APACHE_DOWNLOAD_MIRROR_DEFAULT}); which mirror is used
+ * can be controlled with the {@value DynoInfraUtils#APACHE_DOWNLOAD_MIRROR_KEY}
+ * property. Note that mirrors normally contain only the latest releases on any
+ * given release line; you may need to use
+ * {@code http://archive.apache.org/dist/} for older releases. The downloaded
+ * tarball will be stored in the test directory and can be reused between test
+ * executions. Alternatively, you can specify the {@value HADOOP_BIN_PATH_KEY}
+ * property to point directly to a Hadoop tarball which is present locally and
+ * no download will occur.
+ */
+public class TestDynamometerInfra {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestDynamometerInfra.class);
+
+  private static final int MINICLUSTER_NUM_NMS = 3;
+  private static final int MINICLUSTER_NUM_DNS = 1;
+
+  private static final String HADOOP_BIN_PATH_KEY = "dyno.hadoop.bin.path";
+  private static final String HADOOP_BIN_VERSION_KEY =
+      "dyno.hadoop.bin.version";
+  private static final String HADOOP_BIN_VERSION_DEFAULT = "3.1.1";
+  private static final String FSIMAGE_FILENAME = "fsimage_0000000000000061740";
+  private static final String VERSION_FILENAME = "VERSION";
+
+  private static final String HADOOP_BIN_UNPACKED_DIR_PREFIX =
+      "hadoop_unpacked_";
+
+  private static final String NAMENODE_NODELABEL = "dyno_namenode";
+  private static final String DATANODE_NODELABEL = "dyno_datanode";
+
+  private static MiniDFSCluster miniDFSCluster;
+  private static MiniYARNCluster miniYARNCluster;
+  private static YarnClient yarnClient;
+  private static FileSystem fs;
+  private static Configuration conf;
+  private static Configuration yarnConf;
+  private static Path fsImageTmpPath;
+  private static Path fsVersionTmpPath;
+  private static Path blockImageOutputDir;
+  private static Path auditTraceDir;
+  private static Path confZip;
+  private static File testBaseDir;
+  private static File hadoopTarballPath;
+  private static File hadoopUnpackedDir;
+
+  private ApplicationId infraAppId;
+
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    PlatformAssumptions.assumeNotWindows("Dynamometer will not run on Windows");
+    Assume.assumeThat("JAVA_HOME must be set properly",
+        System.getenv("JAVA_HOME"), notNullValue());
+    try {
+      Shell.ShellCommandExecutor tarCheck = new Shell.ShellCommandExecutor(
+          new String[]{"bash", "-c", "command -v tar"});
+      tarCheck.execute();
+      Assume.assumeTrue("tar command is not available",
+          tarCheck.getExitCode() == 0);
+    } catch (IOException ioe) {
+      Assume.assumeNoException("Unable to execute a shell command", ioe);
+    }
+
+    conf = new Configuration();
+    // Follow the conventions of MiniDFSCluster
+    testBaseDir = new File(
+        System.getProperty(PROP_TEST_BUILD_DATA, "build/test/data"));
+    String hadoopBinVersion = System.getProperty(HADOOP_BIN_VERSION_KEY,
+        HADOOP_BIN_VERSION_DEFAULT);
+    if (System.getProperty(HADOOP_BIN_PATH_KEY) == null) {
+      hadoopTarballPath = fetchHadoopTarball(testBaseDir, hadoopBinVersion,
+          conf, LOG);
+    } else {
+      hadoopTarballPath = new File(System.getProperty(HADOOP_BIN_PATH_KEY));
+    }
+    if (testBaseDir.exists()) {
+      // Delete any old unpacked bin dirs that weren't previously cleaned up
+      File[] oldUnpackedDirs = testBaseDir.listFiles(
+          (dir, name) -> name.startsWith(HADOOP_BIN_UNPACKED_DIR_PREFIX));
+      if (oldUnpackedDirs != null) {
+        for (File oldDir : oldUnpackedDirs) {
+          FileUtils.deleteQuietly(oldDir);
+        }
+      }
+    }
+    // Set up the Hadoop binary to be used as the system-level Hadoop install
+    hadoopUnpackedDir = new File(testBaseDir,
+        HADOOP_BIN_UNPACKED_DIR_PREFIX + UUID.randomUUID());
+    assertTrue("Failed to make temporary directory",
+        hadoopUnpackedDir.mkdirs());
+    Shell.ShellCommandExecutor shexec = new Shell.ShellCommandExecutor(
+        new String[] {"tar", "xzf", hadoopTarballPath.getAbsolutePath(), "-C",
+            hadoopUnpackedDir.getAbsolutePath()});
+    shexec.execute();
+    if (shexec.getExitCode() != 0) {
+      fail("Unable to execute tar to expand Hadoop binary");
+    }
+
+    conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128);
+    conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
+    for (String q : new String[] {"root", "root.default"}) {
+      conf.setInt(CapacitySchedulerConfiguration.PREFIX + q + "."
+          + CapacitySchedulerConfiguration.CAPACITY, 100);
+      String accessibleNodeLabelPrefix = CapacitySchedulerConfiguration.PREFIX
+          + q + "." + CapacitySchedulerConfiguration.ACCESSIBLE_NODE_LABELS;
+      conf.set(accessibleNodeLabelPrefix,
+          CapacitySchedulerConfiguration.ALL_ACL);
+      conf.setInt(accessibleNodeLabelPrefix + "." + DATANODE_NODELABEL + "."
+          + CapacitySchedulerConfiguration.CAPACITY, 100);
+      conf.setInt(accessibleNodeLabelPrefix + "." + NAMENODE_NODELABEL + "."
+          + CapacitySchedulerConfiguration.CAPACITY, 100);
+    }
+    // This is necessary to have the RM respect our vcore allocation request
+    conf.setClass(CapacitySchedulerConfiguration.RESOURCE_CALCULATOR_CLASS,
+        DominantResourceCalculator.class, ResourceCalculator.class);
+    conf.setBoolean(YarnConfiguration.NM_DISK_HEALTH_CHECK_ENABLE, false);
+    miniYARNCluster = new MiniYARNCluster(TestDynamometerInfra.class.getName(),
+        1, MINICLUSTER_NUM_NMS, 1, 1);
+    miniYARNCluster.init(conf);
+    miniYARNCluster.start();
+
+    yarnConf = miniYARNCluster.getConfig();
+    miniDFSCluster = new MiniDFSCluster.Builder(conf).format(true)
+        .numDataNodes(MINICLUSTER_NUM_DNS).build();
+    miniDFSCluster.waitClusterUp();
+    FileSystem.setDefaultUri(conf, miniDFSCluster.getURI());
+    FileSystem.setDefaultUri(yarnConf, miniDFSCluster.getURI());
+    fs = miniDFSCluster.getFileSystem();
+
+    URL url = Thread.currentThread().getContextClassLoader()
+        .getResource("yarn-site.xml");
+    if (url == null) {
+      throw new RuntimeException(
+          "Could not find 'yarn-site.xml' dummy file in classpath");
+    }
+    yarnConf.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH,
+        new File(url.getPath()).getParent());
+    // Write the XML to a buffer before writing to the file. writeXml() can
+    // trigger a read of the existing yarn-site.xml, so writing directly could
+    // trigger a read of the file while it is in an inconsistent state
+    // (partially written)
+    try (ByteArrayOutputStream bytesOut = new ByteArrayOutputStream()) {
+      yarnConf.writeXml(bytesOut);
+      try (OutputStream fileOut = new FileOutputStream(
+          new File(url.getPath()))) {
+        fileOut.write(bytesOut.toByteArray());
+      }
+    }
+
+    yarnClient = YarnClient.createYarnClient();
+    yarnClient.init(new Configuration(yarnConf));
+    yarnClient.start();
+
+    fsImageTmpPath = fs.makeQualified(new Path("/tmp/" + FSIMAGE_FILENAME));
+    fsVersionTmpPath = fs.makeQualified(new Path("/tmp/" + VERSION_FILENAME));
+    blockImageOutputDir = fs.makeQualified(new Path("/tmp/blocks"));
+    auditTraceDir = fs.makeQualified(new Path("/tmp/audit_trace_direct"));
+    confZip = fs.makeQualified(new Path("/tmp/conf.zip"));
+
+    uploadFsimageResourcesToHDFS(hadoopBinVersion);
+
+    miniYARNCluster.waitForNodeManagersToConnect(30000);
+
+    RMNodeLabelsManager nodeLabelManager = miniYARNCluster.getResourceManager()
+        .getRMContext().getNodeLabelManager();
+    nodeLabelManager.addToCluserNodeLabelsWithDefaultExclusivity(
+        Sets.newHashSet(NAMENODE_NODELABEL, DATANODE_NODELABEL));
+    Map<NodeId, Set<String>> nodeLabels = new HashMap<>();
+    nodeLabels.put(miniYARNCluster.getNodeManager(0).getNMContext().getNodeId(),
+        Sets.newHashSet(NAMENODE_NODELABEL));
+    nodeLabels.put(miniYARNCluster.getNodeManager(1).getNMContext().getNodeId(),
+        Sets.newHashSet(DATANODE_NODELABEL));
+    nodeLabelManager.addLabelsToNode(nodeLabels);
+  }
+
+  @AfterClass
+  public static void teardownClass() throws Exception {
+    if (miniDFSCluster != null) {
+      miniDFSCluster.shutdown(true);
+      miniDFSCluster = null;
+    }
+    if (yarnClient != null) {
+      yarnClient.stop();
+      yarnClient = null;
+    }
+    if (miniYARNCluster != null) {
+      miniYARNCluster.getResourceManager().stop();
+      miniYARNCluster.getResourceManager().waitForServiceToStop(30000);
+      miniYARNCluster.stop();
+      miniYARNCluster.waitForServiceToStop(30000);
+      FileUtils.deleteDirectory(miniYARNCluster.getTestWorkDir());
+      miniYARNCluster = null;
+    }
+    if (hadoopUnpackedDir != null) {
+      FileUtils.deleteDirectory(hadoopUnpackedDir);
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (infraAppId != null && yarnClient != null) {
+      yarnClient.killApplication(infraAppId);
+    }
+    infraAppId = null;
+  }
+
+  @Test(timeout = 15 * 60 * 1000)
+  public void testNameNodeInYARN() throws Exception {
+    final Client client = new Client(JarFinder.getJar(ApplicationMaster.class),
+        JarFinder.getJar(Assert.class));
+    Configuration localConf = new Configuration(yarnConf);
+    localConf.setLong(AuditLogDirectParser.AUDIT_START_TIMESTAMP_KEY, 60000);
+    client.setConf(localConf);
+
+    Thread appThread = new Thread(() -> {
+      try {
+        client.run(new String[] {"-" + Client.MASTER_MEMORY_MB_ARG, "128",
+            "-" + Client.CONF_PATH_ARG, confZip.toString(),
+            "-" + Client.BLOCK_LIST_PATH_ARG,
+            blockImageOutputDir.toString(), "-" + Client.FS_IMAGE_DIR_ARG,
+            fsImageTmpPath.getParent().toString(),
+            "-" + Client.HADOOP_BINARY_PATH_ARG,
+            hadoopTarballPath.getAbsolutePath(),
+            "-" + AMOptions.DATANODES_PER_CLUSTER_ARG, "2",
+            "-" + AMOptions.DATANODE_MEMORY_MB_ARG, "128",
+            "-" + AMOptions.DATANODE_NODELABEL_ARG, DATANODE_NODELABEL,
+            "-" + AMOptions.NAMENODE_MEMORY_MB_ARG, "256",
+            "-" + AMOptions.NAMENODE_METRICS_PERIOD_ARG, "1",
+            "-" + AMOptions.NAMENODE_NODELABEL_ARG, NAMENODE_NODELABEL,
+            "-" + AMOptions.SHELL_ENV_ARG,
+            "HADOOP_HOME=" + getHadoopHomeLocation(),
+            "-" + AMOptions.SHELL_ENV_ARG,
+            "HADOOP_CONF_DIR=" + getHadoopHomeLocation() + "/etc/hadoop",
+            "-" + Client.WORKLOAD_REPLAY_ENABLE_ARG,
+            "-" + Client.WORKLOAD_INPUT_PATH_ARG,
+            fs.makeQualified(new Path("/tmp/audit_trace_direct")).toString(),
+            "-" + Client.WORKLOAD_THREADS_PER_MAPPER_ARG, "1",
+            "-" + Client.WORKLOAD_START_DELAY_ARG, "10s",
+            "-" + AMOptions.NAMENODE_ARGS_ARG,
+            "-Ddfs.namenode.safemode.extension=0"});
+      } catch (Exception e) {
+        LOG.error("Error running client", e);
+      }
+    });
+
+    appThread.start();
+    LOG.info("Waiting for application ID to become available");
+    GenericTestUtils.waitFor(() -> {
+      try {
+        List<ApplicationReport> apps = yarnClient.getApplications();
+        if (apps.size() == 1) {
+          infraAppId = apps.get(0).getApplicationId();
+          return true;
+        } else if (apps.size() > 1) {
+          fail("Unexpected: more than one application");
+        }
+      } catch (IOException | YarnException e) {
+        fail("Unexpected exception: " + e);
+      }
+      return false;
+    }, 1000, 60000);
+
+    Supplier<Boolean> falseSupplier = () -> false;
+    Optional<Properties> namenodeProperties = DynoInfraUtils
+        .waitForAndGetNameNodeProperties(falseSupplier, localConf,
+            client.getNameNodeInfoPath(), LOG);
+    if (!namenodeProperties.isPresent()) {
+      fail("Unable to fetch NameNode properties");
+    }
+
+    DynoInfraUtils.waitForNameNodeReadiness(namenodeProperties.get(), 3, false,
+        falseSupplier, localConf, LOG);
+
+    // Test that we can successfully write to / read from the cluster
+    try {
+      URI nameNodeUri =
+          DynoInfraUtils.getNameNodeHdfsUri(namenodeProperties.get());
+      DistributedFileSystem dynoFS =
+          (DistributedFileSystem) FileSystem.get(nameNodeUri, localConf);
+      Path testFile = new Path("/tmp/test/foo");
+      dynoFS.mkdir(testFile.getParent(), FsPermission.getDefault());
+      FSDataOutputStream out = dynoFS.create(testFile, (short) 1);
+      out.write(42);
+      out.hsync();
+      out.close();
+      FileStatus[] stats = dynoFS.listStatus(testFile.getParent());
+      assertEquals(1, stats.length);
+      assertEquals("foo", stats[0].getPath().getName());
+    } catch (IOException e) {
+      LOG.error("Failed to write or read", e);
+      throw e;
+    }
+
+    Map<ContainerId, Container> namenodeContainers = miniYARNCluster
+        .getNodeManager(0).getNMContext().getContainers();
+    Map<ContainerId, Container> datanodeContainers = miniYARNCluster
+        .getNodeManager(1).getNMContext().getContainers();
+    Map<ContainerId, Container> amContainers = miniYARNCluster.getNodeManager(2)
+        .getNMContext().getContainers();
+    assertEquals(1, namenodeContainers.size());
+    assertEquals(2,
+        namenodeContainers.keySet().iterator().next().getContainerId());
+    assertEquals(2, datanodeContainers.size());
+    assertEquals(1, amContainers.size());
+    assertEquals(1, amContainers.keySet().iterator().next().getContainerId());
+
+    LOG.info("Waiting for workload job to start and complete");
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return client.getWorkloadJob() != null
+            && client.getWorkloadJob().isComplete();
+      } catch (IOException | IllegalStateException e) {
+        return false;
+      }
+    }, 3000, 60000);
+    LOG.info("Workload job completed");
+
+    if (!client.getWorkloadJob().isSuccessful()) {
+      fail("Workload job failed");
+    }
+    Counters counters = client.getWorkloadJob().getCounters();
+    assertEquals(6,
+        counters.findCounter(AuditReplayMapper.REPLAYCOUNTERS.TOTALCOMMANDS)
+            .getValue());
+    assertEquals(1,
+        counters
+            .findCounter(AuditReplayMapper.REPLAYCOUNTERS.TOTALINVALIDCOMMANDS)
+            .getValue());
+
+    LOG.info("Waiting for infra application to exit");
+    GenericTestUtils.waitFor(() -> {
+      try {
+        ApplicationReport report = yarnClient
+            .getApplicationReport(infraAppId);
+        return report
+              .getYarnApplicationState() == YarnApplicationState.KILLED;
+      } catch (IOException | YarnException e) {
+        return false;
+      }
+    }, 3000, 300000);
+
+    LOG.info("Waiting for metrics file to be ready");
+    // Try to read the metrics file
+    Path hdfsStoragePath = new Path(fs.getHomeDirectory(),
+        DynoConstants.DYNAMOMETER_STORAGE_DIR + "/" + infraAppId);
+    final Path metricsPath = new Path(hdfsStoragePath, "namenode_metrics");
+    GenericTestUtils.waitFor(() -> {
+      try {
+        FSDataInputStream in = fs.open(metricsPath);
+        String metricsOutput = in.readUTF();
+        in.close();
+        // Just assert that there is some metrics content in there
+        assertTrue(metricsOutput.contains("JvmMetrics"));
+        return true;
+      } catch (IOException ioe) {
+        return false;
+      }
+    }, 3000, 60000);
+  }
+
+  private static URI getResourcePath(String resourceName) {
+    try {
+      return TestDynamometerInfra.class.getClassLoader()
+          .getResource(resourceName).toURI();
+    } catch (URISyntaxException e) {
+      return null;
+    }
+  }
+
+  /**
+   * Get the Hadoop home location (i.e. for {@code HADOOP_HOME}) as the only
+   * directory within the unpacked location of the Hadoop tarball.
+   *
+   * @return The absolute path to the Hadoop home directory.
+   */
+  private String getHadoopHomeLocation() {
+    File[] files = hadoopUnpackedDir.listFiles();
+    if (files == null || files.length != 1) {
+      fail("Should be 1 directory within the Hadoop unpacked dir");
+    }
+    return files[0].getAbsolutePath();
+  }
+
+  /**
+   * Look for the resource files relevant to {@code hadoopBinVersion} and upload
+   * them onto the MiniDFSCluster's HDFS for use by the subsequent jobs.
+   *
+   * @param hadoopBinVersion
+   *          The version string (e.g. "3.1.1") for which to look for resources.
+   */
+  private static void uploadFsimageResourcesToHDFS(String hadoopBinVersion)
+      throws IOException {
+    // Keep only the major/minor version for the resources path
+    String[] versionComponents = hadoopBinVersion.split("\\.");
+    if (versionComponents.length < 2) {
+      fail(
+          "At least major and minor version are required to be specified; got: "
+              + hadoopBinVersion);
+    }
+    String hadoopResourcesPath = "hadoop_" + versionComponents[0] + "_"
+        + versionComponents[1];
+    String fsImageResourcePath = hadoopResourcesPath + "/" + FSIMAGE_FILENAME;
+    fs.copyFromLocalFile(new Path(getResourcePath(fsImageResourcePath)),
+        fsImageTmpPath);
+    fs.copyFromLocalFile(
+        new Path(getResourcePath(fsImageResourcePath + ".md5")),
+        fsImageTmpPath.suffix(".md5"));
+    fs.copyFromLocalFile(
+        new Path(getResourcePath(hadoopResourcesPath + "/" + VERSION_FILENAME)),
+        fsVersionTmpPath);
+    fs.mkdirs(auditTraceDir);
+    IOUtils.copyBytes(
+        TestDynamometerInfra.class.getClassLoader()
+            .getResourceAsStream("audit_trace_direct/audit0"),
+        fs.create(new Path(auditTraceDir, "audit0")), conf, true);
+    fs.mkdirs(blockImageOutputDir);
+    for (String blockFile : new String[] {"dn0-a-0-r-00000", "dn1-a-0-r-00001",
+        "dn2-a-0-r-00002"}) {
+      IOUtils.copyBytes(
+          TestDynamometerInfra.class.getClassLoader()
+              .getResourceAsStream("blocks/" + blockFile),
+          fs.create(new Path(blockImageOutputDir, blockFile)), conf, true);
+    }
+    File tempConfZip = new File(testBaseDir, "conf.zip");
+    ZipOutputStream zos = new ZipOutputStream(
+        new FileOutputStream(tempConfZip));
+    for (String file : new String[] {"core-site.xml", "hdfs-site.xml",
+        "log4j.properties"}) {
+      zos.putNextEntry(new ZipEntry("etc/hadoop/" + file));
+      InputStream is = TestDynamometerInfra.class.getClassLoader()
+          .getResourceAsStream("conf/etc/hadoop/" + file);
+      IOUtils.copyBytes(is, zos, conf, false);
+      is.close();
+      zos.closeEntry();
+    }
+    zos.close();
+    fs.copyFromLocalFile(new Path(tempConfZip.toURI()), confZip);
+    tempConfZip.delete();
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/java/org/apache/hadoop/tools/dynamometer/TestDynoInfraUtils.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/java/org/apache/hadoop/tools/dynamometer/TestDynoInfraUtils.java
new file mode 100644
index 0000000..0881347
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/java/org/apache/hadoop/tools/dynamometer/TestDynoInfraUtils.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.hadoop.tools.dynamometer;
+
+import java.util.Set;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+
+/** Tests for {@link DynoInfraUtils}. */
+public class TestDynoInfraUtils {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestDynoInfraUtils.class);
+
+  @Test
+  public void testParseStaleDatanodeListSingleDatanode() throws Exception {
+    // Confirm all types of values can be properly parsed
+    String json = "{"
+        + "\"1.2.3.4:5\": {"
+        + "  \"numBlocks\": 5,"
+        + "  \"fooString\":\"stringValue\","
+        + "  \"fooInteger\": 1,"
+        + "  \"fooFloat\": 1.0,"
+        + "  \"fooArray\": []"
+        + "}"
+        + "}";
+    Set<String> out = DynoInfraUtils.parseStaleDataNodeList(json, 10, LOG);
+    assertEquals(1, out.size());
+    assertTrue(out.contains("1.2.3.4:5"));
+  }
+
+  @Test
+  public void testParseStaleDatanodeListMultipleDatanodes() throws Exception {
+    String json = "{"
+        + "\"1.2.3.4:1\": {\"numBlocks\": 0}, "
+        + "\"1.2.3.4:2\": {\"numBlocks\": 15}, "
+        + "\"1.2.3.4:3\": {\"numBlocks\": 5}, "
+        + "\"1.2.3.4:4\": {\"numBlocks\": 10} "
+        + "}";
+    Set<String> out = DynoInfraUtils.parseStaleDataNodeList(json, 10, LOG);
+    assertEquals(2, out.size());
+    assertTrue(out.contains("1.2.3.4:1"));
+    assertTrue(out.contains("1.2.3.4:3"));
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/blocks/dn0-a-0-r-00000 b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/blocks/dn0-a-0-r-00000
new file mode 100644
index 0000000..d2768e8
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/blocks/dn0-a-0-r-00000
@@ -0,0 +1,34 @@
+1073751825,11001,4
+1073751921,11097,4
+1073751918,11094,4
+1073751915,11091,4
+1073751912,11088,4
+1073751909,11085,4
+1073751906,11082,4
+1073751903,11079,4
+1073751900,11076,4
+1073751897,11073,4
+1073751894,11070,4
+1073751891,11067,4
+1073751888,11064,4
+1073751885,11061,4
+1073751882,11058,4
+1073751879,11055,4
+1073751876,11052,4
+1073751924,11100,4
+1073751873,11049,4
+1073751870,11046,4
+1073751867,11043,4
+1073751864,11040,4
+1073751861,11037,4
+1073751858,11034,4
+1073751855,11031,4
+1073751852,11028,4
+1073751849,11025,4
+1073751846,11022,4
+1073751843,11019,4
+1073751840,11016,4
+1073751837,11013,4
+1073751834,11010,4
+1073751831,11007,4
+1073751828,11004,4
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/blocks/dn1-a-0-r-00001 b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/blocks/dn1-a-0-r-00001
new file mode 100644
index 0000000..f8b3d74
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/blocks/dn1-a-0-r-00001
@@ -0,0 +1,33 @@
+1073751847,11023,4
+1073751922,11098,4
+1073751841,11017,4
+1073751829,11005,4
+1073751919,11095,4
+1073751856,11032,4
+1073751886,11062,4
+1073751916,11092,4
+1073751871,11047,4
+1073751880,11056,4
+1073751913,11089,4
+1073751826,11002,4
+1073751868,11044,4
+1073751910,11086,4
+1073751853,11029,4
+1073751838,11014,4
+1073751907,11083,4
+1073751889,11065,4
+1073751865,11041,4
+1073751904,11080,4
+1073751874,11050,4
+1073751877,11053,4
+1073751901,11077,4
+1073751850,11026,4
+1073751862,11038,4
+1073751898,11074,4
+1073751832,11008,4
+1073751883,11059,4
+1073751895,11071,4
+1073751844,11020,4
+1073751859,11035,4
+1073751892,11068,4
+1073751835,11011,4
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/blocks/dn2-a-0-r-00002 b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/blocks/dn2-a-0-r-00002
new file mode 100644
index 0000000..d191112
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/blocks/dn2-a-0-r-00002
@@ -0,0 +1,33 @@
+1073751890,11066,4
+1073751887,11063,4
+1073751845,11021,4
+1073751884,11060,4
+1073751881,11057,4
+1073751830,11006,4
+1073751878,11054,4
+1073751842,11018,4
+1073751875,11051,4
+1073751872,11048,4
+1073751839,11015,4
+1073751869,11045,4
+1073751866,11042,4
+1073751863,11039,4
+1073751836,11012,4
+1073751860,11036,4
+1073751857,11033,4
+1073751920,11096,4
+1073751917,11093,4
+1073751827,11003,4
+1073751914,11090,4
+1073751854,11030,4
+1073751911,11087,4
+1073751908,11084,4
+1073751833,11009,4
+1073751905,11081,4
+1073751851,11027,4
+1073751902,11078,4
+1073751899,11075,4
+1073751896,11072,4
+1073751848,11024,4
+1073751893,11069,4
+1073751923,11099,4
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/conf/etc/hadoop/core-site.xml b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/conf/etc/hadoop/core-site.xml
new file mode 100644
index 0000000..cdb482a
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/conf/etc/hadoop/core-site.xml
@@ -0,0 +1,20 @@
+<?xml version='1.0' encoding='UTF-8'?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<configuration>
+  <property>
+  <name>hadoop.security.impersonation.provider.class</name>
+  <value>org.apache.hadoop.tools.dynamometer.AllowAllImpersonationProvider</value>
+  </property>
+</configuration>
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/conf/etc/hadoop/hdfs-site.xml b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/conf/etc/hadoop/hdfs-site.xml
new file mode 100644
index 0000000..34069d4
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/conf/etc/hadoop/hdfs-site.xml
@@ -0,0 +1,20 @@
+<?xml version='1.0' encoding='UTF-8'?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<configuration>
+  <property>
+    <name>dfs.datanode.data.dir</name>
+    <value>file:///dynamometer/dfs/data1,file:///dynamometer/dfs/data2</value>
+  </property>
+</configuration>
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/conf/etc/hadoop/log4j.properties b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/conf/etc/hadoop/log4j.properties
new file mode 100644
index 0000000..d06da87
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/conf/etc/hadoop/log4j.properties
@@ -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.
+
+hadoop.root.logger=INFO,RFA
+hadoop.log.dir=.
+hadoop.log.file=hadoop.log
+
+log4j.rootLogger=${hadoop.root.logger}
+log4j.appender.RFA=org.apache.log4j.ConsoleAppender
+log4j.appender.RFA.target=System.err
+log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
\ No newline at end of file
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/hadoop_3_1/VERSION b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/hadoop_3_1/VERSION
new file mode 100644
index 0000000..ff3f84e
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/hadoop_3_1/VERSION
@@ -0,0 +1,7 @@
+#Wed Jan 09 11:28:34 PST 2019
+namespaceID=1157798435
+clusterID=CID-91db90f5-16f6-4309-a88e-263223234059
+cTime=1547062025277
+storageType=NAME_NODE
+blockpoolID=BP-963098314-0.0.0.0-1494453277231
+layoutVersion=-64
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/hadoop_3_1/fsimage_0000000000000061740 b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/hadoop_3_1/fsimage_0000000000000061740
new file mode 100644
index 0000000..6ea4f23
Binary files /dev/null and b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/hadoop_3_1/fsimage_0000000000000061740 differ
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/hadoop_3_1/fsimage_0000000000000061740.md5 b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/hadoop_3_1/fsimage_0000000000000061740.md5
new file mode 100644
index 0000000..178794a
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/hadoop_3_1/fsimage_0000000000000061740.md5
@@ -0,0 +1 @@
+211ac2f198a2d8f0488638d3b11f5238 *fsimage_0000000000000061740
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/yarn-site.xml b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/yarn-site.xml
new file mode 100644
index 0000000..bbe1799
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/resources/yarn-site.xml
@@ -0,0 +1,19 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+<configuration>
+  <!-- Dummy (invalid) config file to be overwritten by tests with MiniCluster configuration. -->
+</configuration>
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/pom.xml b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/pom.xml
new file mode 100644
index 0000000..25a1f91
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/pom.xml
@@ -0,0 +1,146 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<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>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.3.0-SNAPSHOT</version>
+    <relativePath>../../../hadoop-project</relativePath>
+  </parent>
+  <artifactId>hadoop-dynamometer-workload</artifactId>
+  <version>3.3.0-SNAPSHOT</version>
+  <description>Apache Hadoop Dynamometer Workload Simulator</description>
+  <name>Apache Hadoop Dynamometer Workload Simulator</name>
+  <packaging>jar</packaging>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+        <configuration>
+          <attach>true</attach>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <phase>test-compile</phase>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <exclude>src/test/resources/audit_trace_direct/audit0</exclude>
+            <exclude>src/test/resources/audit_trace_hive/audit0</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+  <profiles>
+    <profile>
+      <id>dist</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-assembly-plugin</artifactId>
+            <dependencies>
+              <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-assemblies</artifactId>
+                <version>${project.version}</version>
+              </dependency>
+            </dependencies>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>prepare-package</phase>
+                <goals>
+                  <goal>single</goal>
+                </goals>
+                <configuration>
+                  <appendAssemblyId>false</appendAssemblyId>
+                  <attach>false</attach>
+                  <finalName>${project.artifactId}-${project.version}</finalName>
+                  <descriptorRefs>
+                    <descriptorRef>hadoop-dynamometer-workload</descriptorRef>
+                  </descriptorRefs>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+       <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>deplist</id>
+            <phase>compile</phase>
+            <goals>
+              <goal>list</goal>
+            </goals>
+            <configuration>
+              <!-- referenced by a built-in command -->
+              <outputFile>${project.basedir}/target/hadoop-tools-deps/${project.artifactId}.tools-builtin.txt</outputFile>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/bash/parse-start-timestamp.sh b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/bash/parse-start-timestamp.sh
new file mode 100755
index 0000000..5b4a827
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/bash/parse-start-timestamp.sh
@@ -0,0 +1,52 @@
+#!/usr/bin/env 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.
+
+# This script will determine the timestamp of the last transaction appearing in a
+# given fsimage by looking at the corresponding edits file. This is useful to determine
+# from whence to start collecting audit logs to replay against the fsimage.
+
+if [ $# -lt 1 ] || [ "$1" == "-h" ] || [ "$1" == "--help" ]; then
+  echo "Usage: ./parse-start-timestamp.sh image-txid [ edits-dir ]"
+  echo "       Finds the last timestamp present in the edit file which ends in"
+  echo "       the specified transaction ID (leading 0s not required)."
+  echo "       If edits-dir is specified, looks for edit files under"
+  echo "       edits-dir/current. Otherwise, looks in the current directory."
+  exit 1
+fi
+if [[ $(command -v gawk) == "" ]]; then
+  echo "This script requires gawk to be available."
+  exit 1
+fi
+image_txid="$1"
+if [[ $# -ge 2 ]]; then
+  edits_dir="$2/current"
+else
+  edits_dir="$(pwd)"
+fi
+
+edits_file_count="$(find -H "${edits_dir}" -depth 1 -type f -name "edits_*-*$image_txid" | wc -l)"
+if [[ "$edits_file_count" != 1 ]]; then
+  echo "Error; found $edits_file_count matching edit files."
+  exit 1
+fi
+edits_file="$(find -H "${edits_dir}" -depth 1 -type f -name "edits_*-*$image_txid")"
+
+# Shellcheck complains about the $ in the single-quote because it won't expand, but this is intentional
+# shellcheck disable=SC2016
+awk_script='/TIMESTAMP/ { line=$0 } END { match(line, />([[:digit:]]+)</, output); print output[1] }'
+echo "Start timestamp for $image_txid is: (this may take a moment)"
+hdfs oev -i "$edits_dir/$edits_file" -o >(gawk "$awk_script")
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/bash/start-workload.sh b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/bash/start-workload.sh
new file mode 100755
index 0000000..50b3ae9
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/bash/start-workload.sh
@@ -0,0 +1,42 @@
+#!/usr/bin/env 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.
+
+# This script simply passes its arguments along to the workload driver
+# driver after finding a hadoop command in PATH/HADOOP_COMMON_HOME/HADOOP_HOME
+# (searching in that order).
+
+if type hadoop &> /dev/null; then
+  hadoop_cmd="hadoop"
+elif type "$HADOOP_COMMON_HOME/bin/hadoop" &> /dev/null; then
+  hadoop_cmd="$HADOOP_COMMON_HOME/bin/hadoop"
+elif type "$HADOOP_HOME/bin/hadoop" &> /dev/null; then
+  hadoop_cmd="$HADOOP_HOME/bin/hadoop"
+else
+  echo "Unable to find a valid hadoop command to execute; exiting."
+  exit 1
+fi
+
+script_pwd="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)/.."
+
+for f in ${script_pwd}/lib/*.jar; do
+  # Skip adding the workload JAR since it is added by the `hadoop jar` command
+  if [[ "$f" != *"dynamometer-workload-"* ]]; then
+    export HADOOP_CLASSPATH="$HADOOP_CLASSPATH:$f"
+  fi
+done
+"$hadoop_cmd" jar "${script_pwd}"/lib/dynamometer-workload-*.jar \
+  org.apache.hadoop.tools.dynamometer.workloadgenerator.WorkloadDriver "$@"
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/CreateFileMapper.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/CreateFileMapper.java
new file mode 100644
index 0000000..8855f44
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/CreateFileMapper.java
@@ -0,0 +1,146 @@
+/**
+ * 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.hadoop.tools.dynamometer.workloadgenerator;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.Mapper;
+
+/**
+ * CreateFileMapper continuously creates 1 byte files for the specified duration
+ * to increase the number of file objects on the NN.
+ *
+ * <p/>Configuration options available:
+ * <ul>
+ *   <li>{@value NUM_MAPPERS_KEY} (required): Number of mappers to launch.</li>
+ *   <li>{@value DURATION_MIN_KEY} (required): Number of minutes to induce
+ *       workload for.</li>
+ *   <li>{@value SHOULD_DELETE_KEY} (default: {@value SHOULD_DELETE_DEFAULT}):
+ *       If true, delete the files after creating them. This can be useful for
+ *       generating constant load without increasing the number of file
+ *       objects.</li>
+ *   <li>{@value FILE_PARENT_PATH_KEY} (default:
+ *       {@value FILE_PARENT_PATH_DEFAULT}): The root directory in which to
+ *       create files.</li>
+ * </ul>
+ */
+public class CreateFileMapper
+    extends WorkloadMapper<NullWritable, NullWritable> {
+
+  public static final String NUM_MAPPERS_KEY = "createfile.num-mappers";
+  public static final String DURATION_MIN_KEY = "createfile.duration-min";
+  public static final String FILE_PARENT_PATH_KEY =
+      "createfile.file-parent-path";
+  public static final String FILE_PARENT_PATH_DEFAULT = "/tmp/createFileMapper";
+  public static final String SHOULD_DELETE_KEY = "createfile.should-delete";
+  public static final boolean SHOULD_DELETE_DEFAULT = false;
+
+  /** Custom {@link org.apache.hadoop.mapreduce.Counter} definitions. */
+  public enum CREATEFILECOUNTERS {
+    NUMFILESCREATED
+  }
+
+  private long startTimestampMs;
+  private FileSystem fs;
+  private Configuration conf;
+  private int taskID;
+  private String fileParentPath;
+  private boolean shouldDelete;
+  private long endTimeStampMs;
+
+  @Override
+  public String getDescription() {
+    return "This mapper creates 1-byte files for the specified duration.";
+  }
+
+  @Override
+  public List<String> getConfigDescriptions() {
+    return Lists.newArrayList(
+        NUM_MAPPERS_KEY + " (required): Number of mappers to launch.",
+        DURATION_MIN_KEY
+            + " (required): Number of minutes to induce workload for.",
+        SHOULD_DELETE_KEY + " (default: " + SHOULD_DELETE_DEFAULT
+            + "): If true, delete the files after creating "
+            + "them. This can be useful for generating constant load without "
+            + "increasing the number of file objects.",
+        FILE_PARENT_PATH_KEY + " (default: " + FILE_PARENT_PATH_DEFAULT
+            + "): The root directory in which to create files.");
+  }
+
+  @Override
+  public boolean verifyConfigurations(Configuration confToVerify) {
+    return confToVerify.get(NUM_MAPPERS_KEY) != null
+        && confToVerify.get(DURATION_MIN_KEY) != null;
+  }
+
+  @Override
+  public void map(NullWritable key, NullWritable value,
+      Mapper.Context mapperContext) throws IOException, InterruptedException {
+    taskID = mapperContext.getTaskAttemptID().getTaskID().getId();
+    conf = mapperContext.getConfiguration();
+    String namenodeURI = conf.get(WorkloadDriver.NN_URI);
+    startTimestampMs = conf.getLong(WorkloadDriver.START_TIMESTAMP_MS, -1);
+    fileParentPath = conf.get(FILE_PARENT_PATH_KEY, FILE_PARENT_PATH_DEFAULT);
+    shouldDelete = conf.getBoolean(SHOULD_DELETE_KEY, SHOULD_DELETE_DEFAULT);
+    int durationMin = conf.getInt(DURATION_MIN_KEY, -1);
+    if (durationMin < 0) {
+      throw new IOException("Duration must be positive; got: " + durationMin);
+    }
+    endTimeStampMs = startTimestampMs
+        + TimeUnit.MILLISECONDS.convert(durationMin, TimeUnit.MINUTES);
+    fs = FileSystem.get(URI.create(namenodeURI), conf);
+    System.out.println("Start timestamp: " + startTimestampMs);
+
+    long currentEpoch = System.currentTimeMillis();
+    long delay = startTimestampMs - currentEpoch;
+    if (delay > 0) {
+      System.out.println("Sleeping for " + delay + " ms");
+      Thread.sleep(delay);
+    }
+
+    String mapperSpecifcPathPrefix = fileParentPath + "/mapper" + taskID;
+    System.out.println("Mapper path prefix: " + mapperSpecifcPathPrefix);
+    long numFilesCreated = 0;
+    Path path;
+    final byte[] content = {0x0};
+    while (System.currentTimeMillis() < endTimeStampMs) {
+      path = new Path(mapperSpecifcPathPrefix + "/file" + numFilesCreated);
+      OutputStream out = fs.create(path);
+      out.write(content);
+      out.close();
+      numFilesCreated++;
+      mapperContext.getCounter(CREATEFILECOUNTERS.NUMFILESCREATED)
+          .increment(1L);
+      if (numFilesCreated % 1000 == 0) {
+        mapperContext.progress();
+        System.out.println("Number of files created: " + numFilesCreated);
+      }
+      if (shouldDelete) {
+        fs.delete(path, true);
+      }
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/VirtualInputFormat.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/VirtualInputFormat.java
new file mode 100644
index 0000000..2206412
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/VirtualInputFormat.java
@@ -0,0 +1,57 @@
+/**
+ * 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.hadoop.tools.dynamometer.workloadgenerator;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+
+
+/**
+ * An input format which does not read any input, but rather starts a
+ * configurable number of mappers and runs them for a configurable duration.
+ */
+public class VirtualInputFormat<K, V> extends FileInputFormat<K, V> {
+  // Number of splits = Number of mappers. Creates fakeSplits to launch
+  // the required number of mappers
+  @Override
+  public List<InputSplit> getSplits(JobContext job) throws IOException {
+    Configuration conf = job.getConfiguration();
+    int numMappers = conf.getInt(CreateFileMapper.NUM_MAPPERS_KEY, -1);
+    if (numMappers == -1) {
+      throw new IOException("Number of mappers should be provided as input");
+    }
+    List<InputSplit> splits = new ArrayList<InputSplit>(numMappers);
+    for (int i = 0; i < numMappers; i++) {
+      splits.add(new VirtualInputSplit());
+    }
+    return splits;
+  }
+
+  @Override
+  public RecordReader<K, V> createRecordReader(InputSplit split,
+      TaskAttemptContext context) throws IOException, InterruptedException {
+    return new VirtualRecordReader<>();
+  }
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/VirtualInputSplit.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/VirtualInputSplit.java
new file mode 100644
index 0000000..cd4c87f
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/VirtualInputSplit.java
@@ -0,0 +1,52 @@
+/**
+ * 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.hadoop.tools.dynamometer.workloadgenerator;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+
+/**
+ * A fake input split.
+ */
+public class VirtualInputSplit extends InputSplit implements Writable {
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    // do Nothing
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    // do Nothing
+  }
+
+  @Override
+  public long getLength() throws IOException, InterruptedException {
+    return 0;
+  }
+
+  @Override
+  public String[] getLocations() throws IOException, InterruptedException {
+    return new String[] {};
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/VirtualRecordReader.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/VirtualRecordReader.java
new file mode 100644
index 0000000..b9c176e
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/VirtualRecordReader.java
@@ -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.
+ */
+package org.apache.hadoop.tools.dynamometer.workloadgenerator;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+
+/**
+ * A simple fake record reader which simply runs for some time duration.
+ */
+@SuppressWarnings("unchecked")
+public class VirtualRecordReader<K, V> extends RecordReader<K, V> {
+  private int durationMs;
+  private long startTimestampInMs;
+  private long endTimestampInMs;
+  private int numRows = 1;
+
+  @Override
+  public void initialize(InputSplit split, TaskAttemptContext context)
+      throws IOException, InterruptedException {
+    Configuration conf = context.getConfiguration();
+    durationMs = conf.getInt(CreateFileMapper.DURATION_MIN_KEY, 0) * 60 * 1000;
+    startTimestampInMs = conf.getInt(WorkloadDriver.START_TIMESTAMP_MS, 0);
+    endTimestampInMs = startTimestampInMs + durationMs;
+  }
+
+  // The map function per split should be invoked only once.
+  @Override
+  public boolean nextKeyValue() throws IOException, InterruptedException {
+    if (numRows > 0) {
+      numRows--;
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public K getCurrentKey() throws IOException, InterruptedException {
+    return (K) NullWritable.get();
+  }
+
+  @Override
+  public V getCurrentValue() throws IOException, InterruptedException {
+    return (V) NullWritable.get();
+  }
+
+  @Override
+  public float getProgress() throws IOException, InterruptedException {
+    long remainingMs = endTimestampInMs - System.currentTimeMillis();
+    return (remainingMs * 100.0f) / durationMs;
+  }
+
+  @Override
+  public void close() throws IOException {
+    // do Nothing
+  }
+};
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/WorkloadDriver.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/WorkloadDriver.java
new file mode 100644
index 0000000..69e3942
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/WorkloadDriver.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.hadoop.tools.dynamometer.workloadgenerator;
+
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditReplayMapper;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.OptionGroup;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.PosixParser;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is the driver for generating generic workloads against a NameNode under
+ * test. It launches a map-only job with a mapper class specified by the
+ * {@value MAPPER_CLASS_NAME} argument. See the specific mappers (currently
+ * {@link AuditReplayMapper} and {@link CreateFileMapper}) for information on
+ * their specific behavior and parameters.
+ */
+public class WorkloadDriver extends Configured implements Tool {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(WorkloadDriver.class);
+
+  public static final String START_TIMESTAMP_MS = "start_timestamp_ms";
+  public static final String START_TIME_OFFSET = "start_time_offset";
+  public static final String START_TIME_OFFSET_DEFAULT = "1m";
+  public static final String NN_URI = "nn_uri";
+  public static final String MAPPER_CLASS_NAME = "mapper_class_name";
+
+  public int run(String[] args) throws Exception {
+    Option helpOption = new Option("h", "help", false,
+        "Shows this message. Additionally specify the " + MAPPER_CLASS_NAME
+            + " argument to show help for a specific mapper class.");
+    Options options = new Options();
+    options.addOption(helpOption);
+    options.addOption(OptionBuilder.withArgName("NN URI").hasArg()
+        .withDescription("URI of the NameNode under test").isRequired()
+        .create(NN_URI));
+    OptionGroup startTimeOptions = new OptionGroup();
+    startTimeOptions.addOption(OptionBuilder.withArgName("Start Timestamp")
+        .hasArg().withDescription("Mapper start UTC timestamp in ms")
+        .create(START_TIMESTAMP_MS));
+    startTimeOptions
+        .addOption(OptionBuilder.withArgName("Start Time Offset").hasArg()
+            .withDescription("Mapper start time as an offset from current "
+                + "time. Human-readable formats accepted, e.g. 10m (default "
+                + START_TIME_OFFSET_DEFAULT + ").")
+            .create(START_TIME_OFFSET));
+    options.addOptionGroup(startTimeOptions);
+    Option mapperClassOption = OptionBuilder.withArgName("Mapper ClassName")
+        .hasArg()
+        .withDescription("Class name of the mapper; must be a WorkloadMapper "
+            + "subclass. Mappers supported currently: \n"
+            + "1. AuditReplayMapper \n"
+            + "2. CreateFileMapper \n"
+            + "Fully specified class names are also supported.")
+        .isRequired().create(MAPPER_CLASS_NAME);
+    options.addOption(mapperClassOption);
+
+    Options helpOptions = new Options();
+    helpOptions.addOption(helpOption);
+    Option mapperClassNotRequiredOption = (Option) mapperClassOption.clone();
+    mapperClassNotRequiredOption.setRequired(false);
+    helpOptions.addOption(mapperClassNotRequiredOption);
+
+    CommandLineParser parser = new PosixParser();
+    CommandLine cli = parser.parse(helpOptions, args, true);
+    if (cli.hasOption("h")) {
+      String footer = null;
+      if (cli.hasOption(MAPPER_CLASS_NAME)) {
+        footer = getMapperUsageInfo(cli.getOptionValue(MAPPER_CLASS_NAME));
+      }
+
+      HelpFormatter formatter = new HelpFormatter();
+      formatter.printHelp(200, "./start-workload [options]", null, options,
+          footer);
+      return 1;
+    }
+
+    cli = parser.parse(options, args);
+    String nnURI = cli.getOptionValue(NN_URI);
+    long startTimestampMs;
+    if (cli.hasOption(START_TIMESTAMP_MS)) {
+      startTimestampMs = Long.parseLong(cli.getOptionValue(START_TIMESTAMP_MS));
+    } else {
+      // Leverage the human-readable time parsing capabilities of Configuration
+      String tmpConfKey = "___temp_config_property___";
+      Configuration tmpConf = new Configuration();
+      tmpConf.set(tmpConfKey,
+          cli.getOptionValue(START_TIME_OFFSET, START_TIME_OFFSET_DEFAULT));
+      startTimestampMs = tmpConf.getTimeDuration(tmpConfKey, 0,
+          TimeUnit.MILLISECONDS) + System.currentTimeMillis();
+    }
+    Class<? extends WorkloadMapper> mapperClass = getMapperClass(
+        cli.getOptionValue(MAPPER_CLASS_NAME));
+    if (!mapperClass.newInstance().verifyConfigurations(getConf())) {
+      System.err
+          .println(getMapperUsageInfo(cli.getOptionValue(MAPPER_CLASS_NAME)));
+      return 1;
+    }
+
+    Job job = getJobForSubmission(getConf(), nnURI, startTimestampMs,
+        mapperClass);
+
+    boolean success = job.waitForCompletion(true);
+    return success ? 0 : 1;
+  }
+
+  public static Job getJobForSubmission(Configuration baseConf, String nnURI,
+      long startTimestampMs, Class<? extends WorkloadMapper> mapperClass)
+      throws IOException, ClassNotFoundException, InstantiationException,
+      IllegalAccessException {
+    Configuration conf = new Configuration(baseConf);
+    conf.set(NN_URI, nnURI);
+    conf.setBoolean(MRJobConfig.MAP_SPECULATIVE, false);
+
+    String startTimeString = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss z")
+        .format(new Date(startTimestampMs));
+    LOG.info("The workload will start at " + startTimestampMs + " ms ("
+        + startTimeString + ")");
+    conf.setLong(START_TIMESTAMP_MS, startTimestampMs);
+
+    Job job = Job.getInstance(conf, "Dynamometer Workload Driver");
+    job.setOutputFormatClass(NullOutputFormat.class);
+    job.setJarByClass(mapperClass);
+    job.setMapperClass(mapperClass);
+    job.setInputFormatClass(mapperClass.newInstance().getInputFormat(conf));
+    job.setOutputFormatClass(NullOutputFormat.class);
+    job.setNumReduceTasks(0);
+    job.setMapOutputKeyClass(NullWritable.class);
+    job.setMapOutputValueClass(NullWritable.class);
+    job.setOutputKeyClass(NullWritable.class);
+    job.setOutputValueClass(NullWritable.class);
+
+    return job;
+  }
+
+  public static void main(String[] args) throws Exception {
+    WorkloadDriver driver = new WorkloadDriver();
+    System.exit(ToolRunner.run(driver, args));
+  }
+
+  private Class<? extends WorkloadMapper> getMapperClass(String className)
+      throws ClassNotFoundException {
+    if (!className.contains(".")) {
+      className = WorkloadDriver.class.getPackage().getName() + "." + className;
+    }
+    Class<?> mapperClass = getConf().getClassByName(className);
+    if (!WorkloadMapper.class.isAssignableFrom(mapperClass)) {
+      throw new IllegalArgumentException(className + " is not a subclass of "
+          + WorkloadMapper.class.getCanonicalName());
+    }
+    return (Class<? extends WorkloadMapper>) mapperClass;
+  }
+
+  private String getMapperUsageInfo(String mapperClassName)
+      throws ClassNotFoundException, InstantiationException,
+      IllegalAccessException {
+    WorkloadMapper<?, ?> mapper = getMapperClass(mapperClassName).newInstance();
+    StringBuilder builder = new StringBuilder("Usage for ");
+    builder.append(mapper.getClass().getSimpleName());
+    builder.append(":\n");
+    builder.append(mapper.getDescription());
+    for (String configDescription : mapper.getConfigDescriptions()) {
+      builder.append("\n    ");
+      builder.append(configDescription);
+    }
+    builder.append("\nConfiguration parameters can be set at the ");
+    builder.append("_start_ of the argument list like:\n");
+    builder.append("  -Dconfiguration.key=configurationValue");
+
+    return builder.toString();
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/WorkloadMapper.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/WorkloadMapper.java
new file mode 100644
index 0000000..1dfe8ce
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/WorkloadMapper.java
@@ -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.
+ */
+package org.apache.hadoop.tools.dynamometer.workloadgenerator;
+
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.Mapper;
+
+/**
+ * Represents the base class for a generic workload-generating mapper. By
+ * default, it will expect to use {@link VirtualInputFormat} as its
+ * {@link InputFormat}. Subclasses expecting a different {@link InputFormat}
+ * should override the {@link #getInputFormat(Configuration)} method.
+ */
+public abstract class WorkloadMapper<KEYIN, VALUEIN>
+    extends Mapper<KEYIN, VALUEIN, NullWritable, NullWritable> {
+
+  /**
+   * Return the input class to be used by this mapper.
+   */
+  public Class<? extends InputFormat> getInputFormat(Configuration conf) {
+    return VirtualInputFormat.class;
+  }
+
+  /**
+   * Get the description of the behavior of this mapper.
+   */
+  public abstract String getDescription();
+
+  /**
+   * Get a list of the description of each configuration that this mapper
+   * accepts.
+   */
+  public abstract List<String> getConfigDescriptions();
+
+  /**
+   * Verify that the provided configuration contains all configurations required
+   * by this mapper.
+   */
+  public abstract boolean verifyConfigurations(Configuration conf);
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditCommandParser.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditCommandParser.java
new file mode 100644
index 0000000..8711cb7
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditCommandParser.java
@@ -0,0 +1,57 @@
+/**
+ * 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.hadoop.tools.dynamometer.workloadgenerator.audit;
+
+import java.io.IOException;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This interface represents a pluggable command parser. It will accept in one
+ * line of {@link Text} input at a time and return an {@link AuditReplayCommand}
+ * which represents the input text. Each input line should produce exactly one
+ * command.
+ */
+public interface AuditCommandParser {
+
+  /**
+   * Initialize this parser with the given configuration. Guaranteed to be
+   * called prior to any calls to {@link #parse(Text, Function)}.
+   *
+   * @param conf The Configuration to be used to set up this parser.
+   */
+  void initialize(Configuration conf) throws IOException;
+
+  /**
+   * Convert a line of input into an {@link AuditReplayCommand}. Since
+   * {@link AuditReplayCommand}s store absolute timestamps, relativeToAbsolute
+   * can be used to convert relative timestamps (i.e., milliseconds elapsed
+   * between the start of the audit log and this command) into absolute
+   * timestamps.
+   *
+   * @param inputLine Single input line to convert.
+   * @param relativeToAbsolute Function converting relative timestamps
+   *                           (in milliseconds) to absolute timestamps
+   *                           (in milliseconds).
+   * @return A command representing the input line.
+   */
+  AuditReplayCommand parse(Text inputLine,
+      Function<Long, Long> relativeToAbsolute) throws IOException;
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditLogDirectParser.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditLogDirectParser.java
new file mode 100644
index 0000000..216e4a3
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditLogDirectParser.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.hadoop.tools.dynamometer.workloadgenerator.audit;
+
+import com.google.common.base.Splitter;
+import java.io.IOException;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This {@link AuditCommandParser} is used to read commands from an audit log in
+ * the original format audit logs are produced in with a standard configuration.
+ * It requires setting the {@value AUDIT_START_TIMESTAMP_KEY} configuration to
+ * specify what the start time of the audit log was to determine when events
+ * occurred relative to this start time.
+ *
+ * <p/>By default, this assumes that the audit log is in the default log format
+ * set up by Hadoop, like:
+ * <pre>{@code
+ *   1970-01-01 00:00:00,000 INFO FSNamesystem.audit: allowed=true ...
+ * }</pre>
+ * You can adjust this parsing behavior using the various configurations
+ * available.
+ */
+public class AuditLogDirectParser implements AuditCommandParser {
+
+  /** See class Javadoc for more detail. */
+  public static final String AUDIT_START_TIMESTAMP_KEY =
+      "auditreplay.log-start-time.ms";
+
+  /**
+   * The format string used to parse the date which is present in the audit
+   * log. This must be a format understood by {@link SimpleDateFormat}.
+   */
+  public static final String AUDIT_LOG_DATE_FORMAT_KEY =
+      "auditreplay.log-date.format";
+  public static final String AUDIT_LOG_DATE_FORMAT_DEFAULT =
+      "yyyy-MM-dd HH:mm:ss,SSS";
+
+  /**
+   * The time zone to use when parsing the audit log timestamp. This must
+   * be a format recognized by {@link TimeZone#getTimeZone(String)}.
+   */
+  public static final String AUDIT_LOG_DATE_TIME_ZONE_KEY =
+      "auditreplay.log-date.time-zone";
+  public static final String AUDIT_LOG_DATE_TIME_ZONE_DEFAULT = "UTC";
+
+  /**
+   * The regex to use when parsing the audit log lines. This should match
+   * against a single log line, and create two named capture groups. One
+   * must be titled "timestamp" and return a timestamp which can be parsed
+   * by a {@link DateFormat date formatter}. The other must be titled "message"
+   * and return the audit content, such as "allowed=true ugi=user ...". See
+   * {@link #AUDIT_LOG_PARSE_REGEX_DEFAULT} for an example.
+   */
+  public static final String AUDIT_LOG_PARSE_REGEX_KEY =
+      "auditreplay.log-parse-regex";
+  public static final String AUDIT_LOG_PARSE_REGEX_DEFAULT =
+      "^(?<timestamp>.+?) INFO [^:]+: (?<message>.+)$";
+
+  private static final Splitter.MapSplitter AUDIT_SPLITTER = Splitter.on("\t")
+      .trimResults().omitEmptyStrings().withKeyValueSeparator("=");
+  private static final Splitter SPACE_SPLITTER = Splitter.on(" ").trimResults()
+      .omitEmptyStrings();
+
+  private long startTimestamp;
+  private DateFormat dateFormat;
+  private Pattern logLineParseRegex;
+
+  @Override
+  public void initialize(Configuration conf) throws IOException {
+    startTimestamp = conf.getLong(AUDIT_START_TIMESTAMP_KEY, -1);
+    if (startTimestamp < 0) {
+      throw new IOException(
+          "Invalid or missing audit start timestamp: " + startTimestamp);
+    }
+    dateFormat = new SimpleDateFormat(conf.get(AUDIT_LOG_DATE_FORMAT_KEY,
+        AUDIT_LOG_DATE_FORMAT_DEFAULT));
+    String timeZoneString = conf.get(AUDIT_LOG_DATE_TIME_ZONE_KEY,
+        AUDIT_LOG_DATE_TIME_ZONE_DEFAULT);
+    dateFormat.setTimeZone(TimeZone.getTimeZone(timeZoneString));
+    String logLineParseRegexString =
+        conf.get(AUDIT_LOG_PARSE_REGEX_KEY, AUDIT_LOG_PARSE_REGEX_DEFAULT);
+    if (!logLineParseRegexString.contains("(?<timestamp>")
+        && logLineParseRegexString.contains("(?<message>")) {
+      throw new IllegalArgumentException("Must configure regex with named "
+          + "capture groups 'timestamp' and 'message'");
+    }
+    logLineParseRegex = Pattern.compile(logLineParseRegexString);
+  }
+
+  @Override
+  public AuditReplayCommand parse(Text inputLine,
+      Function<Long, Long> relativeToAbsolute) throws IOException {
+    Matcher m = logLineParseRegex.matcher(inputLine.toString());
+    if (!m.find()) {
+      throw new IOException(
+          "Unable to find valid message pattern from audit log line: `"
+              + inputLine + "` using regex `" + logLineParseRegex + "`");
+    }
+    long relativeTimestamp;
+    try {
+      relativeTimestamp = dateFormat.parse(m.group("timestamp")).getTime()
+          - startTimestamp;
+    } catch (ParseException p) {
+      throw new IOException("Exception while parsing timestamp from audit log",
+          p);
+    }
+    // Sanitize the = in the rename options field into a : so we can split on =
+    String auditMessageSanitized =
+        m.group("message").replace("(options=", "(options:");
+    Map<String, String> parameterMap = AUDIT_SPLITTER
+        .split(auditMessageSanitized);
+    return new AuditReplayCommand(relativeToAbsolute.apply(relativeTimestamp),
+        // Split the UGI on space to remove the auth and proxy portions of it
+        SPACE_SPLITTER.split(parameterMap.get("ugi")).iterator().next(),
+        parameterMap.get("cmd").replace("(options:", "(options="),
+        parameterMap.get("src"), parameterMap.get("dst"),
+        parameterMap.get("ip"));
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditLogHiveTableParser.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditLogHiveTableParser.java
new file mode 100644
index 0000000..4d36eff
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditLogHiveTableParser.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.hadoop.tools.dynamometer.workloadgenerator.audit;
+
+import java.io.IOException;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+
+/**
+ * This {@link AuditCommandParser} is used to read commands assuming that the
+ * input was generated by a Hive query storing uncompressed output files, in
+ * which fields should be separated by the start-of-heading (U+0001) character.
+ * The fields available should be, in order:
+ *
+ * <pre>
+ *   relativeTimestampMs,ugi,command,src,dest,sourceIP
+ * </pre>
+ *
+ * Where relativeTimestampMs represents the time elapsed between the start of
+ * the audit log and the occurrence of the audit event. Assuming your audit logs
+ * are available in Hive, this can be generated with a query looking like:
+ *
+ * <pre>
+ *   INSERT OVERWRITE DIRECTORY '${outputPath}'
+ *   SELECT (timestamp - ${startTime} AS relTime, ugi, cmd, src, dst, ip
+ *   FROM '${auditLogTableLocation}'
+ *   WHERE timestamp >= ${startTime} AND timestamp < ${endTime}
+ *   DISTRIBUTE BY src
+ *   SORT BY relTime ASC;
+ * </pre>
+ *
+ * Note that the sorting step is important; events in each distinct file must be
+ * in time-ascending order.
+ */
+public class AuditLogHiveTableParser implements AuditCommandParser {
+
+  private static final String FIELD_SEPARATOR = "\u0001";
+
+  @Override
+  public void initialize(Configuration conf) throws IOException {
+    // Nothing to be done
+  }
+
+  @Override
+  public AuditReplayCommand parse(Text inputLine,
+      Function<Long, Long> relativeToAbsolute) throws IOException {
+    String[] fields = inputLine.toString().split(FIELD_SEPARATOR);
+    long absoluteTimestamp = relativeToAbsolute
+        .apply(Long.parseLong(fields[0]));
+    return new AuditReplayCommand(absoluteTimestamp, fields[1], fields[2],
+        fields[3], fields[4], fields[5]);
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditReplayCommand.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditReplayCommand.java
new file mode 100644
index 0000000..038312d
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditReplayCommand.java
@@ -0,0 +1,163 @@
+/**
+ * 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.hadoop.tools.dynamometer.workloadgenerator.audit;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class represents a single command to be replayed by the workload
+ * generator. It implements the {@link Delayed} interface so that they can be
+ * fetched in timestamp order from a {@link java.util.concurrent.DelayQueue}.
+ * You can use the {@link #getPoisonPill(long)} method to retrieve "Poison Pill"
+ * {@link AuditReplayCommand} which has {@link #isPoison()} as true,
+ * representing to a consumer(s) of the {@link java.util.concurrent.DelayQueue}
+ * that it should stop processing further items and instead terminate itself.
+ */
+class AuditReplayCommand implements Delayed {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(AuditReplayCommand.class);
+  private static final Pattern SIMPLE_UGI_PATTERN = Pattern
+      .compile("([^/@ ]*).*?");
+
+  private long absoluteTimestamp;
+  private String ugi;
+  private String command;
+  private String src;
+  private String dest;
+  private String sourceIP;
+
+  AuditReplayCommand(long absoluteTimestamp, String ugi, String command,
+      String src, String dest, String sourceIP) {
+    this.absoluteTimestamp = absoluteTimestamp;
+    this.ugi = ugi;
+    this.command = command;
+    this.src = src;
+    this.dest = dest;
+    this.sourceIP = sourceIP;
+  }
+
+  long getAbsoluteTimestamp() {
+    return absoluteTimestamp;
+  }
+
+  String getSimpleUgi() {
+    Matcher m = SIMPLE_UGI_PATTERN.matcher(ugi);
+    if (m.matches()) {
+      return m.group(1);
+    } else {
+      LOG.error("Error parsing simple UGI <{}>; falling back to current user",
+          ugi);
+      try {
+        return UserGroupInformation.getCurrentUser().getShortUserName();
+      } catch (IOException ioe) {
+        return "";
+      }
+    }
+  }
+
+  String getCommand() {
+    return command;
+  }
+
+  String getSrc() {
+    return src;
+  }
+
+  String getDest() {
+    return dest;
+  }
+
+  String getSourceIP() {
+    return sourceIP;
+  }
+
+  @Override
+  public long getDelay(TimeUnit unit) {
+    return unit.convert(absoluteTimestamp - System.currentTimeMillis(),
+        TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public int compareTo(Delayed o) {
+    return Long.compare(absoluteTimestamp,
+        ((AuditReplayCommand) o).absoluteTimestamp);
+  }
+
+  /**
+   * If true, the thread which consumes this item should not process any further
+   * items and instead simply terminate itself.
+   */
+  boolean isPoison() {
+    return false;
+  }
+
+  /**
+   * A command representing a Poison Pill, indicating that the processing thread
+   * should not process any further items and instead should terminate itself.
+   * Always returns true for {@link #isPoison()}. It does not contain any other
+   * information besides a timestamp; other getter methods wil return null.
+   */
+  private static final class PoisonPillCommand extends AuditReplayCommand {
+
+    private PoisonPillCommand(long absoluteTimestamp) {
+      super(absoluteTimestamp, null, null, null, null, null);
+    }
+
+    @Override
+    boolean isPoison() {
+      return true;
+    }
+
+  }
+
+  static AuditReplayCommand getPoisonPill(long relativeTimestamp) {
+    return new PoisonPillCommand(relativeTimestamp);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof AuditReplayCommand)) {
+      return false;
+    }
+    AuditReplayCommand o = (AuditReplayCommand) other;
+    return absoluteTimestamp == o.absoluteTimestamp && ugi.equals(o.ugi)
+        && command.equals(o.command) && src.equals(o.src) && dest.equals(o.dest)
+        && sourceIP.equals(o.sourceIP);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(absoluteTimestamp, ugi, command, src, dest, sourceIP);
+  }
+
+  @Override
+  public String toString() {
+    return String.format("AuditReplayCommand(absoluteTimestamp=%d, ugi=%s, "
+            + "command=%s, src=%s, dest=%s, sourceIP=%s",
+        absoluteTimestamp, ugi, command, src, dest, sourceIP);
+  }
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditReplayMapper.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditReplayMapper.java
new file mode 100644
index 0000000..8e279ae
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditReplayMapper.java
@@ -0,0 +1,290 @@
+/**
+ * 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.hadoop.tools.dynamometer.workloadgenerator.audit;
+
+import com.google.common.collect.Lists;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.WorkloadDriver;
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.WorkloadMapper;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditReplayMapper.CommandType.READ;
+import static org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditReplayMapper.CommandType.WRITE;
+
+/**
+ * AuditReplayMapper replays the given audit trace against the NameNode under
+ * test. Each mapper spawns a number of threads equal to the
+ * {@value NUM_THREADS_KEY} configuration (by default
+ * {@value NUM_THREADS_DEFAULT}) to use for replaying. Each mapper reads a
+ * single input file which will be consumed by all of the available threads. A
+ * {@link FileInputFormat} with splitting disabled is used so any files present
+ * in the input path directory (given by the {@value INPUT_PATH_KEY}
+ * configuration) will be used as input; one file per mapper. The expected
+ * format of these files is determined by the value of the
+ * {@value COMMAND_PARSER_KEY} configuration, which defaults to
+ * {@link AuditLogDirectParser}.
+ *
+ * <p/>This generates a number of {@link org.apache.hadoop.mapreduce.Counter}
+ * values which can be used to get information into the replay, including the
+ * number of commands replayed, how many of them were "invalid" (threw an
+ * exception), how many were "late" (replayed later than they should have been),
+ * and the latency (from client perspective) of each command. If there are a
+ * large number of "late" commands, you likely need to increase the number of
+ * threads used and/or the number of mappers.
+ *
+ * <p/>By default, commands will be replayed at the same rate as they were
+ * originally performed. However a rate factor can be specified via the
+ * {@value RATE_FACTOR_KEY} configuration; all of the (relative) timestamps will
+ * be divided by this rate factor, effectively changing the rate at which they
+ * are replayed. For example, a rate factor of 2 would make the replay occur
+ * twice as fast, and a rate factor of 0.5 would make it occur half as fast.
+ */
+public class AuditReplayMapper extends WorkloadMapper<LongWritable, Text> {
+
+  public static final String INPUT_PATH_KEY = "auditreplay.input-path";
+  public static final String NUM_THREADS_KEY = "auditreplay.num-threads";
+  public static final int NUM_THREADS_DEFAULT = 1;
+  public static final String CREATE_BLOCKS_KEY = "auditreplay.create-blocks";
+  public static final boolean CREATE_BLOCKS_DEFAULT = true;
+  public static final String RATE_FACTOR_KEY = "auditreplay.rate-factor";
+  public static final double RATE_FACTOR_DEFAULT = 1.0;
+  public static final String COMMAND_PARSER_KEY =
+      "auditreplay.command-parser.class";
+  public static final Class<AuditLogDirectParser> COMMAND_PARSER_DEFAULT =
+      AuditLogDirectParser.class;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AuditReplayMapper.class);
+  // This is the maximum amount that the mapper should read ahead from the input
+  // as compared to the replay time. Setting this to one minute avoids reading
+  // too
+  // many entries into memory simultaneously but ensures that the replay threads
+  // should not ever run out of entries to replay.
+  private static final long MAX_READAHEAD_MS = 60000;
+
+  public static final String INDIVIDUAL_COMMANDS_COUNTER_GROUP =
+      "INDIVIDUAL_COMMANDS";
+  public static final String INDIVIDUAL_COMMANDS_LATENCY_SUFFIX = "_LATENCY";
+  public static final String INDIVIDUAL_COMMANDS_INVALID_SUFFIX = "_INVALID";
+  public static final String INDIVIDUAL_COMMANDS_COUNT_SUFFIX = "_COUNT";
+
+  /** {@link org.apache.hadoop.mapreduce.Counter} definitions for replay. */
+  public enum REPLAYCOUNTERS {
+    // Total number of commands that were replayed
+    TOTALCOMMANDS,
+    // Total number of commands that returned an error during replay (incl
+    // unsupported)
+    TOTALINVALIDCOMMANDS,
+    // Total number of commands that are unsupported for replay
+    TOTALUNSUPPORTEDCOMMANDS,
+    // Total number of commands that were performed later than they should have
+    // been
+    LATECOMMANDS,
+    // Total delay time of all commands that were performed later than they
+    // should have been
+    LATECOMMANDSTOTALTIME,
+    // Total number of write operations
+    TOTALWRITECOMMANDS,
+    // Total latency for all write operations
+    TOTALWRITECOMMANDLATENCY,
+    // Total number of read operations
+    TOTALREADCOMMANDS,
+    // Total latency for all read operations
+    TOTALREADCOMMANDLATENCY
+  }
+
+  /** Definitions of the various commands that can be replayed. */
+  public enum ReplayCommand {
+    APPEND(WRITE),
+    CREATE(WRITE),
+    GETFILEINFO(READ),
+    CONTENTSUMMARY(READ),
+    MKDIRS(WRITE),
+    RENAME(WRITE),
+    LISTSTATUS(READ),
+    DELETE(WRITE),
+    OPEN(READ),
+    SETPERMISSION(WRITE),
+    SETOWNER(WRITE),
+    SETTIMES(WRITE),
+    SETREPLICATION(WRITE),
+    CONCAT(WRITE);
+
+    private final CommandType type;
+
+    ReplayCommand(CommandType type) {
+      this.type = type;
+    }
+
+    public CommandType getType() {
+      return type;
+    }
+  }
+
+  /** Define the type of command, either read or write. */
+  public enum CommandType {
+    READ, WRITE
+  }
+
+  private long startTimestampMs;
+  private int numThreads;
+  private double rateFactor;
+  private long highestTimestamp;
+  private List<AuditReplayThread> threads;
+  private DelayQueue<AuditReplayCommand> commandQueue;
+  private Function<Long, Long> relativeToAbsoluteTimestamp;
+  private AuditCommandParser commandParser;
+  private ScheduledThreadPoolExecutor progressExecutor;
+
+  @Override
+  public Class<? extends InputFormat> getInputFormat(Configuration conf) {
+    return NoSplitTextInputFormat.class;
+  }
+
+  @Override
+  public String getDescription() {
+    return "This mapper replays audit log files.";
+  }
+
+  @Override
+  public List<String> getConfigDescriptions() {
+    return Lists.newArrayList(
+        INPUT_PATH_KEY
+            + " (required): Path to directory containing input files.",
+        NUM_THREADS_KEY + " (default " + NUM_THREADS_DEFAULT
+            + "): Number of threads to use per mapper for replay.",
+        CREATE_BLOCKS_KEY + " (default " + CREATE_BLOCKS_DEFAULT
+            + "): Whether or not to create 1-byte blocks when "
+            + "performing `create` commands.",
+        RATE_FACTOR_KEY + " (default " + RATE_FACTOR_DEFAULT
+            + "): Multiplicative speed at which to replay the audit "
+            + "log; e.g. a value of 2.0 would make the replay occur at "
+            + "twice the original speed. This can be useful "
+            + "to induce heavier loads.");
+  }
+
+  @Override
+  public boolean verifyConfigurations(Configuration conf) {
+    return conf.get(INPUT_PATH_KEY) != null;
+  }
+
+  @Override
+  public void setup(final Mapper.Context context) throws IOException {
+    Configuration conf = context.getConfiguration();
+    // WorkloadDriver ensures that the starttimestamp is set
+    startTimestampMs = conf.getLong(WorkloadDriver.START_TIMESTAMP_MS, -1);
+    numThreads = conf.getInt(NUM_THREADS_KEY, NUM_THREADS_DEFAULT);
+    rateFactor = conf.getDouble(RATE_FACTOR_KEY, RATE_FACTOR_DEFAULT);
+    try {
+      commandParser = conf.getClass(COMMAND_PARSER_KEY, COMMAND_PARSER_DEFAULT,
+          AuditCommandParser.class).getConstructor().newInstance();
+    } catch (NoSuchMethodException | InstantiationException
+        | IllegalAccessException | InvocationTargetException e) {
+      throw new IOException(
+          "Exception encountered while instantiating the command parser", e);
+    }
+    commandParser.initialize(conf);
+    relativeToAbsoluteTimestamp =
+        (input) -> startTimestampMs + Math.round(input / rateFactor);
+
+    LOG.info("Starting " + numThreads + " threads");
+
+    progressExecutor = new ScheduledThreadPoolExecutor(1);
+    // half of the timeout or once per minute if none specified
+    long progressFrequencyMs = conf.getLong(MRJobConfig.TASK_TIMEOUT,
+        2 * 60 * 1000) / 2;
+    progressExecutor.scheduleAtFixedRate(context::progress,
+        progressFrequencyMs, progressFrequencyMs, TimeUnit.MILLISECONDS);
+
+    threads = new ArrayList<>();
+    ConcurrentMap<String, FileSystem> fsCache = new ConcurrentHashMap<>();
+    commandQueue = new DelayQueue<>();
+    for (int i = 0; i < numThreads; i++) {
+      AuditReplayThread thread = new AuditReplayThread(context, commandQueue,
+          fsCache);
+      threads.add(thread);
+      thread.start();
+    }
+  }
+
+  @Override
+  public void map(LongWritable lineNum, Text inputLine, Mapper.Context context)
+      throws IOException, InterruptedException {
+    AuditReplayCommand cmd = commandParser.parse(inputLine,
+        relativeToAbsoluteTimestamp);
+    long delay = cmd.getDelay(TimeUnit.MILLISECONDS);
+    // Prevent from loading too many elements into memory all at once
+    if (delay > MAX_READAHEAD_MS) {
+      Thread.sleep(delay - (MAX_READAHEAD_MS / 2));
+    }
+    commandQueue.put(cmd);
+    highestTimestamp = cmd.getAbsoluteTimestamp();
+  }
+
+  @Override
+  public void cleanup(Mapper.Context context) throws InterruptedException {
+    for (AuditReplayThread t : threads) {
+      // Add in an indicator for each thread to shut down after the last real
+      // command
+      t.addToQueue(AuditReplayCommand.getPoisonPill(highestTimestamp + 1));
+    }
+    Optional<Exception> threadException = Optional.empty();
+    for (AuditReplayThread t : threads) {
+      t.join();
+      t.drainCounters(context);
+      if (t.getException() != null) {
+        threadException = Optional.of(t.getException());
+      }
+    }
+    progressExecutor.shutdown();
+
+    if (threadException.isPresent()) {
+      throw new RuntimeException("Exception in AuditReplayThread",
+          threadException.get());
+    }
+    LOG.info("Time taken to replay the logs in ms: "
+        + (System.currentTimeMillis() - startTimestampMs));
+    long totalCommands = context.getCounter(REPLAYCOUNTERS.TOTALCOMMANDS)
+        .getValue();
+    if (totalCommands != 0) {
+      double percentageOfInvalidOps =
+          context.getCounter(REPLAYCOUNTERS.TOTALINVALIDCOMMANDS).getValue()
+              * 100.0 / totalCommands;
+      LOG.info("Percentage of invalid ops: " + percentageOfInvalidOps);
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditReplayThread.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditReplayThread.java
new file mode 100644
index 0000000..e63c7a3
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/AuditReplayThread.java
@@ -0,0 +1,310 @@
+/**
+ * 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.hadoop.tools.dynamometer.workloadgenerator.audit;
+
+import com.google.common.base.Splitter;
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.WorkloadDriver;
+import java.io.IOException;
+import java.net.URI;
+import java.security.PrivilegedAction;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.counters.GenericCounter;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditReplayMapper.REPLAYCOUNTERS;
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditReplayMapper.ReplayCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditReplayMapper.INDIVIDUAL_COMMANDS_COUNTER_GROUP;
+import static org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditReplayMapper.INDIVIDUAL_COMMANDS_COUNT_SUFFIX;
+import static org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditReplayMapper.INDIVIDUAL_COMMANDS_INVALID_SUFFIX;
+import static org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditReplayMapper.INDIVIDUAL_COMMANDS_LATENCY_SUFFIX;
+
+/**
+ * This class replays each audit log entry at a specified timestamp in the
+ * future. Each of these threads maintains a {@link DelayQueue} into which items
+ * are inserted by the {@link AuditReplayMapper}. Once an item is ready, this
+ * thread will fetch the command from the queue and attempt to replay it.
+ */
+public class AuditReplayThread extends Thread {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AuditReplayThread.class);
+
+  private DelayQueue<AuditReplayCommand> commandQueue;
+  private ConcurrentMap<String, FileSystem> fsCache;
+  private URI namenodeUri;
+  private UserGroupInformation loginUser;
+  private Configuration mapperConf;
+  // If any exception is encountered it will be stored here
+  private Exception exception;
+  private long startTimestampMs;
+  private boolean createBlocks;
+
+  // Counters are not thread-safe so we store a local mapping in our thread
+  // and merge them all together at the end.
+  private Map<REPLAYCOUNTERS, Counter> replayCountersMap = new HashMap<>();
+  private Map<String, Counter> individualCommandsMap = new HashMap<>();
+
+  AuditReplayThread(Mapper.Context mapperContext,
+      DelayQueue<AuditReplayCommand> queue,
+      ConcurrentMap<String, FileSystem> fsCache) throws IOException {
+    commandQueue = queue;
+    this.fsCache = fsCache;
+    loginUser = UserGroupInformation.getLoginUser();
+    mapperConf = mapperContext.getConfiguration();
+    namenodeUri = URI.create(mapperConf.get(WorkloadDriver.NN_URI));
+    startTimestampMs = mapperConf.getLong(WorkloadDriver.START_TIMESTAMP_MS,
+        -1);
+    createBlocks = mapperConf.getBoolean(AuditReplayMapper.CREATE_BLOCKS_KEY,
+        AuditReplayMapper.CREATE_BLOCKS_DEFAULT);
+    LOG.info("Start timestamp: " + startTimestampMs);
+    for (REPLAYCOUNTERS rc : REPLAYCOUNTERS.values()) {
+      replayCountersMap.put(rc, new GenericCounter());
+    }
+    for (ReplayCommand replayCommand : ReplayCommand.values()) {
+      individualCommandsMap.put(
+          replayCommand + INDIVIDUAL_COMMANDS_COUNT_SUFFIX,
+          new GenericCounter());
+      individualCommandsMap.put(
+          replayCommand + INDIVIDUAL_COMMANDS_LATENCY_SUFFIX,
+          new GenericCounter());
+      individualCommandsMap.put(
+          replayCommand + INDIVIDUAL_COMMANDS_INVALID_SUFFIX,
+          new GenericCounter());
+    }
+  }
+
+  /**
+   * Merge all of this thread's counter values into the counters contained
+   * within the passed context.
+   *
+   * @param context The context holding the counters to increment.
+   */
+  void drainCounters(Mapper.Context context) {
+    for (Map.Entry<REPLAYCOUNTERS, Counter> ent : replayCountersMap
+        .entrySet()) {
+      context.getCounter(ent.getKey()).increment(ent.getValue().getValue());
+    }
+    for (Map.Entry<String, Counter> ent : individualCommandsMap.entrySet()) {
+      context.getCounter(INDIVIDUAL_COMMANDS_COUNTER_GROUP, ent.getKey())
+          .increment(ent.getValue().getValue());
+    }
+  }
+
+  /**
+   * Add a command to this thread's processing queue.
+   *
+   * @param cmd Command to add.
+   */
+  void addToQueue(AuditReplayCommand cmd) {
+    commandQueue.put(cmd);
+  }
+
+  /**
+   * Get the Exception that caused this thread to stop running, if any, else
+   * null. Should not be called until this thread has already completed (i.e.,
+   * after {@link #join()} has been called).
+   *
+   * @return The exception which was thrown, if any.
+   */
+  Exception getException() {
+    return exception;
+  }
+
+  @Override
+  public void run() {
+    long currentEpoch = System.currentTimeMillis();
+    long delay = startTimestampMs - currentEpoch;
+    try {
+      if (delay > 0) {
+        LOG.info("Sleeping for " + delay + " ms");
+        Thread.sleep(delay);
+      } else {
+        LOG.warn("Starting late by " + (-1 * delay) + " ms");
+      }
+
+      AuditReplayCommand cmd = commandQueue.take();
+      while (!cmd.isPoison()) {
+        replayCountersMap.get(REPLAYCOUNTERS.TOTALCOMMANDS).increment(1);
+        delay = cmd.getDelay(TimeUnit.MILLISECONDS);
+        if (delay < -5) { // allow some tolerance here
+          replayCountersMap.get(REPLAYCOUNTERS.LATECOMMANDS).increment(1);
+          replayCountersMap.get(REPLAYCOUNTERS.LATECOMMANDSTOTALTIME)
+              .increment(-1 * delay);
+        }
+        if (!replayLog(cmd)) {
+          replayCountersMap.get(REPLAYCOUNTERS.TOTALINVALIDCOMMANDS)
+              .increment(1);
+        }
+        cmd = commandQueue.take();
+      }
+    } catch (InterruptedException e) {
+      LOG.error("Interrupted; exiting from thread.", e);
+    } catch (Exception e) {
+      exception = e;
+      LOG.error("ReplayThread encountered exception; exiting.", e);
+    }
+  }
+
+  /**
+   * Attempt to replay the provided command. Updates counters accordingly.
+   *
+   * @param command The command to replay
+   * @return True iff the command was successfully replayed (i.e., no exceptions
+   *         were thrown).
+   */
+  private boolean replayLog(final AuditReplayCommand command) {
+    final String src = command.getSrc();
+    final String dst = command.getDest();
+    FileSystem proxyFs = fsCache.get(command.getSimpleUgi());
+    if (proxyFs == null) {
+      UserGroupInformation ugi = UserGroupInformation
+          .createProxyUser(command.getSimpleUgi(), loginUser);
+      proxyFs = ugi.doAs((PrivilegedAction<FileSystem>) () -> {
+        try {
+          FileSystem fs = new DistributedFileSystem();
+          fs.initialize(namenodeUri, mapperConf);
+          return fs;
+        } catch (IOException ioe) {
+          throw new RuntimeException(ioe);
+        }
+      });
+      fsCache.put(command.getSimpleUgi(), proxyFs);
+    }
+    final FileSystem fs = proxyFs;
+    ReplayCommand replayCommand;
+    try {
+      replayCommand = ReplayCommand
+          .valueOf(command.getCommand().split(" ")[0].toUpperCase());
+    } catch (IllegalArgumentException iae) {
+      LOG.warn("Unsupported/invalid command: " + command);
+      replayCountersMap.get(REPLAYCOUNTERS.TOTALUNSUPPORTEDCOMMANDS)
+          .increment(1);
+      return false;
+    }
+    try {
+      long startTime = System.currentTimeMillis();
+      switch (replayCommand) {
+      case CREATE:
+        FSDataOutputStream fsDos = fs.create(new Path(src));
+        if (createBlocks) {
+          fsDos.writeByte(0);
+        }
+        fsDos.close();
+        break;
+      case GETFILEINFO:
+        fs.getFileStatus(new Path(src));
+        break;
+      case CONTENTSUMMARY:
+        fs.getContentSummary(new Path(src));
+        break;
+      case MKDIRS:
+        fs.mkdirs(new Path(src));
+        break;
+      case RENAME:
+        fs.rename(new Path(src), new Path(dst));
+        break;
+      case LISTSTATUS:
+        ((DistributedFileSystem) fs).getClient().listPaths(src,
+            HdfsFileStatus.EMPTY_NAME);
+        break;
+      case APPEND:
+        fs.append(new Path(src));
+        return true;
+      case DELETE:
+        fs.delete(new Path(src), true);
+        break;
+      case OPEN:
+        fs.open(new Path(src)).close();
+        break;
+      case SETPERMISSION:
+        fs.setPermission(new Path(src), FsPermission.getDefault());
+        break;
+      case SETOWNER:
+        fs.setOwner(new Path(src),
+            UserGroupInformation.getCurrentUser().getShortUserName(),
+            UserGroupInformation.getCurrentUser().getPrimaryGroupName());
+        break;
+      case SETTIMES:
+        fs.setTimes(new Path(src), System.currentTimeMillis(),
+            System.currentTimeMillis());
+        break;
+      case SETREPLICATION:
+        fs.setReplication(new Path(src), (short) 1);
+        break;
+      case CONCAT:
+        // dst is like [path1, path2] - strip brackets and split on comma
+        String bareDist = dst.length() < 2 ? ""
+            : dst.substring(1, dst.length() - 1).trim();
+        List<Path> dsts = new ArrayList<>();
+        for (String s : Splitter.on(",").omitEmptyStrings().trimResults()
+            .split(bareDist)) {
+          dsts.add(new Path(s));
+        }
+        fs.concat(new Path(src), dsts.toArray(new Path[] {}));
+        break;
+      default:
+        throw new RuntimeException("Unexpected command: " + replayCommand);
+      }
+      long latency = System.currentTimeMillis() - startTime;
+      switch (replayCommand.getType()) {
+      case WRITE:
+        replayCountersMap.get(REPLAYCOUNTERS.TOTALWRITECOMMANDLATENCY)
+            .increment(latency);
+        replayCountersMap.get(REPLAYCOUNTERS.TOTALWRITECOMMANDS).increment(1);
+        break;
+      case READ:
+        replayCountersMap.get(REPLAYCOUNTERS.TOTALREADCOMMANDLATENCY)
+            .increment(latency);
+        replayCountersMap.get(REPLAYCOUNTERS.TOTALREADCOMMANDS).increment(1);
+        break;
+      default:
+        throw new RuntimeException("Unexpected command type: "
+            + replayCommand.getType());
+      }
+      individualCommandsMap
+          .get(replayCommand + INDIVIDUAL_COMMANDS_LATENCY_SUFFIX)
+          .increment(latency);
+      individualCommandsMap
+          .get(replayCommand + INDIVIDUAL_COMMANDS_COUNT_SUFFIX).increment(1);
+      return true;
+    } catch (IOException e) {
+      LOG.debug("IOException: " + e.getLocalizedMessage());
+      individualCommandsMap
+          .get(replayCommand + INDIVIDUAL_COMMANDS_INVALID_SUFFIX).increment(1);
+      return false;
+    }
+  }
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/NoSplitTextInputFormat.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/NoSplitTextInputFormat.java
new file mode 100644
index 0000000..e941d9d
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/NoSplitTextInputFormat.java
@@ -0,0 +1,47 @@
+/**
+ * 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.hadoop.tools.dynamometer.workloadgenerator.audit;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+
+/**
+ * A simple {@link TextInputFormat} that disables splitting of files. This is
+ * the {@link org.apache.hadoop.mapreduce.InputFormat} used by
+ * {@link AuditReplayMapper}.
+ */
+public class NoSplitTextInputFormat extends TextInputFormat {
+
+  @Override
+  public List<FileStatus> listStatus(JobContext context) throws IOException {
+    context.getConfiguration().set(FileInputFormat.INPUT_DIR,
+        context.getConfiguration().get(AuditReplayMapper.INPUT_PATH_KEY));
+    return super.listStatus(context);
+  }
+
+  @Override
+  public boolean isSplitable(JobContext context, Path file) {
+    return false;
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/package-info.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/package-info.java
new file mode 100644
index 0000000..745302a
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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 contains the tooling necessary to use audit logs as a way to generate
+ * a realistic workload against a Dynamometer-simulated NameNode.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.tools.dynamometer.workloadgenerator.audit;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/package-info.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/package-info.java
new file mode 100644
index 0000000..5903a85
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/main/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/package-info.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.
+ */
+
+/**
+ * This contains the tooling necessary to generate a realistic workload,
+ * for example collected from audit logs, against a Dynamometer-simulated
+ * NameNode.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.tools.dynamometer.workloadgenerator;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/test/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/TestWorkloadGenerator.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/test/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/TestWorkloadGenerator.java
new file mode 100644
index 0000000..5b2a2e7
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/test/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/TestWorkloadGenerator.java
@@ -0,0 +1,136 @@
+/**
+ * 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.hadoop.tools.dynamometer.workloadgenerator;
+
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditCommandParser;
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditLogDirectParser;
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditLogHiveTableParser;
+import org.apache.hadoop.tools.dynamometer.workloadgenerator.audit.AuditReplayMapper;
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.authorize.ImpersonationProvider;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_IMPERSONATION_PROVIDER_CLASS;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+
+/** Tests for {@link WorkloadDriver} and related classes. */
+public class TestWorkloadGenerator {
+
+  private Configuration conf;
+  private MiniDFSCluster miniCluster;
+  private FileSystem dfs;
+
+  @Before
+  public void setup() throws Exception {
+    conf = new Configuration();
+    conf.setClass(HADOOP_SECURITY_IMPERSONATION_PROVIDER_CLASS,
+        AllowUserImpersonationProvider.class, ImpersonationProvider.class);
+    miniCluster = new MiniDFSCluster.Builder(conf).build();
+    miniCluster.waitClusterUp();
+    dfs = miniCluster.getFileSystem();
+    dfs.mkdirs(new Path("/tmp"),
+        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
+    dfs.setOwner(new Path("/tmp"), "hdfs", "hdfs");
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (miniCluster != null) {
+      miniCluster.shutdown();
+      miniCluster = null;
+    }
+  }
+
+  @Test
+  public void testAuditWorkloadDirectParser() throws Exception {
+    String workloadInputPath = TestWorkloadGenerator.class.getClassLoader()
+        .getResource("audit_trace_direct").toString();
+    conf.set(AuditReplayMapper.INPUT_PATH_KEY, workloadInputPath);
+    conf.setLong(AuditLogDirectParser.AUDIT_START_TIMESTAMP_KEY, 60 * 1000);
+    testAuditWorkload();
+  }
+
+  @Test
+  public void testAuditWorkloadHiveParser() throws Exception {
+    String workloadInputPath = TestWorkloadGenerator.class.getClassLoader()
+        .getResource("audit_trace_hive").toString();
+    conf.set(AuditReplayMapper.INPUT_PATH_KEY, workloadInputPath);
+    conf.setClass(AuditReplayMapper.COMMAND_PARSER_KEY,
+        AuditLogHiveTableParser.class, AuditCommandParser.class);
+    testAuditWorkload();
+  }
+
+  /**
+   * {@link ImpersonationProvider} that confirms the user doing the
+   * impersonating is the same as the user running the MiniCluster.
+   */
+  private static class AllowUserImpersonationProvider extends Configured
+      implements ImpersonationProvider {
+    public void init(String configurationPrefix) {
+      // Do nothing
+    }
+
+    public void authorize(UserGroupInformation user, String remoteAddress)
+        throws AuthorizationException {
+      try {
+        if (!user.getRealUser().getShortUserName()
+            .equals(UserGroupInformation.getCurrentUser().getShortUserName())) {
+          throw new AuthorizationException();
+        }
+      } catch (IOException ioe) {
+        throw new AuthorizationException(ioe);
+      }
+    }
+  }
+
+  private void testAuditWorkload() throws Exception {
+    long workloadStartTime = System.currentTimeMillis() + 10000;
+    Job workloadJob = WorkloadDriver.getJobForSubmission(conf,
+        dfs.getUri().toString(), workloadStartTime, AuditReplayMapper.class);
+    boolean success = workloadJob.waitForCompletion(true);
+    assertTrue("workload job should succeed", success);
+    Counters counters = workloadJob.getCounters();
+    assertEquals(6,
+        counters.findCounter(AuditReplayMapper.REPLAYCOUNTERS.TOTALCOMMANDS)
+            .getValue());
+    assertEquals(1,
+        counters
+            .findCounter(AuditReplayMapper.REPLAYCOUNTERS.TOTALINVALIDCOMMANDS)
+            .getValue());
+    assertTrue(dfs.getFileStatus(new Path("/tmp/test1")).isFile());
+    assertTrue(
+        dfs.getFileStatus(new Path("/tmp/testDirRenamed")).isDirectory());
+    assertFalse(dfs.exists(new Path("/denied")));
+  }
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/test/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/TestAuditLogDirectParser.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/test/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/TestAuditLogDirectParser.java
new file mode 100644
index 0000000..f3a6eed
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/test/java/org/apache/hadoop/tools/dynamometer/workloadgenerator/audit/TestAuditLogDirectParser.java
@@ -0,0 +1,143 @@
+/**
+ * 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.hadoop.tools.dynamometer.workloadgenerator.audit;
+
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/** Tests for {@link AuditLogDirectParser}. */
+public class TestAuditLogDirectParser {
+
+  private static final long START_TIMESTAMP = 10000;
+  private AuditLogDirectParser parser;
+
+  @Before
+  public void setup() throws Exception {
+    parser = new AuditLogDirectParser();
+    Configuration conf = new Configuration();
+    conf.setLong(AuditLogDirectParser.AUDIT_START_TIMESTAMP_KEY,
+        START_TIMESTAMP);
+    parser.initialize(conf);
+  }
+
+  private Text getAuditString(String timestamp, String ugi, String cmd,
+      String src, String dst) {
+    return new Text(
+        String.format("%s INFO FSNamesystem.audit: "
+                + "allowed=true\tugi=%s\tip=0.0.0.0\tcmd=%s\tsrc=%s\t"
+                + "dst=%s\tperm=null\tproto=rpc",
+            timestamp, ugi, cmd, src, dst));
+  }
+
+  @Test
+  public void testSimpleInput() throws Exception {
+    Text in = getAuditString("1970-01-01 00:00:11,000", "fakeUser",
+        "listStatus", "sourcePath", "null");
+    AuditReplayCommand expected = new AuditReplayCommand(1000, "fakeUser",
+        "listStatus", "sourcePath", "null", "0.0.0.0");
+    assertEquals(expected, parser.parse(in, Function.identity()));
+  }
+
+  @Test
+  public void testInputWithRenameOptions() throws Exception {
+    Text in = getAuditString("1970-01-01 00:00:11,000", "fakeUser",
+        "rename (options=[TO_TRASH])", "sourcePath", "destPath");
+    AuditReplayCommand expected = new AuditReplayCommand(1000, "fakeUser",
+        "rename (options=[TO_TRASH])", "sourcePath", "destPath", "0.0.0.0");
+    assertEquals(expected, parser.parse(in, Function.identity()));
+  }
+
+  @Test
+  public void testInputWithTokenAuth() throws Exception {
+    Text in = getAuditString("1970-01-01 00:00:11,000", "fakeUser (auth:TOKEN)",
+        "create", "sourcePath", "null");
+    AuditReplayCommand expected = new AuditReplayCommand(1000, "fakeUser",
+        "create", "sourcePath", "null", "0.0.0.0");
+    assertEquals(expected, parser.parse(in, Function.identity()));
+  }
+
+  @Test
+  public void testInputWithProxyUser() throws Exception {
+    Text in = getAuditString("1970-01-01 00:00:11,000",
+        "proxyUser (auth:TOKEN) via fakeUser", "create", "sourcePath", "null");
+    AuditReplayCommand expected = new AuditReplayCommand(1000, "proxyUser",
+        "create", "sourcePath", "null", "0.0.0.0");
+    assertEquals(expected, parser.parse(in, Function.identity()));
+  }
+
+  @Test
+  public void testParseDefaultDateFormat() throws Exception {
+    Text in = getAuditString("1970-01-01 13:00:00,000",
+        "ignored", "ignored", "ignored", "ignored");
+    AuditReplayCommand expected = new AuditReplayCommand(
+        13 * 60 * 60 * 1000 - START_TIMESTAMP,
+        "ignored", "ignored", "ignored", "ignored", "0.0.0.0");
+    assertEquals(expected, parser.parse(in, Function.identity()));
+  }
+
+  @Test
+  public void testParseCustomDateFormat() throws Exception {
+    parser = new AuditLogDirectParser();
+    Configuration conf = new Configuration();
+    conf.setLong(AuditLogDirectParser.AUDIT_START_TIMESTAMP_KEY, 0);
+    conf.set(AuditLogDirectParser.AUDIT_LOG_DATE_FORMAT_KEY,
+        "yyyy-MM-dd hh:mm:ss,SSS a");
+    parser.initialize(conf);
+    Text in = getAuditString("1970-01-01 01:00:00,000 PM",
+        "ignored", "ignored", "ignored", "ignored");
+    AuditReplayCommand expected = new AuditReplayCommand(13 * 60 * 60 * 1000,
+        "ignored", "ignored", "ignored", "ignored", "0.0.0.0");
+    assertEquals(expected, parser.parse(in, Function.identity()));
+  }
+
+  @Test
+  public void testParseCustomTimeZone() throws Exception {
+    parser = new AuditLogDirectParser();
+    Configuration conf = new Configuration();
+    conf.setLong(AuditLogDirectParser.AUDIT_START_TIMESTAMP_KEY, 0);
+    conf.set(AuditLogDirectParser.AUDIT_LOG_DATE_TIME_ZONE_KEY, "Etc/GMT-1");
+    parser.initialize(conf);
+    Text in = getAuditString("1970-01-01 01:00:00,000",
+        "ignored", "ignored", "ignored", "ignored");
+    AuditReplayCommand expected = new AuditReplayCommand(0,
+        "ignored", "ignored", "ignored", "ignored", "0.0.0.0");
+    assertEquals(expected, parser.parse(in, Function.identity()));
+  }
+
+  @Test
+  public void testParseCustomAuditLineFormat() throws Exception {
+    Text auditLine = new Text("CUSTOM FORMAT (1970-01-01 00:00:00,000) "
+        + "allowed=true\tugi=fakeUser\tip=0.0.0.0\tcmd=fakeCommand\tsrc=src\t"
+        + "dst=null\tperm=null\tproto=rpc");
+    parser = new AuditLogDirectParser();
+    Configuration conf = new Configuration();
+    conf.setLong(AuditLogDirectParser.AUDIT_START_TIMESTAMP_KEY, 0);
+    conf.set(AuditLogDirectParser.AUDIT_LOG_PARSE_REGEX_KEY,
+        "CUSTOM FORMAT \\((?<timestamp>.+?)\\) (?<message>.+)");
+    parser.initialize(conf);
+    AuditReplayCommand expected = new AuditReplayCommand(0,
+        "fakeUser", "fakeCommand", "src", "null", "0.0.0.0");
+    assertEquals(expected, parser.parse(auditLine, Function.identity()));
+  }
+
+}
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/test/resources/audit_trace_direct/audit0 b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/test/resources/audit_trace_direct/audit0
new file mode 100644
index 0000000..9e7bc2c
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/test/resources/audit_trace_direct/audit0
@@ -0,0 +1,6 @@
+1970-01-01 00:00:01,010 INFO FSNamesystem.audit: allowed=true	ugi=hdfs	ip=/0.0.0.0	cmd=create	src=/tmp/test1	dst=null	perm=null	proto=rpc
+1970-01-01 00:00:01,020 INFO FSNamesystem.audit: allowed=true	ugi=hdfs	ip=/0.0.0.0	cmd=mkdirs	src=/tmp/testDir	dst=null	perm=null	proto=rpc
+1970-01-01 00:00:01,030 INFO FSNamesystem.audit: allowed=true	ugi=hdfs	ip=/0.0.0.0	cmd=rename	src=/tmp/testDir	dst=/tmp/testDirRenamed	perm=null	proto=rpc
+1970-01-01 00:00:01,040 INFO FSNamesystem.audit: allowed=true	ugi=hdfs@REALM.COM	ip=/0.0.0.0	cmd=mkdirs	src=/tmp/testDir2	dst=null	perm=null	proto=rpc
+1970-01-01 00:00:01,050 INFO FSNamesystem.audit: allowed=true	ugi=hdfs/127.0.0.1@REALM.COM	ip=/0.0.0.0	cmd=mkdirs	src=/tmp/testDir3	dst=null	perm=null	proto=rpc
+1970-01-01 00:00:01,060 INFO FSNamesystem.audit: allowed=true	ugi=otherUser	ip=/0.0.0.0	cmd=mkdirs	src=/denied	dst=null	perm=null	proto=rpc
diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/test/resources/audit_trace_hive/audit0 b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/test/resources/audit_trace_hive/audit0
new file mode 100644
index 0000000..5f49bbe
--- /dev/null
+++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-workload/src/test/resources/audit_trace_hive/audit0
@@ -0,0 +1,6 @@
++10hdfscreate/tmp/test1 0.0.0.0
++20hdfsmkdirs/tmp/testDir 0.0.0.0
++30hdfsrename/tmp/testDir/tmp/testDirRenamed0.0.0.0
++40hdfs@REALM.COMmkdirs/tmp/testDir2 0.0.0.0
++50hdfs/127.0.0.1@REALM.COMmkdirs/tmp/testDir3 0.0.0.0
++60otherUsermkdirs/denied 0.0.0.0
\ No newline at end of file
diff --git a/hadoop-tools/pom.xml b/hadoop-tools/hadoop-dynamometer/pom.xml
similarity index 60%
copy from hadoop-tools/pom.xml
copy to hadoop-tools/hadoop-dynamometer/pom.xml
index af1977a..dd96039 100644
--- a/hadoop-tools/pom.xml
+++ b/hadoop-tools/hadoop-dynamometer/pom.xml
@@ -15,40 +15,25 @@
 <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
-                      https://maven.apache.org/xsd/maven-4.0.0.xsd">
+                      http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
     <version>3.3.0-SNAPSHOT</version>
-    <relativePath>../hadoop-project</relativePath>
+    <relativePath>../../hadoop-project</relativePath>
   </parent>
-  <artifactId>hadoop-tools</artifactId>
+  <artifactId>hadoop-dynamometer</artifactId>
   <version>3.3.0-SNAPSHOT</version>
-  <description>Apache Hadoop Tools</description>
-  <name>Apache Hadoop Tools</name>
+  <description>Apache Hadoop Dynamometer</description>
+  <name>Apache Hadoop Dynamometer</name>
   <packaging>pom</packaging>
 
   <modules>
-    <module>hadoop-streaming</module>
-    <module>hadoop-distcp</module>
-    <module>hadoop-archives</module>
-    <module>hadoop-archive-logs</module>
-    <module>hadoop-rumen</module>
-    <module>hadoop-gridmix</module>
-    <module>hadoop-datajoin</module>
-    <module>hadoop-tools-dist</module>
-    <module>hadoop-extras</module>
-    <module>hadoop-pipes</module>
-    <module>hadoop-openstack</module>
-    <module>hadoop-sls</module>
-    <module>hadoop-resourceestimator</module>
-    <module>hadoop-azure</module>
-    <module>hadoop-aws</module>
-    <module>hadoop-kafka</module>
-    <module>hadoop-azure-datalake</module>
-    <module>hadoop-aliyun</module>
-    <module>hadoop-fs2img</module>
+    <module>hadoop-dynamometer-infra</module>
+    <module>hadoop-dynamometer-workload</module>
+    <module>hadoop-dynamometer-blockgen</module>
+    <module>hadoop-dynamometer-dist</module>
   </modules>
 
   <build>
diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml
index af1977a..31b26ac 100644
--- a/hadoop-tools/pom.xml
+++ b/hadoop-tools/pom.xml
@@ -32,6 +32,7 @@
   <modules>
     <module>hadoop-streaming</module>
     <module>hadoop-distcp</module>
+    <module>hadoop-dynamometer</module>
     <module>hadoop-archives</module>
     <module>hadoop-archive-logs</module>
     <module>hadoop-rumen</module>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org