You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by ni...@apache.org on 2022/10/10 18:45:11 UTC

[samza] branch master updated: SAMZA-2757: Make Samza Compatible with Java 11 (#1628)

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

nickpan47 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/samza.git


The following commit(s) were added to refs/heads/master by this push:
     new 5580bcfac SAMZA-2757: Make Samza Compatible with Java 11 (#1628)
5580bcfac is described below

commit 5580bcfac68768ea3e6c1871a03a50361df10889
Author: Jamie DeMichele <ja...@redfin.com>
AuthorDate: Mon Oct 10 13:45:05 2022 -0500

    SAMZA-2757: Make Samza Compatible with Java 11 (#1628)
    
    Feature: Make Samza build artifacts runnable in JDK11
    
    Changes:
       - introduced new build modules samza-yarn3 and samza-shell-yarn3 to build Samza w/ YARN 3.3.4, which is runtime compatible w/ JDK11
    
    
Tests:
       - local test and hello-samza succeeded with Java11
---
 README.md                                          |  21 +-
 build.gradle                                       | 138 ++++
 gradle.properties                                  |   5 +-
 gradle/dependency-versions.gradle                  |   1 +
 gradle/wrapper/gradle-wrapper.properties           |   2 +-
 .../apache/samza/system/hdfs/HdfsSystemAdmin.java  |   3 +-
 .../samza/system/hdfs/HdfsSystemConsumer.java      |   4 +-
 .../hdfs/partitioner/DirectoryPartitioner.java     |  19 +-
 .../org/apache/samza/config/KafkaConfig.scala      |  16 +-
 .../samza/monitor/LocalStoreMonitorConfig.java     |   3 +-
 .../org/apache/samza/monitor/MonitorConfig.java    |   3 +-
 .../apache/samza/rest/SamzaRestApplication.java    |   4 +-
 .../rest/proxy/task/SamzaTaskProxyFactory.java     |   3 +-
 .../apache/samza/rest/resources/TasksResource.java |   3 +-
 samza-shell/src/main/bash/run-class.sh             |  13 +-
 .../samza/job/yarn/LocalizerResourceConfig.java    |   3 +-
 .../org/apache/samza/job/yarn/ClientHelper.scala   |   5 +-
 .../java/org/apache/samza/config/YarnConfig.java   | 251 +++++++
 .../samza/job/yarn/FileSystemImplConfig.java       |  71 ++
 .../samza/job/yarn/LocalizerResourceConfig.java    |   3 +-
 .../samza/job/yarn/LocalizerResourceException.java |  72 ++
 .../samza/job/yarn/LocalizerResourceMapper.java    | 101 +++
 .../org/apache/samza/job/yarn/YarnAppState.java    | 138 ++++
 .../samza/job/yarn/YarnClusterResourceManager.java | 774 +++++++++++++++++++++
 .../samza/job/yarn/YarnFaultDomainManager.java     | 131 ++++
 .../job/yarn/YarnFaultDomainManagerFactory.java    |  34 +
 .../samza/job/yarn/YarnResourceManagerFactory.java |  44 ++
 .../samza/validation/YarnJobValidationTool.java    | 207 ++++++
 .../samza/webapp/ApplicationMasterRestClient.java  | 111 +++
 .../webapp/YarnContainerHeartbeatServlet.java      |  92 +++
 samza-yarn3/src/main/less/main.less                | 164 +++++
 samza-yarn3/src/main/less/variables.less           |  20 +
 .../scalate/WEB-INF/layouts/default.scaml          |  48 ++
 .../resources/scalate/WEB-INF/views/index.scaml    | 239 +++++++
 .../main/resources/scalate/css/bootstrap.min.css   |   7 +
 .../resources/scalate/css/font-awesome.min.css     |   4 +
 .../src/main/resources/scalate/css/ropa-sans.css   |   6 +
 .../main/resources/scalate/fonts/FontAwesome.otf   | Bin 0 -> 62856 bytes
 .../scalate/fonts/RopaSans-Regular-webfont.woff    | Bin 0 -> 23224 bytes
 .../scalate/fonts/fontawesome-webfont.eot          | Bin 0 -> 38205 bytes
 .../scalate/fonts/fontawesome-webfont.svg          | 414 +++++++++++
 .../scalate/fonts/fontawesome-webfont.ttf          | Bin 0 -> 80652 bytes
 .../scalate/fonts/fontawesome-webfont.woff         | Bin 0 -> 44432 bytes
 .../scalate/fonts/glyphicons-halflings-regular.eot | Bin 0 -> 20335 bytes
 .../scalate/fonts/glyphicons-halflings-regular.svg | 229 ++++++
 .../scalate/fonts/glyphicons-halflings-regular.ttf | Bin 0 -> 41280 bytes
 .../fonts/glyphicons-halflings-regular.woff        | Bin 0 -> 23320 bytes
 samza-yarn3/src/main/resources/scalate/img/asc.gif | Bin 0 -> 54 bytes
 samza-yarn3/src/main/resources/scalate/img/bg.gif  | Bin 0 -> 64 bytes
 .../src/main/resources/scalate/img/desc.gif        | Bin 0 -> 54 bytes
 .../src/main/resources/scalate/img/samza-icon.png  | Bin 0 -> 7722 bytes
 .../src/main/resources/scalate/js/bootstrap.min.js |   6 +
 .../main/resources/scalate/js/jquery-1.11.1.min.js |   4 +
 .../resources/scalate/js/jquery.tablesorter.min.js |   4 +
 .../org/apache/samza/job/yarn/ClientHelper.scala   |   5 +-
 .../samza/job/yarn/SamzaAppMasterMetrics.scala     |  67 ++
 .../job/yarn/SamzaAppMasterSecurityManager.scala   | 126 ++++
 .../job/yarn/SamzaContainerSecurityManager.scala   | 102 +++
 .../job/yarn/SamzaYarnAppMasterLifecycle.scala     | 121 ++++
 .../samza/job/yarn/SamzaYarnAppMasterService.scala | 103 +++
 .../job/yarn/SamzaYarnSecurityManagerFactory.scala |  30 +
 .../samza/job/yarn/YarnAppMasterListener.scala     |  68 ++
 .../org/apache/samza/job/yarn/YarnContainer.scala  |  47 ++
 .../scala/org/apache/samza/job/yarn/YarnJob.scala  | 191 +++++
 .../org/apache/samza/job/yarn/YarnJobFactory.scala |  55 ++
 .../org/apache/samza/job/yarn/YarnJobUtil.scala    | 109 +++
 .../apache/samza/util/hadoop/HttpFileSystem.scala  | 107 +++
 .../apache/samza/util/hadoop/HttpInputStream.scala |  46 ++
 .../webapp/ApplicationMasterRestServlet.scala      | 139 ++++
 .../samza/webapp/ApplicationMasterWebServlet.scala |  47 ++
 .../org/apache/samza/config/TestYarnConfig.java    |  40 ++
 .../samza/job/yarn/TestFileSystemImplConfig.java   |  84 +++
 .../job/yarn/TestLocalizerResourceConfig.java      | 125 ++++
 .../job/yarn/TestLocalizerResourceMapper.java      | 174 +++++
 .../job/yarn/TestYarnClusterResourceManager.java   | 253 +++++++
 .../samza/job/yarn/TestYarnFaultDomainManager.java | 198 ++++++
 .../org/apache/samza/job/yarn/TestYarnJob.java     | 114 +++
 .../apache/samza/job/yarn/TestYarnJobFactory.java  |  59 ++
 .../samza/job/yarn/util/MockContainerListener.java | 147 ++++
 .../apache/samza/job/yarn/util/MockHttpServer.java |  55 ++
 .../apache/samza/job/yarn/util/MockNMClient.java   |  68 ++
 .../samza/job/yarn/util/TestAMRMClientImpl.java    |  97 +++
 .../job/yarn/util/hadoop/TestHttpFileSystem.java   | 161 +++++
 .../samza/validation/MockMetricsValidator.java     |  52 ++
 .../validation/TestYarnJobValidationTool.java      | 142 ++++
 .../webapp/TestApplicationMasterRestClient.java    | 331 +++++++++
 .../apache/samza/webapp/TestLocalityServlet.java   | 131 ++++
 .../webapp/TestYarnContainerHeartbeatServlet.java  |  99 +++
 .../apache/samza/job/yarn/MockSystemAdmin.scala    |  42 ++
 .../apache/samza/job/yarn/MockSystemFactory.scala  |  43 ++
 .../apache/samza/job/yarn/TestClientHelper.scala   | 160 +++++
 .../job/yarn/TestSamzaYarnAppMasterLifecycle.scala | 204 ++++++
 .../job/yarn/TestSamzaYarnAppMasterService.scala   | 131 ++++
 settings.gradle                                    |   6 +-
 94 files changed, 7659 insertions(+), 40 deletions(-)

diff --git a/README.md b/README.md
index e5c91a415..1a46f8303 100644
--- a/README.md
+++ b/README.md
@@ -26,11 +26,28 @@ To build Samza from a source release, it is first necessary to download the grad
 
 After the bootstrap script has completed, the regular gradlew instructions below are available.
 
+### Java Version Support
+
+This project is built with Java 8 and can run in a Java 8 runtime enviornment. Additionally, it also supports running in a Java 11 runtime environment. 
+If you intend to use Samza in a Java 11 runtime environment, it means you will also need to use YARN 3.3.4+ and in which case, you should also use 
+the `samza-yarn3` module (built with YARN 3.3.4) instead of the `samza-yarn` (built with YARN 2.10.1). There is also a `samza-shell-yarn3` that
+depends on the `samza-yarn3` module, so use that shell module if you intend on using Yarn 3.
+
 #### Scala and YARN
 
-Samza builds with [Scala](http://www.scala-lang.org/) 2.11 or 2.12 and [YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html) 2.6.1, by default. Use the -PscalaSuffix switches to change Scala versions. Samza supports building Scala with 2.11 and 2.12.
+Samza builds with [Scala](http://www.scala-lang.org/) 2.11 or 2.12 and [YARN](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html) 2.10.1, by default. 
+Use the -PscalaSuffix switches to change Scala versions. Samza supports building Scala with 2.11 and 2.12 and provides a YARN 2 module (`samze-yarn`) and a YARN 3 module (`samza-yarn3`).
+
+NOTE: Some modules currently do **not** officially support Java 11 Runtime and are still using the YARN 2.10.1 dependency:
+* `samza-yarn`
+* `samza-shell`
+* `samza-test`
+* `samza-hdfs`
+
+
+    ./gradlew -PscalaSuffix=2.12 clean build 
 
-    ./gradlew -PscalaSuffix=2.11 clean build
+Also, you can make use of `bin/check-all.sh` in order to test multiple variants of Java JDKs, Scala, and Yarn.
 
 ### Testing Samza
 
diff --git a/build.gradle b/build.gradle
index 2ed9b921e..39687057f 100644
--- a/build.gradle
+++ b/build.gradle
@@ -523,6 +523,76 @@ project(":samza-yarn_$scalaSuffix") {
   jar.dependsOn("lesscss")
 }
 
+project(":samza-yarn3_$scalaSuffix") {
+  apply plugin: 'scala'
+  apply plugin: 'lesscss'
+
+  // Force scala joint compilation
+  sourceSets.main.scala.srcDir "src/main/java"
+  sourceSets.test.scala.srcDir "src/test/java"
+
+  // Disable the Javac compiler by forcing joint compilation by scalac. This is equivalent to setting
+  // tasks.compileTestJava.enabled = false
+  sourceSets.main.java.srcDirs = []
+  sourceSets.test.java.srcDirs = []
+
+  dependencies {
+    compile project(':samza-api')
+    compile project(":samza-core_$scalaSuffix")
+    compile "org.scala-lang:scala-library:$scalaVersion"
+    compile "org.scala-lang:scala-compiler:$scalaVersion"
+    compile "com.google.guava:guava:$guavaVersion"
+    compile "com.fasterxml.jackson.core:jackson-databind:$jacksonVersion"
+    compile "commons-httpclient:commons-httpclient:$commonsHttpClientVersion"
+    compile "org.apache.httpcomponents:httpclient:$httpClientVersion"
+    compile("org.apache.hadoop:hadoop-yarn-api:$yarn3Version") {
+    }
+    compile("org.apache.hadoop:hadoop-yarn-common:$yarn3Version") {
+      exclude module: 'servlet-api'
+    }
+    compile("org.apache.hadoop:hadoop-yarn-client:$yarn3Version") {
+      exclude module: 'servlet-api'
+    }
+    compile("org.apache.hadoop:hadoop-common:$yarn3Version") {
+      exclude module: 'servlet-api'
+      // Exclude because YARN's 3.4.5 ZK version is incompatbile with Kafka's 3.3.4.
+      exclude module: 'zookeeper'
+    }
+    compile("org.apache.hadoop:hadoop-hdfs:$yarn3Version") {
+      exclude module: 'servlet-api'
+    }
+    compile("org.scalatra:scalatra_$scalaSuffix:$scalatraVersion") {
+      exclude module: 'scala-compiler'
+      exclude module: 'slf4j-api'
+    }
+    compile("org.scalatra:scalatra-scalate_$scalaSuffix:$scalatraVersion") {
+      exclude module: 'scala-compiler'
+      exclude module: 'slf4j-api'
+    }
+    compile "joda-time:joda-time:$jodaTimeVersion"
+    compile "org.apache.zookeeper:zookeeper:$zookeeperVersion"
+    testCompile "junit:junit:$junitVersion"
+    testCompile "org.mockito:mockito-core:$mockitoVersion"
+    testCompile project(":samza-core_$scalaSuffix").sourceSets.test.output
+    testCompile "org.scalatest:scalatest_$scalaSuffix:$scalaTestVersion"
+  }
+
+  repositories {
+    maven {
+      url "https://repo.typesafe.com/typesafe/releases"
+    }
+  }
+
+  lesscss {
+    source = fileTree('src/main/less') {
+      include 'main.less'
+    }
+    dest = "$buildDir/resources/main/scalate/css"
+  }
+
+  jar.dependsOn("lesscss")
+}
+
 project(":samza-shell") {
   apply plugin: 'java'
 
@@ -590,6 +660,73 @@ project(":samza-shell") {
   }
 }
 
+project(":samza-shell-yarn3") {
+  apply plugin: 'java'
+
+  configurations {
+    gradleShell
+  }
+
+  dependencies {
+    gradleShell project(":samza-core_$scalaSuffix")
+    gradleShell project(":samza-kafka_$scalaSuffix")
+    gradleShell project(":samza-test_$scalaSuffix")
+    gradleShell project(":samza-yarn3_$scalaSuffix")
+  }
+
+  task shellTarGz(type: Tar) {
+    compression = Compression.GZIP
+    classifier = 'dist'
+    from(project(":samza-shell").file("./src/main/bash"))
+    from(project(":samza-shell").file("./src/main/resources"))
+    from(project(":samza-shell").file("./src/main/visualizer"))
+  }
+
+  artifacts {
+    archives(shellTarGz) {
+      name 'samza-shell-yarn3'
+      classifier 'dist'
+    }
+  }
+
+  // Usage: ./gradlew samza-shell:runJob \
+  //  -PconfigPath=/path/to/job/config.properties
+  task runJob(type:JavaExec) {
+    description 'To run a job (defined in a properties file)'
+    main = 'org.apache.samza.job.JobRunner'
+    classpath = configurations.gradleShell
+    if (project.hasProperty('configPath')) args += [
+            '--config', 'job.config.loader.factory=org.apache.samza.config.loaders.PropertiesConfigLoaderFactory',
+            '--config', 'job.config.loader.properties.path=' + configPath]
+    jvmArgs = ["-Dlog4j.configurationFile=file:src/main/resources/log4j2-console.xml"]
+  }
+
+  // Usage: ./gradlew samza-shell:checkpointTool \
+  //  -PconfigPath=/path/to/job/config.properties -PnewOffsets=/path/to/new/offsets.properties
+  task checkpointTool(type:JavaExec) {
+    description 'Command-line tool to inspect and manipulate the job’s checkpoint'
+    main = 'org.apache.samza.checkpoint.CheckpointTool'
+    classpath = configurations.gradleShell
+    if (project.hasProperty('configPath')) args += [
+            '--config', 'job.config.loader.factory=org.apache.samza.config.loaders.PropertiesConfigLoaderFactory',
+            '--config', 'job.config.loader.properties.path=' + configPath]
+    if (project.hasProperty('newOffsets')) args += ['--new-offsets', newOffsets]
+    jvmArgs = ["-Dlog4j.configurationFile=file:src/main/resources/log4j2-console.xml"]
+  }
+
+  // Usage: ./gradlew samza-shell:kvPerformanceTest
+  //  -PconfigPath=/path/to/job/config.properties
+  task kvPerformanceTest(type:JavaExec) {
+    description 'Command-line tool to run key-value performance tests'
+    main = 'org.apache.samza.test.performance.TestKeyValuePerformance'
+    classpath = configurations.gradleShell
+    if (project.hasProperty('configPath')) args += [
+            '--config', 'job.config.loader.factory=org.apache.samza.config.loaders.PropertiesConfigLoaderFactory',
+            '--config', 'job.config.loader.properties.path=' + configPath]
+    jvmArgs = ["-Dlog4j.configurationFile=file:src/main/resources/log4j2-console.xml"]
+  }
+}
+
 project(":samza-kv_$scalaSuffix") {
   apply plugin: 'scala'
 
@@ -710,6 +847,7 @@ project(":samza-rest_$scalaSuffix") {
     compile "org.glassfish.jersey.media:jersey-media-moxy:$jerseyVersion"
     compile "org.eclipse.jetty.aggregate:jetty-all:$jettyVersion"
     compile "commons-httpclient:commons-httpclient:$commonsHttpClientVersion"
+    compile "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:$jacksonVersion"
     compile("org.apache.hadoop:hadoop-yarn-client:$yarnVersion") {
       exclude module: 'servlet-api'
       exclude group: 'com.sun.jersey'
diff --git a/gradle.properties b/gradle.properties
index cab76e044..1a1366426 100644
--- a/gradle.properties
+++ b/gradle.properties
@@ -17,9 +17,12 @@
 group=org.apache.samza
 version=1.7.0-SNAPSHOT
 scalaSuffix=2.12
+yarnVersion=2.10.1
+# This version of YARN supports Java11 and allows Samza to be run in a Java11 runtime environment
+yarn3Version=3.3.4
 
 # after changing this value, run `$ ./gradlew wrapper` and commit the resulting changed files
-gradleVersion=5.2.1
+gradleVersion=6.9.2
 
 org.gradle.jvmargs="-XX:MaxPermSize=512m"
 
diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle
index 4e848de43..a01dde5d3 100644
--- a/gradle/dependency-versions.gradle
+++ b/gradle/dependency-versions.gradle
@@ -53,4 +53,5 @@
   jnaVersion = "4.5.1"
   couchbaseClientVersion = "2.7.2"
   couchbaseMockVersion = "1.5.22"
+  yarn3Version = "3.3.4"
 }
diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties
index 44e7c4d1d..ec991f9aa 100644
--- a/gradle/wrapper/gradle-wrapper.properties
+++ b/gradle/wrapper/gradle-wrapper.properties
@@ -1,5 +1,5 @@
 distributionBase=GRADLE_USER_HOME
 distributionPath=wrapper/dists
-distributionUrl=https\://services.gradle.org/distributions/gradle-5.2.1-bin.zip
+distributionUrl=https\://services.gradle.org/distributions/gradle-6.9.2-bin.zip
 zipStoreBase=GRADLE_USER_HOME
 zipStorePath=wrapper/dists
diff --git a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java
index 7ffbfc7a2..cc7bda2b8 100644
--- a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java
+++ b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemAdmin.java
@@ -28,7 +28,8 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang.StringUtils;
+
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
diff --git a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java
index 1ceb5d61c..10165583f 100644
--- a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java
+++ b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/HdfsSystemConsumer.java
@@ -30,8 +30,8 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang.Validate;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.Validate;
 import org.apache.samza.Partition;
 import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
diff --git a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/DirectoryPartitioner.java b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/DirectoryPartitioner.java
index 82445043e..4f07ef402 100644
--- a/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/DirectoryPartitioner.java
+++ b/samza-hdfs/src/main/java/org/apache/samza/system/hdfs/partitioner/DirectoryPartitioner.java
@@ -19,6 +19,15 @@
 
 package org.apache.samza.system.hdfs.partitioner;
 
+import org.apache.commons.collections.MapUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.Partition;
+import org.apache.samza.SamzaException;
+import org.apache.samza.system.hdfs.reader.MultiFileHdfsReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -29,16 +38,6 @@ import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import javax.annotation.Nullable;
-
-import org.apache.commons.collections.MapUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.samza.Partition;
-import org.apache.samza.SamzaException;
-import org.apache.samza.system.hdfs.reader.MultiFileHdfsReader;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import static org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata;
 import static org.apache.samza.system.hdfs.partitioner.FileSystemAdapter.FileMetadata;
 
diff --git a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
index 391536a7e..dbd3bb4ff 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
+++ b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConfig.scala
@@ -366,16 +366,14 @@ class KafkaConfig(config: Config) extends ScalaMapConfig(config) {
     val properties = new Properties()
 
     if (isStreamMode) {
-      properties.putAll(ImmutableMap.of(
-        "cleanup.policy", "compact",
-        "segment.bytes", String.valueOf(segmentBytes),
-        "max.message.bytes", String.valueOf(maxMessageBytes)))
+      properties.put("cleanup.policy", "compact")
+      properties.put("segment.bytes", String.valueOf(segmentBytes))
+      properties.put("max.message.bytes", String.valueOf(maxMessageBytes))
     } else {
-      properties.putAll(ImmutableMap.of(
-        "cleanup.policy", "compact,delete",
-        "retention.ms", String.valueOf(KafkaConfig.DEFAULT_RETENTION_MS_FOR_BATCH),
-        "segment.bytes", String.valueOf(segmentBytes),
-        "max.message.bytes", String.valueOf(maxMessageBytes)))
+      properties.put("cleanup.policy", "compact,delete")
+      properties.put("retention.ms", String.valueOf(KafkaConfig.DEFAULT_RETENTION_MS_FOR_BATCH))
+      properties.put("segment.bytes", String.valueOf(segmentBytes))
+      properties.put("max.message.bytes", String.valueOf(maxMessageBytes))
     }
     properties
   }
diff --git a/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitorConfig.java b/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitorConfig.java
index 6923f2f6f..22eee7730 100644
--- a/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitorConfig.java
+++ b/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitorConfig.java
@@ -20,7 +20,8 @@ package org.apache.samza.monitor;
 
 import java.util.Arrays;
 import java.util.List;
-import org.apache.commons.lang.StringUtils;
+
+import org.apache.commons.lang3.StringUtils;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.MapConfig;
diff --git a/samza-rest/src/main/java/org/apache/samza/monitor/MonitorConfig.java b/samza-rest/src/main/java/org/apache/samza/monitor/MonitorConfig.java
index 229125814..786205544 100644
--- a/samza-rest/src/main/java/org/apache/samza/monitor/MonitorConfig.java
+++ b/samza-rest/src/main/java/org/apache/samza/monitor/MonitorConfig.java
@@ -23,7 +23,8 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
-import org.apache.commons.lang.StringUtils;
+
+import org.apache.commons.lang3.StringUtils;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.MapConfig;
 
diff --git a/samza-rest/src/main/java/org/apache/samza/rest/SamzaRestApplication.java b/samza-rest/src/main/java/org/apache/samza/rest/SamzaRestApplication.java
index 7b4cd68a6..c1f58f6b4 100644
--- a/samza-rest/src/main/java/org/apache/samza/rest/SamzaRestApplication.java
+++ b/samza-rest/src/main/java/org/apache/samza/rest/SamzaRestApplication.java
@@ -19,12 +19,14 @@
 package org.apache.samza.rest;
 
 import java.util.Collection;
+
+import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
 import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
 import org.apache.samza.rest.resources.DefaultResourceFactory;
 import org.apache.samza.rest.resources.ResourceFactory;
 import org.apache.samza.util.ReflectionUtil;
-import org.codehaus.jackson.jaxrs.JacksonJsonProvider;
+
 import org.glassfish.jersey.server.ResourceConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/samza-rest/src/main/java/org/apache/samza/rest/proxy/task/SamzaTaskProxyFactory.java b/samza-rest/src/main/java/org/apache/samza/rest/proxy/task/SamzaTaskProxyFactory.java
index d36e8b1ef..4e72e1e88 100644
--- a/samza-rest/src/main/java/org/apache/samza/rest/proxy/task/SamzaTaskProxyFactory.java
+++ b/samza-rest/src/main/java/org/apache/samza/rest/proxy/task/SamzaTaskProxyFactory.java
@@ -19,7 +19,8 @@
 package org.apache.samza.rest.proxy.task;
 
 import com.google.common.base.Preconditions;
-import org.apache.commons.lang.StringUtils;
+
+import org.apache.commons.lang3.StringUtils;
 import org.apache.samza.SamzaException;
 import org.apache.samza.config.ConfigFactory;
 import org.apache.samza.rest.proxy.installation.InstallationFinder;
diff --git a/samza-rest/src/main/java/org/apache/samza/rest/resources/TasksResource.java b/samza-rest/src/main/java/org/apache/samza/rest/resources/TasksResource.java
index 12db787cf..d342b9a32 100644
--- a/samza-rest/src/main/java/org/apache/samza/rest/resources/TasksResource.java
+++ b/samza-rest/src/main/java/org/apache/samza/rest/resources/TasksResource.java
@@ -26,7 +26,8 @@ import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import org.apache.commons.lang.StringUtils;
+
+import org.apache.commons.lang3.StringUtils;
 import org.apache.samza.SamzaException;
 import org.apache.samza.rest.proxy.job.JobInstance;
 import org.apache.samza.rest.proxy.task.TaskProxyFactory;
diff --git a/samza-shell/src/main/bash/run-class.sh b/samza-shell/src/main/bash/run-class.sh
index 3de6dcf40..e95ec6d89 100755
--- a/samza-shell/src/main/bash/run-class.sh
+++ b/samza-shell/src/main/bash/run-class.sh
@@ -114,6 +114,14 @@ function check_and_enable_64_bit_mode {
   fi
 }
 
+# Try and use the -XX:+PrintGCDateStamps jvm argument. Java11 will fail
+function check_and_enable_print_gc_datestamps {
+  `$JAVA -XX:+PrintGCDateStamps -version`
+  if [ $? -eq 0 ] ; then
+    JAVA_OPTS="$JAVA_OPTS -XX:+PrintGCDateStamps"
+  fi
+}
+
 ### Inherit JVM_OPTS from task.opts configuration, and initialize defaults ###
 
 # Make the MDC inheritable to child threads by setting the system property to true if config not explicitly specified
@@ -153,7 +161,7 @@ fi
 [[ $JAVA_OPTS != *-Xmx* ]] && JAVA_OPTS="$JAVA_OPTS -Xmx768M"
 
 # Check if the GC related flags are specified. If not - add the respective flags to JVM_OPTS.
-[[ $JAVA_OPTS != *PrintGCDateStamps* && $JAVA_OPTS != *-Xloggc* ]] && JAVA_OPTS="$JAVA_OPTS -XX:+PrintGCDateStamps -Xloggc:$SAMZA_LOG_DIR/gc.log"
+[[ $JAVA_OPTS != *-Xloggc* ]] && JAVA_OPTS="$JAVA_OPTS -Xloggc:$SAMZA_LOG_DIR/gc.log"
 
 # Check if GC log rotation is already enabled. If not - add the respective flags to JVM_OPTS
 [[ $JAVA_OPTS != *UseGCLogFileRotation* ]] && check_and_enable_gc_log_rotation
@@ -161,6 +169,9 @@ fi
 # Check if 64 bit is set. If not - try and set it if it's supported
 [[ $JAVA_OPTS != *-d64* ]] && check_and_enable_64_bit_mode
 
+# Check if we can use PrintGCDateStamps. Java 11 will fail if this is provided, Java 8 is fine
+[[ $JAVA_OPTS != *PrintGCDateStamps* ]] && check_and_enable_print_gc_datestamps
+
 # HADOOP_CONF_DIR should be supplied to classpath explicitly for Yarn to parse configs
 echo $JAVA $JAVA_OPTS -cp $HADOOP_CONF_DIR:$PATHING_JAR_FILE "$@"
 
diff --git a/samza-yarn/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java b/samza-yarn/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java
index ca94783e6..bfc3d809f 100644
--- a/samza-yarn/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java
+++ b/samza-yarn/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java
@@ -20,7 +20,8 @@ package org.apache.samza.job.yarn;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.commons.lang.StringUtils;
+
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala
index 8e0c3d1b2..207eb6b12 100644
--- a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala
+++ b/samza-yarn/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala
@@ -19,7 +19,8 @@
 
 package org.apache.samza.job.yarn
 
-import org.apache.commons.lang.StringUtils
+
+import org.apache.commons.lang3.StringUtils
 import org.apache.hadoop.fs.permission.FsPermission
 import org.apache.samza.config.{Config, JobConfig, YarnConfig}
 import org.apache.samza.coordinator.stream.CoordinatorStreamWriter
@@ -53,9 +54,9 @@ import org.apache.samza.job.ApplicationStatus.Running
 import org.apache.samza.job.ApplicationStatus.SuccessfulFinish
 import org.apache.samza.job.ApplicationStatus.UnsuccessfulFinish
 import org.apache.samza.util.Logging
+
 import java.io.IOException
 import java.nio.ByteBuffer
-
 import org.apache.http.impl.client.HttpClientBuilder
 import org.apache.samza.webapp.ApplicationMasterRestClient
 
diff --git a/samza-yarn3/src/main/java/org/apache/samza/config/YarnConfig.java b/samza-yarn3/src/main/java/org/apache/samza/config/YarnConfig.java
new file mode 100644
index 000000000..4b37a8114
--- /dev/null
+++ b/samza-yarn3/src/main/java/org/apache/samza/config/YarnConfig.java
@@ -0,0 +1,251 @@
+/*
+ * 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.samza.config;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.samza.SamzaException;
+
+public class YarnConfig extends MapConfig {
+  /**
+   * (Required) URL from which the job package can be downloaded
+   */
+  public static final String PACKAGE_PATH = "yarn.package.path";
+
+  /**
+   * Type of Samza application submitted to YARN
+   * Can be used to distinguish between different job types/APIs (eg: HighLevel, LowLevel, SQL, Beam etc)
+   */
+  public static final String YARN_APPLICATION_TYPE = "yarn.application.type";
+  public static final String DEFAULT_SAMZA_APPLICATION_TYPE = "Samza";
+
+
+  /**
+   * Name of YARN queue to run jobs on
+   */
+  public static final String QUEUE_NAME = "yarn.queue";
+
+  /**
+   * Label to request from YARN for containers
+   */
+  public static final String CONTAINER_LABEL = "yarn.container.label";
+
+  // Configs related to the Samza Application Master (AM)
+  /**
+   * (Optional) JVM options to include in the command line when executing the AM
+   */
+  public static final String AM_JVM_OPTIONS = "yarn.am.opts";
+
+  /**
+   * Determines whether a JMX server should be started on the AM
+   * Default: true
+   */
+  public static final String AM_JMX_ENABLED = "yarn.am.jmx.enabled";
+
+  /**
+   * Memory, in megabytes, to request from YARN for running the AM
+   */
+  public static final String AM_CONTAINER_MAX_MEMORY_MB = "yarn.am.container.memory.mb";
+  private static final int DEFAULT_AM_CONTAINER_MAX_MEMORY_MB = 1024;
+
+  /**
+   * Label to request from YARN for running the AM
+   */
+  public static final String AM_CONTAINER_LABEL = "yarn.am.container.label";
+
+  /**
+   * Number of CPU cores to request from YARN for running the AM
+   */
+  public static final String AM_CONTAINER_MAX_CPU_CORES = "yarn.am.container.cpu.cores";
+  private static final int DEFAULT_AM_CPU_CORES = 1;
+
+  /**
+   * Determines the interval for the Heartbeat between the AM and the Yarn RM
+   */
+  public static final String AM_POLL_INTERVAL_MS = "yarn.am.poll.interval.ms";
+  private static final int DEFAULT_POLL_INTERVAL_MS = 1000;
+
+  /**
+   * (Optional) JAVA_HOME path for Samza AM
+   */
+  public static final String AM_JAVA_HOME = "yarn.am.java.home";
+
+  // Configs related to the ContainerAllocator thread
+  /**
+   * Sleep interval for the allocator thread in milliseconds
+   */
+  public static final String ALLOCATOR_SLEEP_MS = "yarn.allocator.sleep.ms";
+  private static final int DEFAULT_ALLOCATOR_SLEEP_MS = 3600;
+  /**
+   * Number of milliseconds before a container request is considered to have to expired
+   */
+  public static final String CONTAINER_REQUEST_TIMEOUT_MS = "yarn.container.request.timeout.ms";
+  private static final int DEFAULT_CONTAINER_REQUEST_TIMEOUT_MS = 5000;
+
+  /**
+   * Principal used to log in on a Kerberized secure cluster
+   */
+  public static final String YARN_KERBEROS_PRINCIPAL = "yarn.kerberos.principal";
+
+  /**
+   * Key tab used to log in on a Kerberized secure cluster
+   */
+  public static final String YARN_KERBEROS_KEYTAB = "yarn.kerberos.keytab";
+
+  /**
+   * Interval in seconds to renew a delegation token in Kerberized secure cluster
+   */
+  public static final String YARN_TOKEN_RENEWAL_INTERVAL_SECONDS = "yarn.token.renewal.interval.seconds";
+  private static final long DEFAULT_YARN_TOKEN_RENEWAL_INTERVAL_SECONDS = 24 * 3600;
+
+  /**
+   * The location on HDFS to store the credentials file
+   */
+  public static final String YARN_CREDENTIALS_FILE = "yarn.credentials.file";
+
+  /**
+   * The staging directory on HDFS for the job
+   */
+  public static final String YARN_JOB_STAGING_DIRECTORY = "yarn.job.staging.directory";
+
+  /**
+   * For secured YARN cluster only.
+   * The 'viewing' acl of the YARN application. This controls who can view the application,
+   * for example, application status, logs.
+   * {@link org.apache.hadoop.yarn.api.records.ApplicationAccessType} for more details
+   */
+  public static final String YARN_APPLICATION_VIEW_ACL = "yarn.job.view.acl";
+
+  /**
+   * For secured YARN cluster only.
+   * The 'modify' acl of the YARN application. This controls who can modify the application,
+   * for example, killing the job.
+   * {@link org.apache.hadoop.yarn.api.records.ApplicationAccessType} for more details
+   */
+  public static final String YARN_APPLICATION_MODIFY_ACL = "yarn.job.modify.acl";
+
+  public YarnConfig(Config config) {
+    super(config);
+  }
+
+  public int getAMPollIntervalMs() {
+    return getInt(AM_POLL_INTERVAL_MS, DEFAULT_POLL_INTERVAL_MS);
+  }
+
+  public String getContainerLabel() {
+    return get(CONTAINER_LABEL, null);
+  }
+
+  public boolean getJmxServerEnabled() {
+    return getBoolean(AM_JMX_ENABLED, true);
+  }
+
+  public String getPackagePath() {
+    String packagePath = get(PACKAGE_PATH);
+    if (packagePath == null) {
+      throw new SamzaException("No YARN package path defined in config.");
+    }
+    return packagePath;
+  }
+
+  public int getAMContainerMaxMemoryMb() {
+    return getInt(AM_CONTAINER_MAX_MEMORY_MB, DEFAULT_AM_CONTAINER_MAX_MEMORY_MB);
+  }
+
+  public String getAMContainerLabel() {
+    return get(AM_CONTAINER_LABEL, null);
+  }
+
+  public int getAMContainerMaxCpuCores() {
+    return getInt(AM_CONTAINER_MAX_CPU_CORES, DEFAULT_AM_CPU_CORES);
+  }
+
+  public String getAmOpts() {
+    return get(AM_JVM_OPTIONS, "");
+  }
+
+  public String getQueueName() {
+    return get(QUEUE_NAME, null);
+  }
+
+  public String getAMJavaHome() {
+    return get(AM_JAVA_HOME, null);
+  }
+
+  public int getAllocatorSleepTime() {
+    return getInt(ALLOCATOR_SLEEP_MS, DEFAULT_ALLOCATOR_SLEEP_MS);
+  }
+
+  public int getContainerRequestTimeout() {
+    return getInt(CONTAINER_REQUEST_TIMEOUT_MS, DEFAULT_CONTAINER_REQUEST_TIMEOUT_MS);
+  }
+
+  public String getYarnKerberosPrincipal() {
+    return get(YARN_KERBEROS_PRINCIPAL, null);
+  }
+
+  public String getYarnKerberosKeytab() {
+    return get(YARN_KERBEROS_KEYTAB, null);
+  }
+
+  public long getYarnTokenRenewalIntervalSeconds() {
+    return getLong(YARN_TOKEN_RENEWAL_INTERVAL_SECONDS, DEFAULT_YARN_TOKEN_RENEWAL_INTERVAL_SECONDS);
+  }
+
+  public String getYarnCredentialsFile() {
+    return get(YARN_CREDENTIALS_FILE, null);
+  }
+
+  public String getYarnJobStagingDirectory() {
+    return get(YARN_JOB_STAGING_DIRECTORY, null);
+  }
+
+  public String getYarnApplicationViewAcl() {
+    return get(YARN_APPLICATION_VIEW_ACL, null);
+  }
+
+  public String getYarnApplicationModifyAcl() {
+    return get(YARN_APPLICATION_MODIFY_ACL, null);
+  }
+
+  public String getYarnApplicationType() {
+    return get(YARN_APPLICATION_TYPE, DEFAULT_SAMZA_APPLICATION_TYPE);
+  }
+
+  /**
+   * Helper function to get all application acls
+   * @return a map of {@link ApplicationAccessType} to {@link String} for all the acls defined
+   */
+  public Map<ApplicationAccessType, String> getYarnApplicationAcls() {
+    Map<ApplicationAccessType, String> acls = new HashMap<>();
+    String viewAcl = getYarnApplicationViewAcl();
+    String modifyAcl = getYarnApplicationModifyAcl();
+    if (viewAcl != null) {
+      acls.put(ApplicationAccessType.VIEW_APP, viewAcl);
+    }
+    if (modifyAcl != null) {
+      acls.put(ApplicationAccessType.MODIFY_APP, modifyAcl);
+    }
+    return acls;
+  }
+
+}
diff --git a/samza-yarn3/src/main/java/org/apache/samza/job/yarn/FileSystemImplConfig.java b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/FileSystemImplConfig.java
new file mode 100644
index 000000000..a582dbbc5
--- /dev/null
+++ b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/FileSystemImplConfig.java
@@ -0,0 +1,71 @@
+/*
+ * 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.samza.job.yarn;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.samza.config.Config;
+
+
+/**
+ * FileSystemImplConfig is intended to manage the Samza config for fs.&lt;scheme&gt;impl.
+ * e.g. fs.http.impl
+ */
+public class FileSystemImplConfig {
+  private static final String FS_IMPL_PREFIX = "fs.";
+  private static final String FS_IMPL_SUFFIX = ".impl";
+  private static final String FS_IMPL_TEMPLATE = "fs.%s.impl";
+
+  private final Config config;
+
+  public FileSystemImplConfig(final Config config) {
+    if (null == config) {
+      throw new IllegalArgumentException("config cannot be null");
+    }
+    this.config = config;
+  }
+
+  /**
+   * Get all schemes
+   * @return List of schemes in strings
+   */
+  public List<String> getSchemes() {
+    Config subConfig = config.subset(FS_IMPL_PREFIX, true);
+    List<String> schemes = new ArrayList<String>();
+    for (String key : subConfig.keySet()) {
+      if (key.endsWith(FS_IMPL_SUFFIX)) {
+        schemes.add(key.substring(0, key.length() - FS_IMPL_SUFFIX.length()));
+      }
+    }
+    return schemes;
+  }
+
+  /**
+   * Get the config subset for fs.&lt;scheme&gt;.impl
+   * It can include config for fs.&lt;scheme&gt;.impl and additional config for the subKeys fs.&lt;scheme&gt;.impl.* from the configuration
+   * e.g. for scheme "myScheme", there could be config for fs.myScheme.impl, fs.myScheme.impl.client and fs.myScheme.impl.server
+   * @param scheme scheme name, such as http, hdfs, myscheme
+   * @return config for the particular scheme
+   */
+  public Config getSchemeConfig(final String scheme) {
+    String fsSchemeImpl = String.format(FS_IMPL_TEMPLATE, scheme);
+    Config schemeConfig = config.subset(fsSchemeImpl, false); // do not strip off the prefix
+    return schemeConfig;
+  }
+}
diff --git a/samza-yarn/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java
similarity index 98%
copy from samza-yarn/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java
copy to samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java
index ca94783e6..bfc3d809f 100644
--- a/samza-yarn/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java
+++ b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java
@@ -20,7 +20,8 @@ package org.apache.samza.job.yarn;
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.commons.lang.StringUtils;
+
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
diff --git a/samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceException.java b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceException.java
new file mode 100644
index 000000000..0df69036e
--- /dev/null
+++ b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceException.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.samza.job.yarn;
+
+public class LocalizerResourceException extends RuntimeException {
+
+  /**
+   * Constructs an {@code LocalizerResourceException} with {@code null}
+   * as its error detail message.
+   */
+  public LocalizerResourceException() {
+    super();
+  }
+
+  /**
+   * Constructs an {@code LocalizerResourceException} with the specified detail message.
+   *
+   * @param message
+   *        The detail message (which is saved for later retrieval
+   *        by the {@link #getMessage()} method)
+   */
+  public LocalizerResourceException(String message) {
+    super(message);
+  }
+
+  /**
+   * Constructs an {@code LocalizerResourceException} with the specified detail message
+   * and cause.
+   *
+   * @param message
+   *        The detail message (which is saved for later retrieval
+   *        by the {@link #getMessage()} method)
+   *
+   * @param cause
+   *        The cause (which is saved for later retrieval by the
+   *        {@link #getCause()} method).  (A null value is permitted,
+   *        and indicates that the cause is nonexistent or unknown.)
+   */
+  public LocalizerResourceException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  /**
+   * Constructs an {@code LocalizerResourceException} with the specified cause and a
+   * detail message of {@code (cause==null ? null : cause.toString())}
+   *
+   * @param cause
+   *        The cause (which is saved for later retrieval by the
+   *        {@link #getCause()} method).  (A null value is permitted,
+   *        and indicates that the cause is nonexistent or unknown.)
+   */
+  public LocalizerResourceException(Throwable cause) {
+    super(cause);
+  }
+
+}
diff --git a/samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceMapper.java b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceMapper.java
new file mode 100644
index 000000000..6dddb0a1d
--- /dev/null
+++ b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceMapper.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.job.yarn;
+
+import com.google.common.collect.ImmutableMap;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A universal approach to generate local resource map which can be put in ContainerLaunchContext directly
+ */
+public class LocalizerResourceMapper {
+  private static final Logger log = LoggerFactory.getLogger(LocalizerResourceMapper.class);
+
+  private final YarnConfiguration yarnConfiguration; //yarn configurations
+  private final LocalizerResourceConfig resourceConfig;
+  private final Map<String, LocalResource> localResourceMap;
+
+  public LocalizerResourceMapper(LocalizerResourceConfig resourceConfig, YarnConfiguration yarnConfiguration) {
+    this.yarnConfiguration = yarnConfiguration;
+    this.resourceConfig = resourceConfig;
+    this.localResourceMap = buildResourceMapping();
+  }
+
+  private Map<String, LocalResource> buildResourceMapping() {
+    ImmutableMap.Builder<String, LocalResource>  localResourceMapBuilder = ImmutableMap.builder();
+
+    List<String> resourceNames = resourceConfig.getResourceNames();
+    for (String resourceName : resourceNames) {
+      String resourceLocalName = resourceConfig.getResourceLocalName(resourceName);
+      LocalResourceType resourceType = resourceConfig.getResourceLocalType(resourceName);
+      LocalResourceVisibility resourceVisibility = resourceConfig.getResourceLocalVisibility(resourceName);
+      Path resourcePath = resourceConfig.getResourcePath(resourceName);
+
+      LocalResource localResource = createLocalResource(resourcePath, resourceType, resourceVisibility);
+
+      localResourceMapBuilder.put(resourceLocalName, localResource);
+      log.info("preparing local resource: {}", resourceLocalName);
+    }
+
+    return localResourceMapBuilder.build();
+  }
+
+  private LocalResource createLocalResource(Path resourcePath, LocalResourceType resourceType, LocalResourceVisibility resourceVisibility) {
+    LocalResource localResource = Records.newRecord(LocalResource.class);
+    URL resourceUrl = ConverterUtils.getYarnUrlFromPath(resourcePath);
+    try {
+      FileStatus resourceFileStatus = resourcePath.getFileSystem(yarnConfiguration).getFileStatus(resourcePath);
+
+      if (null == resourceFileStatus) {
+        throw new LocalizerResourceException("Check getFileStatus implementation. getFileStatus gets unexpected null for resourcePath " + resourcePath);
+      }
+
+      localResource.setResource(resourceUrl);
+      log.info("setLocalizerResource for {}", resourceUrl);
+      localResource.setSize(resourceFileStatus.getLen());
+      localResource.setTimestamp(resourceFileStatus.getModificationTime());
+      localResource.setType(resourceType);
+      localResource.setVisibility(resourceVisibility);
+      return localResource;
+    } catch (IOException ioe) {
+      log.error("IO Exception when accessing the resource file status from the filesystem: " + resourcePath, ioe);
+      throw new LocalizerResourceException("IO Exception when accessing the resource file status from the filesystem: " + resourcePath);
+    }
+
+  }
+
+  public Map<String, LocalResource> getResourceMap() {
+    return ImmutableMap.copyOf(localResourceMap);
+  }
+
+}
diff --git a/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnAppState.java b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnAppState.java
new file mode 100644
index 000000000..15ca92fec
--- /dev/null
+++ b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnAppState.java
@@ -0,0 +1,138 @@
+/*
+ * 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.samza.job.yarn;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+
+import java.net.URL;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * YarnAppState encapsulates Yarn specific state variables that are Yarn specific. This class
+ * is useful for information to display in the UI.
+ *
+ * TODO: make these variables private, provide thread-safe accessors.
+ * Saving making changes to variables in YarnAppState because it is used by the UI, and changes to
+ * variable names, data structure etc. will require changes to the UI scaml templates too. This is tracked
+ * as a part of SAMZA-902
+ */
+
+public class YarnAppState {
+
+  /**
+   /**
+  * State indicating whether the job is healthy or not
+  * Modified by both the AMRMCallbackThread and the ContainerAllocator thread
+  */
+
+  public Map<String, YarnContainer> runningProcessors = new ConcurrentHashMap<>();
+
+  public Map<String, YarnContainer> pendingProcessors = new ConcurrentHashMap<>();
+
+  public ConcurrentMap<String, ContainerStatus> failedContainersStatus = new ConcurrentHashMap<>();
+
+  public YarnAppState(int taskId,
+                      ContainerId amContainerId,
+                      String nodeHost,
+                      int nodePort,
+                      int nodeHttpPort
+                      ) {
+    this.taskId = taskId;
+    this.amContainerId = amContainerId;
+    this.nodeHost = nodeHost;
+    this.nodePort = nodePort;
+    this.nodeHttpPort = nodeHttpPort;
+    this.appAttemptId = amContainerId.getApplicationAttemptId();
+  }
+
+
+  @Override
+  public String toString() {
+    return "YarnAppState{" +
+        ", taskId=" + taskId +
+        ", amContainerId=" + amContainerId +
+        ", nodeHost='" + nodeHost + '\'' +
+        ", nodePort=" + nodePort +
+        ", nodeHttpPort=" + nodeHttpPort +
+        ", appAttemptId=" + appAttemptId +
+        ", coordinatorUrl=" + coordinatorUrl +
+        ", rpcUrl=" + rpcUrl +
+        ", trackingUrl=" + trackingUrl +
+        ", runningProcessors=" + runningProcessors +
+        ", failedContainersStatus=" + failedContainersStatus +
+        '}';
+  }
+
+   /* The following state variables are primarily used for reference in the AM web services   */
+
+  /**
+   * Task Id of the AM
+   * Used for displaying in the AM UI. Usage found in {@link org.apache.samza.webapp.ApplicationMasterRestServlet}
+   * and scalate/WEB-INF/views/index.scaml
+   */
+
+  public final int taskId;
+  /**
+   * Id of the AM container (as allocated by the RM)
+   * Used for displaying in the AM UI. Usage in {@link org.apache.samza.webapp.ApplicationMasterRestServlet}
+   * and scalate/WEB-INF/views/index.scaml
+   */
+  public final ContainerId amContainerId;
+  /**
+   * Host name of the NM on which the AM is running
+   * Used for displaying in the AM UI. See scalate/WEB-INF/views/index.scaml
+   */
+  public final String nodeHost;
+  /**
+   * NM port on which the AM is running
+   * Used for displaying in the AM UI. See scalate/WEB-INF/views/index.scaml
+   */
+  public final int nodePort;
+  /**
+   * Http port of the NM on which the AM is running
+   * Used for displaying in the AM UI. See scalate/WEB-INF/views/index.scaml
+   */
+  public final int nodeHttpPort;
+  /**
+   * Application Attempt Id as provided by Yarn
+   * Used for displaying in the AM UI. See scalate/WEB-INF/views/index.scaml
+   * and {@link org.apache.samza.webapp.ApplicationMasterRestServlet}
+   */
+  public final ApplicationAttemptId appAttemptId;
+
+  //TODO: Make the below 3 variables immutable. Tracked as a part of SAMZA-902. Save for later.
+  /**
+   * Job Coordinator URL
+   * Usage in {@link org.apache.samza.job.yarn.SamzaYarnAppMasterService} &amp; YarnContainerRunner
+   */
+  public URL coordinatorUrl = null;
+  /**
+   * URL of the {@link org.apache.samza.webapp.ApplicationMasterRestServlet}
+   */
+  public URL rpcUrl = null;
+  /**
+   * URL of the {@link org.apache.samza.webapp.ApplicationMasterWebServlet}
+   */
+  public URL trackingUrl = null;
+}
\ No newline at end of file
diff --git a/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnClusterResourceManager.java b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnClusterResourceManager.java
new file mode 100644
index 000000000..c07a45b9a
--- /dev/null
+++ b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnClusterResourceManager.java
@@ -0,0 +1,774 @@
+/*
+ * 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.samza.job.yarn;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.time.Duration;
+import java.util.Set;
+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.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.*;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
+import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.samza.SamzaException;
+import org.apache.samza.clustermanager.*;
+import org.apache.samza.clustermanager.SamzaApplicationState;
+import org.apache.samza.clustermanager.ProcessorLaunchException;
+import org.apache.samza.config.ClusterManagerConfig;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.ShellCommandConfig;
+import org.apache.samza.config.YarnConfig;
+import org.apache.samza.coordinator.JobModelManager;
+import org.apache.samza.job.CommandBuilder;
+import org.apache.samza.metrics.MetricsRegistryMap;
+import org.apache.samza.util.Util;
+import org.apache.samza.util.hadoop.HttpFileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+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.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ *
+ * An {@link YarnClusterResourceManager} implements a ClusterResourceManager using Yarn as the underlying
+ * resource manager. This class is as an adaptor between Yarn and translates Yarn callbacks into
+ * Samza specific callback methods as specified in Callback.
+ *
+ * Thread-safety:
+ * 1.Start and stop methods should  NOT be called from multiple threads.
+ * 2.ALL callbacks from the YarnContainerManager are invoked from a single Callback thread of the AMRMClient.
+ * 3.Stop should not be called more than once.
+ *
+ */
+
+public class YarnClusterResourceManager extends ClusterResourceManager implements AMRMClientAsync.CallbackHandler, NMClientAsync.CallbackHandler {
+
+  private static final int PREFERRED_HOST_PRIORITY = 0;
+  private static final int ANY_HOST_PRIORITY = 1;
+
+  private static final String INVALID_PROCESSOR_ID = "-1";
+
+  /**
+   * The AMClient instance to request resources from yarn.
+   */
+  private final AMRMClientAsync<AMRMClient.ContainerRequest> amClient;
+
+  /**
+   * Configuration and state specific to Yarn.
+   */
+  private final YarnConfiguration yarnConfiguration;
+  private final YarnAppState state;
+
+  /**
+   * SamzaYarnAppMasterLifecycle is responsible for registering, unregistering the AM client.
+   */
+  private final SamzaYarnAppMasterLifecycle lifecycle;
+
+  /**
+   * SamzaAppMasterService is responsible for hosting an AM web UI. This picks up data from both
+   * SamzaAppState and YarnAppState.
+   */
+  private final SamzaYarnAppMasterService service;
+
+  private final YarnConfig yarnConfig;
+
+  /**
+   * State variables to map Yarn specific callbacks into Samza specific callbacks.
+   */
+  private final ConcurrentHashMap<SamzaResource, Container> allocatedResources = new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<SamzaResourceRequest, AMRMClient.ContainerRequest> requestsMap = new ConcurrentHashMap<>();
+
+  private final SamzaAppMasterMetrics metrics;
+
+  private final AtomicBoolean started = new AtomicBoolean(false);
+  private final Object lock = new Object();
+  private final NMClientAsync nmClientAsync;
+
+  private static final Logger log = LoggerFactory.getLogger(YarnClusterResourceManager.class);
+  private final Config config;
+
+  YarnClusterResourceManager(AMRMClientAsync amClientAsync, NMClientAsync nmClientAsync, Callback callback,
+      YarnAppState yarnAppState, SamzaYarnAppMasterLifecycle lifecycle, SamzaYarnAppMasterService service,
+      SamzaAppMasterMetrics metrics, YarnConfiguration yarnConfiguration, Config config) {
+    super(callback);
+    this.yarnConfiguration  = yarnConfiguration;
+    this.metrics = metrics;
+    this.yarnConfig = new YarnConfig(config);
+    this.config = config;
+    this.amClient = amClientAsync;
+    this.state = yarnAppState;
+    this.lifecycle = lifecycle;
+    this.service = service;
+    this.nmClientAsync = nmClientAsync;
+  }
+
+  /**
+   * Creates an YarnClusterResourceManager from config, a jobModelReader and a callback.
+   * @param config to instantiate the cluster manager with
+   * @param jobModelManager the jobModel manager to get the job model (mostly for the UI)
+   * @param callback the callback to receive events from Yarn.
+   * @param samzaAppState samza app state for display in the UI
+   */
+  public YarnClusterResourceManager(Config config, JobModelManager jobModelManager,
+      ClusterResourceManager.Callback callback, SamzaApplicationState samzaAppState) {
+    super(callback);
+    yarnConfiguration = new YarnConfiguration();
+    yarnConfiguration.set("fs.http.impl", HttpFileSystem.class.getName());
+
+    // Use the Samza job config "fs.<scheme>.impl" and "fs.<scheme>.impl.*" for YarnConfiguration
+    FileSystemImplConfig fsImplConfig = new FileSystemImplConfig(config);
+    fsImplConfig.getSchemes().forEach(
+      scheme -> {
+        fsImplConfig.getSchemeConfig(scheme).forEach(
+          (confKey, confValue) -> yarnConfiguration.set(confKey, confValue)
+        );
+      }
+    );
+
+    MetricsRegistryMap registry = new MetricsRegistryMap();
+    metrics = new SamzaAppMasterMetrics(config, samzaAppState, registry);
+
+    // parse configs from the Yarn environment
+    String containerIdStr = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.toString());
+    ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
+    String nodeHostString = System.getenv(ApplicationConstants.Environment.NM_HOST.toString());
+    String nodePortString = System.getenv(ApplicationConstants.Environment.NM_PORT.toString());
+    String nodeHttpPortString = System.getenv(ApplicationConstants.Environment.NM_HTTP_PORT.toString());
+
+    int nodePort = Integer.parseInt(nodePortString);
+    int nodeHttpPort = Integer.parseInt(nodeHttpPortString);
+    YarnConfig yarnConfig = new YarnConfig(config);
+    this.yarnConfig = yarnConfig;
+    this.config = config;
+    int interval = yarnConfig.getAMPollIntervalMs();
+
+    //Instantiate the AM Client.
+    this.amClient = AMRMClientAsync.createAMRMClientAsync(interval, this);
+
+    this.state = new YarnAppState(-1, containerId, nodeHostString, nodePort, nodeHttpPort);
+
+    log.info("Initialized YarnAppState: {}", state.toString());
+    this.service = new SamzaYarnAppMasterService(config, samzaAppState, this.state, registry, yarnConfiguration);
+
+    log.info("Container ID: {}, Nodehost:  {} , Nodeport : {} , NodeHttpport: {}", containerIdStr, nodeHostString, nodePort, nodeHttpPort);
+    ClusterManagerConfig clusterManagerConfig = new ClusterManagerConfig(config);
+    this.lifecycle = new SamzaYarnAppMasterLifecycle(
+        clusterManagerConfig.getContainerMemoryMb(),
+        clusterManagerConfig.getNumCores(),
+        samzaAppState,
+        state,
+        amClient,
+        new JobConfig(config).getApplicationMasterHighAvailabilityEnabled()
+    );
+    this.nmClientAsync = NMClientAsync.createNMClientAsync(this);
+
+  }
+
+  /**
+   * Starts the YarnClusterResourceManager and initialize all its sub-systems.
+   * Attempting to start an already started cluster manager will return immediately.
+   */
+  @Override
+  public void start() {
+    if (!started.compareAndSet(false, true)) {
+      log.info("Attempting to start an already started YarnClusterResourceManager");
+      return;
+    }
+    metrics.start();
+    service.onInit();
+    log.info("Starting YarnClusterResourceManager.");
+    amClient.init(yarnConfiguration);
+    amClient.start();
+    nmClientAsync.init(yarnConfiguration);
+    nmClientAsync.start();
+    Set<ContainerId> previousAttemptsContainers = lifecycle.onInit();
+    metrics.setContainersFromPreviousAttempts(previousAttemptsContainers.size());
+
+    if (new JobConfig(config).getApplicationMasterHighAvailabilityEnabled()) {
+      log.info("Received running containers from previous attempt. Invoking launch success for them.");
+      previousAttemptsContainers.forEach(this::handleOnContainerStarted);
+    }
+
+    if (lifecycle.shouldShutdown()) {
+      clusterManagerCallback.onError(new SamzaException("Invalid resource request."));
+    }
+
+    log.info("Finished starting YarnClusterResourceManager");
+  }
+
+  /**
+   * Request resources for running container processes.
+   */
+  @Override
+  public void requestResources(SamzaResourceRequest resourceRequest) {
+    String processorId = resourceRequest.getProcessorId();
+    String requestId = resourceRequest.getRequestId();
+    String preferredHost = resourceRequest.getPreferredHost();
+    String[] racks = resourceRequest.getFaultDomains().stream().map(FaultDomain::getId).toArray(String[]::new);
+    int memoryMb = resourceRequest.getMemoryMB();
+    int cpuCores = resourceRequest.getNumCores();
+    Resource capability = Resource.newInstance(memoryMb, cpuCores);
+    String nodeLabelsExpression = yarnConfig.getContainerLabel();
+
+    AMRMClient.ContainerRequest issuedRequest;
+
+    /*
+     * Yarn enforces these two checks:
+     *   1. ANY_HOST requests should always be made with relax-locality = true
+     *   2. A request with relax-locality = false should not be in the same priority as another with relax-locality = true
+     *
+     * Since the Samza AM makes preferred-host requests with relax-locality = false, it follows that ANY_HOST requests
+     * should specify a different priority-level. We can safely set priority of preferred-host requests to be higher than
+     * any-host requests since data-locality is critical.
+     */
+    if (preferredHost.equals("ANY_HOST")) {
+      Priority priority = Priority.newInstance(ANY_HOST_PRIORITY);
+      boolean relaxLocality = true;
+      log.info("Requesting resources for Processor ID: {} on nodes: {} on racks: {} with capability: {}, priority: {}, relaxLocality: {}, nodeLabelsExpression: {}",
+          processorId, null, Arrays.toString(racks), capability, priority, relaxLocality, nodeLabelsExpression);
+      issuedRequest = new AMRMClient.ContainerRequest(capability, null, null, priority, relaxLocality, nodeLabelsExpression);
+    } else {
+      String[] nodes = {preferredHost};
+      Priority priority = Priority.newInstance(PREFERRED_HOST_PRIORITY);
+      boolean relaxLocality = false;
+      log.info("Requesting resources for Processor ID: {} on nodes: {} on racks: {} with capability: {}, priority: {}, relaxLocality: {}, nodeLabelsExpression: {}",
+          processorId, Arrays.toString(nodes), Arrays.toString(racks), capability, priority, relaxLocality, nodeLabelsExpression);
+      issuedRequest = new AMRMClient.ContainerRequest(capability, nodes, racks, priority, relaxLocality, nodeLabelsExpression);
+    }
+    // ensure that updating the state and making the request are done atomically.
+    synchronized (lock) {
+      requestsMap.put(resourceRequest, issuedRequest);
+      amClient.addContainerRequest(issuedRequest);
+    }
+  }
+
+  /**
+   * Requests the YarnContainerManager to release a resource. If the app cannot use the resource or wants to give up
+   * the resource, it can release them.
+   *
+   * @param resource to be released
+   */
+  @Override
+  public void releaseResources(SamzaResource resource) {
+    log.info("Releasing Container ID: {} on host: {}", resource.getContainerId(), resource.getHost());
+    // ensure that updating state and removing the request are done atomically
+    synchronized (lock) {
+      Container container = allocatedResources.get(resource);
+      if (container == null) {
+        log.info("Container ID: {} on host: {} was already released.", resource.getContainerId(), resource.getHost());
+        return;
+      }
+      amClient.releaseAssignedContainer(container.getId());
+      allocatedResources.remove(resource);
+    }
+  }
+
+  /**
+   *
+   * Requests the launch of a StreamProcessor with the specified ID on the resource
+   * @param resource the SamzaResource on which to launch the StreamProcessor
+   * @param builder the builder to build the resource launch command from
+   *
+   * TODO: Support non-builder methods to launch resources. Maybe, refactor into a ContainerLaunchStrategy interface
+   */
+  @Override
+  public void launchStreamProcessor(SamzaResource resource, CommandBuilder builder) {
+    String processorId = builder.buildEnvironment().get(ShellCommandConfig.ENV_CONTAINER_ID);
+    String containerId = resource.getContainerId();
+    String host = resource.getHost();
+    log.info("Starting Processor ID: {} on Container ID: {} on host: {}", processorId, containerId, host);
+    synchronized (lock) {
+      try {
+        Container container = allocatedResources.get(resource);
+        if (container == null) {
+          log.info("Container ID: {} on host: {} was already allocated / released.", containerId, host);
+          return;
+        }
+
+        runProcessor(processorId, container, builder);
+      } catch (Throwable t) {
+        log.info("Error starting Processor ID: {} on Container ID: {} on host: {}", processorId, containerId, host, t);
+        clusterManagerCallback.onStreamProcessorLaunchFailure(resource, t);
+      }
+    }
+  }
+
+  public void stopStreamProcessor(SamzaResource resource) {
+    synchronized (lock) {
+      Container container = allocatedResources.get(resource);
+      String containerId = resource.getContainerId();
+      String containerHost = resource.getHost();
+      /*
+       * 1. Stop the container through NMClient if the container was instantiated as part of NMClient lifecycle.
+       * 2. Stop the container through AMClient by release the assigned container if the container was from the previous
+       *    attempt and managed by the AM due to AM-HA
+       * 3. Ignore the request if the container associated with the resource isn't present in the book keeping.
+       */
+      if (container != null) {
+        log.info("Stopping Container ID: {} on host: {}", containerId, containerHost);
+        this.nmClientAsync.stopContainerAsync(container.getId(), container.getNodeId());
+      } else {
+        YarnContainer yarnContainer = state.runningProcessors.get(getRunningProcessorId(containerId));
+        if (yarnContainer != null) {
+          log.info("Stopping container from previous attempt with Container ID: {} on host: {}",
+              containerId, containerHost);
+          amClient.releaseAssignedContainer(yarnContainer.id());
+        } else {
+          log.info("No container with Container ID: {} exists. Ignoring the stop request", containerId);
+        }
+      }
+    }
+  }
+
+  /**
+   * Given a containerId from Yarn (for example: containerId_app_12345, this method returns the processor ID
+   * in the range [0,N-1] that maps to it.
+   * @param containerId  the Yarn container ID.
+   * @return  the Samza processor ID.
+   */
+  //TODO: Get rid of the YarnContainer object and just use Container in state.runningProcessors hashmap.
+  //In that case, this scan will turn into a lookup. This change will require changes/testing in the UI files because
+  //those UI stub templates operate on the YarnContainer object.
+  private String getRunningProcessorId(String containerId) {
+    for (Map.Entry<String, YarnContainer> entry : state.runningProcessors.entrySet()) {
+      String key = entry.getKey();
+      YarnContainer yarnContainer = entry.getValue();
+      String yarnContainerId = yarnContainer.id().toString();
+      if (yarnContainerId.equals(containerId)) {
+        return key;
+      }
+    }
+    return INVALID_PROCESSOR_ID;
+  }
+
+
+  /**
+   *
+   * Remove a previously submitted resource request. The previous container request may have
+   * been submitted. Even after the remove request, a Callback implementation must
+   * be prepared to receive an allocation for the previous request. This is merely a best effort cancellation.
+   *
+   * @param request the request to be cancelled
+   */
+  @Override
+  public void cancelResourceRequest(SamzaResourceRequest request) {
+    String processorId = request.getProcessorId();
+    String preferredHost = request.getPreferredHost();
+    String requestId = request.getRequestId();
+    log.info("Cancelling resource request for Processor ID: {} on host: {} with Request ID: {}",
+        processorId, preferredHost, requestId);
+    //ensure that removal and cancellation are done atomically.
+    synchronized (lock) {
+      AMRMClient.ContainerRequest containerRequest = requestsMap.get(request);
+      if (containerRequest == null) {
+        log.info("Resource request for Processor ID: {} on host: {} with Request ID: {} already cancelled.",
+            processorId, preferredHost, requestId);
+        return;
+      }
+      requestsMap.remove(request);
+      amClient.removeContainerRequest(containerRequest);
+    }
+  }
+
+
+  /**
+   * Stops the YarnContainerManager and all its sub-components.
+   * Stop should NOT be called from multiple threads.
+   * TODO: fix this to make stop idempotent?.
+   */
+  @Override
+  public void stop(SamzaApplicationState.SamzaAppStatus status) {
+    log.info("Stopping the AM client on shutdown request.");
+    lifecycle.onShutdown(status);
+    amClient.stop();
+    log.info("Stopping the NM client on shutdown request.");
+    nmClientAsync.stop();
+    log.info("Stopping the SamzaYarnAppMasterService service on shutdown request.");
+    service.onShutdown();
+    log.info("Stopping SamzaAppMasterMetrics on shutdown request.");
+    metrics.stop();
+
+    if (status != SamzaApplicationState.SamzaAppStatus.UNDEFINED) {
+      cleanupStagingDir();
+    }
+  }
+
+  /**
+   * Cleans up the staging directory of the job. All exceptions during the cleanup
+   * are swallowed.
+   */
+  private void cleanupStagingDir() {
+    String yarnJobStagingDirectory = yarnConfig.getYarnJobStagingDirectory();
+    if (yarnJobStagingDirectory != null) {
+      JobContext context = new JobContext();
+      context.setAppStagingDir(new Path(yarnJobStagingDirectory));
+
+      FileSystem fs = null;
+      try {
+        fs = FileSystem.get(yarnConfiguration);
+      } catch (IOException e) {
+        log.error("Unable to clean up file system.", e);
+        return;
+      }
+      if (fs != null) {
+        YarnJobUtil.cleanupStagingDir(context, fs);
+      }
+    }
+  }
+
+  /**
+   * Callback invoked from Yarn when containers complete. This translates the yarn callbacks into Samza specific
+   * ones.
+   *
+   * @param statuses the YarnContainerStatus callbacks from Yarn.
+   */
+  @Override
+  public void onContainersCompleted(List<ContainerStatus> statuses) {
+    List<SamzaResourceStatus> samzaResourceStatuses = new ArrayList<>();
+
+    for (ContainerStatus status : statuses) {
+      log.info("Got completion notification for Container ID: {} with status: {} and state: {}. Diagnostics information: {}.",
+          status.getContainerId(), status.getExitStatus(), status.getState(), status.getDiagnostics());
+
+      SamzaResourceStatus samzaResourceStatus = new SamzaResourceStatus(status.getContainerId().toString(), status.getDiagnostics(), status.getExitStatus());
+      samzaResourceStatuses.add(samzaResourceStatus);
+
+      String completedProcessorID = getRunningProcessorId(status.getContainerId().toString());
+      log.info("Completed Container ID: {} had Processor ID: {}", status.getContainerId(), completedProcessorID);
+
+      //remove the container from the list of running containers, if failed with a non-zero exit code, add it to the list of
+      //failed containers.
+      if (!completedProcessorID.equals(INVALID_PROCESSOR_ID)) {
+        if (state.runningProcessors.containsKey(completedProcessorID)) {
+          log.info("Removing Processor ID: {} from YarnClusterResourceManager running processors.", completedProcessorID);
+          state.runningProcessors.remove(completedProcessorID);
+
+          if (status.getExitStatus() != ContainerExitStatus.SUCCESS)
+            state.failedContainersStatus.put(status.getContainerId().toString(), status);
+        }
+      }
+    }
+    clusterManagerCallback.onResourcesCompleted(samzaResourceStatuses);
+  }
+
+  /**
+   * Callback invoked from Yarn when containers are allocated. This translates the yarn callbacks into Samza
+   * specific ones.
+   * @param containers the list of {@link Container} returned by Yarn.
+   */
+  @Override
+  public void onContainersAllocated(List<Container> containers) {
+    List<SamzaResource> resources = new ArrayList<SamzaResource>();
+    for (Container container : containers) {
+      log.info("Got allocation notification for Container ID: {} on host: {}", container.getId(),
+          container.getNodeId().getHost());
+      String containerId = container.getId().toString();
+      String host = container.getNodeId().getHost();
+      int memory = container.getResource().getMemory();
+      int numCores = container.getResource().getVirtualCores();
+
+      SamzaResource resource = new SamzaResource(numCores, memory, host, containerId);
+      allocatedResources.put(resource, container);
+      resources.add(resource);
+    }
+    clusterManagerCallback.onResourcesAvailable(resources);
+  }
+
+  //The below methods are specific to the Yarn AMRM Client. We currently don't handle scenarios where there are
+  //nodes being updated. We always return 0 when asked for progress by Yarn.
+  @Override
+  public void onShutdownRequest() {
+    stop(SamzaApplicationState.SamzaAppStatus.FAILED);
+  }
+
+  @Override
+  public void onNodesUpdated(List<NodeReport> updatedNodes) {
+    //not implemented currently.
+  }
+
+  @Override
+  public float getProgress() {
+    //not implemented currently.
+    return 0;
+  }
+
+  /**
+   * Callback invoked when there is an error in the Yarn client. This delegates the callback handling to
+   * the {@link org.apache.samza.clustermanager.ClusterResourceManager.Callback} instance.
+   *
+   */
+  @Override
+  public void onError(Throwable e) {
+    log.error("Exception in the Yarn callback", e);
+    clusterManagerCallback.onError(e);
+  }
+
+  @Override
+  public void onContainerStarted(ContainerId containerId, Map<String, ByteBuffer> allServiceResponse) {
+    handleOnContainerStarted(containerId);
+  }
+
+  @Override
+  public void onContainerStatusReceived(ContainerId containerId, ContainerStatus containerStatus) {
+    log.info("Got status notification for Container ID: {} for Processor ID: {}. Status: {}",
+        containerId, getRunningProcessorId(containerId.toString()), containerStatus.getState());
+  }
+
+  @Override
+  public void onContainerStopped(ContainerId containerId) {
+    log.info("Got stop notification for Container ID: {} for Processor ID: {}",
+        containerId, getRunningProcessorId(containerId.toString()));
+  }
+
+  @Override
+  public void onStartContainerError(ContainerId containerId, Throwable t) {
+    String processorId = getPendingProcessorId(containerId);
+
+    if (processorId != null) {
+      log.info("Got start error notification for Container ID: {} for Processor ID: {} ", containerId, processorId, t);
+      YarnContainer container = state.pendingProcessors.remove(processorId);
+      SamzaResource resource = new SamzaResource(container.resource().getVirtualCores(),
+          container.resource().getMemory(), container.nodeId().getHost(), containerId.toString());
+      clusterManagerCallback.onStreamProcessorLaunchFailure(resource, new ProcessorLaunchException(t));
+    } else {
+      log.warn("Did not find the pending Processor ID for the start error notification for Container ID: {}. " +
+          "Ignoring notification", containerId);
+    }
+  }
+
+  @Override
+  public void onGetContainerStatusError(ContainerId containerId, Throwable t) {
+    log.info("Got status error notification for Container ID: {} for Processor ID: {}",
+        containerId, getRunningProcessorId(containerId.toString()), t);
+  }
+
+  @Override
+  public void onStopContainerError(ContainerId containerId, Throwable t) {
+    String processorId = getRunningProcessorId(containerId.toString());
+
+    if (processorId != null) {
+      log.info("Got stop error notification for Container ID: {} for Processor ID: {}", containerId, processorId, t);
+      YarnContainer container = state.runningProcessors.get(processorId);
+      SamzaResource resource = new SamzaResource(container.resource().getVirtualCores(),
+          container.resource().getMemory(), container.nodeId().getHost(), containerId.toString());
+      clusterManagerCallback.onStreamProcessorStopFailure(resource, t);
+    } else {
+      log.warn("Did not find the running Processor ID for the stop error notification for Container ID: {}. " +
+          "Ignoring notification", containerId);
+    }
+  }
+
+  @Override
+  public boolean isResourceExpired(SamzaResource resource) {
+    // Time from which resource was allocated > Yarn Expiry Timeout - 30 sec (to account for clock skew)
+    Duration yarnAllocatedResourceExpiry =
+        Duration.ofMillis(YarnConfiguration.DEFAULT_RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS)
+            .minus(Duration.ofSeconds(30));
+    return System.currentTimeMillis() - resource.getTimestamp() > yarnAllocatedResourceExpiry.toMillis();
+  }
+
+  /**
+   * Runs a process as specified by the command builder on the container.
+   * @param processorId id of the samza processor to run (passed as a command line parameter to the process)
+   * @param container the yarn container to run the processor on.
+   * @param cmdBuilder the command builder that encapsulates the command, and the context
+   * @throws IOException on IO exceptions running the container
+   */
+  public void runProcessor(String processorId, Container container, CommandBuilder cmdBuilder) throws IOException {
+    String containerIdStr = ConverterUtils.toString(container.getId());
+    String cmdPath = "./__package/";
+    cmdBuilder.setCommandPath(cmdPath);
+    String command = cmdBuilder.buildCommand();
+
+    Map<String, String> env = getEscapedEnvironmentVariablesMap(cmdBuilder);
+    env.put(ShellCommandConfig.ENV_EXECUTION_ENV_CONTAINER_ID, Util.envVarEscape(container.getId().toString()));
+
+    Path packagePath = new Path(yarnConfig.getPackagePath());
+    String formattedCommand =
+        getFormattedCommand(ApplicationConstants.LOG_DIR_EXPANSION_VAR, command, ApplicationConstants.STDOUT,
+            ApplicationConstants.STDERR);
+
+    log.info("Running Processor ID: {} on Container ID: {} on host: {} using command: {} and env: {} and package path: {}",
+        processorId, containerIdStr, container.getNodeHttpAddress(), formattedCommand, env, packagePath);
+    state.pendingProcessors.put(processorId, new YarnContainer(container));
+
+    startContainer(packagePath, container, env, formattedCommand);
+
+    log.info("Made start request for Processor ID: {} on Container ID: {} on host: {} (http://{}/node/containerlogs/{}).",
+        processorId, containerIdStr, container.getNodeId().getHost(), container.getNodeHttpAddress(), containerIdStr);
+  }
+
+  /**
+   * Runs a command as a process on the container. All binaries needed by the physical process are packaged in the URL
+   * specified by packagePath.
+   */
+  private void startContainer(Path packagePath,
+                              Container container,
+                              Map<String, String> env,
+                              final String cmd) throws IOException {
+    LocalResource packageResource = Records.newRecord(LocalResource.class);
+    URL packageUrl = ConverterUtils.getYarnUrlFromPath(packagePath);
+    FileStatus fileStatus;
+    fileStatus = packagePath.getFileSystem(yarnConfiguration).getFileStatus(packagePath);
+    packageResource.setResource(packageUrl);
+    log.debug("Set package resource in YarnContainerRunner for {}", packageUrl);
+    packageResource.setSize(fileStatus.getLen());
+    packageResource.setTimestamp(fileStatus.getModificationTime());
+    packageResource.setType(LocalResourceType.ARCHIVE);
+    packageResource.setVisibility(LocalResourceVisibility.APPLICATION);
+
+    ByteBuffer allTokens;
+    // copy tokens to start the container
+    Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();
+    DataOutputBuffer dob = new DataOutputBuffer();
+    credentials.writeTokenStorageToStream(dob);
+
+    // now remove the AM->RM token so that containers cannot access it
+    Iterator iter = credentials.getAllTokens().iterator();
+    while (iter.hasNext()) {
+      TokenIdentifier token = ((org.apache.hadoop.security.token.Token) iter.next()).decodeIdentifier();
+      if (token != null && token.getKind().equals(AMRMTokenIdentifier.KIND_NAME)) {
+        iter.remove();
+      }
+    }
+    allTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+
+    Map<String, LocalResource> localResourceMap = new HashMap<>();
+    localResourceMap.put("__package", packageResource);
+
+    // include the resources from the universal resource configurations
+    LocalizerResourceMapper resourceMapper = new LocalizerResourceMapper(new LocalizerResourceConfig(config), yarnConfiguration);
+    localResourceMap.putAll(resourceMapper.getResourceMap());
+
+    ContainerLaunchContext context = Records.newRecord(ContainerLaunchContext.class);
+    context.setEnvironment(env);
+    context.setTokens(allTokens.duplicate());
+    context.setCommands(new ArrayList<String>() {
+      {
+        add(cmd);
+      }
+    });
+    context.setLocalResources(localResourceMap);
+
+    if (UserGroupInformation.isSecurityEnabled()) {
+      Map<ApplicationAccessType, String> acls = yarnConfig.getYarnApplicationAcls();
+      if (!acls.isEmpty()) {
+        context.setApplicationACLs(acls);
+      }
+    }
+
+    log.debug("Setting localResourceMap to {}", localResourceMap);
+    log.debug("Setting context to {}", context);
+
+    StartContainerRequest startContainerRequest = Records.newRecord(StartContainerRequest.class);
+    startContainerRequest.setContainerLaunchContext(context);
+
+    log.info("Making an async start request for Container ID: {} on host: {} with local resource map: {} and context: {}",
+        container.getId(), container.getNodeHttpAddress(), localResourceMap.toString(), context);
+    nmClientAsync.startContainerAsync(container, context);
+  }
+
+  /**
+   * Gets the environment variables from the specified {@link CommandBuilder} and escapes certain characters.
+   *
+   * @param cmdBuilder        the command builder containing the environment variables.
+   * @return                  the map containing the escaped environment variables.
+   */
+  private Map<String, String> getEscapedEnvironmentVariablesMap(CommandBuilder cmdBuilder) {
+    Map<String, String> env = new HashMap<String, String>();
+    for (Map.Entry<String, String> entry : cmdBuilder.buildEnvironment().entrySet()) {
+      String escapedValue = Util.envVarEscape(entry.getValue());
+      env.put(entry.getKey(), escapedValue);
+    }
+    return env;
+  }
+
+
+  private String getFormattedCommand(String logDirExpansionVar, String command, String stdOut, String stdErr) {
+    return String.format("export SAMZA_LOG_DIR=%s && ln -sfn %s logs && exec %s 1>logs/%s 2>logs/%s",
+        logDirExpansionVar, logDirExpansionVar, command, stdOut, stdErr);
+  }
+
+  /**
+   * Returns the Id of the Samza container that corresponds to the provided Yarn {@link ContainerId}
+   * @param containerId the Yarn ContainerId
+   * @return the id of the Samza container corresponding to the {@link ContainerId} that is pending launch
+   */
+  private String getPendingProcessorId(ContainerId containerId) {
+    for (String pendingProcessorId: state.pendingProcessors.keySet()) {
+      YarnContainer yarnContainer = state.pendingProcessors.get(pendingProcessorId);
+      if (yarnContainer != null && yarnContainer.id().equals(containerId)) {
+        return pendingProcessorId;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Handles container started call back for a yarn container.
+   * updates the YarnAppState's pendingProcessors and runningProcessors
+   * and also invokes clusterManagerCallback.s stream processor launch success
+   * @param containerId yarn container id which has started
+   */
+  private void handleOnContainerStarted(ContainerId containerId) {
+    String processorId = getPendingProcessorId(containerId);
+    if (processorId != null) {
+      log.info("Got start notification for Container ID: {} for Processor ID: {}", containerId, processorId);
+      // 1. Move the processor from pending to running state
+      final YarnContainer container = state.pendingProcessors.remove(processorId);
+
+      state.runningProcessors.put(processorId, container);
+
+      // 2. Invoke the success callback.
+      SamzaResource resource = new SamzaResource(container.resource().getVirtualCores(),
+          container.resource().getMemory(), container.nodeId().getHost(), containerId.toString());
+      clusterManagerCallback.onStreamProcessorLaunchSuccess(resource);
+    } else {
+      log.warn("Did not find the Processor ID for the start notification for Container ID: {}. " +
+          "Ignoring notification.", containerId);
+    }
+  }
+
+  @VisibleForTesting
+  ConcurrentHashMap<SamzaResource, Container> getAllocatedResources() {
+    return allocatedResources;
+  }
+}
diff --git a/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnFaultDomainManager.java b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnFaultDomainManager.java
new file mode 100644
index 000000000..2bafa781d
--- /dev/null
+++ b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnFaultDomainManager.java
@@ -0,0 +1,131 @@
+/*
+ * 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.samza.job.yarn;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.samza.SamzaException;
+import org.apache.samza.clustermanager.FaultDomain;
+import org.apache.samza.clustermanager.FaultDomainManager;
+import org.apache.samza.clustermanager.FaultDomainType;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class functionality works with the assumption that the job.standbytasks.replication.factor is 2.
+ * For values greater than 2, it is possible that the standby containers could be on the same rack as the active, or the already existing standby racks.
+ */
+public class YarnFaultDomainManager implements FaultDomainManager {
+
+  private static final Logger log = LoggerFactory.getLogger(FaultDomainManager.class);
+  private static final String FAULT_DOMAIN_MANAGER_GROUP = "yarn-fault-domain-manager";
+  private static final String HOST_TO_FAULT_DOMAIN_CACHE_UPDATES = "host-to-fault-domain-cache-updates";
+  private Multimap<String, FaultDomain> hostToRackMap;
+  private final YarnClientImpl yarnClient;
+  private Counter hostToFaultDomainCacheUpdates;
+
+  public YarnFaultDomainManager(MetricsRegistry metricsRegistry) {
+    this.yarnClient = new YarnClientImpl();
+    yarnClient.init(new YarnConfiguration());
+    yarnClient.start();
+    this.hostToRackMap = computeHostToFaultDomainMap();
+    hostToFaultDomainCacheUpdates = metricsRegistry.newCounter(FAULT_DOMAIN_MANAGER_GROUP, HOST_TO_FAULT_DOMAIN_CACHE_UPDATES);
+  }
+
+  @VisibleForTesting
+  YarnFaultDomainManager(MetricsRegistry metricsRegistry, YarnClientImpl yarnClient, Multimap<String, FaultDomain> hostToRackMap) {
+    this.yarnClient = yarnClient;
+    yarnClient.init(new YarnConfiguration());
+    yarnClient.start();
+    this.hostToRackMap = hostToRackMap;
+    hostToFaultDomainCacheUpdates = metricsRegistry.newCounter(FAULT_DOMAIN_MANAGER_GROUP, HOST_TO_FAULT_DOMAIN_CACHE_UPDATES);
+  }
+
+  /**
+   * This method returns all the last cached rack values in a cluster, for all hosts that are healthy, up and running.
+   * @return a set of {@link FaultDomain}s
+   */
+  @Override
+  public Set<FaultDomain> getAllFaultDomains() {
+    return new HashSet<>(hostToRackMap.values());
+  }
+
+  /**
+   * This method returns the rack a particular host resides on based on the internal cache.
+   * In case the rack of a host does not exist in this cache, we update the cache by computing the host to rack map again using Yarn.
+   * @param host the host
+   * @return the {@link FaultDomain}
+   */
+  @Override
+  public Set<FaultDomain> getFaultDomainsForHost(String host) {
+    if (!hostToRackMap.containsKey(host)) {
+      hostToRackMap = computeHostToFaultDomainMap();
+      hostToFaultDomainCacheUpdates.inc();
+    }
+    return new HashSet<>(hostToRackMap.get(host));
+  }
+
+  /**
+   * This method checks if the two hostnames provided reside on the same rack.
+   * @param host1 hostname
+   * @param host2 hostname
+   * @return true if the hosts exist on the same rack
+   */
+  @Override
+  public boolean hasSameFaultDomains(String host1, String host2) {
+    if (!hostToRackMap.keySet().contains(host1) || !hostToRackMap.keySet().contains(host2)) {
+      hostToRackMap = computeHostToFaultDomainMap();
+      hostToFaultDomainCacheUpdates.inc();
+    }
+    return hostToRackMap.get(host1).equals(hostToRackMap.get(host2));
+  }
+
+  /**
+   * This method computes the host to rack map from Yarn.
+   * Only the hosts that are running in the cluster will be a part of this map.
+   * @return map of the host and the rack it resides on
+   */
+  @VisibleForTesting
+  Multimap<String, FaultDomain> computeHostToFaultDomainMap() {
+    Multimap<String, FaultDomain> hostToRackMap = HashMultimap.create();
+    try {
+      List<NodeReport> nodeReport = yarnClient.getNodeReports(NodeState.RUNNING);
+      nodeReport.forEach(report -> {
+        FaultDomain rack = new FaultDomain(FaultDomainType.RACK, report.getRackName());
+        hostToRackMap.put(report.getNodeId().getHost(), rack);
+      });
+      log.info("Computed the host to rack map successfully from Yarn.");
+    } catch (YarnException | IOException e) {
+      throw new SamzaException("Yarn threw an exception while getting NodeReports.", e);
+    }
+    return hostToRackMap;
+  }
+}
diff --git a/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnFaultDomainManagerFactory.java b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnFaultDomainManagerFactory.java
new file mode 100644
index 000000000..e4e547c49
--- /dev/null
+++ b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnFaultDomainManagerFactory.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.job.yarn;
+
+import org.apache.samza.clustermanager.FaultDomainManager;
+import org.apache.samza.clustermanager.FaultDomainManagerFactory;
+import org.apache.samza.config.Config;
+import org.apache.samza.metrics.MetricsRegistry;
+
+/**
+ * A factory to build a {@link YarnFaultDomainManager}.
+ */
+public class YarnFaultDomainManagerFactory implements FaultDomainManagerFactory {
+  @Override
+  public FaultDomainManager getFaultDomainManager(Config config, MetricsRegistry metricsRegistry) {
+    return new YarnFaultDomainManager(metricsRegistry);
+  }
+}
diff --git a/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnResourceManagerFactory.java b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnResourceManagerFactory.java
new file mode 100644
index 000000000..3f9a84dd5
--- /dev/null
+++ b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnResourceManagerFactory.java
@@ -0,0 +1,44 @@
+/*
+ * 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.samza.job.yarn;
+
+import org.apache.samza.clustermanager.ClusterResourceManager;
+import org.apache.samza.clustermanager.ResourceManagerFactory;
+import org.apache.samza.clustermanager.SamzaApplicationState;
+import org.apache.samza.config.Config;
+import org.apache.samza.coordinator.JobModelManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A YarnContainerProcessManagerFactory returns an implementation of a {@link ClusterResourceManager} for Yarn.
+ */
+public class YarnResourceManagerFactory implements ResourceManagerFactory {
+
+  private static Logger log = LoggerFactory.getLogger(YarnResourceManagerFactory.class);
+
+  @Override
+  public ClusterResourceManager getClusterResourceManager(ClusterResourceManager.Callback callback, SamzaApplicationState state) {
+    log.info("Creating an instance of a cluster resource manager for Yarn. ");
+    JobModelManager jobModelManager = state.jobModelManager;
+    Config config = jobModelManager.jobModel().getConfig();
+    YarnClusterResourceManager manager = new YarnClusterResourceManager(config, jobModelManager, callback, state);
+    return manager;
+  }
+}
diff --git a/samza-yarn3/src/main/java/org/apache/samza/validation/YarnJobValidationTool.java b/samza-yarn3/src/main/java/org/apache/samza/validation/YarnJobValidationTool.java
new file mode 100644
index 000000000..74fef67bd
--- /dev/null
+++ b/samza-yarn3/src/main/java/org/apache/samza/validation/YarnJobValidationTool.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.validation;
+
+import joptsimple.OptionParser;
+import joptsimple.OptionSet;
+import joptsimple.OptionSpec;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.container.LocalityManager;
+import org.apache.samza.coordinator.metadatastore.CoordinatorStreamStore;
+import org.apache.samza.coordinator.metadatastore.NamespaceAwareCoordinatorStreamStore;
+import org.apache.samza.coordinator.stream.messages.SetConfig;
+import org.apache.samza.job.model.ProcessorLocality;
+import org.apache.samza.job.model.LocalityModel;
+import org.apache.samza.job.yarn.ClientHelper;
+import org.apache.samza.metrics.JmxMetricsAccessor;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.metrics.MetricsRegistryMap;
+import org.apache.samza.metrics.MetricsValidator;
+import org.apache.samza.util.CommandLine;
+import org.apache.samza.util.ReflectionUtil;
+import org.apache.samza.util.hadoop.HttpFileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Command-line tool for validating the status of a Yarn job.
+ * It checks the job has been successfully submitted to the Yarn cluster, the status of
+ * the application attempt is running and the running container count matches the expectation.
+ * It also supports an optional MetricsValidator plugin through arguments so job metrics can
+ * be validated too using JMX. This tool can be used, for example, as an automated validation
+ * step after starting a job.
+ *
+ * When running this tool, please provide the configuration URI of job. For example:
+ *
+ * deploy/samza/bin/validate-yarn-job.sh --config job.config.loader.factory=org.apache.samza.config.loaders.PropertiesConfigLoaderFactory --config job.config.loader.properties.path=$PWD/deploy/samza/config/wikipedia-feed.properties [--metrics-validator=com.foo.bar.SomeMetricsValidator]
+ *
+ * The tool prints out the validation result in each step and throws an exception when the
+ * validation fails.
+ */
+public class YarnJobValidationTool {
+  private static final Logger log = LoggerFactory.getLogger(YarnJobValidationTool.class);
+
+  private final JobConfig config;
+  private final YarnClient client;
+  private final String jobName;
+  private final MetricsValidator validator;
+
+  public YarnJobValidationTool(JobConfig config, YarnClient client, MetricsValidator validator) {
+    this.config = config;
+    this.client = client;
+    String name = this.config.getName().get();
+    String jobId = this.config.getJobId();
+    this.jobName =  name + "_" + jobId;
+    this.validator = validator;
+  }
+
+  public void run() {
+    ApplicationId appId;
+    ApplicationAttemptId attemptId;
+
+    try {
+      log.info("Start validating job " + this.jobName);
+
+      appId = validateAppId();
+      attemptId = validateRunningAttemptId(appId);
+      validateContainerCount(attemptId);
+      if (validator != null) {
+        validateJmxMetrics();
+      }
+
+      log.info("End of validation");
+    } catch (Exception e) {
+      log.error(e.getMessage(), e);
+      System.exit(1);
+    }
+  }
+
+  public ApplicationId validateAppId() throws Exception {
+    // fetch only the last created application with the job name and id
+    // i.e. get the application with max appId
+    ApplicationId appId = null;
+    for (ApplicationReport applicationReport : this.client.getApplications()) {
+      if (applicationReport.getName().equals(this.jobName)) {
+        ApplicationId id = applicationReport.getApplicationId();
+        if (appId == null || appId.compareTo(id) < 0) {
+          appId = id;
+        }
+      }
+    }
+    if (appId != null) {
+      log.info("Job lookup success. ApplicationId " + appId.toString());
+      return appId;
+    } else {
+      throw new SamzaException("Job lookup failure " + this.jobName);
+    }
+  }
+
+  public ApplicationAttemptId validateRunningAttemptId(ApplicationId appId) throws Exception {
+    ApplicationAttemptId attemptId = this.client.getApplicationReport(appId).getCurrentApplicationAttemptId();
+    ApplicationAttemptReport attemptReport = this.client.getApplicationAttemptReport(attemptId);
+    if (attemptReport.getYarnApplicationAttemptState() == YarnApplicationAttemptState.RUNNING) {
+      log.info("Job is running. AttempId " + attemptId.toString());
+      return attemptId;
+    } else {
+      throw new SamzaException("Job not running " + this.jobName);
+    }
+  }
+
+  public int validateContainerCount(ApplicationAttemptId attemptId) throws Exception {
+    int runningContainerCount = 0;
+    for (ContainerReport containerReport : this.client.getContainers(attemptId)) {
+      if (containerReport.getContainerState() == ContainerState.RUNNING) {
+        ++runningContainerCount;
+      }
+    }
+    // expected containers to be the configured job containers plus the AppMaster container
+    int containerExpected = this.config.getContainerCount() + 1;
+
+    if (runningContainerCount == containerExpected) {
+      log.info("Container count matches. " + runningContainerCount + " containers are running.");
+      return runningContainerCount;
+    } else {
+      throw new SamzaException("Container count does not match. " + runningContainerCount + " containers are running, while " + containerExpected + " is expected.");
+    }
+  }
+
+  public void validateJmxMetrics() throws Exception {
+    MetricsRegistry metricsRegistry = new MetricsRegistryMap();
+    CoordinatorStreamStore coordinatorStreamStore = new CoordinatorStreamStore(config, metricsRegistry);
+    coordinatorStreamStore.init();
+    try {
+      LocalityManager localityManager =
+          new LocalityManager(new NamespaceAwareCoordinatorStreamStore(coordinatorStreamStore, SetConfig.TYPE));
+      validator.init(config);
+      LocalityModel localityModel = localityManager.readLocality();
+
+      for (ProcessorLocality processorLocality : localityModel.getProcessorLocalities().values()) {
+        String containerId = processorLocality.id();
+        String jmxUrl = processorLocality.jmxTunnelingUrl();
+        if (StringUtils.isNotBlank(jmxUrl)) {
+          log.info("validate container " + containerId + " metrics with JMX: " + jmxUrl);
+          JmxMetricsAccessor jmxMetrics = new JmxMetricsAccessor(jmxUrl);
+          jmxMetrics.connect();
+          validator.validate(jmxMetrics);
+          jmxMetrics.close();
+          log.info("validate container " + containerId + " successfully");
+        }
+      }
+
+      validator.complete();
+    } finally {
+      coordinatorStreamStore.close();
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    CommandLine cmdline = new CommandLine();
+    OptionParser parser = cmdline.parser();
+    OptionSpec<String> validatorOpt = parser.accepts("metrics-validator", "The metrics validator class.")
+                                            .withOptionalArg()
+                                            .ofType(String.class).describedAs("com.foo.bar.ClassName");
+    OptionSet options = cmdline.parser().parse(args);
+    Config config = cmdline.loadConfig(options);
+    MetricsValidator validator = null;
+    if (options.has(validatorOpt)) {
+      String validatorClass = options.valueOf(validatorOpt);
+      validator = ReflectionUtil.getObj(validatorClass, MetricsValidator.class);
+    }
+
+    YarnConfiguration hadoopConfig = new YarnConfiguration();
+    hadoopConfig.set("fs.http.impl", HttpFileSystem.class.getName());
+    hadoopConfig.set("fs.https.impl", HttpFileSystem.class.getName());
+    ClientHelper clientHelper = new ClientHelper(hadoopConfig);
+
+    new YarnJobValidationTool(new JobConfig(config), clientHelper.yarnClient(), validator).run();
+  }
+}
\ No newline at end of file
diff --git a/samza-yarn3/src/main/java/org/apache/samza/webapp/ApplicationMasterRestClient.java b/samza-yarn3/src/main/java/org/apache/samza/webapp/ApplicationMasterRestClient.java
new file mode 100644
index 000000000..f890d2b6a
--- /dev/null
+++ b/samza-yarn3/src/main/java/org/apache/samza/webapp/ApplicationMasterRestClient.java
@@ -0,0 +1,111 @@
+/*
+ * 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.samza.webapp;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.core.type.TypeReference;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.http.HttpHost;
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpStatus;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.util.EntityUtils;
+import org.apache.samza.SamzaException;
+import org.apache.samza.serializers.model.SamzaObjectMapper;
+
+
+/**
+ * Client for the {@link ApplicationMasterRestServlet}.
+ */
+public class ApplicationMasterRestClient implements Closeable {
+  private final CloseableHttpClient httpClient;
+  private final HttpHost appMasterHost;
+  private final ObjectMapper jsonMapper = SamzaObjectMapper.getObjectMapper();
+
+  public ApplicationMasterRestClient(CloseableHttpClient client, String amHostName, int amRpcPort) {
+    httpClient = client;
+    appMasterHost = new HttpHost(amHostName, amRpcPort);
+  }
+
+  /**
+   * @return  the metrics as a map of groupName to metricName to metricValue.
+   * @throws IOException if there was an error fetching the metrics from the servlet.
+   */
+  public Map<String, Map<String, Object>> getMetrics() throws IOException {
+    String jsonString = getEntityAsJson("/metrics", "metrics");
+    return jsonMapper.readValue(jsonString, new TypeReference<Map<String, Map<String, Object>>>() { });
+  }
+
+  /**
+   * @return  the task context as a map of key to value
+   * @throws IOException if there was an error fetching the task context from the servlet.
+   */
+  public Map<String, Object> getTaskContext() throws IOException {
+    String jsonString = getEntityAsJson("/task-context", "task context");
+    return jsonMapper.readValue(jsonString, new TypeReference<Map<String, Object>>() { });
+  }
+
+  /**
+   * @return  the AM state as a map of key to value
+   * @throws IOException if there was an error fetching the AM state from the servlet.
+   */
+  public Map<String, Object> getAmState() throws IOException {
+    String jsonString = getEntityAsJson("/am", "AM state");
+    return jsonMapper.readValue(jsonString, new TypeReference<Map<String, Object>>() { });
+  }
+
+  /**
+   * @return  the config as a map of key to value
+   * @throws IOException if there was an error fetching the config from the servlet.
+   */
+  public Map<String, Object> getConfig() throws IOException {
+    String jsonString = getEntityAsJson("/config", "config");
+    return jsonMapper.readValue(jsonString, new TypeReference<Map<String, Object>>() { });
+  }
+
+  @Override
+  public void close() throws IOException {
+    httpClient.close();
+  }
+
+  private String getEntityAsJson(String path, String entityName) throws IOException {
+    HttpGet getRequest = new HttpGet(path);
+    HttpResponse httpResponse = httpClient.execute(appMasterHost, getRequest);
+
+    StatusLine status = httpResponse.getStatusLine();
+    if (status.getStatusCode() != HttpStatus.SC_OK) {
+      throw new SamzaException(String.format(
+          "Error retrieving %s from host %s. Response: %s",
+          entityName,
+          appMasterHost.toURI(),
+          status.getReasonPhrase()));
+    }
+
+    return EntityUtils.toString(httpResponse.getEntity());
+  }
+
+  @Override
+  public String toString() {
+    return "AppMasterClient for uri: " + appMasterHost.toURI().toString();
+  }
+}
diff --git a/samza-yarn3/src/main/java/org/apache/samza/webapp/YarnContainerHeartbeatServlet.java b/samza-yarn3/src/main/java/org/apache/samza/webapp/YarnContainerHeartbeatServlet.java
new file mode 100644
index 000000000..5b1f28229
--- /dev/null
+++ b/samza-yarn3/src/main/java/org/apache/samza/webapp/YarnContainerHeartbeatServlet.java
@@ -0,0 +1,92 @@
+/*
+ * 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.samza.webapp;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.samza.container.ContainerHeartbeatResponse;
+import org.apache.samza.coordinator.CoordinationConstants;
+import org.apache.samza.job.yarn.SamzaAppMasterMetrics;
+import org.apache.samza.job.yarn.YarnAppState;
+import org.apache.samza.job.yarn.YarnContainer;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Responds to heartbeat requests from the containers with a {@link ContainerHeartbeatResponse}.
+ * The heartbeat request contains the <code> executionContainerId </code>
+ * which in YARN's case is the YARN container Id.
+ * This servlet validates the container Id against the list
+ * of running containers maintained in the {@link YarnAppState}.
+ * The returned {@link ContainerHeartbeatResponse#isAlive()} is
+ * <code> true </code> iff. the container Id exists in {@link YarnAppState#runningProcessors}.
+ */
+public class YarnContainerHeartbeatServlet extends HttpServlet {
+
+  private static final Logger LOG = LoggerFactory.getLogger(YarnContainerHeartbeatServlet.class);
+  private static final String APPLICATION_JSON = "application/json";
+  private static final String GROUP = SamzaAppMasterMetrics.class.getName();
+  private final Counter heartbeatsExpiredCount;
+
+  private YarnAppState yarnAppState;
+  private ObjectMapper mapper;
+
+  public YarnContainerHeartbeatServlet(YarnAppState yarnAppState, ReadableMetricsRegistry registry) {
+    this.yarnAppState = yarnAppState;
+    this.mapper = new ObjectMapper();
+    this.heartbeatsExpiredCount = registry.newCounter(GROUP, "heartbeats-expired");
+  }
+
+  @Override
+  protected void doGet(HttpServletRequest req, HttpServletResponse resp)
+      throws ServletException, IOException {
+    ContainerId yarnContainerId;
+    PrintWriter printWriter = resp.getWriter();
+    String containerIdParam = req.getParameter(CoordinationConstants.YARN_EXECUTION_ENVIRONMENT_CONTAINER_ID);
+    ContainerHeartbeatResponse response;
+    resp.setContentType(APPLICATION_JSON);
+    boolean alive = false;
+    try {
+      yarnContainerId = ContainerId.fromString(containerIdParam);
+      for (YarnContainer yarnContainer : yarnAppState.runningProcessors.values()) {
+        if (yarnContainer.id().compareTo(yarnContainerId) == 0) {
+          alive = true;
+          break;
+        }
+      }
+      if (!alive) {
+        heartbeatsExpiredCount.inc();
+      }
+      response = new ContainerHeartbeatResponse(alive);
+      printWriter.write(mapper.writeValueAsString(response));
+    } catch (IllegalArgumentException e) {
+      LOG.error("Container ID {} passed is invalid", containerIdParam);
+      resp.sendError(HttpServletResponse.SC_BAD_REQUEST, e.getMessage());
+    }
+  }
+}
diff --git a/samza-yarn3/src/main/less/main.less b/samza-yarn3/src/main/less/main.less
new file mode 100644
index 000000000..c826ecb42
--- /dev/null
+++ b/samza-yarn3/src/main/less/main.less
@@ -0,0 +1,164 @@
+// 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.
+
+@import "variables.less";
+
+html, body {
+  height: 100%;
+}
+
+h1 {
+  font-size: 24px;
+}
+
+h2 {
+  font-size: 20px;
+}
+
+h3 {
+  font-size: 18px;
+}
+
+h4 {
+  font-size: 16px;
+}
+
+@media (min-width: 1200px) {
+  .container {
+    width: 1270px;
+  }
+}
+
+.page-header {
+  margin-bottom: 5px !important;
+  margin-top: 10px !important;
+  h1 {
+    margin-bottom: 0px;
+  }
+}
+
+.wrapper {
+  min-height: 100%;
+
+  .wrapper-content {
+    overflow: auto;
+    padding-bottom: 120px;
+  }
+}
+
+.masthead {
+	background-color: @masthead-background-color;
+
+  .masthead-logo {
+    display: inline-block;
+    background-color: @masthead-logo-color;
+    font-family: 'Ropa Sans', sans-serif;
+    font-size: 3em;
+    color: #fff;
+
+    a {
+      color: #fff;
+      &:hover {
+        text-decoration: none;
+      }
+    }
+  }
+
+  .masthead-icons {
+    font-size: 3em;
+
+    .masthead-icon {
+      margin-left: .2em;
+      color: @masthead-icon-color;
+    }
+  }
+}
+
+pre {
+  border: 0px !important;
+  border-radius: 0px !important;
+}
+
+th.header {
+  cursor: pointer;
+  background-image: url(../img/bg.gif);
+  background-repeat: no-repeat;
+  background-position: center right;
+  padding-right: 15px !important;
+}
+
+th.headerSortUp {
+  background-color: #f5f5f5;
+  background-image: url(../img/desc.gif);
+}
+
+th.headerSortDown {
+  background-color: #f5f5f5;
+  background-image: url(../img/asc.gif);
+}
+
+td.key {
+  width: 25%;
+  font-weight: bold;
+}
+
+div.value {
+    max-height: 200px;
+    overflow-y: scroll;
+}
+
+.menu {
+  padding-top: 15px;
+
+  .nav {
+    margin-left: -15px;
+    margin-right: -15px;
+  }
+
+  .nav-pills > li > a {
+    border-radius: 0px;
+  }
+
+  .nav-pills > li.active > a,
+  .nav-pills > li.active > a:hover,
+  .nav-pills > li.active > a:focus {
+    //background-color: #f0f0f0;
+  }
+
+  h1 {
+    font-size: 16px;
+    margin: 10px 0px;
+  }
+
+  ul {
+    list-style: none;
+    padding: 0;
+
+    li {
+      line-height: 150%;
+    }
+  }
+}
+
+.footer {
+  clear: both;
+  position: relative;
+	background-color: @masthead-background-color;
+  z-index: 10;
+  height: 100px;
+  margin-top: -100px;
+}
diff --git a/samza-yarn3/src/main/less/variables.less b/samza-yarn3/src/main/less/variables.less
new file mode 100644
index 000000000..13c5b920c
--- /dev/null
+++ b/samza-yarn3/src/main/less/variables.less
@@ -0,0 +1,20 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+@masthead-background-color: #F5F4F0;
+@masthead-logo-color: #ff0000;
+@masthead-icon-color: #d3d2d0;
diff --git a/samza-yarn3/src/main/resources/scalate/WEB-INF/layouts/default.scaml b/samza-yarn3/src/main/resources/scalate/WEB-INF/layouts/default.scaml
new file mode 100644
index 000000000..7ddccac08
--- /dev/null
+++ b/samza-yarn3/src/main/resources/scalate/WEB-INF/layouts/default.scaml
@@ -0,0 +1,48 @@
+-#
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing,
+  software distributed under the License is distributed on an
+  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  KIND, either express or implied.  See the License for the
+  specific language governing permissions and limitations
+  under the License.
+-@ val title: String
+-@ val body: String
+
+!!!
+%html
+  %head
+    %title= title
+    %link(href="css/ropa-sans.css" rel="stylesheet" type="text/css")
+    %link(href="css/bootstrap.min.css" rel="stylesheet" type="text/css")
+    %link(href="css/font-awesome.min.css" rel="stylesheet")
+    %link(href="css/main.css" rel="stylesheet")
+    %link(href="img/samza-icon.png" rel="icon" type="image/png")
+
+    %script(src="js/jquery-1.11.1.min.js")
+    %script(src="js/jquery.tablesorter.min.js")
+    %script(src="js/bootstrap.min.js")
+  %body
+    %div.wrapper
+      %div.wrapper-content
+
+        %div.masthead
+          %div.container
+            %div.col-xs-2.masthead-logo
+              %a.logo(href="/") samza
+            %div.col-xs-10.masthead-icons
+
+        %div.container
+          != body
+
+    %div.footer
+      %div.container
diff --git a/samza-yarn3/src/main/resources/scalate/WEB-INF/views/index.scaml b/samza-yarn3/src/main/resources/scalate/WEB-INF/views/index.scaml
new file mode 100644
index 000000000..6064d78a4
--- /dev/null
+++ b/samza-yarn3/src/main/resources/scalate/WEB-INF/views/index.scaml
@@ -0,0 +1,239 @@
+-#
+  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.
+-@ val state: org.apache.samza.job.yarn.YarnAppState
+-@ val samzaAppState: org.apache.samza.clustermanager.SamzaApplicationState
+-@ val config: scala.collection.immutable.TreeMap[String, String]
+-@ val rmHttpAddress: String
+-@ val jobName: String = config.get("job.name").getOrElse("MISSING JOB NAME")
+-@ val packagePath: String = config.get("yarn.package.path").getOrElse("MISSING PACKAGE PATH")
+-@ val username: String = org.apache.hadoop.security.UserGroupInformation.getCurrentUser.getShortUserName
+-@ val appMasterClasspath: String = scala.util.Properties.javaClassPath
+-@ val javaVmVersion: String = scala.util.Properties.javaVmVersion
+-@ val javaVmName: String = scala.util.Properties.javaVmName
+-@ val samzaVersion: String = classOf[org.apache.samza.util.Util].getPackage.getImplementationVersion
+- attributes("title") = jobName
+
+%div.col-xs-2.menu
+  %ul.nav.nav-pills.nav-stacked
+    %li.active
+      %a(href="#application-master" data-toggle="tab") Application Master
+    %li
+      %a(href="#containers" data-toggle="tab") Containers
+    %li
+      %a(href="#task-groups" data-toggle="tab") Task Groups
+    %li
+      %a(href="#config" data-toggle="tab") Config
+
+%div.col-xs-10
+  %div.page-header
+    %h1= jobName
+
+  %div.tab-content
+    %div.tab-pane.active#application-master
+      %h2 Application Master
+      %table.table.table-striped.table-bordered
+        %tbody
+          %tr
+            %td.key Hostname
+            %td
+              %a(target="_blank" href="http://#{state.nodeHost}:#{state.nodeHttpPort.toString}")= state.nodeHost
+          %tr
+            %td.key User
+            %td= username
+          %tr
+            %td.key Tracking port
+            %td= state.trackingUrl.getPort.toString
+          %tr
+            %td.key RPC port
+            %td= state.rpcUrl.getPort.toString
+          %tr
+            %td.key Attempt ID
+            %td= state.appAttemptId
+          %tr
+            %td.key Application ID
+            %td= state.appAttemptId.getApplicationId
+          %tr
+            %td.key Application master classpath
+            %td
+              %div.value= appMasterClasspath
+          %tr
+            %td.key Package path
+            %td= packagePath
+          %tr
+            %td.key Java VM name
+            %td= javaVmName
+          %tr
+            %td.key Java VM version
+            %td= javaVmVersion
+          %tr
+            %td.key Samza version
+            %td= samzaVersion
+          %tr
+            %td.key Application master task ID
+            %td= state.taskId
+          %tr
+            %td.key Application master container
+            %td
+              %a(target="_blank" href="http://#{state.nodeHost}:#{state.nodeHttpPort.toString}/node/containerlogs/#{state.amContainerId.toString}/#{username}")= state.amContainerId.toString
+          %tr
+            %td.key JMX server url
+            %td= samzaAppState.jmxUrl
+          %tr
+            %td.key JMX server tunneling url
+            %td= samzaAppState.jmxTunnelingUrl
+
+    %div.tab-pane#containers
+      %h2 Containers
+      %table.table.table-bordered.table-striped
+        %tr
+          %tr
+            %td.key Completed
+            %td= samzaAppState.completedProcessors.toString
+          %tr
+            %td.key Needed
+            %td= samzaAppState.neededProcessors.toString
+          %tr
+            %td.key Failed
+            %td= samzaAppState.failedContainers.toString
+          %tr
+            %td.key Released
+            %td= samzaAppState.releasedContainers.toString
+
+      %h2 Running Containers
+      %table.table.table-striped.table-bordered.tablesorter#containers-table
+        %thead
+          %tr
+            %th Task Group
+            %th Container
+            %th Node
+            %th Start Time
+            %th Up Time
+            %th JMX access
+        %tbody
+          - for((processorId, container) <- state.runningProcessors.asScala)
+            %tr
+              %td #{processorId.toString}
+              %td
+                %a(target="_blank" href="http://#{container.nodeHttpAddress}/node/containerlogs/#{container.id.toString}/#{username}")= container.id.toString
+              %td
+                %a(target="_blank" href="http://#{container.nodeHttpAddress}")= container.nodeHttpAddress
+              %td
+                Start time: #{container.startTimeStr()}
+              %td
+                Up time: #{container.upTimeStr()}
+              %td
+                %a(target="_blank" href="#{state.coordinatorUrl.toString}locality?processorId=#{processorId.toString}") JMX
+
+      %h2 Failed Containers
+      %table.table.table-striped.table-bordered.tablesorter#containers-table
+        %thead
+          %tr
+            %th Container
+            %th Exit code
+            %th Message
+        %tbody
+          - for((containerId, containerStatus) <- state.failedContainersStatus.asScala)
+            %tr
+              %td
+                #{containerId}
+              %td
+                Exit code: #{containerStatus.getExitStatus}
+              %td
+                %div.value= containerStatus.getDiagnostics
+
+    %div.tab-pane#task-groups
+      %h2 Task Groups
+      %table.table.table-striped.table-bordered
+        %tbody
+          %tr
+            %td.key Total
+            %td= samzaAppState.processorCount.toString
+          %tr
+            %td.key Finished
+            %td= samzaAppState.finishedProcessors.toString
+
+      %h3 TaskName Assignment
+      %table.table.table-striped.table-bordered.tablesorter#taskids-table
+        %thead
+          %tr
+            %th Task Group ID
+            %th TaskName
+            %th SystemStreamPartitions
+            %th Container
+        %tbody
+          - for((processorId, container) <- state.runningProcessors.asScala)
+            - val containerModel = samzaAppState.jobModelManager.jobModel.getContainers.get(processorId)
+            - for((taskName, taskModel) <- containerModel.getTasks.asScala)
+              %tr
+                %td= processorId
+                %td= taskName
+                %td= taskModel.getSystemStreamPartitions.asScala.map(_.toString).toList.sorted.mkString(", ")
+                %td
+                  %a(target="_blank" href="http://#{container.nodeHttpAddress}/node/containerlogs/#{container.id.toString}/#{username}")= container.id.toString
+
+    %div.tab-pane#config
+      %h2 Config
+      %div.panel.panel-default
+        %div.panel-heading
+          %input.form-control#config-table-filter(type="text" placeholder="Type '/' to search")
+        %table.table.table-striped.table-bordered.tablesorter#config-table
+          %thead
+            %tr
+              %th Key
+              %th Value
+          %tbody.searchable
+            - for(entrySet <- config.asInstanceOf[Map[String, String]])
+              %tr
+                %td.key= entrySet.getKey
+                %td= entrySet.getValue
+
+    :javascript
+      $(document).ready(function() {
+        // Persist tabs.
+        if (location.hash !== '') {
+          $('a[href="' + location.hash + '"]').tab('show');
+        }
+        $('a[data-toggle="tab"]').on('shown.bs.tab', function(e) {
+          return location.hash = $(e.target).attr('href').substr(1);
+        });
+
+        // Make tables sortable.
+        $('#containers-table').tablesorter();
+        $('#taskids-table').tablesorter();
+        $('#config-table').tablesorter();
+
+        // Type '/' to search.
+        $(document).keyup(function(e) {
+          if (e.keyCode == 191 && $('#config').is(':visible')) {
+            $('#config-table-filter').focus();
+          }
+        });
+
+        // Make config table searchable.
+        $('#config-table-filter').keyup(function(e) {
+          // Press ESC to exit search box.
+          if (e.keyCode == 27) {
+            $('#config-table-filter').blur();
+          }
+          var regex = new RegExp($(this).val(), 'i');
+          $('.searchable tr').hide();
+          $('.searchable tr').filter(function() {
+            return regex.test($(this).text());
+          }).show();
+        });
+      });
diff --git a/samza-yarn3/src/main/resources/scalate/css/bootstrap.min.css b/samza-yarn3/src/main/resources/scalate/css/bootstrap.min.css
new file mode 100644
index 000000000..679272d25
--- /dev/null
+++ b/samza-yarn3/src/main/resources/scalate/css/bootstrap.min.css
@@ -0,0 +1,7 @@
+/*!
+ * Bootstrap v3.1.1 (http://getbootstrap.com)
+ * Copyright 2011-2014 Twitter, Inc.
+ * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE)
+ */
+
+/*! normalize.css v3.0.0 | MIT License | git.io/normalize */html{font-family:sans-serif;-ms-text-size-adjust:100%;-webkit-text-size-adjust:100%}body{margin:0}article,aside,details,figcaption,figure,footer,header,hgroup,main,nav,section,summary{display:block}audio,canvas,progress,video{display:inline-block;vertical-align:baseline}audio:not([controls]){display:none;height:0}[hidden],template{display:none}a{background:0 0}a:active,a:hover{outline:0}abbr[title]{border-bottom:1px dotted}b,str [...]
\ No newline at end of file
diff --git a/samza-yarn3/src/main/resources/scalate/css/font-awesome.min.css b/samza-yarn3/src/main/resources/scalate/css/font-awesome.min.css
new file mode 100644
index 000000000..449d6ac55
--- /dev/null
+++ b/samza-yarn3/src/main/resources/scalate/css/font-awesome.min.css
@@ -0,0 +1,4 @@
+/*!
+ *  Font Awesome 4.0.3 by @davegandy - http://fontawesome.io - @fontawesome
+ *  License - http://fontawesome.io/license (Font: SIL OFL 1.1, CSS: MIT License)
+ */@font-face{font-family:'FontAwesome';src:url('../fonts/fontawesome-webfont.eot?v=4.0.3');src:url('../fonts/fontawesome-webfont.eot?#iefix&v=4.0.3') format('embedded-opentype'),url('../fonts/fontawesome-webfont.woff?v=4.0.3') format('woff'),url('../fonts/fontawesome-webfont.ttf?v=4.0.3') format('truetype'),url('../fonts/fontawesome-webfont.svg?v=4.0.3#fontawesomeregular') format('svg');font-weight:normal;font-style:normal}.fa{display:inline-block;font-family:FontAwesome;font-style:norm [...]
\ No newline at end of file
diff --git a/samza-yarn3/src/main/resources/scalate/css/ropa-sans.css b/samza-yarn3/src/main/resources/scalate/css/ropa-sans.css
new file mode 100644
index 000000000..d36ebed2f
--- /dev/null
+++ b/samza-yarn3/src/main/resources/scalate/css/ropa-sans.css
@@ -0,0 +1,6 @@
+@font-face {
+  font-family: 'Ropa Sans';
+  font-style: normal;
+  font-weight: 400;
+  src: local('Ropa Sans'), local('RopaSans-Regular'), url('../fonts/RopaSans-Regular-webfont.woff') format('woff');
+}
diff --git a/samza-yarn3/src/main/resources/scalate/fonts/FontAwesome.otf b/samza-yarn3/src/main/resources/scalate/fonts/FontAwesome.otf
new file mode 100644
index 000000000..8b0f54e47
Binary files /dev/null and b/samza-yarn3/src/main/resources/scalate/fonts/FontAwesome.otf differ
diff --git a/samza-yarn3/src/main/resources/scalate/fonts/RopaSans-Regular-webfont.woff b/samza-yarn3/src/main/resources/scalate/fonts/RopaSans-Regular-webfont.woff
new file mode 100644
index 000000000..e4eda8525
Binary files /dev/null and b/samza-yarn3/src/main/resources/scalate/fonts/RopaSans-Regular-webfont.woff differ
diff --git a/samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.eot b/samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.eot
new file mode 100755
index 000000000..7c79c6a6b
Binary files /dev/null and b/samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.eot differ
diff --git a/samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.svg b/samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.svg
new file mode 100755
index 000000000..45fdf3383
--- /dev/null
+++ b/samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.svg
@@ -0,0 +1,414 @@
+<?xml version="1.0" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
+<svg xmlns="http://www.w3.org/2000/svg">
+<metadata></metadata>
+<defs>
+<font id="fontawesomeregular" horiz-adv-x="1536" >
+<font-face units-per-em="1792" ascent="1536" descent="-256" />
+<missing-glyph horiz-adv-x="448" />
+<glyph unicode=" "  horiz-adv-x="448" />
+<glyph unicode="&#x09;" horiz-adv-x="448" />
+<glyph unicode="&#xa0;" horiz-adv-x="448" />
+<glyph unicode="&#xa8;" horiz-adv-x="1792" />
+<glyph unicode="&#xa9;" horiz-adv-x="1792" />
+<glyph unicode="&#xae;" horiz-adv-x="1792" />
+<glyph unicode="&#xb4;" horiz-adv-x="1792" />
+<glyph unicode="&#xc6;" horiz-adv-x="1792" />
+<glyph unicode="&#x2000;" horiz-adv-x="768" />
+<glyph unicode="&#x2001;" />
+<glyph unicode="&#x2002;" horiz-adv-x="768" />
+<glyph unicode="&#x2003;" />
+<glyph unicode="&#x2004;" horiz-adv-x="512" />
+<glyph unicode="&#x2005;" horiz-adv-x="384" />
+<glyph unicode="&#x2006;" horiz-adv-x="256" />
+<glyph unicode="&#x2007;" horiz-adv-x="256" />
+<glyph unicode="&#x2008;" horiz-adv-x="192" />
+<glyph unicode="&#x2009;" horiz-adv-x="307" />
+<glyph unicode="&#x200a;" horiz-adv-x="85" />
+<glyph unicode="&#x202f;" horiz-adv-x="307" />
+<glyph unicode="&#x205f;" horiz-adv-x="384" />
+<glyph unicode="&#x2122;" horiz-adv-x="1792" />
+<glyph unicode="&#x221e;" horiz-adv-x="1792" />
+<glyph unicode="&#x2260;" horiz-adv-x="1792" />
+<glyph unicode="&#xe000;" horiz-adv-x="500" d="M0 0z" />
+<glyph unicode="&#xf000;" horiz-adv-x="1792" d="M1699 1350q0 -35 -43 -78l-632 -632v-768h320q26 0 45 -19t19 -45t-19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45t45 19h320v768l-632 632q-43 43 -43 78q0 23 18 36.5t38 17.5t43 4h1408q23 0 43 -4t38 -17.5t18 -36.5z" />
+<glyph unicode="&#xf001;" d="M1536 1312v-1120q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v537l-768 -237v-709q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89 t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v967q0 31 19 56.5t49 35.5l832 256q12 4 28 4q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf002;" horiz-adv-x="1664" d="M1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -52 -38 -90t-90 -38q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5 t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
+<glyph unicode="&#xf003;" horiz-adv-x="1792" d="M1664 32v768q-32 -36 -69 -66q-268 -206 -426 -338q-51 -43 -83 -67t-86.5 -48.5t-102.5 -24.5h-1h-1q-48 0 -102.5 24.5t-86.5 48.5t-83 67q-158 132 -426 338q-37 30 -69 66v-768q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1664 1083v11v13.5t-0.5 13 t-3 12.5t-5.5 9t-9 7.5t-14 2.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5q0 -168 147 -284q193 -152 401 -317q6 -5 35 -29.5t46 -37.5t44.5 -31.5t50.5 -27.5t43 -9h1h1q20 0 43 9t50.5 27.5t44.5 31.5t46 37.5t35 [...]
+<glyph unicode="&#xf004;" horiz-adv-x="1792" d="M896 -128q-26 0 -44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5q224 0 351 -124t127 -344q0 -221 -229 -450l-623 -600 q-18 -18 -44 -18z" />
+<glyph unicode="&#xf005;" horiz-adv-x="1664" d="M1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -21 -10.5 -35.5t-30.5 -14.5q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455 l502 -73q56 -9 56 -46z" />
+<glyph unicode="&#xf006;" horiz-adv-x="1664" d="M1137 532l306 297l-422 62l-189 382l-189 -382l-422 -62l306 -297l-73 -421l378 199l377 -199zM1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -50 -41 -50q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500 l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455l502 -73q56 -9 56 -46z" />
+<glyph unicode="&#xf007;" horiz-adv-x="1408" d="M1408 131q0 -120 -73 -189.5t-194 -69.5h-874q-121 0 -194 69.5t-73 189.5q0 53 3.5 103.5t14 109t26.5 108.5t43 97.5t62 81t85.5 53.5t111.5 20q9 0 42 -21.5t74.5 -48t108 -48t133.5 -21.5t133.5 21.5t108 48t74.5 48t42 21.5q61 0 111.5 -20t85.5 -53.5t62 -81 t43 -97.5t26.5 -108.5t14 -109t3.5 -103.5zM1088 1024q0 -159 -112.5 -271.5t-271.5 -112.5t-271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5t271.5 -112.5t112.5 -271.5z" />
+<glyph unicode="&#xf008;" horiz-adv-x="1920" d="M384 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 320v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 704v128q0 26 -19 45t-45 19h-128 q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 -64v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM384 1088v128q0 26 -19 [...]
+<glyph unicode="&#xf009;" horiz-adv-x="1664" d="M768 512v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM768 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 512v-384q0 -52 -38 -90t-90 -38 h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90z" />
+<glyph unicode="&#xf00a;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 288v-192q0 -40 -28 -68t-68 -28h-320 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 800v-192q0 -40  [...]
+<glyph unicode="&#xf00b;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-960 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192q0 -40  [...]
+<glyph unicode="&#xf00c;" horiz-adv-x="1792" d="M1671 970q0 -40 -28 -68l-724 -724l-136 -136q-28 -28 -68 -28t-68 28l-136 136l-362 362q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -295l656 657q28 28 68 28t68 -28l136 -136q28 -28 28 -68z" />
+<glyph unicode="&#xf00d;" horiz-adv-x="1408" d="M1298 214q0 -40 -28 -68l-136 -136q-28 -28 -68 -28t-68 28l-294 294l-294 -294q-28 -28 -68 -28t-68 28l-136 136q-28 28 -28 68t28 68l294 294l-294 294q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -294l294 294q28 28 68 28t68 -28l136 -136q28 -28 28 -68 t-28 -68l-294 -294l294 -294q28 -28 28 -68z" />
+<glyph unicode="&#xf00e;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-224q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v224h-224q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h224v224q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5v-224h224 q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343  [...]
+<glyph unicode="&#xf010;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-576q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h576q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5z M1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150  [...]
+<glyph unicode="&#xf011;" d="M1536 640q0 -156 -61 -298t-164 -245t-245 -164t-298 -61t-298 61t-245 164t-164 245t-61 298q0 182 80.5 343t226.5 270q43 32 95.5 25t83.5 -50q32 -42 24.5 -94.5t-49.5 -84.5q-98 -74 -151.5 -181t-53.5 -228q0 -104 40.5 -198.5t109.5 -163.5t163.5 -109.5 t198.5 -40.5t198.5 40.5t163.5 109.5t109.5 163.5t40.5 198.5q0 121 -53.5 228t-151.5 181q-42 32 -49.5 84.5t24.5 94.5q31 43 84 50t95 -25q146 -109 226.5 -270t80.5 -343zM896 1408v-640q0 -52 -38 -90t-90 -38t-90 38t-38 90v640q0  [...]
+<glyph unicode="&#xf012;" horiz-adv-x="1792" d="M256 96v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM640 224v-320q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1024 480v-576q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23 v576q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1408 864v-960q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v960q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1792 1376v-1472q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t [...]
+<glyph unicode="&#xf013;" d="M1024 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1536 749v-222q0 -12 -8 -23t-20 -13l-185 -28q-19 -54 -39 -91q35 -50 107 -138q10 -12 10 -25t-9 -23q-27 -37 -99 -108t-94 -71q-12 0 -26 9l-138 108q-44 -23 -91 -38 q-16 -136 -29 -186q-7 -28 -36 -28h-222q-14 0 -24.5 8.5t-11.5 21.5l-28 184q-49 16 -90 37l-141 -107q-10 -9 -25 -9q-14 0 -25 11q-126 114 -165 168q-7 10 -7 23q0 12 8 23q15 21 51 66.5t54 70.5q-27 50 -41 99l-183 27q-13 2 -21 12.5 [...]
+<glyph unicode="&#xf014;" horiz-adv-x="1408" d="M512 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM768 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1024 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1152 76v948h-896v-948q0 -22 7 -40.5t14.5 -27t10.5 -8.5h832q3 0 10.5 8.5t14.5 27t7 40.5zM480 1152h448l-48 117q-7 9 -17 11h-317q-10 -2 -17 -11zM1408 1120v-64q [...]
+<glyph unicode="&#xf015;" horiz-adv-x="1664" d="M1408 544v-480q0 -26 -19 -45t-45 -19h-384v384h-256v-384h-384q-26 0 -45 19t-19 45v480q0 1 0.5 3t0.5 3l575 474l575 -474q1 -2 1 -6zM1631 613l-62 -74q-8 -9 -21 -11h-3q-13 0 -21 7l-692 577l-692 -577q-12 -8 -24 -7q-13 2 -21 11l-62 74q-8 10 -7 23.5t11 21.5 l719 599q32 26 76 26t76 -26l244 -204v195q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-408l219 -182q10 -8 11 -21.5t-7 -23.5z" />
+<glyph unicode="&#xf016;" horiz-adv-x="1280" d="M128 0h1024v768h-416q-40 0 -68 28t-28 68v416h-512v-1280zM768 896h376q-10 29 -22 41l-313 313q-12 12 -41 22v-376zM1280 864v-896q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v1344q0 40 28 68t68 28h640q40 0 88 -20t76 -48l312 -312q28 -28 48 -76t20 -88z " />
+<glyph unicode="&#xf017;" d="M896 992v-448q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h224v352q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf018;" horiz-adv-x="1920" d="M1111 540v4l-24 320q-1 13 -11 22.5t-23 9.5h-186q-13 0 -23 -9.5t-11 -22.5l-24 -320v-4q-1 -12 8 -20t21 -8h244q12 0 21 8t8 20zM1870 73q0 -73 -46 -73h-704q13 0 22 9.5t8 22.5l-20 256q-1 13 -11 22.5t-23 9.5h-272q-13 0 -23 -9.5t-11 -22.5l-20 -256 q-1 -13 8 -22.5t22 -9.5h-704q-46 0 -46 73q0 54 26 116l417 1044q8 19 26 33t38 14h339q-13 0 -23 -9.5t-11 -22.5l-15 -192q-1 -14 8 -23t22 -9h166q13 0 22 9t8 23l-15 192q-1 13 -11 22.5t-23 9.5h339q20 0 38 -14t2 [...]
+<glyph unicode="&#xf019;" horiz-adv-x="1664" d="M1280 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 416v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h465l135 -136 q58 -56 136 -56t136 56l136 136h464q40 0 68 -28t28 -68zM1339 985q17 -41 -14 -70l-448 -448q-18 -19 -45 -19t-45 19l-448 448q-31 29 -14 70q17 39 59 39h256v448q0 26 19 45t45 19h256q26 0 45 -19t19 -45v-448h256q4 [...]
+<glyph unicode="&#xf01a;" d="M1120 608q0 -12 -10 -24l-319 -319q-11 -9 -23 -9t-23 9l-320 320q-15 16 -7 35q8 20 30 20h192v352q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-352h192q14 0 23 -9t9 -23zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273 t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf01b;" d="M1118 660q-8 -20 -30 -20h-192v-352q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v352h-192q-14 0 -23 9t-9 23q0 12 10 24l319 319q11 9 23 9t23 -9l320 -320q15 -16 7 -35zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198 t73 273t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf01c;" d="M1023 576h316q-1 3 -2.5 8t-2.5 8l-212 496h-708l-212 -496q-1 -2 -2.5 -8t-2.5 -8h316l95 -192h320zM1536 546v-482q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v482q0 62 25 123l238 552q10 25 36.5 42t52.5 17h832q26 0 52.5 -17t36.5 -42l238 -552 q25 -61 25 -123z" />
+<glyph unicode="&#xf01d;" d="M1184 640q0 -37 -32 -55l-544 -320q-15 -9 -32 -9q-16 0 -32 8q-32 19 -32 56v640q0 37 32 56q33 18 64 -1l544 -320q32 -18 32 -55zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf01e;" d="M1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-42 0 -59 40q-17 39 14 69l138 138q-148 137 -349 137q-104 0 -198.5 -40.5t-163.5 -109.5t-109.5 -163.5t-40.5 -198.5t40.5 -198.5t109.5 -163.5t163.5 -109.5t198.5 -40.5q119 0 225 52t179 147q7 10 23 12q14 0 25 -9 l137 -138q9 -8 9.5 -20.5t-7.5 -22.5q-109 -132 -264 -204.5t-327 -72.5q-156 0 -298 61t-245 164t-164 245t-61 298t61 298t164 245t245 164t298 61q147 0 284.5 -55.5t244.5 -156.5l130 129q29 31 70 14q39 -17 39 -59z" />
+<glyph unicode="&#xf021;" d="M1511 480q0 -5 -1 -7q-64 -268 -268 -434.5t-478 -166.5q-146 0 -282.5 55t-243.5 157l-129 -129q-19 -19 -45 -19t-45 19t-19 45v448q0 26 19 45t45 19h448q26 0 45 -19t19 -45t-19 -45l-137 -137q71 -66 161 -102t187 -36q134 0 250 65t186 179q11 17 53 117 q8 23 30 23h192q13 0 22.5 -9.5t9.5 -22.5zM1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-26 0 -45 19t-19 45t19 45l138 138q-148 137 -349 137q-134 0 -250 -65t-186 -179q-11 -17 -53 -117q-8 -23 -30 -23h-199q-13 0 -22.5 9.5t-9.5 2 [...]
+<glyph unicode="&#xf022;" horiz-adv-x="1792" d="M384 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M384 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1536 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5  [...]
+<glyph unicode="&#xf023;" horiz-adv-x="1152" d="M320 768h512v192q0 106 -75 181t-181 75t-181 -75t-75 -181v-192zM1152 672v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h32v192q0 184 132 316t316 132t316 -132t132 -316v-192h32q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf024;" horiz-adv-x="1792" d="M320 1280q0 -72 -64 -110v-1266q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v1266q-64 38 -64 110q0 53 37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1792 1216v-763q0 -25 -12.5 -38.5t-39.5 -27.5q-215 -116 -369 -116q-61 0 -123.5 22t-108.5 48 t-115.5 48t-142.5 22q-192 0 -464 -146q-17 -9 -33 -9q-26 0 -45 19t-19 45v742q0 32 31 55q21 14 79 43q236 120 421 120q107 0 200 -29t219 -88q38 -19 88 -19q54 0 117.5 21t110 47t88 47t54.5 21q26 0 45 -1 [...]
+<glyph unicode="&#xf025;" horiz-adv-x="1664" d="M1664 650q0 -166 -60 -314l-20 -49l-185 -33q-22 -83 -90.5 -136.5t-156.5 -53.5v-32q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-32q71 0 130 -35.5t93 -95.5l68 12q29 95 29 193q0 148 -88 279t-236.5 209t-315.5 78 t-315.5 -78t-236.5 -209t-88 -279q0 -98 29 -193l68 -12q34 60 93 95.5t130 35.5v32q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v32q-88 0 -156.5 53.5t-90.5 136.5l-185  [...]
+<glyph unicode="&#xf026;" horiz-adv-x="768" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45z" />
+<glyph unicode="&#xf027;" horiz-adv-x="1152" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142z" />
+<glyph unicode="&#xf028;" horiz-adv-x="1664" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142zM1408 640q0 -153 -85 -282.5t-225 -188.5q-13 -5 -25 -5q-27 0 -46 19t-19 45q0 39 39 59q56  [...]
+<glyph unicode="&#xf029;" horiz-adv-x="1408" d="M384 384v-128h-128v128h128zM384 1152v-128h-128v128h128zM1152 1152v-128h-128v128h128zM128 129h384v383h-384v-383zM128 896h384v384h-384v-384zM896 896h384v384h-384v-384zM640 640v-640h-640v640h640zM1152 128v-128h-128v128h128zM1408 128v-128h-128v128h128z M1408 640v-384h-384v128h-128v-384h-128v640h384v-128h128v128h128zM640 1408v-640h-640v640h640zM1408 1408v-640h-640v640h640z" />
+<glyph unicode="&#xf02a;" horiz-adv-x="1792" d="M63 0h-63v1408h63v-1408zM126 1h-32v1407h32v-1407zM220 1h-31v1407h31v-1407zM377 1h-31v1407h31v-1407zM534 1h-62v1407h62v-1407zM660 1h-31v1407h31v-1407zM723 1h-31v1407h31v-1407zM786 1h-31v1407h31v-1407zM943 1h-63v1407h63v-1407zM1100 1h-63v1407h63v-1407z M1226 1h-63v1407h63v-1407zM1352 1h-63v1407h63v-1407zM1446 1h-63v1407h63v-1407zM1635 1h-94v1407h94v-1407zM1698 1h-32v1407h32v-1407zM1792 0h-63v1408h63v-1408z" />
+<glyph unicode="&#xf02b;" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91z" />
+<glyph unicode="&#xf02c;" horiz-adv-x="1920" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91zM1899 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-36 0 -59 14t-53 45l470 470q37 37 37 90q0 52 -37 91l-715 714q-38 38 -102 64.5t-117 26.5h224q53 0 117 -26.5t102 -64.5l7 [...]
+<glyph unicode="&#xf02d;" horiz-adv-x="1664" d="M1639 1058q40 -57 18 -129l-275 -906q-19 -64 -76.5 -107.5t-122.5 -43.5h-923q-77 0 -148.5 53.5t-99.5 131.5q-24 67 -2 127q0 4 3 27t4 37q1 8 -3 21.5t-3 19.5q2 11 8 21t16.5 23.5t16.5 23.5q23 38 45 91.5t30 91.5q3 10 0.5 30t-0.5 28q3 11 17 28t17 23 q21 36 42 92t25 90q1 9 -2.5 32t0.5 28q4 13 22 30.5t22 22.5q19 26 42.5 84.5t27.5 96.5q1 8 -3 25.5t-2 26.5q2 8 9 18t18 23t17 21q8 12 16.5 30.5t15 35t16 36t19.5 32t26.5 23.5t36 11.5t47.5 -5.5l-1 -3q38 9 51 [...]
+<glyph unicode="&#xf02e;" horiz-adv-x="1280" d="M1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
+<glyph unicode="&#xf02f;" horiz-adv-x="1664" d="M384 0h896v256h-896v-256zM384 640h896v384h-160q-40 0 -68 28t-28 68v160h-640v-640zM1536 576q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 576v-416q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-160q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68 v160h-224q-13 0 -22.5 9.5t-9.5 22.5v416q0 79 56.5 135.5t135.5 56.5h64v544q0 40 28 68t68 28h672q40 0 88 -20t76 -48l152 -152q28 -28 48 -76t20 -88v-256h64q79 0 135.5 -56.5t56.5 -135.5z" />
+<glyph unicode="&#xf030;" horiz-adv-x="1920" d="M960 864q119 0 203.5 -84.5t84.5 -203.5t-84.5 -203.5t-203.5 -84.5t-203.5 84.5t-84.5 203.5t84.5 203.5t203.5 84.5zM1664 1280q106 0 181 -75t75 -181v-896q0 -106 -75 -181t-181 -75h-1408q-106 0 -181 75t-75 181v896q0 106 75 181t181 75h224l51 136 q19 49 69.5 84.5t103.5 35.5h512q53 0 103.5 -35.5t69.5 -84.5l51 -136h224zM960 128q185 0 316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" />
+<glyph unicode="&#xf031;" horiz-adv-x="1664" d="M725 977l-170 -450q73 -1 153.5 -2t119 -1.5t52.5 -0.5l29 2q-32 95 -92 241q-53 132 -92 211zM21 -128h-21l2 79q22 7 80 18q89 16 110 31q20 16 48 68l237 616l280 724h75h53l11 -21l205 -480q103 -242 124 -297q39 -102 96 -235q26 -58 65 -164q24 -67 65 -149 q22 -49 35 -57q22 -19 69 -23q47 -6 103 -27q6 -39 6 -57q0 -14 -1 -26q-80 0 -192 8q-93 8 -189 8q-79 0 -135 -2l-200 -11l-58 -2q0 45 4 78l131 28q56 13 68 23q12 12 12 27t-6 32l-47 114l-92 228l-450 2q-29 - [...]
+<glyph unicode="&#xf032;" horiz-adv-x="1408" d="M555 15q76 -32 140 -32q131 0 216 41t122 113q38 70 38 181q0 114 -41 180q-58 94 -141 126q-80 32 -247 32q-74 0 -101 -10v-144l-1 -173l3 -270q0 -15 12 -44zM541 761q43 -7 109 -7q175 0 264 65t89 224q0 112 -85 187q-84 75 -255 75q-52 0 -130 -13q0 -44 2 -77 q7 -122 6 -279l-1 -98q0 -43 1 -77zM0 -128l2 94q45 9 68 12q77 12 123 31q17 27 21 51q9 66 9 194l-2 497q-5 256 -9 404q-1 87 -11 109q-1 4 -12 12q-18 12 -69 15q-30 2 -114 13l-4 83l260 6l380 13l45 1q5 0 [...]
+<glyph unicode="&#xf033;" horiz-adv-x="1024" d="M0 -126l17 85q4 1 77 20q76 19 116 39q29 37 41 101l27 139l56 268l12 64q8 44 17 84.5t16 67t12.5 46.5t9 30.5t3.5 11.5l29 157l16 63l22 135l8 50v38q-41 22 -144 28q-28 2 -38 4l19 103l317 -14q39 -2 73 -2q66 0 214 9q33 2 68 4.5t36 2.5q-2 -19 -6 -38 q-7 -29 -13 -51q-55 -19 -109 -31q-64 -16 -101 -31q-12 -31 -24 -88q-9 -44 -13 -82q-44 -199 -66 -306l-61 -311l-38 -158l-43 -235l-12 -45q-2 -7 1 -27q64 -15 119 -21q36 -5 66 -10q-1 -29 -7 -58q-7 -31 -9 -41q- [...]
+<glyph unicode="&#xf034;" horiz-adv-x="1792" d="M81 1407l54 -27q20 -5 211 -5h130l19 3l115 1l215 -1h293l34 -2q14 -1 28 7t21 16l7 8l42 1q15 0 28 -1v-104.5t1 -131.5l1 -100l-1 -58q0 -32 -4 -51q-39 -15 -68 -18q-25 43 -54 128q-8 24 -15.5 62.5t-11.5 65.5t-6 29q-13 15 -27 19q-7 2 -42.5 2t-103.5 -1t-111 -1 q-34 0 -67 -5q-10 -97 -8 -136l1 -152v-332l3 -359l-1 -147q-1 -46 11 -85q49 -25 89 -32q2 0 18 -5t44 -13t43 -12q30 -8 50 -18q5 -45 5 -50q0 -10 -3 -29q-14 -1 -34 -1q-110 0 -187 10q-72 8 -238 8q-88  [...]
+<glyph unicode="&#xf035;" d="M81 1407l54 -27q20 -5 211 -5h130l19 3l115 1l446 -1h318l34 -2q14 -1 28 7t21 16l7 8l42 1q15 0 28 -1v-104.5t1 -131.5l1 -100l-1 -58q0 -32 -4 -51q-39 -15 -68 -18q-25 43 -54 128q-8 24 -15.5 62.5t-11.5 65.5t-6 29q-13 15 -27 19q-7 2 -58.5 2t-138.5 -1t-128 -1 q-94 0 -127 -5q-10 -97 -8 -136l1 -152v52l3 -359l-1 -147q-1 -46 11 -85q49 -25 89 -32q2 0 18 -5t44 -13t43 -12q30 -8 50 -18q5 -45 5 -50q0 -10 -3 -29q-14 -1 -34 -1q-110 0 -187 10q-72 8 -238 8q-82 0 -233 -13q-45 -5 -7 [...]
+<glyph unicode="&#xf036;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf037;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h896q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45t-45 -19 h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-640q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h640q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf038;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf039;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf03a;" horiz-adv-x="1792" d="M256 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM256 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5 t9.5 -22.5zM256 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 - [...]
+<glyph unicode="&#xf03b;" horiz-adv-x="1792" d="M384 992v-576q0 -13 -9.5 -22.5t-22.5 -9.5q-14 0 -23 9l-288 288q-9 9 -9 23t9 23l288 288q9 9 23 9q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22 [...]
+<glyph unicode="&#xf03c;" horiz-adv-x="1792" d="M352 704q0 -14 -9 -23l-288 -288q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v576q0 13 9.5 22.5t22.5 9.5q14 0 23 -9l288 -288q9 -9 9 -23zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q- [...]
+<glyph unicode="&#xf03d;" horiz-adv-x="1792" d="M1792 1184v-1088q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-403 403v-166q0 -119 -84.5 -203.5t-203.5 -84.5h-704q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h704q119 0 203.5 -84.5t84.5 -203.5v-165l403 402q18 19 45 19q12 0 25 -5 q39 -17 39 -59z" />
+<glyph unicode="&#xf03e;" horiz-adv-x="1920" d="M640 960q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1664 576v-448h-1408v192l320 320l160 -160l512 512zM1760 1280h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-1216q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5v1216 q0 13 -9.5 22.5t-22.5 9.5zM1920 1248v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
+<glyph unicode="&#xf040;" d="M363 0l91 91l-235 235l-91 -91v-107h128v-128h107zM886 928q0 22 -22 22q-10 0 -17 -7l-542 -542q-7 -7 -7 -17q0 -22 22 -22q10 0 17 7l542 542q7 7 7 17zM832 1120l416 -416l-832 -832h-416v416zM1515 1024q0 -53 -37 -90l-166 -166l-416 416l166 165q36 38 90 38 q53 0 91 -38l235 -234q37 -39 37 -91z" />
+<glyph unicode="&#xf041;" horiz-adv-x="1024" d="M768 896q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1024 896q0 -109 -33 -179l-364 -774q-16 -33 -47.5 -52t-67.5 -19t-67.5 19t-46.5 52l-365 774q-33 70 -33 179q0 212 150 362t362 150t362 -150t150 -362z" />
+<glyph unicode="&#xf042;" d="M768 96v1088q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf043;" horiz-adv-x="1024" d="M512 384q0 36 -20 69q-1 1 -15.5 22.5t-25.5 38t-25 44t-21 50.5q-4 16 -21 16t-21 -16q-7 -23 -21 -50.5t-25 -44t-25.5 -38t-15.5 -22.5q-20 -33 -20 -69q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1024 512q0 -212 -150 -362t-362 -150t-362 150t-150 362 q0 145 81 275q6 9 62.5 90.5t101 151t99.5 178t83 201.5q9 30 34 47t51 17t51.5 -17t33.5 -47q28 -93 83 -201.5t99.5 -178t101 -151t62.5 -90.5q81 -127 81 -275z" />
+<glyph unicode="&#xf044;" horiz-adv-x="1792" d="M888 352l116 116l-152 152l-116 -116v-56h96v-96h56zM1328 1072q-16 16 -33 -1l-350 -350q-17 -17 -1 -33t33 1l350 350q17 17 1 33zM1408 478v-190q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-14 -14 -32 -8q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v126q0 13 9 22l64 64q15 15 35 7t20 -29zM1312 12 [...]
+<glyph unicode="&#xf045;" horiz-adv-x="1664" d="M1408 547v-259q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h255v0q13 0 22.5 -9.5t9.5 -22.5q0 -27 -26 -32q-77 -26 -133 -60q-10 -4 -16 -4h-112q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832 q66 0 113 47t47 113v214q0 19 18 29q28 13 54 37q16 16 35 8q21 -9 21 -29zM1645 1043l-384 -384q-18 -19 -45 -19q-12 0 -25 5q-39 17 -39 59v192h-160q-323 0 -438 -131q-119 -137 -74 -473q3 -23 -20 -34 [...]
+<glyph unicode="&#xf046;" horiz-adv-x="1664" d="M1408 606v-318q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-10 -10 -23 -10q-3 0 -9 2q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832 q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v254q0 13 9 22l64 64q10 10 23 10q6 0 12 -3q20 -8 20 -29zM1639 1095l-814 -814q-24 -24 -57 -24t-57 24l-430 430q-24 24 -24 57t24 57l110 110q24 24 57 24t57 -24l263  [...]
+<glyph unicode="&#xf047;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-384v-384h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v384h-384v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45 t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h384v384h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45t-19 -45t-45 -19h-128v-384h384v128q0 26 19 45t45 19t45 -19l256  [...]
+<glyph unicode="&#xf048;" horiz-adv-x="1024" d="M979 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19z" />
+<glyph unicode="&#xf049;" horiz-adv-x="1792" d="M1747 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19l710 710 q19 19 32 13t13 -32v-710q4 11 13 19z" />
+<glyph unicode="&#xf04a;" horiz-adv-x="1664" d="M1619 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-8 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-19 19 -19 45t19 45l710 710q19 19 32 13t13 -32v-710q5 11 13 19z" />
+<glyph unicode="&#xf04b;" horiz-adv-x="1408" d="M1384 609l-1328 -738q-23 -13 -39.5 -3t-16.5 36v1472q0 26 16.5 36t39.5 -3l1328 -738q23 -13 23 -31t-23 -31z" />
+<glyph unicode="&#xf04c;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45zM640 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf04d;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h1408q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf04e;" horiz-adv-x="1664" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q19 -19 19 -45t-19 -45l-710 -710q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
+<glyph unicode="&#xf050;" horiz-adv-x="1792" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19l-710 -710 q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
+<glyph unicode="&#xf051;" horiz-adv-x="1024" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19z" />
+<glyph unicode="&#xf052;" horiz-adv-x="1538" d="M14 557l710 710q19 19 45 19t45 -19l710 -710q19 -19 13 -32t-32 -13h-1472q-26 0 -32 13t13 32zM1473 0h-1408q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1408q26 0 45 -19t19 -45v-256q0 -26 -19 -45t-45 -19z" />
+<glyph unicode="&#xf053;" horiz-adv-x="1152" d="M742 -37l-652 651q-37 37 -37 90.5t37 90.5l652 651q37 37 90.5 37t90.5 -37l75 -75q37 -37 37 -90.5t-37 -90.5l-486 -486l486 -485q37 -38 37 -91t-37 -90l-75 -75q-37 -37 -90.5 -37t-90.5 37z" />
+<glyph unicode="&#xf054;" horiz-adv-x="1152" d="M1099 704q0 -52 -37 -91l-652 -651q-37 -37 -90 -37t-90 37l-76 75q-37 39 -37 91q0 53 37 90l486 486l-486 485q-37 39 -37 91q0 53 37 90l76 75q36 38 90 38t90 -38l652 -651q37 -37 37 -90z" />
+<glyph unicode="&#xf055;" d="M1216 576v128q0 26 -19 45t-45 19h-256v256q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-256h-256q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h256v-256q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v256h256q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5 t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf056;" d="M1216 576v128q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5 t103 -385.5z" />
+<glyph unicode="&#xf057;" d="M1149 414q0 26 -19 45l-181 181l181 181q19 19 19 45q0 27 -19 46l-90 90q-19 19 -46 19q-26 0 -45 -19l-181 -181l-181 181q-19 19 -45 19q-27 0 -46 -19l-90 -90q-19 -19 -19 -46q0 -26 19 -45l181 -181l-181 -181q-19 -19 -19 -45q0 -27 19 -46l90 -90q19 -19 46 -19 q26 0 45 19l181 181l181 -181q19 -19 45 -19q27 0 46 19l90 90q19 19 19 46zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 - [...]
+<glyph unicode="&#xf058;" d="M1284 802q0 28 -18 46l-91 90q-19 19 -45 19t-45 -19l-408 -407l-226 226q-19 19 -45 19t-45 -19l-91 -90q-18 -18 -18 -46q0 -27 18 -45l362 -362q19 -19 45 -19q27 0 46 19l543 543q18 18 18 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103 t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf059;" d="M896 160v192q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h192q14 0 23 9t9 23zM1152 832q0 88 -55.5 163t-138.5 116t-170 41q-243 0 -371 -213q-15 -24 8 -42l132 -100q7 -6 19 -6q16 0 25 12q53 68 86 92q34 24 86 24q48 0 85.5 -26t37.5 -59 q0 -38 -20 -61t-68 -45q-63 -28 -115.5 -86.5t-52.5 -125.5v-36q0 -14 9 -23t23 -9h192q14 0 23 9t9 23q0 19 21.5 49.5t54.5 49.5q32 18 49 28.5t46 35t44.5 48t28 60.5t12.5 81zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-38 [...]
+<glyph unicode="&#xf05a;" d="M1024 160v160q0 14 -9 23t-23 9h-96v512q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h96v-320h-96q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h448q14 0 23 9t9 23zM896 1056v160q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23 t23 -9h192q14 0 23 9t9 23zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf05b;" d="M1197 512h-109q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h109q-32 108 -112.5 188.5t-188.5 112.5v-109q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v109q-108 -32 -188.5 -112.5t-112.5 -188.5h109q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-109 q32 -108 112.5 -188.5t188.5 -112.5v109q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-109q108 32 188.5 112.5t112.5 188.5zM1536 704v-128q0 -26 -19 -45t-45 -19h-143q-37 -161 -154.5 -278.5t-278.5 -154.5v-143q0 -26 -19 -45t-45 -19h [...]
+<glyph unicode="&#xf05c;" d="M1097 457l-146 -146q-10 -10 -23 -10t-23 10l-137 137l-137 -137q-10 -10 -23 -10t-23 10l-146 146q-10 10 -10 23t10 23l137 137l-137 137q-10 10 -10 23t10 23l146 146q10 10 23 10t23 -10l137 -137l137 137q10 10 23 10t23 -10l146 -146q10 -10 10 -23t-10 -23 l-137 -137l137 -137q10 -10 10 -23t-10 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385. [...]
+<glyph unicode="&#xf05d;" d="M1171 723l-422 -422q-19 -19 -45 -19t-45 19l-294 294q-19 19 -19 45t19 45l102 102q19 19 45 19t45 -19l147 -147l275 275q19 19 45 19t45 -19l102 -102q19 -19 19 -45t-19 -45zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198 t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf05e;" d="M1312 643q0 161 -87 295l-754 -753q137 -89 297 -89q111 0 211.5 43.5t173.5 116.5t116 174.5t43 212.5zM313 344l755 754q-135 91 -300 91q-148 0 -273 -73t-198 -199t-73 -274q0 -162 89 -299zM1536 643q0 -157 -61 -300t-163.5 -246t-245 -164t-298.5 -61t-298.5 61 t-245 164t-163.5 246t-61 300t61 299.5t163.5 245.5t245 164t298.5 61t298.5 -61t245 -164t163.5 -245.5t61 -299.5z" />
+<glyph unicode="&#xf060;" d="M1536 640v-128q0 -53 -32.5 -90.5t-84.5 -37.5h-704l293 -294q38 -36 38 -90t-38 -90l-75 -76q-37 -37 -90 -37q-52 0 -91 37l-651 652q-37 37 -37 90q0 52 37 91l651 650q38 38 91 38q52 0 90 -38l75 -74q38 -38 38 -91t-38 -91l-293 -293h704q52 0 84.5 -37.5 t32.5 -90.5z" />
+<glyph unicode="&#xf061;" d="M1472 576q0 -54 -37 -91l-651 -651q-39 -37 -91 -37q-51 0 -90 37l-75 75q-38 38 -38 91t38 91l293 293h-704q-52 0 -84.5 37.5t-32.5 90.5v128q0 53 32.5 90.5t84.5 37.5h704l-293 294q-38 36 -38 90t38 90l75 75q38 38 90 38q53 0 91 -38l651 -651q37 -35 37 -90z" />
+<glyph unicode="&#xf062;" horiz-adv-x="1664" d="M1611 565q0 -51 -37 -90l-75 -75q-38 -38 -91 -38q-54 0 -90 38l-294 293v-704q0 -52 -37.5 -84.5t-90.5 -32.5h-128q-53 0 -90.5 32.5t-37.5 84.5v704l-294 -293q-36 -38 -90 -38t-90 38l-75 75q-38 38 -38 90q0 53 38 91l651 651q35 37 90 37q54 0 91 -37l651 -651 q37 -39 37 -91z" />
+<glyph unicode="&#xf063;" horiz-adv-x="1664" d="M1611 704q0 -53 -37 -90l-651 -652q-39 -37 -91 -37q-53 0 -90 37l-651 652q-38 36 -38 90q0 53 38 91l74 75q39 37 91 37q53 0 90 -37l294 -294v704q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-704l294 294q37 37 90 37q52 0 91 -37l75 -75q37 -39 37 -91z" />
+<glyph unicode="&#xf064;" horiz-adv-x="1792" d="M1792 896q0 -26 -19 -45l-512 -512q-19 -19 -45 -19t-45 19t-19 45v256h-224q-98 0 -175.5 -6t-154 -21.5t-133 -42.5t-105.5 -69.5t-80 -101t-48.5 -138.5t-17.5 -181q0 -55 5 -123q0 -6 2.5 -23.5t2.5 -26.5q0 -15 -8.5 -25t-23.5 -10q-16 0 -28 17q-7 9 -13 22 t-13.5 30t-10.5 24q-127 285 -127 451q0 199 53 333q162 403 875 403h224v256q0 26 19 45t45 19t45 -19l512 -512q19 -19 19 -45z" />
+<glyph unicode="&#xf065;" d="M755 480q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23zM1536 1344v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332 q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf066;" d="M768 576v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45zM1523 1248q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45 t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23z" />
+<glyph unicode="&#xf067;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-416v-416q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v416h-416q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h416v416q0 40 28 68t68 28h192q40 0 68 -28t28 -68v-416h416q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf068;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-1216q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h1216q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf069;" horiz-adv-x="1664" d="M1482 486q46 -26 59.5 -77.5t-12.5 -97.5l-64 -110q-26 -46 -77.5 -59.5t-97.5 12.5l-266 153v-307q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v307l-266 -153q-46 -26 -97.5 -12.5t-77.5 59.5l-64 110q-26 46 -12.5 97.5t59.5 77.5l266 154l-266 154 q-46 26 -59.5 77.5t12.5 97.5l64 110q26 46 77.5 59.5t97.5 -12.5l266 -153v307q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-307l266 153q46 26 97.5 12.5t77.5 -59.5l64 -110q26 -46 12.5 -97.5t-59.5 -77.5l-266 -154z" />
+<glyph unicode="&#xf06a;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM896 161v190q0 14 -9 23.5t-22 9.5h-192q-13 0 -23 -10t-10 -23v-190q0 -13 10 -23t23 -10h192 q13 0 22 9.5t9 23.5zM894 505l18 621q0 12 -10 18q-10 8 -24 8h-220q-14 0 -24 -8q-10 -6 -10 -18l17 -621q0 -10 10 -17.5t24 -7.5h185q14 0 23.5 7.5t10.5 17.5z" />
+<glyph unicode="&#xf06b;" d="M928 180v56v468v192h-320v-192v-468v-56q0 -25 18 -38.5t46 -13.5h192q28 0 46 13.5t18 38.5zM472 1024h195l-126 161q-26 31 -69 31q-40 0 -68 -28t-28 -68t28 -68t68 -28zM1160 1120q0 40 -28 68t-68 28q-43 0 -69 -31l-125 -161h194q40 0 68 28t28 68zM1536 864v-320 q0 -14 -9 -23t-23 -9h-96v-416q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v416h-96q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h440q-93 0 -158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5q107 0 168 -77l128 -165l128 165q61 [...]
+<glyph unicode="&#xf06c;" horiz-adv-x="1792" d="M1280 832q0 26 -19 45t-45 19q-172 0 -318 -49.5t-259.5 -134t-235.5 -219.5q-19 -21 -19 -45q0 -26 19 -45t45 -19q24 0 45 19q27 24 74 71t67 66q137 124 268.5 176t313.5 52q26 0 45 19t19 45zM1792 1030q0 -95 -20 -193q-46 -224 -184.5 -383t-357.5 -268 q-214 -108 -438 -108q-148 0 -286 47q-15 5 -88 42t-96 37q-16 0 -39.5 -32t-45 -70t-52.5 -70t-60 -32q-30 0 -51 11t-31 24t-27 42q-2 4 -6 11t-5.5 10t-3 9.5t-1.5 13.5q0 35 31 73.5t68 65.5t68 56t31 48q0 4 -14 3 [...]
+<glyph unicode="&#xf06d;" horiz-adv-x="1408" d="M1408 -160v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1152 896q0 -78 -24.5 -144t-64 -112.5t-87.5 -88t-96 -77.5t-87.5 -72t-64 -81.5t-24.5 -96.5q0 -96 67 -224l-4 1l1 -1 q-90 41 -160 83t-138.5 100t-113.5 122.5t-72.5 150.5t-27.5 184q0 78 24.5 144t64 112.5t87.5 88t96 77.5t87.5 72t64 81.5t24.5 96.5q0 94 -66 224l3 -1l-1 1q90 -41 160 -83t138.5 -100t113.5 -122.5t72.5 -150. [...]
+<glyph unicode="&#xf06e;" horiz-adv-x="1792" d="M1664 576q-152 236 -381 353q61 -104 61 -225q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 121 61 225q-229 -117 -381 -353q133 -205 333.5 -326.5t434.5 -121.5t434.5 121.5t333.5 326.5zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5 t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1792 576q0 -34 -20 -69q-140 -230 -376.5 -368.5t-499.5 -138.5t-499.5 139t-376.5 368q-20 35 -20 69t20 69q140 229 376.5 [...]
+<glyph unicode="&#xf070;" horiz-adv-x="1792" d="M555 201l78 141q-87 63 -136 159t-49 203q0 121 61 225q-229 -117 -381 -353q167 -258 427 -375zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1307 1151q0 -7 -1 -9 q-105 -188 -315 -566t-316 -567l-49 -89q-10 -16 -28 -16q-12 0 -134 70q-16 10 -16 28q0 12 44 87q-143 65 -263.5 173t-208.5 245q-20 31 -20 69t20 69q153 235 380 371t496 136q89 0 180 -17l54 97q10 16 28 16q5 0 [...]
+<glyph unicode="&#xf071;" horiz-adv-x="1792" d="M1024 161v190q0 14 -9.5 23.5t-22.5 9.5h-192q-13 0 -22.5 -9.5t-9.5 -23.5v-190q0 -14 9.5 -23.5t22.5 -9.5h192q13 0 22.5 9.5t9.5 23.5zM1022 535l18 459q0 12 -10 19q-13 11 -24 11h-220q-11 0 -24 -11q-10 -7 -10 -21l17 -457q0 -10 10 -16.5t24 -6.5h185 q14 0 23.5 6.5t10.5 16.5zM1008 1469l768 -1408q35 -63 -2 -126q-17 -29 -46.5 -46t-63.5 -17h-1536q-34 0 -63.5 17t-46.5 46q-37 63 -2 126l768 1408q17 31 47 49t65 18t65 -18t47 -49z" />
+<glyph unicode="&#xf072;" horiz-adv-x="1408" d="M1376 1376q44 -52 12 -148t-108 -172l-161 -161l160 -696q5 -19 -12 -33l-128 -96q-7 -6 -19 -6q-4 0 -7 1q-15 3 -21 16l-279 508l-259 -259l53 -194q5 -17 -8 -31l-96 -96q-9 -9 -23 -9h-2q-15 2 -24 13l-189 252l-252 189q-11 7 -13 23q-1 13 9 25l96 97q9 9 23 9 q6 0 8 -1l194 -53l259 259l-508 279q-14 8 -17 24q-2 16 9 27l128 128q14 13 30 8l665 -159l160 160q76 76 172 108t148 -12z" />
+<glyph unicode="&#xf073;" horiz-adv-x="1664" d="M128 -128h288v288h-288v-288zM480 -128h320v288h-320v-288zM128 224h288v320h-288v-320zM480 224h320v320h-320v-320zM128 608h288v288h-288v-288zM864 -128h320v288h-320v-288zM480 608h320v288h-320v-288zM1248 -128h288v288h-288v-288zM864 224h320v320h-320v-320z M512 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1248 224h288v320h-288v-320zM864 608h320v288h-320v-288zM1248 608h28 [...]
+<glyph unicode="&#xf074;" horiz-adv-x="1792" d="M666 1055q-60 -92 -137 -273q-22 45 -37 72.5t-40.5 63.5t-51 56.5t-63 35t-81.5 14.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q250 0 410 -225zM1792 256q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v192q-32 0 -85 -0.5t-81 -1t-73 1 t-71 5t-64 10.5t-63 18.5t-58 28.5t-59 40t-55 53.5t-56 69.5q59 93 136 273q22 -45 37 -72.5t40.5 -63.5t51 -56.5t63 -35t81.5 -14.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23zM1792 1152q0 -1 [...]
+<glyph unicode="&#xf075;" horiz-adv-x="1792" d="M1792 640q0 -174 -120 -321.5t-326 -233t-450 -85.5q-70 0 -145 8q-198 -175 -460 -242q-49 -14 -114 -22q-17 -2 -30.5 9t-17.5 29v1q-3 4 -0.5 12t2 10t4.5 9.5l6 9t7 8.5t8 9q7 8 31 34.5t34.5 38t31 39.5t32.5 51t27 59t26 76q-157 89 -247.5 220t-90.5 281 q0 130 71 248.5t191 204.5t286 136.5t348 50.5q244 0 450 -85.5t326 -233t120 -321.5z" />
+<glyph unicode="&#xf076;" d="M1536 704v-128q0 -201 -98.5 -362t-274 -251.5t-395.5 -90.5t-395.5 90.5t-274 251.5t-98.5 362v128q0 26 19 45t45 19h384q26 0 45 -19t19 -45v-128q0 -52 23.5 -90t53.5 -57t71 -30t64 -13t44 -2t44 2t64 13t71 30t53.5 57t23.5 90v128q0 26 19 45t45 19h384 q26 0 45 -19t19 -45zM512 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45zM1536 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 [...]
+<glyph unicode="&#xf077;" horiz-adv-x="1664" d="M1611 320q0 -53 -37 -90l-75 -75q-38 -38 -91 -38q-54 0 -90 38l-486 485l-486 -485q-36 -38 -90 -38t-90 38l-75 75q-38 36 -38 90q0 53 38 91l651 651q37 37 90 37q52 0 91 -37l650 -651q38 -38 38 -91z" />
+<glyph unicode="&#xf078;" horiz-adv-x="1664" d="M1611 832q0 -53 -37 -90l-651 -651q-38 -38 -91 -38q-54 0 -90 38l-651 651q-38 36 -38 90q0 53 38 91l74 75q39 37 91 37q53 0 90 -37l486 -486l486 486q37 37 90 37q52 0 91 -37l75 -75q37 -39 37 -91z" />
+<glyph unicode="&#xf079;" horiz-adv-x="1920" d="M1280 32q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-8 0 -13.5 2t-9 7t-5.5 8t-3 11.5t-1 11.5v13v11v160v416h-192q-26 0 -45 19t-19 45q0 24 15 41l320 384q19 22 49 22t49 -22l320 -384q15 -17 15 -41q0 -26 -19 -45t-45 -19h-192v-384h576q16 0 25 -11l160 -192q7 -11 7 -21 zM1920 448q0 -24 -15 -41l-320 -384q-20 -23 -49 -23t-49 23l-320 384q-15 17 -15 41q0 26 19 45t45 19h192v384h-576q-16 0 -25 12l-160 192q-7 9 -7 20q0 13 9.5 22.5t22.5 9.5h960q8 0 13.5 -2t9 -7t5.5  [...]
+<glyph unicode="&#xf07a;" horiz-adv-x="1664" d="M640 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1536 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1664 1088v-512q0 -24 -16 -42.5t-41 -21.5 l-1044 -122q1 -7 4.5 -21.5t6 -26.5t2.5 -22q0 -16 -24 -64h920q26 0 45 -19t19 -45t-19 -45t-45 -19h-1024q-26 0 -45 19t-19 45q0 14 11 39.5t29.5 59.5t20.5 38l-177 823h-204q-26 0 -45 19t-19 45t19 45t45 19h [...]
+<glyph unicode="&#xf07b;" horiz-adv-x="1664" d="M1664 928v-704q0 -92 -66 -158t-158 -66h-1216q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h672q92 0 158 -66t66 -158z" />
+<glyph unicode="&#xf07c;" horiz-adv-x="1920" d="M1879 584q0 -31 -31 -66l-336 -396q-43 -51 -120.5 -86.5t-143.5 -35.5h-1088q-34 0 -60.5 13t-26.5 43q0 31 31 66l336 396q43 51 120.5 86.5t143.5 35.5h1088q34 0 60.5 -13t26.5 -43zM1536 928v-160h-832q-94 0 -197 -47.5t-164 -119.5l-337 -396l-5 -6q0 4 -0.5 12.5 t-0.5 12.5v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h544q92 0 158 -66t66 -158z" />
+<glyph unicode="&#xf07d;" horiz-adv-x="768" d="M704 1216q0 -26 -19 -45t-45 -19h-128v-1024h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v1024h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45z" />
+<glyph unicode="&#xf07e;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-1024v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h1024v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
+<glyph unicode="&#xf080;" horiz-adv-x="1920" d="M512 512v-384h-256v384h256zM896 1024v-896h-256v896h256zM1280 768v-640h-256v640h256zM1664 1152v-1024h-256v1024h256zM1792 32v1216q0 13 -9.5 22.5t-22.5 9.5h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-1216q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5z M1920 1248v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
+<glyph unicode="&#xf081;" d="M1280 926q-56 -25 -121 -34q68 40 93 117q-65 -38 -134 -51q-61 66 -153 66q-87 0 -148.5 -61.5t-61.5 -148.5q0 -29 5 -48q-129 7 -242 65t-192 155q-29 -50 -29 -106q0 -114 91 -175q-47 1 -100 26v-2q0 -75 50 -133.5t123 -72.5q-29 -8 -51 -8q-13 0 -39 4 q21 -63 74.5 -104t121.5 -42q-116 -90 -261 -90q-26 0 -50 3q148 -94 322 -94q112 0 210 35.5t168 95t120.5 137t75 162t24.5 168.5q0 18 -1 27q63 45 105 109zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t [...]
+<glyph unicode="&#xf082;" d="M1307 618l23 219h-198v109q0 49 15.5 68.5t71.5 19.5h110v219h-175q-152 0 -218 -72t-66 -213v-131h-131v-219h131v-635h262v635h175zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960 q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf083;" horiz-adv-x="1792" d="M928 704q0 14 -9 23t-23 9q-66 0 -113 -47t-47 -113q0 -14 9 -23t23 -9t23 9t9 23q0 40 28 68t68 28q14 0 23 9t9 23zM1152 574q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181zM128 0h1536v128h-1536v-128zM1280 574q0 159 -112.5 271.5 t-271.5 112.5t-271.5 -112.5t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5zM256 1216h384v128h-384v-128zM128 1024h1536v118v138h-828l-64 -128h-644v-128zM1792 1280v-1280q0 -53 -37.5 - [...]
+<glyph unicode="&#xf084;" horiz-adv-x="1792" d="M832 1024q0 80 -56 136t-136 56t-136 -56t-56 -136q0 -42 19 -83q-41 19 -83 19q-80 0 -136 -56t-56 -136t56 -136t136 -56t136 56t56 136q0 42 -19 83q41 -19 83 -19q80 0 136 56t56 136zM1683 320q0 -17 -49 -66t-66 -49q-9 0 -28.5 16t-36.5 33t-38.5 40t-24.5 26 l-96 -96l220 -220q28 -28 28 -68q0 -42 -39 -81t-81 -39q-40 0 -68 28l-671 671q-176 -131 -365 -131q-163 0 -265.5 102.5t-102.5 265.5q0 160 95 313t248 248t313 95q163 0 265.5 -102.5t102.5 -265.5q0 -189  [...]
+<glyph unicode="&#xf085;" horiz-adv-x="1920" d="M896 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1664 128q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1664 1152q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5 t90.5 37.5t37.5 90.5zM1280 731v-185q0 -10 -7 -19.5t-16 -10.5l-155 -24q-11 -35 -32 -76q34 -48 90 -115q7 -10 7 -20q0 -12 -7 -19q-23 -30 -82.5 -89.5t-78.5 -59.5q-11 0 -21 7l-115 90q-37 -19 -77 -31q-11  [...]
+<glyph unicode="&#xf086;" horiz-adv-x="1792" d="M1408 768q0 -139 -94 -257t-256.5 -186.5t-353.5 -68.5q-86 0 -176 16q-124 -88 -278 -128q-36 -9 -86 -16h-3q-11 0 -20.5 8t-11.5 21q-1 3 -1 6.5t0.5 6.5t2 6l2.5 5t3.5 5.5t4 5t4.5 5t4 4.5q5 6 23 25t26 29.5t22.5 29t25 38.5t20.5 44q-124 72 -195 177t-71 224 q0 139 94 257t256.5 186.5t353.5 68.5t353.5 -68.5t256.5 -186.5t94 -257zM1792 512q0 -120 -71 -224.5t-195 -176.5q10 -24 20.5 -44t25 -38.5t22.5 -29t26 -29.5t23 -25q1 -1 4 -4.5t4.5 -5t4 -5t3.5 -5.5l2.5 [...]
+<glyph unicode="&#xf087;" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 768q0 51 -39 89.5t-89 38.5h-352q0 58 48 159.5t48 160.5q0 98 -32 145t-128 47q-26 -26 -38 -85t-30.5 -125.5t-59.5 -109.5q-22 -23 -77 -91q-4 -5 -23 -30t-31.5 -41t-34.5 -42.5 t-40 -44t-38.5 -35.5t-40 -27t-35.5 -9h-32v-640h32q13 0 31.5 -3t33 -6.5t38 -11t35 -11.5t35.5 -12.5t29 -10.5q211 -73 342 -73h121q192 0 192 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 4 [...]
+<glyph unicode="&#xf088;" d="M256 1088q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 512q0 35 -21.5 81t-53.5 47q15 17 25 47.5t10 55.5q0 69 -53 119q18 32 18 69t-17.5 73.5t-47.5 52.5q5 30 5 56q0 85 -49 126t-136 41h-128q-131 0 -342 -73q-5 -2 -29 -10.5 t-35.5 -12.5t-35 -11.5t-38 -11t-33 -6.5t-31.5 -3h-32v-640h32q16 0 35.5 -9t40 -27t38.5 -35.5t40 -44t34.5 -42.5t31.5 -41t23 -30q55 -68 77 -91q41 -43 59.5 -109.5t30.5 -125.5t38 -85q96 0 128 47t32 145q0 59 -48 160.5t-48 159.5h [...]
+<glyph unicode="&#xf089;" horiz-adv-x="896" d="M832 1504v-1339l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41z" />
+<glyph unicode="&#xf08a;" horiz-adv-x="1792" d="M1664 940q0 81 -21.5 143t-55 98.5t-81.5 59.5t-94 31t-98 8t-112 -25.5t-110.5 -64t-86.5 -72t-60 -61.5q-18 -22 -49 -22t-49 22q-24 28 -60 61.5t-86.5 72t-110.5 64t-112 25.5t-98 -8t-94 -31t-81.5 -59.5t-55 -98.5t-21.5 -143q0 -168 187 -355l581 -560l580 559 q188 188 188 356zM1792 940q0 -221 -229 -450l-623 -600q-18 -18 -44 -18t-44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -6 [...]
+<glyph unicode="&#xf08b;" horiz-adv-x="1664" d="M640 96q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h320q13 0 22.5 -9.5t9.5 -22.5q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-66 0 -113 -47t-47 -113v-704 q0 -66 47 -113t113 -47h288h11h13t11.5 -1t11.5 -3t8 -5.5t7 -9t2 -13.5zM1568 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t4 [...]
+<glyph unicode="&#xf08c;" d="M237 122h231v694h-231v-694zM483 1030q-1 52 -36 86t-93 34t-94.5 -34t-36.5 -86q0 -51 35.5 -85.5t92.5 -34.5h1q59 0 95 34.5t36 85.5zM1068 122h231v398q0 154 -73 233t-193 79q-136 0 -209 -117h2v101h-231q3 -66 0 -694h231v388q0 38 7 56q15 35 45 59.5t74 24.5 q116 0 116 -157v-371zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf08d;" horiz-adv-x="1152" d="M480 672v448q0 14 -9 23t-23 9t-23 -9t-9 -23v-448q0 -14 9 -23t23 -9t23 9t9 23zM1152 320q0 -26 -19 -45t-45 -19h-429l-51 -483q-2 -12 -10.5 -20.5t-20.5 -8.5h-1q-27 0 -32 27l-76 485h-404q-26 0 -45 19t-19 45q0 123 78.5 221.5t177.5 98.5v512q-52 0 -90 38 t-38 90t38 90t90 38h640q52 0 90 -38t38 -90t-38 -90t-90 -38v-512q99 0 177.5 -98.5t78.5 -221.5z" />
+<glyph unicode="&#xf08e;" horiz-adv-x="1792" d="M1408 608v-320q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h704q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v320 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1792 1472v-512q0 -26 -19 -45t-45 -19t-45 19l-176 176l-652 -652q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l652 652l-176 176q-19 19 -19 45t19 45t45 19h512q2 [...]
+<glyph unicode="&#xf090;" d="M1184 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45zM1536 992v-704q0 -119 -84.5 -203.5t-203.5 -84.5h-320q-13 0 -22.5 9.5t-9.5 22.5 q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q66 0 113 47t47 113v704q0 66 -47 113t-113 47h-288h-11h-13t-11.5 1t-11.5 3t-8 5.5t-7 9t-2 13.5q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q119 0 203.5 -84.5t84.5 -2 [...]
+<glyph unicode="&#xf091;" horiz-adv-x="1664" d="M458 653q-74 162 -74 371h-256v-96q0 -78 94.5 -162t235.5 -113zM1536 928v96h-256q0 -209 -74 -371q141 29 235.5 113t94.5 162zM1664 1056v-128q0 -71 -41.5 -143t-112 -130t-173 -97.5t-215.5 -44.5q-42 -54 -95 -95q-38 -34 -52.5 -72.5t-14.5 -89.5q0 -54 30.5 -91 t97.5 -37q75 0 133.5 -45.5t58.5 -114.5v-64q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v64q0 69 58.5 114.5t133.5 45.5q67 0 97.5 37t30.5 91q0 51 -14.5 89.5t-52.5 72.5q-53 41 -95 95q-113 5 -215.5 4 [...]
+<glyph unicode="&#xf092;" d="M394 184q-8 -9 -20 3q-13 11 -4 19q8 9 20 -3q12 -11 4 -19zM352 245q9 -12 0 -19q-8 -6 -17 7t0 18q9 7 17 -6zM291 305q-5 -7 -13 -2q-10 5 -7 12q3 5 13 2q10 -5 7 -12zM322 271q-6 -7 -16 3q-9 11 -2 16q6 6 16 -3q9 -11 2 -16zM451 159q-4 -12 -19 -6q-17 4 -13 15 t19 7q16 -5 13 -16zM514 154q0 -11 -16 -11q-17 -2 -17 11q0 11 16 11q17 2 17 -11zM572 164q2 -10 -14 -14t-18 8t14 15q16 2 18 -9zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-224q-16 0 -24.5 1t-19.5 5t-16 14.5t-5 [...]
+<glyph unicode="&#xf093;" horiz-adv-x="1664" d="M1280 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 288v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h427q21 -56 70.5 -92 t110.5 -36h256q61 0 110.5 36t70.5 92h427q40 0 68 -28t28 -68zM1339 936q-17 -40 -59 -40h-256v-448q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v448h-256q-42 0 -59 40q-17 39 14 69l448 448q18 19 45 19t45 [...]
+<glyph unicode="&#xf094;" d="M1407 710q0 44 -7 113.5t-18 96.5q-12 30 -17 44t-9 36.5t-4 48.5q0 23 5 68.5t5 67.5q0 37 -10 55q-4 1 -13 1q-19 0 -58 -4.5t-59 -4.5q-60 0 -176 24t-175 24q-43 0 -94.5 -11.5t-85 -23.5t-89.5 -34q-137 -54 -202 -103q-96 -73 -159.5 -189.5t-88 -236t-24.5 -248.5 q0 -40 12.5 -120t12.5 -121q0 -23 -11 -66.5t-11 -65.5t12 -36.5t34 -14.5q24 0 72.5 11t73.5 11q57 0 169.5 -15.5t169.5 -15.5q181 0 284 36q129 45 235.5 152.5t166 245.5t59.5 275zM1535 712q0 -165 -70 -327.5t-196 -288t- [...]
+<glyph unicode="&#xf095;" horiz-adv-x="1408" d="M1408 296q0 -27 -10 -70.5t-21 -68.5q-21 -50 -122 -106q-94 -51 -186 -51q-27 0 -52.5 3.5t-57.5 12.5t-47.5 14.5t-55.5 20.5t-49 18q-98 35 -175 83q-128 79 -264.5 215.5t-215.5 264.5q-48 77 -83 175q-3 9 -18 49t-20.5 55.5t-14.5 47.5t-12.5 57.5t-3.5 52.5 q0 92 51 186q56 101 106 122q25 11 68.5 21t70.5 10q14 0 21 -3q18 -6 53 -76q11 -19 30 -54t35 -63.5t31 -53.5q3 -4 17.5 -25t21.5 -35.5t7 -28.5q0 -20 -28.5 -50t-62 -55t-62 -53t-28.5 -46q0 -9 5 -22.5t8.5  [...]
+<glyph unicode="&#xf096;" horiz-adv-x="1408" d="M1120 1280h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v832q0 66 -47 113t-113 47zM1408 1120v-832q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf097;" horiz-adv-x="1280" d="M1152 1280h-1024v-1242l423 406l89 85l89 -85l423 -406v1242zM1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289 q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
+<glyph unicode="&#xf098;" d="M1280 343q0 11 -2 16q-3 8 -38.5 29.5t-88.5 49.5l-53 29q-5 3 -19 13t-25 15t-21 5q-18 0 -47 -32.5t-57 -65.5t-44 -33q-7 0 -16.5 3.5t-15.5 6.5t-17 9.5t-14 8.5q-99 55 -170.5 126.5t-126.5 170.5q-2 3 -8.5 14t-9.5 17t-6.5 15.5t-3.5 16.5q0 13 20.5 33.5t45 38.5 t45 39.5t20.5 36.5q0 10 -5 21t-15 25t-13 19q-3 6 -15 28.5t-25 45.5t-26.5 47.5t-25 40.5t-16.5 18t-16 2q-48 0 -101 -22q-46 -21 -80 -94.5t-34 -130.5q0 -16 2.5 -34t5 -30.5t9 -33t10 -29.5t12.5 -33t11 -30q60 -164 216. [...]
+<glyph unicode="&#xf099;" horiz-adv-x="1664" d="M1620 1128q-67 -98 -162 -167q1 -14 1 -42q0 -130 -38 -259.5t-115.5 -248.5t-184.5 -210.5t-258 -146t-323 -54.5q-271 0 -496 145q35 -4 78 -4q225 0 401 138q-105 2 -188 64.5t-114 159.5q33 -5 61 -5q43 0 85 11q-112 23 -185.5 111.5t-73.5 205.5v4q68 -38 146 -41 q-66 44 -105 115t-39 154q0 88 44 163q121 -149 294.5 -238.5t371.5 -99.5q-8 38 -8 74q0 134 94.5 228.5t228.5 94.5q140 0 236 -102q109 21 205 78q-37 -115 -142 -178q93 10 186 50z" />
+<glyph unicode="&#xf09a;" horiz-adv-x="768" d="M511 980h257l-30 -284h-227v-824h-341v824h-170v284h170v171q0 182 86 275.5t283 93.5h227v-284h-142q-39 0 -62.5 -6.5t-34 -23.5t-13.5 -34.5t-3 -49.5v-142z" />
+<glyph unicode="&#xf09b;" d="M1536 640q0 -251 -146.5 -451.5t-378.5 -277.5q-27 -5 -39.5 7t-12.5 30v211q0 97 -52 142q57 6 102.5 18t94 39t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5 q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 - [...]
+<glyph unicode="&#xf09c;" horiz-adv-x="1664" d="M1664 960v-256q0 -26 -19 -45t-45 -19h-64q-26 0 -45 19t-19 45v256q0 106 -75 181t-181 75t-181 -75t-75 -181v-192h96q40 0 68 -28t28 -68v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h672v192q0 185 131.5 316.5t316.5 131.5 t316.5 -131.5t131.5 -316.5z" />
+<glyph unicode="&#xf09d;" horiz-adv-x="1920" d="M1760 1408q66 0 113 -47t47 -113v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600zM160 1280q-13 0 -22.5 -9.5t-9.5 -22.5v-224h1664v224q0 13 -9.5 22.5t-22.5 9.5h-1600zM1760 0q13 0 22.5 9.5t9.5 22.5v608h-1664v-608 q0 -13 9.5 -22.5t22.5 -9.5h1600zM256 128v128h256v-128h-256zM640 128v128h384v-128h-384z" />
+<glyph unicode="&#xf09e;" horiz-adv-x="1408" d="M384 192q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM896 69q2 -28 -17 -48q-18 -21 -47 -21h-135q-25 0 -43 16.5t-20 41.5q-22 229 -184.5 391.5t-391.5 184.5q-25 2 -41.5 20t-16.5 43v135q0 29 21 47q17 17 43 17h5q160 -13 306 -80.5 t259 -181.5q114 -113 181.5 -259t80.5 -306zM1408 67q2 -27 -18 -47q-18 -20 -46 -20h-143q-26 0 -44.5 17.5t-19.5 42.5q-12 215 -101 408.5t-231.5 336t-336 231.5t-408.5 102q-25 1 -42.5 19.5t-17.5 43.5 [...]
+<glyph unicode="&#xf0a0;" d="M1040 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1296 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1408 160v320q0 13 -9.5 22.5t-22.5 9.5 h-1216q-13 0 -22.5 -9.5t-9.5 -22.5v-320q0 -13 9.5 -22.5t22.5 -9.5h1216q13 0 22.5 9.5t9.5 22.5zM178 640h1180l-157 482q-4 13 -16 21.5t-26 8.5h-782q-14 0 -26 -8.5t-16 -21.5zM1536 480v-320q0 -66 -47 -113t-113 -47h-1216q- [...]
+<glyph unicode="&#xf0a1;" horiz-adv-x="1792" d="M1664 896q53 0 90.5 -37.5t37.5 -90.5t-37.5 -90.5t-90.5 -37.5v-384q0 -52 -38 -90t-90 -38q-417 347 -812 380q-58 -19 -91 -66t-31 -100.5t40 -92.5q-20 -33 -23 -65.5t6 -58t33.5 -55t48 -50t61.5 -50.5q-29 -58 -111.5 -83t-168.5 -11.5t-132 55.5q-7 23 -29.5 87.5 t-32 94.5t-23 89t-15 101t3.5 98.5t22 110.5h-122q-66 0 -113 47t-47 113v192q0 66 47 113t113 47h480q435 0 896 384q52 0 90 -38t38 -90v-384zM1536 292v954q-394 -302 -768 -343v-270q377 -42 768 -341z" />
+<glyph unicode="&#xf0a2;" horiz-adv-x="1664" d="M848 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM183 128h1298q-164 181 -246.5 411.5t-82.5 484.5q0 256 -320 256t-320 -256q0 -254 -82.5 -484.5t-246.5 -411.5zM1664 128q0 -52 -38 -90t-90 -38 h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q190 161 287 397.5t97 498.5q0 165 96 262t264 117q-8 18 -8 37q0 40 28 68t68 28t68 -28t28 -68q0 -19 -8 -37q168 -20 264 -117 [...]
+<glyph unicode="&#xf0a3;" d="M1376 640l138 -135q30 -28 20 -70q-12 -41 -52 -51l-188 -48l53 -186q12 -41 -19 -70q-29 -31 -70 -19l-186 53l-48 -188q-10 -40 -51 -52q-12 -2 -19 -2q-31 0 -51 22l-135 138l-135 -138q-28 -30 -70 -20q-41 11 -51 52l-48 188l-186 -53q-41 -12 -70 19q-31 29 -19 70 l53 186l-188 48q-40 10 -52 51q-10 42 20 70l138 135l-138 135q-30 28 -20 70q12 41 52 51l188 48l-53 186q-12 41 19 70q29 31 70 19l186 -53l48 188q10 41 51 51q41 12 70 -19l135 -139l135 139q29 30 70 19q41 -10 51 -51l48 [...]
+<glyph unicode="&#xf0a4;" horiz-adv-x="1792" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 768q0 51 -39 89.5t-89 38.5h-576q0 20 15 48.5t33 55t33 68t15 84.5q0 67 -44.5 97.5t-115.5 30.5q-24 0 -90 -139q-24 -44 -37 -65q-40 -64 -112 -145q-71 -81 -101 -106 q-69 -57 -140 -57h-32v-640h32q72 0 167 -32t193.5 -64t179.5 -32q189 0 189 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5h331q52 0 90 38t38 90zM1792 769q0 -105 -75.5 -181t-1 [...]
+<glyph unicode="&#xf0a5;" horiz-adv-x="1792" d="M1376 128h32v640h-32q-35 0 -67.5 12t-62.5 37t-50 46t-49 54q-2 3 -3.5 4.5t-4 4.5t-4.5 5q-72 81 -112 145q-14 22 -38 68q-1 3 -10.5 22.5t-18.5 36t-20 35.5t-21.5 30.5t-18.5 11.5q-71 0 -115.5 -30.5t-44.5 -97.5q0 -43 15 -84.5t33 -68t33 -55t15 -48.5h-576 q-50 0 -89 -38.5t-39 -89.5q0 -52 38 -90t90 -38h331q-15 -17 -25 -47.5t-10 -55.5q0 -69 53 -119q-18 -32 -18 -69t17.5 -73.5t47.5 -52.5q-4 -24 -4 -56q0 -85 48.5 -126t135.5 -41q84 0 183 32t194 64t167 32z [...]
+<glyph unicode="&#xf0a6;" d="M1280 -64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 700q0 189 -167 189q-26 0 -56 -5q-16 30 -52.5 47.5t-73.5 17.5t-69 -18q-50 53 -119 53q-25 0 -55.5 -10t-47.5 -25v331q0 52 -38 90t-90 38q-51 0 -89.5 -39t-38.5 -89v-576 q-20 0 -48.5 15t-55 33t-68 33t-84.5 15q-67 0 -97.5 -44.5t-30.5 -115.5q0 -24 139 -90q44 -24 65 -37q64 -40 145 -112q81 -71 106 -101q57 -69 57 -140v-32h640v32q0 72 32 167t64 193.5t32 179.5zM1536 705q0 -133 -69 -322q-59 -164 - [...]
+<glyph unicode="&#xf0a7;" d="M1408 576q0 84 -32 183t-64 194t-32 167v32h-640v-32q0 -35 -12 -67.5t-37 -62.5t-46 -50t-54 -49q-9 -8 -14 -12q-81 -72 -145 -112q-22 -14 -68 -38q-3 -1 -22.5 -10.5t-36 -18.5t-35.5 -20t-30.5 -21.5t-11.5 -18.5q0 -71 30.5 -115.5t97.5 -44.5q43 0 84.5 15t68 33 t55 33t48.5 15v-576q0 -50 38.5 -89t89.5 -39q52 0 90 38t38 90v331q46 -35 103 -35q69 0 119 53q32 -18 69 -18t73.5 17.5t52.5 47.5q24 -4 56 -4q85 0 126 48.5t41 135.5zM1280 1344q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -4 [...]
+<glyph unicode="&#xf0a8;" d="M1280 576v128q0 26 -19 45t-45 19h-502l189 189q19 19 19 45t-19 45l-91 91q-18 18 -45 18t-45 -18l-362 -362l-91 -91q-18 -18 -18 -45t18 -45l91 -91l362 -362q18 -18 45 -18t45 18l91 91q18 18 18 45t-18 45l-189 189h502q26 0 45 19t19 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf0a9;" d="M1285 640q0 27 -18 45l-91 91l-362 362q-18 18 -45 18t-45 -18l-91 -91q-18 -18 -18 -45t18 -45l189 -189h-502q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h502l-189 -189q-19 -19 -19 -45t19 -45l91 -91q18 -18 45 -18t45 18l362 362l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf0aa;" d="M1284 641q0 27 -18 45l-362 362l-91 91q-18 18 -45 18t-45 -18l-91 -91l-362 -362q-18 -18 -18 -45t18 -45l91 -91q18 -18 45 -18t45 18l189 189v-502q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v502l189 -189q19 -19 45 -19t45 19l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf0ab;" d="M1284 639q0 27 -18 45l-91 91q-18 18 -45 18t-45 -18l-189 -189v502q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-502l-189 189q-19 19 -45 19t-45 -19l-91 -91q-18 -18 -18 -45t18 -45l362 -362l91 -91q18 -18 45 -18t45 18l91 91l362 362q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf0ac;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM1042 887q-2 -1 -9.5 -9.5t-13.5 -9.5q2 0 4.5 5t5 11t3.5 7q6 7 22 15q14 6 52 12q34 8 51 -11 q-2 2 9.5 13t14.5 12q3 2 15 4.5t15 7.5l2 22q-12 -1 -17.5 7t-6.5 21q0 -2 -6 -8q0 7 -4.5 8t-11.5 -1t-9 -1q-10 3 -15 7.5t-8 16.5t-4 15q-2 5 -9.5 10.5t-9.5 10.5q-1 2 -2.5 5.5t-3 6.5t-4 5.5t-5.5 2.5t-7 -5t-7.5 -10t-4.5 -5 [...]
+<glyph unicode="&#xf0ad;" horiz-adv-x="1664" d="M384 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1028 484l-682 -682q-37 -37 -90 -37q-52 0 -91 37l-106 108q-38 36 -38 90q0 53 38 91l681 681q39 -98 114.5 -173.5t173.5 -114.5zM1662 919q0 -39 -23 -106q-47 -134 -164.5 -217.5 t-258.5 -83.5q-185 0 -316.5 131.5t-131.5 316.5t131.5 316.5t316.5 131.5q58 0 121.5 -16.5t107.5 -46.5q16 -11 16 -28t-16 -28l-293 -169v-224l193 -107q5 3 79 48.5t135.5 81t70.5 35.5q15 0 23.5 -10t8.5 -25z" />
+<glyph unicode="&#xf0ae;" horiz-adv-x="1792" d="M1024 128h640v128h-640v-128zM640 640h1024v128h-1024v-128zM1280 1152h384v128h-384v-128zM1792 320v-256q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 832v-256q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19 t-19 45v256q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 1344v-256q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1664q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf0b0;" horiz-adv-x="1408" d="M1403 1241q17 -41 -14 -70l-493 -493v-742q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-256 256q-19 19 -19 45v486l-493 493q-31 29 -14 70q17 39 59 39h1280q42 0 59 -39z" />
+<glyph unicode="&#xf0b1;" horiz-adv-x="1792" d="M640 1280h512v128h-512v-128zM1792 640v-480q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v480h672v-160q0 -26 19 -45t45 -19h320q26 0 45 19t19 45v160h672zM1024 640v-128h-256v128h256zM1792 1120v-384h-1792v384q0 66 47 113t113 47h352v160q0 40 28 68 t68 28h576q40 0 68 -28t28 -68v-160h352q66 0 113 -47t47 -113z" />
+<glyph unicode="&#xf0b2;" d="M1283 995l-355 -355l355 -355l144 144q29 31 70 14q39 -17 39 -59v-448q0 -26 -19 -45t-45 -19h-448q-42 0 -59 40q-17 39 14 69l144 144l-355 355l-355 -355l144 -144q31 -30 14 -69q-17 -40 -59 -40h-448q-26 0 -45 19t-19 45v448q0 42 40 59q39 17 69 -14l144 -144 l355 355l-355 355l-144 -144q-19 -19 -45 -19q-12 0 -24 5q-40 17 -40 59v448q0 26 19 45t45 19h448q42 0 59 -40q17 -39 -14 -69l-144 -144l355 -355l355 355l-144 144q-31 30 -14 69q17 40 59 40h448q26 0 45 -19t19 -45v-448q0  [...]
+<glyph unicode="&#xf0c0;" horiz-adv-x="1920" d="M593 640q-162 -5 -265 -128h-134q-82 0 -138 40.5t-56 118.5q0 353 124 353q6 0 43.5 -21t97.5 -42.5t119 -21.5q67 0 133 23q-5 -37 -5 -66q0 -139 81 -256zM1664 3q0 -120 -73 -189.5t-194 -69.5h-874q-121 0 -194 69.5t-73 189.5q0 53 3.5 103.5t14 109t26.5 108.5 t43 97.5t62 81t85.5 53.5t111.5 20q10 0 43 -21.5t73 -48t107 -48t135 -21.5t135 21.5t107 48t73 48t43 21.5q61 0 111.5 -20t85.5 -53.5t62 -81t43 -97.5t26.5 -108.5t14 -109t3.5 -103.5zM640 1280q0 -106 -7 [...]
+<glyph unicode="&#xf0c1;" horiz-adv-x="1664" d="M1456 320q0 40 -28 68l-208 208q-28 28 -68 28q-42 0 -72 -32q3 -3 19 -18.5t21.5 -21.5t15 -19t13 -25.5t3.5 -27.5q0 -40 -28 -68t-68 -28q-15 0 -27.5 3.5t-25.5 13t-19 15t-21.5 21.5t-18.5 19q-33 -31 -33 -73q0 -40 28 -68l206 -207q27 -27 68 -27q40 0 68 26 l147 146q28 28 28 67zM753 1025q0 40 -28 68l-206 207q-28 28 -68 28q-39 0 -68 -27l-147 -146q-28 -28 -28 -67q0 -40 28 -68l208 -208q27 -27 68 -27q42 0 72 31q-3 3 -19 18.5t-21.5 21.5t-15 19t-13 25.5t-3. [...]
+<glyph unicode="&#xf0c2;" horiz-adv-x="1920" d="M1920 384q0 -159 -112.5 -271.5t-271.5 -112.5h-1088q-185 0 -316.5 131.5t-131.5 316.5q0 132 71 241.5t187 163.5q-2 28 -2 43q0 212 150 362t362 150q158 0 286.5 -88t187.5 -230q70 62 166 62q106 0 181 -75t75 -181q0 -75 -41 -138q129 -30 213 -134.5t84 -239.5z " />
+<glyph unicode="&#xf0c3;" horiz-adv-x="1664" d="M1527 88q56 -89 21.5 -152.5t-140.5 -63.5h-1152q-106 0 -140.5 63.5t21.5 152.5l503 793v399h-64q-26 0 -45 19t-19 45t19 45t45 19h512q26 0 45 -19t19 -45t-19 -45t-45 -19h-64v-399zM748 813l-272 -429h712l-272 429l-20 31v37v399h-128v-399v-37z" />
+<glyph unicode="&#xf0c4;" horiz-adv-x="1792" d="M960 640q26 0 45 -19t19 -45t-19 -45t-45 -19t-45 19t-19 45t19 45t45 19zM1260 576l507 -398q28 -20 25 -56q-5 -35 -35 -51l-128 -64q-13 -7 -29 -7q-17 0 -31 8l-690 387l-110 -66q-8 -4 -12 -5q14 -49 10 -97q-7 -77 -56 -147.5t-132 -123.5q-132 -84 -277 -84 q-136 0 -222 78q-90 84 -79 207q7 76 56 147t131 124q132 84 278 84q83 0 151 -31q9 13 22 22l122 73l-122 73q-13 9 -22 22q-68 -31 -151 -31q-146 0 -278 84q-82 53 -131 124t-56 147q-5 59 15.5 113t63.5 93q85 [...]
+<glyph unicode="&#xf0c5;" horiz-adv-x="1792" d="M1696 1152q40 0 68 -28t28 -68v-1216q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v288h-544q-40 0 -68 28t-28 68v672q0 40 20 88t48 76l408 408q28 28 76 48t88 20h416q40 0 68 -28t28 -68v-328q68 40 128 40h416zM1152 939l-299 -299h299v299zM512 1323l-299 -299 h299v299zM708 676l316 316v416h-384v-416q0 -40 -28 -68t-68 -28h-416v-640h512v256q0 40 20 88t48 76zM1664 -128v1152h-384v-416q0 -40 -28 -68t-68 -28h-416v-640h896z" />
+<glyph unicode="&#xf0c6;" horiz-adv-x="1408" d="M1404 151q0 -117 -79 -196t-196 -79q-135 0 -235 100l-777 776q-113 115 -113 271q0 159 110 270t269 111q158 0 273 -113l605 -606q10 -10 10 -22q0 -16 -30.5 -46.5t-46.5 -30.5q-13 0 -23 10l-606 607q-79 77 -181 77q-106 0 -179 -75t-73 -181q0 -105 76 -181 l776 -777q63 -63 145 -63q64 0 106 42t42 106q0 82 -63 145l-581 581q-26 24 -60 24q-29 0 -48 -19t-19 -48q0 -32 25 -59l410 -410q10 -10 10 -22q0 -16 -31 -47t-47 -31q-12 0 -22 10l-410 410q-63 61 -63 149q0  [...]
+<glyph unicode="&#xf0c7;" d="M384 0h768v384h-768v-384zM1280 0h128v896q0 14 -10 38.5t-20 34.5l-281 281q-10 10 -34 20t-39 10v-416q0 -40 -28 -68t-68 -28h-576q-40 0 -68 28t-28 68v416h-128v-1280h128v416q0 40 28 68t68 28h832q40 0 68 -28t28 -68v-416zM896 928v320q0 13 -9.5 22.5t-22.5 9.5 h-192q-13 0 -22.5 -9.5t-9.5 -22.5v-320q0 -13 9.5 -22.5t22.5 -9.5h192q13 0 22.5 9.5t9.5 22.5zM1536 896v-928q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1344q0 40 28 68t68 28h928q40 0 88 -20t76 -48l280 -280q28 [...]
+<glyph unicode="&#xf0c8;" d="M1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf0c9;" d="M1536 192v-128q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1536 704v-128q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1536 1216v-128q0 -26 -19 -45 t-45 -19h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf0ca;" horiz-adv-x="1792" d="M384 128q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM384 640q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1216q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5 t22.5 9.5h1216q13 0 22.5 -9.5t9.5 -22.5zM384 1152q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1792 736v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1216q-13 0 -22.5 9.5t-9.5 22. [...]
+<glyph unicode="&#xf0cb;" horiz-adv-x="1792" d="M381 -84q0 -80 -54.5 -126t-135.5 -46q-106 0 -172 66l57 88q49 -45 106 -45q29 0 50.5 14.5t21.5 42.5q0 64 -105 56l-26 56q8 10 32.5 43.5t42.5 54t37 38.5v1q-16 0 -48.5 -1t-48.5 -1v-53h-106v152h333v-88l-95 -115q51 -12 81 -49t30 -88zM383 543v-159h-362 q-6 36 -6 54q0 51 23.5 93t56.5 68t66 47.5t56.5 43.5t23.5 45q0 25 -14.5 38.5t-39.5 13.5q-46 0 -81 -58l-85 59q24 51 71.5 79.5t105.5 28.5q73 0 123 -41.5t50 -112.5q0 -50 -34 -91.5t-75 -64.5t-75.5 -50.5t- [...]
+<glyph unicode="&#xf0cc;" horiz-adv-x="1792" d="M1760 640q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-1728q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h1728zM483 704q-28 35 -51 80q-48 97 -48 188q0 181 134 309q133 127 393 127q50 0 167 -19q66 -12 177 -48q10 -38 21 -118q14 -123 14 -183q0 -18 -5 -45l-12 -3l-84 6 l-14 2q-50 149 -103 205q-88 91 -210 91q-114 0 -182 -59q-67 -58 -67 -146q0 -73 66 -140t279 -129q69 -20 173 -66q58 -28 95 -52h-743zM990 448h411q7 -39 7 -92q0 -111 -41 -212q-23 -55 -71 -104q-37 -3 [...]
+<glyph unicode="&#xf0cd;" d="M48 1313q-37 2 -45 4l-3 88q13 1 40 1q60 0 112 -4q132 -7 166 -7q86 0 168 3q116 4 146 5q56 0 86 2l-1 -14l2 -64v-9q-60 -9 -124 -9q-60 0 -79 -25q-13 -14 -13 -132q0 -13 0.5 -32.5t0.5 -25.5l1 -229l14 -280q6 -124 51 -202q35 -59 96 -92q88 -47 177 -47 q104 0 191 28q56 18 99 51q48 36 65 64q36 56 53 114q21 73 21 229q0 79 -3.5 128t-11 122.5t-13.5 159.5l-4 59q-5 67 -24 88q-34 35 -77 34l-100 -2l-14 3l2 86h84l205 -10q76 -3 196 10l18 -2q6 -38 6 -51q0 -7 -4 -31q-45 -12 -84 -1 [...]
+<glyph unicode="&#xf0ce;" horiz-adv-x="1664" d="M512 160v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM512 544v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM1024 160v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23 v-192q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM512 928v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h320q14 0 23 9t9 23zM1024 544v192q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -2 [...]
+<glyph unicode="&#xf0d0;" horiz-adv-x="1664" d="M1190 955l293 293l-107 107l-293 -293zM1637 1248q0 -27 -18 -45l-1286 -1286q-18 -18 -45 -18t-45 18l-198 198q-18 18 -18 45t18 45l1286 1286q18 18 45 18t45 -18l198 -198q18 -18 18 -45zM286 1438l98 -30l-98 -30l-30 -98l-30 98l-98 30l98 30l30 98zM636 1276 l196 -60l-196 -60l-60 -196l-60 196l-196 60l196 60l60 196zM1566 798l98 -30l-98 -30l-30 -98l-30 98l-98 30l98 30l30 98zM926 1438l98 -30l-98 -30l-30 -98l-30 98l-98 30l98 30l30 98z" />
+<glyph unicode="&#xf0d1;" horiz-adv-x="1792" d="M640 128q0 52 -38 90t-90 38t-90 -38t-38 -90t38 -90t90 -38t90 38t38 90zM256 640h384v256h-158q-13 0 -22 -9l-195 -195q-9 -9 -9 -22v-30zM1536 128q0 52 -38 90t-90 38t-90 -38t-38 -90t38 -90t90 -38t90 38t38 90zM1792 1216v-1024q0 -15 -4 -26.5t-13.5 -18.5 t-16.5 -11.5t-23.5 -6t-22.5 -2t-25.5 0t-22.5 0.5q0 -106 -75 -181t-181 -75t-181 75t-75 181h-384q0 -106 -75 -181t-181 -75t-181 75t-75 181h-64q-3 0 -22.5 -0.5t-25.5 0t-22.5 2t-23.5 6t-16.5 11.5t-13.5  [...]
+<glyph unicode="&#xf0d2;" d="M1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103q-111 0 -218 32q59 93 78 164q9 34 54 211q20 -39 73 -67.5t114 -28.5q121 0 216 68.5t147 188.5t52 270q0 114 -59.5 214t-172.5 163t-255 63q-105 0 -196 -29t-154.5 -77t-109 -110.5t-67 -129.5t-21.5 -134 q0 -104 40 -183t117 -111q30 -12 38 20q2 7 8 31t8 30q6 23 -11 43q-51 61 -51 151q0 151 104.5 259.5t273.5 108.5q151 0 235.5 -82t84.5 -213q0 -170 -68.5 -289t-175.5 -119q-61 0 -98 43.5t-23 104.5q8 35 26.5 93.5t30 103t11 [...]
+<glyph unicode="&#xf0d3;" d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-725q85 122 108 210q9 34 53 209q21 -39 73.5 -67t112.5 -28q181 0 295.5 147.5t114.5 373.5q0 84 -35 162.5t-96.5 139t-152.5 97t-197 36.5q-104 0 -194.5 -28.5t-153 -76.5 t-107.5 -109.5t-66.5 -128t-21.5 -132.5q0 -102 39.5 -180t116.5 -110q13 -5 23.5 0t14.5 19q10 44 15 61q6 23 -11 42q-50 62 -50 150q0 150 103.5 256.5t270.5 106.5q149 0 232.5 -81t83.5 -210q0 -168 -67.5 -286t-173.5 -118q-60 0 - [...]
+<glyph unicode="&#xf0d4;" d="M678 -57q0 -38 -10 -71h-380q-95 0 -171.5 56.5t-103.5 147.5q24 45 69 77.5t100 49.5t107 24t107 7q32 0 49 -2q6 -4 30.5 -21t33 -23t31 -23t32 -25.5t27.5 -25.5t26.5 -29.5t21 -30.5t17.5 -34.5t9.5 -36t4.5 -40.5zM385 294q-234 -7 -385 -85v433q103 -118 273 -118 q32 0 70 5q-21 -61 -21 -86q0 -67 63 -149zM558 805q0 -100 -43.5 -160.5t-140.5 -60.5q-51 0 -97 26t-78 67.5t-56 93.5t-35.5 104t-11.5 99q0 96 51.5 165t144.5 69q66 0 119 -41t84 -104t47 -130t16 -128zM1536 896v-736q0 -1 [...]
+<glyph unicode="&#xf0d5;" horiz-adv-x="1664" d="M876 71q0 21 -4.5 40.5t-9.5 36t-17.5 34.5t-21 30.5t-26.5 29.5t-27.5 25.5t-32 25.5t-31 23t-33 23t-30.5 21q-17 2 -50 2q-54 0 -106 -7t-108 -25t-98 -46t-69 -75t-27 -107q0 -68 35.5 -121.5t93 -84t120.5 -45.5t127 -15q59 0 112.5 12.5t100.5 39t74.5 73.5 t27.5 110zM756 933q0 60 -16.5 127.5t-47 130.5t-84 104t-119.5 41q-93 0 -144 -69t-51 -165q0 -47 11.5 -99t35.5 -104t56 -93.5t78 -67.5t97 -26q97 0 140.5 60.5t43.5 160.5zM625 1408h437l-135 -79h-135q71 -45 [...]
+<glyph unicode="&#xf0d6;" horiz-adv-x="1920" d="M768 384h384v96h-128v448h-114l-148 -137l77 -80q42 37 55 57h2v-288h-128v-96zM1280 640q0 -70 -21 -142t-59.5 -134t-101.5 -101t-138 -39t-138 39t-101.5 101t-59.5 134t-21 142t21 142t59.5 134t101.5 101t138 39t138 -39t101.5 -101t59.5 -134t21 -142zM1792 384 v512q-106 0 -181 75t-75 181h-1152q0 -106 -75 -181t-181 -75v-512q106 0 181 -75t75 -181h1152q0 106 75 181t181 75zM1920 1216v-1152q0 -26 -19 -45t-45 -19h-1792q-26 0 -45 19t-19 45v1152q0 26 19 45t45  [...]
+<glyph unicode="&#xf0d7;" horiz-adv-x="1024" d="M1024 832q0 -26 -19 -45l-448 -448q-19 -19 -45 -19t-45 19l-448 448q-19 19 -19 45t19 45t45 19h896q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf0d8;" horiz-adv-x="1024" d="M1024 320q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45l448 448q19 19 45 19t45 -19l448 -448q19 -19 19 -45z" />
+<glyph unicode="&#xf0d9;" horiz-adv-x="640" d="M640 1088v-896q0 -26 -19 -45t-45 -19t-45 19l-448 448q-19 19 -19 45t19 45l448 448q19 19 45 19t45 -19t19 -45z" />
+<glyph unicode="&#xf0da;" horiz-adv-x="640" d="M576 640q0 -26 -19 -45l-448 -448q-19 -19 -45 -19t-45 19t-19 45v896q0 26 19 45t45 19t45 -19l448 -448q19 -19 19 -45z" />
+<glyph unicode="&#xf0db;" horiz-adv-x="1664" d="M160 0h608v1152h-640v-1120q0 -13 9.5 -22.5t22.5 -9.5zM1536 32v1120h-640v-1152h608q13 0 22.5 9.5t9.5 22.5zM1664 1248v-1216q0 -66 -47 -113t-113 -47h-1344q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1344q66 0 113 -47t47 -113z" />
+<glyph unicode="&#xf0dc;" horiz-adv-x="1024" d="M1024 448q0 -26 -19 -45l-448 -448q-19 -19 -45 -19t-45 19l-448 448q-19 19 -19 45t19 45t45 19h896q26 0 45 -19t19 -45zM1024 832q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45l448 448q19 19 45 19t45 -19l448 -448q19 -19 19 -45z" />
+<glyph unicode="&#xf0dd;" horiz-adv-x="1024" d="M1024 448q0 -26 -19 -45l-448 -448q-19 -19 -45 -19t-45 19l-448 448q-19 19 -19 45t19 45t45 19h896q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf0de;" horiz-adv-x="1024" d="M1024 832q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45l448 448q19 19 45 19t45 -19l448 -448q19 -19 19 -45z" />
+<glyph unicode="&#xf0e0;" horiz-adv-x="1792" d="M1792 826v-794q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v794q44 -49 101 -87q362 -246 497 -345q57 -42 92.5 -65.5t94.5 -48t110 -24.5h1h1q51 0 110 24.5t94.5 48t92.5 65.5q170 123 498 345q57 39 100 87zM1792 1120q0 -79 -49 -151t-122 -123 q-376 -261 -468 -325q-10 -7 -42.5 -30.5t-54 -38t-52 -32.5t-57.5 -27t-50 -9h-1h-1q-23 0 -50 9t-57.5 27t-52 32.5t-54 38t-42.5 30.5q-91 64 -262 182.5t-205 142.5q-62 42 -117 115.5t-55 136.5q0 78 41.5 130t11 [...]
+<glyph unicode="&#xf0e1;" d="M349 911v-991h-330v991h330zM370 1217q1 -73 -50.5 -122t-135.5 -49h-2q-82 0 -132 49t-50 122q0 74 51.5 122.5t134.5 48.5t133 -48.5t51 -122.5zM1536 488v-568h-329v530q0 105 -40.5 164.5t-126.5 59.5q-63 0 -105.5 -34.5t-63.5 -85.5q-11 -30 -11 -81v-553h-329 q2 399 2 647t-1 296l-1 48h329v-144h-2q20 32 41 56t56.5 52t87 43.5t114.5 15.5q171 0 275 -113.5t104 -332.5z" />
+<glyph unicode="&#xf0e2;" d="M1536 640q0 -156 -61 -298t-164 -245t-245 -164t-298 -61q-172 0 -327 72.5t-264 204.5q-7 10 -6.5 22.5t8.5 20.5l137 138q10 9 25 9q16 -2 23 -12q73 -95 179 -147t225 -52q104 0 198.5 40.5t163.5 109.5t109.5 163.5t40.5 198.5t-40.5 198.5t-109.5 163.5 t-163.5 109.5t-198.5 40.5q-98 0 -188 -35.5t-160 -101.5l137 -138q31 -30 14 -69q-17 -40 -59 -40h-448q-26 0 -45 19t-19 45v448q0 42 40 59q39 17 69 -14l130 -129q107 101 244.5 156.5t284.5 55.5q156 0 298 -61t245 -164t164 -245t61 - [...]
+<glyph unicode="&#xf0e3;" horiz-adv-x="1792" d="M1771 0q0 -53 -37 -90l-107 -108q-39 -37 -91 -37q-53 0 -90 37l-363 364q-38 36 -38 90q0 53 43 96l-256 256l-126 -126q-14 -14 -34 -14t-34 14q2 -2 12.5 -12t12.5 -13t10 -11.5t10 -13.5t6 -13.5t5.5 -16.5t1.5 -18q0 -38 -28 -68q-3 -3 -16.5 -18t-19 -20.5 t-18.5 -16.5t-22 -15.5t-22 -9t-26 -4.5q-40 0 -68 28l-408 408q-28 28 -28 68q0 13 4.5 26t9 22t15.5 22t16.5 18.5t20.5 19t18 16.5q30 28 68 28q10 0 18 -1.5t16.5 -5.5t13.5 -6t13.5 -10t11.5 -10t13 -12.5t12 - [...]
+<glyph unicode="&#xf0e4;" horiz-adv-x="1792" d="M384 384q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM576 832q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1004 351l101 382q6 26 -7.5 48.5t-38.5 29.5 t-48 -6.5t-30 -39.5l-101 -382q-60 -5 -107 -43.5t-63 -98.5q-20 -77 20 -146t117 -89t146 20t89 117q16 60 -6 117t-72 91zM1664 384q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37 [...]
+<glyph unicode="&#xf0e5;" horiz-adv-x="1792" d="M896 1152q-204 0 -381.5 -69.5t-282 -187.5t-104.5 -255q0 -112 71.5 -213.5t201.5 -175.5l87 -50l-27 -96q-24 -91 -70 -172q152 63 275 171l43 38l57 -6q69 -8 130 -8q204 0 381.5 69.5t282 187.5t104.5 255t-104.5 255t-282 187.5t-381.5 69.5zM1792 640 q0 -174 -120 -321.5t-326 -233t-450 -85.5q-70 0 -145 8q-198 -175 -460 -242q-49 -14 -114 -22h-5q-15 0 -27 10.5t-16 27.5v1q-3 4 -0.5 12t2 10t4.5 9.5l6 9t7 8.5t8 9q7 8 31 34.5t34.5 38t31 39.5t32.5 51t27 59t26  [...]
+<glyph unicode="&#xf0e6;" horiz-adv-x="1792" d="M704 1152q-153 0 -286 -52t-211.5 -141t-78.5 -191q0 -82 53 -158t149 -132l97 -56l-35 -84q34 20 62 39l44 31l53 -10q78 -14 153 -14q153 0 286 52t211.5 141t78.5 191t-78.5 191t-211.5 141t-286 52zM704 1280q191 0 353.5 -68.5t256.5 -186.5t94 -257t-94 -257 t-256.5 -186.5t-353.5 -68.5q-86 0 -176 16q-124 -88 -278 -128q-36 -9 -86 -16h-3q-11 0 -20.5 8t-11.5 21q-1 3 -1 6.5t0.5 6.5t2 6l2.5 5t3.5 5.5t4 5t4.5 5t4 4.5q5 6 23 25t26 29.5t22.5 29t25 38.5t20.5 44q [...]
+<glyph unicode="&#xf0e7;" horiz-adv-x="896" d="M885 970q18 -20 7 -44l-540 -1157q-13 -25 -42 -25q-4 0 -14 2q-17 5 -25.5 19t-4.5 30l197 808l-406 -101q-4 -1 -12 -1q-18 0 -31 11q-18 15 -13 39l201 825q4 14 16 23t28 9h328q19 0 32 -12.5t13 -29.5q0 -8 -5 -18l-171 -463l396 98q8 2 12 2q19 0 34 -15z" />
+<glyph unicode="&#xf0e8;" horiz-adv-x="1792" d="M1792 288v-320q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h96v192h-512v-192h96q40 0 68 -28t28 -68v-320q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h96v192h-512v-192h96q40 0 68 -28t28 -68v-320 q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h96v192q0 52 38 90t90 38h512v192h-96q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h320q40 0 68 -28t28 -68v-320q0 -40 -28 -68t-68 -28h-96v-192h [...]
+<glyph unicode="&#xf0e9;" horiz-adv-x="1664" d="M896 708v-580q0 -104 -76 -180t-180 -76t-180 76t-76 180q0 26 19 45t45 19t45 -19t19 -45q0 -50 39 -89t89 -39t89 39t39 89v580q33 11 64 11t64 -11zM1664 681q0 -13 -9.5 -22.5t-22.5 -9.5q-11 0 -23 10q-49 46 -93 69t-102 23q-68 0 -128 -37t-103 -97 q-7 -10 -17.5 -28t-14.5 -24q-11 -17 -28 -17q-18 0 -29 17q-4 6 -14.5 24t-17.5 28q-43 60 -102.5 97t-127.5 37t-127.5 -37t-102.5 -97q-7 -10 -17.5 -28t-14.5 -24q-11 -17 -29 -17q-17 0 -28 17q-4 6 -14.5 24t-17.5 2 [...]
+<glyph unicode="&#xf0ea;" horiz-adv-x="1792" d="M768 -128h896v640h-416q-40 0 -68 28t-28 68v416h-384v-1152zM1024 1312v64q0 13 -9.5 22.5t-22.5 9.5h-704q-13 0 -22.5 -9.5t-9.5 -22.5v-64q0 -13 9.5 -22.5t22.5 -9.5h704q13 0 22.5 9.5t9.5 22.5zM1280 640h299l-299 299v-299zM1792 512v-672q0 -40 -28 -68t-68 -28 h-960q-40 0 -68 28t-28 68v160h-544q-40 0 -68 28t-28 68v1344q0 40 28 68t68 28h1088q40 0 68 -28t28 -68v-328q21 -13 36 -28l408 -408q28 -28 48 -76t20 -88z" />
+<glyph unicode="&#xf0eb;" horiz-adv-x="1024" d="M736 960q0 -13 -9.5 -22.5t-22.5 -9.5t-22.5 9.5t-9.5 22.5q0 46 -54 71t-106 25q-13 0 -22.5 9.5t-9.5 22.5t9.5 22.5t22.5 9.5q50 0 99.5 -16t87 -54t37.5 -90zM896 960q0 72 -34.5 134t-90 101.5t-123 62t-136.5 22.5t-136.5 -22.5t-123 -62t-90 -101.5t-34.5 -134 q0 -101 68 -180q10 -11 30.5 -33t30.5 -33q128 -153 141 -298h228q13 145 141 298q10 11 30.5 33t30.5 33q68 79 68 180zM1024 960q0 -155 -103 -268q-45 -49 -74.5 -87t-59.5 -95.5t-34 -107.5q47 -28 47 -82q [...]
+<glyph unicode="&#xf0ec;" horiz-adv-x="1792" d="M1792 352v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5q-12 0 -24 10l-319 320q-9 9 -9 22q0 14 9 23l320 320q9 9 23 9q13 0 22.5 -9.5t9.5 -22.5v-192h1376q13 0 22.5 -9.5t9.5 -22.5zM1792 896q0 -14 -9 -23l-320 -320q-9 -9 -23 -9 q-13 0 -22.5 9.5t-9.5 22.5v192h-1376q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1376v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23z" />
+<glyph unicode="&#xf0ed;" horiz-adv-x="1920" d="M1280 608q0 14 -9 23t-23 9h-224v352q0 13 -9.5 22.5t-22.5 9.5h-192q-13 0 -22.5 -9.5t-9.5 -22.5v-352h-224q-13 0 -22.5 -9.5t-9.5 -22.5q0 -14 9 -23l352 -352q9 -9 23 -9t23 9l351 351q10 12 10 24zM1920 384q0 -159 -112.5 -271.5t-271.5 -112.5h-1088 q-185 0 -316.5 131.5t-131.5 316.5q0 130 70 240t188 165q-2 30 -2 43q0 212 150 362t362 150q156 0 285.5 -87t188.5 -231q71 62 166 62q106 0 181 -75t75 -181q0 -76 -41 -138q130 -31 213.5 -135.5t83.5 -238.5z" />
+<glyph unicode="&#xf0ee;" horiz-adv-x="1920" d="M1280 672q0 14 -9 23l-352 352q-9 9 -23 9t-23 -9l-351 -351q-10 -12 -10 -24q0 -14 9 -23t23 -9h224v-352q0 -13 9.5 -22.5t22.5 -9.5h192q13 0 22.5 9.5t9.5 22.5v352h224q13 0 22.5 9.5t9.5 22.5zM1920 384q0 -159 -112.5 -271.5t-271.5 -112.5h-1088 q-185 0 -316.5 131.5t-131.5 316.5q0 130 70 240t188 165q-2 30 -2 43q0 212 150 362t362 150q156 0 285.5 -87t188.5 -231q71 62 166 62q106 0 181 -75t75 -181q0 -76 -41 -138q130 -31 213.5 -135.5t83.5 -238.5z" />
+<glyph unicode="&#xf0f0;" horiz-adv-x="1408" d="M384 192q0 -26 -19 -45t-45 -19t-45 19t-19 45t19 45t45 19t45 -19t19 -45zM1408 131q0 -121 -73 -190t-194 -69h-874q-121 0 -194 69t-73 190q0 68 5.5 131t24 138t47.5 132.5t81 103t120 60.5q-22 -52 -22 -120v-203q-58 -20 -93 -70t-35 -111q0 -80 56 -136t136 -56 t136 56t56 136q0 61 -35.5 111t-92.5 70v203q0 62 25 93q132 -104 295 -104t295 104q25 -31 25 -93v-64q-106 0 -181 -75t-75 -181v-89q-32 -29 -32 -71q0 -40 28 -68t68 -28t68 28t28 68q0 42 -32 71v89q0 52 [...]
+<glyph unicode="&#xf0f1;" horiz-adv-x="1408" d="M1280 832q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 832q0 -62 -35.5 -111t-92.5 -70v-395q0 -159 -131.5 -271.5t-316.5 -112.5t-316.5 112.5t-131.5 271.5v132q-164 20 -274 128t-110 252v512q0 26 19 45t45 19q6 0 16 -2q17 30 47 48 t65 18q53 0 90.5 -37.5t37.5 -90.5t-37.5 -90.5t-90.5 -37.5q-33 0 -64 18v-402q0 -106 94 -181t226 -75t226 75t94 181v402q-31 -18 -64 -18q-53 0 -90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5q35 0 65 -18t47 - [...]
+<glyph unicode="&#xf0f2;" horiz-adv-x="1792" d="M640 1152h512v128h-512v-128zM288 1152v-1280h-64q-92 0 -158 66t-66 158v832q0 92 66 158t158 66h64zM1408 1152v-1280h-1024v1280h128v160q0 40 28 68t68 28h576q40 0 68 -28t28 -68v-160h128zM1792 928v-832q0 -92 -66 -158t-158 -66h-64v1280h64q92 0 158 -66 t66 -158z" />
+<glyph unicode="&#xf0f3;" horiz-adv-x="1664" d="M848 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM1664 128q0 -52 -38 -90t-90 -38h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q190 161 287 397.5t97 498.5 q0 165 96 262t264 117q-8 18 -8 37q0 40 28 68t68 28t68 -28t28 -68q0 -19 -8 -37q168 -20 264 -117t96 -262q0 -262 97 -498.5t287 -397.5z" />
+<glyph unicode="&#xf0f4;" horiz-adv-x="1920" d="M1664 896q0 80 -56 136t-136 56h-64v-384h64q80 0 136 56t56 136zM0 128h1792q0 -106 -75 -181t-181 -75h-1280q-106 0 -181 75t-75 181zM1856 896q0 -159 -112.5 -271.5t-271.5 -112.5h-64v-32q0 -92 -66 -158t-158 -66h-704q-92 0 -158 66t-66 158v736q0 26 19 45 t45 19h1152q159 0 271.5 -112.5t112.5 -271.5z" />
+<glyph unicode="&#xf0f5;" horiz-adv-x="1408" d="M640 1472v-640q0 -61 -35.5 -111t-92.5 -70v-779q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v779q-57 20 -92.5 70t-35.5 111v640q0 26 19 45t45 19t45 -19t19 -45v-416q0 -26 19 -45t45 -19t45 19t19 45v416q0 26 19 45t45 19t45 -19t19 -45v-416q0 -26 19 -45 t45 -19t45 19t19 45v416q0 26 19 45t45 19t45 -19t19 -45zM1408 1472v-1600q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v512h-224q-13 0 -22.5 9.5t-9.5 22.5v800q0 132 94 226t226 94h256q26 0 45 -19t1 [...]
+<glyph unicode="&#xf0f6;" horiz-adv-x="1280" d="M1024 352v-64q0 -14 -9 -23t-23 -9h-704q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h704q14 0 23 -9t9 -23zM1024 608v-64q0 -14 -9 -23t-23 -9h-704q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h704q14 0 23 -9t9 -23zM128 0h1024v768h-416q-40 0 -68 28t-28 68v416h-512v-1280z M768 896h376q-10 29 -22 41l-313 313q-12 12 -41 22v-376zM1280 864v-896q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v1344q0 40 28 68t68 28h640q40 0 88 -20t76 -48l312 -312q28 -28 48 -76t20 -88z" />
+<glyph unicode="&#xf0f7;" horiz-adv-x="1408" d="M384 224v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M640 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22 [...]
+<glyph unicode="&#xf0f8;" horiz-adv-x="1408" d="M384 224v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M640 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22 [...]
+<glyph unicode="&#xf0f9;" horiz-adv-x="1920" d="M640 128q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM256 640h384v256h-158q-14 -2 -22 -9l-195 -195q-7 -12 -9 -22v-30zM1536 128q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5 t90.5 37.5t37.5 90.5zM1664 800v192q0 14 -9 23t-23 9h-224v224q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-224h-224q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h224v-224q0 -14 9 -23t23 -9h192q14 0 23  [...]
+<glyph unicode="&#xf0fa;" horiz-adv-x="1792" d="M1280 416v192q0 14 -9 23t-23 9h-224v224q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-224h-224q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h224v-224q0 -14 9 -23t23 -9h192q14 0 23 9t9 23v224h224q14 0 23 9t9 23zM640 1152h512v128h-512v-128zM256 1152v-1280h-32 q-92 0 -158 66t-66 158v832q0 92 66 158t158 66h32zM1440 1152v-1280h-1088v1280h160v160q0 40 28 68t68 28h576q40 0 68 -28t28 -68v-160h160zM1792 928v-832q0 -92 -66 -158t-158 -66h-32v1280h32q92 0 15 [...]
+<glyph unicode="&#xf0fb;" horiz-adv-x="1920" d="M1920 576q-1 -32 -288 -96l-352 -32l-224 -64h-64l-293 -352h69q26 0 45 -4.5t19 -11.5t-19 -11.5t-45 -4.5h-96h-160h-64v32h64v416h-160l-192 -224h-96l-32 32v192h32v32h128v8l-192 24v128l192 24v8h-128v32h-32v192l32 32h96l192 -224h160v416h-64v32h64h160h96 q26 0 45 -4.5t19 -11.5t-19 -11.5t-45 -4.5h-69l293 -352h64l224 -64l352 -32q261 -58 287 -93z" />
+<glyph unicode="&#xf0fc;" horiz-adv-x="1664" d="M640 640v384h-256v-256q0 -53 37.5 -90.5t90.5 -37.5h128zM1664 192v-192h-1152v192l128 192h-128q-159 0 -271.5 112.5t-112.5 271.5v320l-64 64l32 128h480l32 128h960l32 -192l-64 -32v-800z" />
+<glyph unicode="&#xf0fd;" d="M1280 192v896q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-320h-512v320q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-896q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v320h512v-320q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1536 1120v-960 q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf0fe;" d="M1280 576v128q0 26 -19 45t-45 19h-320v320q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-320h-320q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h320v-320q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v320h320q26 0 45 19t19 45zM1536 1120v-960 q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf100;" horiz-adv-x="1024" d="M627 160q0 -13 -10 -23l-50 -50q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23t10 23l466 466q10 10 23 10t23 -10l50 -50q10 -10 10 -23t-10 -23l-393 -393l393 -393q10 -10 10 -23zM1011 160q0 -13 -10 -23l-50 -50q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23 t10 23l466 466q10 10 23 10t23 -10l50 -50q10 -10 10 -23t-10 -23l-393 -393l393 -393q10 -10 10 -23z" />
+<glyph unicode="&#xf101;" horiz-adv-x="1024" d="M595 576q0 -13 -10 -23l-466 -466q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l393 393l-393 393q-10 10 -10 23t10 23l50 50q10 10 23 10t23 -10l466 -466q10 -10 10 -23zM979 576q0 -13 -10 -23l-466 -466q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23 l393 393l-393 393q-10 10 -10 23t10 23l50 50q10 10 23 10t23 -10l466 -466q10 -10 10 -23z" />
+<glyph unicode="&#xf102;" horiz-adv-x="1152" d="M1075 224q0 -13 -10 -23l-50 -50q-10 -10 -23 -10t-23 10l-393 393l-393 -393q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l466 466q10 10 23 10t23 -10l466 -466q10 -10 10 -23zM1075 608q0 -13 -10 -23l-50 -50q-10 -10 -23 -10t-23 10l-393 393l-393 -393 q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l466 466q10 10 23 10t23 -10l466 -466q10 -10 10 -23z" />
+<glyph unicode="&#xf103;" horiz-adv-x="1152" d="M1075 672q0 -13 -10 -23l-466 -466q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23t10 23l50 50q10 10 23 10t23 -10l393 -393l393 393q10 10 23 10t23 -10l50 -50q10 -10 10 -23zM1075 1056q0 -13 -10 -23l-466 -466q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23 t10 23l50 50q10 10 23 10t23 -10l393 -393l393 393q10 10 23 10t23 -10l50 -50q10 -10 10 -23z" />
+<glyph unicode="&#xf104;" horiz-adv-x="640" d="M627 992q0 -13 -10 -23l-393 -393l393 -393q10 -10 10 -23t-10 -23l-50 -50q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23t10 23l466 466q10 10 23 10t23 -10l50 -50q10 -10 10 -23z" />
+<glyph unicode="&#xf105;" horiz-adv-x="640" d="M595 576q0 -13 -10 -23l-466 -466q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l393 393l-393 393q-10 10 -10 23t10 23l50 50q10 10 23 10t23 -10l466 -466q10 -10 10 -23z" />
+<glyph unicode="&#xf106;" horiz-adv-x="1152" d="M1075 352q0 -13 -10 -23l-50 -50q-10 -10 -23 -10t-23 10l-393 393l-393 -393q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l466 466q10 10 23 10t23 -10l466 -466q10 -10 10 -23z" />
+<glyph unicode="&#xf107;" horiz-adv-x="1152" d="M1075 800q0 -13 -10 -23l-466 -466q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23t10 23l50 50q10 10 23 10t23 -10l393 -393l393 393q10 10 23 10t23 -10l50 -50q10 -10 10 -23z" />
+<glyph unicode="&#xf108;" horiz-adv-x="1920" d="M1792 544v832q0 13 -9.5 22.5t-22.5 9.5h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-832q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5zM1920 1376v-1088q0 -66 -47 -113t-113 -47h-544q0 -37 16 -77.5t32 -71t16 -43.5q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19 t-19 45q0 14 16 44t32 70t16 78h-544q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
+<glyph unicode="&#xf109;" horiz-adv-x="1920" d="M416 256q-66 0 -113 47t-47 113v704q0 66 47 113t113 47h1088q66 0 113 -47t47 -113v-704q0 -66 -47 -113t-113 -47h-1088zM384 1120v-704q0 -13 9.5 -22.5t22.5 -9.5h1088q13 0 22.5 9.5t9.5 22.5v704q0 13 -9.5 22.5t-22.5 9.5h-1088q-13 0 -22.5 -9.5t-9.5 -22.5z M1760 192h160v-96q0 -40 -47 -68t-113 -28h-1600q-66 0 -113 28t-47 68v96h160h1600zM1040 96q16 0 16 16t-16 16h-160q-16 0 -16 -16t16 -16h160z" />
+<glyph unicode="&#xf10a;" horiz-adv-x="1152" d="M640 128q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1024 288v960q0 13 -9.5 22.5t-22.5 9.5h-832q-13 0 -22.5 -9.5t-9.5 -22.5v-960q0 -13 9.5 -22.5t22.5 -9.5h832q13 0 22.5 9.5t9.5 22.5zM1152 1248v-1088q0 -66 -47 -113t-113 -47h-832 q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h832q66 0 113 -47t47 -113z" />
+<glyph unicode="&#xf10b;" horiz-adv-x="768" d="M464 128q0 33 -23.5 56.5t-56.5 23.5t-56.5 -23.5t-23.5 -56.5t23.5 -56.5t56.5 -23.5t56.5 23.5t23.5 56.5zM672 288v704q0 13 -9.5 22.5t-22.5 9.5h-512q-13 0 -22.5 -9.5t-9.5 -22.5v-704q0 -13 9.5 -22.5t22.5 -9.5h512q13 0 22.5 9.5t9.5 22.5zM480 1136 q0 16 -16 16h-160q-16 0 -16 -16t16 -16h160q16 0 16 16zM768 1152v-1024q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v1024q0 52 38 90t90 38h512q52 0 90 -38t38 -90z" />
+<glyph unicode="&#xf10c;" d="M768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103 t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf10d;" horiz-adv-x="1664" d="M768 576v-384q0 -80 -56 -136t-136 -56h-384q-80 0 -136 56t-56 136v704q0 104 40.5 198.5t109.5 163.5t163.5 109.5t198.5 40.5h64q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-64q-106 0 -181 -75t-75 -181v-32q0 -40 28 -68t68 -28h224q80 0 136 -56t56 -136z M1664 576v-384q0 -80 -56 -136t-136 -56h-384q-80 0 -136 56t-56 136v704q0 104 40.5 198.5t109.5 163.5t163.5 109.5t198.5 40.5h64q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-64q-106 0 -181 -75t-75  [...]
+<glyph unicode="&#xf10e;" horiz-adv-x="1664" d="M768 1216v-704q0 -104 -40.5 -198.5t-109.5 -163.5t-163.5 -109.5t-198.5 -40.5h-64q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h64q106 0 181 75t75 181v32q0 40 -28 68t-68 28h-224q-80 0 -136 56t-56 136v384q0 80 56 136t136 56h384q80 0 136 -56t56 -136zM1664 1216 v-704q0 -104 -40.5 -198.5t-109.5 -163.5t-163.5 -109.5t-198.5 -40.5h-64q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h64q106 0 181 75t75 181v32q0 40 -28 68t-68 28h-224q-80 0 -136 56t-56 136v384q0 80 [...]
+<glyph unicode="&#xf110;" horiz-adv-x="1568" d="M496 192q0 -60 -42.5 -102t-101.5 -42q-60 0 -102 42t-42 102t42 102t102 42q59 0 101.5 -42t42.5 -102zM928 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM320 640q0 -66 -47 -113t-113 -47t-113 47t-47 113 t47 113t113 47t113 -47t47 -113zM1360 192q0 -46 -33 -79t-79 -33t-79 33t-33 79t33 79t79 33t79 -33t33 -79zM528 1088q0 -73 -51.5 -124.5t-124.5 -51.5t-124.5 51.5t-51.5 124.5t51.5 124.5t124.5 51.5t124.5 [...]
+<glyph unicode="&#xf111;" d="M1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf112;" horiz-adv-x="1792" d="M1792 416q0 -166 -127 -451q-3 -7 -10.5 -24t-13.5 -30t-13 -22q-12 -17 -28 -17q-15 0 -23.5 10t-8.5 25q0 9 2.5 26.5t2.5 23.5q5 68 5 123q0 101 -17.5 181t-48.5 138.5t-80 101t-105.5 69.5t-133 42.5t-154 21.5t-175.5 6h-224v-256q0 -26 -19 -45t-45 -19t-45 19 l-512 512q-19 19 -19 45t19 45l512 512q19 19 45 19t45 -19t19 -45v-256h224q713 0 875 -403q53 -134 53 -333z" />
+<glyph unicode="&#xf113;" horiz-adv-x="1664" d="M640 320q0 -40 -12.5 -82t-43 -76t-72.5 -34t-72.5 34t-43 76t-12.5 82t12.5 82t43 76t72.5 34t72.5 -34t43 -76t12.5 -82zM1280 320q0 -40 -12.5 -82t-43 -76t-72.5 -34t-72.5 34t-43 76t-12.5 82t12.5 82t43 76t72.5 34t72.5 -34t43 -76t12.5 -82zM1440 320 q0 120 -69 204t-187 84q-41 0 -195 -21q-71 -11 -157 -11t-157 11q-152 21 -195 21q-118 0 -187 -84t-69 -204q0 -88 32 -153.5t81 -103t122 -60t140 -29.5t149 -7h168q82 0 149 7t140 29.5t122 60t81 103t32 153.5zM16 [...]
+<glyph unicode="&#xf114;" horiz-adv-x="1664" d="M1536 224v704q0 40 -28 68t-68 28h-704q-40 0 -68 28t-28 68v64q0 40 -28 68t-68 28h-320q-40 0 -68 -28t-28 -68v-960q0 -40 28 -68t68 -28h1216q40 0 68 28t28 68zM1664 928v-704q0 -92 -66 -158t-158 -66h-1216q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320 q92 0 158 -66t66 -158v-32h672q92 0 158 -66t66 -158z" />
+<glyph unicode="&#xf115;" horiz-adv-x="1920" d="M1781 605q0 35 -53 35h-1088q-40 0 -85.5 -21.5t-71.5 -52.5l-294 -363q-18 -24 -18 -40q0 -35 53 -35h1088q40 0 86 22t71 53l294 363q18 22 18 39zM640 768h768v160q0 40 -28 68t-68 28h-576q-40 0 -68 28t-28 68v64q0 40 -28 68t-68 28h-320q-40 0 -68 -28t-28 -68 v-853l256 315q44 53 116 87.5t140 34.5zM1909 605q0 -62 -46 -120l-295 -363q-43 -53 -116 -87.5t-140 -34.5h-1088q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h544q92 0 158 [...]
+<glyph unicode="&#xf116;" horiz-adv-x="1792" />
+<glyph unicode="&#xf117;" horiz-adv-x="1792" />
+<glyph unicode="&#xf118;" d="M1134 461q-37 -121 -138 -195t-228 -74t-228 74t-138 195q-8 25 4 48.5t38 31.5q25 8 48.5 -4t31.5 -38q25 -80 92.5 -129.5t151.5 -49.5t151.5 49.5t92.5 129.5q8 26 32 38t49 4t37 -31.5t4 -48.5zM640 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5 t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1152 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1408 640q0 130 -51 248.5t-136.5 204t-204 136.5t-248.5 51t-248.5 -51t-2 [...]
+<glyph unicode="&#xf119;" d="M1134 307q8 -25 -4 -48.5t-37 -31.5t-49 4t-32 38q-25 80 -92.5 129.5t-151.5 49.5t-151.5 -49.5t-92.5 -129.5q-8 -26 -31.5 -38t-48.5 -4q-26 8 -38 31.5t-4 48.5q37 121 138 195t228 74t228 -74t138 -195zM640 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5 t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1152 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1408 640q0 130 -51 248.5t-136.5 204t-204 136.5t-248.5 51t-248 [...]
+<glyph unicode="&#xf11a;" d="M1152 448q0 -26 -19 -45t-45 -19h-640q-26 0 -45 19t-19 45t19 45t45 19h640q26 0 45 -19t19 -45zM640 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1152 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5 t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1408 640q0 130 -51 248.5t-136.5 204t-204 136.5t-248.5 51t-248.5 -51t-204 -136.5t-136.5 -204t-51 -248.5t51 -248.5t136.5 -204t204 -136.5t248.5 -51t248.5 51t204 136. [...]
+<glyph unicode="&#xf11b;" horiz-adv-x="1920" d="M832 448v128q0 14 -9 23t-23 9h-192v192q0 14 -9 23t-23 9h-128q-14 0 -23 -9t-9 -23v-192h-192q-14 0 -23 -9t-9 -23v-128q0 -14 9 -23t23 -9h192v-192q0 -14 9 -23t23 -9h128q14 0 23 9t9 23v192h192q14 0 23 9t9 23zM1408 384q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5 t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1664 640q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1920 512q0 -212 -150 -362t-362 -1 [...]
+<glyph unicode="&#xf11c;" horiz-adv-x="1920" d="M384 368v-96q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16h96q16 0 16 -16zM512 624v-96q0 -16 -16 -16h-224q-16 0 -16 16v96q0 16 16 16h224q16 0 16 -16zM384 880v-96q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16h96q16 0 16 -16zM1408 368v-96q0 -16 -16 -16 h-864q-16 0 -16 16v96q0 16 16 16h864q16 0 16 -16zM768 624v-96q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16h96q16 0 16 -16zM640 880v-96q0 -16 -16 -16h-96q-16 0 -16 16v96q0 16 16 16h96q16 0 16 -16z [...]
+<glyph unicode="&#xf11d;" horiz-adv-x="1792" d="M1664 491v616q-169 -91 -306 -91q-82 0 -145 32q-100 49 -184 76.5t-178 27.5q-173 0 -403 -127v-599q245 113 433 113q55 0 103.5 -7.5t98 -26t77 -31t82.5 -39.5l28 -14q44 -22 101 -22q120 0 293 92zM320 1280q0 -35 -17.5 -64t-46.5 -46v-1266q0 -14 -9 -23t-23 -9 h-64q-14 0 -23 9t-9 23v1266q-29 17 -46.5 46t-17.5 64q0 53 37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1792 1216v-763q0 -39 -35 -57q-10 -5 -17 -9q-218 -116 -369 -116q-88 0 -158 35l-28 14q-64 33 -9 [...]
+<glyph unicode="&#xf11e;" horiz-adv-x="1792" d="M832 536v192q-181 -16 -384 -117v-185q205 96 384 110zM832 954v197q-172 -8 -384 -126v-189q215 111 384 118zM1664 491v184q-235 -116 -384 -71v224q-20 6 -39 15q-5 3 -33 17t-34.5 17t-31.5 15t-34.5 15.5t-32.5 13t-36 12.5t-35 8.5t-39.5 7.5t-39.5 4t-44 2 q-23 0 -49 -3v-222h19q102 0 192.5 -29t197.5 -82q19 -9 39 -15v-188q42 -17 91 -17q120 0 293 92zM1664 918v189q-169 -91 -306 -91q-45 0 -78 8v-196q148 -42 384 90zM320 1280q0 -35 -17.5 -64t-46.5 -46v-1266q [...]
+<glyph unicode="&#xf120;" horiz-adv-x="1664" d="M585 553l-466 -466q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l393 393l-393 393q-10 10 -10 23t10 23l50 50q10 10 23 10t23 -10l466 -466q10 -10 10 -23t-10 -23zM1664 96v-64q0 -14 -9 -23t-23 -9h-960q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h960q14 0 23 -9 t9 -23z" />
+<glyph unicode="&#xf121;" horiz-adv-x="1920" d="M617 137l-50 -50q-10 -10 -23 -10t-23 10l-466 466q-10 10 -10 23t10 23l466 466q10 10 23 10t23 -10l50 -50q10 -10 10 -23t-10 -23l-393 -393l393 -393q10 -10 10 -23t-10 -23zM1208 1204l-373 -1291q-4 -13 -15.5 -19.5t-23.5 -2.5l-62 17q-13 4 -19.5 15.5t-2.5 24.5 l373 1291q4 13 15.5 19.5t23.5 2.5l62 -17q13 -4 19.5 -15.5t2.5 -24.5zM1865 553l-466 -466q-10 -10 -23 -10t-23 10l-50 50q-10 10 -10 23t10 23l393 393l-393 393q-10 10 -10 23t10 23l50 50q10 10 23 10 [...]
+<glyph unicode="&#xf122;" horiz-adv-x="1792" d="M640 454v-70q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-512 512q-19 19 -19 45t19 45l512 512q29 31 70 14q39 -17 39 -59v-69l-397 -398q-19 -19 -19 -45t19 -45zM1792 416q0 -58 -17 -133.5t-38.5 -138t-48 -125t-40.5 -90.5l-20 -40q-8 -17 -28 -17q-6 0 -9 1 q-25 8 -23 34q43 400 -106 565q-64 71 -170.5 110.5t-267.5 52.5v-251q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-512 512q-19 19 -19 45t19 45l512 512q29 31 70 14q39 -17 39 -59v-262q411 -28 599 -221q169 - [...]
+<glyph unicode="&#xf123;" horiz-adv-x="1664" d="M1186 579l257 250l-356 52l-66 10l-30 60l-159 322v-963l59 -31l318 -168l-60 355l-12 66zM1638 841l-363 -354l86 -500q5 -33 -6 -51.5t-34 -18.5q-17 0 -40 12l-449 236l-449 -236q-23 -12 -40 -12q-23 0 -34 18.5t-6 51.5l86 500l-364 354q-32 32 -23 59.5t54 34.5 l502 73l225 455q20 41 49 41q28 0 49 -41l225 -455l502 -73q45 -7 54 -34.5t-24 -59.5z" />
+<glyph unicode="&#xf124;" horiz-adv-x="1408" d="M1401 1187l-640 -1280q-17 -35 -57 -35q-5 0 -15 2q-22 5 -35.5 22.5t-13.5 39.5v576h-576q-22 0 -39.5 13.5t-22.5 35.5t4 42t29 30l1280 640q13 7 29 7q27 0 45 -19q15 -14 18.5 -34.5t-6.5 -39.5z" />
+<glyph unicode="&#xf125;" horiz-adv-x="1664" d="M557 256h595v595zM512 301l595 595h-595v-595zM1664 224v-192q0 -14 -9 -23t-23 -9h-224v-224q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v224h-864q-14 0 -23 9t-9 23v864h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224v224q0 14 9 23t23 9h192q14 0 23 -9t9 -23 v-224h851l246 247q10 9 23 9t23 -9q9 -10 9 -23t-9 -23l-247 -246v-851h224q14 0 23 -9t9 -23z" />
+<glyph unicode="&#xf126;" horiz-adv-x="1024" d="M288 64q0 40 -28 68t-68 28t-68 -28t-28 -68t28 -68t68 -28t68 28t28 68zM288 1216q0 40 -28 68t-68 28t-68 -28t-28 -68t28 -68t68 -28t68 28t28 68zM928 1088q0 40 -28 68t-68 28t-68 -28t-28 -68t28 -68t68 -28t68 28t28 68zM1024 1088q0 -52 -26 -96.5t-70 -69.5 q-2 -287 -226 -414q-68 -38 -203 -81q-128 -40 -169.5 -71t-41.5 -100v-26q44 -25 70 -69.5t26 -96.5q0 -80 -56 -136t-136 -56t-136 56t-56 136q0 52 26 96.5t70 69.5v820q-44 25 -70 69.5t-26 96.5q0 80 56 13 [...]
+<glyph unicode="&#xf127;" horiz-adv-x="1664" d="M439 265l-256 -256q-10 -9 -23 -9q-12 0 -23 9q-9 10 -9 23t9 23l256 256q10 9 23 9t23 -9q9 -10 9 -23t-9 -23zM608 224v-320q0 -14 -9 -23t-23 -9t-23 9t-9 23v320q0 14 9 23t23 9t23 -9t9 -23zM384 448q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9t-9 23t9 23t23 9h320 q14 0 23 -9t9 -23zM1648 320q0 -120 -85 -203l-147 -146q-83 -83 -203 -83q-121 0 -204 85l-334 335q-21 21 -42 56l239 18l273 -274q27 -27 68 -27.5t68 26.5l147 146q28 28 28 67q0 40 -28 68l-274 275l18 239 [...]
+<glyph unicode="&#xf128;" horiz-adv-x="1024" d="M704 280v-240q0 -16 -12 -28t-28 -12h-240q-16 0 -28 12t-12 28v240q0 16 12 28t28 12h240q16 0 28 -12t12 -28zM1020 880q0 -54 -15.5 -101t-35 -76.5t-55 -59.5t-57.5 -43.5t-61 -35.5q-41 -23 -68.5 -65t-27.5 -67q0 -17 -12 -32.5t-28 -15.5h-240q-15 0 -25.5 18.5 t-10.5 37.5v45q0 83 65 156.5t143 108.5q59 27 84 56t25 76q0 42 -46.5 74t-107.5 32q-65 0 -108 -29q-35 -25 -107 -115q-13 -16 -31 -16q-12 0 -25 8l-164 125q-13 10 -15.5 25t5.5 28q160 266 464 266q80 0 [...]
+<glyph unicode="&#xf129;" horiz-adv-x="640" d="M640 192v-128q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h64v384h-64q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h384q26 0 45 -19t19 -45v-576h64q26 0 45 -19t19 -45zM512 1344v-192q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v192 q0 26 19 45t45 19h256q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf12a;" horiz-adv-x="640" d="M512 288v-224q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v224q0 26 19 45t45 19h256q26 0 45 -19t19 -45zM542 1344l-28 -768q-1 -26 -20.5 -45t-45.5 -19h-256q-26 0 -45.5 19t-20.5 45l-28 768q-1 26 17.5 45t44.5 19h320q26 0 44.5 -19t17.5 -45z" />
+<glyph unicode="&#xf12b;" d="M897 167v-167h-248l-159 252l-24 42q-8 9 -11 21h-3l-9 -21q-10 -20 -25 -44l-155 -250h-258v167h128l197 291l-185 272h-137v168h276l139 -228q2 -4 23 -42q8 -9 11 -21h3q3 9 11 21l25 42l140 228h257v-168h-125l-184 -267l204 -296h109zM1534 846v-206h-514l-3 27 q-4 28 -4 46q0 64 26 117t65 86.5t84 65t84 54.5t65 54t26 64q0 38 -29.5 62.5t-70.5 24.5q-51 0 -97 -39q-14 -11 -36 -38l-105 92q26 37 63 66q83 65 188 65q110 0 178 -59.5t68 -158.5q0 -56 -24.5 -103t-62 -76.5t-81.5 -58.5t- [...]
+<glyph unicode="&#xf12c;" d="M897 167v-167h-248l-159 252l-24 42q-8 9 -11 21h-3l-9 -21q-10 -20 -25 -44l-155 -250h-258v167h128l197 291l-185 272h-137v168h276l139 -228q2 -4 23 -42q8 -9 11 -21h3q3 9 11 21l25 42l140 228h257v-168h-125l-184 -267l204 -296h109zM1536 -50v-206h-514l-4 27 q-3 45 -3 46q0 64 26 117t65 86.5t84 65t84 54.5t65 54t26 64q0 38 -29.5 62.5t-70.5 24.5q-51 0 -97 -39q-14 -11 -36 -38l-105 92q26 37 63 66q80 65 188 65q110 0 178 -59.5t68 -158.5q0 -66 -34.5 -118.5t-84 -86t-99.5 -62.5t- [...]
+<glyph unicode="&#xf12d;" horiz-adv-x="1920" d="M896 128l336 384h-768l-336 -384h768zM1909 1205q15 -34 9.5 -71.5t-30.5 -65.5l-896 -1024q-38 -44 -96 -44h-768q-38 0 -69.5 20.5t-47.5 54.5q-15 34 -9.5 71.5t30.5 65.5l896 1024q38 44 96 44h768q38 0 69.5 -20.5t47.5 -54.5z" />
+<glyph unicode="&#xf12e;" horiz-adv-x="1664" d="M1664 438q0 -81 -44.5 -135t-123.5 -54q-41 0 -77.5 17.5t-59 38t-56.5 38t-71 17.5q-110 0 -110 -124q0 -39 16 -115t15 -115v-5q-22 0 -33 -1q-34 -3 -97.5 -11.5t-115.5 -13.5t-98 -5q-61 0 -103 26.5t-42 83.5q0 37 17.5 71t38 56.5t38 59t17.5 77.5q0 79 -54 123.5 t-135 44.5q-84 0 -143 -45.5t-59 -127.5q0 -43 15 -83t33.5 -64.5t33.5 -53t15 -50.5q0 -45 -46 -89q-37 -35 -117 -35q-95 0 -245 24q-9 2 -27.5 4t-27.5 4l-13 2q-1 0 -3 1q-2 0 -2 1v1024q2 -1 17.5 -3.5t [...]
+<glyph unicode="&#xf130;" horiz-adv-x="1152" d="M1152 832v-128q0 -221 -147.5 -384.5t-364.5 -187.5v-132h256q26 0 45 -19t19 -45t-19 -45t-45 -19h-640q-26 0 -45 19t-19 45t19 45t45 19h256v132q-217 24 -364.5 187.5t-147.5 384.5v128q0 26 19 45t45 19t45 -19t19 -45v-128q0 -185 131.5 -316.5t316.5 -131.5 t316.5 131.5t131.5 316.5v128q0 26 19 45t45 19t45 -19t19 -45zM896 1216v-512q0 -132 -94 -226t-226 -94t-226 94t-94 226v512q0 132 94 226t226 94t226 -94t94 -226z" />
+<glyph unicode="&#xf131;" horiz-adv-x="1408" d="M271 591l-101 -101q-42 103 -42 214v128q0 26 19 45t45 19t45 -19t19 -45v-128q0 -53 15 -113zM1385 1193l-361 -361v-128q0 -132 -94 -226t-226 -94q-55 0 -109 19l-96 -96q97 -51 205 -51q185 0 316.5 131.5t131.5 316.5v128q0 26 19 45t45 19t45 -19t19 -45v-128 q0 -221 -147.5 -384.5t-364.5 -187.5v-132h256q26 0 45 -19t19 -45t-19 -45t-45 -19h-640q-26 0 -45 19t-19 45t19 45t45 19h256v132q-125 13 -235 81l-254 -254q-10 -10 -23 -10t-23 10l-82 82q-10 10 -10 23t10 [...]
+<glyph unicode="&#xf132;" horiz-adv-x="1280" d="M1088 576v640h-448v-1137q119 63 213 137q235 184 235 360zM1280 1344v-768q0 -86 -33.5 -170.5t-83 -150t-118 -127.5t-126.5 -103t-121 -77.5t-89.5 -49.5t-42.5 -20q-12 -6 -26 -6t-26 6q-16 7 -42.5 20t-89.5 49.5t-121 77.5t-126.5 103t-118 127.5t-83 150 t-33.5 170.5v768q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf133;" horiz-adv-x="1664" d="M128 -128h1408v1024h-1408v-1024zM512 1088v288q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-288q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1280 1088v288q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-288q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1664 1152v-1280 q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h384v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h128q52 0 90 -38t38 -90z" />
+<glyph unicode="&#xf134;" horiz-adv-x="1408" d="M512 1344q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 1376v-320q0 -16 -12 -25q-8 -7 -20 -7q-4 0 -7 1l-448 96q-11 2 -18 11t-7 20h-256v-102q111 -23 183.5 -111t72.5 -203v-800q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v800 q0 106 62.5 190.5t161.5 114.5v111h-32q-59 0 -115 -23.5t-91.5 -53t-66 -66.5t-40.5 -53.5t-14 -24.5q-17 -35 -57 -35q-16 0 -29 7q-23 12 -31.5 37t3.5 49q5 10 14.5 26t37.5 53.5t60.5 70t85 67t108.5 52.5q-2 [...]
+<glyph unicode="&#xf135;" horiz-adv-x="1664" d="M1440 1088q0 40 -28 68t-68 28t-68 -28t-28 -68t28 -68t68 -28t68 28t28 68zM1664 1376q0 -249 -75.5 -430.5t-253.5 -360.5q-81 -80 -195 -176l-20 -379q-2 -16 -16 -26l-384 -224q-7 -4 -16 -4q-12 0 -23 9l-64 64q-13 14 -8 32l85 276l-281 281l-276 -85q-3 -1 -9 -1 q-14 0 -23 9l-64 64q-17 19 -5 39l224 384q10 14 26 16l379 20q96 114 176 195q188 187 358 258t431 71q14 0 24 -9.5t10 -22.5z" />
+<glyph unicode="&#xf136;" horiz-adv-x="1792" d="M1745 763l-164 -763h-334l178 832q13 56 -15 88q-27 33 -83 33h-169l-204 -953h-334l204 953h-286l-204 -953h-334l204 953l-153 327h1276q101 0 189.5 -40.5t147.5 -113.5q60 -73 81 -168.5t0 -194.5z" />
+<glyph unicode="&#xf137;" d="M909 141l102 102q19 19 19 45t-19 45l-307 307l307 307q19 19 19 45t-19 45l-102 102q-19 19 -45 19t-45 -19l-454 -454q-19 -19 -19 -45t19 -45l454 -454q19 -19 45 -19t45 19zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf138;" d="M717 141l454 454q19 19 19 45t-19 45l-454 454q-19 19 -45 19t-45 -19l-102 -102q-19 -19 -19 -45t19 -45l307 -307l-307 -307q-19 -19 -19 -45t19 -45l102 -102q19 -19 45 -19t45 19zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf139;" d="M1165 397l102 102q19 19 19 45t-19 45l-454 454q-19 19 -45 19t-45 -19l-454 -454q-19 -19 -19 -45t19 -45l102 -102q19 -19 45 -19t45 19l307 307l307 -307q19 -19 45 -19t45 19zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf13a;" d="M813 237l454 454q19 19 19 45t-19 45l-102 102q-19 19 -45 19t-45 -19l-307 -307l-307 307q-19 19 -45 19t-45 -19l-102 -102q-19 -19 -19 -45t19 -45l454 -454q19 -19 45 -19t45 19zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf13b;" horiz-adv-x="1408" d="M1130 939l16 175h-884l47 -534h612l-22 -228l-197 -53l-196 53l-13 140h-175l22 -278l362 -100h4v1l359 99l50 544h-644l-15 181h674zM0 1408h1408l-128 -1438l-578 -162l-574 162z" />
+<glyph unicode="&#xf13c;" horiz-adv-x="1792" d="M275 1408h1505l-266 -1333l-804 -267l-698 267l71 356h297l-29 -147l422 -161l486 161l68 339h-1208l58 297h1209l38 191h-1208z" />
+<glyph unicode="&#xf13d;" horiz-adv-x="1792" d="M960 1280q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1792 352v-352q0 -22 -20 -30q-8 -2 -12 -2q-13 0 -23 9l-93 93q-119 -143 -318.5 -226.5t-429.5 -83.5t-429.5 83.5t-318.5 226.5l-93 -93q-9 -9 -23 -9q-4 0 -12 2q-20 8 -20 30v352 q0 14 9 23t23 9h352q22 0 30 -20q8 -19 -7 -35l-100 -100q67 -91 189.5 -153.5t271.5 -82.5v647h-192q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h192v163q-58 34 -93 92.5t-35 128.5q0 106 75 181t181 75t181 -75t75 [...]
+<glyph unicode="&#xf13e;" horiz-adv-x="1152" d="M1056 768q40 0 68 -28t28 -68v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h32v320q0 185 131.5 316.5t316.5 131.5t316.5 -131.5t131.5 -316.5q0 -26 -19 -45t-45 -19h-64q-26 0 -45 19t-19 45q0 106 -75 181t-181 75t-181 -75t-75 -181 v-320h736z" />
+<glyph unicode="&#xf140;" d="M1024 640q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181zM1152 640q0 159 -112.5 271.5t-271.5 112.5t-271.5 -112.5t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5zM1280 640q0 -212 -150 -362t-362 -150t-362 150 t-150 362t150 362t362 150t362 -150t150 -362zM1408 640q0 130 -51 248.5t-136.5 204t-204 136.5t-248.5 51t-248.5 -51t-204 -136.5t-136.5 -204t-51 -248.5t51 -248.5t136.5 -204t204 -136.5t248.5 -51t248.5 51t204 136.5t136.5 2 [...]
+<glyph unicode="&#xf141;" horiz-adv-x="1408" d="M384 800v-192q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h192q40 0 68 -28t28 -68zM896 800v-192q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h192q40 0 68 -28t28 -68zM1408 800v-192q0 -40 -28 -68t-68 -28h-192 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h192q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf142;" horiz-adv-x="384" d="M384 288v-192q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h192q40 0 68 -28t28 -68zM384 800v-192q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h192q40 0 68 -28t28 -68zM384 1312v-192q0 -40 -28 -68t-68 -28h-192 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h192q40 0 68 -28t28 -68z" />
+<glyph unicode="&#xf143;" d="M512 256q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM863 162q-13 232 -177 396t-396 177q-14 1 -24 -9t-10 -23v-128q0 -13 8.5 -22t21.5 -10q154 -11 264 -121t121 -264q1 -13 10 -21.5t22 -8.5h128q13 0 23 10 t9 24zM1247 161q-5 154 -56 297.5t-139.5 260t-205 205t-260 139.5t-297.5 56q-14 1 -23 -9q-10 -10 -10 -23v-128q0 -13 9 -22t22 -10q204 -7 378 -111.5t278.5 -278.5t111.5 -378q1 -13 10 -22t22 -9h128q13 0 23 10q11 9 9 23z [...]
+<glyph unicode="&#xf144;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM1152 585q32 18 32 55t-32 55l-544 320q-31 19 -64 1q-32 -19 -32 -56v-640q0 -37 32 -56 q16 -8 32 -8q17 0 32 9z" />
+<glyph unicode="&#xf145;" horiz-adv-x="1792" d="M1024 1084l316 -316l-572 -572l-316 316zM813 105l618 618q19 19 19 45t-19 45l-362 362q-18 18 -45 18t-45 -18l-618 -618q-19 -19 -19 -45t19 -45l362 -362q18 -18 45 -18t45 18zM1702 742l-907 -908q-37 -37 -90.5 -37t-90.5 37l-126 126q56 56 56 136t-56 136 t-136 56t-136 -56l-125 126q-37 37 -37 90.5t37 90.5l907 906q37 37 90.5 37t90.5 -37l125 -125q-56 -56 -56 -136t56 -136t136 -56t136 56l126 -125q37 -37 37 -90.5t-37 -90.5z" />
+<glyph unicode="&#xf146;" d="M1280 576v128q0 26 -19 45t-45 19h-896q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h896q26 0 45 19t19 45zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5 t84.5 -203.5z" />
+<glyph unicode="&#xf147;" horiz-adv-x="1408" d="M1152 736v-64q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h832q14 0 23 -9t9 -23zM1280 288v832q0 66 -47 113t-113 47h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113zM1408 1120v-832q0 -119 -84.5 -203.5 t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf148;" horiz-adv-x="1024" d="M1018 933q-18 -37 -58 -37h-192v-864q0 -14 -9 -23t-23 -9h-704q-21 0 -29 18q-8 20 4 35l160 192q9 11 25 11h320v640h-192q-40 0 -58 37q-17 37 9 68l320 384q18 22 49 22t49 -22l320 -384q27 -32 9 -68z" />
+<glyph unicode="&#xf149;" horiz-adv-x="1024" d="M32 1280h704q13 0 22.5 -9.5t9.5 -23.5v-863h192q40 0 58 -37t-9 -69l-320 -384q-18 -22 -49 -22t-49 22l-320 384q-26 31 -9 69q18 37 58 37h192v640h-320q-14 0 -25 11l-160 192q-13 14 -4 34q9 19 29 19z" />
+<glyph unicode="&#xf14a;" d="M685 237l614 614q19 19 19 45t-19 45l-102 102q-19 19 -45 19t-45 -19l-467 -467l-211 211q-19 19 -45 19t-45 -19l-102 -102q-19 -19 -19 -45t19 -45l358 -358q19 -19 45 -19t45 19zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5 t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf14b;" d="M404 428l152 -152l-52 -52h-56v96h-96v56zM818 818q14 -13 -3 -30l-291 -291q-17 -17 -30 -3q-14 13 3 30l291 291q17 17 30 3zM544 128l544 544l-288 288l-544 -544v-288h288zM1152 736l92 92q28 28 28 68t-28 68l-152 152q-28 28 -68 28t-68 -28l-92 -92zM1536 1120 v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf14c;" d="M1280 608v480q0 26 -19 45t-45 19h-480q-42 0 -59 -39q-17 -41 14 -70l144 -144l-534 -534q-19 -19 -19 -45t19 -45l102 -102q19 -19 45 -19t45 19l534 534l144 -144q18 -19 45 -19q12 0 25 5q39 17 39 59zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960 q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf14d;" d="M1005 435l352 352q19 19 19 45t-19 45l-352 352q-30 31 -69 14q-40 -17 -40 -59v-160q-119 0 -216 -19.5t-162.5 -51t-114 -79t-76.5 -95.5t-44.5 -109t-21.5 -111.5t-5 -110.5q0 -181 167 -404q10 -12 25 -12q7 0 13 3q22 9 19 33q-44 354 62 473q46 52 130 75.5 t224 23.5v-160q0 -42 40 -59q12 -5 24 -5q26 0 45 19zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf14e;" d="M640 448l256 128l-256 128v-256zM1024 1039v-542l-512 -256v542zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103 t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf150;" d="M1145 861q18 -35 -5 -66l-320 -448q-19 -27 -52 -27t-52 27l-320 448q-23 31 -5 66q17 35 57 35h640q40 0 57 -35zM1280 160v960q0 13 -9.5 22.5t-22.5 9.5h-960q-13 0 -22.5 -9.5t-9.5 -22.5v-960q0 -13 9.5 -22.5t22.5 -9.5h960q13 0 22.5 9.5t9.5 22.5zM1536 1120 v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf151;" d="M1145 419q-17 -35 -57 -35h-640q-40 0 -57 35q-18 35 5 66l320 448q19 27 52 27t52 -27l320 -448q23 -31 5 -66zM1280 160v960q0 13 -9.5 22.5t-22.5 9.5h-960q-13 0 -22.5 -9.5t-9.5 -22.5v-960q0 -13 9.5 -22.5t22.5 -9.5h960q13 0 22.5 9.5t9.5 22.5zM1536 1120v-960 q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf152;" d="M1088 640q0 -33 -27 -52l-448 -320q-31 -23 -66 -5q-35 17 -35 57v640q0 40 35 57q35 18 66 -5l448 -320q27 -19 27 -52zM1280 160v960q0 14 -9 23t-23 9h-960q-14 0 -23 -9t-9 -23v-960q0 -14 9 -23t23 -9h960q14 0 23 9t9 23zM1536 1120v-960q0 -119 -84.5 -203.5 t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf153;" horiz-adv-x="1024" d="M976 229l35 -159q3 -12 -3 -22.5t-17 -14.5l-5 -1q-4 -2 -10.5 -3.5t-16 -4.5t-21.5 -5.5t-25.5 -5t-30 -5t-33.5 -4.5t-36.5 -3t-38.5 -1q-234 0 -409 130.5t-238 351.5h-95q-13 0 -22.5 9.5t-9.5 22.5v113q0 13 9.5 22.5t22.5 9.5h66q-2 57 1 105h-67q-14 0 -23 9 t-9 23v114q0 14 9 23t23 9h98q67 210 243.5 338t400.5 128q102 0 194 -23q11 -3 20 -15q6 -11 3 -24l-43 -159q-3 -13 -14 -19.5t-24 -2.5l-4 1q-4 1 -11.5 2.5l-17.5 3.5t-22.5 3.5t-26 3t-29 2.5t-29.5 1q-126  [...]
+<glyph unicode="&#xf154;" horiz-adv-x="1024" d="M1020 399v-367q0 -14 -9 -23t-23 -9h-956q-14 0 -23 9t-9 23v150q0 13 9.5 22.5t22.5 9.5h97v383h-95q-14 0 -23 9.5t-9 22.5v131q0 14 9 23t23 9h95v223q0 171 123.5 282t314.5 111q185 0 335 -125q9 -8 10 -20.5t-7 -22.5l-103 -127q-9 -11 -22 -12q-13 -2 -23 7 q-5 5 -26 19t-69 32t-93 18q-85 0 -137 -47t-52 -123v-215h305q13 0 22.5 -9t9.5 -23v-131q0 -13 -9.5 -22.5t-22.5 -9.5h-305v-379h414v181q0 13 9 22.5t23 9.5h162q14 0 23 -9.5t9 -22.5z" />
+<glyph unicode="&#xf155;" horiz-adv-x="1024" d="M978 351q0 -153 -99.5 -263.5t-258.5 -136.5v-175q0 -14 -9 -23t-23 -9h-135q-13 0 -22.5 9.5t-9.5 22.5v175q-66 9 -127.5 31t-101.5 44.5t-74 48t-46.5 37.5t-17.5 18q-17 21 -2 41l103 135q7 10 23 12q15 2 24 -9l2 -2q113 -99 243 -125q37 -8 74 -8q81 0 142.5 43 t61.5 122q0 28 -15 53t-33.5 42t-58.5 37.5t-66 32t-80 32.5q-39 16 -61.5 25t-61.5 26.5t-62.5 31t-56.5 35.5t-53.5 42.5t-43.5 49t-35.5 58t-21 66.5t-8.5 78q0 138 98 242t255 134v180q0 13 9.5 22.5t22.5  [...]
+<glyph unicode="&#xf156;" horiz-adv-x="898" d="M898 1066v-102q0 -14 -9 -23t-23 -9h-168q-23 -144 -129 -234t-276 -110q167 -178 459 -536q14 -16 4 -34q-8 -18 -29 -18h-195q-16 0 -25 12q-306 367 -498 571q-9 9 -9 22v127q0 13 9.5 22.5t22.5 9.5h112q132 0 212.5 43t102.5 125h-427q-14 0 -23 9t-9 23v102 q0 14 9 23t23 9h413q-57 113 -268 113h-145q-13 0 -22.5 9.5t-9.5 22.5v133q0 14 9 23t23 9h832q14 0 23 -9t9 -23v-102q0 -14 -9 -23t-23 -9h-233q47 -61 64 -144h171q14 0 23 -9t9 -23z" />
+<glyph unicode="&#xf157;" horiz-adv-x="1027" d="M603 0h-172q-13 0 -22.5 9t-9.5 23v330h-288q-13 0 -22.5 9t-9.5 23v103q0 13 9.5 22.5t22.5 9.5h288v85h-288q-13 0 -22.5 9t-9.5 23v104q0 13 9.5 22.5t22.5 9.5h214l-321 578q-8 16 0 32q10 16 28 16h194q19 0 29 -18l215 -425q19 -38 56 -125q10 24 30.5 68t27.5 61 l191 420q8 19 29 19h191q17 0 27 -16q9 -14 1 -31l-313 -579h215q13 0 22.5 -9.5t9.5 -22.5v-104q0 -14 -9.5 -23t-22.5 -9h-290v-85h290q13 0 22.5 -9.5t9.5 -22.5v-103q0 -14 -9.5 -23t-22.5 -9h-290v-330q [...]
+<glyph unicode="&#xf158;" horiz-adv-x="1280" d="M1043 971q0 100 -65 162t-171 62h-320v-448h320q106 0 171 62t65 162zM1280 971q0 -193 -126.5 -315t-326.5 -122h-340v-118h505q14 0 23 -9t9 -23v-128q0 -14 -9 -23t-23 -9h-505v-192q0 -14 -9.5 -23t-22.5 -9h-167q-14 0 -23 9t-9 23v192h-224q-14 0 -23 9t-9 23v128 q0 14 9 23t23 9h224v118h-224q-14 0 -23 9t-9 23v149q0 13 9 22.5t23 9.5h224v629q0 14 9 23t23 9h539q200 0 326.5 -122t126.5 -315z" />
+<glyph unicode="&#xf159;" horiz-adv-x="1792" d="M514 341l81 299h-159l75 -300q1 -1 1 -3t1 -3q0 1 0.5 3.5t0.5 3.5zM630 768l35 128h-292l32 -128h225zM822 768h139l-35 128h-70zM1271 340l78 300h-162l81 -299q0 -1 0.5 -3.5t1.5 -3.5q0 1 0.5 3t0.5 3zM1382 768l33 128h-297l34 -128h230zM1792 736v-64q0 -14 -9 -23 t-23 -9h-213l-164 -616q-7 -24 -31 -24h-159q-24 0 -31 24l-166 616h-209l-167 -616q-7 -24 -31 -24h-159q-11 0 -19.5 7t-10.5 17l-160 616h-208q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h175l-33 128h-142q-1 [...]
+<glyph unicode="&#xf15a;" horiz-adv-x="1280" d="M1167 896q18 -182 -131 -258q117 -28 175 -103t45 -214q-7 -71 -32.5 -125t-64.5 -89t-97 -58.5t-121.5 -34.5t-145.5 -15v-255h-154v251q-80 0 -122 1v-252h-154v255q-18 0 -54 0.5t-55 0.5h-200l31 183h111q50 0 58 51v402h16q-6 1 -16 1v287q-13 68 -89 68h-111v164 l212 -1q64 0 97 1v252h154v-247q82 2 122 2v245h154v-252q79 -7 140 -22.5t113 -45t82.5 -78t36.5 -114.5zM952 351q0 36 -15 64t-37 46t-57.5 30.5t-65.5 18.5t-74 9t-69 3t-64.5 -1t-47.5 -1v-338q8 0 37 -0 [...]
+<glyph unicode="&#xf15b;" horiz-adv-x="1280" d="M1280 768v-800q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v1344q0 40 28 68t68 28h544v-544q0 -40 28 -68t68 -28h544zM1277 896h-509v509q82 -15 132 -65l312 -312q50 -50 65 -132z" />
+<glyph unicode="&#xf15c;" horiz-adv-x="1280" d="M1024 160v64q0 14 -9 23t-23 9h-704q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h704q14 0 23 9t9 23zM1024 416v64q0 14 -9 23t-23 9h-704q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h704q14 0 23 9t9 23zM1280 768v-800q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28 t-28 68v1344q0 40 28 68t68 28h544v-544q0 -40 28 -68t68 -28h544zM1277 896h-509v509q82 -15 132 -65l312 -312q50 -50 65 -132z" />
+<glyph unicode="&#xf15d;" horiz-adv-x="1664" d="M1191 1128h177l-72 218l-12 47q-2 16 -2 20h-4l-3 -20q0 -1 -3.5 -18t-7.5 -29zM736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192q14 0 23 -9t9 -23zM1572 -23 v-233h-584v90l369 529q12 18 21 27l11 9v3q-2 0 -6.5 -0.5t-7.5 -0.5q-12 -3 -30 -3h-232v-115h-120v229h567v-89l-369 -530q-6 -8 -21 -26l-11 -11v-2l14 2q9 2 30 2h248v119h121zM1661 874v-106h-288v106h75l- [...]
+<glyph unicode="&#xf15e;" horiz-adv-x="1664" d="M1191 104h177l-72 218l-12 47q-2 16 -2 20h-4l-3 -20q0 -1 -3.5 -18t-7.5 -29zM736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192q14 0 23 -9t9 -23zM1661 -150 v-106h-288v106h75l-47 144h-243l-47 -144h75v-106h-287v106h70l230 662h162l230 -662h70zM1572 1001v-233h-584v90l369 529q12 18 21 27l11 9v3q-2 0 -6.5 -0.5t-7.5 -0.5q-12 -3 -30 -3h-232v-115h-120v229h567 [...]
+<glyph unicode="&#xf160;" horiz-adv-x="1792" d="M736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192q14 0 23 -9t9 -23zM1792 -32v-192q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h832 q14 0 23 -9t9 -23zM1600 480v-192q0 -14 -9 -23t-23 -9h-640q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h640q14 0 23 -9t9 -23zM1408 992v-192q0 -14 -9 -23t-23 -9h-448q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h448q14  [...]
+<glyph unicode="&#xf161;" horiz-adv-x="1792" d="M1216 -32v-192q0 -14 -9 -23t-23 -9h-256q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h256q14 0 23 -9t9 -23zM736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192 q14 0 23 -9t9 -23zM1408 480v-192q0 -14 -9 -23t-23 -9h-448q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h448q14 0 23 -9t9 -23zM1600 992v-192q0 -14 -9 -23t-23 -9h-640q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h640q14  [...]
+<glyph unicode="&#xf162;" d="M1346 223q0 63 -44 116t-103 53q-52 0 -83 -37t-31 -94t36.5 -95t104.5 -38q50 0 85 27t35 68zM736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192q14 0 23 -9t9 -23 zM1486 165q0 -62 -13 -121.5t-41 -114t-68 -95.5t-98.5 -65.5t-127.5 -24.5q-62 0 -108 16q-24 8 -42 15l39 113q15 -7 31 -11q37 -13 75 -13q84 0 134.5 58.5t66.5 145.5h-2q-21 -23 -61.5 -37t-84.5 -14q-106 0 -173 71.5t-67 [...]
+<glyph unicode="&#xf163;" d="M1346 1247q0 63 -44 116t-103 53q-52 0 -83 -37t-31 -94t36.5 -95t104.5 -38q50 0 85 27t35 68zM736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192q14 0 23 -9 t9 -23zM1456 -142v-114h-469v114h167v432q0 7 0.5 19t0.5 17v16h-2l-7 -12q-8 -13 -26 -31l-62 -58l-82 86l192 185h123v-654h165zM1486 1189q0 -62 -13 -121.5t-41 -114t-68 -95.5t-98.5 -65.5t-127.5 -24.5q-62 0 -108 16q-24 8 -4 [...]
+<glyph unicode="&#xf164;" horiz-adv-x="1664" d="M256 192q0 26 -19 45t-45 19q-27 0 -45.5 -19t-18.5 -45q0 -27 18.5 -45.5t45.5 -18.5q26 0 45 18.5t19 45.5zM416 704v-640q0 -26 -19 -45t-45 -19h-288q-26 0 -45 19t-19 45v640q0 26 19 45t45 19h288q26 0 45 -19t19 -45zM1600 704q0 -86 -55 -149q15 -44 15 -76 q3 -76 -43 -137q17 -56 0 -117q-15 -57 -54 -94q9 -112 -49 -181q-64 -76 -197 -78h-36h-76h-17q-66 0 -144 15.5t-121.5 29t-120.5 39.5q-123 43 -158 44q-26 1 -45 19.5t-19 44.5v641q0 25 18 43.5t43 20.5q24  [...]
+<glyph unicode="&#xf165;" horiz-adv-x="1664" d="M256 960q0 -26 -19 -45t-45 -19q-27 0 -45.5 19t-18.5 45q0 27 18.5 45.5t45.5 18.5q26 0 45 -18.5t19 -45.5zM416 448v640q0 26 -19 45t-45 19h-288q-26 0 -45 -19t-19 -45v-640q0 -26 19 -45t45 -19h288q26 0 45 19t19 45zM1545 597q55 -61 55 -149q-1 -78 -57.5 -135 t-134.5 -57h-277q4 -14 8 -24t11 -22t10 -18q18 -37 27 -57t19 -58.5t10 -76.5q0 -24 -0.5 -39t-5 -45t-12 -50t-24 -45t-40 -40.5t-60 -26t-82.5 -10.5q-26 0 -45 19q-20 20 -34 50t-19.5 52t-12.5 61q-9 42 [...]
+<glyph unicode="&#xf166;" d="M919 233v157q0 50 -29 50q-17 0 -33 -16v-224q16 -16 33 -16q29 0 29 49zM1103 355h66v34q0 51 -33 51t-33 -51v-34zM532 621v-70h-80v-423h-74v423h-78v70h232zM733 495v-367h-67v40q-39 -45 -76 -45q-33 0 -42 28q-6 16 -6 54v290h66v-270q0 -24 1 -26q1 -15 15 -15 q20 0 42 31v280h67zM985 384v-146q0 -52 -7 -73q-12 -42 -53 -42q-35 0 -68 41v-36h-67v493h67v-161q32 40 68 40q41 0 53 -42q7 -21 7 -74zM1236 255v-9q0 -29 -2 -43q-3 -22 -15 -40q-27 -40 -80 -40q-52 0 -81 38q-21 27 -21 86 [...]
+<glyph unicode="&#xf167;" d="M971 292v-211q0 -67 -39 -67q-23 0 -45 22v301q22 22 45 22q39 0 39 -67zM1309 291v-46h-90v46q0 68 45 68t45 -68zM343 509h107v94h-312v-94h105v-569h100v569zM631 -60h89v494h-89v-378q-30 -42 -57 -42q-18 0 -21 21q-1 3 -1 35v364h-89v-391q0 -49 8 -73 q12 -37 58 -37q48 0 102 61v-54zM1060 88v197q0 73 -9 99q-17 56 -71 56q-50 0 -93 -54v217h-89v-663h89v48q45 -55 93 -55q54 0 71 55q9 27 9 100zM1398 98v13h-91q0 -51 -2 -61q-7 -36 -40 -36q-46 0 -46 69v87h179v103q0 79 -27 116q-39  [...]
+<glyph unicode="&#xf168;" horiz-adv-x="1408" d="M597 869q-10 -18 -257 -456q-27 -46 -65 -46h-239q-21 0 -31 17t0 36l253 448q1 0 0 1l-161 279q-12 22 -1 37q9 15 32 15h239q40 0 66 -45zM1403 1511q11 -16 0 -37l-528 -934v-1l336 -615q11 -20 1 -37q-10 -15 -32 -15h-239q-42 0 -66 45l-339 622q18 32 531 942 q25 45 64 45h241q22 0 31 -15z" />
+<glyph unicode="&#xf169;" d="M685 771q0 1 -126 222q-21 34 -52 34h-184q-18 0 -26 -11q-7 -12 1 -29l125 -216v-1l-196 -346q-9 -14 0 -28q8 -13 24 -13h185q31 0 50 36zM1309 1268q-7 12 -24 12h-187q-30 0 -49 -35l-411 -729q1 -2 262 -481q20 -35 52 -35h184q18 0 25 12q8 13 -1 28l-260 476v1 l409 723q8 16 0 28zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf16a;" horiz-adv-x="1792" d="M1280 640q0 37 -30 54l-512 320q-31 20 -65 2q-33 -18 -33 -56v-640q0 -38 33 -56q16 -8 31 -8q20 0 34 10l512 320q30 17 30 54zM1792 640q0 -96 -1 -150t-8.5 -136.5t-22.5 -147.5q-16 -73 -69 -123t-124 -58q-222 -25 -671 -25t-671 25q-71 8 -124.5 58t-69.5 123 q-14 65 -21.5 147.5t-8.5 136.5t-1 150t1 150t8.5 136.5t22.5 147.5q16 73 69 123t124 58q222 25 671 25t671 -25q71 -8 124.5 -58t69.5 -123q14 -65 21.5 -147.5t8.5 -136.5t1 -150z" />
+<glyph unicode="&#xf16b;" horiz-adv-x="1792" d="M402 829l494 -305l-342 -285l-490 319zM1388 274v-108l-490 -293v-1l-1 1l-1 -1v1l-489 293v108l147 -96l342 284v2l1 -1l1 1v-2l343 -284zM554 1418l342 -285l-494 -304l-338 270zM1390 829l338 -271l-489 -319l-343 285zM1239 1418l489 -319l-338 -270l-494 304z" />
+<glyph unicode="&#xf16c;" horiz-adv-x="1408" d="M928 135v-151l-707 -1v151zM1169 481v-701l-1 -35v-1h-1132l-35 1h-1v736h121v-618h928v618h120zM241 393l704 -65l-13 -150l-705 65zM309 709l683 -183l-39 -146l-683 183zM472 1058l609 -360l-77 -130l-609 360zM832 1389l398 -585l-124 -85l-399 584zM1285 1536 l121 -697l-149 -26l-121 697z" />
+<glyph unicode="&#xf16d;" d="M1362 110v648h-135q20 -63 20 -131q0 -126 -64 -232.5t-174 -168.5t-240 -62q-197 0 -337 135.5t-140 327.5q0 68 20 131h-141v-648q0 -26 17.5 -43.5t43.5 -17.5h1069q25 0 43 17.5t18 43.5zM1078 643q0 124 -90.5 211.5t-218.5 87.5q-127 0 -217.5 -87.5t-90.5 -211.5 t90.5 -211.5t217.5 -87.5q128 0 218.5 87.5t90.5 211.5zM1362 1003v165q0 28 -20 48.5t-49 20.5h-174q-29 0 -49 -20.5t-20 -48.5v-165q0 -29 20 -49t49 -20h174q29 0 49 20t20 49zM1536 1211v-1142q0 -81 -58 -139t-139 -58h-11 [...]
+<glyph unicode="&#xf16e;" d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960zM698 640q0 88 -62 150t-150 62t-150 -62t-62 -150t62 -150t150 -62t150 62t62 150zM1262 640q0 88 -62 150 t-150 62t-150 -62t-62 -150t62 -150t150 -62t150 62t62 150z" />
+<glyph unicode="&#xf170;" d="M768 914l201 -306h-402zM1133 384h94l-459 691l-459 -691h94l104 160h522zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf171;" horiz-adv-x="1408" d="M815 677q8 -63 -50.5 -101t-111.5 -6q-39 17 -53.5 58t-0.5 82t52 58q36 18 72.5 12t64 -35.5t27.5 -67.5zM926 698q-14 107 -113 164t-197 13q-63 -28 -100.5 -88.5t-34.5 -129.5q4 -91 77.5 -155t165.5 -56q91 8 152 84t50 168zM1165 1240q-20 27 -56 44.5t-58 22 t-71 12.5q-291 47 -566 -2q-43 -7 -66 -12t-55 -22t-50 -43q30 -28 76 -45.5t73.5 -22t87.5 -11.5q228 -29 448 -1q63 8 89.5 12t72.5 21.5t75 46.5zM1222 205q-8 -26 -15.5 -76.5t-14 -84t-28.5 -70t-58 -56.5q- [...]
+<glyph unicode="&#xf172;" d="M848 666q0 43 -41 66t-77 1q-43 -20 -42.5 -72.5t43.5 -70.5q39 -23 81 4t36 72zM928 682q8 -66 -36 -121t-110 -61t-119 40t-56 113q-2 49 25.5 93t72.5 64q70 31 141.5 -10t81.5 -118zM1100 1073q-20 -21 -53.5 -34t-53 -16t-63.5 -8q-155 -20 -324 0q-44 6 -63 9.5 t-52.5 16t-54.5 32.5q13 19 36 31t40 15.5t47 8.5q198 35 408 1q33 -5 51 -8.5t43 -16t39 -31.5zM1142 327q0 7 5.5 26.5t3 32t-17.5 16.5q-161 -106 -365 -106t-366 106l-12 -6l-5 -12q26 -154 41 -210q47 -81 204 -108q249 -46 4 [...]
+<glyph unicode="&#xf173;" horiz-adv-x="1024" d="M390 1408h219v-388h364v-241h-364v-394q0 -136 14 -172q13 -37 52 -60q50 -31 117 -31q117 0 232 76v-242q-102 -48 -178 -65q-77 -19 -173 -19q-105 0 -186 27q-78 25 -138 75q-58 51 -79 105q-22 54 -22 161v539h-170v217q91 30 155 84q64 55 103 132q39 78 54 196z " />
+<glyph unicode="&#xf174;" d="M1123 127v181q-88 -56 -174 -56q-51 0 -88 23q-29 17 -39 45q-11 30 -11 129v295h274v181h-274v291h-164q-11 -90 -40 -147t-78 -99q-48 -40 -116 -63v-163h127v-404q0 -78 17 -121q17 -42 59 -78q43 -37 104 -57q62 -20 140 -20q67 0 129 14q57 13 134 49zM1536 1120 v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf175;" horiz-adv-x="768" d="M765 237q8 -19 -5 -35l-350 -384q-10 -10 -23 -10q-14 0 -24 10l-355 384q-13 16 -5 35q9 19 29 19h224v1248q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1248h224q21 0 29 -19z" />
+<glyph unicode="&#xf176;" horiz-adv-x="768" d="M765 1043q-9 -19 -29 -19h-224v-1248q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v1248h-224q-21 0 -29 19t5 35l350 384q10 10 23 10q14 0 24 -10l355 -384q13 -16 5 -35z" />
+<glyph unicode="&#xf177;" horiz-adv-x="1792" d="M1792 736v-192q0 -14 -9 -23t-23 -9h-1248v-224q0 -21 -19 -29t-35 5l-384 350q-10 10 -10 23q0 14 10 24l384 354q16 14 35 6q19 -9 19 -29v-224h1248q14 0 23 -9t9 -23z" />
+<glyph unicode="&#xf178;" horiz-adv-x="1792" d="M1728 643q0 -14 -10 -24l-384 -354q-16 -14 -35 -6q-19 9 -19 29v224h-1248q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h1248v224q0 21 19 29t35 -5l384 -350q10 -10 10 -23z" />
+<glyph unicode="&#xf179;" horiz-adv-x="1408" d="M1393 321q-39 -125 -123 -250q-129 -196 -257 -196q-49 0 -140 32q-86 32 -151 32q-61 0 -142 -33q-81 -34 -132 -34q-152 0 -301 259q-147 261 -147 503q0 228 113 374q112 144 284 144q72 0 177 -30q104 -30 138 -30q45 0 143 34q102 34 173 34q119 0 213 -65 q52 -36 104 -100q-79 -67 -114 -118q-65 -94 -65 -207q0 -124 69 -223t158 -126zM1017 1494q0 -61 -29 -136q-30 -75 -93 -138q-54 -54 -108 -72q-37 -11 -104 -17q3 149 78 257q74 107 250 148q1 -3 2.5 -11t2.5 -11 [...]
+<glyph unicode="&#xf17a;" horiz-adv-x="1664" d="M682 530v-651l-682 94v557h682zM682 1273v-659h-682v565zM1664 530v-786l-907 125v661h907zM1664 1408v-794h-907v669z" />
+<glyph unicode="&#xf17b;" horiz-adv-x="1408" d="M493 1053q16 0 27.5 11.5t11.5 27.5t-11.5 27.5t-27.5 11.5t-27 -11.5t-11 -27.5t11 -27.5t27 -11.5zM915 1053q16 0 27 11.5t11 27.5t-11 27.5t-27 11.5t-27.5 -11.5t-11.5 -27.5t11.5 -27.5t27.5 -11.5zM103 869q42 0 72 -30t30 -72v-430q0 -43 -29.5 -73t-72.5 -30 t-73 30t-30 73v430q0 42 30 72t73 30zM1163 850v-666q0 -46 -32 -78t-77 -32h-75v-227q0 -43 -30 -73t-73 -30t-73 30t-30 73v227h-138v-227q0 -43 -30 -73t-73 -30q-42 0 -72 30t-30 73l-1 227h-74q-46 0 -78  [...]
+<glyph unicode="&#xf17c;" d="M663 1125q-11 -1 -15.5 -10.5t-8.5 -9.5q-5 -1 -5 5q0 12 19 15h10zM750 1111q-4 -1 -11.5 6.5t-17.5 4.5q24 11 32 -2q3 -6 -3 -9zM399 684q-4 1 -6 -3t-4.5 -12.5t-5.5 -13.5t-10 -13q-7 -10 -1 -12q4 -1 12.5 7t12.5 18q1 3 2 7t2 6t1.5 4.5t0.5 4v3t-1 2.5t-3 2z M1254 325q0 18 -55 42q4 15 7.5 27.5t5 26t3 21.5t0.5 22.5t-1 19.5t-3.5 22t-4 20.5t-5 25t-5.5 26.5q-10 48 -47 103t-72 75q24 -20 57 -83q87 -162 54 -278q-11 -40 -50 -42q-31 -4 -38.5 18.5t-8 83.5t-11.5 107q-9 39 -19.5 69 [...]
+<glyph unicode="&#xf17d;" d="M1024 36q-42 241 -140 498h-2l-2 -1q-16 -6 -43 -16.5t-101 -49t-137 -82t-131 -114.5t-103 -148l-15 11q184 -150 418 -150q132 0 256 52zM839 643q-21 49 -53 111q-311 -93 -673 -93q-1 -7 -1 -21q0 -124 44 -236.5t124 -201.5q50 89 123.5 166.5t142.5 124.5t130.5 81 t99.5 48l37 13q4 1 13 3.5t13 4.5zM732 855q-120 213 -244 378q-138 -65 -234 -186t-128 -272q302 0 606 80zM1416 536q-210 60 -409 29q87 -239 128 -469q111 75 185 189.5t96 250.5zM611 1277q-1 0 -2 -1q1 1 2 1zM1201 1132q [...]
+<glyph unicode="&#xf17e;" d="M1173 473q0 50 -19.5 91.5t-48.5 68.5t-73 49t-82.5 34t-87.5 23l-104 24q-30 7 -44 10.5t-35 11.5t-30 16t-16.5 21t-7.5 30q0 77 144 77q43 0 77 -12t54 -28.5t38 -33.5t40 -29t48 -12q47 0 75.5 32t28.5 77q0 55 -56 99.5t-142 67.5t-182 23q-68 0 -132 -15.5 t-119.5 -47t-89 -87t-33.5 -128.5q0 -61 19 -106.5t56 -75.5t80 -48.5t103 -32.5l146 -36q90 -22 112 -36q32 -20 32 -60q0 -39 -40 -64.5t-105 -25.5q-51 0 -91.5 16t-65 38.5t-45.5 45t-46 38.5t-54 16q-50 0 -75.5 -30t-25.5 -75q0 - [...]
+<glyph unicode="&#xf180;" horiz-adv-x="1664" d="M1483 512l-587 -587q-52 -53 -127.5 -53t-128.5 53l-587 587q-53 53 -53 128t53 128l587 587q53 53 128 53t128 -53l265 -265l-398 -399l-188 188q-42 42 -99 42q-59 0 -100 -41l-120 -121q-42 -40 -42 -99q0 -58 42 -100l406 -408q30 -28 67 -37l6 -4h28q60 0 99 41 l619 619l2 -3q53 -53 53 -128t-53 -128zM1406 1138l120 -120q14 -15 14 -36t-14 -36l-730 -730q-17 -15 -37 -15v0q-4 0 -6 1q-18 2 -30 14l-407 408q-14 15 -14 36t14 35l121 120q13 15 35 15t36 -15l252 -252l [...]
+<glyph unicode="&#xf181;" d="M704 192v1024q0 14 -9 23t-23 9h-480q-14 0 -23 -9t-9 -23v-1024q0 -14 9 -23t23 -9h480q14 0 23 9t9 23zM1376 576v640q0 14 -9 23t-23 9h-480q-14 0 -23 -9t-9 -23v-640q0 -14 9 -23t23 -9h480q14 0 23 9t9 23zM1536 1344v-1408q0 -26 -19 -45t-45 -19h-1408 q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h1408q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf182;" horiz-adv-x="1280" d="M1280 480q0 -40 -28 -68t-68 -28q-51 0 -80 43l-227 341h-45v-132l247 -411q9 -15 9 -33q0 -26 -19 -45t-45 -19h-192v-272q0 -46 -33 -79t-79 -33h-160q-46 0 -79 33t-33 79v272h-192q-26 0 -45 19t-19 45q0 18 9 33l247 411v132h-45l-227 -341q-29 -43 -80 -43 q-40 0 -68 28t-28 68q0 29 16 53l256 384q73 107 176 107h384q103 0 176 -107l256 -384q16 -24 16 -53zM864 1280q0 -93 -65.5 -158.5t-158.5 -65.5t-158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5t158.5 -65.5t65. [...]
+<glyph unicode="&#xf183;" horiz-adv-x="1024" d="M1024 832v-416q0 -40 -28 -68t-68 -28t-68 28t-28 68v352h-64v-912q0 -46 -33 -79t-79 -33t-79 33t-33 79v464h-64v-464q0 -46 -33 -79t-79 -33t-79 33t-33 79v912h-64v-352q0 -40 -28 -68t-68 -28t-68 28t-28 68v416q0 80 56 136t136 56h640q80 0 136 -56t56 -136z M736 1280q0 -93 -65.5 -158.5t-158.5 -65.5t-158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5t158.5 -65.5t65.5 -158.5z" />
+<glyph unicode="&#xf184;" d="M773 234l350 473q16 22 24.5 59t-6 85t-61.5 79q-40 26 -83 25.5t-73.5 -17.5t-54.5 -45q-36 -40 -96 -40q-59 0 -95 40q-24 28 -54.5 45t-73.5 17.5t-84 -25.5q-46 -31 -60.5 -79t-6 -85t24.5 -59zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103 t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf185;" horiz-adv-x="1792" d="M1472 640q0 117 -45.5 223.5t-123 184t-184 123t-223.5 45.5t-223.5 -45.5t-184 -123t-123 -184t-45.5 -223.5t45.5 -223.5t123 -184t184 -123t223.5 -45.5t223.5 45.5t184 123t123 184t45.5 223.5zM1748 363q-4 -15 -20 -20l-292 -96v-306q0 -16 -13 -26q-15 -10 -29 -4 l-292 94l-180 -248q-10 -13 -26 -13t-26 13l-180 248l-292 -94q-14 -6 -29 4q-13 10 -13 26v306l-292 96q-16 5 -20 20q-5 17 4 29l180 248l-180 248q-9 13 -4 29q4 15 20 20l292 96v306q0 16 13 26q15 10 2 [...]
+<glyph unicode="&#xf186;" d="M1262 233q-54 -9 -110 -9q-182 0 -337 90t-245 245t-90 337q0 192 104 357q-201 -60 -328.5 -229t-127.5 -384q0 -130 51 -248.5t136.5 -204t204 -136.5t248.5 -51q144 0 273.5 61.5t220.5 171.5zM1465 318q-94 -203 -283.5 -324.5t-413.5 -121.5q-156 0 -298 61 t-245 164t-164 245t-61 298q0 153 57.5 292.5t156 241.5t235.5 164.5t290 68.5q44 2 61 -39q18 -41 -15 -72q-86 -78 -131.5 -181.5t-45.5 -218.5q0 -148 73 -273t198 -198t273 -73q118 0 228 51q41 18 72 -13q14 -14 17.5 -34t-4.5 -38z" />
+<glyph unicode="&#xf187;" horiz-adv-x="1792" d="M1088 704q0 26 -19 45t-45 19h-256q-26 0 -45 -19t-19 -45t19 -45t45 -19h256q26 0 45 19t19 45zM1664 896v-960q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v960q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1728 1344v-256q0 -26 -19 -45t-45 -19h-1536 q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1536q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf188;" horiz-adv-x="1664" d="M1632 576q0 -26 -19 -45t-45 -19h-224q0 -171 -67 -290l208 -209q19 -19 19 -45t-19 -45q-18 -19 -45 -19t-45 19l-198 197q-5 -5 -15 -13t-42 -28.5t-65 -36.5t-82 -29t-97 -13v896h-128v-896q-51 0 -101.5 13.5t-87 33t-66 39t-43.5 32.5l-15 14l-183 -207 q-20 -21 -48 -21q-24 0 -43 16q-19 18 -20.5 44.5t15.5 46.5l202 227q-58 114 -58 274h-224q-26 0 -45 19t-19 45t19 45t45 19h224v294l-173 173q-19 19 -19 45t19 45t45 19t45 -19l173 -173h844l173 173q19 19 45 19t45 [...]
+<glyph unicode="&#xf189;" horiz-adv-x="1920" d="M1917 1016q23 -64 -150 -294q-24 -32 -65 -85q-78 -100 -90 -131q-17 -41 14 -81q17 -21 81 -82h1l1 -1l1 -1l2 -2q141 -131 191 -221q3 -5 6.5 -12.5t7 -26.5t-0.5 -34t-25 -27.5t-59 -12.5l-256 -4q-24 -5 -56 5t-52 22l-20 12q-30 21 -70 64t-68.5 77.5t-61 58 t-56.5 15.5q-3 -1 -8 -3.5t-17 -14.5t-21.5 -29.5t-17 -52t-6.5 -77.5q0 -15 -3.5 -27.5t-7.5 -18.5l-4 -5q-18 -19 -53 -22h-115q-71 -4 -146 16.5t-131.5 53t-103 66t-70.5 57.5l-25 24q-10 10 -27.5 30t-71.5 91 [...]
+<glyph unicode="&#xf18a;" horiz-adv-x="1792" d="M675 252q21 34 11 69t-45 50q-34 14 -73 1t-60 -46q-22 -34 -13 -68.5t43 -50.5t74.5 -2.5t62.5 47.5zM769 373q8 13 3.5 26.5t-17.5 18.5q-14 5 -28.5 -0.5t-21.5 -18.5q-17 -31 13 -45q14 -5 29 0.5t22 18.5zM943 266q-45 -102 -158 -150t-224 -12 q-107 34 -147.5 126.5t6.5 187.5q47 93 151.5 139t210.5 19q111 -29 158.5 -119.5t2.5 -190.5zM1255 426q-9 96 -89 170t-208.5 109t-274.5 21q-223 -23 -369.5 -141.5t-132.5 -264.5q9 -96 89 -170t208.5 -109t274.5 -21q223 23 [...]
+<glyph unicode="&#xf18b;" d="M1133 -34q-171 -94 -368 -94q-196 0 -367 94q138 87 235.5 211t131.5 268q35 -144 132.5 -268t235.5 -211zM638 1394v-485q0 -252 -126.5 -459.5t-330.5 -306.5q-181 215 -181 495q0 187 83.5 349.5t229.5 269.5t325 137zM1536 638q0 -280 -181 -495 q-204 99 -330.5 306.5t-126.5 459.5v485q179 -30 325 -137t229.5 -269.5t83.5 -349.5z" />
+<glyph unicode="&#xf18c;" horiz-adv-x="1408" d="M1402 433q-32 -80 -76 -138t-91 -88.5t-99 -46.5t-101.5 -14.5t-96.5 8.5t-86.5 22t-69.5 27.5t-46 22.5l-17 10q-113 -228 -289.5 -359.5t-384.5 -132.5q-19 0 -32 13t-13 32t13 31.5t32 12.5q173 1 322.5 107.5t251.5 294.5q-36 -14 -72 -23t-83 -13t-91 2.5t-93 28.5 t-92 59t-84.5 100t-74.5 146q114 47 214 57t167.5 -7.5t124.5 -56.5t88.5 -77t56.5 -82q53 131 79 291q-7 -1 -18 -2.5t-46.5 -2.5t-69.5 0.5t-81.5 10t-88.5 23t-84 42.5t-75 65t-54.5 94.5t-28.5 127.5q70  [...]
+<glyph unicode="&#xf18d;" horiz-adv-x="1280" d="M1259 283v-66q0 -85 -57.5 -144.5t-138.5 -59.5h-57l-260 -269v269h-529q-81 0 -138.5 59.5t-57.5 144.5v66h1238zM1259 609v-255h-1238v255h1238zM1259 937v-255h-1238v255h1238zM1259 1077v-67h-1238v67q0 84 57.5 143.5t138.5 59.5h846q81 0 138.5 -59.5t57.5 -143.5z " />
+<glyph unicode="&#xf18e;" d="M1152 640q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v192h-352q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h352v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198 t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf190;" d="M1152 736v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-352v-192q0 -14 -9 -23t-23 -9q-12 0 -24 10l-319 319q-9 9 -9 23t9 23l320 320q9 9 23 9q13 0 22.5 -9.5t9.5 -22.5v-192h352q13 0 22.5 -9.5t9.5 -22.5zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198 t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf191;" d="M1024 960v-640q0 -26 -19 -45t-45 -19q-20 0 -37 12l-448 320q-27 19 -27 52t27 52l448 320q17 12 37 12q26 0 45 -19t19 -45zM1280 160v960q0 13 -9.5 22.5t-22.5 9.5h-960q-13 0 -22.5 -9.5t-9.5 -22.5v-960q0 -13 9.5 -22.5t22.5 -9.5h960q13 0 22.5 9.5t9.5 22.5z M1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf192;" d="M1024 640q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5 t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf193;" horiz-adv-x="1664" d="M1023 349l102 -204q-58 -179 -210 -290t-339 -111q-156 0 -288.5 77.5t-210 210t-77.5 288.5q0 181 104.5 330t274.5 211l17 -131q-122 -54 -195 -165.5t-73 -244.5q0 -185 131.5 -316.5t316.5 -131.5q126 0 232.5 65t165 175.5t49.5 236.5zM1571 249l58 -114l-256 -128 q-13 -7 -29 -7q-40 0 -57 35l-239 477h-472q-24 0 -42.5 16.5t-21.5 40.5l-96 779q-2 16 6 42q14 51 57 82.5t97 31.5q66 0 113 -47t47 -113q0 -69 -52 -117.5t-120 -41.5l37 -289h423v-128h-407l16 -128h455 [...]
+<glyph unicode="&#xf194;" d="M1254 899q16 85 -21 132q-52 65 -187 45q-17 -3 -41 -12.5t-57.5 -30.5t-64.5 -48.5t-59.5 -70t-44.5 -91.5q80 7 113.5 -16t26.5 -99q-5 -52 -52 -143q-43 -78 -71 -99q-44 -32 -87 14q-23 24 -37.5 64.5t-19 73t-10 84t-8.5 71.5q-23 129 -34 164q-12 37 -35.5 69 t-50.5 40q-57 16 -127 -25q-54 -32 -136.5 -106t-122.5 -102v-7q16 -8 25.5 -26t21.5 -20q21 -3 54.5 8.5t58 10.5t41.5 -30q11 -18 18.5 -38.5t15 -48t12.5 -40.5q17 -46 53 -187q36 -146 57 -197q42 -99 103 -125q43 -12 85 -1.5t7 [...]
+<glyph unicode="&#xf195;" horiz-adv-x="1152" d="M1152 704q0 -191 -94.5 -353t-256.5 -256.5t-353 -94.5h-160q-14 0 -23 9t-9 23v611l-215 -66q-3 -1 -9 -1q-10 0 -19 6q-13 10 -13 26v128q0 23 23 31l233 71v93l-215 -66q-3 -1 -9 -1q-10 0 -19 6q-13 10 -13 26v128q0 23 23 31l233 71v250q0 14 9 23t23 9h160 q14 0 23 -9t9 -23v-181l375 116q15 5 28 -5t13 -26v-128q0 -23 -23 -31l-393 -121v-93l375 116q15 5 28 -5t13 -26v-128q0 -23 -23 -31l-393 -121v-487q188 13 318 151t130 328q0 14 9 23t23 9h160q14 0 23 -9t9 -23z" />
+<glyph unicode="&#xf196;" horiz-adv-x="1408" d="M1152 736v-64q0 -14 -9 -23t-23 -9h-352v-352q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v352h-352q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h352v352q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-352h352q14 0 23 -9t9 -23zM1280 288v832q0 66 -47 113t-113 47h-832 q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113zM1408 1120v-832q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832q119 0 20 [...]
+<glyph unicode="&#xf197;" horiz-adv-x="1792" />
+<glyph unicode="&#xf198;" horiz-adv-x="1792" />
+<glyph unicode="&#xf199;" horiz-adv-x="1792" />
+<glyph unicode="&#xf19a;" horiz-adv-x="1792" />
+<glyph unicode="&#xf19b;" horiz-adv-x="1792" />
+<glyph unicode="&#xf19c;" horiz-adv-x="1792" />
+<glyph unicode="&#xf19d;" horiz-adv-x="1792" />
+<glyph unicode="&#xf19e;" horiz-adv-x="1792" />
+<glyph unicode="&#xf500;" horiz-adv-x="1792" />
+</font>
+</defs></svg> 
\ No newline at end of file
diff --git a/samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.ttf b/samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.ttf
new file mode 100755
index 000000000..e89738de5
Binary files /dev/null and b/samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.ttf differ
diff --git a/samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.woff b/samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.woff
new file mode 100755
index 000000000..8c1748aab
Binary files /dev/null and b/samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.woff differ
diff --git a/samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.eot b/samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.eot
new file mode 100644
index 000000000..4a4ca865d
Binary files /dev/null and b/samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.eot differ
diff --git a/samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.svg b/samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.svg
new file mode 100644
index 000000000..e3e2dc739
--- /dev/null
+++ b/samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.svg
@@ -0,0 +1,229 @@
+<?xml version="1.0" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
+<svg xmlns="http://www.w3.org/2000/svg">
+<metadata></metadata>
+<defs>
+<font id="glyphicons_halflingsregular" horiz-adv-x="1200" >
+<font-face units-per-em="1200" ascent="960" descent="-240" />
+<missing-glyph horiz-adv-x="500" />
+<glyph />
+<glyph />
+<glyph unicode="&#xd;" />
+<glyph unicode=" " />
+<glyph unicode="*" d="M100 500v200h259l-183 183l141 141l183 -183v259h200v-259l183 183l141 -141l-183 -183h259v-200h-259l183 -183l-141 -141l-183 183v-259h-200v259l-183 -183l-141 141l183 183h-259z" />
+<glyph unicode="+" d="M0 400v300h400v400h300v-400h400v-300h-400v-400h-300v400h-400z" />
+<glyph unicode="&#xa0;" />
+<glyph unicode="&#x2000;" horiz-adv-x="652" />
+<glyph unicode="&#x2001;" horiz-adv-x="1304" />
+<glyph unicode="&#x2002;" horiz-adv-x="652" />
+<glyph unicode="&#x2003;" horiz-adv-x="1304" />
+<glyph unicode="&#x2004;" horiz-adv-x="434" />
+<glyph unicode="&#x2005;" horiz-adv-x="326" />
+<glyph unicode="&#x2006;" horiz-adv-x="217" />
+<glyph unicode="&#x2007;" horiz-adv-x="217" />
+<glyph unicode="&#x2008;" horiz-adv-x="163" />
+<glyph unicode="&#x2009;" horiz-adv-x="260" />
+<glyph unicode="&#x200a;" horiz-adv-x="72" />
+<glyph unicode="&#x202f;" horiz-adv-x="260" />
+<glyph unicode="&#x205f;" horiz-adv-x="326" />
+<glyph unicode="&#x20ac;" d="M100 500l100 100h113q0 47 5 100h-218l100 100h135q37 167 112 257q117 141 297 141q242 0 354 -189q60 -103 66 -209h-181q0 55 -25.5 99t-63.5 68t-75 36.5t-67 12.5q-24 0 -52.5 -10t-62.5 -32t-65.5 -67t-50.5 -107h379l-100 -100h-300q-6 -46 -6 -100h406l-100 -100 h-300q9 -74 33 -132t52.5 -91t62 -54.5t59 -29t46.5 -7.5q29 0 66 13t75 37t63.5 67.5t25.5 96.5h174q-31 -172 -128 -278q-107 -117 -274 -117q-205 0 -324 158q-36 46 -69 131.5t-45 205.5h-217z" />
+<glyph unicode="&#x2212;" d="M200 400h900v300h-900v-300z" />
+<glyph unicode="&#x25fc;" horiz-adv-x="500" d="M0 0z" />
+<glyph unicode="&#x2601;" d="M-14 494q0 -80 56.5 -137t135.5 -57h750q120 0 205 86.5t85 207.5t-85 207t-205 86q-46 0 -90 -14q-44 97 -134.5 156.5t-200.5 59.5q-152 0 -260 -107.5t-108 -260.5q0 -25 2 -37q-66 -14 -108.5 -67.5t-42.5 -122.5z" />
+<glyph unicode="&#x2709;" d="M0 100l400 400l200 -200l200 200l400 -400h-1200zM0 300v600l300 -300zM0 1100l600 -603l600 603h-1200zM900 600l300 300v-600z" />
+<glyph unicode="&#x270f;" d="M-13 -13l333 112l-223 223zM187 403l214 -214l614 614l-214 214zM887 1103l214 -214l99 92q13 13 13 32.5t-13 33.5l-153 153q-15 13 -33 13t-33 -13z" />
+<glyph unicode="&#xe001;" d="M0 1200h1200l-500 -550v-550h300v-100h-800v100h300v550z" />
+<glyph unicode="&#xe002;" d="M14 84q18 -55 86 -75.5t147 5.5q65 21 109 69t44 90v606l600 155v-521q-64 16 -138 -7q-79 -26 -122.5 -83t-25.5 -111q18 -55 86 -75.5t147 4.5q70 23 111.5 63.5t41.5 95.5v881q0 10 -7 15.5t-17 2.5l-752 -193q-10 -3 -17 -12.5t-7 -19.5v-689q-64 17 -138 -7 q-79 -25 -122.5 -82t-25.5 -112z" />
+<glyph unicode="&#xe003;" d="M23 693q0 200 142 342t342 142t342 -142t142 -342q0 -142 -78 -261l300 -300q7 -8 7 -18t-7 -18l-109 -109q-8 -7 -18 -7t-18 7l-300 300q-119 -78 -261 -78q-200 0 -342 142t-142 342zM176 693q0 -136 97 -233t234 -97t233.5 96.5t96.5 233.5t-96.5 233.5t-233.5 96.5 t-234 -97t-97 -233z" />
+<glyph unicode="&#xe005;" d="M100 784q0 64 28 123t73 100.5t104.5 64t119 20.5t120 -38.5t104.5 -104.5q48 69 109.5 105t121.5 38t118.5 -20.5t102.5 -64t71 -100.5t27 -123q0 -57 -33.5 -117.5t-94 -124.5t-126.5 -127.5t-150 -152.5t-146 -174q-62 85 -145.5 174t-149.5 152.5t-126.5 127.5 t-94 124.5t-33.5 117.5z" />
+<glyph unicode="&#xe006;" d="M-72 800h479l146 400h2l146 -400h472l-382 -278l145 -449l-384 275l-382 -275l146 447zM168 71l2 1z" />
+<glyph unicode="&#xe007;" d="M-72 800h479l146 400h2l146 -400h472l-382 -278l145 -449l-384 275l-382 -275l146 447zM168 71l2 1zM237 700l196 -142l-73 -226l192 140l195 -141l-74 229l193 140h-235l-77 211l-78 -211h-239z" />
+<glyph unicode="&#xe008;" d="M0 0v143l400 257v100q-37 0 -68.5 74.5t-31.5 125.5v200q0 124 88 212t212 88t212 -88t88 -212v-200q0 -51 -31.5 -125.5t-68.5 -74.5v-100l400 -257v-143h-1200z" />
+<glyph unicode="&#xe009;" d="M0 0v1100h1200v-1100h-1200zM100 100h100v100h-100v-100zM100 300h100v100h-100v-100zM100 500h100v100h-100v-100zM100 700h100v100h-100v-100zM100 900h100v100h-100v-100zM300 100h600v400h-600v-400zM300 600h600v400h-600v-400zM1000 100h100v100h-100v-100z M1000 300h100v100h-100v-100zM1000 500h100v100h-100v-100zM1000 700h100v100h-100v-100zM1000 900h100v100h-100v-100z" />
+<glyph unicode="&#xe010;" d="M0 50v400q0 21 14.5 35.5t35.5 14.5h400q21 0 35.5 -14.5t14.5 -35.5v-400q0 -21 -14.5 -35.5t-35.5 -14.5h-400q-21 0 -35.5 14.5t-14.5 35.5zM0 650v400q0 21 14.5 35.5t35.5 14.5h400q21 0 35.5 -14.5t14.5 -35.5v-400q0 -21 -14.5 -35.5t-35.5 -14.5h-400 q-21 0 -35.5 14.5t-14.5 35.5zM600 50v400q0 21 14.5 35.5t35.5 14.5h400q21 0 35.5 -14.5t14.5 -35.5v-400q0 -21 -14.5 -35.5t-35.5 -14.5h-400q-21 0 -35.5 14.5t-14.5 35.5zM600 650v400q0 21 14.5 35.5t35.5 14.5h400q21 0 35.5 -14.5 [...]
+<glyph unicode="&#xe011;" d="M0 50v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5zM0 450v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200 q-21 0 -35.5 14.5t-14.5 35.5zM0 850v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5zM400 50v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t1 [...]
+<glyph unicode="&#xe012;" d="M0 50v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5zM0 450q0 -21 14.5 -35.5t35.5 -14.5h200q21 0 35.5 14.5t14.5 35.5v200q0 21 -14.5 35.5t-35.5 14.5h-200q-21 0 -35.5 -14.5 t-14.5 -35.5v-200zM0 850v200q0 21 14.5 35.5t35.5 14.5h200q21 0 35.5 -14.5t14.5 -35.5v-200q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5zM400 50v200q0 21 14.5 35.5t35.5 14.5h700q21 0 35.5 -14.5t1 [...]
+<glyph unicode="&#xe013;" d="M29 454l419 -420l818 820l-212 212l-607 -607l-206 207z" />
+<glyph unicode="&#xe014;" d="M106 318l282 282l-282 282l212 212l282 -282l282 282l212 -212l-282 -282l282 -282l-212 -212l-282 282l-282 -282z" />
+<glyph unicode="&#xe015;" d="M23 693q0 200 142 342t342 142t342 -142t142 -342q0 -142 -78 -261l300 -300q7 -8 7 -18t-7 -18l-109 -109q-8 -7 -18 -7t-18 7l-300 300q-119 -78 -261 -78q-200 0 -342 142t-142 342zM176 693q0 -136 97 -233t234 -97t233.5 96.5t96.5 233.5t-96.5 233.5t-233.5 96.5 t-234 -97t-97 -233zM300 600v200h100v100h200v-100h100v-200h-100v-100h-200v100h-100z" />
+<glyph unicode="&#xe016;" d="M23 694q0 200 142 342t342 142t342 -142t142 -342q0 -141 -78 -262l300 -299q7 -7 7 -18t-7 -18l-109 -109q-8 -8 -18 -8t-18 8l-300 300q-119 -78 -261 -78q-200 0 -342 142t-142 342zM176 694q0 -136 97 -233t234 -97t233.5 97t96.5 233t-96.5 233t-233.5 97t-234 -97 t-97 -233zM300 601h400v200h-400v-200z" />
+<glyph unicode="&#xe017;" d="M23 600q0 183 105 331t272 210v-166q-103 -55 -165 -155t-62 -220q0 -177 125 -302t302 -125t302 125t125 302q0 120 -62 220t-165 155v166q167 -62 272 -210t105 -331q0 -118 -45.5 -224.5t-123 -184t-184 -123t-224.5 -45.5t-224.5 45.5t-184 123t-123 184t-45.5 224.5 zM500 750q0 -21 14.5 -35.5t35.5 -14.5h100q21 0 35.5 14.5t14.5 35.5v400q0 21 -14.5 35.5t-35.5 14.5h-100q-21 0 -35.5 -14.5t-14.5 -35.5v-400z" />
+<glyph unicode="&#xe018;" d="M100 1h200v300h-200v-300zM400 1v500h200v-500h-200zM700 1v800h200v-800h-200zM1000 1v1200h200v-1200h-200z" />
+<glyph unicode="&#xe019;" d="M26 601q0 -33 6 -74l151 -38l2 -6q14 -49 38 -93l3 -5l-80 -134q45 -59 105 -105l133 81l5 -3q45 -26 94 -39l5 -2l38 -151q40 -5 74 -5q27 0 74 5l38 151l6 2q46 13 93 39l5 3l134 -81q56 44 104 105l-80 134l3 5q24 44 39 93l1 6l152 38q5 40 5 74q0 28 -5 73l-152 38 l-1 6q-16 51 -39 93l-3 5l80 134q-44 58 -104 105l-134 -81l-5 3q-45 25 -93 39l-6 1l-38 152q-40 5 -74 5q-27 0 -74 -5l-38 -152l-5 -1q-50 -14 -94 -39l-5 -3l-133 81q-59 -47 -105 -105l80 -134l-3 -5q-25 -47 -38 -93l-2 -6 [...]
+<glyph unicode="&#xe020;" d="M100 1025v50q0 10 7.5 17.5t17.5 7.5h275v100q0 41 29.5 70.5t70.5 29.5h300q41 0 70.5 -29.5t29.5 -70.5v-100h275q10 0 17.5 -7.5t7.5 -17.5v-50q0 -11 -7 -18t-18 -7h-1050q-11 0 -18 7t-7 18zM200 100v800h900v-800q0 -41 -29.5 -71t-70.5 -30h-700q-41 0 -70.5 30 t-29.5 71zM300 100h100v700h-100v-700zM500 100h100v700h-100v-700zM500 1100h300v100h-300v-100zM700 100h100v700h-100v-700zM900 100h100v700h-100v-700z" />
+<glyph unicode="&#xe021;" d="M1 601l656 644l644 -644h-200v-600h-300v400h-300v-400h-300v600h-200z" />
+<glyph unicode="&#xe022;" d="M100 25v1150q0 11 7 18t18 7h475v-500h400v-675q0 -11 -7 -18t-18 -7h-850q-11 0 -18 7t-7 18zM700 800v300l300 -300h-300z" />
+<glyph unicode="&#xe023;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM500 500v400h100 v-300h200v-100h-300z" />
+<glyph unicode="&#xe024;" d="M-100 0l431 1200h209l-21 -300h162l-20 300h208l431 -1200h-538l-41 400h-242l-40 -400h-539zM488 500h224l-27 300h-170z" />
+<glyph unicode="&#xe025;" d="M0 0v400h490l-290 300h200v500h300v-500h200l-290 -300h490v-400h-1100zM813 200h175v100h-175v-100z" />
+<glyph unicode="&#xe026;" d="M1 600q0 122 47.5 233t127.5 191t191 127.5t233 47.5t233 -47.5t191 -127.5t127.5 -191t47.5 -233t-47.5 -233t-127.5 -191t-191 -127.5t-233 -47.5t-233 47.5t-191 127.5t-127.5 191t-47.5 233zM188 600q0 -170 121 -291t291 -121t291 121t121 291t-121 291t-291 121 t-291 -121t-121 -291zM350 600h150v300h200v-300h150l-250 -300z" />
+<glyph unicode="&#xe027;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM350 600l250 300 l250 -300h-150v-300h-200v300h-150z" />
+<glyph unicode="&#xe028;" d="M0 25v475l200 700h800l199 -700l1 -475q0 -11 -7 -18t-18 -7h-1150q-11 0 -18 7t-7 18zM200 500h200l50 -200h300l50 200h200l-97 500h-606z" />
+<glyph unicode="&#xe029;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -172 121.5 -293t292.5 -121t292.5 121t121.5 293q0 171 -121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM500 397v401 l297 -200z" />
+<glyph unicode="&#xe030;" d="M23 600q0 -118 45.5 -224.5t123 -184t184 -123t224.5 -45.5t224.5 45.5t184 123t123 184t45.5 224.5h-150q0 -177 -125 -302t-302 -125t-302 125t-125 302t125 302t302 125q136 0 246 -81l-146 -146h400v400l-145 -145q-157 122 -355 122q-118 0 -224.5 -45.5t-184 -123 t-123 -184t-45.5 -224.5z" />
+<glyph unicode="&#xe031;" d="M23 600q0 118 45.5 224.5t123 184t184 123t224.5 45.5q198 0 355 -122l145 145v-400h-400l147 147q-112 80 -247 80q-177 0 -302 -125t-125 -302h-150zM100 0v400h400l-147 -147q112 -80 247 -80q177 0 302 125t125 302h150q0 -118 -45.5 -224.5t-123 -184t-184 -123 t-224.5 -45.5q-198 0 -355 122z" />
+<glyph unicode="&#xe032;" d="M100 0h1100v1200h-1100v-1200zM200 100v900h900v-900h-900zM300 200v100h100v-100h-100zM300 400v100h100v-100h-100zM300 600v100h100v-100h-100zM300 800v100h100v-100h-100zM500 200h500v100h-500v-100zM500 400v100h500v-100h-500zM500 600v100h500v-100h-500z M500 800v100h500v-100h-500z" />
+<glyph unicode="&#xe033;" d="M0 100v600q0 41 29.5 70.5t70.5 29.5h100v200q0 82 59 141t141 59h300q82 0 141 -59t59 -141v-200h100q41 0 70.5 -29.5t29.5 -70.5v-600q0 -41 -29.5 -70.5t-70.5 -29.5h-900q-41 0 -70.5 29.5t-29.5 70.5zM400 800h300v150q0 21 -14.5 35.5t-35.5 14.5h-200 q-21 0 -35.5 -14.5t-14.5 -35.5v-150z" />
+<glyph unicode="&#xe034;" d="M100 0v1100h100v-1100h-100zM300 400q60 60 127.5 84t127.5 17.5t122 -23t119 -30t110 -11t103 42t91 120.5v500q-40 -81 -101.5 -115.5t-127.5 -29.5t-138 25t-139.5 40t-125.5 25t-103 -29.5t-65 -115.5v-500z" />
+<glyph unicode="&#xe035;" d="M0 275q0 -11 7 -18t18 -7h50q11 0 18 7t7 18v300q0 127 70.5 231.5t184.5 161.5t245 57t245 -57t184.5 -161.5t70.5 -231.5v-300q0 -11 7 -18t18 -7h50q11 0 18 7t7 18v300q0 116 -49.5 227t-131 192.5t-192.5 131t-227 49.5t-227 -49.5t-192.5 -131t-131 -192.5 t-49.5 -227v-300zM200 20v460q0 8 6 14t14 6h160q8 0 14 -6t6 -14v-460q0 -8 -6 -14t-14 -6h-160q-8 0 -14 6t-6 14zM800 20v460q0 8 6 14t14 6h160q8 0 14 -6t6 -14v-460q0 -8 -6 -14t-14 -6h-160q-8 0 -14 6t-6 14z" />
+<glyph unicode="&#xe036;" d="M0 400h300l300 -200v800l-300 -200h-300v-400zM688 459l141 141l-141 141l71 71l141 -141l141 141l71 -71l-141 -141l141 -141l-71 -71l-141 141l-141 -141z" />
+<glyph unicode="&#xe037;" d="M0 400h300l300 -200v800l-300 -200h-300v-400zM700 857l69 53q111 -135 111 -310q0 -169 -106 -302l-67 54q86 110 86 248q0 146 -93 257z" />
+<glyph unicode="&#xe038;" d="M0 401v400h300l300 200v-800l-300 200h-300zM702 858l69 53q111 -135 111 -310q0 -170 -106 -303l-67 55q86 110 86 248q0 145 -93 257zM889 951l7 -8q123 -151 123 -344q0 -189 -119 -339l-7 -8l81 -66l6 8q142 178 142 405q0 230 -144 408l-6 8z" />
+<glyph unicode="&#xe039;" d="M0 0h500v500h-200v100h-100v-100h-200v-500zM0 600h100v100h400v100h100v100h-100v300h-500v-600zM100 100v300h300v-300h-300zM100 800v300h300v-300h-300zM200 200v100h100v-100h-100zM200 900h100v100h-100v-100zM500 500v100h300v-300h200v-100h-100v-100h-200v100 h-100v100h100v200h-200zM600 0v100h100v-100h-100zM600 1000h100v-300h200v-300h300v200h-200v100h200v500h-600v-200zM800 800v300h300v-300h-300zM900 0v100h300v-100h-300zM900 900v100h100v-100h-100zM1100 200v100h100v-100h [...]
+<glyph unicode="&#xe040;" d="M0 200h100v1000h-100v-1000zM100 0v100h300v-100h-300zM200 200v1000h100v-1000h-100zM500 0v91h100v-91h-100zM500 200v1000h200v-1000h-200zM700 0v91h100v-91h-100zM800 200v1000h100v-1000h-100zM900 0v91h200v-91h-200zM1000 200v1000h200v-1000h-200z" />
+<glyph unicode="&#xe041;" d="M0 700l1 475q0 10 7.5 17.5t17.5 7.5h474l700 -700l-500 -500zM148 953q0 -42 29 -71q30 -30 71.5 -30t71.5 30q29 29 29 71t-29 71q-30 30 -71.5 30t-71.5 -30q-29 -29 -29 -71z" />
+<glyph unicode="&#xe042;" d="M1 700l1 475q0 11 7 18t18 7h474l700 -700l-500 -500zM148 953q0 -42 30 -71q29 -30 71 -30t71 30q30 29 30 71t-30 71q-29 30 -71 30t-71 -30q-30 -29 -30 -71zM701 1200h100l700 -700l-500 -500l-50 50l450 450z" />
+<glyph unicode="&#xe043;" d="M100 0v1025l175 175h925v-1000l-100 -100v1000h-750l-100 -100h750v-1000h-900z" />
+<glyph unicode="&#xe044;" d="M200 0l450 444l450 -443v1150q0 20 -14.5 35t-35.5 15h-800q-21 0 -35.5 -15t-14.5 -35v-1151z" />
+<glyph unicode="&#xe045;" d="M0 100v700h200l100 -200h600l100 200h200v-700h-200v200h-800v-200h-200zM253 829l40 -124h592l62 124l-94 346q-2 11 -10 18t-18 7h-450q-10 0 -18 -7t-10 -18zM281 24l38 152q2 10 11.5 17t19.5 7h500q10 0 19.5 -7t11.5 -17l38 -152q2 -10 -3.5 -17t-15.5 -7h-600 q-10 0 -15.5 7t-3.5 17z" />
+<glyph unicode="&#xe046;" d="M0 200q0 -41 29.5 -70.5t70.5 -29.5h1000q41 0 70.5 29.5t29.5 70.5v600q0 41 -29.5 70.5t-70.5 29.5h-150q-4 8 -11.5 21.5t-33 48t-53 61t-69 48t-83.5 21.5h-200q-41 0 -82 -20.5t-70 -50t-52 -59t-34 -50.5l-12 -20h-150q-41 0 -70.5 -29.5t-29.5 -70.5v-600z M356 500q0 100 72 172t172 72t172 -72t72 -172t-72 -172t-172 -72t-172 72t-72 172zM494 500q0 -44 31 -75t75 -31t75 31t31 75t-31 75t-75 31t-75 -31t-31 -75zM900 700v100h100v-100h-100z" />
+<glyph unicode="&#xe047;" d="M53 0h365v66q-41 0 -72 11t-49 38t1 71l92 234h391l82 -222q16 -45 -5.5 -88.5t-74.5 -43.5v-66h417v66q-34 1 -74 43q-18 19 -33 42t-21 37l-6 13l-385 998h-93l-399 -1006q-24 -48 -52 -75q-12 -12 -33 -25t-36 -20l-15 -7v-66zM416 521l178 457l46 -140l116 -317h-340 z" />
+<glyph unicode="&#xe048;" d="M100 0v89q41 7 70.5 32.5t29.5 65.5v827q0 28 -1 39.5t-5.5 26t-15.5 21t-29 14t-49 14.5v71l471 -1q120 0 213 -88t93 -228q0 -55 -11.5 -101.5t-28 -74t-33.5 -47.5t-28 -28l-12 -7q8 -3 21.5 -9t48 -31.5t60.5 -58t47.5 -91.5t21.5 -129q0 -84 -59 -156.5t-142 -111 t-162 -38.5h-500zM400 200h161q89 0 153 48.5t64 132.5q0 90 -62.5 154.5t-156.5 64.5h-159v-400zM400 700h139q76 0 130 61.5t54 138.5q0 82 -84 130.5t-239 48.5v-379z" />
+<glyph unicode="&#xe049;" d="M200 0v57q77 7 134.5 40.5t65.5 80.5l173 849q10 56 -10 74t-91 37q-6 1 -10.5 2.5t-9.5 2.5v57h425l2 -57q-33 -8 -62 -25.5t-46 -37t-29.5 -38t-17.5 -30.5l-5 -12l-128 -825q-10 -52 14 -82t95 -36v-57h-500z" />
+<glyph unicode="&#xe050;" d="M-75 200h75v800h-75l125 167l125 -167h-75v-800h75l-125 -167zM300 900v300h150h700h150v-300h-50q0 29 -8 48.5t-18.5 30t-33.5 15t-39.5 5.5t-50.5 1h-200v-850l100 -50v-100h-400v100l100 50v850h-200q-34 0 -50.5 -1t-40 -5.5t-33.5 -15t-18.5 -30t-8.5 -48.5h-49z " />
+<glyph unicode="&#xe051;" d="M33 51l167 125v-75h800v75l167 -125l-167 -125v75h-800v-75zM100 901v300h150h700h150v-300h-50q0 29 -8 48.5t-18 30t-33.5 15t-40 5.5t-50.5 1h-200v-650l100 -50v-100h-400v100l100 50v650h-200q-34 0 -50.5 -1t-39.5 -5.5t-33.5 -15t-18.5 -30t-8 -48.5h-50z" />
+<glyph unicode="&#xe052;" d="M0 50q0 -20 14.5 -35t35.5 -15h1100q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-1100q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM0 350q0 -20 14.5 -35t35.5 -15h800q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-800q-21 0 -35.5 -14.5t-14.5 -35.5 v-100zM0 650q0 -20 14.5 -35t35.5 -15h1000q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-1000q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM0 950q0 -20 14.5 -35t35.5 -15h600q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-3 [...]
+<glyph unicode="&#xe053;" d="M0 50q0 -20 14.5 -35t35.5 -15h1100q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-1100q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM0 650q0 -20 14.5 -35t35.5 -15h1100q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-1100q-21 0 -35.5 -14.5t-14.5 -35.5 v-100zM200 350q0 -20 14.5 -35t35.5 -15h700q21 0 35.5 15t14.5 35v100q0 21 -14.5 35.5t-35.5 14.5h-700q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM200 950q0 -20 14.5 -35t35.5 -15h700q21 0 35.5 15t14.5 35v100q0 21 -14.5 35. [...]
+<glyph unicode="&#xe054;" d="M0 50v100q0 21 14.5 35.5t35.5 14.5h1100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-1100q-21 0 -35.5 15t-14.5 35zM100 650v100q0 21 14.5 35.5t35.5 14.5h1000q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-1000q-21 0 -35.5 15 t-14.5 35zM300 350v100q0 21 14.5 35.5t35.5 14.5h800q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-800q-21 0 -35.5 15t-14.5 35zM500 950v100q0 21 14.5 35.5t35.5 14.5h600q21 0 35.5 -14.5t14.5 -35.5v-100q [...]
+<glyph unicode="&#xe055;" d="M0 50v100q0 21 14.5 35.5t35.5 14.5h1100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-1100q-21 0 -35.5 15t-14.5 35zM0 350v100q0 21 14.5 35.5t35.5 14.5h1100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-1100q-21 0 -35.5 15 t-14.5 35zM0 650v100q0 21 14.5 35.5t35.5 14.5h1100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-1100q-21 0 -35.5 15t-14.5 35zM0 950v100q0 21 14.5 35.5t35.5 14.5h1100q21 0 35.5 -14.5t14.5 -35.5v-100q0 - [...]
+<glyph unicode="&#xe056;" d="M0 50v100q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-100q-21 0 -35.5 15t-14.5 35zM0 350v100q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-100q-21 0 -35.5 15 t-14.5 35zM0 650v100q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14.5 -35t-35.5 -15h-100q-21 0 -35.5 15t-14.5 35zM0 950v100q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-100q0 -20 -14. [...]
+<glyph unicode="&#xe057;" d="M-101 500v100h201v75l166 -125l-166 -125v75h-201zM300 0h100v1100h-100v-1100zM500 50q0 -20 14.5 -35t35.5 -15h600q20 0 35 15t15 35v100q0 21 -15 35.5t-35 14.5h-600q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM500 350q0 -20 14.5 -35t35.5 -15h300q20 0 35 15t15 35 v100q0 21 -15 35.5t-35 14.5h-300q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM500 650q0 -20 14.5 -35t35.5 -15h500q20 0 35 15t15 35v100q0 21 -15 35.5t-35 14.5h-500q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM500 950q0 -20 14.5 -35t3 [...]
+<glyph unicode="&#xe058;" d="M1 50q0 -20 14.5 -35t35.5 -15h600q20 0 35 15t15 35v100q0 21 -15 35.5t-35 14.5h-600q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM1 350q0 -20 14.5 -35t35.5 -15h300q20 0 35 15t15 35v100q0 21 -15 35.5t-35 14.5h-300q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM1 650 q0 -20 14.5 -35t35.5 -15h500q20 0 35 15t15 35v100q0 21 -15 35.5t-35 14.5h-500q-21 0 -35.5 -14.5t-14.5 -35.5v-100zM1 950q0 -20 14.5 -35t35.5 -15h100q20 0 35 15t15 35v100q0 21 -15 35.5t-35 14.5h-100q-21 0 -35.5 -14.5t-14. [...]
+<glyph unicode="&#xe059;" d="M0 275v650q0 31 22 53t53 22h750q31 0 53 -22t22 -53v-650q0 -31 -22 -53t-53 -22h-750q-31 0 -53 22t-22 53zM900 600l300 300v-600z" />
+<glyph unicode="&#xe060;" d="M0 44v1012q0 18 13 31t31 13h1112q19 0 31.5 -13t12.5 -31v-1012q0 -18 -12.5 -31t-31.5 -13h-1112q-18 0 -31 13t-13 31zM100 263l247 182l298 -131l-74 156l293 318l236 -288v500h-1000v-737zM208 750q0 56 39 95t95 39t95 -39t39 -95t-39 -95t-95 -39t-95 39t-39 95z " />
+<glyph unicode="&#xe062;" d="M148 745q0 124 60.5 231.5t165 172t226.5 64.5q123 0 227 -63t164.5 -169.5t60.5 -229.5t-73 -272q-73 -114 -166.5 -237t-150.5 -189l-57 -66q-10 9 -27 26t-66.5 70.5t-96 109t-104 135.5t-100.5 155q-63 139 -63 262zM342 772q0 -107 75.5 -182.5t181.5 -75.5 q107 0 182.5 75.5t75.5 182.5t-75.5 182t-182.5 75t-182 -75.5t-75 -181.5z" />
+<glyph unicode="&#xe063;" d="M1 600q0 122 47.5 233t127.5 191t191 127.5t233 47.5t233 -47.5t191 -127.5t127.5 -191t47.5 -233t-47.5 -233t-127.5 -191t-191 -127.5t-233 -47.5t-233 47.5t-191 127.5t-127.5 191t-47.5 233zM173 600q0 -177 125.5 -302t301.5 -125v854q-176 0 -301.5 -125 t-125.5 -302z" />
+<glyph unicode="&#xe064;" d="M117 406q0 94 34 186t88.5 172.5t112 159t115 177t87.5 194.5q21 -71 57.5 -142.5t76 -130.5t83 -118.5t82 -117t70 -116t50 -125.5t18.5 -136q0 -89 -39 -165.5t-102 -126.5t-140 -79.5t-156 -33.5q-114 6 -211.5 53t-161.5 139t-64 210zM243 414q14 -82 59.5 -136 t136.5 -80l16 98q-7 6 -18 17t-34 48t-33 77q-15 73 -14 143.5t10 122.5l9 51q-92 -110 -119.5 -185t-12.5 -156z" />
+<glyph unicode="&#xe065;" d="M0 400v300q0 165 117.5 282.5t282.5 117.5q366 -6 397 -14l-186 -186h-311q-41 0 -70.5 -29.5t-29.5 -70.5v-500q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5v125l200 200v-225q0 -165 -117.5 -282.5t-282.5 -117.5h-300q-165 0 -282.5 117.5 t-117.5 282.5zM436 341l161 50l412 412l-114 113l-405 -405zM995 1015l113 -113l113 113l-21 85l-92 28z" />
+<glyph unicode="&#xe066;" d="M0 400v300q0 165 117.5 282.5t282.5 117.5h261l2 -80q-133 -32 -218 -120h-145q-41 0 -70.5 -29.5t-29.5 -70.5v-500q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5l200 153v-53q0 -165 -117.5 -282.5t-282.5 -117.5h-300q-165 0 -282.5 117.5t-117.5 282.5 zM423 524q30 38 81.5 64t103 35.5t99 14t77.5 3.5l29 -1v-209l360 324l-359 318v-216q-7 0 -19 -1t-48 -8t-69.5 -18.5t-76.5 -37t-76.5 -59t-62 -88t-39.5 -121.5z" />
+<glyph unicode="&#xe067;" d="M0 400v300q0 165 117.5 282.5t282.5 117.5h300q61 0 127 -23l-178 -177h-349q-41 0 -70.5 -29.5t-29.5 -70.5v-500q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5v69l200 200v-169q0 -165 -117.5 -282.5t-282.5 -117.5h-300q-165 0 -282.5 117.5 t-117.5 282.5zM342 632l283 -284l567 567l-137 137l-430 -431l-146 147z" />
+<glyph unicode="&#xe068;" d="M0 603l300 296v-198h200v200h-200l300 300l295 -300h-195v-200h200v198l300 -296l-300 -300v198h-200v-200h195l-295 -300l-300 300h200v200h-200v-198z" />
+<glyph unicode="&#xe069;" d="M200 50v1000q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-437l500 487v-1100l-500 488v-438q0 -21 -14.5 -35.5t-35.5 -14.5h-100q-21 0 -35.5 14.5t-14.5 35.5z" />
+<glyph unicode="&#xe070;" d="M0 50v1000q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-437l500 487v-487l500 487v-1100l-500 488v-488l-500 488v-438q0 -21 -14.5 -35.5t-35.5 -14.5h-100q-21 0 -35.5 14.5t-14.5 35.5z" />
+<glyph unicode="&#xe071;" d="M136 550l564 550v-487l500 487v-1100l-500 488v-488z" />
+<glyph unicode="&#xe072;" d="M200 0l900 550l-900 550v-1100z" />
+<glyph unicode="&#xe073;" d="M200 150q0 -21 14.5 -35.5t35.5 -14.5h200q21 0 35.5 14.5t14.5 35.5v800q0 21 -14.5 35.5t-35.5 14.5h-200q-21 0 -35.5 -14.5t-14.5 -35.5v-800zM600 150q0 -21 14.5 -35.5t35.5 -14.5h200q21 0 35.5 14.5t14.5 35.5v800q0 21 -14.5 35.5t-35.5 14.5h-200 q-21 0 -35.5 -14.5t-14.5 -35.5v-800z" />
+<glyph unicode="&#xe074;" d="M200 150q0 -20 14.5 -35t35.5 -15h800q21 0 35.5 15t14.5 35v800q0 21 -14.5 35.5t-35.5 14.5h-800q-21 0 -35.5 -14.5t-14.5 -35.5v-800z" />
+<glyph unicode="&#xe075;" d="M0 0v1100l500 -487v487l564 -550l-564 -550v488z" />
+<glyph unicode="&#xe076;" d="M0 0v1100l500 -487v487l500 -487v437q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-1000q0 -21 -14.5 -35.5t-35.5 -14.5h-100q-21 0 -35.5 14.5t-14.5 35.5v438l-500 -488v488z" />
+<glyph unicode="&#xe077;" d="M300 0v1100l500 -487v437q0 21 14.5 35.5t35.5 14.5h100q21 0 35.5 -14.5t14.5 -35.5v-1000q0 -21 -14.5 -35.5t-35.5 -14.5h-100q-21 0 -35.5 14.5t-14.5 35.5v438z" />
+<glyph unicode="&#xe078;" d="M100 250v100q0 21 14.5 35.5t35.5 14.5h1000q21 0 35.5 -14.5t14.5 -35.5v-100q0 -21 -14.5 -35.5t-35.5 -14.5h-1000q-21 0 -35.5 14.5t-14.5 35.5zM100 500h1100l-550 564z" />
+<glyph unicode="&#xe079;" d="M185 599l592 -592l240 240l-353 353l353 353l-240 240z" />
+<glyph unicode="&#xe080;" d="M272 194l353 353l-353 353l241 240l572 -571l21 -22l-1 -1v-1l-592 -591z" />
+<glyph unicode="&#xe081;" d="M3 600q0 162 80 299.5t217.5 217.5t299.5 80t299.5 -80t217.5 -217.5t80 -299.5t-80 -299.5t-217.5 -217.5t-299.5 -80t-299.5 80t-217.5 217.5t-80 299.5zM300 500h200v-200h200v200h200v200h-200v200h-200v-200h-200v-200z" />
+<glyph unicode="&#xe082;" d="M3 600q0 162 80 299.5t217.5 217.5t299.5 80t299.5 -80t217.5 -217.5t80 -299.5t-80 -299.5t-217.5 -217.5t-299.5 -80t-299.5 80t-217.5 217.5t-80 299.5zM300 500h600v200h-600v-200z" />
+<glyph unicode="&#xe083;" d="M3 600q0 162 80 299.5t217.5 217.5t299.5 80t299.5 -80t217.5 -217.5t80 -299.5t-80 -299.5t-217.5 -217.5t-299.5 -80t-299.5 80t-217.5 217.5t-80 299.5zM246 459l213 -213l141 142l141 -142l213 213l-142 141l142 141l-213 212l-141 -141l-141 142l-212 -213l141 -141 z" />
+<glyph unicode="&#xe084;" d="M3 600q0 162 80 299.5t217.5 217.5t299.5 80t299.5 -80t217.5 -217.5t80 -299.5t-80 -299.5t-217.5 -217.5t-299.5 -80t-299.5 80t-217.5 217.5t-80 299.5zM270 551l276 -277l411 411l-175 174l-236 -236l-102 102z" />
+<glyph unicode="&#xe085;" d="M3 600q0 162 80 299.5t217.5 217.5t299.5 80t299.5 -80t217.5 -217.5t80 -299.5t-80 -299.5t-217.5 -217.5t-299.5 -80t-299.5 80t-217.5 217.5t-80 299.5zM364 700h143q4 0 11.5 -1t11 -1t6.5 3t3 9t1 11t3.5 8.5t3.5 6t5.5 4t6.5 2.5t9 1.5t9 0.5h11.5h12.5 q19 0 30 -10t11 -26q0 -22 -4 -28t-27 -22q-5 -1 -12.5 -3t-27 -13.5t-34 -27t-26.5 -46t-11 -68.5h200q5 3 14 8t31.5 25.5t39.5 45.5t31 69t14 94q0 51 -17.5 89t-42 58t-58.5 32t-58.5 15t-51.5 3q-50 0 -90.5 -12t-75 -38.5t-53.5 -74. [...]
+<glyph unicode="&#xe086;" d="M3 600q0 162 80 299.5t217.5 217.5t299.5 80t299.5 -80t217.5 -217.5t80 -299.5t-80 -299.5t-217.5 -217.5t-299.5 -80t-299.5 80t-217.5 217.5t-80 299.5zM400 300h400v100h-100v300h-300v-100h100v-200h-100v-100zM500 800h200v100h-200v-100z" />
+<glyph unicode="&#xe087;" d="M0 500v200h195q31 125 98.5 199.5t206.5 100.5v200h200v-200q54 -20 113 -60t112.5 -105.5t71.5 -134.5h203v-200h-203q-25 -102 -116.5 -186t-180.5 -117v-197h-200v197q-140 27 -208 102.5t-98 200.5h-194zM290 500q24 -73 79.5 -127.5t130.5 -78.5v206h200v-206 q149 48 201 206h-201v200h200q-25 74 -75.5 127t-124.5 77v-204h-200v203q-75 -23 -130 -77t-79 -126h209v-200h-210z" />
+<glyph unicode="&#xe088;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM356 465l135 135 l-135 135l109 109l135 -135l135 135l109 -109l-135 -135l135 -135l-109 -109l-135 135l-135 -135z" />
+<glyph unicode="&#xe089;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM322 537l141 141 l87 -87l204 205l142 -142l-346 -345z" />
+<glyph unicode="&#xe090;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -115 62 -215l568 567q-100 62 -216 62q-171 0 -292.5 -121.5t-121.5 -292.5zM391 245q97 -59 209 -59q171 0 292.5 121.5t121.5 292.5 q0 112 -59 209z" />
+<glyph unicode="&#xe091;" d="M0 547l600 453v-300h600v-300h-600v-301z" />
+<glyph unicode="&#xe092;" d="M0 400v300h600v300l600 -453l-600 -448v301h-600z" />
+<glyph unicode="&#xe093;" d="M204 600l450 600l444 -600h-298v-600h-300v600h-296z" />
+<glyph unicode="&#xe094;" d="M104 600h296v600h300v-600h298l-449 -600z" />
+<glyph unicode="&#xe095;" d="M0 200q6 132 41 238.5t103.5 193t184 138t271.5 59.5v271l600 -453l-600 -448v301q-95 -2 -183 -20t-170 -52t-147 -92.5t-100 -135.5z" />
+<glyph unicode="&#xe096;" d="M0 0v400l129 -129l294 294l142 -142l-294 -294l129 -129h-400zM635 777l142 -142l294 294l129 -129v400h-400l129 -129z" />
+<glyph unicode="&#xe097;" d="M34 176l295 295l-129 129h400v-400l-129 130l-295 -295zM600 600v400l129 -129l295 295l142 -141l-295 -295l129 -130h-400z" />
+<glyph unicode="&#xe101;" d="M23 600q0 118 45.5 224.5t123 184t184 123t224.5 45.5t224.5 -45.5t184 -123t123 -184t45.5 -224.5t-45.5 -224.5t-123 -184t-184 -123t-224.5 -45.5t-224.5 45.5t-184 123t-123 184t-45.5 224.5zM456 851l58 -302q4 -20 21.5 -34.5t37.5 -14.5h54q20 0 37.5 14.5 t21.5 34.5l58 302q4 20 -8 34.5t-32 14.5h-207q-21 0 -33 -14.5t-8 -34.5zM500 300h200v100h-200v-100z" />
+<glyph unicode="&#xe102;" d="M0 800h100v-200h400v300h200v-300h400v200h100v100h-111q1 1 1 6.5t-1.5 15t-3.5 17.5l-34 172q-11 39 -41.5 63t-69.5 24q-32 0 -61 -17l-239 -144q-22 -13 -40 -35q-19 24 -40 36l-238 144q-33 18 -62 18q-39 0 -69.5 -23t-40.5 -61l-35 -177q-2 -8 -3 -18t-1 -15v-6 h-111v-100zM100 0h400v400h-400v-400zM200 900q-3 0 14 48t36 96l18 47l213 -191h-281zM700 0v400h400v-400h-400zM731 900l202 197q5 -12 12 -32.5t23 -64t25 -72t7 -28.5h-269z" />
+<glyph unicode="&#xe103;" d="M0 -22v143l216 193q-9 53 -13 83t-5.5 94t9 113t38.5 114t74 124q47 60 99.5 102.5t103 68t127.5 48t145.5 37.5t184.5 43.5t220 58.5q0 -189 -22 -343t-59 -258t-89 -181.5t-108.5 -120t-122 -68t-125.5 -30t-121.5 -1.5t-107.5 12.5t-87.5 17t-56.5 7.5l-99 -55z M238.5 300.5q19.5 -6.5 86.5 76.5q55 66 367 234q70 38 118.5 69.5t102 79t99 111.5t86.5 148q22 50 24 60t-6 19q-7 5 -17 5t-26.5 -14.5t-33.5 -39.5q-35 -51 -113.5 -108.5t-139.5 -89.5l-61 -32q-369 -197 -458 -401q-48 -111 -28 [...]
+<glyph unicode="&#xe104;" d="M111 408q0 -33 5 -63q9 -56 44 -119.5t105 -108.5q31 -21 64 -16t62 23.5t57 49.5t48 61.5t35 60.5q32 66 39 184.5t-13 157.5q79 -80 122 -164t26 -184q-5 -33 -20.5 -69.5t-37.5 -80.5q-10 -19 -14.5 -29t-12 -26t-9 -23.5t-3 -19t2.5 -15.5t11 -9.5t19.5 -5t30.5 2.5 t42 8q57 20 91 34t87.5 44.5t87 64t65.5 88.5t47 122q38 172 -44.5 341.5t-246.5 278.5q22 -44 43 -129q39 -159 -32 -154q-15 2 -33 9q-79 33 -120.5 100t-44 175.5t48.5 257.5q-13 -8 -34 -23.5t-72.5 -66.5t-88.5 -105.5t-60  [...]
+<glyph unicode="&#xe105;" d="M-61 600l26 40q6 10 20 30t49 63.5t74.5 85.5t97 90t116.5 83.5t132.5 59t145.5 23.5t145.5 -23.5t132.5 -59t116.5 -83.5t97 -90t74.5 -85.5t49 -63.5t20 -30l26 -40l-26 -40q-6 -10 -20 -30t-49 -63.5t-74.5 -85.5t-97 -90t-116.5 -83.5t-132.5 -59t-145.5 -23.5 t-145.5 23.5t-132.5 59t-116.5 83.5t-97 90t-74.5 85.5t-49 63.5t-20 30zM120 600q7 -10 40.5 -58t56 -78.5t68 -77.5t87.5 -75t103 -49.5t125 -21.5t123.5 20t100.5 45.5t85.5 71.5t66.5 75.5t58 81.5t47 66q-1 1 -28.5 37.5t-42 55t [...]
+<glyph unicode="&#xe106;" d="M-61 600l26 40q6 10 20 30t49 63.5t74.5 85.5t97 90t116.5 83.5t132.5 59t145.5 23.5q61 0 121 -17l37 142h148l-314 -1200h-148l37 143q-82 21 -165 71.5t-140 102t-109.5 112t-72 88.5t-29.5 43zM120 600q210 -282 393 -336l37 141q-107 18 -178.5 101.5t-71.5 193.5 q0 85 46 158q-102 -87 -226 -258zM377 656q49 -124 154 -191l47 47l23 87q-30 28 -59 69t-44 68l-14 26zM780 161l38 145q22 15 44.5 34t46 44t40.5 44t41 50.5t33.5 43.5t33 44t24.5 34q-97 127 -140 175l39 146q67 -54 131.5 -1 [...]
+<glyph unicode="&#xe107;" d="M-97.5 34q13.5 -34 50.5 -34h1294q37 0 50.5 35.5t-7.5 67.5l-642 1056q-20 34 -48 36.5t-48 -29.5l-642 -1066q-21 -32 -7.5 -66zM155 200l445 723l445 -723h-345v100h-200v-100h-345zM500 600l100 -300l100 300v100h-200v-100z" />
+<glyph unicode="&#xe108;" d="M100 262v41q0 20 11 44.5t26 38.5l363 325v339q0 62 44 106t106 44t106 -44t44 -106v-339l363 -325q15 -14 26 -38.5t11 -44.5v-41q0 -20 -12 -26.5t-29 5.5l-359 249v-263q100 -91 100 -113v-64q0 -20 -13 -28.5t-32 0.5l-94 78h-222l-94 -78q-19 -9 -32 -0.5t-13 28.5 v64q0 22 100 113v263l-359 -249q-17 -12 -29 -5.5t-12 26.5z" />
+<glyph unicode="&#xe109;" d="M0 50q0 -20 14.5 -35t35.5 -15h1000q21 0 35.5 15t14.5 35v750h-1100v-750zM0 900h1100v150q0 21 -14.5 35.5t-35.5 14.5h-150v100h-100v-100h-500v100h-100v-100h-150q-21 0 -35.5 -14.5t-14.5 -35.5v-150zM100 100v100h100v-100h-100zM100 300v100h100v-100h-100z M100 500v100h100v-100h-100zM300 100v100h100v-100h-100zM300 300v100h100v-100h-100zM300 500v100h100v-100h-100zM500 100v100h100v-100h-100zM500 300v100h100v-100h-100zM500 500v100h100v-100h-100zM700 100v100h100v-100h-100z [...]
+<glyph unicode="&#xe110;" d="M0 200v200h259l600 600h241v198l300 -295l-300 -300v197h-159l-600 -600h-341zM0 800h259l122 -122l141 142l-181 180h-341v-200zM678 381l141 142l122 -123h159v198l300 -295l-300 -300v197h-241z" />
+<glyph unicode="&#xe111;" d="M0 400v600q0 41 29.5 70.5t70.5 29.5h1000q41 0 70.5 -29.5t29.5 -70.5v-600q0 -41 -29.5 -70.5t-70.5 -29.5h-596l-304 -300v300h-100q-41 0 -70.5 29.5t-29.5 70.5z" />
+<glyph unicode="&#xe112;" d="M100 600v200h300v-250q0 -113 6 -145q17 -92 102 -117q39 -11 92 -11q37 0 66.5 5.5t50 15.5t36 24t24 31.5t14 37.5t7 42t2.5 45t0 47v25v250h300v-200q0 -42 -3 -83t-15 -104t-31.5 -116t-58 -109.5t-89 -96.5t-129 -65.5t-174.5 -25.5t-174.5 25.5t-129 65.5t-89 96.5 t-58 109.5t-31.5 116t-15 104t-3 83zM100 900v300h300v-300h-300zM800 900v300h300v-300h-300z" />
+<glyph unicode="&#xe113;" d="M-30 411l227 -227l352 353l353 -353l226 227l-578 579z" />
+<glyph unicode="&#xe114;" d="M70 797l580 -579l578 579l-226 227l-353 -353l-352 353z" />
+<glyph unicode="&#xe115;" d="M-198 700l299 283l300 -283h-203v-400h385l215 -200h-800v600h-196zM402 1000l215 -200h381v-400h-198l299 -283l299 283h-200v600h-796z" />
+<glyph unicode="&#xe116;" d="M18 939q-5 24 10 42q14 19 39 19h896l38 162q5 17 18.5 27.5t30.5 10.5h94q20 0 35 -14.5t15 -35.5t-15 -35.5t-35 -14.5h-54l-201 -961q-2 -4 -6 -10.5t-19 -17.5t-33 -11h-31v-50q0 -20 -14.5 -35t-35.5 -15t-35.5 15t-14.5 35v50h-300v-50q0 -20 -14.5 -35t-35.5 -15 t-35.5 15t-14.5 35v50h-50q-21 0 -35.5 15t-14.5 35q0 21 14.5 35.5t35.5 14.5h535l48 200h-633q-32 0 -54.5 21t-27.5 43z" />
+<glyph unicode="&#xe117;" d="M0 0v800h1200v-800h-1200zM0 900v100h200q0 41 29.5 70.5t70.5 29.5h300q41 0 70.5 -29.5t29.5 -70.5h500v-100h-1200z" />
+<glyph unicode="&#xe118;" d="M1 0l300 700h1200l-300 -700h-1200zM1 400v600h200q0 41 29.5 70.5t70.5 29.5h300q41 0 70.5 -29.5t29.5 -70.5h500v-200h-1000z" />
+<glyph unicode="&#xe119;" d="M302 300h198v600h-198l298 300l298 -300h-198v-600h198l-298 -300z" />
+<glyph unicode="&#xe120;" d="M0 600l300 298v-198h600v198l300 -298l-300 -297v197h-600v-197z" />
+<glyph unicode="&#xe121;" d="M0 100v100q0 41 29.5 70.5t70.5 29.5h1000q41 0 70.5 -29.5t29.5 -70.5v-100q0 -41 -29.5 -70.5t-70.5 -29.5h-1000q-41 0 -70.5 29.5t-29.5 70.5zM31 400l172 739q5 22 23 41.5t38 19.5h672q19 0 37.5 -22.5t23.5 -45.5l172 -732h-1138zM800 100h100v100h-100v-100z M1000 100h100v100h-100v-100z" />
+<glyph unicode="&#xe122;" d="M-101 600v50q0 24 25 49t50 38l25 13v-250l-11 5.5t-24 14t-30 21.5t-24 27.5t-11 31.5zM100 500v250v8v8v7t0.5 7t1.5 5.5t2 5t3 4t4.5 3.5t6 1.5t7.5 0.5h200l675 250v-850l-675 200h-38l47 -276q2 -12 -3 -17.5t-11 -6t-21 -0.5h-8h-83q-20 0 -34.5 14t-18.5 35 q-55 337 -55 351zM1100 200v850q0 21 14.5 35.5t35.5 14.5q20 0 35 -14.5t15 -35.5v-850q0 -20 -15 -35t-35 -15q-21 0 -35.5 15t-14.5 35z" />
+<glyph unicode="&#xe123;" d="M74 350q0 21 13.5 35.5t33.5 14.5h18l117 173l63 327q15 77 76 140t144 83l-18 32q-6 19 3 32t29 13h94q20 0 29 -10.5t3 -29.5q-18 -36 -18 -37q83 -19 144 -82.5t76 -140.5l63 -327l118 -173h17q20 0 33.5 -14.5t13.5 -35.5q0 -20 -13 -40t-31 -27q-8 -3 -23 -8.5 t-65 -20t-103 -25t-132.5 -19.5t-158.5 -9q-125 0 -245.5 20.5t-178.5 40.5l-58 20q-18 7 -31 27.5t-13 40.5zM497 110q12 -49 40 -79.5t63 -30.5t63 30.5t39 79.5q-48 -6 -102 -6t-103 6z" />
+<glyph unicode="&#xe124;" d="M21 445l233 -45l-78 -224l224 78l45 -233l155 179l155 -179l45 233l224 -78l-78 224l234 45l-180 155l180 156l-234 44l78 225l-224 -78l-45 233l-155 -180l-155 180l-45 -233l-224 78l78 -225l-233 -44l179 -156z" />
+<glyph unicode="&#xe125;" d="M0 200h200v600h-200v-600zM300 275q0 -75 100 -75h61q124 -100 139 -100h250q46 0 83 57l238 344q29 31 29 74v100q0 44 -30.5 84.5t-69.5 40.5h-328q28 118 28 125v150q0 44 -30.5 84.5t-69.5 40.5h-50q-27 0 -51 -20t-38 -48l-96 -198l-145 -196q-20 -26 -20 -63v-400z M400 300v375l150 213l100 212h50v-175l-50 -225h450v-125l-250 -375h-214l-136 100h-100z" />
+<glyph unicode="&#xe126;" d="M0 400v600h200v-600h-200zM300 525v400q0 75 100 75h61q124 100 139 100h250q46 0 83 -57l238 -344q29 -31 29 -74v-100q0 -44 -30.5 -84.5t-69.5 -40.5h-328q28 -118 28 -125v-150q0 -44 -30.5 -84.5t-69.5 -40.5h-50q-27 0 -51 20t-38 48l-96 198l-145 196 q-20 26 -20 63zM400 525l150 -212l100 -213h50v175l-50 225h450v125l-250 375h-214l-136 -100h-100v-375z" />
+<glyph unicode="&#xe127;" d="M8 200v600h200v-600h-200zM308 275v525q0 17 14 35.5t28 28.5l14 9l362 230q14 6 25 6q17 0 29 -12l109 -112q14 -14 14 -34q0 -18 -11 -32l-85 -121h302q85 0 138.5 -38t53.5 -110t-54.5 -111t-138.5 -39h-107l-130 -339q-7 -22 -20.5 -41.5t-28.5 -19.5h-341 q-7 0 -90 81t-83 94zM408 289l100 -89h293l131 339q6 21 19.5 41t28.5 20h203q16 0 25 15t9 36q0 20 -9 34.5t-25 14.5h-457h-6.5h-7.5t-6.5 0.5t-6 1t-5 1.5t-5.5 2.5t-4 4t-4 5.5q-5 12 -5 20q0 14 10 27l147 183l-86 83l-339 -236v-503z" />
+<glyph unicode="&#xe128;" d="M-101 651q0 72 54 110t139 38l302 -1l-85 121q-11 16 -11 32q0 21 14 34l109 113q13 12 29 12q11 0 25 -6l365 -230q7 -4 17 -10.5t26.5 -26t16.5 -36.5v-526q0 -13 -86 -93.5t-94 -80.5h-341q-16 0 -29.5 20t-19.5 41l-130 339h-107q-84 0 -139 39t-55 111zM-1 601h222 q15 0 28.5 -20.5t19.5 -40.5l131 -339h293l107 89v502l-343 237l-87 -83l145 -184q10 -11 10 -26q0 -11 -5 -20q-1 -3 -3.5 -5.5l-4 -4t-5 -2.5t-5.5 -1.5t-6.5 -1t-6.5 -0.5h-7.5h-6.5h-476v-100zM1000 201v600h200v-600h-200z" />
+<glyph unicode="&#xe129;" d="M97 719l230 -363q4 -6 10.5 -15.5t26 -25t36.5 -15.5h525q13 0 94 83t81 90v342q0 15 -20 28.5t-41 19.5l-339 131v106q0 84 -39 139t-111 55t-110 -53.5t-38 -138.5v-302l-121 84q-15 12 -33.5 11.5t-32.5 -13.5l-112 -110q-22 -22 -6 -53zM172 739l83 86l183 -146 q22 -18 47 -5q3 1 5.5 3.5l4 4t2.5 5t1.5 5.5t1 6.5t0.5 6.5v7.5v6.5v456q0 22 25 31t50 -0.5t25 -30.5v-202q0 -16 20 -29.5t41 -19.5l339 -130v-294l-89 -100h-503zM400 0v200h600v-200h-600z" />
+<glyph unicode="&#xe130;" d="M2 585q-16 -31 6 -53l112 -110q13 -13 32 -13.5t34 10.5l121 85q0 -51 -0.5 -153.5t-0.5 -148.5q0 -84 38.5 -138t110.5 -54t111 55t39 139v106l339 131q20 6 40.5 19.5t20.5 28.5v342q0 7 -81 90t-94 83h-525q-17 0 -35.5 -14t-28.5 -28l-10 -15zM77 565l236 339h503 l89 -100v-294l-340 -130q-20 -6 -40 -20t-20 -29v-202q0 -22 -25 -31t-50 0t-25 31v456v14.5t-1.5 11.5t-5 12t-9.5 7q-24 13 -46 -5l-184 -146zM305 1104v200h600v-200h-600z" />
+<glyph unicode="&#xe131;" d="M5 597q0 122 47.5 232.5t127.5 190.5t190.5 127.5t232.5 47.5q162 0 299.5 -80t217.5 -218t80 -300t-80 -299.5t-217.5 -217.5t-299.5 -80t-300 80t-218 217.5t-80 299.5zM298 701l2 -201h300l-2 -194l402 294l-402 298v-197h-300z" />
+<glyph unicode="&#xe132;" d="M0 597q0 122 47.5 232.5t127.5 190.5t190.5 127.5t231.5 47.5q122 0 232.5 -47.5t190.5 -127.5t127.5 -190.5t47.5 -232.5q0 -162 -80 -299.5t-218 -217.5t-300 -80t-299.5 80t-217.5 217.5t-80 299.5zM200 600l402 -294l-2 194h300l2 201h-300v197z" />
+<glyph unicode="&#xe133;" d="M5 597q0 122 47.5 232.5t127.5 190.5t190.5 127.5t232.5 47.5q162 0 299.5 -80t217.5 -218t80 -300t-80 -299.5t-217.5 -217.5t-299.5 -80t-300 80t-218 217.5t-80 299.5zM300 600h200v-300h200v300h200l-300 400z" />
+<glyph unicode="&#xe134;" d="M5 597q0 122 47.5 232.5t127.5 190.5t190.5 127.5t232.5 47.5q162 0 299.5 -80t217.5 -218t80 -300t-80 -299.5t-217.5 -217.5t-299.5 -80t-300 80t-218 217.5t-80 299.5zM300 600l300 -400l300 400h-200v300h-200v-300h-200z" />
+<glyph unicode="&#xe135;" d="M5 597q0 122 47.5 232.5t127.5 190.5t190.5 127.5t232.5 47.5q121 0 231.5 -47.5t190.5 -127.5t127.5 -190.5t47.5 -232.5q0 -162 -80 -299.5t-217.5 -217.5t-299.5 -80t-300 80t-218 217.5t-80 299.5zM254 780q-8 -33 5.5 -92.5t7.5 -87.5q0 -9 17 -44t16 -60 q12 0 23 -5.5t23 -15t20 -13.5q24 -12 108 -42q22 -8 53 -31.5t59.5 -38.5t57.5 -11q8 -18 -15 -55t-20 -57q42 -71 87 -80q0 -6 -3 -15.5t-3.5 -14.5t4.5 -17q104 -3 221 112q30 29 47 47t34.5 49t20.5 62q-14 9 -37 9.5t-36 7.5q-14 7 - [...]
+<glyph unicode="&#xe136;" d="M0 164.5q0 21.5 15 37.5l600 599q-33 101 6 201.5t135 154.5q164 92 306 -9l-259 -138l145 -232l251 126q13 -175 -151 -267q-123 -70 -253 -23l-596 -596q-15 -16 -36.5 -16t-36.5 16l-111 110q-15 15 -15 36.5z" />
+<glyph unicode="&#xe137;" horiz-adv-x="1220" d="M0 196v100q0 41 29.5 70.5t70.5 29.5h1000q41 0 70.5 -29.5t29.5 -70.5v-100q0 -41 -29.5 -70.5t-70.5 -29.5h-1000q-41 0 -70.5 29.5t-29.5 70.5zM0 596v100q0 41 29.5 70.5t70.5 29.5h1000q41 0 70.5 -29.5t29.5 -70.5v-100q0 -41 -29.5 -70.5t-70.5 -29.5h-1000 q-41 0 -70.5 29.5t-29.5 70.5zM0 996v100q0 41 29.5 70.5t70.5 29.5h1000q41 0 70.5 -29.5t29.5 -70.5v-100q0 -41 -29.5 -70.5t-70.5 -29.5h-1000q-41 0 -70.5 29.5t-29.5 70.5zM600 596h500v100h-500v-100zM800  [...]
+<glyph unicode="&#xe138;" d="M100 1100v100h1000v-100h-1000zM150 1000h900l-350 -500v-300l-200 -200v500z" />
+<glyph unicode="&#xe139;" d="M0 200v200h1200v-200q0 -41 -29.5 -70.5t-70.5 -29.5h-1000q-41 0 -70.5 29.5t-29.5 70.5zM0 500v400q0 41 29.5 70.5t70.5 29.5h300v100q0 41 29.5 70.5t70.5 29.5h200q41 0 70.5 -29.5t29.5 -70.5v-100h300q41 0 70.5 -29.5t29.5 -70.5v-400h-500v100h-200v-100h-500z M500 1000h200v100h-200v-100z" />
+<glyph unicode="&#xe140;" d="M0 0v400l129 -129l200 200l142 -142l-200 -200l129 -129h-400zM0 800l129 129l200 -200l142 142l-200 200l129 129h-400v-400zM729 329l142 142l200 -200l129 129v-400h-400l129 129zM729 871l200 200l-129 129h400v-400l-129 129l-200 -200z" />
+<glyph unicode="&#xe141;" d="M0 596q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM182 596q0 -172 121.5 -293t292.5 -121t292.5 121t121.5 293q0 171 -121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM291 655 q0 23 15.5 38.5t38.5 15.5t39 -16t16 -38q0 -23 -16 -39t-39 -16q-22 0 -38 16t-16 39zM400 850q0 22 16 38.5t39 16.5q22 0 38 -16t16 -39t-16 -39t-38 -16q-23 0 -39 16.5t-16 38.5zM514 609q0 32 20.5 56.5t51.5 29.5l122 126l1 1q-9 14  [...]
+<glyph unicode="&#xe142;" d="M-40 375q-13 -95 35 -173q35 -57 94 -89t129 -32q63 0 119 28q33 16 65 40.5t52.5 45.5t59.5 64q40 44 57 61l394 394q35 35 47 84t-3 96q-27 87 -117 104q-20 2 -29 2q-46 0 -78.5 -16.5t-67.5 -51.5l-389 -396l-7 -7l69 -67l377 373q20 22 39 38q23 23 50 23 q38 0 53 -36q16 -39 -20 -75l-547 -547q-52 -52 -125 -52q-55 0 -100 33t-54 96q-5 35 2.5 66t31.5 63t42 50t56 54q24 21 44 41l348 348q52 52 82.5 79.5t84 54t107.5 26.5q25 0 48 -4q95 -17 154 -94.5t51 -175.5q-7 -101 -98 -192l-252 [...]
+<glyph unicode="&#xe143;" d="M80 784q0 131 98.5 229.5t230.5 98.5q143 0 241 -129q103 129 246 129q129 0 226 -98.5t97 -229.5q0 -46 -17.5 -91t-61 -99t-77 -89.5t-104.5 -105.5q-197 -191 -293 -322l-17 -23l-16 23q-43 58 -100 122.5t-92 99.5t-101 100q-71 70 -104.5 105.5t-77 89.5t-61 99 t-17.5 91zM250 784q0 -27 30.5 -70t61.5 -75.5t95 -94.5l22 -22q93 -90 190 -201q82 92 195 203l12 12q64 62 97.5 97t64.5 79t31 72q0 71 -48 119.5t-105 48.5q-74 0 -132 -83l-118 -171l-114 174q-51 80 -123 80q-60 0 -109.5 -49 [...]
+<glyph unicode="&#xe144;" d="M57 353q0 -95 66 -159l141 -142q68 -66 159 -66q93 0 159 66l283 283q66 66 66 159t-66 159l-141 141q-8 9 -19 17l-105 -105l212 -212l-389 -389l-247 248l95 95l-18 18q-46 45 -75 101l-55 -55q-66 -66 -66 -159zM269 706q0 -93 66 -159l141 -141q7 -7 19 -17l105 105 l-212 212l389 389l247 -247l-95 -96l18 -17q47 -49 77 -100l29 29q35 35 62.5 88t27.5 96q0 93 -66 159l-141 141q-66 66 -159 66q-95 0 -159 -66l-283 -283q-66 -64 -66 -159z" />
+<glyph unicode="&#xe145;" d="M200 100v953q0 21 30 46t81 48t129 38t163 15t162 -15t127 -38t79 -48t29 -46v-953q0 -41 -29.5 -70.5t-70.5 -29.5h-600q-41 0 -70.5 29.5t-29.5 70.5zM300 300h600v700h-600v-700zM496 150q0 -43 30.5 -73.5t73.5 -30.5t73.5 30.5t30.5 73.5t-30.5 73.5t-73.5 30.5 t-73.5 -30.5t-30.5 -73.5z" />
+<glyph unicode="&#xe146;" d="M0 0l303 380l207 208l-210 212h300l267 279l-35 36q-15 14 -15 35t15 35q14 15 35 15t35 -15l283 -282q15 -15 15 -36t-15 -35q-14 -15 -35 -15t-35 15l-36 35l-279 -267v-300l-212 210l-208 -207z" />
+<glyph unicode="&#xe148;" d="M295 433h139q5 -77 48.5 -126.5t117.5 -64.5v335q-6 1 -15.5 4t-11.5 3q-46 14 -79 26.5t-72 36t-62.5 52t-40 72.5t-16.5 99q0 92 44 159.5t109 101t144 40.5v78h100v-79q38 -4 72.5 -13.5t75.5 -31.5t71 -53.5t51.5 -84t24.5 -118.5h-159q-8 72 -35 109.5t-101 50.5 v-307l64 -14q34 -7 64 -16.5t70 -31.5t67.5 -52t47.5 -80.5t20 -112.5q0 -139 -89 -224t-244 -96v-77h-100v78q-152 17 -237 104q-40 40 -52.5 93.5t-15.5 139.5zM466 889q0 -29 8 -51t16.5 -34t29.5 -22.5t31 -13.5t38 -10q7 -2 1 [...]
+<glyph unicode="&#xe149;" d="M100 600v100h166q-24 49 -44 104q-10 26 -14.5 55.5t-3 72.5t25 90t68.5 87q97 88 263 88q129 0 230 -89t101 -208h-153q0 52 -34 89.5t-74 51.5t-76 14q-37 0 -79 -14.5t-62 -35.5q-41 -44 -41 -101q0 -28 16.5 -69.5t28 -62.5t41.5 -72h241v-100h-197q8 -50 -2.5 -115 t-31.5 -94q-41 -59 -99 -113q35 11 84 18t70 7q33 1 103 -16t103 -17q76 0 136 30l50 -147q-41 -25 -80.5 -36.5t-59 -13t-61.5 -1.5q-23 0 -128 33t-155 29q-39 -4 -82 -17t-66 -25l-24 -11l-55 145l16.5 11t15.5 10t13.5 9.5t1 [...]
+<glyph unicode="&#xe150;" d="M2 300l298 -300l298 300h-198v900h-200v-900h-198zM602 900l298 300l298 -300h-198v-900h-200v900h-198z" />
+<glyph unicode="&#xe151;" d="M2 300h198v900h200v-900h198l-298 -300zM700 0v200h100v-100h200v-100h-300zM700 400v100h300v-200h-99v-100h-100v100h99v100h-200zM700 700v500h300v-500h-100v100h-100v-100h-100zM801 900h100v200h-100v-200z" />
+<glyph unicode="&#xe152;" d="M2 300h198v900h200v-900h198l-298 -300zM700 0v500h300v-500h-100v100h-100v-100h-100zM700 700v200h100v-100h200v-100h-300zM700 1100v100h300v-200h-99v-100h-100v100h99v100h-200zM801 200h100v200h-100v-200z" />
+<glyph unicode="&#xe153;" d="M2 300l298 -300l298 300h-198v900h-200v-900h-198zM800 100v400h300v-500h-100v100h-200zM800 1100v100h200v-500h-100v400h-100zM901 200h100v200h-100v-200z" />
+<glyph unicode="&#xe154;" d="M2 300l298 -300l298 300h-198v900h-200v-900h-198zM800 400v100h200v-500h-100v400h-100zM800 800v400h300v-500h-100v100h-200zM901 900h100v200h-100v-200z" />
+<glyph unicode="&#xe155;" d="M2 300l298 -300l298 300h-198v900h-200v-900h-198zM700 100v200h500v-200h-500zM700 400v200h400v-200h-400zM700 700v200h300v-200h-300zM700 1000v200h200v-200h-200z" />
+<glyph unicode="&#xe156;" d="M2 300l298 -300l298 300h-198v900h-200v-900h-198zM700 100v200h200v-200h-200zM700 400v200h300v-200h-300zM700 700v200h400v-200h-400zM700 1000v200h500v-200h-500z" />
+<glyph unicode="&#xe157;" d="M0 400v300q0 165 117.5 282.5t282.5 117.5h300q162 0 281 -118.5t119 -281.5v-300q0 -165 -118.5 -282.5t-281.5 -117.5h-300q-165 0 -282.5 117.5t-117.5 282.5zM200 300q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5v500q0 41 -29.5 70.5t-70.5 29.5 h-500q-41 0 -70.5 -29.5t-29.5 -70.5v-500z" />
+<glyph unicode="&#xe158;" d="M0 400v300q0 163 119 281.5t281 118.5h300q165 0 282.5 -117.5t117.5 -282.5v-300q0 -165 -117.5 -282.5t-282.5 -117.5h-300q-163 0 -281.5 117.5t-118.5 282.5zM200 300q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5v500q0 41 -29.5 70.5t-70.5 29.5 h-500q-41 0 -70.5 -29.5t-29.5 -70.5v-500zM400 300l333 250l-333 250v-500z" />
+<glyph unicode="&#xe159;" d="M0 400v300q0 163 117.5 281.5t282.5 118.5h300q163 0 281.5 -119t118.5 -281v-300q0 -165 -117.5 -282.5t-282.5 -117.5h-300q-165 0 -282.5 117.5t-117.5 282.5zM200 300q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5v500q0 41 -29.5 70.5t-70.5 29.5 h-500q-41 0 -70.5 -29.5t-29.5 -70.5v-500zM300 700l250 -333l250 333h-500z" />
+<glyph unicode="&#xe160;" d="M0 400v300q0 165 117.5 282.5t282.5 117.5h300q165 0 282.5 -117.5t117.5 -282.5v-300q0 -162 -118.5 -281t-281.5 -119h-300q-165 0 -282.5 118.5t-117.5 281.5zM200 300q0 -41 29.5 -70.5t70.5 -29.5h500q41 0 70.5 29.5t29.5 70.5v500q0 41 -29.5 70.5t-70.5 29.5 h-500q-41 0 -70.5 -29.5t-29.5 -70.5v-500zM300 400h500l-250 333z" />
+<glyph unicode="&#xe161;" d="M0 400v300h300v200l400 -350l-400 -350v200h-300zM500 0v200h500q41 0 70.5 29.5t29.5 70.5v500q0 41 -29.5 70.5t-70.5 29.5h-500v200h400q165 0 282.5 -117.5t117.5 -282.5v-300q0 -165 -117.5 -282.5t-282.5 -117.5h-400z" />
+<glyph unicode="&#xe162;" d="M217 519q8 -19 31 -19h302q-155 -438 -160 -458q-5 -21 4 -32l9 -8h9q14 0 26 15q11 13 274.5 321.5t264.5 308.5q14 19 5 36q-8 17 -31 17l-301 -1q1 4 78 219.5t79 227.5q2 15 -5 27l-9 9h-9q-15 0 -25 -16q-4 -6 -98 -111.5t-228.5 -257t-209.5 -237.5q-16 -19 -6 -41 z" />
+<glyph unicode="&#xe163;" d="M0 400q0 -165 117.5 -282.5t282.5 -117.5h300q47 0 100 15v185h-500q-41 0 -70.5 29.5t-29.5 70.5v500q0 41 29.5 70.5t70.5 29.5h500v185q-14 4 -114 7.5t-193 5.5l-93 2q-165 0 -282.5 -117.5t-117.5 -282.5v-300zM600 400v300h300v200l400 -350l-400 -350v200h-300z " />
+<glyph unicode="&#xe164;" d="M0 400q0 -165 117.5 -282.5t282.5 -117.5h300q163 0 281.5 117.5t118.5 282.5v98l-78 73l-122 -123v-148q0 -41 -29.5 -70.5t-70.5 -29.5h-500q-41 0 -70.5 29.5t-29.5 70.5v500q0 41 29.5 70.5t70.5 29.5h156l118 122l-74 78h-100q-165 0 -282.5 -117.5t-117.5 -282.5 v-300zM496 709l353 342l-149 149h500v-500l-149 149l-342 -353z" />
+<glyph unicode="&#xe165;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM406 600 q0 80 57 137t137 57t137 -57t57 -137t-57 -137t-137 -57t-137 57t-57 137z" />
+<glyph unicode="&#xe166;" d="M0 0v275q0 11 7 18t18 7h1048q11 0 19 -7.5t8 -17.5v-275h-1100zM100 800l445 -500l450 500h-295v400h-300v-400h-300zM900 150h100v50h-100v-50z" />
+<glyph unicode="&#xe167;" d="M0 0v275q0 11 7 18t18 7h1048q11 0 19 -7.5t8 -17.5v-275h-1100zM100 700h300v-300h300v300h295l-445 500zM900 150h100v50h-100v-50z" />
+<glyph unicode="&#xe168;" d="M0 0v275q0 11 7 18t18 7h1048q11 0 19 -7.5t8 -17.5v-275h-1100zM100 705l305 -305l596 596l-154 155l-442 -442l-150 151zM900 150h100v50h-100v-50z" />
+<glyph unicode="&#xe169;" d="M0 0v275q0 11 7 18t18 7h1048q11 0 19 -7.5t8 -17.5v-275h-1100zM100 988l97 -98l212 213l-97 97zM200 400l697 1l3 699l-250 -239l-149 149l-212 -212l149 -149zM900 150h100v50h-100v-50z" />
+<glyph unicode="&#xe170;" d="M0 0v275q0 11 7 18t18 7h1048q11 0 19 -7.5t8 -17.5v-275h-1100zM200 612l212 -212l98 97l-213 212zM300 1200l239 -250l-149 -149l212 -212l149 148l249 -237l-1 697zM900 150h100v50h-100v-50z" />
+<glyph unicode="&#xe171;" d="M23 415l1177 784v-1079l-475 272l-310 -393v416h-392zM494 210l672 938l-672 -712v-226z" />
+<glyph unicode="&#xe172;" d="M0 150v1000q0 20 14.5 35t35.5 15h250v-300h500v300h100l200 -200v-850q0 -21 -15 -35.5t-35 -14.5h-150v400h-700v-400h-150q-21 0 -35.5 14.5t-14.5 35.5zM600 1000h100v200h-100v-200z" />
+<glyph unicode="&#xe173;" d="M0 150v1000q0 20 14.5 35t35.5 15h250v-300h500v300h100l200 -200v-218l-276 -275l-120 120l-126 -127h-378v-400h-150q-21 0 -35.5 14.5t-14.5 35.5zM581 306l123 123l120 -120l353 352l123 -123l-475 -476zM600 1000h100v200h-100v-200z" />
+<glyph unicode="&#xe174;" d="M0 150v1000q0 20 14.5 35t35.5 15h250v-300h500v300h100l200 -200v-269l-103 -103l-170 170l-298 -298h-329v-400h-150q-21 0 -35.5 14.5t-14.5 35.5zM600 1000h100v200h-100v-200zM700 133l170 170l-170 170l127 127l170 -170l170 170l127 -128l-170 -169l170 -170 l-127 -127l-170 170l-170 -170z" />
+<glyph unicode="&#xe175;" d="M0 150v1000q0 20 14.5 35t35.5 15h250v-300h500v300h100l200 -200v-300h-400v-200h-500v-400h-150q-21 0 -35.5 14.5t-14.5 35.5zM600 300l300 -300l300 300h-200v300h-200v-300h-200zM600 1000v200h100v-200h-100z" />
+<glyph unicode="&#xe176;" d="M0 150v1000q0 20 14.5 35t35.5 15h250v-300h500v300h100l200 -200v-402l-200 200l-298 -298h-402v-400h-150q-21 0 -35.5 14.5t-14.5 35.5zM600 300h200v-300h200v300h200l-300 300zM600 1000v200h100v-200h-100z" />
+<glyph unicode="&#xe177;" d="M0 250q0 -21 14.5 -35.5t35.5 -14.5h1100q21 0 35.5 14.5t14.5 35.5v550h-1200v-550zM0 900h1200v150q0 21 -14.5 35.5t-35.5 14.5h-1100q-21 0 -35.5 -14.5t-14.5 -35.5v-150zM100 300v200h400v-200h-400z" />
+<glyph unicode="&#xe178;" d="M0 400l300 298v-198h400v-200h-400v-198zM100 800v200h100v-200h-100zM300 800v200h100v-200h-100zM500 800v200h400v198l300 -298l-300 -298v198h-400zM800 300v200h100v-200h-100zM1000 300h100v200h-100v-200z" />
+<glyph unicode="&#xe179;" d="M100 700v400l50 100l50 -100v-300h100v300l50 100l50 -100v-300h100v300l50 100l50 -100v-400l-100 -203v-447q0 -21 -14.5 -35.5t-35.5 -14.5h-200q-21 0 -35.5 14.5t-14.5 35.5v447zM800 597q0 -29 10.5 -55.5t25 -43t29 -28.5t25.5 -18l10 -5v-397q0 -21 14.5 -35.5 t35.5 -14.5h200q21 0 35.5 14.5t14.5 35.5v1106q0 31 -18 40.5t-44 -7.5l-276 -116q-25 -17 -43.5 -51.5t-18.5 -65.5v-359z" />
+<glyph unicode="&#xe180;" d="M100 0h400v56q-75 0 -87.5 6t-12.5 44v394h500v-394q0 -38 -12.5 -44t-87.5 -6v-56h400v56q-4 0 -11 0.5t-24 3t-30 7t-24 15t-11 24.5v888q0 22 25 34.5t50 13.5l25 2v56h-400v-56q75 0 87.5 -6t12.5 -44v-394h-500v394q0 38 12.5 44t87.5 6v56h-400v-56q4 0 11 -0.5 t24 -3t30 -7t24 -15t11 -24.5v-888q0 -22 -25 -34.5t-50 -13.5l-25 -2v-56z" />
+<glyph unicode="&#xe181;" d="M0 300q0 -41 29.5 -70.5t70.5 -29.5h300q41 0 70.5 29.5t29.5 70.5v500q0 41 -29.5 70.5t-70.5 29.5h-300q-41 0 -70.5 -29.5t-29.5 -70.5v-500zM100 100h400l200 200h105l295 98v-298h-425l-100 -100h-375zM100 300v200h300v-200h-300zM100 600v200h300v-200h-300z M100 1000h400l200 -200v-98l295 98h105v200h-425l-100 100h-375zM700 402v163l400 133v-163z" />
+<glyph unicode="&#xe182;" d="M16.5 974.5q0.5 -21.5 16 -90t46.5 -140t104 -177.5t175 -208q103 -103 207.5 -176t180 -103.5t137 -47t92.5 -16.5l31 1l163 162q17 18 13.5 41t-22.5 37l-192 136q-19 14 -45 12t-42 -19l-118 -118q-142 101 -268 227t-227 268l118 118q17 17 20 41.5t-11 44.5 l-139 194q-14 19 -36.5 22t-40.5 -14l-162 -162q-1 -11 -0.5 -32.5z" />
+<glyph unicode="&#xe183;" d="M0 50v212q0 20 10.5 45.5t24.5 39.5l365 303v50q0 4 1 10.5t12 22.5t30 28.5t60 23t97 10.5t97 -10t60 -23.5t30 -27.5t12 -24l1 -10v-50l365 -303q14 -14 24.5 -39.5t10.5 -45.5v-212q0 -21 -14.5 -35.5t-35.5 -14.5h-1100q-20 0 -35 14.5t-15 35.5zM0 712 q0 -21 14.5 -33.5t34.5 -8.5l202 33q20 4 34.5 21t14.5 38v146q141 24 300 24t300 -24v-146q0 -21 14.5 -38t34.5 -21l202 -33q20 -4 34.5 8.5t14.5 33.5v200q-6 8 -19 20.5t-63 45t-112 57t-171 45t-235 20.5q-92 0 -175 -10.5t-141.5 -27t- [...]
+<glyph unicode="&#xe184;" d="M100 0v100h1100v-100h-1100zM175 200h950l-125 150v250l100 100v400h-100v-200h-100v200h-200v-200h-100v200h-200v-200h-100v200h-100v-400l100 -100v-250z" />
+<glyph unicode="&#xe185;" d="M100 0h300v400q0 41 -29.5 70.5t-70.5 29.5h-100q-41 0 -70.5 -29.5t-29.5 -70.5v-400zM500 0v1000q0 41 29.5 70.5t70.5 29.5h100q41 0 70.5 -29.5t29.5 -70.5v-1000h-300zM900 0v700q0 41 29.5 70.5t70.5 29.5h100q41 0 70.5 -29.5t29.5 -70.5v-700h-300z" />
+<glyph unicode="&#xe186;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 300h300v300h-200v100h200v100h-300v-300h200v-100h-200v-100zM600 300h200v100h100v300h-100v100h-200v-500 zM700 400v300h100v-300h-100z" />
+<glyph unicode="&#xe187;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 300h100v200h100v-200h100v500h-100v-200h-100v200h-100v-500zM600 300h200v100h100v300h-100v100h-200v-500 zM700 400v300h100v-300h-100z" />
+<glyph unicode="&#xe188;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 300h300v100h-200v300h200v100h-300v-500zM600 300h300v100h-200v300h200v100h-300v-500z" />
+<glyph unicode="&#xe189;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 550l300 -150v300zM600 400l300 150l-300 150v-300z" />
+<glyph unicode="&#xe190;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 300v500h700v-500h-700zM300 400h130q41 0 68 42t27 107t-28.5 108t-66.5 43h-130v-300zM575 549 q0 -65 27 -107t68 -42h130v300h-130q-38 0 -66.5 -43t-28.5 -108z" />
+<glyph unicode="&#xe191;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 300h300v300h-200v100h200v100h-300v-300h200v-100h-200v-100zM601 300h100v100h-100v-100zM700 700h100 v-400h100v500h-200v-100z" />
+<glyph unicode="&#xe192;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 300h300v400h-200v100h-100v-500zM301 400v200h100v-200h-100zM601 300h100v100h-100v-100zM700 700h100 v-400h100v500h-200v-100z" />
+<glyph unicode="&#xe193;" d="M-100 300v500q0 124 88 212t212 88h700q124 0 212 -88t88 -212v-500q0 -124 -88 -212t-212 -88h-700q-124 0 -212 88t-88 212zM100 200h900v700h-900v-700zM200 700v100h300v-300h-99v-100h-100v100h99v200h-200zM201 300v100h100v-100h-100zM601 300v100h100v-100h-100z M700 700v100h200v-500h-100v400h-100z" />
+<glyph unicode="&#xe194;" d="M4 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM186 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM400 500v200 l100 100h300v-100h-300v-200h300v-100h-300z" />
+<glyph unicode="&#xe195;" d="M0 600q0 162 80 299t217 217t299 80t299 -80t217 -217t80 -299t-80 -299t-217 -217t-299 -80t-299 80t-217 217t-80 299zM182 600q0 -171 121.5 -292.5t292.5 -121.5t292.5 121.5t121.5 292.5t-121.5 292.5t-292.5 121.5t-292.5 -121.5t-121.5 -292.5zM400 400v400h300 l100 -100v-100h-100v100h-200v-100h200v-100h-200v-100h-100zM700 400v100h100v-100h-100z" />
+<glyph unicode="&#xe197;" d="M-14 494q0 -80 56.5 -137t135.5 -57h222v300h400v-300h128q120 0 205 86.5t85 207.5t-85 207t-205 86q-46 0 -90 -14q-44 97 -134.5 156.5t-200.5 59.5q-152 0 -260 -107.5t-108 -260.5q0 -25 2 -37q-66 -14 -108.5 -67.5t-42.5 -122.5zM300 200h200v300h200v-300h200 l-300 -300z" />
+<glyph unicode="&#xe198;" d="M-14 494q0 -80 56.5 -137t135.5 -57h8l414 414l403 -403q94 26 154.5 104.5t60.5 178.5q0 120 -85 206.5t-205 86.5q-46 0 -90 -14q-44 97 -134.5 156.5t-200.5 59.5q-152 0 -260 -107.5t-108 -260.5q0 -25 2 -37q-66 -14 -108.5 -67.5t-42.5 -122.5zM300 200l300 300 l300 -300h-200v-300h-200v300h-200z" />
+<glyph unicode="&#xe199;" d="M100 200h400v-155l-75 -45h350l-75 45v155h400l-270 300h170l-270 300h170l-300 333l-300 -333h170l-270 -300h170z" />
+<glyph unicode="&#xe200;" d="M121 700q0 -53 28.5 -97t75.5 -65q-4 -16 -4 -38q0 -74 52.5 -126.5t126.5 -52.5q56 0 100 30v-306l-75 -45h350l-75 45v306q46 -30 100 -30q74 0 126.5 52.5t52.5 126.5q0 24 -9 55q50 32 79.5 83t29.5 112q0 90 -61.5 155.5t-150.5 71.5q-26 89 -99.5 145.5 t-167.5 56.5q-116 0 -197.5 -81.5t-81.5 -197.5q0 -4 1 -11.5t1 -11.5q-14 2 -23 2q-74 0 -126.5 -52.5t-52.5 -126.5z" />
+</font>
+</defs></svg> 
\ No newline at end of file
diff --git a/samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.ttf b/samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.ttf
new file mode 100644
index 000000000..67fa00bf8
Binary files /dev/null and b/samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.ttf differ
diff --git a/samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.woff b/samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.woff
new file mode 100644
index 000000000..8c54182aa
Binary files /dev/null and b/samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.woff differ
diff --git a/samza-yarn3/src/main/resources/scalate/img/asc.gif b/samza-yarn3/src/main/resources/scalate/img/asc.gif
new file mode 100644
index 000000000..74157867f
Binary files /dev/null and b/samza-yarn3/src/main/resources/scalate/img/asc.gif differ
diff --git a/samza-yarn3/src/main/resources/scalate/img/bg.gif b/samza-yarn3/src/main/resources/scalate/img/bg.gif
new file mode 100644
index 000000000..fac668fcf
Binary files /dev/null and b/samza-yarn3/src/main/resources/scalate/img/bg.gif differ
diff --git a/samza-yarn3/src/main/resources/scalate/img/desc.gif b/samza-yarn3/src/main/resources/scalate/img/desc.gif
new file mode 100644
index 000000000..3b30b3c58
Binary files /dev/null and b/samza-yarn3/src/main/resources/scalate/img/desc.gif differ
diff --git a/samza-yarn3/src/main/resources/scalate/img/samza-icon.png b/samza-yarn3/src/main/resources/scalate/img/samza-icon.png
new file mode 100644
index 000000000..4aa6c792d
Binary files /dev/null and b/samza-yarn3/src/main/resources/scalate/img/samza-icon.png differ
diff --git a/samza-yarn3/src/main/resources/scalate/js/bootstrap.min.js b/samza-yarn3/src/main/resources/scalate/js/bootstrap.min.js
new file mode 100755
index 000000000..b04a0e82f
--- /dev/null
+++ b/samza-yarn3/src/main/resources/scalate/js/bootstrap.min.js
@@ -0,0 +1,6 @@
+/*!
+ * Bootstrap v3.1.1 (http://getbootstrap.com)
+ * Copyright 2011-2014 Twitter, Inc.
+ * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE)
+ */
+if("undefined"==typeof jQuery)throw new Error("Bootstrap's JavaScript requires jQuery");+function(a){"use strict";function b(){var a=document.createElement("bootstrap"),b={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"};for(var c in b)if(void 0!==a.style[c])return{end:b[c]};return!1}a.fn.emulateTransitionEnd=function(b){var c=!1,d=this;a(this).one(a.support.transition.end,function(){c=!0});var e= [...]
\ No newline at end of file
diff --git a/samza-yarn3/src/main/resources/scalate/js/jquery-1.11.1.min.js b/samza-yarn3/src/main/resources/scalate/js/jquery-1.11.1.min.js
new file mode 100644
index 000000000..ab28a2472
--- /dev/null
+++ b/samza-yarn3/src/main/resources/scalate/js/jquery-1.11.1.min.js
@@ -0,0 +1,4 @@
+/*! jQuery v1.11.1 | (c) 2005, 2014 jQuery Foundation, Inc. | jquery.org/license */
+!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=c.slice,e=c.concat,f=c.push,g=c.indexOf,h={},i=h.toString,j=h.hasOwnProperty,k={},l="1.11.1",m=function(a,b){return new m.fn.init(a,b)},n=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,o=/^-ms-/,p=/-([\da-z])/gi,q=function(a,b [...]
+if(k&&j[k]&&(e||j[k].data)||void 0!==d||"string"!=typeof b)return k||(k=i?a[h]=c.pop()||m.guid++:h),j[k]||(j[k]=i?{}:{toJSON:m.noop}),("object"==typeof b||"function"==typeof b)&&(e?j[k]=m.extend(j[k],b):j[k].data=m.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[m.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[m.camelCase(b)])):f=g,f}}function R(a,b,c){if(m.acceptData(a)){var d,e,f=a.nodeType,g=f?m.cache:a,h=f?a[m.expando]:m.expando;if(g[h]){if(b&& [...]
+},cur:function(){var a=Zb.propHooks[this.prop];return a&&a.get?a.get(this):Zb.propHooks._default.get(this)},run:function(a){var b,c=Zb.propHooks[this.prop];return this.pos=b=this.options.duration?m.easing[this.easing](a,this.options.duration*a,0,1,this.options.duration):a,this.now=(this.end-this.start)*b+this.start,this.options.step&&this.options.step.call(this.elem,this.now,this),c&&c.set?c.set(this):Zb.propHooks._default.set(this),this}},Zb.prototype.init.prototype=Zb.prototype,Zb.prop [...]
diff --git a/samza-yarn3/src/main/resources/scalate/js/jquery.tablesorter.min.js b/samza-yarn3/src/main/resources/scalate/js/jquery.tablesorter.min.js
new file mode 100644
index 000000000..b8605df1e
--- /dev/null
+++ b/samza-yarn3/src/main/resources/scalate/js/jquery.tablesorter.min.js
@@ -0,0 +1,4 @@
+
+(function($){$.extend({tablesorter:new
+function(){var parsers=[],widgets=[];this.defaults={cssHeader:"header",cssAsc:"headerSortUp",cssDesc:"headerSortDown",cssChildRow:"expand-child",sortInitialOrder:"asc",sortMultiSortKey:"shiftKey",sortForce:null,sortAppend:null,sortLocaleCompare:true,textExtraction:"simple",parsers:{},widgets:[],widgetZebra:{css:["even","odd"]},headers:{},widthFixed:false,cancelSelection:true,sortList:[],headerList:[],dateFormat:"us",decimal:'/\.|\,/g',onRenderHeader:null,selectorHeaders:'thead th',debug: [...]
+var firstAvailCol;if(typeof(matrix[rowIndex])=="undefined"){matrix[rowIndex]=[];}for(var k=0;k<matrix[rowIndex].length+1;k++){if(typeof(matrix[rowIndex][k])=="undefined"){firstAvailCol=k;break;}}lookup[cellId]=firstAvailCol;for(var k=rowIndex;k<rowIndex+rowSpan;k++){if(typeof(matrix[k])=="undefined"){matrix[k]=[];}var matrixrow=matrix[k];for(var l=firstAvailCol;l<firstAvailCol+colSpan;l++){matrixrow[l]="x";}}}}return lookup;}function checkCellColSpan(table,rows,row){var arr=[],r=table.tH [...]
\ No newline at end of file
diff --git a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala
similarity index 99%
copy from samza-yarn/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala
copy to samza-yarn3/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala
index 8e0c3d1b2..207eb6b12 100644
--- a/samza-yarn/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala
+++ b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala
@@ -19,7 +19,8 @@
 
 package org.apache.samza.job.yarn
 
-import org.apache.commons.lang.StringUtils
+
+import org.apache.commons.lang3.StringUtils
 import org.apache.hadoop.fs.permission.FsPermission
 import org.apache.samza.config.{Config, JobConfig, YarnConfig}
 import org.apache.samza.coordinator.stream.CoordinatorStreamWriter
@@ -53,9 +54,9 @@ import org.apache.samza.job.ApplicationStatus.Running
 import org.apache.samza.job.ApplicationStatus.SuccessfulFinish
 import org.apache.samza.job.ApplicationStatus.UnsuccessfulFinish
 import org.apache.samza.util.Logging
+
 import java.io.IOException
 import java.nio.ByteBuffer
-
 import org.apache.http.impl.client.HttpClientBuilder
 import org.apache.samza.webapp.ApplicationMasterRestClient
 
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterMetrics.scala b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterMetrics.scala
new file mode 100644
index 000000000..fb540f77f
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterMetrics.scala
@@ -0,0 +1,67 @@
+/*
+ * 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.samza.job.yarn
+
+import org.apache.samza.clustermanager.SamzaApplicationState
+import org.apache.samza.config.{Config, MetricsConfig}
+import org.apache.samza.util.Logging
+import org.apache.samza.util.MetricsReporterLoader
+import org.apache.samza.metrics.{Counter, MetricsHelper, ReadableMetricsRegistry}
+
+import scala.collection.JavaConverters._
+
+object SamzaAppMasterMetrics {
+  val sourceName = "ApplicationMaster"
+}
+
+/**
+ * Responsible for wiring up Samza's metrics. Given that Samza has a metric
+ * registry, we might as well use it. This class takes Samza's application
+ * master state, and converts it to metrics.
+ */
+class SamzaAppMasterMetrics(val config: Config,
+  val state: SamzaApplicationState,
+  val registry: ReadableMetricsRegistry) extends MetricsHelper with Logging {
+
+  private val metricsConfig = new MetricsConfig(config)
+  val containersFromPreviousAttempts = newGauge("container-from-previous-attempt", 0L)
+  val reporters = MetricsReporterLoader.getMetricsReporters(metricsConfig, SamzaAppMasterMetrics.sourceName).asScala
+  reporters.values.foreach(_.register(SamzaAppMasterMetrics.sourceName, registry))
+
+  def setContainersFromPreviousAttempts(containerCount: Int) {
+    containersFromPreviousAttempts.set(containerCount)
+  }
+
+  def start() {
+    val mRunningContainers = newGauge("running-containers", () => state.runningProcessors.size)
+    val mNeededContainers = newGauge("needed-containers", () => state.neededProcessors.get())
+    val mCompletedContainers = newGauge("completed-containers", () => state.completedProcessors.get())
+    val mFailedContainers = newGauge("failed-containers", () => state.failedContainers.get())
+    val mReleasedContainers = newGauge("released-containers", () => state.releasedContainers.get())
+    val mContainers = newGauge("container-count", () => state.processorCount.get())
+    val mJobHealthy = newGauge("job-healthy", () => if (state.jobHealthy.get()) 1 else 0)
+
+    reporters.values.foreach(_.start)
+  }
+
+  def stop() {
+    reporters.values.foreach(_.stop)
+  }
+}
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterSecurityManager.scala b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterSecurityManager.scala
new file mode 100644
index 000000000..bb590c5d3
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterSecurityManager.scala
@@ -0,0 +1,126 @@
+/*
+ * 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.samza.job.yarn
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.security.Credentials
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.samza.SamzaException
+import org.apache.samza.config.Config
+import org.apache.samza.config.YarnConfig
+import org.apache.samza.container.SecurityManager
+import org.apache.samza.util.Logging
+
+import java.security.PrivilegedExceptionAction
+import java.util.concurrent.Executors
+import java.util.concurrent.TimeUnit
+
+
+/**
+  * The SamzaAppMasterSecurityManager is responsible for renewing and distributing HDFS delegation tokens on a secure YARN
+  * cluster.
+  *
+  * <p />
+  *
+  * It runs in a daemon thread and periodically requests new delegation tokens and writes the fresh tokens in a configured
+  * staging directory at the configured frequency.
+  *
+  * @param config     Samza config for the application
+  * @param hadoopConf the hadoop configuration
+  */
+class SamzaAppMasterSecurityManager(config: Config, hadoopConf: Configuration) extends SecurityManager with Logging {
+  private val tokenRenewExecutor = Executors.newSingleThreadScheduledExecutor(
+    new ThreadFactoryBuilder()
+      .setNameFormat("Samza AMSecurityManager TokenRenewer Thread-%d")
+      .setDaemon(true)
+      .build())
+
+  def start() = {
+    val yarnConfig = new YarnConfig(config)
+    val principal = yarnConfig.getYarnKerberosPrincipal
+    // only get the name part of the keytab config, the keytab file will in the working directory
+    val keytab = new Path(yarnConfig.getYarnKerberosKeytab).getName
+    val renewalInterval = yarnConfig.getYarnTokenRenewalIntervalSeconds
+    val credentialsFile = yarnConfig.getYarnCredentialsFile
+
+    val tokenRenewRunnable = new Runnable {
+      override def run(): Unit = {
+        try {
+          loginFromKeytab(principal, keytab, credentialsFile)
+        } catch {
+          case e: Exception =>
+            warn("Failed to renew token and write out new credentials", e)
+        }
+      }
+    }
+
+    tokenRenewExecutor.scheduleAtFixedRate(tokenRenewRunnable, 0, renewalInterval, TimeUnit.SECONDS)
+  }
+
+  private def loginFromKeytab(principal: String, keytab: String, credentialsFile: String) = {
+    info(s"Logging to KDC using principal: $principal")
+    val keytabUser = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab)
+    val credentials = keytabUser.getCredentials
+
+    // get the new delegation token from the key tab user
+    keytabUser.doAs(new PrivilegedExceptionAction[Void] {
+      override def run(): Void = {
+        getNewDelegationToken(credentials)
+        null
+      }
+    })
+
+    UserGroupInformation.getCurrentUser.addCredentials(credentials)
+
+    val credentialsFilePath = new Path(credentialsFile)
+    writeNewDelegationToken(credentialsFilePath, credentials)
+  }
+
+  private def getNewDelegationToken(credentials: Credentials) = {
+    val fs = FileSystem.get(hadoopConf)
+    val tokenRenewer = UserGroupInformation.getCurrentUser.getShortUserName
+    // HDFS will not issue new delegation token if there are existing ones in the credentials. This is hacked
+    // by creating a new credentials object from the login via the given keytab and principle, passing the new
+    // credentials object to FileSystem.addDelegationTokens to force a new delegation token created and adding
+    // the credentials to the current user's credential object
+    fs.addDelegationTokens(tokenRenewer, credentials)
+  }
+
+  private def writeNewDelegationToken(credentialsFilePath: Path, credentials: Credentials) = {
+    val fs = FileSystem.get(hadoopConf)
+    if (fs.exists(credentialsFilePath)) {
+      logger.info(s"Deleting existing credentials file $credentialsFilePath")
+      val success = fs.delete(credentialsFilePath, false)
+      if (!success) {
+        throw new SamzaException(s"Failed deleting existing credentials file $credentialsFilePath")
+      }
+    }
+
+    logger.info(s"Writing new delegation to the token file $credentialsFilePath")
+    credentials.writeTokenStorageFile(credentialsFilePath, hadoopConf)
+  }
+
+  def stop() = {
+    tokenRenewExecutor.shutdown
+  }
+}
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaContainerSecurityManager.scala b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaContainerSecurityManager.scala
new file mode 100644
index 000000000..fce840cca
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaContainerSecurityManager.scala
@@ -0,0 +1,102 @@
+/*
+ * 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.samza.job.yarn
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.security.Credentials
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.samza.config.Config
+import org.apache.samza.config.YarnConfig
+import org.apache.samza.container.SecurityManager
+import org.apache.samza.util.Logging
+
+import java.util.concurrent.Executors
+import java.util.concurrent.TimeUnit
+
+class SamzaContainerSecurityManager(config: Config, hadoopConfig: Configuration) extends SecurityManager with Logging {
+  private val InitialDelayInSeconds = 60
+  
+  private val tokenRenewExecutor = Executors.newSingleThreadScheduledExecutor(
+    new ThreadFactoryBuilder()
+      .setNameFormat("Samza ContainerSecurityManager TokenRenewer Thread-%d")
+      .setDaemon(true)
+      .build())
+
+  private var lastRefreshTimestamp = 0L
+
+  def start() = {
+    val yarnConfig = new YarnConfig(config)
+    val renewalInterval = yarnConfig.getYarnTokenRenewalIntervalSeconds
+    val tokenFilePath = new Path(yarnConfig.getYarnCredentialsFile)
+
+    val tokenRenewRunnable = new Runnable {
+      override def run(): Unit = {
+        try {
+          val fs = FileSystem.get(hadoopConfig)
+          if (fs.exists(tokenFilePath)) {
+            val fileStatus = fs.getFileStatus(tokenFilePath)
+            if (lastRefreshTimestamp > fileStatus.getModificationTime) {
+              // credentials have not been updated, retry after 5 minutes
+              info("Expecting to update delegation tokens, but AM has not updated credentials file yet, will retry in 5 minutes")
+              tokenRenewExecutor.schedule(this, 5, TimeUnit.MINUTES)
+            } else {
+              val credentials = getCredentialsFromHDFS(fs, tokenFilePath)
+              UserGroupInformation.getCurrentUser.addCredentials(credentials)
+              info("Successfully renewed tokens from credentials file")
+              lastRefreshTimestamp = System.currentTimeMillis
+              info(s"Schedule the next fetch in $renewalInterval seconds")
+              tokenRenewExecutor.schedule(this, renewalInterval, TimeUnit.SECONDS)
+            }
+          } else {
+            info(s"Credentials file not found yet. Schedule the next fetch in $renewalInterval seconds")
+            tokenRenewExecutor.schedule(this, renewalInterval, TimeUnit.SECONDS)
+          }
+        } catch {
+          case e: Exception =>
+            val retrySeconds = Math.min(renewalInterval, 3600)
+            warn(s"Failed to renew tokens, will retry in $retrySeconds seconds", e)
+
+            tokenRenewExecutor.schedule(this, retrySeconds, TimeUnit.SECONDS)
+        }
+      }
+    }
+
+    info(s"Schedule the next fetch in ${renewalInterval + InitialDelayInSeconds} seconds")
+    tokenRenewExecutor.schedule(tokenRenewRunnable, renewalInterval + InitialDelayInSeconds, TimeUnit.SECONDS)
+  }
+
+  private def getCredentialsFromHDFS(fs: FileSystem, tokenPath: Path): Credentials = {
+    val stream = fs.open(tokenPath)
+    try {
+      val newCredentials = new Credentials()
+      newCredentials.readTokenStorageStream(stream)
+      newCredentials
+    } finally {
+      stream.close()
+    }
+  }
+
+  def stop(): Unit = {
+    tokenRenewExecutor.shutdown()
+  }
+}
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaYarnAppMasterLifecycle.scala b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaYarnAppMasterLifecycle.scala
new file mode 100644
index 000000000..27e0b1f5a
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaYarnAppMasterLifecycle.scala
@@ -0,0 +1,121 @@
+/*
+ * 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.samza.job.yarn
+
+import java.io.IOException
+import java.util
+import java.util.HashMap
+
+import org.apache.hadoop.yarn.api.records.{Container, ContainerId, FinalApplicationStatus}
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync
+import org.apache.hadoop.yarn.exceptions.{InvalidApplicationMasterRequestException, YarnException}
+import org.apache.samza.SamzaException
+import org.apache.samza.clustermanager.{SamzaApplicationState, SamzaResource}
+import SamzaApplicationState.SamzaAppStatus
+import org.apache.samza.util.Logging
+
+import scala.collection.JavaConverters._
+
+/**
+ * Responsible for managing the lifecycle of the Yarn application master. Mostly,
+ * this means registering and unregistering with the RM, and shutting down
+ * when the RM tells us to Reboot.
+ */
+//This class is used in the refactored code path as called by run-jc.sh
+class SamzaYarnAppMasterLifecycle(containerMem: Int, containerCpu: Int, samzaAppState: SamzaApplicationState, state: YarnAppState, amClient: AMRMClientAsync[ContainerRequest],
+  isApplicationMasterHighAvailabilityEnabled: Boolean) extends Logging {
+  var validResourceRequest = true
+  var shutdownMessage: String = null
+  var webApp: SamzaYarnAppMasterService = null
+  def onInit(): util.Set[ContainerId] = {
+    val host = state.nodeHost
+    val response = amClient.registerApplicationMaster(host, state.rpcUrl.getPort, "%s:%d" format (host, state.trackingUrl.getPort))
+
+    // validate that the YARN cluster can handle our container resource requirements
+    val maxCapability = response.getMaximumResourceCapability
+    val maxMem = maxCapability.getMemory
+    val maxCpu = maxCapability.getVirtualCores
+    val previousAttemptContainers = new util.HashSet[ContainerId]()
+    if (isApplicationMasterHighAvailabilityEnabled) {
+      val yarnIdToprocIdMap = new HashMap[String, String]()
+      samzaAppState.processorToExecutionId.asScala foreach { entry => yarnIdToprocIdMap.put(entry._2, entry._1) }
+      response.getContainersFromPreviousAttempts.asScala foreach { (ctr: Container) =>
+        val samzaProcId = yarnIdToprocIdMap.get(ctr.getId.toString)
+        info("Received container from previous attempt with samza processor id %s and yarn container id %s" format(samzaProcId, ctr.getId.toString))
+        samzaAppState.pendingProcessors.put(samzaProcId,
+          new SamzaResource(ctr.getResource.getVirtualCores, ctr.getResource.getMemory, ctr.getNodeId.getHost, ctr.getId.toString))
+        state.pendingProcessors.put(samzaProcId, new YarnContainer(ctr))
+        previousAttemptContainers.add(ctr.getId)
+      }
+    }
+    info("Got AM register response. The YARN RM supports container requests with max-mem: %s, max-cpu: %s" format (maxMem, maxCpu))
+
+    if (containerMem > maxMem || containerCpu > maxCpu) {
+      shutdownMessage = "The YARN cluster is unable to run your job due to unsatisfiable resource requirements. You asked for mem: %s, and cpu: %s." format (containerMem, containerCpu)
+      error(shutdownMessage)
+      validResourceRequest = false
+      samzaAppState.status = SamzaAppStatus.FAILED;
+      samzaAppState.jobHealthy.set(false)
+    }
+    previousAttemptContainers
+  }
+
+  def onReboot() {
+    throw new SamzaException("Received a reboot signal from the RM, so throwing an exception to reboot the AM.")
+  }
+
+  def onShutdown(samzaAppStatus: SamzaAppStatus) {
+    val yarnStatus: FinalApplicationStatus = getStatus(samzaAppStatus)
+    info("Shutting down SamzaAppStatus: " + samzaAppStatus + " yarn status: " + yarnStatus)
+    //The value of state.status is set to either SUCCEEDED or FAILED for errors we catch and handle - like container failures
+    //All other AM failures (errors in callbacks/connection failures after retries/token expirations) should not unregister the AM,
+    //allowing the RM to restart it (potentially on a different host)
+    if(samzaAppStatus != SamzaAppStatus.UNDEFINED) {
+      info("Unregistering AM from the RM.")
+      try {
+        amClient.unregisterApplicationMaster(yarnStatus, shutdownMessage, null)
+        info("Unregister complete.")
+      } catch {
+        case ex: InvalidApplicationMasterRequestException =>
+          // Once the NM dies, the corresponding app attempt ID is removed from the RM cache so that the RM can spin up a new AM and its containers.
+          // Hence, this throws InvalidApplicationMasterRequestException since that AM is unregistered with the RM already.
+          info("Removed application attempt from RM cache because the AM died. Unregister complete.")
+        case ex @ (_ : YarnException | _ : IOException) =>
+          error("Caught an exception while trying to unregister AM. Trying to stop other components.", ex)
+      }
+    }
+    else {
+      info("Not unregistering AM from the RM. This will enable RM retries")
+    }
+  }
+
+  def getStatus(samzaAppStatus: SamzaAppStatus): FinalApplicationStatus = {
+    if (samzaAppStatus == SamzaAppStatus.FAILED)
+       return FinalApplicationStatus.FAILED
+    if(samzaAppStatus == SamzaAppStatus.SUCCEEDED)
+       return FinalApplicationStatus.SUCCEEDED
+
+   return FinalApplicationStatus.UNDEFINED
+  }
+
+
+  def shouldShutdown = !validResourceRequest
+}
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaYarnAppMasterService.scala b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaYarnAppMasterService.scala
new file mode 100644
index 000000000..0f512ad64
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaYarnAppMasterService.scala
@@ -0,0 +1,103 @@
+/*
+ * 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.samza.job.yarn
+
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.samza.clustermanager.SamzaApplicationState
+import org.apache.samza.config.Config
+import org.apache.samza.coordinator.CoordinationConstants
+import org.apache.samza.coordinator.server.HttpServer
+import org.apache.samza.coordinator.stream.CoordinatorStreamWriter
+import org.apache.samza.coordinator.stream.messages.SetConfig
+import org.apache.samza.metrics.ReadableMetricsRegistry
+import org.apache.samza.util.Logging
+import org.apache.samza.webapp.{ApplicationMasterRestServlet, ApplicationMasterWebServlet, YarnContainerHeartbeatServlet}
+
+/**
+  * Samza's application master runs a very basic HTTP/JSON service to allow
+  * dashboards to check on the status of a job. SamzaAppMasterService starts
+  * up the web service when initialized.
+  */
+//This class is used in the refactored code path as called by run-jc.sh
+
+class SamzaYarnAppMasterService(config: Config, samzaAppState: SamzaApplicationState, state: YarnAppState, registry: ReadableMetricsRegistry, yarnConfiguration: YarnConfiguration) extends  Logging {
+  var rpcApp: HttpServer = null
+  var webApp: HttpServer = null
+  val SERVER_URL_OPT: String = CoordinationConstants.YARN_COORDINATOR_URL;
+  var securityManager: Option[SamzaAppMasterSecurityManager] = None
+
+  def onInit() {
+    // try starting the samza AM dashboard at a random rpc and tracking port
+    info("Starting webapp at a random rpc and tracking port")
+
+    rpcApp = new HttpServer(resourceBasePath = "scalate")
+
+    rpcApp.addServlet("/*", new ApplicationMasterRestServlet(config, samzaAppState, state, registry))
+    rpcApp.start
+
+    webApp = new HttpServer(resourceBasePath = "scalate")
+    webApp.addServlet("/*", new ApplicationMasterWebServlet(config, samzaAppState, state))
+    webApp.start
+
+    samzaAppState.jobModelManager.server.addServlet("/" + CoordinationConstants.YARN_CONTAINER_HEARTBEAT_SERVELET,
+      new YarnContainerHeartbeatServlet(state, registry))
+    samzaAppState.jobModelManager.start
+    state.rpcUrl = rpcApp.getUrl
+    state.trackingUrl = webApp.getUrl
+    state.coordinatorUrl = samzaAppState.jobModelManager.server.getUrl
+
+    //write server url to coordinator stream
+    val coordinatorStreamWriter: CoordinatorStreamWriter = new CoordinatorStreamWriter(config)
+    coordinatorStreamWriter.start()
+    coordinatorStreamWriter.sendMessage(SetConfig.TYPE, SERVER_URL_OPT, state.coordinatorUrl.toString)
+    coordinatorStreamWriter.stop()
+    debug("Sent server url message with value: %s " format state.coordinatorUrl.toString)
+
+    info("Webapp is started at (rpc %s, tracking %s, coordinator %s)" format(state.rpcUrl, state.trackingUrl, state.coordinatorUrl))
+
+    // start YarnSecurityManger for a secure cluster
+    if (UserGroupInformation.isSecurityEnabled) {
+      securityManager = Option {
+        val securityManager = new SamzaAppMasterSecurityManager(config, yarnConfiguration)
+        securityManager.start
+        securityManager
+      }
+    }
+
+  }
+
+  def onShutdown() {
+    if (rpcApp != null) {
+      rpcApp.stop
+    }
+
+    if (webApp != null) {
+      webApp.stop
+    }
+
+    samzaAppState.jobModelManager.stop
+
+    securityManager.map {
+      securityManager => securityManager.stop
+    }
+
+  }
+}
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaYarnSecurityManagerFactory.scala b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaYarnSecurityManagerFactory.scala
new file mode 100644
index 000000000..824e81d2d
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaYarnSecurityManagerFactory.scala
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.job.yarn
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.samza.config.Config
+import org.apache.samza.container.{SecurityManager, SecurityManagerFactory}
+
+class SamzaYarnSecurityManagerFactory extends SecurityManagerFactory {
+  override def getSecurityManager(config: Config): SecurityManager = {
+    val yarnConfig = new YarnConfiguration
+    new SamzaContainerSecurityManager(config, yarnConfig)
+  }
+}
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnAppMasterListener.scala b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnAppMasterListener.scala
new file mode 100644
index 000000000..6bf3046a1
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnAppMasterListener.scala
@@ -0,0 +1,68 @@
+/*
+ * 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.samza.job.yarn
+import org.apache.hadoop.yarn.api.records.Container
+import org.apache.hadoop.yarn.api.records.ContainerStatus
+
+/**
+ * Classes that wish to listen to callback events from YarnAppMaster must
+ * implement this trait.
+ */
+trait YarnAppMasterListener {
+  /**
+   * If true, YarnAppMaster will cease to poll the RM, and call onShutdown for
+   * all listeners.
+   */
+  def shouldShutdown: Boolean = false
+
+  /**
+   * Invoked by YarnAppMaster once per listener, before entering the RM polling
+   * event loop.
+   */
+  def onInit() {}
+
+  /**
+   * Invoked whenever the RM responds with a reboot request. Usually, reboots
+   * are triggered by the YARN RM when its state gets out of sync with the
+   * application master (usually the result of restarting the RM).
+   * YarnAppMaster does not actually restart anything. It is up to one or more
+   * listeners to trigger a failure, or shutdown.
+   */
+  def onReboot() {}
+
+  /**
+   * Signifies that the YarnAppMaster has exited the RM polling event loop, and
+   * is about to exit.
+   */
+  def onShutdown() {}
+
+  /**
+   * Whenever the RM allocates a container for the application master, this
+   * callback is invoked (once per container).
+   */
+  def onContainerAllocated(container: Container) {}
+
+  /**
+   * Whenever a container completes (either failure, or success), this callback
+   * will be invoked.
+   */
+  def onContainerCompleted(containerStatus: ContainerStatus) {}
+
+}
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnContainer.scala b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnContainer.scala
new file mode 100644
index 000000000..eefefb7a0
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnContainer.scala
@@ -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.samza.job.yarn
+
+import org.apache.hadoop.yarn.api.records.{Container, ContainerId, NodeId, Priority, Resource, Token}
+import org.joda.time.Period
+import org.joda.time.format.{DateTimeFormatter, ISODateTimeFormat, ISOPeriodFormat, PeriodFormatter}
+
+object YarnContainerUtils {
+  val dateFormater: DateTimeFormatter = ISODateTimeFormat.dateTime
+  val periodFormater: PeriodFormatter = ISOPeriodFormat.standard
+}
+
+/**
+ * YARN container information plus start time and up time
+ */
+class YarnContainer(container: Container) {
+  val id: ContainerId = container.getId
+  val nodeId: NodeId = container.getNodeId
+  val nodeHttpAddress: String = container.getNodeHttpAddress
+  val resource: Resource = container.getResource
+  val priority: Priority = container.getPriority
+  val containerToken: Token = container.getContainerToken
+  val startTime: Long = System.currentTimeMillis()
+  def startTimeStr(dtFormatter: Option[DateTimeFormatter] = None): String =
+    dtFormatter.getOrElse(YarnContainerUtils.dateFormater).print(startTime)
+  val upTime: Long = System.currentTimeMillis()
+  def upTimeStr(periodFormatter: Option[PeriodFormatter] = None): String =
+    periodFormatter.getOrElse(YarnContainerUtils.periodFormater).print(new Period(startTime, upTime))
+}
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala
new file mode 100644
index 000000000..16ff8a465
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala
@@ -0,0 +1,191 @@
+/*
+ * 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.samza.job.yarn
+
+import com.google.common.annotations.VisibleForTesting
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.api.ApplicationConstants
+import org.apache.hadoop.yarn.api.records.ApplicationId
+import org.apache.samza.SamzaException
+import org.apache.samza.config.{Config, JobConfig, ShellCommandConfig, YarnConfig}
+import org.apache.samza.job.ApplicationStatus.{SuccessfulFinish, UnsuccessfulFinish}
+import org.apache.samza.job.{ApplicationStatus, StreamJob}
+import org.apache.samza.serializers.model.SamzaObjectMapper
+import org.apache.samza.util.ScalaJavaUtil.JavaOptionals
+import org.apache.samza.util.{CoordinatorStreamUtil, Logging, Util}
+
+/**
+ * Starts the application manager
+ */
+class YarnJob(config: Config, hadoopConfig: Configuration) extends StreamJob with Logging {
+
+  val client = new ClientHelper(hadoopConfig)
+  var appId: Option[ApplicationId] = None
+  val yarnConfig = new YarnConfig(config)
+
+  def submit: YarnJob = {
+    try {
+      val jobConfig = new JobConfig(config)
+      val cmdExec = "./__package/" + jobConfig.getCoordinatorExecuteCommand
+      val environment = YarnJob.buildEnvironment(config, this.yarnConfig, jobConfig)
+
+      appId = client.submitApplication(
+        config,
+        List(
+          "export SAMZA_LOG_DIR=%s && ln -sfn %s logs && exec %s 1>logs/%s 2>logs/%s"
+            format (ApplicationConstants.LOG_DIR_EXPANSION_VAR, ApplicationConstants.LOG_DIR_EXPANSION_VAR,
+            cmdExec, ApplicationConstants.STDOUT, ApplicationConstants.STDERR)),
+        Some(environment),
+        Some("%s_%s" format(jobConfig.getName.get, jobConfig.getJobId))
+      )
+    } catch {
+      case e: Throwable =>
+        logger.error("Exception submitting yarn job.", e )
+        try {
+          // try to clean up. this may throw an exception depending on how far into launching the job we got.
+          // we don't want to mask the original problem by throwing this.
+          client.cleanupStagingDir
+        } catch {
+          case ce: Throwable => logger.warn("Exception cleaning Staging Directory after failed launch attempt.", ce)
+        } finally {
+          throw e
+        }
+    }
+
+    this
+  }
+
+  def waitForFinish(timeoutMs: Long): ApplicationStatus = {
+    val startTimeMs = System.currentTimeMillis()
+
+    while (System.currentTimeMillis() - startTimeMs < timeoutMs) {
+      Option(getStatus) match {
+        case Some(s) => if (SuccessfulFinish.equals(s) || UnsuccessfulFinish.equals(s))
+          client.cleanupStagingDir
+          return s
+        case None =>
+      }
+
+      Thread.sleep(1000)
+    }
+
+    getStatus
+  }
+
+  def waitForStatus(status: ApplicationStatus, timeoutMs: Long): ApplicationStatus = {
+    val startTimeMs = System.currentTimeMillis()
+
+    while (System.currentTimeMillis() - startTimeMs < timeoutMs) {
+      Option(getStatus) match {
+        case Some(s) => if (status.equals(s)) return status
+        case None => null
+      }
+
+      Thread.sleep(1000)
+    }
+
+    getStatus
+  }
+
+  def getStatus: ApplicationStatus = {
+    getAppId match {
+      case Some(appId) =>
+        logger.info("Getting status for applicationId %s" format appId)
+        client.status(appId).getOrElse(
+          throw new SamzaException("No status was determined for applicationId %s" format appId))
+      case None =>
+        logger.info("Unable to report status because no applicationId could be found.")
+        ApplicationStatus.SuccessfulFinish
+    }
+  }
+
+  def kill: YarnJob = {
+    // getAppId only returns one appID. Run multiple times to kill dupes (erroneous case)
+    getAppId match {
+      case Some(appId) =>
+        try {
+          logger.info("Killing applicationId {}", appId)
+          client.kill(appId)
+        } finally {
+          client.cleanupStagingDir
+        }
+      case None =>
+    }
+    this
+  }
+
+  private def getAppId: Option[ApplicationId] = {
+    appId match {
+      case Some(applicationId) =>
+       appId
+      case None =>
+        // Get by name
+        val jobConfig = new JobConfig(config)
+        JavaOptionals.toRichOptional(jobConfig.getName).toOption match {
+          case Some(jobName) =>
+            val applicationName = "%s_%s" format(jobName, jobConfig.getJobId)
+            logger.info("Fetching status from YARN for application name %s" format applicationName)
+            val applicationIds = client.getActiveApplicationIds(applicationName)
+
+            if (applicationIds.nonEmpty) {
+              // Only return latest one, because there should only be one.
+              logger.info("Matching active ids: " + applicationIds.sorted.reverse.toString())
+              applicationIds.sorted.reverse.headOption
+            } else {
+              // Couldn't find an active applicationID. Use one the latest finished ID.
+              val pastApplicationIds = client.getPreviousApplicationIds(applicationName)
+              // Don't log because there could be many, many previous app IDs for an application.
+              pastApplicationIds.sorted.reverse.headOption  // Get latest
+            }
+
+          case None =>
+            None
+        }
+    }
+  }
+}
+
+object YarnJob extends Logging {
+  /**
+    * Build the environment variable map for the job coordinator execution.
+    * Passing multiple separate config objects so that they can be reused for other logic.
+    */
+  @VisibleForTesting
+  private[yarn] def buildEnvironment(config: Config, yarnConfig: YarnConfig,
+    jobConfig: JobConfig): Map[String, String] = {
+    val envMapBuilder = Map.newBuilder[String, String]
+    if (jobConfig.getConfigLoaderFactory.isPresent) {
+      envMapBuilder += ShellCommandConfig.ENV_SUBMISSION_CONFIG ->
+        Util.envVarEscape(SamzaObjectMapper.getObjectMapper.writeValueAsString(config))
+    } else {
+      // TODO SAMZA-2432: Clean this up once SAMZA-2405 is completed when legacy flow is removed.
+      val coordinatorSystemConfig = CoordinatorStreamUtil.buildCoordinatorStreamConfig(config)
+      envMapBuilder += ShellCommandConfig.ENV_COORDINATOR_SYSTEM_CONFIG ->
+        Util.envVarEscape(SamzaObjectMapper.getObjectMapper.writeValueAsString(coordinatorSystemConfig))
+    }
+    envMapBuilder += ShellCommandConfig.ENV_JAVA_OPTS -> Util.envVarEscape(yarnConfig.getAmOpts)
+    Option.apply(yarnConfig.getAMJavaHome).foreach {
+      amJavaHome => envMapBuilder += ShellCommandConfig.ENV_JAVA_HOME -> amJavaHome
+    }
+    envMapBuilder += ShellCommandConfig.ENV_ADDITIONAL_CLASSPATH_DIR ->
+      Util.envVarEscape(config.get(ShellCommandConfig.ADDITIONAL_CLASSPATH_DIR, ""))
+    envMapBuilder.result()
+  }
+}
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnJobFactory.scala b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnJobFactory.scala
new file mode 100644
index 000000000..9433cdac4
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnJobFactory.scala
@@ -0,0 +1,55 @@
+/*
+ * 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.samza.job.yarn
+
+
+import org.apache.hadoop.hdfs.DistributedFileSystem
+import org.apache.samza.job.StreamJobFactory
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.samza.config.Config
+import org.apache.samza.util.hadoop.HttpFileSystem
+import org.apache.samza.util.Logging
+import scala.collection.JavaConverters._
+
+class YarnJobFactory extends StreamJobFactory with Logging {
+  def getJob(config: Config) = {
+    // TODO fix this. needed to support http package locations.
+    val hConfig = new YarnConfiguration
+    hConfig.set("fs.http.impl", classOf[HttpFileSystem].getName)
+    hConfig.set("fs.https.impl", classOf[HttpFileSystem].getName)
+    hConfig.set("fs.hdfs.impl", classOf[DistributedFileSystem].getName)
+    // pass along the RM config if has any
+    if (config.containsKey(YarnConfiguration.RM_ADDRESS)) {
+      hConfig.set(YarnConfiguration.RM_ADDRESS, config.get(YarnConfiguration.RM_ADDRESS, "0.0.0.0:8032"))
+    }
+
+    // Use the Samza job config "fs.<scheme>.impl" and "fs.<scheme>.impl.*" for YarnConfiguration
+    val fsImplConfig = new FileSystemImplConfig(config)
+    fsImplConfig.getSchemes.asScala.foreach(
+      scheme => {
+        fsImplConfig.getSchemeConfig(scheme).asScala.foreach {
+          case(confKey, confValue) => hConfig.set(confKey, confValue)
+        }
+      }
+    )
+
+    new YarnJob(config, hConfig)
+  }
+}
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnJobUtil.scala b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnJobUtil.scala
new file mode 100644
index 000000000..d7afa155a
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnJobUtil.scala
@@ -0,0 +1,109 @@
+/*
+ * 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.samza.job.yarn
+
+import java.io.IOException
+
+import org.apache.hadoop.fs.permission.FsPermission
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.yarn.api.records.ApplicationId
+import org.apache.samza.util.Logging
+
+object JobContext {
+  val STAGING_DIR = ".samzaStaging"
+}
+
+/**
+  * JobContext is used to store the meta information about the job running on Yarn.
+  *
+  * Optionally, one can set Application Id and Application Staging Directory to the job context object.
+  */
+class JobContext {
+  private var appId: ApplicationId = null
+  private var appStagingDir: Path = null
+
+  def getAppId: Option[ApplicationId] = {
+    Option(appId)
+  }
+
+  def getAppStagingDir: Option[Path] = {
+    Option(appStagingDir)
+  }
+
+  def setAppId(appId: ApplicationId) = {
+    this.appId = appId
+  }
+
+  def setAppStagingDir(appStagingDir: Path) = {
+    this.appStagingDir = appStagingDir
+  }
+
+  /**
+    * Return the staging directory path to the given application.
+    */
+  def defaultAppStagingDir: Option[String] = {
+    getAppId.map(JobContext.STAGING_DIR + Path.SEPARATOR + _.toString)
+  }
+}
+
+object YarnJobUtil extends Logging {
+  /**
+    * Create the staging directory for the application.
+    *
+    * @param jobContext
+    * @param fs
+    * @return the Option of the Path object to the staging directory
+    */
+  def createStagingDir(jobContext: JobContext, fs: FileSystem) = {
+    val defaultStagingDir = jobContext.defaultAppStagingDir.map(new Path(fs.getHomeDirectory, _))
+    val stagingDir = jobContext.getAppStagingDir match {
+      case appStagingDir: Some[Path] => appStagingDir
+      case None => defaultStagingDir
+  }
+    stagingDir.map {
+      appStagingDir =>
+        val appStagingDirPermission = FsPermission.createImmutable(Integer.parseInt("700", 8).toShort)
+        if (FileSystem.mkdirs(fs, appStagingDir, appStagingDirPermission)) {
+          appStagingDir
+        } else {
+          null
+        }
+    }
+  }
+
+  /**
+    * Clean up the application staging directory.
+    */
+  def cleanupStagingDir(jobContext: JobContext, fs: FileSystem): Unit = {
+    jobContext.getAppStagingDir match {
+      case Some(appStagingDir) => try {
+        if (fs.exists(appStagingDir)) {
+          info("Deleting staging directory " + appStagingDir)
+          fs.delete(appStagingDir, true)
+        }
+      } catch {
+        case ioe: IOException =>
+          warn("Failed to cleanup staging dir " + appStagingDir, ioe)
+      }
+      case None => info("No staging dir exists")
+    }
+
+  }
+}
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/util/hadoop/HttpFileSystem.scala b/samza-yarn3/src/main/scala/org/apache/samza/util/hadoop/HttpFileSystem.scala
new file mode 100644
index 000000000..fa65a4ed0
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/util/hadoop/HttpFileSystem.scala
@@ -0,0 +1,107 @@
+/*
+ * 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.samza.util.hadoop
+
+import java.io.IOException
+import java.net.URI
+
+import org.apache.commons.httpclient.methods.GetMethod
+import org.apache.commons.httpclient.HttpClient
+import org.apache.commons.httpclient.HttpStatus
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.permission.FsPermission
+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.util.Progressable
+
+import org.apache.samza.util.Logging
+
+class HttpFileSystem extends FileSystem with Logging {
+  val DEFAULT_BLOCK_SIZE = 4 * 1024
+  var uri: URI = null
+  var connectionTimeoutMs = 5 * 60 * 1000
+  var socketReadTimeoutMs = 5 * 60 * 1000
+
+  def setConnectionTimeoutMs(timeout: Int): Unit = connectionTimeoutMs = timeout
+
+  def setSocketReadTimeoutMs(timeout: Int): Unit = socketReadTimeoutMs = timeout
+
+  override def initialize(uri: URI, conf: Configuration) {
+    super.initialize(uri, conf)
+    info("init uri %s" format (uri))
+    this.uri = uri
+  }
+
+  override def getUri = uri
+
+  override def open(f: Path, bufferSize: Int): FSDataInputStream = {
+    info("open http file %s" format (f))
+    val client = new HttpClient
+    client.getHttpConnectionManager.getParams.setConnectionTimeout(connectionTimeoutMs)
+    client.getHttpConnectionManager.getParams.setSoTimeout(socketReadTimeoutMs)
+
+    val method = new GetMethod(f.toUri.toString)
+    val statusCode = client.executeMethod(method)
+
+    if (statusCode != HttpStatus.SC_OK) {
+      warn("got status code %d for uri %s" format (statusCode, uri))
+      throw new IOException("Bad status code returned by http server " + f + ": " + statusCode)
+    }
+
+    new FSDataInputStream(new HttpInputStream(method.getResponseBodyAsStream))
+  }
+
+  override def create(f: Path,
+    permission: FsPermission,
+    overwrite: Boolean,
+    bufferSize: Int,
+    replication: Short,
+    blockSize: Long,
+    progress: Progressable): FSDataOutputStream = null
+
+  override def append(f: Path, bufferSize: Int, progress: Progressable): FSDataOutputStream = null
+
+  override def rename(src: Path, dst: Path): Boolean = false
+
+  override def delete(f: Path, recursive: Boolean): Boolean = false
+
+  override def listStatus(f: Path): Array[FileStatus] = null
+
+  override def setWorkingDirectory(newDir: Path) {}
+
+  override def getWorkingDirectory(): Path = new Path("/")
+
+  override def mkdirs(f: Path, permission: FsPermission): Boolean = false
+
+  override def getFileStatus(f: Path): FileStatus = {
+    val length = -1
+    val isDir = false
+    val blockReplication = 1
+    val blockSize = DEFAULT_BLOCK_SIZE
+    val modTime = 0
+    val fs = new FileStatus(length, isDir, blockReplication, blockSize, modTime, f)
+    debug("file status for %s is %s" format (f, fs))
+    return fs
+  }
+}
+
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/util/hadoop/HttpInputStream.scala b/samza-yarn3/src/main/scala/org/apache/samza/util/hadoop/HttpInputStream.scala
new file mode 100644
index 000000000..a5458fb43
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/util/hadoop/HttpInputStream.scala
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.util.hadoop
+
+import java.io.IOException
+import java.io.InputStream
+
+import org.apache.hadoop.fs.FSInputStream
+
+class HttpInputStream(is: InputStream) extends FSInputStream {
+  val lock: AnyRef = new Object
+  var pos: Long = 0
+
+  override def seek(pos: Long) = throw new IOException("Seek not supported");
+
+  override def getPos: Long = pos
+
+  override def seekToNewSource(targetPos: Long): Boolean = throw new IOException("Seek not supported");
+
+  override def read: Int = {
+    lock.synchronized {
+      var byteRead = is.read()
+      if (byteRead >= 0) {
+        pos += 1
+      }
+      byteRead
+    }
+  }
+}
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala b/samza-yarn3/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala
new file mode 100644
index 000000000..b7bd9fd99
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala
@@ -0,0 +1,139 @@
+/*
+ * 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.samza.webapp
+
+import java.{lang, util}
+
+import org.apache.samza.clustermanager.SamzaApplicationState
+import org.scalatra._
+import scalate.ScalateSupport
+import org.apache.samza.config.Config
+import org.apache.samza.job.yarn.{ClientHelper, YarnAppState}
+import org.apache.samza.metrics._
+
+import scala.collection.JavaConverters._
+import com.fasterxml.jackson.databind.ObjectMapper
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import java.util.HashMap
+
+import org.apache.samza.serializers.model.SamzaObjectMapper
+
+object ApplicationMasterRestServlet {
+  def getMetrics(jsonMapper: ObjectMapper, metricsRegistry: ReadableMetricsRegistry) = {
+    val metricMap = new HashMap[String, util.Map[String, Object]]
+
+    // build metric map
+    metricsRegistry.getGroups.asScala.foreach(group => {
+      val groupMap = new HashMap[String, Object]
+
+      metricsRegistry.getGroup(group).asScala.foreach {
+        case (name, metric) =>
+          metric.visit(new MetricsVisitor() {
+            def counter(counter: Counter) =
+              groupMap.put(counter.getName, counter.getCount: lang.Long)
+
+            def gauge[T](gauge: Gauge[T]) =
+              groupMap.put(gauge.getName, gauge.getValue.asInstanceOf[Object])
+
+            def timer(timer: Timer) =
+              groupMap.put(timer.getName, timer.getSnapshot().getAverage: lang.Double)
+          })
+      }
+
+      metricMap.put(group, groupMap)
+    })
+
+    jsonMapper.writeValueAsString(metricMap)
+  }
+
+  def getTaskContext(jsonMapper: ObjectMapper, state: YarnAppState) = {
+    // sick of fighting with scala.. just using java map for now
+    val contextMap = new HashMap[String, Object]
+
+    contextMap.put("task-id", state.taskId: Integer)
+    contextMap.put("name", state.amContainerId.toString)
+
+    jsonMapper.writeValueAsString(contextMap)
+  }
+
+  def getAmState(jsonMapper: ObjectMapper, samzaAppState: SamzaApplicationState, state: YarnAppState) = {
+    val containers = new HashMap[String, util.HashMap[String, Object]]
+
+    state.runningProcessors.asScala.foreach {
+      case (containerId, container) =>
+        val yarnContainerId = container.id.toString
+        val containerMap = new HashMap[String, Object]
+        val taskModels = samzaAppState.jobModelManager.jobModel.getContainers.get(containerId).getTasks
+        containerMap.put("yarn-address", container.nodeHttpAddress)
+        containerMap.put("start-time", container.startTime.toString)
+        containerMap.put("up-time", container.upTime.toString)
+        containerMap.put("task-models", taskModels)
+        containerMap.put("container-id", containerId.toString)
+        containers.put(yarnContainerId, containerMap)
+    }
+
+    val status = Map[String, Object](
+      "app-attempt-id" -> state.appAttemptId.toString,
+      "container-id" -> state.amContainerId.toString,
+      "containers" -> containers,
+      "host" -> "%s:%s".format(state.nodeHost, state.rpcUrl.getPort))
+
+    jsonMapper.writeValueAsString(new HashMap[String, Object](status.asJava))
+  }
+
+  def getConfig(jsonMapper: ObjectMapper, samzaConfig: Config) = {
+    jsonMapper.writeValueAsString(new HashMap[String, Object](samzaConfig.sanitize))
+  }
+}
+
+/**
+  * Defines the Scalatra routes for the servlet.
+  */
+class ApplicationMasterRestServlet(samzaConfig: Config, samzaAppState: SamzaApplicationState, state: YarnAppState, registry: ReadableMetricsRegistry) extends ScalatraServlet with ScalateSupport {
+  val yarnConfig = new YarnConfiguration
+  val client = new ClientHelper(yarnConfig)
+  val jsonMapper = SamzaObjectMapper.getObjectMapper
+
+  before() {
+    contentType = "application/json"
+  }
+
+  get("/metrics") {
+    ApplicationMasterRestServlet.getMetrics(jsonMapper, registry)
+  }
+
+
+
+  get("/task-context") {
+    ApplicationMasterRestServlet.getTaskContext(jsonMapper, state)
+  }
+
+
+
+  get("/am") {
+    ApplicationMasterRestServlet.getAmState(jsonMapper, samzaAppState, state)
+  }
+
+
+
+  get("/config") {
+    ApplicationMasterRestServlet.getConfig(jsonMapper, samzaConfig)
+  }
+}
diff --git a/samza-yarn3/src/main/scala/org/apache/samza/webapp/ApplicationMasterWebServlet.scala b/samza-yarn3/src/main/scala/org/apache/samza/webapp/ApplicationMasterWebServlet.scala
new file mode 100644
index 000000000..2b62b96f8
--- /dev/null
+++ b/samza-yarn3/src/main/scala/org/apache/samza/webapp/ApplicationMasterWebServlet.scala
@@ -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.samza.webapp
+
+import org.apache.samza.clustermanager.SamzaApplicationState
+import org.scalatra._
+import scalate.ScalateSupport
+import org.apache.samza.job.yarn.YarnAppState
+import org.apache.samza.config.Config
+import scala.collection.JavaConverters._
+import scala.collection.immutable.TreeMap
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils
+
+class ApplicationMasterWebServlet(samzaConfig: Config, samzaAppState: SamzaApplicationState, state: YarnAppState) extends ScalatraServlet with ScalateSupport {
+  val yarnConfig = new YarnConfiguration
+
+  before() {
+    contentType = "text/html"
+  }
+
+  // Due to AMHA, the uptime and start time of containers (within state) from previous attempt is reset to the time the new AM becomes alive.
+  get("/") {
+    layoutTemplate("/WEB-INF/views/index.scaml",
+      "config" -> TreeMap(samzaConfig.sanitize.asScala.toMap.toArray: _*),
+      "state" -> state,
+      "samzaAppState" -> samzaAppState,
+      "rmHttpAddress" -> WebAppUtils.getRMWebAppURLWithScheme(yarnConfig))
+  }
+}
diff --git a/samza-yarn3/src/test/java/org/apache/samza/config/TestYarnConfig.java b/samza-yarn3/src/test/java/org/apache/samza/config/TestYarnConfig.java
new file mode 100644
index 000000000..ae4bf25f4
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/config/TestYarnConfig.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.config;
+
+import java.util.Collections;
+import org.apache.samza.SamzaException;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class TestYarnConfig {
+  @Test
+  public void testGetPackagePath() {
+    String packagePath = "http://some.package.path";
+    Config config = new MapConfig(Collections.singletonMap(YarnConfig.PACKAGE_PATH, packagePath));
+    assertEquals(packagePath, new YarnConfig(config).getPackagePath());
+  }
+
+  @Test(expected = SamzaException.class)
+  public void testGetPackagePathMissingConfig() {
+    new YarnConfig(new MapConfig()).getPackagePath();
+  }
+}
\ No newline at end of file
diff --git a/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestFileSystemImplConfig.java b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestFileSystemImplConfig.java
new file mode 100644
index 000000000..b906efd67
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestFileSystemImplConfig.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.job.yarn;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestFileSystemImplConfig {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void testFileSystemImplConfigSuccess() {
+    Map<String, String> configMap = new HashMap<>();
+
+    configMap.put("fs.http.impl", "org.apache.samza.HttpFileSystem");
+    configMap.put("fs.myscheme.impl", "org.apache.samza.MySchemeFileSystem");
+
+    Config conf = new MapConfig(configMap);
+
+    FileSystemImplConfig manager = new FileSystemImplConfig(conf);
+    assertEquals(2, manager.getSchemes().size());
+    assertEquals("http", manager.getSchemes().get(0));
+    assertEquals("myscheme", manager.getSchemes().get(1));
+  }
+
+  @Test
+  public void testNullConfig() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("config cannot be null");
+    FileSystemImplConfig manager = new FileSystemImplConfig(null);
+  }
+
+  @Test
+  public void testSchemeWithSubkeys() {
+    Map<String, String> configMap = new HashMap<>();
+    configMap.put("fs.http.impl", "org.apache.samza.HttpFileSystem");
+    configMap.put("fs.myscheme.impl", "org.apache.samza.MySchemeFileSystem");
+    configMap.put("fs.http.impl.key1", "val1");
+    configMap.put("fs.http.impl.key2", "val2");
+    Config conf = new MapConfig(configMap);
+
+    FileSystemImplConfig manager = new FileSystemImplConfig(conf);
+
+    Map<String, String> expectedFsHttpImplConfs = ImmutableMap.of(//Scheme with additional subkeys
+        "fs.http.impl", "org.apache.samza.HttpFileSystem",
+        "fs.http.impl.key1", "val1",
+        "fs.http.impl.key2", "val2"
+    );
+
+    Map<String, String> expectedFsMyschemeImplConfs = ImmutableMap.of(// Scheme without subkeys
+        "fs.myscheme.impl", "org.apache.samza.MySchemeFileSystem"
+    );
+
+    assertEquals(Arrays.asList("http", "myscheme"), manager.getSchemes());
+    assertEquals(expectedFsHttpImplConfs, manager.getSchemeConfig("http"));
+    assertEquals(expectedFsMyschemeImplConfs, manager.getSchemeConfig("myscheme"));
+  }
+}
diff --git a/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestLocalizerResourceConfig.java b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestLocalizerResourceConfig.java
new file mode 100644
index 000000000..6d0b8af8d
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestLocalizerResourceConfig.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.job.yarn;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.junit.Rule;
+import org.junit.rules.ExpectedException;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+
+public class TestLocalizerResourceConfig {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void testResourceConfigIncluded() {
+    Map<String, String> configMap = new HashMap<>();
+
+    configMap.put("yarn.resources.myResource1.path", "http://host1.com/readme");
+    configMap.put("yarn.resources.myResource1.local.name", "readme");
+    configMap.put("yarn.resources.myResource1.local.type", "file");
+    configMap.put("yarn.resources.myResource1.local.visibility", "public");
+
+    Config conf = new MapConfig(configMap);
+
+    LocalizerResourceConfig manager = new LocalizerResourceConfig(conf);
+    assertEquals(1, manager.getResourceNames().size());
+    assertEquals("myResource1", manager.getResourceNames().get(0));
+    assertEquals("readme", manager.getResourceLocalName("myResource1"));
+    assertEquals(LocalResourceType.FILE, manager.getResourceLocalType("myResource1"));
+    assertEquals(LocalResourceVisibility.PUBLIC, manager.getResourceLocalVisibility("myResource1"));
+  }
+
+  @Test
+  public void testResourcrConfigNotIncluded() {
+    Map<String, String> configMap = new HashMap<>();
+
+    configMap.put("otherconfig", "https://host2.com/not_included");
+    configMap.put("yarn.resources.myResource2.local.name", "notExisting");
+    configMap.put("yarn.resources.myResource2.local.type", "file");
+    configMap.put("yarn.resources.myResource2.local.visibility", "application");
+
+    Config conf = new MapConfig(configMap);
+
+    LocalizerResourceConfig manager = new LocalizerResourceConfig(conf);
+    assertEquals(0, manager.getResourceNames().size());
+  }
+
+  @Test
+  public void testNullConfig() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("config cannot be null");
+    LocalizerResourceConfig manager = new LocalizerResourceConfig(null);
+  }
+
+  @Test
+  public void testInvalidVisibility() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("No enum constant org.apache.hadoop.yarn.api.records.LocalResourceVisibility.INVALIDVISIBILITY");
+
+    Map<String, String> configMap = new HashMap<>();
+    configMap.put("yarn.resources.myResource1.path", "http://host1.com/readme");
+    configMap.put("yarn.resources.myResource1.local.name", "readme");
+    configMap.put("yarn.resources.myResource1.local.type", "file");
+    configMap.put("yarn.resources.myResource1.local.visibility", "invalidVisibility");
+    Config conf = new MapConfig(configMap);
+
+    LocalizerResourceConfig manager = new LocalizerResourceConfig(conf);
+    manager.getResourceLocalVisibility("myResource1");
+  }
+
+  @Test
+  public void testInvalidType() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("No enum constant org.apache.hadoop.yarn.api.records.LocalResourceType.INVALIDTYPE");
+
+    Map<String, String> configMap = new HashMap<>();
+    configMap.put("yarn.resources.myResource1.path", "http://host1.com/readme");
+    configMap.put("yarn.resources.myResource1.local.name", "readme");
+    configMap.put("yarn.resources.myResource1.local.type", "invalidType");
+    configMap.put("yarn.resources.myResource1.local.visibility", "application");
+    Config conf = new MapConfig(configMap);
+
+    LocalizerResourceConfig manager = new LocalizerResourceConfig(conf);
+    manager.getResourceLocalType("myResource1");
+  }
+
+  @Test
+  public void testInvalidPath() {
+    thrown.expect(LocalizerResourceException.class);
+    thrown.expectMessage("resource path is required but not defined in config for resource myResource1");
+    Map<String, String> configMap = new HashMap<>();
+    configMap.put("yarn.resources.myResource1.path", "");
+    configMap.put("yarn.resources.myResource1.local.name", "readme");
+    configMap.put("yarn.resources.myResource1.local.type", "invalidType");
+    configMap.put("yarn.resources.myResource1.local.visibility", "application");
+    Config conf = new MapConfig(configMap);
+
+    LocalizerResourceConfig manager = new LocalizerResourceConfig(conf);
+    manager.getResourcePath("myResource1");
+  }
+
+}
diff --git a/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestLocalizerResourceMapper.java b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestLocalizerResourceMapper.java
new file mode 100644
index 000000000..6ff8df18c
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestLocalizerResourceMapper.java
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.job.yarn;
+
+import java.util.HashMap;
+import java.util.Map;
+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.conf.YarnConfiguration;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.util.hadoop.HttpFileSystem;
+import org.junit.Rule;
+import org.junit.rules.ExpectedException;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+public class TestLocalizerResourceMapper {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void testResourceMapSuccess() {
+
+    Map<String, String> configMap = new HashMap<>();
+
+    configMap.put("yarn.resources.myResource1.path", "http://host1.com/readme");
+    configMap.put("yarn.resources.myResource1.local.name", "readme");
+    configMap.put("yarn.resources.myResource1.local.type", "file");
+    configMap.put("yarn.resources.myResource1.local.visibility", "public");
+
+    configMap.put("yarn.resources.myResource2.path", "https://host2.com/package");
+    configMap.put("yarn.resources.myResource2.local.name", "__package");
+    configMap.put("yarn.resources.myResource2.local.type", "archive");
+    configMap.put("yarn.resources.myResource2.local.visibility", "private");
+
+    configMap.put("yarn.resources.myResource3.path", "https://host3.com/csr");
+    configMap.put("yarn.resources.myResource3.local.name", "csr");
+    configMap.put("yarn.resources.myResource3.local.type", "file");
+    configMap.put("yarn.resources.myResource3.local.visibility", "application");
+
+    configMap.put("otherconfig", "https://host4.com/not_included");
+    configMap.put("yarn.resources.myResource4.local.name", "notExisting");
+    configMap.put("yarn.resources.myResource4.local.type", "file");
+    configMap.put("yarn.resources.myResource4.local.visibility", "application");
+
+    Config conf = new MapConfig(configMap);
+
+    YarnConfiguration yarnConfiguration = new YarnConfiguration();
+    yarnConfiguration.set("fs.http.impl", HttpFileSystem.class.getName());
+    yarnConfiguration.set("fs.https.impl", HttpFileSystem.class.getName());
+
+    LocalizerResourceMapper mapper = new LocalizerResourceMapper(new LocalizerResourceConfig(conf), yarnConfiguration);
+    Map<String, LocalResource> resourceMap = mapper.getResourceMap();
+
+    assertEquals("resourceMap has 3 resources", 3, resourceMap.size());
+
+    // resource1
+    assertEquals("host1.com", resourceMap.get("readme").getResource().getHost());
+    assertEquals(LocalResourceType.FILE, resourceMap.get("readme").getType());
+    assertEquals(LocalResourceVisibility.PUBLIC, resourceMap.get("readme").getVisibility());
+
+    // resource 2
+    assertEquals("host2.com", resourceMap.get("__package").getResource().getHost());
+    assertEquals(LocalResourceType.ARCHIVE, resourceMap.get("__package").getType());
+    assertEquals(LocalResourceVisibility.PRIVATE, resourceMap.get("__package").getVisibility());
+
+    // resource 3
+    assertEquals("host3.com", resourceMap.get("csr").getResource().getHost());
+    assertEquals(LocalResourceType.FILE, resourceMap.get("csr").getType());
+    assertEquals(LocalResourceVisibility.APPLICATION, resourceMap.get("csr").getVisibility());
+
+    // resource 4 should not exist
+    assertNull("Resource does not exist with the name myResource4", resourceMap.get("myResource4"));
+    assertNull("Resource does not exist with the defined config name notExisting for myResource4 either", resourceMap.get("notExisting"));
+  }
+
+  @Test
+  public void testResourceMapWithDefaultValues() {
+
+    Map<String, String> configMap = new HashMap<>();
+
+    configMap.put("yarn.resources.myResource1.path", "http://host1.com/readme");
+
+    Config conf = new MapConfig(configMap);
+
+    YarnConfiguration yarnConfiguration = new YarnConfiguration();
+    yarnConfiguration.set("fs.http.impl", HttpFileSystem.class.getName());
+
+    LocalizerResourceMapper mapper = new LocalizerResourceMapper(new LocalizerResourceConfig(conf), yarnConfiguration);
+    Map<String, LocalResource> resourceMap = mapper.getResourceMap();
+
+    assertNull("Resource does not exist with a name readme", resourceMap.get("readme"));
+    assertNotNull("Resource exists with a name myResource1", resourceMap.get("myResource1"));
+    assertEquals("host1.com", resourceMap.get("myResource1").getResource().getHost());
+    assertEquals(LocalResourceType.FILE, resourceMap.get("myResource1").getType());
+    assertEquals(LocalResourceVisibility.APPLICATION, resourceMap.get("myResource1").getVisibility());
+  }
+
+  @Test
+  public void testResourceMapWithFileStatusFailure() {
+    thrown.expect(LocalizerResourceException.class);
+    thrown.expectMessage("IO Exception when accessing the resource file status from the filesystem");
+
+    Map<String, String> configMap = new HashMap<>();
+    configMap.put("yarn.resources.myResource1.path", "unknown://host1.com/readme");
+    configMap.put("yarn.resources.myResource1.local.name", "readme");
+    configMap.put("yarn.resources.myResource1.local.type", "file");
+    configMap.put("yarn.resources.myResource1.local.visibility", "public");
+    Config conf = new MapConfig(configMap);
+
+    YarnConfiguration yarnConfiguration = new YarnConfiguration();
+    yarnConfiguration.set("fs.http.impl", HttpFileSystem.class.getName());
+    yarnConfiguration.set("fs.https.impl", HttpFileSystem.class.getName());
+    LocalizerResourceMapper mapper = new LocalizerResourceMapper(new LocalizerResourceConfig(conf), yarnConfiguration);
+  }
+
+  @Test
+  public void testResourceMapWithInvalidVisibilityFailure() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("No enum constant org.apache.hadoop.yarn.api.records.LocalResourceVisibility.INVALIDVISIBILITY");
+
+    Map<String, String> configMap = new HashMap<>();
+    configMap.put("yarn.resources.myResource1.path", "http://host1.com/readme");
+    configMap.put("yarn.resources.myResource1.local.name", "readme");
+    configMap.put("yarn.resources.myResource1.local.type", "file");
+    configMap.put("yarn.resources.myResource1.local.visibility", "invalidVisibility");
+    Config conf = new MapConfig(configMap);
+
+    YarnConfiguration yarnConfiguration = new YarnConfiguration();
+    yarnConfiguration.set("fs.http.impl", HttpFileSystem.class.getName());
+    yarnConfiguration.set("fs.https.impl", HttpFileSystem.class.getName());
+    LocalizerResourceMapper mapper = new LocalizerResourceMapper(new LocalizerResourceConfig(conf), yarnConfiguration);
+  }
+
+  @Test
+  public void testResourceMapWithInvalidTypeFailure() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("No enum constant org.apache.hadoop.yarn.api.records.LocalResourceType.INVALIDTYPE");
+
+    Map<String, String> configMap = new HashMap<>();
+    configMap.put("yarn.resources.myResource1.path", "http://host1.com/readme");
+    configMap.put("yarn.resources.myResource1.local.name", "readme");
+    configMap.put("yarn.resources.myResource1.local.type", "invalidType");
+    configMap.put("yarn.resources.myResource1.local.visibility", "public");
+    Config conf = new MapConfig(configMap);
+
+    YarnConfiguration yarnConfiguration = new YarnConfiguration();
+    yarnConfiguration.set("fs.http.impl", HttpFileSystem.class.getName());
+    yarnConfiguration.set("fs.https.impl", HttpFileSystem.class.getName());
+    LocalizerResourceMapper mapper = new LocalizerResourceMapper(new LocalizerResourceConfig(conf), yarnConfiguration);
+  }
+
+}
diff --git a/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnClusterResourceManager.java b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnClusterResourceManager.java
new file mode 100644
index 000000000..c7ae46f97
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnClusterResourceManager.java
@@ -0,0 +1,253 @@
+/*
+ * 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.samza.job.yarn;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
+import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.samza.clustermanager.ClusterResourceManager;
+import org.apache.samza.clustermanager.SamzaApplicationState;
+import org.apache.samza.clustermanager.SamzaResource;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.MapConfig;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.*;
+
+
+public class TestYarnClusterResourceManager {
+
+  private YarnConfiguration yarnConfiguration;
+  private Config config;
+  private SamzaAppMasterMetrics metrics;
+  private AMRMClientAsync asyncClient;
+  private SamzaYarnAppMasterLifecycle lifecycle;
+  private SamzaYarnAppMasterService service;
+  private NMClientAsync asyncNMClient;
+  private ClusterResourceManager.Callback callback;
+  private YarnAppState yarnAppState;
+
+  @Before
+  public void setup() {
+    yarnConfiguration = mock(YarnConfiguration.class);
+    config = mock(Config.class);
+    metrics = mock(SamzaAppMasterMetrics.class);
+    asyncClient = mock(AMRMClientAsync.class);
+    lifecycle = mock(SamzaYarnAppMasterLifecycle.class);
+    service = mock(SamzaYarnAppMasterService.class);
+    asyncNMClient = mock(NMClientAsync.class);
+    callback = mock(ClusterResourceManager.Callback.class);
+    yarnAppState = new YarnAppState(0, mock(ContainerId.class), "host", 8080, 8081);
+  }
+
+  @Test
+  public void testErrorInStartContainerShouldUpdateState() {
+    // create mocks
+    final int samzaContainerId = 1;
+
+    // start the cluster manager
+    YarnClusterResourceManager yarnClusterResourceManager =
+        new YarnClusterResourceManager(asyncClient, asyncNMClient, callback, yarnAppState, lifecycle, service, metrics,
+            yarnConfiguration, config);
+
+    yarnAppState.pendingProcessors.put(String.valueOf(samzaContainerId), new YarnContainer(Container.newInstance(
+        ContainerId.newContainerId(ApplicationAttemptId.newInstance(ApplicationId.newInstance(10000L, 1), 1), 1),
+        NodeId.newInstance("host1", 8088), "http://host1", Resource.newInstance(1024, 1), Priority.newInstance(1),
+        Token.newInstance("id".getBytes(), "read", "password".getBytes(), "service"))));
+
+    yarnClusterResourceManager.start();
+    assertEquals(1, yarnAppState.pendingProcessors.size());
+
+    yarnClusterResourceManager.onStartContainerError(ContainerId.newContainerId(ApplicationAttemptId.newInstance(ApplicationId.newInstance(10000L, 1), 1), 1),
+        new Exception());
+
+    assertEquals(0, yarnAppState.pendingProcessors.size());
+    verify(callback, times(1)).onStreamProcessorLaunchFailure(anyObject(), any(Exception.class));
+  }
+
+  @Test
+  public void testAllocatedResourceExpiryForYarn() {
+    // start the cluster manager
+    YarnClusterResourceManager yarnClusterResourceManager =
+        new YarnClusterResourceManager(asyncClient, asyncNMClient, callback, yarnAppState, lifecycle, service, metrics,
+            yarnConfiguration, config);
+
+    SamzaResource allocatedResource = mock(SamzaResource.class);
+    when(allocatedResource.getTimestamp()).thenReturn(System.currentTimeMillis() - Duration.ofMinutes(10).toMillis());
+
+    Assert.assertTrue(yarnClusterResourceManager.isResourceExpired(allocatedResource));
+  }
+
+  @Test
+  public void testAMShutdownOnRMCallback() throws IOException, YarnException {
+    // create mocks
+    SamzaYarnAppMasterLifecycle lifecycle = Mockito.spy(new SamzaYarnAppMasterLifecycle(512, 2, mock(SamzaApplicationState.class), yarnAppState, asyncClient, false));
+
+    // start the cluster manager
+    YarnClusterResourceManager yarnClusterResourceManager =
+        new YarnClusterResourceManager(asyncClient, asyncNMClient, callback, yarnAppState, lifecycle, service, metrics,
+            yarnConfiguration, config);
+
+    yarnClusterResourceManager.onShutdownRequest();
+
+    verify(lifecycle, times(1)).onShutdown(SamzaApplicationState.SamzaAppStatus.FAILED);
+    verify(asyncClient, times(1)).unregisterApplicationMaster(FinalApplicationStatus.FAILED, null, null);
+    verify(asyncClient, times(1)).stop();
+    verify(asyncNMClient, times(1)).stop();
+    verify(service, times(1)).onShutdown();
+    verify(metrics, times(1)).stop();
+  }
+
+  @Test
+  public void testAMShutdownThrowingExceptionOnRMCallback() throws IOException, YarnException {
+    // create mocks
+    SamzaYarnAppMasterLifecycle lifecycle = Mockito.spy(new SamzaYarnAppMasterLifecycle(512, 2, mock(SamzaApplicationState.class), yarnAppState, asyncClient, false));
+
+    doThrow(InvalidApplicationMasterRequestException.class).when(asyncClient).unregisterApplicationMaster(FinalApplicationStatus.FAILED, null, null);
+
+    // start the cluster manager
+    YarnClusterResourceManager yarnClusterResourceManager =
+        new YarnClusterResourceManager(asyncClient, asyncNMClient, callback, yarnAppState, lifecycle, service, metrics,
+            yarnConfiguration, config);
+
+    yarnClusterResourceManager.onShutdownRequest();
+
+    verify(lifecycle, times(1)).onShutdown(SamzaApplicationState.SamzaAppStatus.FAILED);
+    verify(asyncClient, times(1)).unregisterApplicationMaster(FinalApplicationStatus.FAILED, null, null);
+    verify(asyncClient, times(1)).stop();
+    verify(asyncNMClient, times(1)).stop();
+    verify(service, times(1)).onShutdown();
+    verify(metrics, times(1)).stop();
+  }
+
+  @Test
+  public void testAMHACallbackInvokedForPreviousAttemptContainers() {
+    String previousAttemptContainerId = "0";
+    String previousAttemptYarnContainerId = "container_1607304997422_0008_02_000002";
+    // create mocks
+    YarnAppState yarnAppState = Mockito.spy(new YarnAppState(0, mock(ContainerId.class), "host", 8080, 8081));
+
+    ContainerId containerId = mock(ContainerId.class);
+    when(containerId.toString()).thenReturn(previousAttemptYarnContainerId);
+
+    YarnContainer yarnContainer = mock(YarnContainer.class);
+    Resource resource = mock(Resource.class);
+    when(resource.getMemory()).thenReturn(1024);
+    Mockito.when(resource.getVirtualCores()).thenReturn(1);
+    Mockito.when(yarnContainer.resource()).thenReturn(resource);
+    Mockito.when(yarnContainer.id()).thenReturn(containerId);
+    NodeId nodeId = mock(NodeId.class);
+    when(nodeId.getHost()).thenReturn("host");
+    when(yarnContainer.nodeId()).thenReturn(nodeId);
+
+    yarnAppState.pendingProcessors.put(previousAttemptContainerId, yarnContainer);
+
+    Set<ContainerId> previousAttemptContainers = new HashSet<>();
+    previousAttemptContainers.add(containerId);
+    when(lifecycle.onInit()).thenReturn(previousAttemptContainers);
+
+    Map<String, String> configMap = new HashMap<>();
+    configMap.put(JobConfig.YARN_AM_HIGH_AVAILABILITY_ENABLED, "true");
+    Config config = new MapConfig(configMap);
+
+    // start the cluster manager
+    YarnClusterResourceManager yarnClusterResourceManager =
+        new YarnClusterResourceManager(asyncClient, asyncNMClient, callback, yarnAppState, lifecycle, service, metrics,
+            yarnConfiguration, config);
+
+    yarnClusterResourceManager.start();
+    verify(lifecycle).onInit();
+    ArgumentCaptor<SamzaResource> samzaResourceArgumentCaptor = ArgumentCaptor.forClass(SamzaResource.class);
+    verify(callback).onStreamProcessorLaunchSuccess(samzaResourceArgumentCaptor.capture());
+    ArgumentCaptor<Integer> containerFromPreviousAttemptCaptor = ArgumentCaptor.forClass(Integer.class);
+    verify(metrics).setContainersFromPreviousAttempts(containerFromPreviousAttemptCaptor.capture());
+    SamzaResource samzaResource = samzaResourceArgumentCaptor.getValue();
+    assertEquals(previousAttemptYarnContainerId, samzaResource.getContainerId());
+    assertEquals(1, containerFromPreviousAttemptCaptor.getValue().intValue());
+  }
+
+  @Test
+  public void testStopStreamProcessorForContainerFromPreviousAttempt() {
+    String containerId = "Yarn_Container_id_0";
+    String processorId = "Container_id_0";
+    YarnContainer runningYarnContainer = mock(YarnContainer.class);
+    ContainerId previousRunningContainerId = mock(ContainerId.class);
+    YarnAppState yarnAppState = Mockito.spy(new YarnAppState(0, mock(ContainerId.class), "host", 8080, 8081));
+
+    yarnAppState.runningProcessors.put(processorId, runningYarnContainer);
+    when(runningYarnContainer.id()).thenReturn(previousRunningContainerId);
+    when(previousRunningContainerId.toString()).thenReturn(containerId);
+
+    YarnClusterResourceManager yarnClusterResourceManager =
+        new YarnClusterResourceManager(asyncClient, asyncNMClient, callback, yarnAppState, lifecycle, service, metrics,
+            yarnConfiguration, config);
+
+    SamzaResource containerResourceFromPreviousRun = mock(SamzaResource.class);
+    when(containerResourceFromPreviousRun.getContainerId()).thenReturn(containerId);
+
+    yarnClusterResourceManager.stopStreamProcessor(containerResourceFromPreviousRun);
+    verify(asyncClient, times(1)).releaseAssignedContainer(previousRunningContainerId);
+  }
+
+  @Test
+  public void testStopStreamProcessorForContainerStartedInCurrentLifecycle() {
+    YarnClusterResourceManager yarnClusterResourceManager =
+        new YarnClusterResourceManager(asyncClient, asyncNMClient, callback, yarnAppState, lifecycle, service, metrics,
+            yarnConfiguration, config);
+
+    SamzaResource allocatedContainerResource = mock(SamzaResource.class);
+    Container runningContainer = mock(Container.class);
+    ContainerId runningContainerId = mock(ContainerId.class);
+    NodeId runningNodeId = mock(NodeId.class);
+
+    when(runningContainer.getId()).thenReturn(runningContainerId);
+    when(runningContainer.getNodeId()).thenReturn(runningNodeId);
+
+    yarnClusterResourceManager.getAllocatedResources().put(allocatedContainerResource, runningContainer);
+    yarnClusterResourceManager.stopStreamProcessor(allocatedContainerResource);
+
+    verify(asyncNMClient, times(1)).stopContainerAsync(runningContainerId, runningNodeId);
+  }
+}
\ No newline at end of file
diff --git a/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java
new file mode 100644
index 000000000..76d6d8842
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java
@@ -0,0 +1,198 @@
+/*
+ * 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.samza.job.yarn;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Multimap;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.samza.clustermanager.FaultDomain;
+import org.apache.samza.clustermanager.FaultDomainType;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestYarnFaultDomainManager {
+  private final Multimap<String, FaultDomain> hostToRackMap = HashMultimap.create();
+  private final String hostName1 = "host1";
+  private final String hostName2 = "host2";
+  private final String hostName3 = "host3";
+  private final String hostName4 = "host4";
+  private final String hostName5 = "host5";
+  private final String hostName6 = "host6";
+  private final String rackName1 = "rack1";
+  private final String rackName2 = "rack2";
+  private final String rackName3 = "rack3";
+
+  private final NodeReport nodeReport1 = createNodeReport(hostName1, 1, NodeState.RUNNING, "httpAddress1",
+          rackName1, 1, 1, 2, 1, 2,
+          "", 60L);
+  private final NodeReport nodeReport2 = createNodeReport(hostName2, 1, NodeState.RUNNING, "httpAddress2",
+          rackName2, 1, 1, 2, 1, 2,
+          "", 60L);
+  private final NodeReport nodeReport3 = createNodeReport(hostName3, 1, NodeState.RUNNING, "httpAddress3",
+          rackName1, 1, 1, 2, 1, 2,
+          "", 60L);
+  private final NodeReport nodeReport4 = createNodeReport(hostName4, 1, NodeState.RUNNING, "httpAddress4",
+          rackName2, 1, 1, 2, 1, 2,
+          "", 60L);
+  private final NodeReport nodeReport5 = createNodeReport(hostName5, 1, NodeState.RUNNING, "httpAddress5",
+          rackName3, 1, 1, 2, 1, 2,
+          "", 60L);
+  private final NodeReport nodeReport6 = createNodeReport(hostName6, 1, NodeState.RUNNING, "httpAddress6",
+          rackName1, 1, 1, 2, 1, 2,
+          "", 60L);
+
+  @Mock
+  YarnClientImpl yarnClient;
+  @Mock
+  ReadableMetricsRegistry mockMetricsRegistry;
+  @Mock
+  Counter mockCounter;
+
+  @Before
+  public void setup() {
+    FaultDomain rack1 = new FaultDomain(FaultDomainType.RACK, rackName1);
+    FaultDomain rack2 = new FaultDomain(FaultDomainType.RACK, rackName2);
+    FaultDomain rack3 = new FaultDomain(FaultDomainType.RACK, rackName3);
+    hostToRackMap.put(hostName1, rack1);
+    hostToRackMap.put(hostName2, rack2);
+    hostToRackMap.put(hostName3, rack1);
+    hostToRackMap.put(hostName4, rack2);
+    hostToRackMap.put(hostName5, rack3);
+
+    when(mockMetricsRegistry.newCounter(anyString(), anyString())).thenReturn(mockCounter);
+  }
+
+  @Test
+  public void testGetFaultDomainOfHostWhichExistsInCache() {
+    YarnFaultDomainManager yarnFaultDomainManager = new YarnFaultDomainManager(mockMetricsRegistry, yarnClient, hostToRackMap);
+
+    Set<FaultDomain> expectedFaultDomainSet = new HashSet<>();
+    expectedFaultDomainSet.add(new FaultDomain(FaultDomainType.RACK, rackName1));
+
+    Set<FaultDomain> actualFaultDomainSet = yarnFaultDomainManager.getFaultDomainsForHost(hostName3);
+
+    assertNotNull(actualFaultDomainSet);
+    assertEquals(expectedFaultDomainSet.iterator().next(), actualFaultDomainSet.iterator().next());
+    verify(mockCounter, times(0)).inc();
+  }
+
+  @Test
+  public void testGetFaultDomainOfHostWhichDoesNotExistInCache() throws IOException, YarnException {
+    YarnFaultDomainManager yarnFaultDomainManager = new YarnFaultDomainManager(mockMetricsRegistry, yarnClient, hostToRackMap);
+
+    Set<FaultDomain> expectedFaultDomainSet = new HashSet<>();
+    expectedFaultDomainSet.add(new FaultDomain(FaultDomainType.RACK, rackName1));
+
+    List<NodeReport> updatedNodeReport = ImmutableList.of(nodeReport1, nodeReport2, nodeReport3, nodeReport4, nodeReport5, nodeReport6);
+    when(yarnClient.getNodeReports(NodeState.RUNNING)).thenReturn(updatedNodeReport);
+
+    Set<FaultDomain> actualFaultDomainSet = yarnFaultDomainManager.getFaultDomainsForHost(hostName6);
+
+    assertNotNull(actualFaultDomainSet);
+    assertEquals(expectedFaultDomainSet.iterator().next(), actualFaultDomainSet.iterator().next());
+    verify(mockCounter, times(1)).inc();
+  }
+
+  @Test
+  public void testHasSameFaultDomainsWhenTrue() {
+    YarnFaultDomainManager yarnFaultDomainManager = new YarnFaultDomainManager(mockMetricsRegistry, yarnClient, hostToRackMap);
+
+    boolean result = yarnFaultDomainManager.hasSameFaultDomains(hostName1, hostName3);
+
+    assertTrue(result);
+  }
+
+  @Test
+  public void testHasSameFaultDomainsWhenFalse() {
+    YarnFaultDomainManager yarnFaultDomainManager = new YarnFaultDomainManager(mockMetricsRegistry, yarnClient, hostToRackMap);
+
+    boolean result = yarnFaultDomainManager.hasSameFaultDomains(hostName1, hostName2);
+
+    assertFalse(result);
+  }
+
+  @Test
+  public void testHasSameFaultDomainsWhenHostDoesNotExistInCache() throws IOException, YarnException {
+    YarnFaultDomainManager yarnFaultDomainManager = new YarnFaultDomainManager(mockMetricsRegistry, yarnClient, hostToRackMap);
+
+    List<NodeReport> updatedNodeReport = ImmutableList.of(nodeReport1, nodeReport2, nodeReport3, nodeReport4, nodeReport5, nodeReport6);
+    when(yarnClient.getNodeReports(NodeState.RUNNING)).thenReturn(updatedNodeReport);
+
+    boolean result = yarnFaultDomainManager.hasSameFaultDomains(hostName1, hostName6);
+
+    assertTrue(result);
+  }
+
+  @Test
+  public void testComputeHostToFaultDomainMap() throws IOException, YarnException {
+    YarnFaultDomainManager yarnFaultDomainManager = new YarnFaultDomainManager(mockMetricsRegistry, yarnClient, null);
+
+    List<NodeReport> nodeReport = ImmutableList.of(nodeReport1, nodeReport2, nodeReport3, nodeReport4, nodeReport5);
+    when(yarnClient.getNodeReports(NodeState.RUNNING)).thenReturn(nodeReport);
+
+    Multimap<String, FaultDomain> hostToRackMap = yarnFaultDomainManager.computeHostToFaultDomainMap();
+
+    assertEquals(this.hostToRackMap.size(), hostToRackMap.size());
+    assertEquals(this.hostToRackMap.keySet(), hostToRackMap.keySet());
+    Iterator<FaultDomain> expectedValues = this.hostToRackMap.values().iterator();
+    Iterator<FaultDomain> computedValues = hostToRackMap.values().iterator();
+    expectedValues.forEachRemaining(expectedRack -> assertFaultDomainEquals(expectedRack, computedValues.next()));
+  }
+
+  private void assertFaultDomainEquals(FaultDomain faultDomain1, FaultDomain faultDomain2) {
+    assertEquals(faultDomain1.getType(), faultDomain2.getType());
+    assertEquals(faultDomain1.getId(), faultDomain2.getId());
+  }
+
+  private NodeReport createNodeReport(String host, int port, NodeState nodeState, String httpAddress, String rackName,
+                                      int memoryUsed, int vcoresUsed, int totalMemory, int totalVcores, int numContainers,
+                                      String healthReport, long lastHealthReportTime) {
+    return NodeReport.newInstance(
+            NodeId.newInstance(host, port),
+            nodeState,
+            httpAddress,
+            rackName,
+            Resource.newInstance(memoryUsed, vcoresUsed),
+            Resource.newInstance(totalMemory, totalVcores),
+            numContainers,
+            healthReport,
+            lastHealthReportTime
+    );
+  }
+}
diff --git a/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnJob.java b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnJob.java
new file mode 100644
index 000000000..4e2c4a7dd
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnJob.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.job.yarn;
+
+import java.io.IOException;
+import java.util.Map;
+import com.google.common.collect.ImmutableMap;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.config.ShellCommandConfig;
+import org.apache.samza.config.YarnConfig;
+import org.apache.samza.serializers.model.SamzaObjectMapper;
+import org.apache.samza.util.CoordinatorStreamUtil;
+import org.apache.samza.util.Util;
+import org.junit.Test;
+import scala.collection.JavaConverters;
+
+import static org.junit.Assert.assertEquals;
+
+
+public class TestYarnJob {
+  @Test
+  public void testBuildEnvironment() throws IOException {
+    String amJvmOptions = "-Xmx1g -Dconfig.key='config value'";
+    Config config = new MapConfig(new ImmutableMap.Builder<String, String>()
+        .put(JobConfig.JOB_NAME, "jobName")
+        .put(JobConfig.JOB_ID, "jobId")
+        .put(JobConfig.JOB_COORDINATOR_SYSTEM, "jobCoordinatorSystem")
+        .put(YarnConfig.AM_JVM_OPTIONS, amJvmOptions) // needs escaping
+        .build());
+    String expectedCoordinatorStreamConfigStringValue = Util.envVarEscape(SamzaObjectMapper.getObjectMapper()
+        .writeValueAsString(CoordinatorStreamUtil.buildCoordinatorStreamConfig(config)));
+    Map<String, String> expected = ImmutableMap.of(
+        ShellCommandConfig.ENV_COORDINATOR_SYSTEM_CONFIG, expectedCoordinatorStreamConfigStringValue,
+        ShellCommandConfig.ENV_JAVA_OPTS, Util.envVarEscape(amJvmOptions),
+        ShellCommandConfig.ENV_ADDITIONAL_CLASSPATH_DIR, "");
+    assertEquals(expected, JavaConverters.mapAsJavaMapConverter(
+        YarnJob$.MODULE$.buildEnvironment(config, new YarnConfig(config), new JobConfig(config))).asJava());
+  }
+
+  @Test
+  public void testBuildEnvironmentWithAMJavaHome() throws IOException {
+    Config config = new MapConfig(new ImmutableMap.Builder<String, String>()
+        .put(JobConfig.JOB_NAME, "jobName")
+        .put(JobConfig.JOB_ID, "jobId")
+        .put(JobConfig.JOB_COORDINATOR_SYSTEM, "jobCoordinatorSystem")
+        .put(YarnConfig.AM_JVM_OPTIONS, "")
+        .put(YarnConfig.AM_JAVA_HOME, "/some/path/to/java/home")
+        .build());
+    String expectedCoordinatorStreamConfigStringValue = Util.envVarEscape(SamzaObjectMapper.getObjectMapper()
+        .writeValueAsString(CoordinatorStreamUtil.buildCoordinatorStreamConfig(config)));
+    Map<String, String> expected = ImmutableMap.of(
+        ShellCommandConfig.ENV_COORDINATOR_SYSTEM_CONFIG, expectedCoordinatorStreamConfigStringValue,
+        ShellCommandConfig.ENV_JAVA_OPTS, "",
+        ShellCommandConfig.ENV_JAVA_HOME, "/some/path/to/java/home",
+        ShellCommandConfig.ENV_ADDITIONAL_CLASSPATH_DIR, "");
+    assertEquals(expected, JavaConverters.mapAsJavaMapConverter(
+        YarnJob$.MODULE$.buildEnvironment(config, new YarnConfig(config), new JobConfig(config))).asJava());
+  }
+
+  @Test
+  public void testBuildJobSubmissionEnvironment() throws IOException {
+    Config config = new MapConfig(new ImmutableMap.Builder<String, String>()
+        .put(JobConfig.JOB_NAME, "jobName")
+        .put(JobConfig.JOB_ID, "jobId")
+        .put(JobConfig.CONFIG_LOADER_FACTORY, "org.apache.samza.config.loaders.PropertiesConfigLoaderFactory")
+        .put(YarnConfig.AM_JVM_OPTIONS, "")
+        .build());
+    String expectedSubmissionConfig = Util.envVarEscape(SamzaObjectMapper.getObjectMapper()
+        .writeValueAsString(config));
+    Map<String, String> expected = ImmutableMap.of(
+        ShellCommandConfig.ENV_SUBMISSION_CONFIG, expectedSubmissionConfig,
+        ShellCommandConfig.ENV_JAVA_OPTS, "",
+        ShellCommandConfig.ENV_ADDITIONAL_CLASSPATH_DIR, "");
+    assertEquals(expected, JavaConverters.mapAsJavaMapConverter(
+        YarnJob$.MODULE$.buildEnvironment(config, new YarnConfig(config), new JobConfig(config))).asJava());
+  }
+
+  @Test
+  public void testBuildJobWithAdditionalClassPath() throws IOException {
+    Config config = new MapConfig(new ImmutableMap.Builder<String, String>()
+        .put(JobConfig.JOB_NAME, "jobName")
+        .put(JobConfig.JOB_ID, "jobId")
+        .put(JobConfig.CONFIG_LOADER_FACTORY, "org.apache.samza.config.loaders.PropertiesConfigLoaderFactory")
+        .put(YarnConfig.AM_JVM_OPTIONS, "")
+        .put(ShellCommandConfig.ADDITIONAL_CLASSPATH_DIR, "./sqlapp/lib/*")
+        .build());
+    String expectedSubmissionConfig = Util.envVarEscape(SamzaObjectMapper.getObjectMapper()
+        .writeValueAsString(config));
+    Map<String, String> expected = ImmutableMap.of(
+        ShellCommandConfig.ENV_SUBMISSION_CONFIG, expectedSubmissionConfig,
+        ShellCommandConfig.ENV_JAVA_OPTS, "",
+        ShellCommandConfig.ENV_ADDITIONAL_CLASSPATH_DIR, "./sqlapp/lib/*");
+    assertEquals(expected, JavaConverters.mapAsJavaMapConverter(
+        YarnJob$.MODULE$.buildEnvironment(config, new YarnConfig(config), new JobConfig(config))).asJava());
+  }
+}
\ No newline at end of file
diff --git a/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnJobFactory.java b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnJobFactory.java
new file mode 100644
index 000000000..9ba623235
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnJobFactory.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.samza.job.yarn;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.util.hadoop.HttpFileSystem;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+
+public class TestYarnJobFactory {
+  @Test
+  public void testGetJobWithDefaultFsImpl() {
+    YarnJobFactory jobFactory = new YarnJobFactory();
+    YarnJob yarnJob = jobFactory.getJob(new MapConfig());
+    Configuration hConfig = yarnJob.client().yarnClient().getConfig();
+    assertEquals(HttpFileSystem.class.getName(), hConfig.get("fs.http.impl"));
+    assertEquals(HttpFileSystem.class.getName(), hConfig.get("fs.https.impl"));
+  }
+
+  @Test
+  public void  testGetJobWithFsImplOverride() {
+    YarnJobFactory jobFactory = new YarnJobFactory();
+    YarnJob yarnJob = jobFactory.getJob(new MapConfig(ImmutableMap.of(
+        "fs.http.impl", "org.apache.myHttp",
+        "fs.myscheme.impl", "org.apache.myScheme")));
+    Configuration hConfig = yarnJob.client().yarnClient().getConfig();
+    assertEquals("org.apache.myHttp", hConfig.get("fs.http.impl"));
+    assertEquals("org.apache.myScheme", hConfig.get("fs.myscheme.impl"));
+  }
+
+  @Test
+  public void  testGetJobWithFsImplSubkeys() {
+    YarnJobFactory jobFactory = new YarnJobFactory();
+    YarnJob yarnJob = jobFactory.getJob(new MapConfig(ImmutableMap.of(
+        "fs.myscheme.impl", "org.apache.myScheme",
+        "fs.myscheme.impl.client", "org.apache.mySchemeClient")));
+    Configuration hConfig = yarnJob.client().yarnClient().getConfig();
+    assertEquals("org.apache.myScheme", hConfig.get("fs.myscheme.impl"));
+    assertEquals("org.apache.mySchemeClient", hConfig.get("fs.myscheme.impl.client"));
+  }
+}
diff --git a/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/MockContainerListener.java b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/MockContainerListener.java
new file mode 100644
index 000000000..69c9745e8
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/MockContainerListener.java
@@ -0,0 +1,147 @@
+/*
+ * 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.samza.job.yarn.util;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertTrue;
+
+public class MockContainerListener {
+  private final CountDownLatch conditionLatch;
+
+
+  private final AsyncCountableCondition containersAdded;
+  private final AsyncCountableCondition containersReleased;
+  private final AsyncCountableCondition containersAssigned;
+  private final AsyncCountableCondition containersRunning;
+
+  private final AsyncCountableCondition[] allConditions;
+
+  public MockContainerListener(int numExpectedContainersAdded,
+      int numExpectedContainersReleased,
+      int numExpectedContainersAssigned,
+      int numExpectedContainersRunning,
+      Runnable addContainerAssertions,
+      Runnable releaseContainerAssertions,
+      Runnable assignContainerAssertions,
+      Runnable runContainerAssertions) {
+    containersAdded = new AsyncCountableCondition("containers added", numExpectedContainersAdded, addContainerAssertions);
+    containersReleased = new AsyncCountableCondition("containers released", numExpectedContainersReleased, releaseContainerAssertions);
+    containersAssigned = new AsyncCountableCondition("containers assigned", numExpectedContainersAssigned, assignContainerAssertions);
+    containersRunning = new AsyncCountableCondition("containers running", numExpectedContainersRunning, runContainerAssertions);
+
+    allConditions = new AsyncCountableCondition[] {containersAdded, containersReleased, containersAssigned, containersRunning};
+
+    int unsatisfiedConditions = 0;
+    for (AsyncCountableCondition condition : allConditions) {
+      if (!condition.isSatisfied()) {
+        unsatisfiedConditions++;
+      }
+    }
+
+    conditionLatch = new CountDownLatch(unsatisfiedConditions);
+  }
+
+  public void postAddContainer(int totalAddedContainers) {
+    if (containersAdded.update(totalAddedContainers)) {
+      conditionLatch.countDown();
+    }
+  }
+
+  public void postReleaseContainers(int totalReleasedContainers) {
+    if (containersReleased.update(totalReleasedContainers)) {
+      conditionLatch.countDown();
+    }
+  }
+
+  public void postUpdateRequestStateAfterAssignment(int totalAssignedContainers) {
+    if (containersAssigned.update(totalAssignedContainers)) {
+      conditionLatch.countDown();
+    }
+  }
+
+  public void postRunContainer(int totalRunningContainers) {
+    if (containersRunning.update(totalRunningContainers)) {
+      conditionLatch.countDown();
+    }
+  }
+
+  /**
+   * This method should be called in the main thread. It waits for all the conditions to occur in the other
+   * threads and then verifies that they were in fact satisfied.
+   */
+  public void verify()
+      throws InterruptedException {
+    conditionLatch.await(5, TimeUnit.SECONDS);
+
+    for (AsyncCountableCondition condition : allConditions) {
+      condition.verify();
+    }
+  }
+
+  private static class AsyncCountableCondition {
+    private boolean satisfied = false;
+    private final int expectedCount;
+    private final Runnable postConditionAssertions;
+    private final String name;
+    private AssertionError assertionError = null;
+
+    private AsyncCountableCondition(String name, int expectedCount, Runnable postConditionAssertions) {
+      this.name = name;
+      this.expectedCount = expectedCount;
+      if (expectedCount == 0) satisfied = true;
+      this.postConditionAssertions = postConditionAssertions;
+    }
+
+    public boolean update(int latestCount) {
+      if (!satisfied && latestCount == expectedCount) {
+        if (postConditionAssertions != null) {
+          try {
+            postConditionAssertions.run();
+          } catch (Throwable t) {
+            assertionError = new AssertionError(String.format("Assertion for '%s' failed", name), t);
+          }
+        }
+
+        satisfied = true;
+        return true;
+      }
+      return false;
+    }
+
+    public boolean isSatisfied() {
+      return satisfied;
+    }
+
+    public void verify() {
+      assertTrue(String.format("Condition '%s' was not satisfied", name), isSatisfied());
+
+      if (assertionError != null) {
+        throw assertionError;
+      }
+    }
+
+    @Override
+    public String toString() {
+      return name;
+    }
+  }
+}
diff --git a/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/MockHttpServer.java b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/MockHttpServer.java
new file mode 100644
index 000000000..7435f33df
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/MockHttpServer.java
@@ -0,0 +1,55 @@
+/*
+ * 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.samza.job.yarn.util;
+
+import org.apache.samza.coordinator.server.HttpServer;
+import org.eclipse.jetty.servlet.ServletHolder;
+
+import java.net.MalformedURLException;
+import java.net.URL;
+
+public class MockHttpServer extends HttpServer {
+
+  public MockHttpServer(String rootPath, int port, String resourceBasePath, ServletHolder defaultHolder) {
+    super(rootPath, port, resourceBasePath, defaultHolder);
+    start();
+  }
+
+  @Override
+  public void start() {
+    super.running_$eq(true);
+  }
+
+  @Override
+  public void stop() {
+    super.running_$eq(false);
+  }
+
+  @Override
+  public URL getUrl() {
+    if (running()) {
+      try {
+        return new URL("http://localhost:12345/");
+      } catch (MalformedURLException mue) {
+        mue.printStackTrace();
+      }
+    }
+    return null;
+  }
+}
diff --git a/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/MockNMClient.java b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/MockNMClient.java
new file mode 100644
index 000000000..642d70910
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/MockNMClient.java
@@ -0,0 +1,68 @@
+/*
+ * 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.samza.job.yarn.util;
+
+import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.client.api.NMClient;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+public class MockNMClient extends NMClient {
+  public MockNMClient(String name) {
+    super(name);
+  }
+
+  @Override
+  public Map<String, ByteBuffer> startContainer(Container container, ContainerLaunchContext containerLaunchContext) throws YarnException, IOException {
+    return new HashMap<>();
+  }
+
+  @Override
+  public void stopContainer(ContainerId containerId, NodeId nodeId) throws YarnException, IOException { }
+
+  @Override
+  public ContainerStatus getContainerStatus(ContainerId containerId, NodeId nodeId) throws YarnException, IOException {
+    return null;
+  }
+
+  @Override
+  public void cleanupRunningContainersOnStop(boolean enabled) { }
+
+  @Override
+  public void commitLastReInitialization(ContainerId containerId) { }
+
+  @Override
+  public void rollbackLastReInitialization(ContainerId containerId) { }
+
+  @Override
+  public void restartContainer(ContainerId containerId) { }
+
+  @Override
+  public void increaseContainerResource(Container container) { }
+
+  @Override
+  public void updateContainerResource(Container container) { }
+
+  @Override
+  public void reInitializeContainer(ContainerId containerId, ContainerLaunchContext containerLaunchContext, boolean shouldAutoCommit) { }
+}
diff --git a/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/TestAMRMClientImpl.java b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/TestAMRMClientImpl.java
new file mode 100644
index 000000000..59226cad6
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/TestAMRMClientImpl.java
@@ -0,0 +1,97 @@
+/*
+ * 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.samza.job.yarn.util;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
+import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+public class TestAMRMClientImpl extends AMRMClientImpl<ContainerRequest> {
+
+  private final AllocateResponse response;
+  public List<ContainerRequest> requests = new ArrayList<ContainerRequest>();
+
+  public TestAMRMClientImpl(AllocateResponse response) {
+    this.response = response;
+  }
+
+  public Set<ContainerId> getRelease() {
+    return release;
+  }
+
+  public void resetRelease() {
+    release.clear();
+  }
+
+  @Override
+  public RegisterApplicationMasterResponse registerApplicationMaster(String appHostName,
+                                                                     int appHostPort,
+                                                                     String appTrackingUrl) {
+    return null;
+  }
+
+  @Override
+  public AllocateResponse allocate(float progressIndicator) throws YarnException, IOException {
+    response.getAMCommand();
+    return response;
+  }
+
+  @Override
+  public synchronized void releaseAssignedContainer(ContainerId containerId) {
+    pendingRelease.add(containerId);
+    release.add(containerId);
+  }
+
+  @Override
+  public void unregisterApplicationMaster(FinalApplicationStatus appStatus, String appMessage, String appTrackingUrl)
+      throws YarnException, IOException { }
+
+  @Override
+  public synchronized void addContainerRequest(ContainerRequest req) {
+    requests.add(req);
+  }
+
+  @Override
+  public synchronized void removeContainerRequest(ContainerRequest req) {
+  }
+
+  @Override
+  public synchronized int getClusterNodeCount() {
+    return 1;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception { }
+
+  @Override
+  protected void serviceStart() throws Exception {  }
+
+  @Override
+  protected void serviceStop() throws Exception { }
+}
diff --git a/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/hadoop/TestHttpFileSystem.java b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/hadoop/TestHttpFileSystem.java
new file mode 100644
index 000000000..53f8c4036
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/hadoop/TestHttpFileSystem.java
@@ -0,0 +1,161 @@
+/*
+* 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.samza.job.yarn.util.hadoop;
+
+import junit.framework.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.samza.coordinator.server.HttpServer;
+import org.apache.samza.util.hadoop.HttpFileSystem;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.SocketTimeoutException;
+import java.net.URI;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Test the behavior of {@link HttpFileSystem}
+ */
+public class TestHttpFileSystem {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestHttpFileSystem.class);
+  /**
+   * Number of bytes the server should stream before hanging the TCP connection.
+   */
+  private static final int THRESHOLD_BYTES = 5;
+  private static final String RESPONSE_STR = "HELLO WORLD";
+
+  private final CountDownLatch serverWaitLatch = new CountDownLatch(1);
+
+  private Exception clientException;
+  private Exception serverException;
+
+  /**
+   * A {@link HttpServlet} implementation that streams its response to the client one byte at a time.
+   */
+  private class PartialFileFetchServlet extends HttpServlet {
+    @Override
+    public void doGet(HttpServletRequest request, HttpServletResponse response) {
+
+      //Mimic download of the package tar-ball
+      response.setContentType("application/gzip");
+      response.setStatus(HttpServletResponse.SC_OK);
+
+      try {
+        int currByteWritten = -1;
+        int numBytesWritten = 0;
+        //Begin streaming a response.
+        InputStream in = new ByteArrayInputStream(RESPONSE_STR.getBytes());
+        OutputStream out = response.getOutputStream();
+
+        while ((currByteWritten = in.read()) != -1) {
+          out.write(currByteWritten);
+          out.flush();
+          numBytesWritten++;
+
+          //Hang the connection until the read timeout expires on the client side.
+          if (numBytesWritten >= THRESHOLD_BYTES) {
+            if (!serverWaitLatch.await(5, TimeUnit.SECONDS)) {
+              throw new IOException("Timed out waiting for latch");
+            }
+            break;
+          }
+        }
+      } catch (Exception e) {
+        //Record any exception that may have occurred
+        LOG.error("{}", e);
+        serverException = e;
+      }
+    }
+  }
+
+  class FileSystemClientThread extends Thread {
+
+    private static final int TIMEOUT_MS = 1000;
+    private final URI resourceURI;
+    private int totalBytesRead = 0;
+
+    FileSystemClientThread(URI resourceURI) {
+      this.resourceURI = resourceURI;
+    }
+
+    public int getTotalBytesRead() {
+      return totalBytesRead;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Path resource = new Path(resourceURI);
+        Configuration conf = new Configuration();
+        HttpFileSystem fs = new HttpFileSystem();
+        fs.setSocketReadTimeoutMs(TIMEOUT_MS);
+        fs.setConnectionTimeoutMs(TIMEOUT_MS);
+        fs.setConf(conf);
+        fs.initialize(resourceURI, conf);
+
+        //Read from the socket one byte at a time.
+        FSDataInputStream in = fs.open(resource);
+        while (in.read() >= 0) {
+          totalBytesRead++;
+        }
+      } catch (SocketTimeoutException e) {
+        //Expect the socket to timeout after THRESHOLD bytes have been read.
+        serverWaitLatch.countDown();
+      } catch (Exception e) {
+        //Record any exception that may have occurred.
+        LOG.error("{}", e);
+        clientException = e;
+      }
+    }
+  }
+
+  @Test
+  public void testHttpFileSystemReadTimeouts() throws Exception {
+    HttpServer server = new HttpServer("/", 0, null, new ServletHolder(DefaultServlet.class));
+    try {
+      server.addServlet("/download", new PartialFileFetchServlet());
+      server.start();
+      String serverUrl = server.getUrl().toString() + "download";
+      FileSystemClientThread fileSystemClientThread = new FileSystemClientThread(new URI(serverUrl));
+      fileSystemClientThread.start();
+      fileSystemClientThread.join();
+      Assert.assertEquals(fileSystemClientThread.getTotalBytesRead(), THRESHOLD_BYTES);
+      Assert.assertNull(clientException);
+      Assert.assertNull(serverException);
+    } finally {
+      server.stop();
+    }
+  }
+}
+
diff --git a/samza-yarn3/src/test/java/org/apache/samza/validation/MockMetricsValidator.java b/samza-yarn3/src/test/java/org/apache/samza/validation/MockMetricsValidator.java
new file mode 100644
index 000000000..de0a980e8
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/validation/MockMetricsValidator.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.samza.validation;
+
+import java.util.Map;
+import org.apache.samza.config.Config;
+import org.apache.samza.container.SamzaContainerMetrics;
+import org.apache.samza.metrics.MetricsAccessor;
+import org.apache.samza.metrics.MetricsValidationFailureException;
+import org.apache.samza.metrics.MetricsValidator;
+
+
+public class MockMetricsValidator implements MetricsValidator {
+
+  @Override
+  public void init(Config config) {
+  }
+
+  @Override
+  public void validate(MetricsAccessor accessor) throws MetricsValidationFailureException {
+    Map<String, Long> commitCalls = accessor.getCounterValues(SamzaContainerMetrics.class.getName(), "commit-calls");
+    if (commitCalls.isEmpty()) {
+      throw new MetricsValidationFailureException("no value");
+    }
+    for (Map.Entry<String, Long> entry : commitCalls.entrySet()) {
+      if (entry.getValue() <= 0) {
+        throw new MetricsValidationFailureException("commit call <= 0");
+      }
+    }
+  }
+
+  @Override
+  public void complete() {
+  }
+}
\ No newline at end of file
diff --git a/samza-yarn3/src/test/java/org/apache/samza/validation/TestYarnJobValidationTool.java b/samza-yarn3/src/test/java/org/apache/samza/validation/TestYarnJobValidationTool.java
new file mode 100644
index 000000000..7a8d291f2
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/validation/TestYarnJobValidationTool.java
@@ -0,0 +1,142 @@
+/*
+ * 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.samza.validation;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.MapConfig;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.samza.container.SamzaContainerMetrics;
+import org.apache.samza.metrics.JmxMetricsAccessor;
+import org.apache.samza.metrics.MetricsValidationFailureException;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.junit.Assert.*;
+
+public class TestYarnJobValidationTool {
+  private YarnClient client;
+  private YarnJobValidationTool tool;
+  private String jobName = "test";
+  private int jobId = 1;
+  private ApplicationId appId;
+  ApplicationAttemptId attemptId;
+  private int containerCount = 9;
+  private Config config = new MapConfig(new HashMap<String, String>() {
+    {
+      put("job.name", jobName);
+      put("job.id", String.valueOf(jobId));
+      put("yarn.container.count", String.valueOf(containerCount));
+    }
+  });
+  private MockMetricsValidator validator = new MockMetricsValidator();
+
+  @Rule
+  public final ExpectedException exception = ExpectedException.none();
+
+  @Before
+  public void setup() throws Exception {
+    client = mock(YarnClient.class);
+    tool = new YarnJobValidationTool(new JobConfig(config), client, validator);
+    appId = mock(ApplicationId.class);
+    when(appId.getId()).thenReturn(1111);
+    attemptId = mock(ApplicationAttemptId.class);
+    when(attemptId.getApplicationId()).thenReturn(appId);
+    when(attemptId.getAttemptId()).thenReturn(2222);
+  }
+
+  @Test
+  public void testValidateAppId() throws Exception {
+    ApplicationReport appReport = mock(ApplicationReport.class);
+    when(appReport.getName()).thenReturn(jobName + "_" + jobId);
+    when(appReport.getApplicationId()).thenReturn(appId);
+    when(client.getApplications()).thenReturn(Collections.singletonList(appReport));
+    assertTrue(tool.validateAppId().equals(appId));
+
+    when(appReport.getName()).thenReturn("dummy");
+    exception.expect(SamzaException.class);
+    tool.validateAppId();
+  }
+
+  @Test
+  public void testValidateRunningAttemptId() throws Exception {
+    ApplicationReport appReport = mock(ApplicationReport.class);
+    when(client.getApplicationReport(appId)).thenReturn(appReport);
+    when(appReport.getCurrentApplicationAttemptId()).thenReturn(attemptId);
+    ApplicationAttemptReport attemptReport = mock(ApplicationAttemptReport.class);
+    when(attemptReport.getYarnApplicationAttemptState()).thenReturn(YarnApplicationAttemptState.RUNNING);
+    when(attemptReport.getApplicationAttemptId()).thenReturn(attemptId);
+    when(client.getApplicationAttemptReport(attemptId)).thenReturn(attemptReport);
+    assertTrue(tool.validateRunningAttemptId(appId).equals(attemptId));
+
+    when(attemptReport.getYarnApplicationAttemptState()).thenReturn(YarnApplicationAttemptState.FAILED);
+    exception.expect(SamzaException.class);
+    tool.validateRunningAttemptId(appId);
+  }
+
+  @Test
+  public void testValidateContainerCount() throws Exception {
+    List<ContainerReport> containerReports = new ArrayList<>();
+    for (int i = 0; i <= containerCount; i++) {
+      ContainerReport report = mock(ContainerReport.class);
+      when(report.getContainerState()).thenReturn(ContainerState.RUNNING);
+      containerReports.add(report);
+    }
+    when(client.getContainers(attemptId)).thenReturn(containerReports);
+    assertTrue(tool.validateContainerCount(attemptId) == (containerCount + 1));
+
+    containerReports.remove(0);
+    exception.expect(SamzaException.class);
+    tool.validateContainerCount(attemptId);
+  }
+
+  @Test
+  public void testValidateJmxMetrics() throws MetricsValidationFailureException {
+    JmxMetricsAccessor jmxMetricsAccessor = mock(JmxMetricsAccessor.class);
+    Map<String, Long> values = new HashMap<>();
+    values.put("samza-container-0", 100L);
+    when(jmxMetricsAccessor.getCounterValues(SamzaContainerMetrics.class.getName(), "commit-calls")).thenReturn(values);
+    validator.validate(jmxMetricsAccessor);
+
+    values.put("samza-container-0", -1L);
+    // the mock validator will fail if the commit-calls are less than or equal to 0
+    exception.expect(MetricsValidationFailureException.class);
+    validator.validate(jmxMetricsAccessor);
+  }
+}
diff --git a/samza-yarn3/src/test/java/org/apache/samza/webapp/TestApplicationMasterRestClient.java b/samza-yarn3/src/test/java/org/apache/samza/webapp/TestApplicationMasterRestClient.java
new file mode 100644
index 000000000..f60f232e6
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/webapp/TestApplicationMasterRestClient.java
@@ -0,0 +1,331 @@
+/*
+ * 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.samza.webapp;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import java.io.IOException;
+import java.io.StringReader;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.io.input.ReaderInputStream;
+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.util.ConverterUtils;
+import org.apache.http.HttpEntity;
+import org.apache.http.HttpHost;
+import org.apache.http.HttpStatus;
+import org.apache.http.StatusLine;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.samza.Partition;
+import org.apache.samza.SamzaException;
+import org.apache.samza.clustermanager.SamzaApplicationState;
+import org.apache.samza.clustermanager.SamzaResource;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.container.grouper.task.GroupByContainerCount;
+import org.apache.samza.coordinator.JobModelManager;
+import org.apache.samza.job.model.ContainerModel;
+import org.apache.samza.job.model.JobModel;
+import org.apache.samza.job.model.TaskModel;
+import org.apache.samza.job.yarn.SamzaAppMasterMetrics;
+import org.apache.samza.job.yarn.YarnAppState;
+import org.apache.samza.job.yarn.YarnContainer;
+import org.apache.samza.metrics.MetricsRegistryMap;
+import org.apache.samza.serializers.model.SamzaObjectMapper;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamPartition;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.core.type.TypeReference;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+
+public class TestApplicationMasterRestClient {
+  private static final String AM_HOST_NAME = "dummyHost";
+  private static final int AM_RPC_PORT = 1337;
+  private static final int AM_HTTP_PORT = 7001;
+  private static final String YARN_CONTAINER_ID_1 = "container_e38_1510966221296_0007_01_000001";
+  private static final String YARN_CONTAINER_ID_2 = "container_e38_1510966221296_0007_01_000002";
+  private static final String YARN_CONTAINER_ID_3 = "container_e38_1510966221296_0007_01_000003";
+  private static final String APP_ATTEMPT_ID = "appattempt_1510966221296_0007_000001";
+
+  private final ObjectMapper jsonMapper = SamzaObjectMapper.getObjectMapper();
+
+  private CloseableHttpClient mockClient;
+
+  @Before
+  public void setup() {
+    mockClient = mock(CloseableHttpClient.class);
+  }
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none(); // Enables us to verify the exception message
+
+  @Test
+  public void testGetMetricsSuccess() throws IOException {
+    SamzaApplicationState samzaAppState = createSamzaApplicationState();
+
+    MetricsRegistryMap registry = new MetricsRegistryMap();
+    assignMetricValues(samzaAppState, registry);
+
+    String response = ApplicationMasterRestServlet.getMetrics(jsonMapper, registry);
+    setupMockClientResponse(HttpStatus.SC_OK, "Success", response);
+
+    ApplicationMasterRestClient client = new ApplicationMasterRestClient(mockClient, AM_HOST_NAME, AM_RPC_PORT);
+    Map<String, Map<String, Object>> metricsResult = client.getMetrics();
+
+    String group = SamzaAppMasterMetrics.class.getCanonicalName();
+    assertEquals(1, metricsResult.size());
+    assertTrue(metricsResult.containsKey(group));
+
+    Map<String, Object> amMetricsGroup = metricsResult.get(group);
+    assertEquals(8, amMetricsGroup.size());
+    assertEquals(samzaAppState.runningProcessors.size(),  amMetricsGroup.get("running-containers"));
+    assertEquals(samzaAppState.neededProcessors.get(),    amMetricsGroup.get("needed-containers"));
+    assertEquals(samzaAppState.completedProcessors.get(), amMetricsGroup.get("completed-containers"));
+    assertEquals(samzaAppState.failedContainers.get(),    amMetricsGroup.get("failed-containers"));
+    assertEquals(samzaAppState.releasedContainers.get(),  amMetricsGroup.get("released-containers"));
+    assertEquals(samzaAppState.processorCount.get(),      amMetricsGroup.get("container-count"));
+    assertEquals(samzaAppState.jobHealthy.get() ? 1 : 0,  amMetricsGroup.get("job-healthy"));
+    assertEquals(0, amMetricsGroup.get("container-from-previous-attempt"));
+  }
+
+  @Test
+  public void testGetMetricsError() throws IOException {
+    setupErrorTest("metrics");
+
+    ApplicationMasterRestClient client = new ApplicationMasterRestClient(mockClient, AM_HOST_NAME, AM_RPC_PORT);
+    client.getMetrics();
+  }
+
+  @Test
+  public void testGetTaskContextSuccess() throws IOException {
+    ContainerId containerId = ConverterUtils.toContainerId(YARN_CONTAINER_ID_1);
+    YarnAppState yarnAppState = createYarnAppState(containerId);
+
+    String response = ApplicationMasterRestServlet.getTaskContext(jsonMapper, yarnAppState);
+    setupMockClientResponse(HttpStatus.SC_OK, "Success", response);
+
+    ApplicationMasterRestClient client = new ApplicationMasterRestClient(mockClient, AM_HOST_NAME, AM_RPC_PORT);
+    Map<String, Object> taskContextResult = client.getTaskContext();
+
+    assertEquals(2, taskContextResult.size());
+    assertEquals(2, taskContextResult.get("task-id"));
+    assertEquals(containerId.toString(), taskContextResult.get("name"));
+  }
+
+  @Test
+  public void testTaskContextError() throws IOException {
+    setupErrorTest("task context");
+
+    ApplicationMasterRestClient client = new ApplicationMasterRestClient(mockClient, AM_HOST_NAME, AM_RPC_PORT);
+    client.getTaskContext();
+  }
+
+  @Test
+  public void testGetAmStateSuccess() throws IOException {
+    SamzaApplicationState samzaAppState = createSamzaApplicationState();
+
+    ApplicationAttemptId attemptId = ConverterUtils.toApplicationAttemptId(APP_ATTEMPT_ID);
+    ContainerId containerId = ConverterUtils.toContainerId(YARN_CONTAINER_ID_1);
+    YarnAppState yarnAppState = createYarnAppState(containerId);
+
+    String response = ApplicationMasterRestServlet.getAmState(jsonMapper, samzaAppState, yarnAppState);
+    setupMockClientResponse(HttpStatus.SC_OK, "Success", response);
+
+    ApplicationMasterRestClient client = new ApplicationMasterRestClient(mockClient, AM_HOST_NAME, AM_RPC_PORT);
+    Map<String, Object> amStateResult = client.getAmState();
+
+    assertEquals(4, amStateResult.size());
+    assertEquals(String.format("%s:%s", yarnAppState.nodeHost, yarnAppState.rpcUrl.getPort()), amStateResult.get("host"));
+    assertEquals(containerId.toString(), amStateResult.get("container-id"));
+    // Can only validate the keys because up-time changes everytime it's requested
+    assertEquals(buildExpectedContainerResponse(yarnAppState.runningProcessors, samzaAppState).keySet(),
+        ((Map<String, Object>) amStateResult.get("containers")).keySet());
+    assertEquals(attemptId.toString(), amStateResult.get("app-attempt-id"));
+  }
+
+  @Test
+  public void testGetAmStateError() throws IOException {
+    setupErrorTest("AM state");
+
+    ApplicationMasterRestClient client = new ApplicationMasterRestClient(mockClient, AM_HOST_NAME, AM_RPC_PORT);
+    client.getAmState();
+  }
+
+  @Test
+  public void testGetConfigSuccess() throws IOException {
+    SamzaApplicationState samzaAppState = createSamzaApplicationState();
+
+    Map<String, String> configMap = ImmutableMap.of("key1", "value1", "key2", "value2");
+    Config config = new MapConfig(configMap);
+
+    String response = ApplicationMasterRestServlet.getConfig(jsonMapper, config);
+    setupMockClientResponse(HttpStatus.SC_OK, "Success", response);
+
+    ApplicationMasterRestClient client = new ApplicationMasterRestClient(mockClient, AM_HOST_NAME, AM_RPC_PORT);
+    Map<String, Object> configResult = client.getConfig();
+
+    assertEquals(configMap, configResult);
+  }
+
+  @Test
+  public void testGetConfigError() throws IOException {
+    setupErrorTest("config");
+
+    ApplicationMasterRestClient client = new ApplicationMasterRestClient(mockClient, AM_HOST_NAME, AM_RPC_PORT);
+    client.getConfig();
+  }
+
+  @Test
+  public void testCloseMethodClosesHttpClient() throws IOException {
+    ApplicationMasterRestClient client = new ApplicationMasterRestClient(mockClient, AM_HOST_NAME, AM_RPC_PORT);
+    client.close();
+
+    verify(mockClient).close();
+  }
+
+  private void setupMockClientResponse(int statusCode, String statusReason, String responseBody) throws IOException {
+    StatusLine statusLine = mock(StatusLine.class);
+    when(statusLine.getStatusCode()).thenReturn(statusCode);
+    when(statusLine.getReasonPhrase()).thenReturn(statusReason);
+
+    HttpEntity entity = mock(HttpEntity.class);
+    when(entity.getContent()).thenReturn(new ReaderInputStream(new StringReader(responseBody)));
+
+    CloseableHttpResponse response = mock(CloseableHttpResponse.class);
+    when(response.getStatusLine()).thenReturn(statusLine);
+    when(response.getEntity()).thenReturn(entity);
+
+    when(mockClient.execute(any(HttpHost.class), any(HttpGet.class))).thenReturn(response);
+  }
+
+  private SamzaApplicationState createSamzaApplicationState() {
+    HashMap<String, ContainerModel> containers = generateContainers();
+
+    JobModel mockJobModel = mock(JobModel.class);
+    when(mockJobModel.getContainers()).thenReturn(containers);
+    JobModelManager mockJobModelManager = mock(JobModelManager.class);
+    when(mockJobModelManager.jobModel()).thenReturn(mockJobModel);
+
+    SamzaApplicationState samzaApplicationState = new SamzaApplicationState(mockJobModelManager);
+
+    samzaApplicationState.runningProcessors.put(YARN_CONTAINER_ID_3,
+        new SamzaResource(1, 2, "dummyNodeHost1", "dummyResourceId1"));
+    samzaApplicationState.runningProcessors.put(YARN_CONTAINER_ID_2,
+        new SamzaResource(2, 4, "dummyNodeHost2", "dummyResourceId2"));
+    return samzaApplicationState;
+  }
+
+  private YarnAppState createYarnAppState(ContainerId containerId) throws MalformedURLException {
+    YarnAppState yarnAppState = new YarnAppState(2, containerId, AM_HOST_NAME, AM_RPC_PORT, AM_HTTP_PORT);
+    yarnAppState.rpcUrl = new URL(new HttpHost(AM_HOST_NAME, AM_RPC_PORT).toURI());
+    yarnAppState.runningProcessors.put("0", new YarnContainer(Container.newInstance(
+        ConverterUtils.toContainerId(YARN_CONTAINER_ID_2),
+        ConverterUtils.toNodeIdWithDefaultPort("dummyNodeHost1"),
+        "dummyNodeHttpHost1",
+        null,
+        null,
+        null
+    )));
+    yarnAppState.runningProcessors.put("1", new YarnContainer(Container.newInstance(
+        ConverterUtils.toContainerId(YARN_CONTAINER_ID_3),
+        ConverterUtils.toNodeIdWithDefaultPort("dummyNodeHost2"),
+        "dummyNodeHttpHost2",
+        null,
+        null,
+        null
+    )));
+    return yarnAppState;
+  }
+
+  private HashMap<String, ContainerModel> generateContainers() {
+    Set<TaskModel> taskModels = ImmutableSet.of(
+        new TaskModel(new TaskName("task1"),
+                      ImmutableSet.of(new SystemStreamPartition(new SystemStream("system1", "stream1"), new Partition(0))),
+                      new Partition(0)),
+        new TaskModel(new TaskName("task2"),
+            ImmutableSet.of(new SystemStreamPartition(new SystemStream("system1", "stream1"), new Partition(1))),
+            new Partition(1)));
+    GroupByContainerCount grouper = new GroupByContainerCount(2);
+    Set<ContainerModel> containerModels = grouper.group(taskModels);
+    HashMap<String, ContainerModel> containers = new HashMap<>();
+    for (ContainerModel containerModel : containerModels) {
+      containers.put(containerModel.getId(), containerModel);
+    }
+    return containers;
+  }
+
+  private Map<String, Map<String, Object>> buildExpectedContainerResponse(Map<String, YarnContainer> runningYarnContainers,
+      SamzaApplicationState samzaAppState) throws IOException {
+    Map<String, Map<String, Object>> containers = new HashMap<>();
+
+    runningYarnContainers.forEach((containerId, container) -> {
+      String yarnContainerId = container.id().toString();
+      Map<String, Object> containerMap = new HashMap();
+      Map<TaskName, TaskModel> taskModels = samzaAppState.jobModelManager.jobModel().getContainers().get(containerId).getTasks();
+      containerMap.put("yarn-address", container.nodeHttpAddress());
+      containerMap.put("start-time", String.valueOf(container.startTime()));
+      containerMap.put("up-time", String.valueOf(container.upTime()));
+      containerMap.put("task-models", taskModels);
+      containerMap.put("container-id", containerId);
+      containers.put(yarnContainerId, containerMap);
+    });
+
+    return jsonMapper.readValue(jsonMapper.writeValueAsString(containers), new TypeReference<Map<String, Map<String, Object>>>() { });
+  }
+
+  private void assignMetricValues(SamzaApplicationState samzaAppState, MetricsRegistryMap registry) {
+    SamzaAppMasterMetrics metrics = new SamzaAppMasterMetrics(new MapConfig(), samzaAppState, registry);
+    metrics.start();
+    samzaAppState.runningProcessors.put("dummyContainer",
+        new SamzaResource(1, 2, AM_HOST_NAME, "dummyResourceId")); // 1 container
+    samzaAppState.neededProcessors.set(2);
+    samzaAppState.completedProcessors.set(3);
+    samzaAppState.failedContainers.set(4);
+    samzaAppState.releasedContainers.set(5);
+    samzaAppState.processorCount.set(6);
+    samzaAppState.jobHealthy.set(true);
+  }
+
+  private void setupErrorTest(String entityToFetch) throws IOException {
+    String statusReason = "Dummy status reason";
+    expectedException.expect(SamzaException.class);
+    expectedException.expectMessage(String.format(
+        "Error retrieving %s from host %s. Response: %s",
+        entityToFetch,
+        new HttpHost(AM_HOST_NAME, AM_RPC_PORT).toURI(),
+        statusReason));
+
+    setupMockClientResponse(HttpStatus.SC_INTERNAL_SERVER_ERROR, statusReason, "");
+  }
+}
diff --git a/samza-yarn3/src/test/java/org/apache/samza/webapp/TestLocalityServlet.java b/samza-yarn3/src/test/java/org/apache/samza/webapp/TestLocalityServlet.java
new file mode 100644
index 000000000..27cf78414
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/webapp/TestLocalityServlet.java
@@ -0,0 +1,131 @@
+/*
+ * 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.samza.webapp;
+
+import com.google.common.collect.ImmutableMap;
+import java.net.URL;
+import java.util.Collections;
+import org.apache.samza.container.LocalityManager;
+import org.apache.samza.coordinator.server.HttpServer;
+import org.apache.samza.coordinator.server.LocalityServlet;
+import org.apache.samza.job.model.ProcessorLocality;
+import org.apache.samza.job.model.LocalityModel;
+import org.apache.samza.serializers.model.SamzaObjectMapper;
+import org.apache.samza.util.ExponentialSleepStrategy;
+import org.apache.samza.util.HttpUtil;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+
+/**
+ * A test class for {@link LocalityServlet}. It validates the servlet directly and Serde Mix-In of {@link ProcessorLocality}
+ * indirectly.
+ */
+public class TestLocalityServlet {
+  private static final String PROCESSOR_ID1 = "1";
+  private static final String PROCESSOR_ID2 = "2";
+  private static final String HOST1 = "host1";
+  private static final String HOST2 = "host2";
+  private static final String JMX_URL = "jmx";
+  private static final String TUNNELING_URL = "tunneling";
+
+  private static final ProcessorLocality PROCESSOR_1_LOCALITY =
+      new ProcessorLocality(PROCESSOR_ID1, HOST1, JMX_URL, TUNNELING_URL);
+  private static final ProcessorLocality PROCESSOR_2_LOCALITY =
+      new ProcessorLocality("2", HOST2, JMX_URL, TUNNELING_URL);
+
+  private final ObjectMapper mapper = SamzaObjectMapper.getObjectMapper();
+  private HttpServer webApp;
+  private LocalityManager localityManager;
+
+
+
+  @Before
+  public void setup()
+      throws Exception {
+    localityManager = mock(LocalityManager.class);
+    when(localityManager.readLocality())
+        .thenReturn(new LocalityModel(ImmutableMap.of(PROCESSOR_ID1, PROCESSOR_1_LOCALITY, PROCESSOR_ID2, PROCESSOR_2_LOCALITY)));
+    webApp = new HttpServer("/", 0, "", new ServletHolder(new DefaultServlet()));
+    webApp.addServlet("/locality", new LocalityServlet(localityManager));
+    webApp.start();
+  }
+
+  @After
+  public void cleanup()
+      throws Exception {
+    webApp.stop();
+  }
+
+  @Test
+  public void testReadContainerLocality() throws Exception {
+    URL url = new URL(webApp.getUrl().toString() + "locality");
+
+    String response = HttpUtil.read(url, 1000, new ExponentialSleepStrategy());
+    LocalityModel locality = mapper.readValue(response, LocalityModel.class);
+
+    assertEquals("Expected locality for two containers", 2, locality.getProcessorLocalities().size());
+    assertEquals("Mismatch in locality for processor " + PROCESSOR_ID1,
+        locality.getProcessorLocality(PROCESSOR_ID1), PROCESSOR_1_LOCALITY);
+    assertEquals("Mismatch in locality for processor " + PROCESSOR_ID2,
+        locality.getProcessorLocality(PROCESSOR_ID2), PROCESSOR_2_LOCALITY);
+  }
+
+  @Test
+  public void testReadContainerLocalityWithNoLocality() throws Exception {
+    final LocalityModel expectedLocality = new LocalityModel(Collections.emptyMap());
+    URL url = new URL(webApp.getUrl().toString() + "locality");
+    when(localityManager.readLocality()).thenReturn(new LocalityModel(ImmutableMap.of()));
+
+    String response = HttpUtil.read(url, 1000, new ExponentialSleepStrategy());
+    LocalityModel locality = mapper.readValue(response, LocalityModel.class);
+
+    assertEquals("Expected empty response but got " + locality, locality, expectedLocality);
+  }
+
+  @Test
+  public void testReadProcessorLocality() throws Exception {
+    URL url = new URL(webApp.getUrl().toString() + "locality?processorId=" + PROCESSOR_ID1);
+    String response = HttpUtil.read(url, 1000, new ExponentialSleepStrategy());
+
+    assertEquals("Mismatch in the locality for processor " + PROCESSOR_ID1,
+        mapper.readValue(response, ProcessorLocality.class), PROCESSOR_1_LOCALITY);
+  }
+
+  @Test
+  public void testReadProcessorLocalityWithNoLocality() throws Exception {
+    final ProcessorLocality expectedProcessorLocality = new ProcessorLocality(PROCESSOR_ID2, "");
+    URL url = new URL(webApp.getUrl().toString() + "locality?processorId=" + PROCESSOR_ID2);
+    when(localityManager.readLocality()).thenReturn(new LocalityModel(ImmutableMap.of()));
+
+    String response = HttpUtil.read(url, 1000, new ExponentialSleepStrategy());
+    ProcessorLocality processorLocality = mapper.readValue(response, ProcessorLocality.class);
+
+    assertEquals("Expected empty response for processor locality " + PROCESSOR_ID2 + " but got " + processorLocality,
+        processorLocality, expectedProcessorLocality);
+  }
+}
diff --git a/samza-yarn3/src/test/java/org/apache/samza/webapp/TestYarnContainerHeartbeatServlet.java b/samza-yarn3/src/test/java/org/apache/samza/webapp/TestYarnContainerHeartbeatServlet.java
new file mode 100644
index 000000000..f3b598018
--- /dev/null
+++ b/samza-yarn3/src/test/java/org/apache/samza/webapp/TestYarnContainerHeartbeatServlet.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.samza.webapp;
+
+import java.io.IOException;
+import java.net.URL;
+import junit.framework.Assert;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.samza.container.ContainerHeartbeatResponse;
+import org.apache.samza.coordinator.CoordinationConstants;
+import org.apache.samza.coordinator.server.HttpServer;
+import org.apache.samza.job.yarn.YarnAppState;
+import org.apache.samza.job.yarn.YarnContainer;
+import org.apache.samza.metrics.MetricsRegistryMap;
+import org.apache.samza.metrics.ReadableMetricsRegistry;
+import org.apache.samza.util.ExponentialSleepStrategy;
+import org.apache.samza.util.HttpUtil;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestYarnContainerHeartbeatServlet {
+
+  private YarnContainer container;
+  private YarnAppState yarnAppState;
+  private HttpServer webApp;
+  private ObjectMapper mapper;
+
+  private ContainerHeartbeatResponse heartbeat;
+
+  @Before
+  public void setup()
+      throws Exception {
+    container = mock(YarnContainer.class);
+    ReadableMetricsRegistry registry = new MetricsRegistryMap();
+
+    yarnAppState =
+        new YarnAppState(-1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "testHost", 1, 1);
+    webApp = new HttpServer("/", 0, "", new ServletHolder(new DefaultServlet()));
+    webApp.addServlet("/", new YarnContainerHeartbeatServlet(yarnAppState, registry));
+    webApp.start();
+    mapper = new ObjectMapper();
+  }
+
+  @After
+  public void cleanup()
+      throws Exception {
+    webApp.stop();
+  }
+
+  @Test
+  public void testContainerHeartbeatWhenValid()
+      throws IOException {
+    String validContainerId = "container_1350670447861_0003_01_000002";
+    when(container.id()).thenReturn(ConverterUtils.toContainerId(validContainerId));
+    yarnAppState.runningProcessors.put(validContainerId, container);
+    URL url = new URL(String.format(CoordinationConstants.YARN_CONTAINER_HEARTBEAT_ENDPOINT_FORMAT, webApp.getUrl().toString(), validContainerId));
+    String response = HttpUtil.read(url, 1000, new ExponentialSleepStrategy());
+    heartbeat = mapper.readValue(response, ContainerHeartbeatResponse.class);
+    Assert.assertTrue(heartbeat.isAlive());
+  }
+
+  @Test
+  public void testContainerHeartbeatWhenInvalid()
+      throws IOException {
+    String validContainerId = "container_1350670447861_0003_01_000003";
+    String invalidContainerId = "container_1350670447861_0003_01_000002";
+    when(container.id()).thenReturn(ConverterUtils.toContainerId(validContainerId));
+    yarnAppState.runningProcessors.put(validContainerId, container);
+    URL url = new URL(String.format(CoordinationConstants.YARN_CONTAINER_HEARTBEAT_ENDPOINT_FORMAT,
+        webApp.getUrl().toString(), invalidContainerId));
+    String response = HttpUtil.read(url, 1000, new ExponentialSleepStrategy());
+    heartbeat = mapper.readValue(response, ContainerHeartbeatResponse.class);
+    Assert.assertFalse(heartbeat.isAlive());
+  }
+}
diff --git a/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/MockSystemAdmin.scala b/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/MockSystemAdmin.scala
new file mode 100644
index 000000000..5650d4b17
--- /dev/null
+++ b/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/MockSystemAdmin.scala
@@ -0,0 +1,42 @@
+/*
+ * 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.samza.job.yarn
+
+import org.apache.samza.Partition
+import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata
+import org.apache.samza.system.{SystemStreamMetadata, SystemStreamPartition, SystemAdmin}
+import scala.collection.JavaConverters._
+
+/**
+ * A mock implementation class that returns metadata for each stream that contains numTasks partitions in it.
+ */
+class MockSystemAdmin(numTasks: Int) extends SystemAdmin {
+  def getOffsetsAfter(offsets: java.util.Map[SystemStreamPartition, String]) = null
+  def getSystemStreamMetadata(streamNames: java.util.Set[String]) = {
+    streamNames.asScala.map(streamName => {
+      val partitionMetadata = (0 until numTasks).map(partitionId => {
+        new Partition(partitionId) -> new SystemStreamPartitionMetadata(null, null, null)
+      }).toMap
+      streamName -> new SystemStreamMetadata(streamName, partitionMetadata.asJava)
+    }).toMap.asJava
+  }
+
+  override def offsetComparator(offset1: String, offset2: String) = null
+}
diff --git a/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/MockSystemFactory.scala b/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/MockSystemFactory.scala
new file mode 100644
index 000000000..458400e66
--- /dev/null
+++ b/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/MockSystemFactory.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.job.yarn
+
+import org.apache.samza.config.{JobConfig, Config}
+import org.apache.samza.metrics.MetricsRegistry
+import org.apache.samza.system.SystemFactory
+
+/**
+  * A {@link org.apache.samza.system.SystemFactory} implementation that returns a {@link org.apache.samza.job.yarn.MockSystemAdmin}.
+  */
+
+class MockSystemFactory extends SystemFactory {
+  def getConsumer(systemName: String, config: Config, registry: MetricsRegistry) = {
+    throw new RuntimeException("Hmm. Not implemented.")
+  }
+
+  def getProducer(systemName: String, config: Config, registry: MetricsRegistry) = {
+    throw new RuntimeException("Hmm. Not implemented.")
+  }
+
+  def getAdmin(systemName: String, config: Config) = {
+    val jobConfig = new JobConfig(config)
+    new MockSystemAdmin(jobConfig.getContainerCount)
+  }
+}
diff --git a/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/TestClientHelper.scala b/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/TestClientHelper.scala
new file mode 100644
index 000000000..f85bb566b
--- /dev/null
+++ b/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/TestClientHelper.scala
@@ -0,0 +1,160 @@
+/*
+ * 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.samza.job.yarn
+
+import java.net.ConnectException
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.permission.FsPermission
+import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
+import org.apache.hadoop.yarn.api.records.{ApplicationId, ApplicationReport, FinalApplicationStatus, YarnApplicationState}
+import org.apache.hadoop.yarn.client.api.YarnClient
+import org.apache.samza.SamzaException
+import org.apache.samza.config.{JobConfig, MapConfig, YarnConfig}
+import org.apache.samza.job.ApplicationStatus
+import org.apache.samza.webapp.ApplicationMasterRestClient
+import org.junit.Assert.{assertEquals, assertNotNull}
+import org.mockito.Matchers.any
+import org.mockito.Mockito._
+import org.scalatest.FunSuite
+import org.scalatest.mockito.MockitoSugar
+
+
+class TestClientHelper extends FunSuite {
+  import MockitoSugar._
+  val hadoopConfig = mock[Configuration]
+  val mockAmClient = mock[ApplicationMasterRestClient]
+
+  val clientHelper = new ClientHelper(hadoopConfig) {
+    override def createYarnClient() = {
+      mock[YarnClient]
+    }
+    override def createAmClient(applicationReport: ApplicationReport) = {
+      mockAmClient
+    }
+  }
+
+  test("test allContainersRunning is false when there are ConnectExceptions") {
+    val exceptionThrowingClientHelper = new ClientHelper(hadoopConfig) {
+      override def createYarnClient() = {
+        mock[YarnClient]
+      }
+      override def createAmClient(applicationReport: ApplicationReport) = {
+        val amClient = mock[ApplicationMasterRestClient]
+        when(amClient.getMetrics).thenThrow(new ConnectException())
+        amClient
+      }
+    }
+
+    val allContainersRunning = exceptionThrowingClientHelper.allContainersRunning(null)
+    assertEquals(false, allContainersRunning)
+  }
+
+  test("test validateJobConfig") {
+    import collection.JavaConverters._
+    var config = new MapConfig()
+
+    intercept[SamzaException] {
+      clientHelper.validateJobConfig(config)
+    }
+
+    config = new MapConfig(Map(JobConfig.JOB_SECURITY_MANAGER_FACTORY -> "some value").asJava)
+
+    clientHelper.validateJobConfig(config)
+  }
+
+  test("test prepareJobConfig") {
+    val jobContext = new JobContext
+    jobContext.setAppStagingDir(new Path("/user/temp/.samzaStaging/app_123"))
+    clientHelper.jobContext = jobContext
+
+    val ret = clientHelper.getSecurityYarnConfig
+
+    assert(ret.size == 2)
+    assert(ret.get(YarnConfig.YARN_JOB_STAGING_DIRECTORY) == Some("/user/temp/.samzaStaging/app_123"))
+    assert(ret.get(YarnConfig.YARN_CREDENTIALS_FILE) == Some("/user/temp/.samzaStaging/app_123/credentials"))
+  }
+
+  test("test setupAMLocalResources") {
+    val applicationId = mock[ApplicationId]
+    when(applicationId.toString).thenReturn("application_123")
+    val jobContext = new JobContext
+    jobContext.setAppId(applicationId)
+    clientHelper.jobContext = jobContext
+
+    val mockFs = mock[FileSystem]
+    val fileStatus = new FileStatus(0, false, 0, 0, System.currentTimeMillis(), null)
+
+    when(mockFs.getHomeDirectory).thenReturn(new Path("/user/test"))
+    when(mockFs.getFileStatus(any[Path])).thenReturn(fileStatus)
+    when(mockFs.mkdirs(any[Path])).thenReturn(true)
+
+    doNothing().when(mockFs).copyFromLocalFile(any[Path], any[Path])
+    doNothing().when(mockFs).setPermission(any[Path], any[FsPermission])
+
+    val ret = clientHelper.setupAMLocalResources(mockFs, Some("some.principal"), Some("some.keytab"))
+
+    assert(ret.size == 1)
+    assert(ret.contains("some.keytab"))
+  }
+
+  test("test toAppStatus") {
+    val appReport = mock[ApplicationReport]
+    when(appReport.getYarnApplicationState).thenReturn(YarnApplicationState.FAILED)
+    when(appReport.getDiagnostics).thenReturn("some yarn diagnostics")
+
+    var appStatus = clientHelper.toAppStatus(appReport).get
+    assertEquals(appStatus, ApplicationStatus.UnsuccessfulFinish)
+    assertNotNull(appStatus.getThrowable)
+
+    when(appReport.getYarnApplicationState).thenReturn(YarnApplicationState.NEW)
+    appStatus = clientHelper.toAppStatus(appReport).get
+    assertEquals(appStatus, ApplicationStatus.New)
+
+    when(appReport.getYarnApplicationState).thenReturn(YarnApplicationState.FINISHED)
+    when(appReport.getFinalApplicationStatus).thenReturn(FinalApplicationStatus.FAILED)
+    appStatus = clientHelper.toAppStatus(appReport).get
+    assertEquals(appStatus, ApplicationStatus.UnsuccessfulFinish)
+
+    when(appReport.getFinalApplicationStatus).thenReturn(FinalApplicationStatus.KILLED)
+    appStatus = clientHelper.toAppStatus(appReport).get
+    assertEquals(appStatus, ApplicationStatus.UnsuccessfulFinish)
+
+    when(appReport.getFinalApplicationStatus).thenReturn(FinalApplicationStatus.SUCCEEDED)
+    appStatus = clientHelper.toAppStatus(appReport).get
+    assertEquals(appStatus, ApplicationStatus.SuccessfulFinish)
+
+    val appMasterMetrics =  new java.util.HashMap[String, Object]()
+    val metrics = new java.util.HashMap[String, java.util.Map[String, Object]]()
+    metrics.put(classOf[SamzaAppMasterMetrics].getCanonicalName(), appMasterMetrics)
+    appMasterMetrics.put("needed-containers", "1")
+    when(mockAmClient.getMetrics).thenReturn(metrics)
+
+    when(appReport.getYarnApplicationState).thenReturn(YarnApplicationState.RUNNING)
+    appStatus = clientHelper.toAppStatus(appReport).get
+    assertEquals(appStatus, ApplicationStatus.New) // Should not be RUNNING if there are still needed containers
+
+    appMasterMetrics.put("needed-containers", "0")
+    when(mockAmClient.getMetrics).thenReturn(metrics)
+
+    when(appReport.getYarnApplicationState).thenReturn(YarnApplicationState.RUNNING)
+    appStatus = clientHelper.toAppStatus(appReport).get
+    assertEquals(appStatus, ApplicationStatus.Running) // Should not be RUNNING if there are still needed containers
+  }
+}
diff --git a/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala b/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala
new file mode 100644
index 000000000..7491d2aaf
--- /dev/null
+++ b/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.job.yarn
+
+import java.net.URL
+import java.nio.ByteBuffer
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler
+import org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes
+import org.apache.hadoop.yarn.util.ConverterUtils
+import org.apache.samza.SamzaException
+import org.apache.samza.clustermanager.SamzaApplicationState
+import org.apache.samza.clustermanager.SamzaApplicationState.SamzaAppStatus
+import org.apache.samza.coordinator.JobModelManager
+import org.junit.Assert._
+import org.junit.Test
+import org.mockito.{ArgumentCaptor, Mockito}
+
+class TestSamzaYarnAppMasterLifecycle {
+  private def YARN_CONTAINER_ID = "container_123_123_123"
+  private def YARN_CONTAINER_HOST = "host"
+  private def YARN_CONTAINER_MEM = 1024
+  private def YARN_CONTAINER_VCORE = 1
+  val coordinator = new JobModelManager(null, null)
+  val amClient = new AMRMClientAsyncImpl[ContainerRequest](1, Mockito.mock(classOf[CallbackHandler])) {
+    var host = ""
+    var port = 0
+    var status: FinalApplicationStatus = null
+    override def registerApplicationMaster(appHostName: String, appHostPort: Int, appTrackingUrl: String): RegisterApplicationMasterResponse = {
+      this.host = appHostName
+      this.port = appHostPort
+      new RegisterApplicationMasterResponse {
+        override def setApplicationACLs(map: java.util.Map[ApplicationAccessType, String]): Unit = ()
+        override def getApplicationACLs = null
+        override def setResourceTypes(types: java.util.List[org.apache.hadoop.yarn.api.records.ResourceTypeInfo]): Unit = ()
+        override def getResourceTypes = null
+        override def setMaximumResourceCapability(r: Resource): Unit = ()
+        override def getMaximumResourceCapability = new Resource {
+          def getMemory = 512
+          def getVirtualCores = 2
+          def setMemory(memory: Int) {}
+          def setVirtualCores(vCores: Int) {}
+          override def compareTo(o: Resource) = 0
+        }
+        override def getClientToAMTokenMasterKey = null
+        override def setClientToAMTokenMasterKey(buffer: ByteBuffer) {}
+        // to test AM high availability - return a running container from previous attempt
+        val prevAttemptCotainers = new java.util.ArrayList[Container]()
+        prevAttemptCotainers.add(getMockContainer)
+        override def getContainersFromPreviousAttempts(): java.util.List[Container] = prevAttemptCotainers
+        override def getNMTokensFromPreviousAttempts(): java.util.List[NMToken] = java.util.Collections.emptyList[NMToken]
+        override def getQueue(): String = null
+        override def setContainersFromPreviousAttempts(containers: java.util.List[Container]): Unit = Unit
+        override def setNMTokensFromPreviousAttempts(nmTokens: java.util.List[NMToken]): Unit = Unit
+        override def setQueue(queue: String): Unit = Unit
+
+        override def setSchedulerResourceTypes(types: java.util.EnumSet[SchedulerResourceTypes]): Unit = {}
+        override def getSchedulerResourceTypes: java.util.EnumSet[SchedulerResourceTypes] = null
+        override def getResourceProfiles(): java.util.Map[String,org.apache.hadoop.yarn.api.records.Resource] = ???
+        override def setResourceProfiles(x$1: java.util.Map[String,org.apache.hadoop.yarn.api.records.Resource]): Unit = ???
+      }
+    }
+    override def unregisterApplicationMaster(appStatus: FinalApplicationStatus,
+      appMessage: String,
+      appTrackingUrl: String) {
+      this.status = appStatus
+    }
+    override def releaseAssignedContainer(containerId: ContainerId) {}
+    override def getClusterNodeCount() = 1
+
+    override def serviceInit(config: Configuration) {}
+    override def serviceStart() {}
+    override def serviceStop() {}
+  }
+
+  @Test
+  def testLifecycleShouldRegisterOnInit {
+    val state = new SamzaApplicationState(coordinator)
+
+    val yarnState = new YarnAppState(1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "testHost", 1, 2);
+    yarnState.rpcUrl = new URL("http://localhost:1")
+    yarnState.trackingUrl = new URL("http://localhost:2")
+
+    val saml = new SamzaYarnAppMasterLifecycle(512, 2, state, yarnState, amClient, false)
+    saml.onInit
+    assertEquals("testHost", amClient.host)
+    assertEquals(1, amClient.port)
+    assertFalse(saml.shouldShutdown)
+  }
+
+  @Test
+  def testLifecycleShouldUnregisterOnShutdown {
+    val state = new SamzaApplicationState(coordinator)
+
+    val yarnState =  new YarnAppState(1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "testHost", 1, 2);
+    new SamzaYarnAppMasterLifecycle(512, 2, state, yarnState, amClient, false).onShutdown (SamzaAppStatus.SUCCEEDED)
+    assertEquals(FinalApplicationStatus.SUCCEEDED, amClient.status)
+  }
+
+  @Test
+  def testLifecycleShouldThrowAnExceptionOnReboot {
+    var gotException = false
+    try {
+      val state = new SamzaApplicationState(coordinator)
+
+      val yarnState =  new YarnAppState(1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "testHost", 1, 2);
+      new SamzaYarnAppMasterLifecycle(512, 2, state, yarnState, amClient, false).onReboot()
+    } catch {
+      // expected
+      case e: SamzaException => gotException = true
+    }
+    assertTrue(gotException)
+  }
+
+  @Test
+  def testLifecycleShouldShutdownOnInvalidContainerSettings {
+    val state = new SamzaApplicationState(coordinator)
+
+    val yarnState =  new YarnAppState(1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "testHost", 1, 2);
+    yarnState.rpcUrl = new URL("http://localhost:1")
+    yarnState.trackingUrl = new URL("http://localhost:2")
+
+    //Request a higher amount of memory from yarn.
+    List(new SamzaYarnAppMasterLifecycle(768, 1, state, yarnState, amClient, false),
+    //Request a higher number of cores from yarn.
+      new SamzaYarnAppMasterLifecycle(368, 3, state, yarnState, amClient, false)).map(saml => {
+        saml.onInit
+        assertTrue(saml.shouldShutdown)
+      })
+  }
+
+  @Test
+  def testAMHighAvailabilityOnInit {
+    val PROCESSOR_ID = "0"
+    val samzaApplicationState = new SamzaApplicationState(coordinator)
+
+    samzaApplicationState.processorToExecutionId.put(PROCESSOR_ID, YARN_CONTAINER_ID);
+
+    val yarnState = new YarnAppState(1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000001"), "testHost", 1, 2);
+    yarnState.rpcUrl = new URL("http://localhost:1")
+    yarnState.trackingUrl = new URL("http://localhost:2")
+
+    val saml = new SamzaYarnAppMasterLifecycle(512, 2, samzaApplicationState, yarnState, amClient, true)
+    saml.onInit
+
+    // verify that the samzaApplicationState is updated to reflect a running container from previous attempt
+    assertEquals(1, samzaApplicationState.pendingProcessors.size())
+    assertTrue(samzaApplicationState.pendingProcessors.containsKey(PROCESSOR_ID))
+    val resource = samzaApplicationState.pendingProcessors.get(PROCESSOR_ID)
+    assertEquals(YARN_CONTAINER_ID, resource.getContainerId)
+    assertEquals(YARN_CONTAINER_HOST, resource.getHost)
+    assertEquals(YARN_CONTAINER_MEM, resource.getMemoryMb)
+    assertEquals(YARN_CONTAINER_VCORE, resource.getNumCores)
+
+    assertEquals(1, yarnState.pendingProcessors.size())
+    assertTrue(yarnState.pendingProcessors.containsKey(PROCESSOR_ID))
+    val yarnCtr = yarnState.pendingProcessors.get(PROCESSOR_ID)
+    assertEquals(YARN_CONTAINER_ID, yarnCtr.id.toString)
+    assertEquals(YARN_CONTAINER_HOST, yarnCtr.nodeId.getHost)
+    assertEquals(YARN_CONTAINER_MEM, yarnCtr.resource.getMemory)
+    assertEquals(YARN_CONTAINER_VCORE, yarnCtr.resource.getVirtualCores)
+  }
+
+  def getMockContainer: Container = {
+    val container = Mockito.mock(classOf[Container])
+
+    val containerId = Mockito.mock(classOf[ContainerId])
+    Mockito.when(containerId.toString).thenReturn(YARN_CONTAINER_ID)
+    Mockito.when(container.getId).thenReturn(containerId)
+
+    val resource = Mockito.mock(classOf[Resource])
+    Mockito.when(resource.getMemory).thenReturn(YARN_CONTAINER_MEM)
+    Mockito.when(resource.getVirtualCores).thenReturn(YARN_CONTAINER_VCORE)
+    Mockito.when(container.getResource).thenReturn(resource)
+
+    val nodeId = Mockito.mock(classOf[NodeId])
+    Mockito.when(nodeId.getHost).thenReturn(YARN_CONTAINER_HOST)
+    Mockito.when(container.getNodeId).thenReturn(nodeId)
+
+    container
+  }
+}
diff --git a/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterService.scala b/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterService.scala
new file mode 100644
index 000000000..f43b991a4
--- /dev/null
+++ b/samza-yarn3/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterService.scala
@@ -0,0 +1,131 @@
+/*
+ * 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.samza.job.yarn
+
+import java.io.{BufferedReader, InputStreamReader}
+import java.net.URL
+
+import org.apache.hadoop.yarn.util.ConverterUtils
+import org.apache.samza.clustermanager.{ClusterBasedJobCoordinator, SamzaApplicationState}
+import org.apache.samza.config.{Config, MapConfig}
+import org.apache.samza.coordinator.JobModelManager
+import org.apache.samza.coordinator.metadatastore.{CoordinatorStreamStore, CoordinatorStreamStoreTestUtil, NamespaceAwareCoordinatorStreamStore}
+import org.apache.samza.coordinator.stream.messages.SetChangelogMapping
+import org.apache.samza.coordinator.stream.MockCoordinatorStreamSystemFactory
+import org.apache.samza.metrics._
+import org.apache.samza.storage.ChangelogStreamManager
+import org.junit.Assert._
+import org.junit.Test
+
+import scala.collection.JavaConverters._
+
+class TestSamzaYarnAppMasterService {
+
+  @Test
+  def testAppMasterDashboardShouldStart {
+    MockCoordinatorStreamSystemFactory.enableMockConsumerCache()
+    val config = getDummyConfig
+    val jobModelManager = getTestJobModelManager(config)
+    val samzaState = new SamzaApplicationState(jobModelManager)
+    val registry = new MetricsRegistryMap()
+
+    val state = new YarnAppState(-1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000002"), "testHost", 1, 1)
+    val service = new SamzaYarnAppMasterService(config, samzaState, state, registry, null)
+
+    // start the dashboard
+    service.onInit
+    assertTrue(state.rpcUrl.getPort > 0)
+    assertTrue(state.trackingUrl.getPort > 0)
+    assertTrue(state.coordinatorUrl.getPort > 0)
+
+    // check to see if it's running
+    val url = new URL(state.rpcUrl.toString + "am")
+    val is = url.openConnection().getInputStream()
+    val reader = new BufferedReader(new InputStreamReader(is))
+    var line: String = null
+
+    do {
+      line = reader.readLine()
+    } while (line != null)
+
+    reader.close()
+  }
+
+  /**
+   * This tests the rendering of the index.scaml file containing some Scala code. The objective
+   * is to ensure that the rendered scala code builds correctly
+   */
+  @Test
+  def testAppMasterDashboardWebServiceShouldStart {
+    // Create some dummy config
+    val config = getDummyConfig
+    val jobModelManager = getTestJobModelManager(config)
+    val samzaState = new SamzaApplicationState(jobModelManager)
+    val state = new YarnAppState(-1, ConverterUtils.toContainerId("container_1350670447861_0003_01_000002"), "testHost", 1, 1)
+    val registry = new MetricsRegistryMap()
+
+    val service = new SamzaYarnAppMasterService(config, samzaState, state, registry, null)
+
+    // start the dashboard
+    service.onInit
+    assertTrue(state.rpcUrl.getPort > 0)
+    assertTrue(state.trackingUrl.getPort > 0)
+
+    // Do a GET Request on the tracking port: This in turn will render index.scaml
+    val url = state.trackingUrl
+    val is = url.openConnection().getInputStream()
+    val reader = new BufferedReader(new InputStreamReader(is))
+    var line: String = null
+
+    do {
+      line = reader.readLine()
+    } while (line != null)
+
+    reader.close
+  }
+
+  private def getTestJobModelManager(config: Config) = {
+    val coordinatorStreamTestUtil: CoordinatorStreamStoreTestUtil = new CoordinatorStreamStoreTestUtil(config)
+    val coordinatorStreamStore: CoordinatorStreamStore = coordinatorStreamTestUtil.getCoordinatorStreamStore
+    val namespaceAwareCoordinatorStore: NamespaceAwareCoordinatorStreamStore = new NamespaceAwareCoordinatorStreamStore(coordinatorStreamStore, SetChangelogMapping.TYPE)
+    try {
+      val changelogPartitionManager = new ChangelogStreamManager(namespaceAwareCoordinatorStore)
+      val jobModelManager = JobModelManager(getDummyConfig, changelogPartitionManager.readPartitionMapping(),
+        coordinatorStreamStore, new MetricsRegistryMap())
+      jobModelManager
+    } finally {
+      coordinatorStreamStore.close()
+    }
+  }
+
+  private def getDummyConfig: Config = new MapConfig(Map[String, String](
+    "job.name" -> "test",
+    "yarn.container.count" -> "1",
+    "systems.test-system.samza.factory" -> "org.apache.samza.job.yarn.MockSystemFactory",
+    "yarn.container.memory.mb" -> "512",
+    "yarn.package.path" -> "/foo",
+    "task.inputs" -> "test-system.test-stream",
+    "systems.test-system.samza.key.serde" -> "org.apache.samza.serializers.JsonSerde",
+    "systems.test-system.samza.msg.serde" -> "org.apache.samza.serializers.JsonSerde",
+    "yarn.container.retry.count" -> "1",
+    "yarn.container.retry.window.ms" -> "1999999999",
+    "job.coordinator.system" -> "coordinator",
+    "systems.coordinator.samza.factory" -> classOf[MockCoordinatorStreamSystemFactory].getCanonicalName).asJava)
+}
\ No newline at end of file
diff --git a/settings.gradle b/settings.gradle
index cf4c9be7a..a29de6aec 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -20,7 +20,8 @@
 include \
   'samza-api',
   'samza-sql',
-  'samza-shell'
+  'samza-shell',
+  'samza-shell-yarn3'
 
 def scalaModules = [
         'samza-aws',
@@ -39,8 +40,9 @@ def scalaModules = [
         'samza-sql',
         'samza-sql-shell',
         'samza-tools',
-        'samza-yarn',
         'samza-test',
+        'samza-yarn',
+        'samza-yarn3'
 ] as HashSet
 
 scalaModules.each {