From 5d6e588fa33844fbe53f96a2a5733a7166420739 Mon Sep 17 00:00:00 2001 From: James DeMichele Date: Thu, 18 Aug 2022 11:11:52 -0500 Subject: [PATCH 01/18] Initial commit for getting Samza to work with Java 11 --- .../org/apache/samza/config/KafkaConfig.scala | 16 +++++++--------- samza-shell/src/main/bash/run-class.sh | 2 +- 2 files changed, 8 insertions(+), 10 deletions(-) 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 391536a7eb..dbd3bb4ffe 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-shell/src/main/bash/run-class.sh b/samza-shell/src/main/bash/run-class.sh index 3de6dcf40f..315014365b 100755 --- a/samza-shell/src/main/bash/run-class.sh +++ b/samza-shell/src/main/bash/run-class.sh @@ -153,7 +153,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 From 1dbd8a0b017dd26960ce2947b9e1df1ff9bcfa06 Mon Sep 17 00:00:00 2001 From: James DeMichele Date: Thu, 18 Aug 2022 13:19:20 -0500 Subject: [PATCH 02/18] Upgrade Hadoop to 3.3.4 --- gradle.properties | 2 +- gradle/dependency-versions.gradle | 2 +- .../samza/system/hdfs/HdfsSystemAdmin.java | 3 ++- .../samza/system/hdfs/HdfsSystemConsumer.java | 4 ++-- .../partitioner/DirectoryPartitioner.java | 19 +++++++++---------- .../monitor/LocalStoreMonitorConfig.java | 3 ++- .../apache/samza/monitor/MonitorConfig.java | 3 ++- .../samza/rest/SamzaRestApplication.java | 4 +++- .../proxy/task/SamzaTaskProxyFactory.java | 3 ++- .../samza/rest/resources/TasksResource.java | 3 ++- .../job/yarn/LocalizerResourceConfig.java | 3 ++- .../apache/samza/job/yarn/ClientHelper.scala | 5 +++-- .../TestSamzaYarnAppMasterLifecycle.scala | 2 ++ 13 files changed, 33 insertions(+), 23 deletions(-) diff --git a/gradle.properties b/gradle.properties index cab76e044a..ae13170f7f 100644 --- a/gradle.properties +++ b/gradle.properties @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. group=org.apache.samza -version=1.7.0-SNAPSHOT +version=redfin-java11 scalaSuffix=2.12 # after changing this value, run `$ ./gradlew wrapper` and commit the resulting changed files diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index 79f30f83ae..b15312b510 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -45,7 +45,7 @@ rocksdbVersion = "7.0.3" scalaTestVersion = "3.0.1" slf4jVersion = "1.7.7" - yarnVersion = "2.10.1" + yarnVersion = "3.3.4" zkClientVersion = "0.11" zookeeperVersion = "3.4.13" failsafeVersion = "2.4.0" 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 7ffbfc7a28..cc7bda2b89 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.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 1ceb5d61c6..10165583fc 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.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 82445043ec..4f07ef402a 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.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-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitorConfig.java b/samza-rest/src/main/java/org/apache/samza/monitor/LocalStoreMonitorConfig.java index 6923f2f6f7..22eee7730d 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 @@ 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 2291258143..786205544b 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.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 7b4cd68a60..c1f58f6b44 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 d36e8b1ef5..4e72e1e88a 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 12db787cf0..d342b9a328 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.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-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 ca94783e6e..bfc3d809f4 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 @@ 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 8e0c3d1b27..207eb6b12a 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-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala index 67e018a44f..7491d2aafc 100644 --- a/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala +++ b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala @@ -78,6 +78,8 @@ class TestSamzaYarnAppMasterLifecycle { 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, From e67cc596a4e095750a6f9da7352cdf0fb2110f8d Mon Sep 17 00:00:00 2001 From: James DeMichele Date: Thu, 18 Aug 2022 15:45:44 -0500 Subject: [PATCH 03/18] Fix test --- .../job/yarn/TestYarnFaultDomainManager.java | 29 ++++++++++++------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java b/samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java index 9216088da2..650a466c24 100644 --- a/samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java +++ b/samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java @@ -29,6 +29,7 @@ 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.NodeUpdateType; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl; import org.apache.hadoop.yarn.exceptions.YarnException; @@ -60,22 +61,22 @@ public class TestYarnFaultDomainManager { private final NodeReport nodeReport1 = createNodeReport(hostName1, 1, NodeState.RUNNING, "httpAddress1", rackName1, 1, 1, 2, 1, 2, - "", 60L, null); + "", 60L); private final NodeReport nodeReport2 = createNodeReport(hostName2, 1, NodeState.RUNNING, "httpAddress2", rackName2, 1, 1, 2, 1, 2, - "", 60L, null); + "", 60L); private final NodeReport nodeReport3 = createNodeReport(hostName3, 1, NodeState.RUNNING, "httpAddress3", rackName1, 1, 1, 2, 1, 2, - "", 60L, null); + "", 60L); private final NodeReport nodeReport4 = createNodeReport(hostName4, 1, NodeState.RUNNING, "httpAddress4", rackName2, 1, 1, 2, 1, 2, - "", 60L, null); + "", 60L); private final NodeReport nodeReport5 = createNodeReport(hostName5, 1, NodeState.RUNNING, "httpAddress5", rackName3, 1, 1, 2, 1, 2, - "", 60L, null); + "", 60L); private final NodeReport nodeReport6 = createNodeReport(hostName6, 1, NodeState.RUNNING, "httpAddress6", rackName1, 1, 1, 2, 1, 2, - "", 60L, null); + "", 60L); @Mock YarnClientImpl yarnClient; @@ -182,9 +183,17 @@ private void assertFaultDomainEquals(FaultDomain faultDomain1, FaultDomain fault 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, Set nodeLabels) { - return NodeReport.newInstance(NodeId.newInstance(host, port), nodeState, httpAddress, rackName, - Resource.newInstance(memoryUsed, vcoresUsed), Resource.newInstance(totalMemory, totalVcores), numContainers, - healthReport, lastHealthReportTime, nodeLabels); + 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 + ); } } From 332ca88178156b8049c01bd58222fbe8df5320d6 Mon Sep 17 00:00:00 2001 From: Jamie DeMichele Date: Thu, 18 Aug 2022 16:13:34 -0500 Subject: [PATCH 04/18] Update gradle.properties --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index ae13170f7f..cab76e044a 100644 --- a/gradle.properties +++ b/gradle.properties @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. group=org.apache.samza -version=redfin-java11 +version=1.7.0-SNAPSHOT scalaSuffix=2.12 # after changing this value, run `$ ./gradlew wrapper` and commit the resulting changed files From 2cd5b5aec7cd7a67fc4dfe8a76cde3e83ca00986 Mon Sep 17 00:00:00 2001 From: James DeMichele Date: Tue, 20 Sep 2022 09:53:30 -0500 Subject: [PATCH 05/18] Move yarn back to 2.10.1 by default. Make changes for some failing issues in tests with Java 11 --- README.md | 7 +++++-- build.gradle | 1 + gradle/dependency-versions.gradle | 3 ++- .../remote/couchbase/TestCouchbaseBucketRegistry.java | 2 ++ .../remote/couchbase/TestCouchbaseTableReadFunction.java | 2 ++ .../remote/couchbase/TestCouchbaseTableWriteFunction.java | 2 ++ .../apache/samza/sql/translator/TestFilterTranslator.java | 2 ++ .../apache/samza/sql/translator/TestProjectTranslator.java | 2 ++ .../apache/samza/job/yarn/TestYarnFaultDomainManager.java | 1 - 9 files changed, 18 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index e5c91a4157..03a06015da 100644 --- a/README.md +++ b/README.md @@ -28,9 +28,12 @@ After the bootstrap script has completed, the regular gradlew instructions below #### 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 or 3.3.4. Scala 2.12 and Yarn 2.10.1 are used by default. Use the -PscalaSuffix and -PyarnVersion switches to change Scala or Yarn versions. Samza supports building Scala with 2.11 and 2.12. - ./gradlew -PscalaSuffix=2.11 clean build + ./gradlew -PscalaSuffix=2.12 -PyarnVersion=2.10.1 clean build + ./gradlew -PscalaSuffix=2.12 -PyarnVersion=3.3.4 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 2ed9b921ef..705261eaaa 100644 --- a/build.gradle +++ b/build.gradle @@ -710,6 +710,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/dependency-versions.gradle b/gradle/dependency-versions.gradle index b15312b510..0745501c47 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -45,7 +45,8 @@ rocksdbVersion = "7.0.3" scalaTestVersion = "3.0.1" slf4jVersion = "1.7.7" - yarnVersion = "3.3.4" + yarnVersion = "2.10.1" + //yarnVersion = "3.3.4" zkClientVersion = "0.11" zookeeperVersion = "3.4.13" failsafeVersion = "2.4.0" diff --git a/samza-kv-couchbase/src/test/java/org/apache/samza/table/remote/couchbase/TestCouchbaseBucketRegistry.java b/samza-kv-couchbase/src/test/java/org/apache/samza/table/remote/couchbase/TestCouchbaseBucketRegistry.java index e2c5ab1fa0..6692a93644 100644 --- a/samza-kv-couchbase/src/test/java/org/apache/samza/table/remote/couchbase/TestCouchbaseBucketRegistry.java +++ b/samza-kv-couchbase/src/test/java/org/apache/samza/table/remote/couchbase/TestCouchbaseBucketRegistry.java @@ -26,6 +26,7 @@ import java.util.List; import org.junit.Test; import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; @@ -39,6 +40,7 @@ */ @RunWith(PowerMockRunner.class) @PrepareForTest(CouchbaseCluster.class) +@PowerMockIgnore("jdk.internal.reflect.*") public class TestCouchbaseBucketRegistry { /** diff --git a/samza-kv-couchbase/src/test/java/org/apache/samza/table/remote/couchbase/TestCouchbaseTableReadFunction.java b/samza-kv-couchbase/src/test/java/org/apache/samza/table/remote/couchbase/TestCouchbaseTableReadFunction.java index f1b0491268..09ac87ce32 100644 --- a/samza-kv-couchbase/src/test/java/org/apache/samza/table/remote/couchbase/TestCouchbaseTableReadFunction.java +++ b/samza-kv-couchbase/src/test/java/org/apache/samza/table/remote/couchbase/TestCouchbaseTableReadFunction.java @@ -42,6 +42,7 @@ import org.junit.runner.RunWith; import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; @@ -57,6 +58,7 @@ */ @RunWith(PowerMockRunner.class) @PrepareForTest(CouchbaseBucketRegistry.class) +@PowerMockIgnore("jdk.internal.reflect.*") public class TestCouchbaseTableReadFunction { private static final String DEFAULT_BUCKET_NAME = "default-bucket-name"; private static final String DEFAULT_CLUSTER_NODE = "localhost"; diff --git a/samza-kv-couchbase/src/test/java/org/apache/samza/table/remote/couchbase/TestCouchbaseTableWriteFunction.java b/samza-kv-couchbase/src/test/java/org/apache/samza/table/remote/couchbase/TestCouchbaseTableWriteFunction.java index 5a4e3c1048..4bba8f7681 100644 --- a/samza-kv-couchbase/src/test/java/org/apache/samza/table/remote/couchbase/TestCouchbaseTableWriteFunction.java +++ b/samza-kv-couchbase/src/test/java/org/apache/samza/table/remote/couchbase/TestCouchbaseTableWriteFunction.java @@ -40,6 +40,7 @@ import org.junit.runner.RunWith; import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; @@ -55,6 +56,7 @@ */ @RunWith(PowerMockRunner.class) @PrepareForTest(CouchbaseBucketRegistry.class) +@PowerMockIgnore("jdk.internal.reflect.*") public class TestCouchbaseTableWriteFunction { private static final String DEFAULT_BUCKET_NAME = "default-bucket-name"; private static final String DEFAULT_CLUSTER_NODE = "localhost"; diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java index 396192d1d8..973b5d5505 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestFilterTranslator.java @@ -45,6 +45,7 @@ import org.junit.runner.RunWith; import org.mockito.internal.util.reflection.Whitebox; import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; @@ -65,6 +66,7 @@ */ @RunWith(PowerMockRunner.class) @PrepareForTest(LogicalFilter.class) +@PowerMockIgnore("jdk.internal.reflect.*") public class TestFilterTranslator extends TranslatorTestBase { private static final String LOGICAL_OP_ID = "sql0_filter_0"; diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java index dc193a2c8d..9d48cdb582 100644 --- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java +++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestProjectTranslator.java @@ -49,6 +49,7 @@ import org.junit.runner.RunWith; import org.mockito.internal.util.reflection.Whitebox; import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; import static org.junit.Assert.*; @@ -66,6 +67,7 @@ */ @RunWith(PowerMockRunner.class) @PrepareForTest(LogicalProject.class) +@PowerMockIgnore("jdk.internal.reflect.*") public class TestProjectTranslator extends TranslatorTestBase { private static final String LOGICAL_OP_ID = "sql0_project_0"; diff --git a/samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java b/samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java index 650a466c24..76d6d88426 100644 --- a/samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java +++ b/samza-yarn/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java @@ -29,7 +29,6 @@ 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.NodeUpdateType; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl; import org.apache.hadoop.yarn.exceptions.YarnException; From 380486d49686d65d1dff4a39b38028bf5762d5c0 Mon Sep 17 00:00:00 2001 From: James DeMichele Date: Tue, 20 Sep 2022 16:23:14 -0500 Subject: [PATCH 06/18] Upgrade gradle, remove yarn 3 change --- gradle.properties | 2 +- gradle/wrapper/gradle-wrapper.properties | 2 +- .../samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/gradle.properties b/gradle.properties index cab76e044a..5d3b1486e6 100644 --- a/gradle.properties +++ b/gradle.properties @@ -19,7 +19,7 @@ version=1.7.0-SNAPSHOT scalaSuffix=2.12 # 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/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 44e7c4d1d7..ec991f9aa1 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-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala index 7491d2aafc..739e5b1f3f 100644 --- a/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala +++ b/samza-yarn/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala @@ -78,8 +78,8 @@ class TestSamzaYarnAppMasterLifecycle { 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 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, From bb5080b2856fbcd341415a20d774e34943b0565c Mon Sep 17 00:00:00 2001 From: James DeMichele Date: Tue, 20 Sep 2022 22:03:07 -0500 Subject: [PATCH 07/18] Make a samza-yarn3 project that can be included given the right yarn version --- build.gradle | 9 +- gradle.properties | 3 + gradle/dependency-versions-yarn-2.gradle | 22 + gradle/dependency-versions-yarn-3.gradle | 22 + gradle/dependency-versions.gradle | 3 +- .../org/apache/samza/config/YarnConfig.java | 251 ++++++ .../samza/job/yarn/FileSystemImplConfig.java | 71 ++ .../job/yarn/LocalizerResourceConfig.java | 103 +++ .../job/yarn/LocalizerResourceException.java | 72 ++ .../job/yarn/LocalizerResourceMapper.java | 101 +++ .../apache/samza/job/yarn/YarnAppState.java | 138 ++++ .../job/yarn/YarnClusterResourceManager.java | 774 ++++++++++++++++++ .../job/yarn/YarnFaultDomainManager.java | 131 +++ .../yarn/YarnFaultDomainManagerFactory.java | 34 + .../job/yarn/YarnResourceManagerFactory.java | 44 + .../validation/YarnJobValidationTool.java | 207 +++++ .../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 ++ .../scalate/WEB-INF/views/index.scaml | 239 ++++++ .../resources/scalate/css/bootstrap.min.css | 7 + .../scalate/css/font-awesome.min.css | 4 + .../main/resources/scalate/css/ropa-sans.css | 6 + .../resources/scalate/fonts/FontAwesome.otf | Bin 0 -> 62856 bytes .../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 .../fonts/glyphicons-halflings-regular.eot | Bin 0 -> 20335 bytes .../fonts/glyphicons-halflings-regular.svg | 229 ++++++ .../fonts/glyphicons-halflings-regular.ttf | Bin 0 -> 41280 bytes .../fonts/glyphicons-halflings-regular.woff | Bin 0 -> 23320 bytes .../src/main/resources/scalate/img/asc.gif | Bin 0 -> 54 bytes .../src/main/resources/scalate/img/bg.gif | Bin 0 -> 64 bytes .../src/main/resources/scalate/img/desc.gif | Bin 0 -> 54 bytes .../main/resources/scalate/img/samza-icon.png | Bin 0 -> 7722 bytes .../resources/scalate/js/bootstrap.min.js | 6 + .../resources/scalate/js/jquery-1.11.1.min.js | 4 + .../scalate/js/jquery.tablesorter.min.js | 4 + .../apache/samza/job/yarn/ClientHelper.scala | 450 ++++++++++ .../job/yarn/SamzaAppMasterMetrics.scala | 67 ++ .../yarn/SamzaAppMasterSecurityManager.scala | 126 +++ .../yarn/SamzaContainerSecurityManager.scala | 102 +++ .../yarn/SamzaYarnAppMasterLifecycle.scala | 121 +++ .../job/yarn/SamzaYarnAppMasterService.scala | 103 +++ .../SamzaYarnSecurityManagerFactory.scala | 30 + .../job/yarn/YarnAppMasterListener.scala | 68 ++ .../apache/samza/job/yarn/YarnContainer.scala | 47 ++ .../org/apache/samza/job/yarn/YarnJob.scala | 191 +++++ .../samza/job/yarn/YarnJobFactory.scala | 55 ++ .../apache/samza/job/yarn/YarnJobUtil.scala | 109 +++ .../samza/util/hadoop/HttpFileSystem.scala | 107 +++ .../samza/util/hadoop/HttpInputStream.scala | 46 ++ .../webapp/ApplicationMasterRestServlet.scala | 139 ++++ .../webapp/ApplicationMasterWebServlet.scala | 47 ++ .../apache/samza/config/TestYarnConfig.java | 40 + .../job/yarn/TestFileSystemImplConfig.java | 84 ++ .../job/yarn/TestLocalizerResourceConfig.java | 125 +++ .../job/yarn/TestLocalizerResourceMapper.java | 174 ++++ .../yarn/TestYarnClusterResourceManager.java | 253 ++++++ .../job/yarn/TestYarnFaultDomainManager.java | 198 +++++ .../apache/samza/job/yarn/TestYarnJob.java | 114 +++ .../samza/job/yarn/TestYarnJobFactory.java | 59 ++ .../job/yarn/util/MockContainerListener.java | 147 ++++ .../samza/job/yarn/util/MockHttpServer.java | 55 ++ .../samza/job/yarn/util/MockNMClient.java | 68 ++ .../job/yarn/util/TestAMRMClientImpl.java | 97 +++ .../yarn/util/hadoop/TestHttpFileSystem.java | 161 ++++ .../validation/MockMetricsValidator.java | 52 ++ .../validation/TestYarnJobValidationTool.java | 142 ++++ .../TestApplicationMasterRestClient.java | 331 ++++++++ .../samza/webapp/TestLocalityServlet.java | 131 +++ .../TestYarnContainerHeartbeatServlet.java | 99 +++ .../samza/job/yarn/MockSystemAdmin.scala | 42 + .../samza/job/yarn/MockSystemFactory.scala | 43 + .../samza/job/yarn/TestClientHelper.scala | 160 ++++ .../TestSamzaYarnAppMasterLifecycle.scala | 204 +++++ .../yarn/TestSamzaYarnAppMasterService.scala | 131 +++ settings.gradle | 8 +- 82 files changed, 8052 insertions(+), 7 deletions(-) create mode 100644 gradle/dependency-versions-yarn-2.gradle create mode 100644 gradle/dependency-versions-yarn-3.gradle create mode 100644 samza-yarn3/src/main/java/org/apache/samza/config/YarnConfig.java create mode 100644 samza-yarn3/src/main/java/org/apache/samza/job/yarn/FileSystemImplConfig.java create mode 100644 samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java create mode 100644 samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceException.java create mode 100644 samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceMapper.java create mode 100644 samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnAppState.java create mode 100644 samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnClusterResourceManager.java create mode 100644 samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnFaultDomainManager.java create mode 100644 samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnFaultDomainManagerFactory.java create mode 100644 samza-yarn3/src/main/java/org/apache/samza/job/yarn/YarnResourceManagerFactory.java create mode 100644 samza-yarn3/src/main/java/org/apache/samza/validation/YarnJobValidationTool.java create mode 100644 samza-yarn3/src/main/java/org/apache/samza/webapp/ApplicationMasterRestClient.java create mode 100644 samza-yarn3/src/main/java/org/apache/samza/webapp/YarnContainerHeartbeatServlet.java create mode 100644 samza-yarn3/src/main/less/main.less create mode 100644 samza-yarn3/src/main/less/variables.less create mode 100644 samza-yarn3/src/main/resources/scalate/WEB-INF/layouts/default.scaml create mode 100644 samza-yarn3/src/main/resources/scalate/WEB-INF/views/index.scaml create mode 100644 samza-yarn3/src/main/resources/scalate/css/bootstrap.min.css create mode 100644 samza-yarn3/src/main/resources/scalate/css/font-awesome.min.css create mode 100644 samza-yarn3/src/main/resources/scalate/css/ropa-sans.css create mode 100644 samza-yarn3/src/main/resources/scalate/fonts/FontAwesome.otf create mode 100644 samza-yarn3/src/main/resources/scalate/fonts/RopaSans-Regular-webfont.woff create mode 100755 samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.eot create mode 100755 samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.svg create mode 100755 samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.ttf create mode 100755 samza-yarn3/src/main/resources/scalate/fonts/fontawesome-webfont.woff create mode 100644 samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.eot create mode 100644 samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.svg create mode 100644 samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.ttf create mode 100644 samza-yarn3/src/main/resources/scalate/fonts/glyphicons-halflings-regular.woff create mode 100644 samza-yarn3/src/main/resources/scalate/img/asc.gif create mode 100644 samza-yarn3/src/main/resources/scalate/img/bg.gif create mode 100644 samza-yarn3/src/main/resources/scalate/img/desc.gif create mode 100644 samza-yarn3/src/main/resources/scalate/img/samza-icon.png create mode 100755 samza-yarn3/src/main/resources/scalate/js/bootstrap.min.js create mode 100644 samza-yarn3/src/main/resources/scalate/js/jquery-1.11.1.min.js create mode 100644 samza-yarn3/src/main/resources/scalate/js/jquery.tablesorter.min.js create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/job/yarn/ClientHelper.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterMetrics.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaAppMasterSecurityManager.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaContainerSecurityManager.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaYarnAppMasterLifecycle.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaYarnAppMasterService.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/job/yarn/SamzaYarnSecurityManagerFactory.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnAppMasterListener.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnContainer.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnJob.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnJobFactory.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/job/yarn/YarnJobUtil.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/util/hadoop/HttpFileSystem.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/util/hadoop/HttpInputStream.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/webapp/ApplicationMasterRestServlet.scala create mode 100644 samza-yarn3/src/main/scala/org/apache/samza/webapp/ApplicationMasterWebServlet.scala create mode 100644 samza-yarn3/src/test/java/org/apache/samza/config/TestYarnConfig.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestFileSystemImplConfig.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestLocalizerResourceConfig.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestLocalizerResourceMapper.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnClusterResourceManager.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnFaultDomainManager.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnJob.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/job/yarn/TestYarnJobFactory.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/MockContainerListener.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/MockHttpServer.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/MockNMClient.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/TestAMRMClientImpl.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/job/yarn/util/hadoop/TestHttpFileSystem.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/validation/MockMetricsValidator.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/validation/TestYarnJobValidationTool.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/webapp/TestApplicationMasterRestClient.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/webapp/TestLocalityServlet.java create mode 100644 samza-yarn3/src/test/java/org/apache/samza/webapp/TestYarnContainerHeartbeatServlet.java create mode 100644 samza-yarn3/src/test/scala/org/apache/samza/job/yarn/MockSystemAdmin.scala create mode 100644 samza-yarn3/src/test/scala/org/apache/samza/job/yarn/MockSystemFactory.scala create mode 100644 samza-yarn3/src/test/scala/org/apache/samza/job/yarn/TestClientHelper.scala create mode 100644 samza-yarn3/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterLifecycle.scala create mode 100644 samza-yarn3/src/test/scala/org/apache/samza/job/yarn/TestSamzaYarnAppMasterService.scala diff --git a/build.gradle b/build.gradle index 705261eaaa..9e447585d9 100644 --- a/build.gradle +++ b/build.gradle @@ -34,6 +34,7 @@ allprojects { apply from: file("gradle/dependency-versions.gradle") apply from: file("gradle/dependency-versions-scala-" + scalaSuffix + ".gradle") +apply from: file("gradle/dependency-versions-yarn-" + yarnVersionPrefix + ".gradle") apply from: file("gradle/release.gradle") apply from: file("gradle/rat.gradle") apply from: file('gradle/customize.gradle') @@ -453,7 +454,7 @@ project(":samza-log4j2_$scalaSuffix") { } } -project(":samza-yarn_$scalaSuffix") { +project(":${yarnProject}_$scalaSuffix") { apply plugin: 'scala' apply plugin: 'lesscss' @@ -534,7 +535,7 @@ project(":samza-shell") { gradleShell project(":samza-core_$scalaSuffix") gradleShell project(":samza-kafka_$scalaSuffix") gradleShell project(":samza-test_$scalaSuffix") - gradleShell project(":samza-yarn_$scalaSuffix") + gradleShell project(":${yarnProject}_$scalaSuffix") } task shellTarGz(type: Tar) { @@ -678,7 +679,7 @@ project(":samza-hdfs_$scalaSuffix") { // currently hdfs system producer/consumer do depend on yarn for two things: // 1. staging directory 2. security // SAMZA-1032 to solve the staging directory dependency - compile project(":samza-yarn_$scalaSuffix") + compile project(":${yarnProject}_$scalaSuffix") compile "org.scala-lang:scala-library:$scalaVersion" compile("org.apache.hadoop:hadoop-hdfs:$yarnVersion") { exclude module: 'servlet-api' @@ -788,7 +789,7 @@ project(":samza-test_$scalaSuffix") { compile project(":samza-kafka_$scalaSuffix") compile project(":samza-sql_$scalaSuffix") runtime project(":samza-log4j2_$scalaSuffix") - runtime project(":samza-yarn_$scalaSuffix") + runtime project(":${yarnProject}_$scalaSuffix") runtime project(":samza-hdfs_$scalaSuffix") compile "org.scala-lang:scala-library:$scalaVersion" compile "net.sf.jopt-simple:jopt-simple:$joptSimpleVersion" diff --git a/gradle.properties b/gradle.properties index 5d3b1486e6..f368e878c6 100644 --- a/gradle.properties +++ b/gradle.properties @@ -16,7 +16,10 @@ # under the License. group=org.apache.samza version=1.7.0-SNAPSHOT +# These 2 are ones that you can override using properties, like: +# -PscalaSuffix=2.12 -PyarnVersion=2.10.1 scalaSuffix=2.12 +yarnVersion=2.10.1 # after changing this value, run `$ ./gradlew wrapper` and commit the resulting changed files gradleVersion=6.9.2 diff --git a/gradle/dependency-versions-yarn-2.gradle b/gradle/dependency-versions-yarn-2.gradle new file mode 100644 index 0000000000..3900cbd6c5 --- /dev/null +++ b/gradle/dependency-versions-yarn-2.gradle @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +ext { + yarnVersion = "2.10.1" + yarnProject = "samza-yarn" +} diff --git a/gradle/dependency-versions-yarn-3.gradle b/gradle/dependency-versions-yarn-3.gradle new file mode 100644 index 0000000000..4d69287c16 --- /dev/null +++ b/gradle/dependency-versions-yarn-3.gradle @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +ext { + yarnVersion = "3.3.4" + yarnProject = "samza-yarn3" +} diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index 0745501c47..8a22c74e2d 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -45,8 +45,6 @@ rocksdbVersion = "7.0.3" scalaTestVersion = "3.0.1" slf4jVersion = "1.7.7" - yarnVersion = "2.10.1" - //yarnVersion = "3.3.4" zkClientVersion = "0.11" zookeeperVersion = "3.4.13" failsafeVersion = "2.4.0" @@ -54,4 +52,5 @@ jnaVersion = "4.5.1" couchbaseClientVersion = "2.7.2" couchbaseMockVersion = "1.5.22" + yarnVersionPrefix = yarnVersion.substring(0, yarnVersion.indexOf('.')) } 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 0000000000..4b37a8114f --- /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 getYarnApplicationAcls() { + Map 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 0000000000..a582dbbc5e --- /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.<scheme>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 getSchemes() { + Config subConfig = config.subset(FS_IMPL_PREFIX, true); + List schemes = new ArrayList(); + 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.<scheme>.impl + * It can include config for fs.<scheme>.impl and additional config for the subKeys fs.<scheme>.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-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java new file mode 100644 index 0000000000..bfc3d809f4 --- /dev/null +++ b/samza-yarn3/src/main/java/org/apache/samza/job/yarn/LocalizerResourceConfig.java @@ -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 java.util.ArrayList; +import java.util.List; + +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; +import org.apache.samza.config.Config; + +/** + * LocalizerResourceConfig is intended to manage/fetch the config values + * for the yarn localizer resource(s) from the configuration. + * + * There are 4 config values + * yarn.resources.<resourceName>.path + * (Required) The path for fetching the resource for localization, + * e.g. http://hostname.com/test. + * yarn.resources.<resourceName>.local.name + * (Optional) The local name used for the localized resource. + * If not set, the default one will be the <resourceName> from the config key. + * yarn.resources.<resourceName>.local.type + * (Optional) The value value is a string format of {@link LocalResourceType}: + * ARCHIVE, FILE, PATTERN. + * If not set, the default value is FILE. + * yarn.resources.<resourceName>.local.visibility + * (Optional) The valid value is a string format of {@link LocalResourceVisibility}: + * PUBLIC, PRIVATE, or APPLICATION. + * If not set, the default value is is APPLICATION. + */ +public class LocalizerResourceConfig { + private static final String RESOURCE_PREFIX = "yarn.resources."; + private static final String PATH_SUFFIX = ".path"; + private static final String RESOURCE_PATH = "yarn.resources.%s.path"; + private static final String RESOURCE_LOCAL_NAME = "yarn.resources.%s.local.name"; + private static final String RESOURCE_LOCAL_TYPE = "yarn.resources.%s.local.type"; + private static final String RESOURCE_LOCAL_VISIBILITY = "yarn.resources.%s.local.visibility"; + private static final String DEFAULT_RESOURCE_LOCAL_TYPE = "FILE"; + private static final String DEFAULT_RESOURCE_LOCAL_VISIBILITY = "APPLICATION"; + + private final Config config; + + public LocalizerResourceConfig(final Config config) { + if (null == config) { + throw new IllegalArgumentException("config cannot be null"); + } + this.config = config; + } + + public List getResourceNames() { + Config subConfig = config.subset(RESOURCE_PREFIX, true); + List resourceNames = new ArrayList(); + for (String key : subConfig.keySet()) { + if (key.endsWith(PATH_SUFFIX)) { + resourceNames.add(key.substring(0, key.length() - PATH_SUFFIX.length())); + } + } + return resourceNames; + } + + public Path getResourcePath(final String resourceName) { + String pathStr = config.get(String.format(RESOURCE_PATH, resourceName)); + if (StringUtils.isEmpty(pathStr)) { + throw new LocalizerResourceException("resource path is required but not defined in config for resource " + resourceName); + } + return new Path(pathStr); + } + + public LocalResourceType getResourceLocalType(final String resourceName) { + String typeStr = config.get(String.format(RESOURCE_LOCAL_TYPE, resourceName), DEFAULT_RESOURCE_LOCAL_TYPE); + return LocalResourceType.valueOf(StringUtils.upperCase(typeStr)); + } + + public LocalResourceVisibility getResourceLocalVisibility(final String resourceName) { + String visibilityStr = config.get(String.format(RESOURCE_LOCAL_VISIBILITY, resourceName), DEFAULT_RESOURCE_LOCAL_VISIBILITY); + return LocalResourceVisibility.valueOf(StringUtils.upperCase(visibilityStr)); + } + + public String getResourceLocalName(final String resourceName) { + String name = config.get(String.format(RESOURCE_LOCAL_NAME, resourceName), resourceName); + return name; + } + +} 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 0000000000..0df69036e6 --- /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 0000000000..6dddb0a1d6 --- /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 localResourceMap; + + public LocalizerResourceMapper(LocalizerResourceConfig resourceConfig, YarnConfiguration yarnConfiguration) { + this.yarnConfiguration = yarnConfiguration; + this.resourceConfig = resourceConfig; + this.localResourceMap = buildResourceMapping(); + } + + private Map buildResourceMapping() { + ImmutableMap.Builder localResourceMapBuilder = ImmutableMap.builder(); + + List 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 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 0000000000..15ca92fec0 --- /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 runningProcessors = new ConcurrentHashMap<>(); + + public Map pendingProcessors = new ConcurrentHashMap<>(); + + public ConcurrentMap 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} & 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 0000000000..c07a45b9a0 --- /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 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 allocatedResources = new ConcurrentHashMap<>(); + private final ConcurrentHashMap 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..impl" and "fs..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 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 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 statuses) { + List 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 containers) { + List resources = new ArrayList(); + 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 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 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 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 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 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() { + { + add(cmd); + } + }); + context.setLocalResources(localResourceMap); + + if (UserGroupInformation.isSecurityEnabled()) { + Map 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 getEscapedEnvironmentVariablesMap(CommandBuilder cmdBuilder) { + Map env = new HashMap(); + for (Map.Entry 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 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 0000000000..2bafa781d7 --- /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 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 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 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 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 computeHostToFaultDomainMap() { + Multimap hostToRackMap = HashMultimap.create(); + try { + List 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 0000000000..e4e547c490 --- /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 0000000000..3f9a84dd5a --- /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 0000000000..74fef67bd1 --- /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 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 0000000000..f890d2b6a1 --- /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> getMetrics() throws IOException { + String jsonString = getEntityAsJson("/metrics", "metrics"); + return jsonMapper.readValue(jsonString, new TypeReference>>() { }); + } + + /** + * @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 getTaskContext() throws IOException { + String jsonString = getEntityAsJson("/task-context", "task context"); + return jsonMapper.readValue(jsonString, new TypeReference>() { }); + } + + /** + * @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 getAmState() throws IOException { + String jsonString = getEntityAsJson("/am", "AM state"); + return jsonMapper.readValue(jsonString, new TypeReference>() { }); + } + + /** + * @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 getConfig() throws IOException { + String jsonString = getEntityAsJson("/config", "config"); + return jsonMapper.readValue(jsonString, new TypeReference>() { }); + } + + @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 0000000000..5b1f282291 --- /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 executionContainerId + * 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 + * true 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 0000000000..c826ecb429 --- /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 0000000000..13c5b920c3 --- /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 0000000000..7ddccac085 --- /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 0000000000..6064d78a48 --- /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 0000000000..679272d258 --- /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,strong{font-weight:700}dfn{font-style:italic}h1{font-size:2em;margin:.67em 0}mark{background:#ff0;color:#000}small{font-size:80%}sub,sup{font-size:75%;line-height:0;position:relative;vertical-align:baseline}sup{top:-.5em}sub{bottom:-.25em}img{border:0}svg:not(:root){overflow:hidden}figure{margin:1em 40px}hr{-moz-box-sizing:content-box;box-sizing:content-box;height:0}pre{overflow:auto}code,kbd,pre,samp{font-family:monospace,monospace;font-size:1em}button,input,optgroup,select,textarea{color:inherit;font:inherit;margin:0}button{overflow:visible}button,select{text-transform:none}button,html input[type=button],input[type=reset],input[type=submit]{-webkit-appearance:button;cursor:pointer}button[disabled],html input[disabled]{cursor:default}button::-moz-focus-inner,input::-moz-focus-inner{border:0;padding:0}input{line-height:normal}input[type=checkbox],input[type=radio]{box-sizing:border-box;padding:0}input[type=number]::-webkit-inner-spin-button,input[type=number]::-webkit-outer-spin-button{height:auto}input[type=search]{-webkit-appearance:textfield;-moz-box-sizing:content-box;-webkit-box-sizing:content-box;box-sizing:content-box}input[type=search]::-webkit-search-cancel-button,input[type=search]::-webkit-search-decoration{-webkit-appearance:none}fieldset{border:1px solid silver;margin:0 2px;padding:.35em .625em .75em}legend{border:0;padding:0}textarea{overflow:auto}optgroup{font-weight:700}table{border-collapse:collapse;border-spacing:0}td,th{padding:0}@media print{*{text-shadow:none!important;color:#000!important;background:transparent!important;box-shadow:none!important}a,a:visited{text-decoration:underline}a[href]:after{content:" (" attr(href) ")"}abbr[title]:after{content:" (" attr(title) ")"}a[href^="javascript:"]:after,a[href^="#"]:after{content:""}pre,blockquote{border:1px solid #999;page-break-inside:avoid}thead{display:table-header-group}tr,img{page-break-inside:avoid}img{max-width:100%!important}p,h2,h3{orphans:3;widows:3}h2,h3{page-break-after:avoid}select{background:#fff!important}.navbar{display:none}.table td,.table th{background-color:#fff!important}.btn>.caret,.dropup>.btn>.caret{border-top-color:#000!important}.label{border:1px solid #000}.table{border-collapse:collapse!important}.table-bordered th,.table-bordered td{border:1px solid #ddd!important}}*{-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}:before,:after{-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}html{font-size:62.5%;-webkit-tap-highlight-color:rgba(0,0,0,0)}body{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:1.42857143;color:#333;background-color:#fff}input,button,select,textarea{font-family:inherit;font-size:inherit;line-height:inherit}a{color:#428bca;text-decoration:none}a:hover,a:focus{color:#2a6496;text-decoration:underline}a:focus{outline:thin dotted;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}figure{margin:0}img{vertical-align:middle}.img-responsive,.thumbnail>img,.thumbnail a>img,.carousel-inner>.item>img,.carousel-inner>.item>a>img{display:block;max-width:100%;height:auto}.img-rounded{border-radius:6px}.img-thumbnail{padding:4px;line-height:1.42857143;background-color:#fff;border:1px solid #ddd;border-radius:4px;-webkit-transition:all .2s ease-in-out;transition:all .2s ease-in-out;display:inline-block;max-width:100%;height:auto}.img-circle{border-radius:50%}hr{margin-top:20px;margin-bottom:20px;border:0;border-top:1px solid #eee}.sr-only{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0}h1,h2,h3,h4,h5,h6,.h1,.h2,.h3,.h4,.h5,.h6{font-family:inherit;font-weight:500;line-height:1.1;color:inherit}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small,.h1 small,.h2 small,.h3 small,.h4 small,.h5 small,.h6 small,h1 .small,h2 .small,h3 .small,h4 .small,h5 .small,h6 .small,.h1 .small,.h2 .small,.h3 .small,.h4 .small,.h5 .small,.h6 .small{font-weight:400;line-height:1;color:#999}h1,.h1,h2,.h2,h3,.h3{margin-top:20px;margin-bottom:10px}h1 small,.h1 small,h2 small,.h2 small,h3 small,.h3 small,h1 .small,.h1 .small,h2 .small,.h2 .small,h3 .small,.h3 .small{font-size:65%}h4,.h4,h5,.h5,h6,.h6{margin-top:10px;margin-bottom:10px}h4 small,.h4 small,h5 small,.h5 small,h6 small,.h6 small,h4 .small,.h4 .small,h5 .small,.h5 .small,h6 .small,.h6 .small{font-size:75%}h1,.h1{font-size:36px}h2,.h2{font-size:30px}h3,.h3{font-size:24px}h4,.h4{font-size:18px}h5,.h5{font-size:14px}h6,.h6{font-size:12px}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:16px;font-weight:200;line-height:1.4}@media (min-width:768px){.lead{font-size:21px}}small,.small{font-size:85%}cite{font-style:normal}.text-left{text-align:left}.text-right{text-align:right}.text-center{text-align:center}.text-justify{text-align:justify}.text-muted{color:#999}.text-primary{color:#428bca}a.text-primary:hover{color:#3071a9}.text-success{color:#3c763d}a.text-success:hover{color:#2b542c}.text-info{color:#31708f}a.text-info:hover{color:#245269}.text-warning{color:#8a6d3b}a.text-warning:hover{color:#66512c}.text-danger{color:#a94442}a.text-danger:hover{color:#843534}.bg-primary{color:#fff;background-color:#428bca}a.bg-primary:hover{background-color:#3071a9}.bg-success{background-color:#dff0d8}a.bg-success:hover{background-color:#c1e2b3}.bg-info{background-color:#d9edf7}a.bg-info:hover{background-color:#afd9ee}.bg-warning{background-color:#fcf8e3}a.bg-warning:hover{background-color:#f7ecb5}.bg-danger{background-color:#f2dede}a.bg-danger:hover{background-color:#e4b9b9}.page-header{padding-bottom:9px;margin:40px 0 20px;border-bottom:1px solid #eee}ul,ol{margin-top:0;margin-bottom:10px}ul ul,ol ul,ul ol,ol ol{margin-bottom:0}.list-unstyled{padding-left:0;list-style:none}.list-inline{padding-left:0;list-style:none;margin-left:-5px}.list-inline>li{display:inline-block;padding-left:5px;padding-right:5px}dl{margin-top:0;margin-bottom:20px}dt,dd{line-height:1.42857143}dt{font-weight:700}dd{margin-left:0}@media (min-width:768px){.dl-horizontal dt{float:left;width:160px;clear:left;text-align:right;overflow:hidden;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:180px}}abbr[title],abbr[data-original-title]{cursor:help;border-bottom:1px dotted #999}.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:10px 20px;margin:0 0 20px;font-size:17.5px;border-left:5px solid #eee}blockquote p:last-child,blockquote ul:last-child,blockquote ol:last-child{margin-bottom:0}blockquote footer,blockquote small,blockquote .small{display:block;font-size:80%;line-height:1.42857143;color:#999}blockquote footer:before,blockquote small:before,blockquote .small:before{content:'\2014 \00A0'}.blockquote-reverse,blockquote.pull-right{padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0;text-align:right}.blockquote-reverse footer:before,blockquote.pull-right footer:before,.blockquote-reverse small:before,blockquote.pull-right small:before,.blockquote-reverse .small:before,blockquote.pull-right .small:before{content:''}.blockquote-reverse footer:after,blockquote.pull-right footer:after,.blockquote-reverse small:after,blockquote.pull-right small:after,.blockquote-reverse .small:after,blockquote.pull-right .small:after{content:'\00A0 \2014'}blockquote:before,blockquote:after{content:""}address{margin-bottom:20px;font-style:normal;line-height:1.42857143}code,kbd,pre,samp{font-family:Menlo,Monaco,Consolas,"Courier New",monospace}code{padding:2px 4px;font-size:90%;color:#c7254e;background-color:#f9f2f4;white-space:nowrap;border-radius:4px}kbd{padding:2px 4px;font-size:90%;color:#fff;background-color:#333;border-radius:3px;box-shadow:inset 0 -1px 0 rgba(0,0,0,.25)}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:1.42857143;word-break:break-all;word-wrap:break-word;color:#333;background-color:#f5f5f5;border:1px solid #ccc;border-radius:4px}pre code{padding:0;font-size:inherit;color:inherit;white-space:pre-wrap;background-color:transparent;border-radius:0}.pre-scrollable{max-height:340px;overflow-y:scroll}.container{margin-right:auto;margin-left:auto;padding-left:15px;padding-right:15px}@media (min-width:768px){.container{width:750px}}@media (min-width:992px){.container{width:970px}}@media (min-width:1200px){.container{width:1170px}}.container-fluid{margin-right:auto;margin-left:auto;padding-left:15px;padding-right:15px}.row{margin-left:-15px;margin-right:-15px}.col-xs-1,.col-sm-1,.col-md-1,.col-lg-1,.col-xs-2,.col-sm-2,.col-md-2,.col-lg-2,.col-xs-3,.col-sm-3,.col-md-3,.col-lg-3,.col-xs-4,.col-sm-4,.col-md-4,.col-lg-4,.col-xs-5,.col-sm-5,.col-md-5,.col-lg-5,.col-xs-6,.col-sm-6,.col-md-6,.col-lg-6,.col-xs-7,.col-sm-7,.col-md-7,.col-lg-7,.col-xs-8,.col-sm-8,.col-md-8,.col-lg-8,.col-xs-9,.col-sm-9,.col-md-9,.col-lg-9,.col-xs-10,.col-sm-10,.col-md-10,.col-lg-10,.col-xs-11,.col-sm-11,.col-md-11,.col-lg-11,.col-xs-12,.col-sm-12,.col-md-12,.col-lg-12{position:relative;min-height:1px;padding-left:15px;padding-right:15px}.col-xs-1,.col-xs-2,.col-xs-3,.col-xs-4,.col-xs-5,.col-xs-6,.col-xs-7,.col-xs-8,.col-xs-9,.col-xs-10,.col-xs-11,.col-xs-12{float:left}.col-xs-12{width:100%}.col-xs-11{width:91.66666667%}.col-xs-10{width:83.33333333%}.col-xs-9{width:75%}.col-xs-8{width:66.66666667%}.col-xs-7{width:58.33333333%}.col-xs-6{width:50%}.col-xs-5{width:41.66666667%}.col-xs-4{width:33.33333333%}.col-xs-3{width:25%}.col-xs-2{width:16.66666667%}.col-xs-1{width:8.33333333%}.col-xs-pull-12{right:100%}.col-xs-pull-11{right:91.66666667%}.col-xs-pull-10{right:83.33333333%}.col-xs-pull-9{right:75%}.col-xs-pull-8{right:66.66666667%}.col-xs-pull-7{right:58.33333333%}.col-xs-pull-6{right:50%}.col-xs-pull-5{right:41.66666667%}.col-xs-pull-4{right:33.33333333%}.col-xs-pull-3{right:25%}.col-xs-pull-2{right:16.66666667%}.col-xs-pull-1{right:8.33333333%}.col-xs-pull-0{right:0}.col-xs-push-12{left:100%}.col-xs-push-11{left:91.66666667%}.col-xs-push-10{left:83.33333333%}.col-xs-push-9{left:75%}.col-xs-push-8{left:66.66666667%}.col-xs-push-7{left:58.33333333%}.col-xs-push-6{left:50%}.col-xs-push-5{left:41.66666667%}.col-xs-push-4{left:33.33333333%}.col-xs-push-3{left:25%}.col-xs-push-2{left:16.66666667%}.col-xs-push-1{left:8.33333333%}.col-xs-push-0{left:0}.col-xs-offset-12{margin-left:100%}.col-xs-offset-11{margin-left:91.66666667%}.col-xs-offset-10{margin-left:83.33333333%}.col-xs-offset-9{margin-left:75%}.col-xs-offset-8{margin-left:66.66666667%}.col-xs-offset-7{margin-left:58.33333333%}.col-xs-offset-6{margin-left:50%}.col-xs-offset-5{margin-left:41.66666667%}.col-xs-offset-4{margin-left:33.33333333%}.col-xs-offset-3{margin-left:25%}.col-xs-offset-2{margin-left:16.66666667%}.col-xs-offset-1{margin-left:8.33333333%}.col-xs-offset-0{margin-left:0}@media (min-width:768px){.col-sm-1,.col-sm-2,.col-sm-3,.col-sm-4,.col-sm-5,.col-sm-6,.col-sm-7,.col-sm-8,.col-sm-9,.col-sm-10,.col-sm-11,.col-sm-12{float:left}.col-sm-12{width:100%}.col-sm-11{width:91.66666667%}.col-sm-10{width:83.33333333%}.col-sm-9{width:75%}.col-sm-8{width:66.66666667%}.col-sm-7{width:58.33333333%}.col-sm-6{width:50%}.col-sm-5{width:41.66666667%}.col-sm-4{width:33.33333333%}.col-sm-3{width:25%}.col-sm-2{width:16.66666667%}.col-sm-1{width:8.33333333%}.col-sm-pull-12{right:100%}.col-sm-pull-11{right:91.66666667%}.col-sm-pull-10{right:83.33333333%}.col-sm-pull-9{right:75%}.col-sm-pull-8{right:66.66666667%}.col-sm-pull-7{right:58.33333333%}.col-sm-pull-6{right:50%}.col-sm-pull-5{right:41.66666667%}.col-sm-pull-4{right:33.33333333%}.col-sm-pull-3{right:25%}.col-sm-pull-2{right:16.66666667%}.col-sm-pull-1{right:8.33333333%}.col-sm-pull-0{right:0}.col-sm-push-12{left:100%}.col-sm-push-11{left:91.66666667%}.col-sm-push-10{left:83.33333333%}.col-sm-push-9{left:75%}.col-sm-push-8{left:66.66666667%}.col-sm-push-7{left:58.33333333%}.col-sm-push-6{left:50%}.col-sm-push-5{left:41.66666667%}.col-sm-push-4{left:33.33333333%}.col-sm-push-3{left:25%}.col-sm-push-2{left:16.66666667%}.col-sm-push-1{left:8.33333333%}.col-sm-push-0{left:0}.col-sm-offset-12{margin-left:100%}.col-sm-offset-11{margin-left:91.66666667%}.col-sm-offset-10{margin-left:83.33333333%}.col-sm-offset-9{margin-left:75%}.col-sm-offset-8{margin-left:66.66666667%}.col-sm-offset-7{margin-left:58.33333333%}.col-sm-offset-6{margin-left:50%}.col-sm-offset-5{margin-left:41.66666667%}.col-sm-offset-4{margin-left:33.33333333%}.col-sm-offset-3{margin-left:25%}.col-sm-offset-2{margin-left:16.66666667%}.col-sm-offset-1{margin-left:8.33333333%}.col-sm-offset-0{margin-left:0}}@media (min-width:992px){.col-md-1,.col-md-2,.col-md-3,.col-md-4,.col-md-5,.col-md-6,.col-md-7,.col-md-8,.col-md-9,.col-md-10,.col-md-11,.col-md-12{float:left}.col-md-12{width:100%}.col-md-11{width:91.66666667%}.col-md-10{width:83.33333333%}.col-md-9{width:75%}.col-md-8{width:66.66666667%}.col-md-7{width:58.33333333%}.col-md-6{width:50%}.col-md-5{width:41.66666667%}.col-md-4{width:33.33333333%}.col-md-3{width:25%}.col-md-2{width:16.66666667%}.col-md-1{width:8.33333333%}.col-md-pull-12{right:100%}.col-md-pull-11{right:91.66666667%}.col-md-pull-10{right:83.33333333%}.col-md-pull-9{right:75%}.col-md-pull-8{right:66.66666667%}.col-md-pull-7{right:58.33333333%}.col-md-pull-6{right:50%}.col-md-pull-5{right:41.66666667%}.col-md-pull-4{right:33.33333333%}.col-md-pull-3{right:25%}.col-md-pull-2{right:16.66666667%}.col-md-pull-1{right:8.33333333%}.col-md-pull-0{right:0}.col-md-push-12{left:100%}.col-md-push-11{left:91.66666667%}.col-md-push-10{left:83.33333333%}.col-md-push-9{left:75%}.col-md-push-8{left:66.66666667%}.col-md-push-7{left:58.33333333%}.col-md-push-6{left:50%}.col-md-push-5{left:41.66666667%}.col-md-push-4{left:33.33333333%}.col-md-push-3{left:25%}.col-md-push-2{left:16.66666667%}.col-md-push-1{left:8.33333333%}.col-md-push-0{left:0}.col-md-offset-12{margin-left:100%}.col-md-offset-11{margin-left:91.66666667%}.col-md-offset-10{margin-left:83.33333333%}.col-md-offset-9{margin-left:75%}.col-md-offset-8{margin-left:66.66666667%}.col-md-offset-7{margin-left:58.33333333%}.col-md-offset-6{margin-left:50%}.col-md-offset-5{margin-left:41.66666667%}.col-md-offset-4{margin-left:33.33333333%}.col-md-offset-3{margin-left:25%}.col-md-offset-2{margin-left:16.66666667%}.col-md-offset-1{margin-left:8.33333333%}.col-md-offset-0{margin-left:0}}@media (min-width:1200px){.col-lg-1,.col-lg-2,.col-lg-3,.col-lg-4,.col-lg-5,.col-lg-6,.col-lg-7,.col-lg-8,.col-lg-9,.col-lg-10,.col-lg-11,.col-lg-12{float:left}.col-lg-12{width:100%}.col-lg-11{width:91.66666667%}.col-lg-10{width:83.33333333%}.col-lg-9{width:75%}.col-lg-8{width:66.66666667%}.col-lg-7{width:58.33333333%}.col-lg-6{width:50%}.col-lg-5{width:41.66666667%}.col-lg-4{width:33.33333333%}.col-lg-3{width:25%}.col-lg-2{width:16.66666667%}.col-lg-1{width:8.33333333%}.col-lg-pull-12{right:100%}.col-lg-pull-11{right:91.66666667%}.col-lg-pull-10{right:83.33333333%}.col-lg-pull-9{right:75%}.col-lg-pull-8{right:66.66666667%}.col-lg-pull-7{right:58.33333333%}.col-lg-pull-6{right:50%}.col-lg-pull-5{right:41.66666667%}.col-lg-pull-4{right:33.33333333%}.col-lg-pull-3{right:25%}.col-lg-pull-2{right:16.66666667%}.col-lg-pull-1{right:8.33333333%}.col-lg-pull-0{right:0}.col-lg-push-12{left:100%}.col-lg-push-11{left:91.66666667%}.col-lg-push-10{left:83.33333333%}.col-lg-push-9{left:75%}.col-lg-push-8{left:66.66666667%}.col-lg-push-7{left:58.33333333%}.col-lg-push-6{left:50%}.col-lg-push-5{left:41.66666667%}.col-lg-push-4{left:33.33333333%}.col-lg-push-3{left:25%}.col-lg-push-2{left:16.66666667%}.col-lg-push-1{left:8.33333333%}.col-lg-push-0{left:0}.col-lg-offset-12{margin-left:100%}.col-lg-offset-11{margin-left:91.66666667%}.col-lg-offset-10{margin-left:83.33333333%}.col-lg-offset-9{margin-left:75%}.col-lg-offset-8{margin-left:66.66666667%}.col-lg-offset-7{margin-left:58.33333333%}.col-lg-offset-6{margin-left:50%}.col-lg-offset-5{margin-left:41.66666667%}.col-lg-offset-4{margin-left:33.33333333%}.col-lg-offset-3{margin-left:25%}.col-lg-offset-2{margin-left:16.66666667%}.col-lg-offset-1{margin-left:8.33333333%}.col-lg-offset-0{margin-left:0}}table{max-width:100%;background-color:transparent}th{text-align:left}.table{width:100%;margin-bottom:20px}.table>thead>tr>th,.table>tbody>tr>th,.table>tfoot>tr>th,.table>thead>tr>td,.table>tbody>tr>td,.table>tfoot>tr>td{padding:8px;line-height:1.42857143;vertical-align:top;border-top:1px solid #ddd}.table>thead>tr>th{vertical-align:bottom;border-bottom:2px solid #ddd}.table>caption+thead>tr:first-child>th,.table>colgroup+thead>tr:first-child>th,.table>thead:first-child>tr:first-child>th,.table>caption+thead>tr:first-child>td,.table>colgroup+thead>tr:first-child>td,.table>thead:first-child>tr:first-child>td{border-top:0}.table>tbody+tbody{border-top:2px solid #ddd}.table .table{background-color:#fff}.table-condensed>thead>tr>th,.table-condensed>tbody>tr>th,.table-condensed>tfoot>tr>th,.table-condensed>thead>tr>td,.table-condensed>tbody>tr>td,.table-condensed>tfoot>tr>td{padding:5px}.table-bordered{border:1px solid #ddd}.table-bordered>thead>tr>th,.table-bordered>tbody>tr>th,.table-bordered>tfoot>tr>th,.table-bordered>thead>tr>td,.table-bordered>tbody>tr>td,.table-bordered>tfoot>tr>td{border:1px solid #ddd}.table-bordered>thead>tr>th,.table-bordered>thead>tr>td{border-bottom-width:2px}.table-striped>tbody>tr:nth-child(odd)>td,.table-striped>tbody>tr:nth-child(odd)>th{background-color:#f9f9f9}.table-hover>tbody>tr:hover>td,.table-hover>tbody>tr:hover>th{background-color:#f5f5f5}table col[class*=col-]{position:static;float:none;display:table-column}table td[class*=col-],table th[class*=col-]{position:static;float:none;display:table-cell}.table>thead>tr>td.active,.table>tbody>tr>td.active,.table>tfoot>tr>td.active,.table>thead>tr>th.active,.table>tbody>tr>th.active,.table>tfoot>tr>th.active,.table>thead>tr.active>td,.table>tbody>tr.active>td,.table>tfoot>tr.active>td,.table>thead>tr.active>th,.table>tbody>tr.active>th,.table>tfoot>tr.active>th{background-color:#f5f5f5}.table-hover>tbody>tr>td.active:hover,.table-hover>tbody>tr>th.active:hover,.table-hover>tbody>tr.active:hover>td,.table-hover>tbody>tr.active:hover>th{background-color:#e8e8e8}.table>thead>tr>td.success,.table>tbody>tr>td.success,.table>tfoot>tr>td.success,.table>thead>tr>th.success,.table>tbody>tr>th.success,.table>tfoot>tr>th.success,.table>thead>tr.success>td,.table>tbody>tr.success>td,.table>tfoot>tr.success>td,.table>thead>tr.success>th,.table>tbody>tr.success>th,.table>tfoot>tr.success>th{background-color:#dff0d8}.table-hover>tbody>tr>td.success:hover,.table-hover>tbody>tr>th.success:hover,.table-hover>tbody>tr.success:hover>td,.table-hover>tbody>tr.success:hover>th{background-color:#d0e9c6}.table>thead>tr>td.info,.table>tbody>tr>td.info,.table>tfoot>tr>td.info,.table>thead>tr>th.info,.table>tbody>tr>th.info,.table>tfoot>tr>th.info,.table>thead>tr.info>td,.table>tbody>tr.info>td,.table>tfoot>tr.info>td,.table>thead>tr.info>th,.table>tbody>tr.info>th,.table>tfoot>tr.info>th{background-color:#d9edf7}.table-hover>tbody>tr>td.info:hover,.table-hover>tbody>tr>th.info:hover,.table-hover>tbody>tr.info:hover>td,.table-hover>tbody>tr.info:hover>th{background-color:#c4e3f3}.table>thead>tr>td.warning,.table>tbody>tr>td.warning,.table>tfoot>tr>td.warning,.table>thead>tr>th.warning,.table>tbody>tr>th.warning,.table>tfoot>tr>th.warning,.table>thead>tr.warning>td,.table>tbody>tr.warning>td,.table>tfoot>tr.warning>td,.table>thead>tr.warning>th,.table>tbody>tr.warning>th,.table>tfoot>tr.warning>th{background-color:#fcf8e3}.table-hover>tbody>tr>td.warning:hover,.table-hover>tbody>tr>th.warning:hover,.table-hover>tbody>tr.warning:hover>td,.table-hover>tbody>tr.warning:hover>th{background-color:#faf2cc}.table>thead>tr>td.danger,.table>tbody>tr>td.danger,.table>tfoot>tr>td.danger,.table>thead>tr>th.danger,.table>tbody>tr>th.danger,.table>tfoot>tr>th.danger,.table>thead>tr.danger>td,.table>tbody>tr.danger>td,.table>tfoot>tr.danger>td,.table>thead>tr.danger>th,.table>tbody>tr.danger>th,.table>tfoot>tr.danger>th{background-color:#f2dede}.table-hover>tbody>tr>td.danger:hover,.table-hover>tbody>tr>th.danger:hover,.table-hover>tbody>tr.danger:hover>td,.table-hover>tbody>tr.danger:hover>th{background-color:#ebcccc}@media (max-width:767px){.table-responsive{width:100%;margin-bottom:15px;overflow-y:hidden;overflow-x:scroll;-ms-overflow-style:-ms-autohiding-scrollbar;border:1px solid #ddd;-webkit-overflow-scrolling:touch}.table-responsive>.table{margin-bottom:0}.table-responsive>.table>thead>tr>th,.table-responsive>.table>tbody>tr>th,.table-responsive>.table>tfoot>tr>th,.table-responsive>.table>thead>tr>td,.table-responsive>.table>tbody>tr>td,.table-responsive>.table>tfoot>tr>td{white-space:nowrap}.table-responsive>.table-bordered{border:0}.table-responsive>.table-bordered>thead>tr>th:first-child,.table-responsive>.table-bordered>tbody>tr>th:first-child,.table-responsive>.table-bordered>tfoot>tr>th:first-child,.table-responsive>.table-bordered>thead>tr>td:first-child,.table-responsive>.table-bordered>tbody>tr>td:first-child,.table-responsive>.table-bordered>tfoot>tr>td:first-child{border-left:0}.table-responsive>.table-bordered>thead>tr>th:last-child,.table-responsive>.table-bordered>tbody>tr>th:last-child,.table-responsive>.table-bordered>tfoot>tr>th:last-child,.table-responsive>.table-bordered>thead>tr>td:last-child,.table-responsive>.table-bordered>tbody>tr>td:last-child,.table-responsive>.table-bordered>tfoot>tr>td:last-child{border-right:0}.table-responsive>.table-bordered>tbody>tr:last-child>th,.table-responsive>.table-bordered>tfoot>tr:last-child>th,.table-responsive>.table-bordered>tbody>tr:last-child>td,.table-responsive>.table-bordered>tfoot>tr:last-child>td{border-bottom:0}}fieldset{padding:0;margin:0;border:0;min-width:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:inherit;color:#333;border:0;border-bottom:1px solid #e5e5e5}label{display:inline-block;margin-bottom:5px;font-weight:700}input[type=search]{-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}input[type=radio],input[type=checkbox]{margin:4px 0 0;margin-top:1px \9;line-height:normal}input[type=file]{display:block}input[type=range]{display:block;width:100%}select[multiple],select[size]{height:auto}input[type=file]:focus,input[type=radio]:focus,input[type=checkbox]:focus{outline:thin dotted;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}output{display:block;padding-top:7px;font-size:14px;line-height:1.42857143;color:#555}.form-control{display:block;width:100%;height:34px;padding:6px 12px;font-size:14px;line-height:1.42857143;color:#555;background-color:#fff;background-image:none;border:1px solid #ccc;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075);box-shadow:inset 0 1px 1px rgba(0,0,0,.075);-webkit-transition:border-color ease-in-out .15s,box-shadow ease-in-out .15s;transition:border-color ease-in-out .15s,box-shadow ease-in-out .15s}.form-control:focus{border-color:#66afe9;outline:0;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 8px rgba(102,175,233,.6);box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 8px rgba(102,175,233,.6)}.form-control::-moz-placeholder{color:#999;opacity:1}.form-control:-ms-input-placeholder{color:#999}.form-control::-webkit-input-placeholder{color:#999}.form-control[disabled],.form-control[readonly],fieldset[disabled] .form-control{cursor:not-allowed;background-color:#eee;opacity:1}textarea.form-control{height:auto}input[type=search]{-webkit-appearance:none}input[type=date]{line-height:34px}.form-group{margin-bottom:15px}.radio,.checkbox{display:block;min-height:20px;margin-top:10px;margin-bottom:10px;padding-left:20px}.radio label,.checkbox label{display:inline;font-weight:400;cursor:pointer}.radio input[type=radio],.radio-inline input[type=radio],.checkbox input[type=checkbox],.checkbox-inline input[type=checkbox]{float:left;margin-left:-20px}.radio+.radio,.checkbox+.checkbox{margin-top:-5px}.radio-inline,.checkbox-inline{display:inline-block;padding-left:20px;margin-bottom:0;vertical-align:middle;font-weight:400;cursor:pointer}.radio-inline+.radio-inline,.checkbox-inline+.checkbox-inline{margin-top:0;margin-left:10px}input[type=radio][disabled],input[type=checkbox][disabled],.radio[disabled],.radio-inline[disabled],.checkbox[disabled],.checkbox-inline[disabled],fieldset[disabled] input[type=radio],fieldset[disabled] input[type=checkbox],fieldset[disabled] .radio,fieldset[disabled] .radio-inline,fieldset[disabled] .checkbox,fieldset[disabled] .checkbox-inline{cursor:not-allowed}.input-sm{height:30px;padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}select.input-sm{height:30px;line-height:30px}textarea.input-sm,select[multiple].input-sm{height:auto}.input-lg{height:46px;padding:10px 16px;font-size:18px;line-height:1.33;border-radius:6px}select.input-lg{height:46px;line-height:46px}textarea.input-lg,select[multiple].input-lg{height:auto}.has-feedback{position:relative}.has-feedback .form-control{padding-right:42.5px}.has-feedback .form-control-feedback{position:absolute;top:25px;right:0;display:block;width:34px;height:34px;line-height:34px;text-align:center}.has-success .help-block,.has-success .control-label,.has-success .radio,.has-success .checkbox,.has-success .radio-inline,.has-success .checkbox-inline{color:#3c763d}.has-success .form-control{border-color:#3c763d;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075);box-shadow:inset 0 1px 1px rgba(0,0,0,.075)}.has-success .form-control:focus{border-color:#2b542c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 6px #67b168;box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 6px #67b168}.has-success .input-group-addon{color:#3c763d;border-color:#3c763d;background-color:#dff0d8}.has-success .form-control-feedback{color:#3c763d}.has-warning .help-block,.has-warning .control-label,.has-warning .radio,.has-warning .checkbox,.has-warning .radio-inline,.has-warning .checkbox-inline{color:#8a6d3b}.has-warning .form-control{border-color:#8a6d3b;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075);box-shadow:inset 0 1px 1px rgba(0,0,0,.075)}.has-warning .form-control:focus{border-color:#66512c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 6px #c0a16b;box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 6px #c0a16b}.has-warning .input-group-addon{color:#8a6d3b;border-color:#8a6d3b;background-color:#fcf8e3}.has-warning .form-control-feedback{color:#8a6d3b}.has-error .help-block,.has-error .control-label,.has-error .radio,.has-error .checkbox,.has-error .radio-inline,.has-error .checkbox-inline{color:#a94442}.has-error .form-control{border-color:#a94442;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075);box-shadow:inset 0 1px 1px rgba(0,0,0,.075)}.has-error .form-control:focus{border-color:#843534;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 6px #ce8483;box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 6px #ce8483}.has-error .input-group-addon{color:#a94442;border-color:#a94442;background-color:#f2dede}.has-error .form-control-feedback{color:#a94442}.form-control-static{margin-bottom:0}.help-block{display:block;margin-top:5px;margin-bottom:10px;color:#737373}@media (min-width:768px){.form-inline .form-group{display:inline-block;margin-bottom:0;vertical-align:middle}.form-inline .form-control{display:inline-block;width:auto;vertical-align:middle}.form-inline .input-group>.form-control{width:100%}.form-inline .control-label{margin-bottom:0;vertical-align:middle}.form-inline .radio,.form-inline .checkbox{display:inline-block;margin-top:0;margin-bottom:0;padding-left:0;vertical-align:middle}.form-inline .radio input[type=radio],.form-inline .checkbox input[type=checkbox]{float:none;margin-left:0}.form-inline .has-feedback .form-control-feedback{top:0}}.form-horizontal .control-label,.form-horizontal .radio,.form-horizontal .checkbox,.form-horizontal .radio-inline,.form-horizontal .checkbox-inline{margin-top:0;margin-bottom:0;padding-top:7px}.form-horizontal .radio,.form-horizontal .checkbox{min-height:27px}.form-horizontal .form-group{margin-left:-15px;margin-right:-15px}.form-horizontal .form-control-static{padding-top:7px}@media (min-width:768px){.form-horizontal .control-label{text-align:right}}.form-horizontal .has-feedback .form-control-feedback{top:0;right:15px}.btn{display:inline-block;margin-bottom:0;font-weight:400;text-align:center;vertical-align:middle;cursor:pointer;background-image:none;border:1px solid transparent;white-space:nowrap;padding:6px 12px;font-size:14px;line-height:1.42857143;border-radius:4px;-webkit-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}.btn:focus,.btn:active:focus,.btn.active:focus{outline:thin dotted;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn:hover,.btn:focus{color:#333;text-decoration:none}.btn:active,.btn.active{outline:0;background-image:none;-webkit-box-shadow:inset 0 3px 5px rgba(0,0,0,.125);box-shadow:inset 0 3px 5px rgba(0,0,0,.125)}.btn.disabled,.btn[disabled],fieldset[disabled] .btn{cursor:not-allowed;pointer-events:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;box-shadow:none}.btn-default{color:#333;background-color:#fff;border-color:#ccc}.btn-default:hover,.btn-default:focus,.btn-default:active,.btn-default.active,.open .dropdown-toggle.btn-default{color:#333;background-color:#ebebeb;border-color:#adadad}.btn-default:active,.btn-default.active,.open .dropdown-toggle.btn-default{background-image:none}.btn-default.disabled,.btn-default[disabled],fieldset[disabled] .btn-default,.btn-default.disabled:hover,.btn-default[disabled]:hover,fieldset[disabled] .btn-default:hover,.btn-default.disabled:focus,.btn-default[disabled]:focus,fieldset[disabled] .btn-default:focus,.btn-default.disabled:active,.btn-default[disabled]:active,fieldset[disabled] .btn-default:active,.btn-default.disabled.active,.btn-default[disabled].active,fieldset[disabled] .btn-default.active{background-color:#fff;border-color:#ccc}.btn-default .badge{color:#fff;background-color:#333}.btn-primary{color:#fff;background-color:#428bca;border-color:#357ebd}.btn-primary:hover,.btn-primary:focus,.btn-primary:active,.btn-primary.active,.open .dropdown-toggle.btn-primary{color:#fff;background-color:#3276b1;border-color:#285e8e}.btn-primary:active,.btn-primary.active,.open .dropdown-toggle.btn-primary{background-image:none}.btn-primary.disabled,.btn-primary[disabled],fieldset[disabled] .btn-primary,.btn-primary.disabled:hover,.btn-primary[disabled]:hover,fieldset[disabled] .btn-primary:hover,.btn-primary.disabled:focus,.btn-primary[disabled]:focus,fieldset[disabled] .btn-primary:focus,.btn-primary.disabled:active,.btn-primary[disabled]:active,fieldset[disabled] .btn-primary:active,.btn-primary.disabled.active,.btn-primary[disabled].active,fieldset[disabled] .btn-primary.active{background-color:#428bca;border-color:#357ebd}.btn-primary .badge{color:#428bca;background-color:#fff}.btn-success{color:#fff;background-color:#5cb85c;border-color:#4cae4c}.btn-success:hover,.btn-success:focus,.btn-success:active,.btn-success.active,.open .dropdown-toggle.btn-success{color:#fff;background-color:#47a447;border-color:#398439}.btn-success:active,.btn-success.active,.open .dropdown-toggle.btn-success{background-image:none}.btn-success.disabled,.btn-success[disabled],fieldset[disabled] .btn-success,.btn-success.disabled:hover,.btn-success[disabled]:hover,fieldset[disabled] .btn-success:hover,.btn-success.disabled:focus,.btn-success[disabled]:focus,fieldset[disabled] .btn-success:focus,.btn-success.disabled:active,.btn-success[disabled]:active,fieldset[disabled] .btn-success:active,.btn-success.disabled.active,.btn-success[disabled].active,fieldset[disabled] .btn-success.active{background-color:#5cb85c;border-color:#4cae4c}.btn-success .badge{color:#5cb85c;background-color:#fff}.btn-info{color:#fff;background-color:#5bc0de;border-color:#46b8da}.btn-info:hover,.btn-info:focus,.btn-info:active,.btn-info.active,.open .dropdown-toggle.btn-info{color:#fff;background-color:#39b3d7;border-color:#269abc}.btn-info:active,.btn-info.active,.open .dropdown-toggle.btn-info{background-image:none}.btn-info.disabled,.btn-info[disabled],fieldset[disabled] .btn-info,.btn-info.disabled:hover,.btn-info[disabled]:hover,fieldset[disabled] .btn-info:hover,.btn-info.disabled:focus,.btn-info[disabled]:focus,fieldset[disabled] .btn-info:focus,.btn-info.disabled:active,.btn-info[disabled]:active,fieldset[disabled] .btn-info:active,.btn-info.disabled.active,.btn-info[disabled].active,fieldset[disabled] .btn-info.active{background-color:#5bc0de;border-color:#46b8da}.btn-info .badge{color:#5bc0de;background-color:#fff}.btn-warning{color:#fff;background-color:#f0ad4e;border-color:#eea236}.btn-warning:hover,.btn-warning:focus,.btn-warning:active,.btn-warning.active,.open .dropdown-toggle.btn-warning{color:#fff;background-color:#ed9c28;border-color:#d58512}.btn-warning:active,.btn-warning.active,.open .dropdown-toggle.btn-warning{background-image:none}.btn-warning.disabled,.btn-warning[disabled],fieldset[disabled] .btn-warning,.btn-warning.disabled:hover,.btn-warning[disabled]:hover,fieldset[disabled] .btn-warning:hover,.btn-warning.disabled:focus,.btn-warning[disabled]:focus,fieldset[disabled] .btn-warning:focus,.btn-warning.disabled:active,.btn-warning[disabled]:active,fieldset[disabled] .btn-warning:active,.btn-warning.disabled.active,.btn-warning[disabled].active,fieldset[disabled] .btn-warning.active{background-color:#f0ad4e;border-color:#eea236}.btn-warning .badge{color:#f0ad4e;background-color:#fff}.btn-danger{color:#fff;background-color:#d9534f;border-color:#d43f3a}.btn-danger:hover,.btn-danger:focus,.btn-danger:active,.btn-danger.active,.open .dropdown-toggle.btn-danger{color:#fff;background-color:#d2322d;border-color:#ac2925}.btn-danger:active,.btn-danger.active,.open .dropdown-toggle.btn-danger{background-image:none}.btn-danger.disabled,.btn-danger[disabled],fieldset[disabled] .btn-danger,.btn-danger.disabled:hover,.btn-danger[disabled]:hover,fieldset[disabled] .btn-danger:hover,.btn-danger.disabled:focus,.btn-danger[disabled]:focus,fieldset[disabled] .btn-danger:focus,.btn-danger.disabled:active,.btn-danger[disabled]:active,fieldset[disabled] .btn-danger:active,.btn-danger.disabled.active,.btn-danger[disabled].active,fieldset[disabled] .btn-danger.active{background-color:#d9534f;border-color:#d43f3a}.btn-danger .badge{color:#d9534f;background-color:#fff}.btn-link{color:#428bca;font-weight:400;cursor:pointer;border-radius:0}.btn-link,.btn-link:active,.btn-link[disabled],fieldset[disabled] .btn-link{background-color:transparent;-webkit-box-shadow:none;box-shadow:none}.btn-link,.btn-link:hover,.btn-link:focus,.btn-link:active{border-color:transparent}.btn-link:hover,.btn-link:focus{color:#2a6496;text-decoration:underline;background-color:transparent}.btn-link[disabled]:hover,fieldset[disabled] .btn-link:hover,.btn-link[disabled]:focus,fieldset[disabled] .btn-link:focus{color:#999;text-decoration:none}.btn-lg,.btn-group-lg>.btn{padding:10px 16px;font-size:18px;line-height:1.33;border-radius:6px}.btn-sm,.btn-group-sm>.btn{padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}.btn-xs,.btn-group-xs>.btn{padding:1px 5px;font-size:12px;line-height:1.5;border-radius:3px}.btn-block{display:block;width:100%;padding-left:0;padding-right:0}.btn-block+.btn-block{margin-top:5px}input[type=submit].btn-block,input[type=reset].btn-block,input[type=button].btn-block{width:100%}.fade{opacity:0;-webkit-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{display:none}.collapse.in{display:block}.collapsing{position:relative;height:0;overflow:hidden;-webkit-transition:height .35s ease;transition:height .35s ease}@font-face{font-family:'Glyphicons Halflings';src:url(../fonts/glyphicons-halflings-regular.eot);src:url(../fonts/glyphicons-halflings-regular.eot?#iefix) format('embedded-opentype'),url(../fonts/glyphicons-halflings-regular.woff) format('woff'),url(../fonts/glyphicons-halflings-regular.ttf) format('truetype'),url(../fonts/glyphicons-halflings-regular.svg#glyphicons_halflingsregular) format('svg')}.glyphicon{position:relative;top:1px;display:inline-block;font-family:'Glyphicons Halflings';font-style:normal;font-weight:400;line-height:1;-webkit-font-smoothing:antialiased;-moz-osx-font-smoothing:grayscale}.glyphicon-asterisk:before{content:"\2a"}.glyphicon-plus:before{content:"\2b"}.glyphicon-euro:before{content:"\20ac"}.glyphicon-minus:before{content:"\2212"}.glyphicon-cloud:before{content:"\2601"}.glyphicon-envelope:before{content:"\2709"}.glyphicon-pencil:before{content:"\270f"}.glyphicon-glass:before{content:"\e001"}.glyphicon-music:before{content:"\e002"}.glyphicon-search:before{content:"\e003"}.glyphicon-heart:before{content:"\e005"}.glyphicon-star:before{content:"\e006"}.glyphicon-star-empty:before{content:"\e007"}.glyphicon-user:before{content:"\e008"}.glyphicon-film:before{content:"\e009"}.glyphicon-th-large:before{content:"\e010"}.glyphicon-th:before{content:"\e011"}.glyphicon-th-list:before{content:"\e012"}.glyphicon-ok:before{content:"\e013"}.glyphicon-remove:before{content:"\e014"}.glyphicon-zoom-in:before{content:"\e015"}.glyphicon-zoom-out:before{content:"\e016"}.glyphicon-off:before{content:"\e017"}.glyphicon-signal:before{content:"\e018"}.glyphicon-cog:before{content:"\e019"}.glyphicon-trash:before{content:"\e020"}.glyphicon-home:before{content:"\e021"}.glyphicon-file:before{content:"\e022"}.glyphicon-time:before{content:"\e023"}.glyphicon-road:before{content:"\e024"}.glyphicon-download-alt:before{content:"\e025"}.glyphicon-download:before{content:"\e026"}.glyphicon-upload:before{content:"\e027"}.glyphicon-inbox:before{content:"\e028"}.glyphicon-play-circle:before{content:"\e029"}.glyphicon-repeat:before{content:"\e030"}.glyphicon-refresh:before{content:"\e031"}.glyphicon-list-alt:before{content:"\e032"}.glyphicon-lock:before{content:"\e033"}.glyphicon-flag:before{content:"\e034"}.glyphicon-headphones:before{content:"\e035"}.glyphicon-volume-off:before{content:"\e036"}.glyphicon-volume-down:before{content:"\e037"}.glyphicon-volume-up:before{content:"\e038"}.glyphicon-qrcode:before{content:"\e039"}.glyphicon-barcode:before{content:"\e040"}.glyphicon-tag:before{content:"\e041"}.glyphicon-tags:before{content:"\e042"}.glyphicon-book:before{content:"\e043"}.glyphicon-bookmark:before{content:"\e044"}.glyphicon-print:before{content:"\e045"}.glyphicon-camera:before{content:"\e046"}.glyphicon-font:before{content:"\e047"}.glyphicon-bold:before{content:"\e048"}.glyphicon-italic:before{content:"\e049"}.glyphicon-text-height:before{content:"\e050"}.glyphicon-text-width:before{content:"\e051"}.glyphicon-align-left:before{content:"\e052"}.glyphicon-align-center:before{content:"\e053"}.glyphicon-align-right:before{content:"\e054"}.glyphicon-align-justify:before{content:"\e055"}.glyphicon-list:before{content:"\e056"}.glyphicon-indent-left:before{content:"\e057"}.glyphicon-indent-right:before{content:"\e058"}.glyphicon-facetime-video:before{content:"\e059"}.glyphicon-picture:before{content:"\e060"}.glyphicon-map-marker:before{content:"\e062"}.glyphicon-adjust:before{content:"\e063"}.glyphicon-tint:before{content:"\e064"}.glyphicon-edit:before{content:"\e065"}.glyphicon-share:before{content:"\e066"}.glyphicon-check:before{content:"\e067"}.glyphicon-move:before{content:"\e068"}.glyphicon-step-backward:before{content:"\e069"}.glyphicon-fast-backward:before{content:"\e070"}.glyphicon-backward:before{content:"\e071"}.glyphicon-play:before{content:"\e072"}.glyphicon-pause:before{content:"\e073"}.glyphicon-stop:before{content:"\e074"}.glyphicon-forward:before{content:"\e075"}.glyphicon-fast-forward:before{content:"\e076"}.glyphicon-step-forward:before{content:"\e077"}.glyphicon-eject:before{content:"\e078"}.glyphicon-chevron-left:before{content:"\e079"}.glyphicon-chevron-right:before{content:"\e080"}.glyphicon-plus-sign:before{content:"\e081"}.glyphicon-minus-sign:before{content:"\e082"}.glyphicon-remove-sign:before{content:"\e083"}.glyphicon-ok-sign:before{content:"\e084"}.glyphicon-question-sign:before{content:"\e085"}.glyphicon-info-sign:before{content:"\e086"}.glyphicon-screenshot:before{content:"\e087"}.glyphicon-remove-circle:before{content:"\e088"}.glyphicon-ok-circle:before{content:"\e089"}.glyphicon-ban-circle:before{content:"\e090"}.glyphicon-arrow-left:before{content:"\e091"}.glyphicon-arrow-right:before{content:"\e092"}.glyphicon-arrow-up:before{content:"\e093"}.glyphicon-arrow-down:before{content:"\e094"}.glyphicon-share-alt:before{content:"\e095"}.glyphicon-resize-full:before{content:"\e096"}.glyphicon-resize-small:before{content:"\e097"}.glyphicon-exclamation-sign:before{content:"\e101"}.glyphicon-gift:before{content:"\e102"}.glyphicon-leaf:before{content:"\e103"}.glyphicon-fire:before{content:"\e104"}.glyphicon-eye-open:before{content:"\e105"}.glyphicon-eye-close:before{content:"\e106"}.glyphicon-warning-sign:before{content:"\e107"}.glyphicon-plane:before{content:"\e108"}.glyphicon-calendar:before{content:"\e109"}.glyphicon-random:before{content:"\e110"}.glyphicon-comment:before{content:"\e111"}.glyphicon-magnet:before{content:"\e112"}.glyphicon-chevron-up:before{content:"\e113"}.glyphicon-chevron-down:before{content:"\e114"}.glyphicon-retweet:before{content:"\e115"}.glyphicon-shopping-cart:before{content:"\e116"}.glyphicon-folder-close:before{content:"\e117"}.glyphicon-folder-open:before{content:"\e118"}.glyphicon-resize-vertical:before{content:"\e119"}.glyphicon-resize-horizontal:before{content:"\e120"}.glyphicon-hdd:before{content:"\e121"}.glyphicon-bullhorn:before{content:"\e122"}.glyphicon-bell:before{content:"\e123"}.glyphicon-certificate:before{content:"\e124"}.glyphicon-thumbs-up:before{content:"\e125"}.glyphicon-thumbs-down:before{content:"\e126"}.glyphicon-hand-right:before{content:"\e127"}.glyphicon-hand-left:before{content:"\e128"}.glyphicon-hand-up:before{content:"\e129"}.glyphicon-hand-down:before{content:"\e130"}.glyphicon-circle-arrow-right:before{content:"\e131"}.glyphicon-circle-arrow-left:before{content:"\e132"}.glyphicon-circle-arrow-up:before{content:"\e133"}.glyphicon-circle-arrow-down:before{content:"\e134"}.glyphicon-globe:before{content:"\e135"}.glyphicon-wrench:before{content:"\e136"}.glyphicon-tasks:before{content:"\e137"}.glyphicon-filter:before{content:"\e138"}.glyphicon-briefcase:before{content:"\e139"}.glyphicon-fullscreen:before{content:"\e140"}.glyphicon-dashboard:before{content:"\e141"}.glyphicon-paperclip:before{content:"\e142"}.glyphicon-heart-empty:before{content:"\e143"}.glyphicon-link:before{content:"\e144"}.glyphicon-phone:before{content:"\e145"}.glyphicon-pushpin:before{content:"\e146"}.glyphicon-usd:before{content:"\e148"}.glyphicon-gbp:before{content:"\e149"}.glyphicon-sort:before{content:"\e150"}.glyphicon-sort-by-alphabet:before{content:"\e151"}.glyphicon-sort-by-alphabet-alt:before{content:"\e152"}.glyphicon-sort-by-order:before{content:"\e153"}.glyphicon-sort-by-order-alt:before{content:"\e154"}.glyphicon-sort-by-attributes:before{content:"\e155"}.glyphicon-sort-by-attributes-alt:before{content:"\e156"}.glyphicon-unchecked:before{content:"\e157"}.glyphicon-expand:before{content:"\e158"}.glyphicon-collapse-down:before{content:"\e159"}.glyphicon-collapse-up:before{content:"\e160"}.glyphicon-log-in:before{content:"\e161"}.glyphicon-flash:before{content:"\e162"}.glyphicon-log-out:before{content:"\e163"}.glyphicon-new-window:before{content:"\e164"}.glyphicon-record:before{content:"\e165"}.glyphicon-save:before{content:"\e166"}.glyphicon-open:before{content:"\e167"}.glyphicon-saved:before{content:"\e168"}.glyphicon-import:before{content:"\e169"}.glyphicon-export:before{content:"\e170"}.glyphicon-send:before{content:"\e171"}.glyphicon-floppy-disk:before{content:"\e172"}.glyphicon-floppy-saved:before{content:"\e173"}.glyphicon-floppy-remove:before{content:"\e174"}.glyphicon-floppy-save:before{content:"\e175"}.glyphicon-floppy-open:before{content:"\e176"}.glyphicon-credit-card:before{content:"\e177"}.glyphicon-transfer:before{content:"\e178"}.glyphicon-cutlery:before{content:"\e179"}.glyphicon-header:before{content:"\e180"}.glyphicon-compressed:before{content:"\e181"}.glyphicon-earphone:before{content:"\e182"}.glyphicon-phone-alt:before{content:"\e183"}.glyphicon-tower:before{content:"\e184"}.glyphicon-stats:before{content:"\e185"}.glyphicon-sd-video:before{content:"\e186"}.glyphicon-hd-video:before{content:"\e187"}.glyphicon-subtitles:before{content:"\e188"}.glyphicon-sound-stereo:before{content:"\e189"}.glyphicon-sound-dolby:before{content:"\e190"}.glyphicon-sound-5-1:before{content:"\e191"}.glyphicon-sound-6-1:before{content:"\e192"}.glyphicon-sound-7-1:before{content:"\e193"}.glyphicon-copyright-mark:before{content:"\e194"}.glyphicon-registration-mark:before{content:"\e195"}.glyphicon-cloud-download:before{content:"\e197"}.glyphicon-cloud-upload:before{content:"\e198"}.glyphicon-tree-conifer:before{content:"\e199"}.glyphicon-tree-deciduous:before{content:"\e200"}.caret{display:inline-block;width:0;height:0;margin-left:2px;vertical-align:middle;border-top:4px solid;border-right:4px solid transparent;border-left:4px solid transparent}.dropdown{position:relative}.dropdown-toggle:focus{outline:0}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;font-size:14px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,.15);border-radius:4px;-webkit-box-shadow:0 6px 12px rgba(0,0,0,.175);box-shadow:0 6px 12px rgba(0,0,0,.175);background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{height:1px;margin:9px 0;overflow:hidden;background-color:#e5e5e5}.dropdown-menu>li>a{display:block;padding:3px 20px;clear:both;font-weight:400;line-height:1.42857143;color:#333;white-space:nowrap}.dropdown-menu>li>a:hover,.dropdown-menu>li>a:focus{text-decoration:none;color:#262626;background-color:#f5f5f5}.dropdown-menu>.active>a,.dropdown-menu>.active>a:hover,.dropdown-menu>.active>a:focus{color:#fff;text-decoration:none;outline:0;background-color:#428bca}.dropdown-menu>.disabled>a,.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{color:#999}.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{text-decoration:none;background-color:transparent;background-image:none;filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);cursor:not-allowed}.open>.dropdown-menu{display:block}.open>a{outline:0}.dropdown-menu-right{left:auto;right:0}.dropdown-menu-left{left:0;right:auto}.dropdown-header{display:block;padding:3px 20px;font-size:12px;line-height:1.42857143;color:#999}.dropdown-backdrop{position:fixed;left:0;right:0;bottom:0;top:0;z-index:990}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid;content:""}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}@media (min-width:768px){.navbar-right .dropdown-menu{left:auto;right:0}.navbar-right .dropdown-menu-left{left:0;right:auto}}.btn-group,.btn-group-vertical{position:relative;display:inline-block;vertical-align:middle}.btn-group>.btn,.btn-group-vertical>.btn{position:relative;float:left}.btn-group>.btn:hover,.btn-group-vertical>.btn:hover,.btn-group>.btn:focus,.btn-group-vertical>.btn:focus,.btn-group>.btn:active,.btn-group-vertical>.btn:active,.btn-group>.btn.active,.btn-group-vertical>.btn.active{z-index:2}.btn-group>.btn:focus,.btn-group-vertical>.btn:focus{outline:0}.btn-group .btn+.btn,.btn-group .btn+.btn-group,.btn-group .btn-group+.btn,.btn-group .btn-group+.btn-group{margin-left:-1px}.btn-toolbar{margin-left:-5px}.btn-toolbar .btn-group,.btn-toolbar .input-group{float:left}.btn-toolbar>.btn,.btn-toolbar>.btn-group,.btn-toolbar>.input-group{margin-left:5px}.btn-group>.btn:not(:first-child):not(:last-child):not(.dropdown-toggle){border-radius:0}.btn-group>.btn:first-child{margin-left:0}.btn-group>.btn:first-child:not(:last-child):not(.dropdown-toggle){border-bottom-right-radius:0;border-top-right-radius:0}.btn-group>.btn:last-child:not(:first-child),.btn-group>.dropdown-toggle:not(:first-child){border-bottom-left-radius:0;border-top-left-radius:0}.btn-group>.btn-group{float:left}.btn-group>.btn-group:not(:first-child):not(:last-child)>.btn{border-radius:0}.btn-group>.btn-group:first-child>.btn:last-child,.btn-group>.btn-group:first-child>.dropdown-toggle{border-bottom-right-radius:0;border-top-right-radius:0}.btn-group>.btn-group:last-child>.btn:first-child{border-bottom-left-radius:0;border-top-left-radius:0}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{padding-left:8px;padding-right:8px}.btn-group>.btn-lg+.dropdown-toggle{padding-left:12px;padding-right:12px}.btn-group.open .dropdown-toggle{-webkit-box-shadow:inset 0 3px 5px rgba(0,0,0,.125);box-shadow:inset 0 3px 5px rgba(0,0,0,.125)}.btn-group.open .dropdown-toggle.btn-link{-webkit-box-shadow:none;box-shadow:none}.btn .caret{margin-left:0}.btn-lg .caret{border-width:5px 5px 0;border-bottom-width:0}.dropup .btn-lg .caret{border-width:0 5px 5px}.btn-group-vertical>.btn,.btn-group-vertical>.btn-group,.btn-group-vertical>.btn-group>.btn{display:block;float:none;width:100%;max-width:100%}.btn-group-vertical>.btn-group>.btn{float:none}.btn-group-vertical>.btn+.btn,.btn-group-vertical>.btn+.btn-group,.btn-group-vertical>.btn-group+.btn,.btn-group-vertical>.btn-group+.btn-group{margin-top:-1px;margin-left:0}.btn-group-vertical>.btn:not(:first-child):not(:last-child){border-radius:0}.btn-group-vertical>.btn:first-child:not(:last-child){border-top-right-radius:4px;border-bottom-right-radius:0;border-bottom-left-radius:0}.btn-group-vertical>.btn:last-child:not(:first-child){border-bottom-left-radius:4px;border-top-right-radius:0;border-top-left-radius:0}.btn-group-vertical>.btn-group:not(:first-child):not(:last-child)>.btn{border-radius:0}.btn-group-vertical>.btn-group:first-child:not(:last-child)>.btn:last-child,.btn-group-vertical>.btn-group:first-child:not(:last-child)>.dropdown-toggle{border-bottom-right-radius:0;border-bottom-left-radius:0}.btn-group-vertical>.btn-group:last-child:not(:first-child)>.btn:first-child{border-top-right-radius:0;border-top-left-radius:0}.btn-group-justified{display:table;width:100%;table-layout:fixed;border-collapse:separate}.btn-group-justified>.btn,.btn-group-justified>.btn-group{float:none;display:table-cell;width:1%}.btn-group-justified>.btn-group .btn{width:100%}[data-toggle=buttons]>.btn>input[type=radio],[data-toggle=buttons]>.btn>input[type=checkbox]{display:none}.input-group{position:relative;display:table;border-collapse:separate}.input-group[class*=col-]{float:none;padding-left:0;padding-right:0}.input-group .form-control{position:relative;z-index:2;float:left;width:100%;margin-bottom:0}.input-group-lg>.form-control,.input-group-lg>.input-group-addon,.input-group-lg>.input-group-btn>.btn{height:46px;padding:10px 16px;font-size:18px;line-height:1.33;border-radius:6px}select.input-group-lg>.form-control,select.input-group-lg>.input-group-addon,select.input-group-lg>.input-group-btn>.btn{height:46px;line-height:46px}textarea.input-group-lg>.form-control,textarea.input-group-lg>.input-group-addon,textarea.input-group-lg>.input-group-btn>.btn,select[multiple].input-group-lg>.form-control,select[multiple].input-group-lg>.input-group-addon,select[multiple].input-group-lg>.input-group-btn>.btn{height:auto}.input-group-sm>.form-control,.input-group-sm>.input-group-addon,.input-group-sm>.input-group-btn>.btn{height:30px;padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}select.input-group-sm>.form-control,select.input-group-sm>.input-group-addon,select.input-group-sm>.input-group-btn>.btn{height:30px;line-height:30px}textarea.input-group-sm>.form-control,textarea.input-group-sm>.input-group-addon,textarea.input-group-sm>.input-group-btn>.btn,select[multiple].input-group-sm>.form-control,select[multiple].input-group-sm>.input-group-addon,select[multiple].input-group-sm>.input-group-btn>.btn{height:auto}.input-group-addon,.input-group-btn,.input-group .form-control{display:table-cell}.input-group-addon:not(:first-child):not(:last-child),.input-group-btn:not(:first-child):not(:last-child),.input-group .form-control:not(:first-child):not(:last-child){border-radius:0}.input-group-addon,.input-group-btn{width:1%;white-space:nowrap;vertical-align:middle}.input-group-addon{padding:6px 12px;font-size:14px;font-weight:400;line-height:1;color:#555;text-align:center;background-color:#eee;border:1px solid #ccc;border-radius:4px}.input-group-addon.input-sm{padding:5px 10px;font-size:12px;border-radius:3px}.input-group-addon.input-lg{padding:10px 16px;font-size:18px;border-radius:6px}.input-group-addon input[type=radio],.input-group-addon input[type=checkbox]{margin-top:0}.input-group .form-control:first-child,.input-group-addon:first-child,.input-group-btn:first-child>.btn,.input-group-btn:first-child>.btn-group>.btn,.input-group-btn:first-child>.dropdown-toggle,.input-group-btn:last-child>.btn:not(:last-child):not(.dropdown-toggle),.input-group-btn:last-child>.btn-group:not(:last-child)>.btn{border-bottom-right-radius:0;border-top-right-radius:0}.input-group-addon:first-child{border-right:0}.input-group .form-control:last-child,.input-group-addon:last-child,.input-group-btn:last-child>.btn,.input-group-btn:last-child>.btn-group>.btn,.input-group-btn:last-child>.dropdown-toggle,.input-group-btn:first-child>.btn:not(:first-child),.input-group-btn:first-child>.btn-group:not(:first-child)>.btn{border-bottom-left-radius:0;border-top-left-radius:0}.input-group-addon:last-child{border-left:0}.input-group-btn{position:relative;font-size:0;white-space:nowrap}.input-group-btn>.btn{position:relative}.input-group-btn>.btn+.btn{margin-left:-1px}.input-group-btn>.btn:hover,.input-group-btn>.btn:focus,.input-group-btn>.btn:active{z-index:2}.input-group-btn:first-child>.btn,.input-group-btn:first-child>.btn-group{margin-right:-1px}.input-group-btn:last-child>.btn,.input-group-btn:last-child>.btn-group{margin-left:-1px}.nav{margin-bottom:0;padding-left:0;list-style:none}.nav>li{position:relative;display:block}.nav>li>a{position:relative;display:block;padding:10px 15px}.nav>li>a:hover,.nav>li>a:focus{text-decoration:none;background-color:#eee}.nav>li.disabled>a{color:#999}.nav>li.disabled>a:hover,.nav>li.disabled>a:focus{color:#999;text-decoration:none;background-color:transparent;cursor:not-allowed}.nav .open>a,.nav .open>a:hover,.nav .open>a:focus{background-color:#eee;border-color:#428bca}.nav .nav-divider{height:1px;margin:9px 0;overflow:hidden;background-color:#e5e5e5}.nav>li>a>img{max-width:none}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{float:left;margin-bottom:-1px}.nav-tabs>li>a{margin-right:2px;line-height:1.42857143;border:1px solid transparent;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>li.active>a,.nav-tabs>li.active>a:hover,.nav-tabs>li.active>a:focus{color:#555;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent;cursor:default}.nav-tabs.nav-justified{width:100%;border-bottom:0}.nav-tabs.nav-justified>li{float:none}.nav-tabs.nav-justified>li>a{text-align:center;margin-bottom:5px}.nav-tabs.nav-justified>.dropdown .dropdown-menu{top:auto;left:auto}@media (min-width:768px){.nav-tabs.nav-justified>li{display:table-cell;width:1%}.nav-tabs.nav-justified>li>a{margin-bottom:0}}.nav-tabs.nav-justified>li>a{margin-right:0;border-radius:4px}.nav-tabs.nav-justified>.active>a,.nav-tabs.nav-justified>.active>a:hover,.nav-tabs.nav-justified>.active>a:focus{border:1px solid #ddd}@media (min-width:768px){.nav-tabs.nav-justified>li>a{border-bottom:1px solid #ddd;border-radius:4px 4px 0 0}.nav-tabs.nav-justified>.active>a,.nav-tabs.nav-justified>.active>a:hover,.nav-tabs.nav-justified>.active>a:focus{border-bottom-color:#fff}}.nav-pills>li{float:left}.nav-pills>li>a{border-radius:4px}.nav-pills>li+li{margin-left:2px}.nav-pills>li.active>a,.nav-pills>li.active>a:hover,.nav-pills>li.active>a:focus{color:#fff;background-color:#428bca}.nav-stacked>li{float:none}.nav-stacked>li+li{margin-top:2px;margin-left:0}.nav-justified{width:100%}.nav-justified>li{float:none}.nav-justified>li>a{text-align:center;margin-bottom:5px}.nav-justified>.dropdown .dropdown-menu{top:auto;left:auto}@media (min-width:768px){.nav-justified>li{display:table-cell;width:1%}.nav-justified>li>a{margin-bottom:0}}.nav-tabs-justified{border-bottom:0}.nav-tabs-justified>li>a{margin-right:0;border-radius:4px}.nav-tabs-justified>.active>a,.nav-tabs-justified>.active>a:hover,.nav-tabs-justified>.active>a:focus{border:1px solid #ddd}@media (min-width:768px){.nav-tabs-justified>li>a{border-bottom:1px solid #ddd;border-radius:4px 4px 0 0}.nav-tabs-justified>.active>a,.nav-tabs-justified>.active>a:hover,.nav-tabs-justified>.active>a:focus{border-bottom-color:#fff}}.tab-content>.tab-pane{display:none}.tab-content>.active{display:block}.nav-tabs .dropdown-menu{margin-top:-1px;border-top-right-radius:0;border-top-left-radius:0}.navbar{position:relative;min-height:50px;margin-bottom:20px;border:1px solid transparent}@media (min-width:768px){.navbar{border-radius:4px}}@media (min-width:768px){.navbar-header{float:left}}.navbar-collapse{max-height:340px;overflow-x:visible;padding-right:15px;padding-left:15px;border-top:1px solid transparent;box-shadow:inset 0 1px 0 rgba(255,255,255,.1);-webkit-overflow-scrolling:touch}.navbar-collapse.in{overflow-y:auto}@media (min-width:768px){.navbar-collapse{width:auto;border-top:0;box-shadow:none}.navbar-collapse.collapse{display:block!important;height:auto!important;padding-bottom:0;overflow:visible!important}.navbar-collapse.in{overflow-y:visible}.navbar-fixed-top .navbar-collapse,.navbar-static-top .navbar-collapse,.navbar-fixed-bottom .navbar-collapse{padding-left:0;padding-right:0}}.container>.navbar-header,.container-fluid>.navbar-header,.container>.navbar-collapse,.container-fluid>.navbar-collapse{margin-right:-15px;margin-left:-15px}@media (min-width:768px){.container>.navbar-header,.container-fluid>.navbar-header,.container>.navbar-collapse,.container-fluid>.navbar-collapse{margin-right:0;margin-left:0}}.navbar-static-top{z-index:1000;border-width:0 0 1px}@media (min-width:768px){.navbar-static-top{border-radius:0}}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030}@media (min-width:768px){.navbar-fixed-top,.navbar-fixed-bottom{border-radius:0}}.navbar-fixed-top{top:0;border-width:0 0 1px}.navbar-fixed-bottom{bottom:0;margin-bottom:0;border-width:1px 0 0}.navbar-brand{float:left;padding:15px;font-size:18px;line-height:20px;height:50px}.navbar-brand:hover,.navbar-brand:focus{text-decoration:none}@media (min-width:768px){.navbar>.container .navbar-brand,.navbar>.container-fluid .navbar-brand{margin-left:-15px}}.navbar-toggle{position:relative;float:right;margin-right:15px;padding:9px 10px;margin-top:8px;margin-bottom:8px;background-color:transparent;background-image:none;border:1px solid transparent;border-radius:4px}.navbar-toggle:focus{outline:0}.navbar-toggle .icon-bar{display:block;width:22px;height:2px;border-radius:1px}.navbar-toggle .icon-bar+.icon-bar{margin-top:4px}@media (min-width:768px){.navbar-toggle{display:none}}.navbar-nav{margin:7.5px -15px}.navbar-nav>li>a{padding-top:10px;padding-bottom:10px;line-height:20px}@media (max-width:767px){.navbar-nav .open .dropdown-menu{position:static;float:none;width:auto;margin-top:0;background-color:transparent;border:0;box-shadow:none}.navbar-nav .open .dropdown-menu>li>a,.navbar-nav .open .dropdown-menu .dropdown-header{padding:5px 15px 5px 25px}.navbar-nav .open .dropdown-menu>li>a{line-height:20px}.navbar-nav .open .dropdown-menu>li>a:hover,.navbar-nav .open .dropdown-menu>li>a:focus{background-image:none}}@media (min-width:768px){.navbar-nav{float:left;margin:0}.navbar-nav>li{float:left}.navbar-nav>li>a{padding-top:15px;padding-bottom:15px}.navbar-nav.navbar-right:last-child{margin-right:-15px}}@media (min-width:768px){.navbar-left{float:left!important}.navbar-right{float:right!important}}.navbar-form{margin-left:-15px;margin-right:-15px;padding:10px 15px;border-top:1px solid transparent;border-bottom:1px solid transparent;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,.1),0 1px 0 rgba(255,255,255,.1);box-shadow:inset 0 1px 0 rgba(255,255,255,.1),0 1px 0 rgba(255,255,255,.1);margin-top:8px;margin-bottom:8px}@media (min-width:768px){.navbar-form .form-group{display:inline-block;margin-bottom:0;vertical-align:middle}.navbar-form .form-control{display:inline-block;width:auto;vertical-align:middle}.navbar-form .input-group>.form-control{width:100%}.navbar-form .control-label{margin-bottom:0;vertical-align:middle}.navbar-form .radio,.navbar-form .checkbox{display:inline-block;margin-top:0;margin-bottom:0;padding-left:0;vertical-align:middle}.navbar-form .radio input[type=radio],.navbar-form .checkbox input[type=checkbox]{float:none;margin-left:0}.navbar-form .has-feedback .form-control-feedback{top:0}}@media (max-width:767px){.navbar-form .form-group{margin-bottom:5px}}@media (min-width:768px){.navbar-form{width:auto;border:0;margin-left:0;margin-right:0;padding-top:0;padding-bottom:0;-webkit-box-shadow:none;box-shadow:none}.navbar-form.navbar-right:last-child{margin-right:-15px}}.navbar-nav>li>.dropdown-menu{margin-top:0;border-top-right-radius:0;border-top-left-radius:0}.navbar-fixed-bottom .navbar-nav>li>.dropdown-menu{border-bottom-right-radius:0;border-bottom-left-radius:0}.navbar-btn{margin-top:8px;margin-bottom:8px}.navbar-btn.btn-sm{margin-top:10px;margin-bottom:10px}.navbar-btn.btn-xs{margin-top:14px;margin-bottom:14px}.navbar-text{margin-top:15px;margin-bottom:15px}@media (min-width:768px){.navbar-text{float:left;margin-left:15px;margin-right:15px}.navbar-text.navbar-right:last-child{margin-right:0}}.navbar-default{background-color:#f8f8f8;border-color:#e7e7e7}.navbar-default .navbar-brand{color:#777}.navbar-default .navbar-brand:hover,.navbar-default .navbar-brand:focus{color:#5e5e5e;background-color:transparent}.navbar-default .navbar-text{color:#777}.navbar-default .navbar-nav>li>a{color:#777}.navbar-default .navbar-nav>li>a:hover,.navbar-default .navbar-nav>li>a:focus{color:#333;background-color:transparent}.navbar-default .navbar-nav>.active>a,.navbar-default .navbar-nav>.active>a:hover,.navbar-default .navbar-nav>.active>a:focus{color:#555;background-color:#e7e7e7}.navbar-default .navbar-nav>.disabled>a,.navbar-default .navbar-nav>.disabled>a:hover,.navbar-default .navbar-nav>.disabled>a:focus{color:#ccc;background-color:transparent}.navbar-default .navbar-toggle{border-color:#ddd}.navbar-default .navbar-toggle:hover,.navbar-default .navbar-toggle:focus{background-color:#ddd}.navbar-default .navbar-toggle .icon-bar{background-color:#888}.navbar-default .navbar-collapse,.navbar-default .navbar-form{border-color:#e7e7e7}.navbar-default .navbar-nav>.open>a,.navbar-default .navbar-nav>.open>a:hover,.navbar-default .navbar-nav>.open>a:focus{background-color:#e7e7e7;color:#555}@media (max-width:767px){.navbar-default .navbar-nav .open .dropdown-menu>li>a{color:#777}.navbar-default .navbar-nav .open .dropdown-menu>li>a:hover,.navbar-default .navbar-nav .open .dropdown-menu>li>a:focus{color:#333;background-color:transparent}.navbar-default .navbar-nav .open .dropdown-menu>.active>a,.navbar-default .navbar-nav .open .dropdown-menu>.active>a:hover,.navbar-default .navbar-nav .open .dropdown-menu>.active>a:focus{color:#555;background-color:#e7e7e7}.navbar-default .navbar-nav .open .dropdown-menu>.disabled>a,.navbar-default .navbar-nav .open .dropdown-menu>.disabled>a:hover,.navbar-default .navbar-nav .open .dropdown-menu>.disabled>a:focus{color:#ccc;background-color:transparent}}.navbar-default .navbar-link{color:#777}.navbar-default .navbar-link:hover{color:#333}.navbar-inverse{background-color:#222;border-color:#080808}.navbar-inverse .navbar-brand{color:#999}.navbar-inverse .navbar-brand:hover,.navbar-inverse .navbar-brand:focus{color:#fff;background-color:transparent}.navbar-inverse .navbar-text{color:#999}.navbar-inverse .navbar-nav>li>a{color:#999}.navbar-inverse .navbar-nav>li>a:hover,.navbar-inverse .navbar-nav>li>a:focus{color:#fff;background-color:transparent}.navbar-inverse .navbar-nav>.active>a,.navbar-inverse .navbar-nav>.active>a:hover,.navbar-inverse .navbar-nav>.active>a:focus{color:#fff;background-color:#080808}.navbar-inverse .navbar-nav>.disabled>a,.navbar-inverse .navbar-nav>.disabled>a:hover,.navbar-inverse .navbar-nav>.disabled>a:focus{color:#444;background-color:transparent}.navbar-inverse .navbar-toggle{border-color:#333}.navbar-inverse .navbar-toggle:hover,.navbar-inverse .navbar-toggle:focus{background-color:#333}.navbar-inverse .navbar-toggle .icon-bar{background-color:#fff}.navbar-inverse .navbar-collapse,.navbar-inverse .navbar-form{border-color:#101010}.navbar-inverse .navbar-nav>.open>a,.navbar-inverse .navbar-nav>.open>a:hover,.navbar-inverse .navbar-nav>.open>a:focus{background-color:#080808;color:#fff}@media (max-width:767px){.navbar-inverse .navbar-nav .open .dropdown-menu>.dropdown-header{border-color:#080808}.navbar-inverse .navbar-nav .open .dropdown-menu .divider{background-color:#080808}.navbar-inverse .navbar-nav .open .dropdown-menu>li>a{color:#999}.navbar-inverse .navbar-nav .open .dropdown-menu>li>a:hover,.navbar-inverse .navbar-nav .open .dropdown-menu>li>a:focus{color:#fff;background-color:transparent}.navbar-inverse .navbar-nav .open .dropdown-menu>.active>a,.navbar-inverse .navbar-nav .open .dropdown-menu>.active>a:hover,.navbar-inverse .navbar-nav .open .dropdown-menu>.active>a:focus{color:#fff;background-color:#080808}.navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a,.navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a:hover,.navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a:focus{color:#444;background-color:transparent}}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.breadcrumb{padding:8px 15px;margin-bottom:20px;list-style:none;background-color:#f5f5f5;border-radius:4px}.breadcrumb>li{display:inline-block}.breadcrumb>li+li:before{content:"/\00a0";padding:0 5px;color:#ccc}.breadcrumb>.active{color:#999}.pagination{display:inline-block;padding-left:0;margin:20px 0;border-radius:4px}.pagination>li{display:inline}.pagination>li>a,.pagination>li>span{position:relative;float:left;padding:6px 12px;line-height:1.42857143;text-decoration:none;color:#428bca;background-color:#fff;border:1px solid #ddd;margin-left:-1px}.pagination>li:first-child>a,.pagination>li:first-child>span{margin-left:0;border-bottom-left-radius:4px;border-top-left-radius:4px}.pagination>li:last-child>a,.pagination>li:last-child>span{border-bottom-right-radius:4px;border-top-right-radius:4px}.pagination>li>a:hover,.pagination>li>span:hover,.pagination>li>a:focus,.pagination>li>span:focus{color:#2a6496;background-color:#eee;border-color:#ddd}.pagination>.active>a,.pagination>.active>span,.pagination>.active>a:hover,.pagination>.active>span:hover,.pagination>.active>a:focus,.pagination>.active>span:focus{z-index:2;color:#fff;background-color:#428bca;border-color:#428bca;cursor:default}.pagination>.disabled>span,.pagination>.disabled>span:hover,.pagination>.disabled>span:focus,.pagination>.disabled>a,.pagination>.disabled>a:hover,.pagination>.disabled>a:focus{color:#999;background-color:#fff;border-color:#ddd;cursor:not-allowed}.pagination-lg>li>a,.pagination-lg>li>span{padding:10px 16px;font-size:18px}.pagination-lg>li:first-child>a,.pagination-lg>li:first-child>span{border-bottom-left-radius:6px;border-top-left-radius:6px}.pagination-lg>li:last-child>a,.pagination-lg>li:last-child>span{border-bottom-right-radius:6px;border-top-right-radius:6px}.pagination-sm>li>a,.pagination-sm>li>span{padding:5px 10px;font-size:12px}.pagination-sm>li:first-child>a,.pagination-sm>li:first-child>span{border-bottom-left-radius:3px;border-top-left-radius:3px}.pagination-sm>li:last-child>a,.pagination-sm>li:last-child>span{border-bottom-right-radius:3px;border-top-right-radius:3px}.pager{padding-left:0;margin:20px 0;list-style:none;text-align:center}.pager li{display:inline}.pager li>a,.pager li>span{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;border-radius:15px}.pager li>a:hover,.pager li>a:focus{text-decoration:none;background-color:#eee}.pager .next>a,.pager .next>span{float:right}.pager .previous>a,.pager .previous>span{float:left}.pager .disabled>a,.pager .disabled>a:hover,.pager .disabled>a:focus,.pager .disabled>span{color:#999;background-color:#fff;cursor:not-allowed}.label{display:inline;padding:.2em .6em .3em;font-size:75%;font-weight:700;line-height:1;color:#fff;text-align:center;white-space:nowrap;vertical-align:baseline;border-radius:.25em}.label[href]:hover,.label[href]:focus{color:#fff;text-decoration:none;cursor:pointer}.label:empty{display:none}.btn .label{position:relative;top:-1px}.label-default{background-color:#999}.label-default[href]:hover,.label-default[href]:focus{background-color:gray}.label-primary{background-color:#428bca}.label-primary[href]:hover,.label-primary[href]:focus{background-color:#3071a9}.label-success{background-color:#5cb85c}.label-success[href]:hover,.label-success[href]:focus{background-color:#449d44}.label-info{background-color:#5bc0de}.label-info[href]:hover,.label-info[href]:focus{background-color:#31b0d5}.label-warning{background-color:#f0ad4e}.label-warning[href]:hover,.label-warning[href]:focus{background-color:#ec971f}.label-danger{background-color:#d9534f}.label-danger[href]:hover,.label-danger[href]:focus{background-color:#c9302c}.badge{display:inline-block;min-width:10px;padding:3px 7px;font-size:12px;font-weight:700;color:#fff;line-height:1;vertical-align:baseline;white-space:nowrap;text-align:center;background-color:#999;border-radius:10px}.badge:empty{display:none}.btn .badge{position:relative;top:-1px}.btn-xs .badge{top:0;padding:1px 5px}a.badge:hover,a.badge:focus{color:#fff;text-decoration:none;cursor:pointer}a.list-group-item.active>.badge,.nav-pills>.active>a>.badge{color:#428bca;background-color:#fff}.nav-pills>li>a>.badge{margin-left:3px}.jumbotron{padding:30px;margin-bottom:30px;color:inherit;background-color:#eee}.jumbotron h1,.jumbotron .h1{color:inherit}.jumbotron p{margin-bottom:15px;font-size:21px;font-weight:200}.container .jumbotron{border-radius:6px}.jumbotron .container{max-width:100%}@media screen and (min-width:768px){.jumbotron{padding-top:48px;padding-bottom:48px}.container .jumbotron{padding-left:60px;padding-right:60px}.jumbotron h1,.jumbotron .h1{font-size:63px}}.thumbnail{display:block;padding:4px;margin-bottom:20px;line-height:1.42857143;background-color:#fff;border:1px solid #ddd;border-radius:4px;-webkit-transition:all .2s ease-in-out;transition:all .2s ease-in-out}.thumbnail>img,.thumbnail a>img{margin-left:auto;margin-right:auto}a.thumbnail:hover,a.thumbnail:focus,a.thumbnail.active{border-color:#428bca}.thumbnail .caption{padding:9px;color:#333}.alert{padding:15px;margin-bottom:20px;border:1px solid transparent;border-radius:4px}.alert h4{margin-top:0;color:inherit}.alert .alert-link{font-weight:700}.alert>p,.alert>ul{margin-bottom:0}.alert>p+p{margin-top:5px}.alert-dismissable{padding-right:35px}.alert-dismissable .close{position:relative;top:-2px;right:-21px;color:inherit}.alert-success{background-color:#dff0d8;border-color:#d6e9c6;color:#3c763d}.alert-success hr{border-top-color:#c9e2b3}.alert-success .alert-link{color:#2b542c}.alert-info{background-color:#d9edf7;border-color:#bce8f1;color:#31708f}.alert-info hr{border-top-color:#a6e1ec}.alert-info .alert-link{color:#245269}.alert-warning{background-color:#fcf8e3;border-color:#faebcc;color:#8a6d3b}.alert-warning hr{border-top-color:#f7e1b5}.alert-warning .alert-link{color:#66512c}.alert-danger{background-color:#f2dede;border-color:#ebccd1;color:#a94442}.alert-danger hr{border-top-color:#e4b9c0}.alert-danger .alert-link{color:#843534}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{overflow:hidden;height:20px;margin-bottom:20px;background-color:#f5f5f5;border-radius:4px;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,.1);box-shadow:inset 0 1px 2px rgba(0,0,0,.1)}.progress-bar{float:left;width:0;height:100%;font-size:12px;line-height:20px;color:#fff;text-align:center;background-color:#428bca;-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,.15);-webkit-transition:width .6s ease;transition:width .6s ease}.progress-striped .progress-bar{background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-size:40px 40px}.progress.active .progress-bar{-webkit-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-bar-success{background-color:#5cb85c}.progress-striped .progress-bar-success{background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent)}.progress-bar-info{background-color:#5bc0de}.progress-striped .progress-bar-info{background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent)}.progress-bar-warning{background-color:#f0ad4e}.progress-striped .progress-bar-warning{background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent)}.progress-bar-danger{background-color:#d9534f}.progress-striped .progress-bar-danger{background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent)}.media,.media-body{overflow:hidden;zoom:1}.media,.media .media{margin-top:15px}.media:first-child{margin-top:0}.media-object{display:block}.media-heading{margin:0 0 5px}.media>.pull-left{margin-right:10px}.media>.pull-right{margin-left:10px}.media-list{padding-left:0;list-style:none}.list-group{margin-bottom:20px;padding-left:0}.list-group-item{position:relative;display:block;padding:10px 15px;margin-bottom:-1px;background-color:#fff;border:1px solid #ddd}.list-group-item:first-child{border-top-right-radius:4px;border-top-left-radius:4px}.list-group-item:last-child{margin-bottom:0;border-bottom-right-radius:4px;border-bottom-left-radius:4px}.list-group-item>.badge{float:right}.list-group-item>.badge+.badge{margin-right:5px}a.list-group-item{color:#555}a.list-group-item .list-group-item-heading{color:#333}a.list-group-item:hover,a.list-group-item:focus{text-decoration:none;background-color:#f5f5f5}a.list-group-item.active,a.list-group-item.active:hover,a.list-group-item.active:focus{z-index:2;color:#fff;background-color:#428bca;border-color:#428bca}a.list-group-item.active .list-group-item-heading,a.list-group-item.active:hover .list-group-item-heading,a.list-group-item.active:focus .list-group-item-heading{color:inherit}a.list-group-item.active .list-group-item-text,a.list-group-item.active:hover .list-group-item-text,a.list-group-item.active:focus .list-group-item-text{color:#e1edf7}.list-group-item-success{color:#3c763d;background-color:#dff0d8}a.list-group-item-success{color:#3c763d}a.list-group-item-success .list-group-item-heading{color:inherit}a.list-group-item-success:hover,a.list-group-item-success:focus{color:#3c763d;background-color:#d0e9c6}a.list-group-item-success.active,a.list-group-item-success.active:hover,a.list-group-item-success.active:focus{color:#fff;background-color:#3c763d;border-color:#3c763d}.list-group-item-info{color:#31708f;background-color:#d9edf7}a.list-group-item-info{color:#31708f}a.list-group-item-info .list-group-item-heading{color:inherit}a.list-group-item-info:hover,a.list-group-item-info:focus{color:#31708f;background-color:#c4e3f3}a.list-group-item-info.active,a.list-group-item-info.active:hover,a.list-group-item-info.active:focus{color:#fff;background-color:#31708f;border-color:#31708f}.list-group-item-warning{color:#8a6d3b;background-color:#fcf8e3}a.list-group-item-warning{color:#8a6d3b}a.list-group-item-warning .list-group-item-heading{color:inherit}a.list-group-item-warning:hover,a.list-group-item-warning:focus{color:#8a6d3b;background-color:#faf2cc}a.list-group-item-warning.active,a.list-group-item-warning.active:hover,a.list-group-item-warning.active:focus{color:#fff;background-color:#8a6d3b;border-color:#8a6d3b}.list-group-item-danger{color:#a94442;background-color:#f2dede}a.list-group-item-danger{color:#a94442}a.list-group-item-danger .list-group-item-heading{color:inherit}a.list-group-item-danger:hover,a.list-group-item-danger:focus{color:#a94442;background-color:#ebcccc}a.list-group-item-danger.active,a.list-group-item-danger.active:hover,a.list-group-item-danger.active:focus{color:#fff;background-color:#a94442;border-color:#a94442}.list-group-item-heading{margin-top:0;margin-bottom:5px}.list-group-item-text{margin-bottom:0;line-height:1.3}.panel{margin-bottom:20px;background-color:#fff;border:1px solid transparent;border-radius:4px;-webkit-box-shadow:0 1px 1px rgba(0,0,0,.05);box-shadow:0 1px 1px rgba(0,0,0,.05)}.panel-body{padding:15px}.panel-heading{padding:10px 15px;border-bottom:1px solid transparent;border-top-right-radius:3px;border-top-left-radius:3px}.panel-heading>.dropdown .dropdown-toggle{color:inherit}.panel-title{margin-top:0;margin-bottom:0;font-size:16px;color:inherit}.panel-title>a{color:inherit}.panel-footer{padding:10px 15px;background-color:#f5f5f5;border-top:1px solid #ddd;border-bottom-right-radius:3px;border-bottom-left-radius:3px}.panel>.list-group{margin-bottom:0}.panel>.list-group .list-group-item{border-width:1px 0;border-radius:0}.panel>.list-group:first-child .list-group-item:first-child{border-top:0;border-top-right-radius:3px;border-top-left-radius:3px}.panel>.list-group:last-child .list-group-item:last-child{border-bottom:0;border-bottom-right-radius:3px;border-bottom-left-radius:3px}.panel-heading+.list-group .list-group-item:first-child{border-top-width:0}.panel>.table,.panel>.table-responsive>.table{margin-bottom:0}.panel>.table:first-child,.panel>.table-responsive:first-child>.table:first-child{border-top-right-radius:3px;border-top-left-radius:3px}.panel>.table:first-child>thead:first-child>tr:first-child td:first-child,.panel>.table-responsive:first-child>.table:first-child>thead:first-child>tr:first-child td:first-child,.panel>.table:first-child>tbody:first-child>tr:first-child td:first-child,.panel>.table-responsive:first-child>.table:first-child>tbody:first-child>tr:first-child td:first-child,.panel>.table:first-child>thead:first-child>tr:first-child th:first-child,.panel>.table-responsive:first-child>.table:first-child>thead:first-child>tr:first-child th:first-child,.panel>.table:first-child>tbody:first-child>tr:first-child th:first-child,.panel>.table-responsive:first-child>.table:first-child>tbody:first-child>tr:first-child th:first-child{border-top-left-radius:3px}.panel>.table:first-child>thead:first-child>tr:first-child td:last-child,.panel>.table-responsive:first-child>.table:first-child>thead:first-child>tr:first-child td:last-child,.panel>.table:first-child>tbody:first-child>tr:first-child td:last-child,.panel>.table-responsive:first-child>.table:first-child>tbody:first-child>tr:first-child td:last-child,.panel>.table:first-child>thead:first-child>tr:first-child th:last-child,.panel>.table-responsive:first-child>.table:first-child>thead:first-child>tr:first-child th:last-child,.panel>.table:first-child>tbody:first-child>tr:first-child th:last-child,.panel>.table-responsive:first-child>.table:first-child>tbody:first-child>tr:first-child th:last-child{border-top-right-radius:3px}.panel>.table:last-child,.panel>.table-responsive:last-child>.table:last-child{border-bottom-right-radius:3px;border-bottom-left-radius:3px}.panel>.table:last-child>tbody:last-child>tr:last-child td:first-child,.panel>.table-responsive:last-child>.table:last-child>tbody:last-child>tr:last-child td:first-child,.panel>.table:last-child>tfoot:last-child>tr:last-child td:first-child,.panel>.table-responsive:last-child>.table:last-child>tfoot:last-child>tr:last-child td:first-child,.panel>.table:last-child>tbody:last-child>tr:last-child th:first-child,.panel>.table-responsive:last-child>.table:last-child>tbody:last-child>tr:last-child th:first-child,.panel>.table:last-child>tfoot:last-child>tr:last-child th:first-child,.panel>.table-responsive:last-child>.table:last-child>tfoot:last-child>tr:last-child th:first-child{border-bottom-left-radius:3px}.panel>.table:last-child>tbody:last-child>tr:last-child td:last-child,.panel>.table-responsive:last-child>.table:last-child>tbody:last-child>tr:last-child td:last-child,.panel>.table:last-child>tfoot:last-child>tr:last-child td:last-child,.panel>.table-responsive:last-child>.table:last-child>tfoot:last-child>tr:last-child td:last-child,.panel>.table:last-child>tbody:last-child>tr:last-child th:last-child,.panel>.table-responsive:last-child>.table:last-child>tbody:last-child>tr:last-child th:last-child,.panel>.table:last-child>tfoot:last-child>tr:last-child th:last-child,.panel>.table-responsive:last-child>.table:last-child>tfoot:last-child>tr:last-child th:last-child{border-bottom-right-radius:3px}.panel>.panel-body+.table,.panel>.panel-body+.table-responsive{border-top:1px solid #ddd}.panel>.table>tbody:first-child>tr:first-child th,.panel>.table>tbody:first-child>tr:first-child td{border-top:0}.panel>.table-bordered,.panel>.table-responsive>.table-bordered{border:0}.panel>.table-bordered>thead>tr>th:first-child,.panel>.table-responsive>.table-bordered>thead>tr>th:first-child,.panel>.table-bordered>tbody>tr>th:first-child,.panel>.table-responsive>.table-bordered>tbody>tr>th:first-child,.panel>.table-bordered>tfoot>tr>th:first-child,.panel>.table-responsive>.table-bordered>tfoot>tr>th:first-child,.panel>.table-bordered>thead>tr>td:first-child,.panel>.table-responsive>.table-bordered>thead>tr>td:first-child,.panel>.table-bordered>tbody>tr>td:first-child,.panel>.table-responsive>.table-bordered>tbody>tr>td:first-child,.panel>.table-bordered>tfoot>tr>td:first-child,.panel>.table-responsive>.table-bordered>tfoot>tr>td:first-child{border-left:0}.panel>.table-bordered>thead>tr>th:last-child,.panel>.table-responsive>.table-bordered>thead>tr>th:last-child,.panel>.table-bordered>tbody>tr>th:last-child,.panel>.table-responsive>.table-bordered>tbody>tr>th:last-child,.panel>.table-bordered>tfoot>tr>th:last-child,.panel>.table-responsive>.table-bordered>tfoot>tr>th:last-child,.panel>.table-bordered>thead>tr>td:last-child,.panel>.table-responsive>.table-bordered>thead>tr>td:last-child,.panel>.table-bordered>tbody>tr>td:last-child,.panel>.table-responsive>.table-bordered>tbody>tr>td:last-child,.panel>.table-bordered>tfoot>tr>td:last-child,.panel>.table-responsive>.table-bordered>tfoot>tr>td:last-child{border-right:0}.panel>.table-bordered>thead>tr:first-child>td,.panel>.table-responsive>.table-bordered>thead>tr:first-child>td,.panel>.table-bordered>tbody>tr:first-child>td,.panel>.table-responsive>.table-bordered>tbody>tr:first-child>td,.panel>.table-bordered>thead>tr:first-child>th,.panel>.table-responsive>.table-bordered>thead>tr:first-child>th,.panel>.table-bordered>tbody>tr:first-child>th,.panel>.table-responsive>.table-bordered>tbody>tr:first-child>th{border-bottom:0}.panel>.table-bordered>tbody>tr:last-child>td,.panel>.table-responsive>.table-bordered>tbody>tr:last-child>td,.panel>.table-bordered>tfoot>tr:last-child>td,.panel>.table-responsive>.table-bordered>tfoot>tr:last-child>td,.panel>.table-bordered>tbody>tr:last-child>th,.panel>.table-responsive>.table-bordered>tbody>tr:last-child>th,.panel>.table-bordered>tfoot>tr:last-child>th,.panel>.table-responsive>.table-bordered>tfoot>tr:last-child>th{border-bottom:0}.panel>.table-responsive{border:0;margin-bottom:0}.panel-group{margin-bottom:20px}.panel-group .panel{margin-bottom:0;border-radius:4px;overflow:hidden}.panel-group .panel+.panel{margin-top:5px}.panel-group .panel-heading{border-bottom:0}.panel-group .panel-heading+.panel-collapse .panel-body{border-top:1px solid #ddd}.panel-group .panel-footer{border-top:0}.panel-group .panel-footer+.panel-collapse .panel-body{border-bottom:1px solid #ddd}.panel-default{border-color:#ddd}.panel-default>.panel-heading{color:#333;background-color:#f5f5f5;border-color:#ddd}.panel-default>.panel-heading+.panel-collapse .panel-body{border-top-color:#ddd}.panel-default>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#ddd}.panel-primary{border-color:#428bca}.panel-primary>.panel-heading{color:#fff;background-color:#428bca;border-color:#428bca}.panel-primary>.panel-heading+.panel-collapse .panel-body{border-top-color:#428bca}.panel-primary>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#428bca}.panel-success{border-color:#d6e9c6}.panel-success>.panel-heading{color:#3c763d;background-color:#dff0d8;border-color:#d6e9c6}.panel-success>.panel-heading+.panel-collapse .panel-body{border-top-color:#d6e9c6}.panel-success>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#d6e9c6}.panel-info{border-color:#bce8f1}.panel-info>.panel-heading{color:#31708f;background-color:#d9edf7;border-color:#bce8f1}.panel-info>.panel-heading+.panel-collapse .panel-body{border-top-color:#bce8f1}.panel-info>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#bce8f1}.panel-warning{border-color:#faebcc}.panel-warning>.panel-heading{color:#8a6d3b;background-color:#fcf8e3;border-color:#faebcc}.panel-warning>.panel-heading+.panel-collapse .panel-body{border-top-color:#faebcc}.panel-warning>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#faebcc}.panel-danger{border-color:#ebccd1}.panel-danger>.panel-heading{color:#a94442;background-color:#f2dede;border-color:#ebccd1}.panel-danger>.panel-heading+.panel-collapse .panel-body{border-top-color:#ebccd1}.panel-danger>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#ebccd1}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.05);box-shadow:inset 0 1px 1px rgba(0,0,0,.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,.15)}.well-lg{padding:24px;border-radius:6px}.well-sm{padding:9px;border-radius:3px}.close{float:right;font-size:21px;font-weight:700;line-height:1;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover,.close:focus{color:#000;text-decoration:none;cursor:pointer;opacity:.5;filter:alpha(opacity=50)}button.close{padding:0;cursor:pointer;background:0 0;border:0;-webkit-appearance:none}.modal-open{overflow:hidden}.modal{display:none;overflow:auto;overflow-y:scroll;position:fixed;top:0;right:0;bottom:0;left:0;z-index:1050;-webkit-overflow-scrolling:touch;outline:0}.modal.fade .modal-dialog{-webkit-transform:translate(0,-25%);-ms-transform:translate(0,-25%);transform:translate(0,-25%);-webkit-transition:-webkit-transform .3s ease-out;-moz-transition:-moz-transform .3s ease-out;-o-transition:-o-transform .3s ease-out;transition:transform .3s ease-out}.modal.in .modal-dialog{-webkit-transform:translate(0,0);-ms-transform:translate(0,0);transform:translate(0,0)}.modal-dialog{position:relative;width:auto;margin:10px}.modal-content{position:relative;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,.2);border-radius:6px;-webkit-box-shadow:0 3px 9px rgba(0,0,0,.5);box-shadow:0 3px 9px rgba(0,0,0,.5);background-clip:padding-box;outline:0}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0;filter:alpha(opacity=0)}.modal-backdrop.in{opacity:.5;filter:alpha(opacity=50)}.modal-header{padding:15px;border-bottom:1px solid #e5e5e5;min-height:16.42857143px}.modal-header .close{margin-top:-2px}.modal-title{margin:0;line-height:1.42857143}.modal-body{position:relative;padding:20px}.modal-footer{margin-top:15px;padding:19px 20px 20px;text-align:right;border-top:1px solid #e5e5e5}.modal-footer .btn+.btn{margin-left:5px;margin-bottom:0}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.modal-footer .btn-block+.btn-block{margin-left:0}@media (min-width:768px){.modal-dialog{width:600px;margin:30px auto}.modal-content{-webkit-box-shadow:0 5px 15px rgba(0,0,0,.5);box-shadow:0 5px 15px rgba(0,0,0,.5)}.modal-sm{width:300px}}@media (min-width:992px){.modal-lg{width:900px}}.tooltip{position:absolute;z-index:1030;display:block;visibility:visible;font-size:12px;line-height:1.4;opacity:0;filter:alpha(opacity=0)}.tooltip.in{opacity:.9;filter:alpha(opacity=90)}.tooltip.top{margin-top:-3px;padding:5px 0}.tooltip.right{margin-left:3px;padding:0 5px}.tooltip.bottom{margin-top:3px;padding:5px 0}.tooltip.left{margin-left:-3px;padding:0 5px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-width:5px 5px 0;border-top-color:#000}.tooltip.top-left .tooltip-arrow{bottom:0;left:5px;border-width:5px 5px 0;border-top-color:#000}.tooltip.top-right .tooltip-arrow{bottom:0;right:5px;border-width:5px 5px 0;border-top-color:#000}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-width:5px 5px 5px 0;border-right-color:#000}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-width:5px 0 5px 5px;border-left-color:#000}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-width:0 5px 5px;border-bottom-color:#000}.tooltip.bottom-left .tooltip-arrow{top:0;left:5px;border-width:0 5px 5px;border-bottom-color:#000}.tooltip.bottom-right .tooltip-arrow{top:0;right:5px;border-width:0 5px 5px;border-bottom-color:#000}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;max-width:276px;padding:1px;text-align:left;background-color:#fff;background-clip:padding-box;border:1px solid #ccc;border:1px solid rgba(0,0,0,.2);border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,.2);box-shadow:0 5px 10px rgba(0,0,0,.2);white-space:normal}.popover.top{margin-top:-10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-left:-10px}.popover-title{margin:0;padding:8px 14px;font-size:14px;font-weight:400;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover>.arrow,.popover>.arrow:after{position:absolute;display:block;width:0;height:0;border-color:transparent;border-style:solid}.popover>.arrow{border-width:11px}.popover>.arrow:after{border-width:10px;content:""}.popover.top>.arrow{left:50%;margin-left:-11px;border-bottom-width:0;border-top-color:#999;border-top-color:rgba(0,0,0,.25);bottom:-11px}.popover.top>.arrow:after{content:" ";bottom:1px;margin-left:-10px;border-bottom-width:0;border-top-color:#fff}.popover.right>.arrow{top:50%;left:-11px;margin-top:-11px;border-left-width:0;border-right-color:#999;border-right-color:rgba(0,0,0,.25)}.popover.right>.arrow:after{content:" ";left:1px;bottom:-10px;border-left-width:0;border-right-color:#fff}.popover.bottom>.arrow{left:50%;margin-left:-11px;border-top-width:0;border-bottom-color:#999;border-bottom-color:rgba(0,0,0,.25);top:-11px}.popover.bottom>.arrow:after{content:" ";top:1px;margin-left:-10px;border-top-width:0;border-bottom-color:#fff}.popover.left>.arrow{top:50%;right:-11px;margin-top:-11px;border-right-width:0;border-left-color:#999;border-left-color:rgba(0,0,0,.25)}.popover.left>.arrow:after{content:" ";right:1px;border-right-width:0;border-left-color:#fff;bottom:-10px}.carousel{position:relative}.carousel-inner{position:relative;overflow:hidden;width:100%}.carousel-inner>.item{display:none;position:relative;-webkit-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel-inner>.item>img,.carousel-inner>.item>a>img{line-height:1}.carousel-inner>.active,.carousel-inner>.next,.carousel-inner>.prev{display:block}.carousel-inner>.active{left:0}.carousel-inner>.next,.carousel-inner>.prev{position:absolute;top:0;width:100%}.carousel-inner>.next{left:100%}.carousel-inner>.prev{left:-100%}.carousel-inner>.next.left,.carousel-inner>.prev.right{left:0}.carousel-inner>.active.left{left:-100%}.carousel-inner>.active.right{left:100%}.carousel-control{position:absolute;top:0;left:0;bottom:0;width:15%;opacity:.5;filter:alpha(opacity=50);font-size:20px;color:#fff;text-align:center;text-shadow:0 1px 2px rgba(0,0,0,.6)}.carousel-control.left{background-image:-webkit-linear-gradient(left,color-stop(rgba(0,0,0,.5) 0),color-stop(rgba(0,0,0,.0001) 100%));background-image:linear-gradient(to right,rgba(0,0,0,.5) 0,rgba(0,0,0,.0001) 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#80000000', endColorstr='#00000000', GradientType=1)}.carousel-control.right{left:auto;right:0;background-image:-webkit-linear-gradient(left,color-stop(rgba(0,0,0,.0001) 0),color-stop(rgba(0,0,0,.5) 100%));background-image:linear-gradient(to right,rgba(0,0,0,.0001) 0,rgba(0,0,0,.5) 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#00000000', endColorstr='#80000000', GradientType=1)}.carousel-control:hover,.carousel-control:focus{outline:0;color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-control .icon-prev,.carousel-control .icon-next,.carousel-control .glyphicon-chevron-left,.carousel-control .glyphicon-chevron-right{position:absolute;top:50%;z-index:5;display:inline-block}.carousel-control .icon-prev,.carousel-control .glyphicon-chevron-left{left:50%}.carousel-control .icon-next,.carousel-control .glyphicon-chevron-right{right:50%}.carousel-control .icon-prev,.carousel-control .icon-next{width:20px;height:20px;margin-top:-10px;margin-left:-10px;font-family:serif}.carousel-control .icon-prev:before{content:'\2039'}.carousel-control .icon-next:before{content:'\203a'}.carousel-indicators{position:absolute;bottom:10px;left:50%;z-index:15;width:60%;margin-left:-30%;padding-left:0;list-style:none;text-align:center}.carousel-indicators li{display:inline-block;width:10px;height:10px;margin:1px;text-indent:-999px;border:1px solid #fff;border-radius:10px;cursor:pointer;background-color:#000 \9;background-color:rgba(0,0,0,0)}.carousel-indicators .active{margin:0;width:12px;height:12px;background-color:#fff}.carousel-caption{position:absolute;left:15%;right:15%;bottom:20px;z-index:10;padding-top:20px;padding-bottom:20px;color:#fff;text-align:center;text-shadow:0 1px 2px rgba(0,0,0,.6)}.carousel-caption .btn{text-shadow:none}@media screen and (min-width:768px){.carousel-control .glyphicon-chevron-left,.carousel-control .glyphicon-chevron-right,.carousel-control .icon-prev,.carousel-control .icon-next{width:30px;height:30px;margin-top:-15px;margin-left:-15px;font-size:30px}.carousel-caption{left:20%;right:20%;padding-bottom:30px}.carousel-indicators{bottom:20px}}.clearfix:before,.clearfix:after,.container:before,.container:after,.container-fluid:before,.container-fluid:after,.row:before,.row:after,.form-horizontal .form-group:before,.form-horizontal .form-group:after,.btn-toolbar:before,.btn-toolbar:after,.btn-group-vertical>.btn-group:before,.btn-group-vertical>.btn-group:after,.nav:before,.nav:after,.navbar:before,.navbar:after,.navbar-header:before,.navbar-header:after,.navbar-collapse:before,.navbar-collapse:after,.pager:before,.pager:after,.panel-body:before,.panel-body:after,.modal-footer:before,.modal-footer:after{content:" ";display:table}.clearfix:after,.container:after,.container-fluid:after,.row:after,.form-horizontal .form-group:after,.btn-toolbar:after,.btn-group-vertical>.btn-group:after,.nav:after,.navbar:after,.navbar-header:after,.navbar-collapse:after,.pager:after,.panel-body:after,.modal-footer:after{clear:both}.center-block{display:block;margin-left:auto;margin-right:auto}.pull-right{float:right!important}.pull-left{float:left!important}.hide{display:none!important}.show{display:block!important}.invisible{visibility:hidden}.text-hide{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.hidden{display:none!important;visibility:hidden!important}.affix{position:fixed}@-ms-viewport{width:device-width}.visible-xs,.visible-sm,.visible-md,.visible-lg{display:none!important}@media (max-width:767px){.visible-xs{display:block!important}table.visible-xs{display:table}tr.visible-xs{display:table-row!important}th.visible-xs,td.visible-xs{display:table-cell!important}}@media (min-width:768px) and (max-width:991px){.visible-sm{display:block!important}table.visible-sm{display:table}tr.visible-sm{display:table-row!important}th.visible-sm,td.visible-sm{display:table-cell!important}}@media (min-width:992px) and (max-width:1199px){.visible-md{display:block!important}table.visible-md{display:table}tr.visible-md{display:table-row!important}th.visible-md,td.visible-md{display:table-cell!important}}@media (min-width:1200px){.visible-lg{display:block!important}table.visible-lg{display:table}tr.visible-lg{display:table-row!important}th.visible-lg,td.visible-lg{display:table-cell!important}}@media (max-width:767px){.hidden-xs{display:none!important}}@media (min-width:768px) and (max-width:991px){.hidden-sm{display:none!important}}@media (min-width:992px) and (max-width:1199px){.hidden-md{display:none!important}}@media (min-width:1200px){.hidden-lg{display:none!important}}.visible-print{display:none!important}@media print{.visible-print{display:block!important}table.visible-print{display:table}tr.visible-print{display:table-row!important}th.visible-print,td.visible-print{display:table-cell!important}}@media print{.hidden-print{display:none!important}} \ 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 0000000000..449d6ac551 --- /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:normal;font-weight:normal;line-height:1;-webkit-font-smoothing:antialiased;-moz-osx-font-smoothing:grayscale}.fa-lg{font-size:1.3333333333333333em;line-height:.75em;vertical-align:-15%}.fa-2x{font-size:2em}.fa-3x{font-size:3em}.fa-4x{font-size:4em}.fa-5x{font-size:5em}.fa-fw{width:1.2857142857142858em;text-align:center}.fa-ul{padding-left:0;margin-left:2.142857142857143em;list-style-type:none}.fa-ul>li{position:relative}.fa-li{position:absolute;left:-2.142857142857143em;width:2.142857142857143em;top:.14285714285714285em;text-align:center}.fa-li.fa-lg{left:-1.8571428571428572em}.fa-border{padding:.2em .25em .15em;border:solid .08em #eee;border-radius:.1em}.pull-right{float:right}.pull-left{float:left}.fa.pull-left{margin-right:.3em}.fa.pull-right{margin-left:.3em}.fa-spin{-webkit-animation:spin 2s infinite linear;-moz-animation:spin 2s infinite linear;-o-animation:spin 2s infinite linear;animation:spin 2s infinite linear}@-moz-keyframes spin{0%{-moz-transform:rotate(0deg)}100%{-moz-transform:rotate(359deg)}}@-webkit-keyframes spin{0%{-webkit-transform:rotate(0deg)}100%{-webkit-transform:rotate(359deg)}}@-o-keyframes spin{0%{-o-transform:rotate(0deg)}100%{-o-transform:rotate(359deg)}}@-ms-keyframes spin{0%{-ms-transform:rotate(0deg)}100%{-ms-transform:rotate(359deg)}}@keyframes spin{0%{transform:rotate(0deg)}100%{transform:rotate(359deg)}}.fa-rotate-90{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=1);-webkit-transform:rotate(90deg);-moz-transform:rotate(90deg);-ms-transform:rotate(90deg);-o-transform:rotate(90deg);transform:rotate(90deg)}.fa-rotate-180{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=2);-webkit-transform:rotate(180deg);-moz-transform:rotate(180deg);-ms-transform:rotate(180deg);-o-transform:rotate(180deg);transform:rotate(180deg)}.fa-rotate-270{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=3);-webkit-transform:rotate(270deg);-moz-transform:rotate(270deg);-ms-transform:rotate(270deg);-o-transform:rotate(270deg);transform:rotate(270deg)}.fa-flip-horizontal{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=0,mirror=1);-webkit-transform:scale(-1,1);-moz-transform:scale(-1,1);-ms-transform:scale(-1,1);-o-transform:scale(-1,1);transform:scale(-1,1)}.fa-flip-vertical{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=2,mirror=1);-webkit-transform:scale(1,-1);-moz-transform:scale(1,-1);-ms-transform:scale(1,-1);-o-transform:scale(1,-1);transform:scale(1,-1)}.fa-stack{position:relative;display:inline-block;width:2em;height:2em;line-height:2em;vertical-align:middle}.fa-stack-1x,.fa-stack-2x{position:absolute;left:0;width:100%;text-align:center}.fa-stack-1x{line-height:inherit}.fa-stack-2x{font-size:2em}.fa-inverse{color:#fff}.fa-glass:before{content:"\f000"}.fa-music:before{content:"\f001"}.fa-search:before{content:"\f002"}.fa-envelope-o:before{content:"\f003"}.fa-heart:before{content:"\f004"}.fa-star:before{content:"\f005"}.fa-star-o:before{content:"\f006"}.fa-user:before{content:"\f007"}.fa-film:before{content:"\f008"}.fa-th-large:before{content:"\f009"}.fa-th:before{content:"\f00a"}.fa-th-list:before{content:"\f00b"}.fa-check:before{content:"\f00c"}.fa-times:before{content:"\f00d"}.fa-search-plus:before{content:"\f00e"}.fa-search-minus:before{content:"\f010"}.fa-power-off:before{content:"\f011"}.fa-signal:before{content:"\f012"}.fa-gear:before,.fa-cog:before{content:"\f013"}.fa-trash-o:before{content:"\f014"}.fa-home:before{content:"\f015"}.fa-file-o:before{content:"\f016"}.fa-clock-o:before{content:"\f017"}.fa-road:before{content:"\f018"}.fa-download:before{content:"\f019"}.fa-arrow-circle-o-down:before{content:"\f01a"}.fa-arrow-circle-o-up:before{content:"\f01b"}.fa-inbox:before{content:"\f01c"}.fa-play-circle-o:before{content:"\f01d"}.fa-rotate-right:before,.fa-repeat:before{content:"\f01e"}.fa-refresh:before{content:"\f021"}.fa-list-alt:before{content:"\f022"}.fa-lock:before{content:"\f023"}.fa-flag:before{content:"\f024"}.fa-headphones:before{content:"\f025"}.fa-volume-off:before{content:"\f026"}.fa-volume-down:before{content:"\f027"}.fa-volume-up:before{content:"\f028"}.fa-qrcode:before{content:"\f029"}.fa-barcode:before{content:"\f02a"}.fa-tag:before{content:"\f02b"}.fa-tags:before{content:"\f02c"}.fa-book:before{content:"\f02d"}.fa-bookmark:before{content:"\f02e"}.fa-print:before{content:"\f02f"}.fa-camera:before{content:"\f030"}.fa-font:before{content:"\f031"}.fa-bold:before{content:"\f032"}.fa-italic:before{content:"\f033"}.fa-text-height:before{content:"\f034"}.fa-text-width:before{content:"\f035"}.fa-align-left:before{content:"\f036"}.fa-align-center:before{content:"\f037"}.fa-align-right:before{content:"\f038"}.fa-align-justify:before{content:"\f039"}.fa-list:before{content:"\f03a"}.fa-dedent:before,.fa-outdent:before{content:"\f03b"}.fa-indent:before{content:"\f03c"}.fa-video-camera:before{content:"\f03d"}.fa-picture-o:before{content:"\f03e"}.fa-pencil:before{content:"\f040"}.fa-map-marker:before{content:"\f041"}.fa-adjust:before{content:"\f042"}.fa-tint:before{content:"\f043"}.fa-edit:before,.fa-pencil-square-o:before{content:"\f044"}.fa-share-square-o:before{content:"\f045"}.fa-check-square-o:before{content:"\f046"}.fa-arrows:before{content:"\f047"}.fa-step-backward:before{content:"\f048"}.fa-fast-backward:before{content:"\f049"}.fa-backward:before{content:"\f04a"}.fa-play:before{content:"\f04b"}.fa-pause:before{content:"\f04c"}.fa-stop:before{content:"\f04d"}.fa-forward:before{content:"\f04e"}.fa-fast-forward:before{content:"\f050"}.fa-step-forward:before{content:"\f051"}.fa-eject:before{content:"\f052"}.fa-chevron-left:before{content:"\f053"}.fa-chevron-right:before{content:"\f054"}.fa-plus-circle:before{content:"\f055"}.fa-minus-circle:before{content:"\f056"}.fa-times-circle:before{content:"\f057"}.fa-check-circle:before{content:"\f058"}.fa-question-circle:before{content:"\f059"}.fa-info-circle:before{content:"\f05a"}.fa-crosshairs:before{content:"\f05b"}.fa-times-circle-o:before{content:"\f05c"}.fa-check-circle-o:before{content:"\f05d"}.fa-ban:before{content:"\f05e"}.fa-arrow-left:before{content:"\f060"}.fa-arrow-right:before{content:"\f061"}.fa-arrow-up:before{content:"\f062"}.fa-arrow-down:before{content:"\f063"}.fa-mail-forward:before,.fa-share:before{content:"\f064"}.fa-expand:before{content:"\f065"}.fa-compress:before{content:"\f066"}.fa-plus:before{content:"\f067"}.fa-minus:before{content:"\f068"}.fa-asterisk:before{content:"\f069"}.fa-exclamation-circle:before{content:"\f06a"}.fa-gift:before{content:"\f06b"}.fa-leaf:before{content:"\f06c"}.fa-fire:before{content:"\f06d"}.fa-eye:before{content:"\f06e"}.fa-eye-slash:before{content:"\f070"}.fa-warning:before,.fa-exclamation-triangle:before{content:"\f071"}.fa-plane:before{content:"\f072"}.fa-calendar:before{content:"\f073"}.fa-random:before{content:"\f074"}.fa-comment:before{content:"\f075"}.fa-magnet:before{content:"\f076"}.fa-chevron-up:before{content:"\f077"}.fa-chevron-down:before{content:"\f078"}.fa-retweet:before{content:"\f079"}.fa-shopping-cart:before{content:"\f07a"}.fa-folder:before{content:"\f07b"}.fa-folder-open:before{content:"\f07c"}.fa-arrows-v:before{content:"\f07d"}.fa-arrows-h:before{content:"\f07e"}.fa-bar-chart-o:before{content:"\f080"}.fa-twitter-square:before{content:"\f081"}.fa-facebook-square:before{content:"\f082"}.fa-camera-retro:before{content:"\f083"}.fa-key:before{content:"\f084"}.fa-gears:before,.fa-cogs:before{content:"\f085"}.fa-comments:before{content:"\f086"}.fa-thumbs-o-up:before{content:"\f087"}.fa-thumbs-o-down:before{content:"\f088"}.fa-star-half:before{content:"\f089"}.fa-heart-o:before{content:"\f08a"}.fa-sign-out:before{content:"\f08b"}.fa-linkedin-square:before{content:"\f08c"}.fa-thumb-tack:before{content:"\f08d"}.fa-external-link:before{content:"\f08e"}.fa-sign-in:before{content:"\f090"}.fa-trophy:before{content:"\f091"}.fa-github-square:before{content:"\f092"}.fa-upload:before{content:"\f093"}.fa-lemon-o:before{content:"\f094"}.fa-phone:before{content:"\f095"}.fa-square-o:before{content:"\f096"}.fa-bookmark-o:before{content:"\f097"}.fa-phone-square:before{content:"\f098"}.fa-twitter:before{content:"\f099"}.fa-facebook:before{content:"\f09a"}.fa-github:before{content:"\f09b"}.fa-unlock:before{content:"\f09c"}.fa-credit-card:before{content:"\f09d"}.fa-rss:before{content:"\f09e"}.fa-hdd-o:before{content:"\f0a0"}.fa-bullhorn:before{content:"\f0a1"}.fa-bell:before{content:"\f0f3"}.fa-certificate:before{content:"\f0a3"}.fa-hand-o-right:before{content:"\f0a4"}.fa-hand-o-left:before{content:"\f0a5"}.fa-hand-o-up:before{content:"\f0a6"}.fa-hand-o-down:before{content:"\f0a7"}.fa-arrow-circle-left:before{content:"\f0a8"}.fa-arrow-circle-right:before{content:"\f0a9"}.fa-arrow-circle-up:before{content:"\f0aa"}.fa-arrow-circle-down:before{content:"\f0ab"}.fa-globe:before{content:"\f0ac"}.fa-wrench:before{content:"\f0ad"}.fa-tasks:before{content:"\f0ae"}.fa-filter:before{content:"\f0b0"}.fa-briefcase:before{content:"\f0b1"}.fa-arrows-alt:before{content:"\f0b2"}.fa-group:before,.fa-users:before{content:"\f0c0"}.fa-chain:before,.fa-link:before{content:"\f0c1"}.fa-cloud:before{content:"\f0c2"}.fa-flask:before{content:"\f0c3"}.fa-cut:before,.fa-scissors:before{content:"\f0c4"}.fa-copy:before,.fa-files-o:before{content:"\f0c5"}.fa-paperclip:before{content:"\f0c6"}.fa-save:before,.fa-floppy-o:before{content:"\f0c7"}.fa-square:before{content:"\f0c8"}.fa-bars:before{content:"\f0c9"}.fa-list-ul:before{content:"\f0ca"}.fa-list-ol:before{content:"\f0cb"}.fa-strikethrough:before{content:"\f0cc"}.fa-underline:before{content:"\f0cd"}.fa-table:before{content:"\f0ce"}.fa-magic:before{content:"\f0d0"}.fa-truck:before{content:"\f0d1"}.fa-pinterest:before{content:"\f0d2"}.fa-pinterest-square:before{content:"\f0d3"}.fa-google-plus-square:before{content:"\f0d4"}.fa-google-plus:before{content:"\f0d5"}.fa-money:before{content:"\f0d6"}.fa-caret-down:before{content:"\f0d7"}.fa-caret-up:before{content:"\f0d8"}.fa-caret-left:before{content:"\f0d9"}.fa-caret-right:before{content:"\f0da"}.fa-columns:before{content:"\f0db"}.fa-unsorted:before,.fa-sort:before{content:"\f0dc"}.fa-sort-down:before,.fa-sort-asc:before{content:"\f0dd"}.fa-sort-up:before,.fa-sort-desc:before{content:"\f0de"}.fa-envelope:before{content:"\f0e0"}.fa-linkedin:before{content:"\f0e1"}.fa-rotate-left:before,.fa-undo:before{content:"\f0e2"}.fa-legal:before,.fa-gavel:before{content:"\f0e3"}.fa-dashboard:before,.fa-tachometer:before{content:"\f0e4"}.fa-comment-o:before{content:"\f0e5"}.fa-comments-o:before{content:"\f0e6"}.fa-flash:before,.fa-bolt:before{content:"\f0e7"}.fa-sitemap:before{content:"\f0e8"}.fa-umbrella:before{content:"\f0e9"}.fa-paste:before,.fa-clipboard:before{content:"\f0ea"}.fa-lightbulb-o:before{content:"\f0eb"}.fa-exchange:before{content:"\f0ec"}.fa-cloud-download:before{content:"\f0ed"}.fa-cloud-upload:before{content:"\f0ee"}.fa-user-md:before{content:"\f0f0"}.fa-stethoscope:before{content:"\f0f1"}.fa-suitcase:before{content:"\f0f2"}.fa-bell-o:before{content:"\f0a2"}.fa-coffee:before{content:"\f0f4"}.fa-cutlery:before{content:"\f0f5"}.fa-file-text-o:before{content:"\f0f6"}.fa-building-o:before{content:"\f0f7"}.fa-hospital-o:before{content:"\f0f8"}.fa-ambulance:before{content:"\f0f9"}.fa-medkit:before{content:"\f0fa"}.fa-fighter-jet:before{content:"\f0fb"}.fa-beer:before{content:"\f0fc"}.fa-h-square:before{content:"\f0fd"}.fa-plus-square:before{content:"\f0fe"}.fa-angle-double-left:before{content:"\f100"}.fa-angle-double-right:before{content:"\f101"}.fa-angle-double-up:before{content:"\f102"}.fa-angle-double-down:before{content:"\f103"}.fa-angle-left:before{content:"\f104"}.fa-angle-right:before{content:"\f105"}.fa-angle-up:before{content:"\f106"}.fa-angle-down:before{content:"\f107"}.fa-desktop:before{content:"\f108"}.fa-laptop:before{content:"\f109"}.fa-tablet:before{content:"\f10a"}.fa-mobile-phone:before,.fa-mobile:before{content:"\f10b"}.fa-circle-o:before{content:"\f10c"}.fa-quote-left:before{content:"\f10d"}.fa-quote-right:before{content:"\f10e"}.fa-spinner:before{content:"\f110"}.fa-circle:before{content:"\f111"}.fa-mail-reply:before,.fa-reply:before{content:"\f112"}.fa-github-alt:before{content:"\f113"}.fa-folder-o:before{content:"\f114"}.fa-folder-open-o:before{content:"\f115"}.fa-smile-o:before{content:"\f118"}.fa-frown-o:before{content:"\f119"}.fa-meh-o:before{content:"\f11a"}.fa-gamepad:before{content:"\f11b"}.fa-keyboard-o:before{content:"\f11c"}.fa-flag-o:before{content:"\f11d"}.fa-flag-checkered:before{content:"\f11e"}.fa-terminal:before{content:"\f120"}.fa-code:before{content:"\f121"}.fa-reply-all:before{content:"\f122"}.fa-mail-reply-all:before{content:"\f122"}.fa-star-half-empty:before,.fa-star-half-full:before,.fa-star-half-o:before{content:"\f123"}.fa-location-arrow:before{content:"\f124"}.fa-crop:before{content:"\f125"}.fa-code-fork:before{content:"\f126"}.fa-unlink:before,.fa-chain-broken:before{content:"\f127"}.fa-question:before{content:"\f128"}.fa-info:before{content:"\f129"}.fa-exclamation:before{content:"\f12a"}.fa-superscript:before{content:"\f12b"}.fa-subscript:before{content:"\f12c"}.fa-eraser:before{content:"\f12d"}.fa-puzzle-piece:before{content:"\f12e"}.fa-microphone:before{content:"\f130"}.fa-microphone-slash:before{content:"\f131"}.fa-shield:before{content:"\f132"}.fa-calendar-o:before{content:"\f133"}.fa-fire-extinguisher:before{content:"\f134"}.fa-rocket:before{content:"\f135"}.fa-maxcdn:before{content:"\f136"}.fa-chevron-circle-left:before{content:"\f137"}.fa-chevron-circle-right:before{content:"\f138"}.fa-chevron-circle-up:before{content:"\f139"}.fa-chevron-circle-down:before{content:"\f13a"}.fa-html5:before{content:"\f13b"}.fa-css3:before{content:"\f13c"}.fa-anchor:before{content:"\f13d"}.fa-unlock-alt:before{content:"\f13e"}.fa-bullseye:before{content:"\f140"}.fa-ellipsis-h:before{content:"\f141"}.fa-ellipsis-v:before{content:"\f142"}.fa-rss-square:before{content:"\f143"}.fa-play-circle:before{content:"\f144"}.fa-ticket:before{content:"\f145"}.fa-minus-square:before{content:"\f146"}.fa-minus-square-o:before{content:"\f147"}.fa-level-up:before{content:"\f148"}.fa-level-down:before{content:"\f149"}.fa-check-square:before{content:"\f14a"}.fa-pencil-square:before{content:"\f14b"}.fa-external-link-square:before{content:"\f14c"}.fa-share-square:before{content:"\f14d"}.fa-compass:before{content:"\f14e"}.fa-toggle-down:before,.fa-caret-square-o-down:before{content:"\f150"}.fa-toggle-up:before,.fa-caret-square-o-up:before{content:"\f151"}.fa-toggle-right:before,.fa-caret-square-o-right:before{content:"\f152"}.fa-euro:before,.fa-eur:before{content:"\f153"}.fa-gbp:before{content:"\f154"}.fa-dollar:before,.fa-usd:before{content:"\f155"}.fa-rupee:before,.fa-inr:before{content:"\f156"}.fa-cny:before,.fa-rmb:before,.fa-yen:before,.fa-jpy:before{content:"\f157"}.fa-ruble:before,.fa-rouble:before,.fa-rub:before{content:"\f158"}.fa-won:before,.fa-krw:before{content:"\f159"}.fa-bitcoin:before,.fa-btc:before{content:"\f15a"}.fa-file:before{content:"\f15b"}.fa-file-text:before{content:"\f15c"}.fa-sort-alpha-asc:before{content:"\f15d"}.fa-sort-alpha-desc:before{content:"\f15e"}.fa-sort-amount-asc:before{content:"\f160"}.fa-sort-amount-desc:before{content:"\f161"}.fa-sort-numeric-asc:before{content:"\f162"}.fa-sort-numeric-desc:before{content:"\f163"}.fa-thumbs-up:before{content:"\f164"}.fa-thumbs-down:before{content:"\f165"}.fa-youtube-square:before{content:"\f166"}.fa-youtube:before{content:"\f167"}.fa-xing:before{content:"\f168"}.fa-xing-square:before{content:"\f169"}.fa-youtube-play:before{content:"\f16a"}.fa-dropbox:before{content:"\f16b"}.fa-stack-overflow:before{content:"\f16c"}.fa-instagram:before{content:"\f16d"}.fa-flickr:before{content:"\f16e"}.fa-adn:before{content:"\f170"}.fa-bitbucket:before{content:"\f171"}.fa-bitbucket-square:before{content:"\f172"}.fa-tumblr:before{content:"\f173"}.fa-tumblr-square:before{content:"\f174"}.fa-long-arrow-down:before{content:"\f175"}.fa-long-arrow-up:before{content:"\f176"}.fa-long-arrow-left:before{content:"\f177"}.fa-long-arrow-right:before{content:"\f178"}.fa-apple:before{content:"\f179"}.fa-windows:before{content:"\f17a"}.fa-android:before{content:"\f17b"}.fa-linux:before{content:"\f17c"}.fa-dribbble:before{content:"\f17d"}.fa-skype:before{content:"\f17e"}.fa-foursquare:before{content:"\f180"}.fa-trello:before{content:"\f181"}.fa-female:before{content:"\f182"}.fa-male:before{content:"\f183"}.fa-gittip:before{content:"\f184"}.fa-sun-o:before{content:"\f185"}.fa-moon-o:before{content:"\f186"}.fa-archive:before{content:"\f187"}.fa-bug:before{content:"\f188"}.fa-vk:before{content:"\f189"}.fa-weibo:before{content:"\f18a"}.fa-renren:before{content:"\f18b"}.fa-pagelines:before{content:"\f18c"}.fa-stack-exchange:before{content:"\f18d"}.fa-arrow-circle-o-right:before{content:"\f18e"}.fa-arrow-circle-o-left:before{content:"\f190"}.fa-toggle-left:before,.fa-caret-square-o-left:before{content:"\f191"}.fa-dot-circle-o:before{content:"\f192"}.fa-wheelchair:before{content:"\f193"}.fa-vimeo-square:before{content:"\f194"}.fa-turkish-lira:before,.fa-try:before{content:"\f195"}.fa-plus-square-o:before{content:"\f196"} \ 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 0000000000..d36ebed2f4 --- /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 0000000000000000000000000000000000000000..8b0f54e47e1d356dcf1496942a50e228e0f1ee14 GIT binary patch literal 62856 zcmcfp2Y3_5)&LBzEbU6(wGF`%u_do$I-wUs=poc3^xzP>t859|l91%ydy%{4ZewH9 zLNU#OK%5)jlp7M#adH#VlN(Y~MSVYG)7F`Dsts8mQIv>+ztD)dFw+9OVG%`1 zdML`ns?&x=Qnp|IfM+dm&(}ePcdqmf37+Ghm#p%f+FVKQ2*chjkzF#ZB~9w-bef!xGBr6D7h{6UGOP@t%*!8rhr zqTX&D_txFJckW8F88SgJDOYWQiq1}9HpST zU`<34PZ)C!_3}_&M2)6kC53tq%16Wv<;B!kk^fL$a$g&o8ZTNrRL|U3FQqy}Aw%^t z%FjbIl=r0M9>Z`rYKq77t>{++@-k0@oM~*1+}p2(7`Q4V*n=HYq=vsI?g5v}-nP z3|{}}ibb1(*R0;YdDD}@+q7nj-e?F6nlWp}oWMD=X3yOms||yGW^I(#9B4HL0`>*2 zG{Pq6qjlCmi#Eba+D94TAv}p9V_D5%k=nR0b4*~E)oRv<#|upiMk~z0GGmR=Yz-V5 ze^pq5HgIj2Au?HKwVD>qoJsnJx#u=RZ=|+Tk5lVmJ2z1#N=q3aw}vu8YK7c-N>4=y zwHEjdq-Iky;2wVdD3u7c7HAy@>636rQ}I+R6-Jq%%_eFi6$}s_rB+ajpcD*stEugP zo136*FtrWZo1wQ}7%h+r0@$R$MYWppE&yKBVk^ODoieQIXI-PMCWPv3^jr9p7*cDDu9q6%xx{?3;;b@n3omixrmwx*YNmZf9p3xm@i;8 zp?TpJjUB@J0D^@;Vq@WEgcj}}s2gf=U*-SLs=qz||El20$!O-RlsfnS_J9)6lK^rf z@F|+|fem;DctSVzuQ6lCs>g=*`}C{(m-TP#-`gM6ukSbXXY`l%AL#GuKiB_u|L6U` z^xwJVb4z_|(yht2X53nKYvZlGw+y#3Zk69U@CS95u-8E9*x%q${UiIw^e^w<+#lK> z-M_Ej)SuN~+27uOroXrU-Tp88`)^UVM&1epcn{s0b!+*p&9_2tnQmp>swD94ennAt zcir7`_tDR9d~W}I%Sf-0+(^%nvXRn}u#+RjBRxinMp7g0j<_@8_K4p{{5Im&i2f13 zj`+pr(-A+9_-Vw=5kHRjVZ`?%z8i6aJ1^|@`u}w?=l`!y{JYkcahKF7zYy(4XAHaLAh7>kswf;WDJ8 zodnW*&mk}LA4ATyzs;HS z&jMIk)X1SUY8WQ8mk8qz!5gX{ac?|#KNXah-`{R{t;jx;+arrw4mTM?C=b`)g9B|K zKbe$=Z!xqbc>xxr!#G3cIJ_43-sk>0XiMsaXE3e+56S@N-W&nebhy1GS=0t{!`!CB zeXl$`20SDCO)=z#yl@A)%foXM<_FJ&aY(!S?qN9ajLc&>wDpF%>BD`=97%ujZX|^{ zkUJb;(Bvllh3Ak$Tkm1o9O@S+z@h#=rtsbrEayd0}DguL&kx00m+ja=Bpt$)C)Jj(+GE#@N5{qN_YooPx`~Xe7HP3 z{%{$_+eqqQIN>I3Ngv^P)=&zdhx-v8M)G7X!|w&{r;s|*7v>g7Gy(!cXqP3lRov@8 zR1fWh=MwT9Zqok0{>Y@@?`{gwSN{7?L`gvE7m2*?lX6LUm1893w2Pdz9?n{^!(W2e zdWpaFl9b@u0BLprBcj#q)KgjW@7iqlGG5Yvz*k2E1b+8G7f(?i1&vA9XxDLyUk5nmBs6~80?xA;He-^DJ8RN^C1NybWMO6ExxOV&s>OP-SKlxQUu zNxCEtRJdwMgQQb(MDmQ}tmIiqujCEMHOY0!HkBMipnS7>{u``WKCv$?i#JtM9$^4u7g87d5nYqQ>kup*r>4Q>U zI$1hRI!8KRx>mYFs*@&5bEW0dI%&J~sPvTdy!1usRp|%PFQwl}f0q6xb;-PBD%k|t zY}tI-V%aj;YS{+aQ?dwIjLaxYk`>BoWsR~9*)iEk*+tn)va7OpWS_{smHjSrdP+V0 zJk_4#J?D9@_1xwe?HTK7@=Wl|@+|Uf_B`o%#`BWri=J_T=4`v|*&UBhl-L)Zv5p0%+J>@(~s_AL7X`wDx7eUJT&{SSMK z9pETV%t<)~r{X4Z^SBk<7A}m7;^H_fm&|2x`CJ88%QbUt++pq*cal5LUErSMUf^El zUgJLCKIVSme)FQdBwi!E`Us0Q z%p9T98WOazMw1pS4`!>y8fGSUh&Ik-O^&x{%~AT;IIAusHq0EYwdzPtZ?PI<%-T3( zf;Poyj0@2lgv1zcHAY2Q^wEZ}*a%}ZXpR=04ir-WpbZI&wOaLYTC*`MGSZl6h=r8Y z4d>%cq(*NDHzt{4!;(WH^yY|Ityyc*hFL*fHES(8GA!v5YmA7AiVce8e_;!6kC&7Z?Hyy8O0n%G}drq zY^2^A7ORi2YLl!XIxW$Sg>0fe(yD_8(T0#%Z4_w&Inczd&{N0@YP37MFWzF+MkX06M(8q>71~9GMQF*2ge2%AwMG*R7f)W-5CO{_W(pxQ1Gtd{5P-01VNw=dm{|+^ z6%j+0-eT37Lc+r$ViLp5kx^l=IKzeEl&qvF4E7NA%LH2ey@o@10m4vTyAQN~fSq7A zx?gWNFHF`H8*d3AI~%7r4CUPWFH{<1gk*m_30u(tfF`iWB#nqQTC}hv2E8F#m?SuDFTQn3UEkkc8@TWC!-F{GC^ww z>q*$~q;*EKK82V{VgW}(B4CfL)4q56 z4)D)xH0hF~^)O1fFcUYy3iJruY7hufKutIFVd8R^gr`Ecp*I_TDL24)U$r5ORbRg-pCjNXR?8@hRjlg!)^B z(D!dOu%iM74)q`)qGOHW+C($Zqs|&;iLn3^gGC89>$Oo4U_&EF=f-R>g=zQ41JxU% z^ai~(IaX`22o=$0BPn|0z*CK8 zK%DqkW2^;?Z85-a0Z6ni9$1JOKmq#-j|FR7G;j-Zd_)ZF6-)}K?p{V%Lg*B4TBUeba0p4h(`{lkhnUa;!S@mlEwb3uRAAna%X|R34lqnNUbFX_%$pF{0bXxjWdRmGt^CFZcG*MWq&*% zpD-JDPJjsSWiSA$4WFQ~!(L z(g@%$q;&`!M=`(;0H;FcJiPEeUTy)bGXu%#O;$^MxH}UvXTe-kd`b#g8@(3xP*30x znc%M+5eqCjy*4&-n6xnX2oC%!5s^Uj?t@SuO@S=#uW(bx z{WX6b2|^FDjXG;w?7RqzWiB8Wa4|QJBTGftngtFZz*C@qy(Q$Y1K?iO@DUL*ch+1% z9wK1j&>$1McLEb&Zk8+5#cF{jf&aTxfx3yPAYib-S%s<1oju2WfRYkWB~Tuak9)I+ z(-1(skh!xT*2bHo!{JN-dNJ<8yjM5m zG60rH7zk-~uZGNixK`kLe=CruA#>*j!96b-j;Z)?t?(j4`6Spia^GJE{4Ojx680Zt zNWe8%t069;H$XAk92OS^LR}2VREDV856=$Q!%mO|6<}C_6UCa{zd}W<5upDiblg`Y z4Cvl7f*bc0-6U;-JxByu&zNWdaxxqBk$}(fNs-__0UlzBNj3priZ@%}*dQl4?7A@u zxFO-}z(C>X2fTOs4u7+;J0*%HiJsMQxqoBiu59bC{I)* zIwpEv)GK;ZbY1kl=qJ%1q5%)ugY$R_l;6D`VIDej?~k_t(Uq#ab(*CcOB-jjSFxlRYtLG(g8nl{qO zbOHT5{ZCLqIVOM^&rD@zGV_^TOav3dn3%)Nr_5K(_smbsZ;XR+Nxh{3(y`L%(je&q z=^E)esaBdKO_%0LE2WLn1JX|EJJNqkKa+kfy&=6R{Z;m$EI>A1Hd!`RHd8iFwn+Af zOe@pN;$&u7o$Qe8lVqKiD_fkJ-=Jui1W386V`Pb1S)E zZZ{Xs={O@7&!utMTpf3Udy%`wead~q-Q@bYKfGjKDz6z{L0&7o9`}0EYlm03m(I)J zmEe`?mG4#O)#laVb=0fN>w?#dUN3vS=Jl4>2VS3feeLyw*Uw(Rc{#l9deh#V_egJz z_ayH*-iy4Kd2jIE?ESR2*4ylzxhxHlZ~0u+4bSNe2Avwqk&^$DHRv=KS#CD3;S~8SQm|;x zN%uXOg<%H!6sOWpT07MECb~&~iaal%Kr~kA@W=0ly z{t+$Uxdi~XHN7!e%}J9R(_7UXGlAu{@LgPTdU`T9mC4D=%h61g=2Yj|)i)V?b+ui? zE#uW(1@DS-MfI`{o?I@T&abi;)~M_?7x@=n*uipt?Z;r>c-GlBp66Pcnp(J_b~W~k zJU4;W8IE;z9Xr-_5FpZ3`8gH2s@$By{Co|!66RIRN3*C1^>ST?V>+@U!LTF2up`?- zL$|?lw4^nqr~{nKnUu7&6b%lRrZlCsr~{Z@h76@~^htykcl!R`V4$yrCB3Hbq$wn746_@NOa-3Klzp2l^gn2VQjbAuo0?#JQLL z$Mz}bSE*b<%<3&$R%={A(pBfD{9}jO88R43TRRf@j!umu(~;H5a&uR%M853YmDj$} zIQyjET)Xy-no~>!4446Ue9XYDW$(ym^9NXsBiI!j&bBmH*VjYd5uCtsQXS7>`8HO> zDbN}`0?ouLy46Rz8=vn%p8Uqm@ezB}D0m6pght^=)w6thX?kgz2G3qG5zoOZl-P#$ z;62Eu9_V9|U>i5{jy^LBsJUYYou6NrldH_F$f?R#6Z}L^@PMpQjwrgSs={8Q zoOChE&E(fDVqJZ+_^S(9K%?|z4Qv@&$Gd6owP0l%>_y%&IxVx)7#jOLcGPC4#d!g42=Yrv!#JYwQRKph}ax;`_tIz`20);H(1 zsJH++i<8d1wvyoE7px2R-tQK>V~5{WU|KHT4=~~?>;J-zTfD!37u?D8Q>s%Z8#$yy z%h5wD_x>xdywB+ughWP$WMyPzRwT*3=TpiXGn-0FZKbMbDvnhisqR1g!-dcPCCh&K zU-?&5z+T@$$>=nPF5$IkC4LdF#0#)`=@RwFOYj1u#w%4&w-#zI;XGu*dusADPKoOm z8YZ0Itm0}4+W;2`1!=edNfwuq23(9Y^AiBwidZ$*g5O$1LZ$6+E(!Uc|#A>nDKry|{>zcC#+K%kF13+aeB` z9VD9p6UpVd$^V7B9CH{zE9`mIIchS3J(9JvNG|5m;2dy7E#^4~49g)Y8pA2@Lg!dK zg2BOf!)Nnef3=~Zrna)izq+0-OJ%Z4GBT8|Rd_LG9C|4SxZ~=3jfW$p9$pYw$y_dg z$>JhlV>uJMiW^X%#R@E9a470Q>roqx9zaWQErSDbk~yp(uQ0DT&%cNvuP5iE^LQ+u z26PNWna=x2;dpDwYtF2PX<;eXb5R_ zZZpZ*jjdH0&h{xRQ82^3_v)+fai0dznTkb#fpNA>TZj!$wMBp(y(a5G+OcF=O-IX7 zI1yn7^P5|gEmh6+^=fi-zRxzcYPfTi=c-TFqDL>HS)ZW?kxW)_xu>W{<;ZnRKUuRK|0& z{yIfL1XJ`OLv>qeQ+d6Ac^h59pu}O!d{)1 zv*gVuu9H;FWrMuddxQ0v#UA3Pz#$I+SM%g3Mhc$GgAw6?7&+-zJQ9zbG>QEFIth(L zBY*uBja2)zlewX3ESktVZS|5(mkM&oHz$Xv$b>E&ZkH^c3ZkKeyP{@`J>81Zl|K725KKL~og7cTUw&+r2C zUk9>oB)d(Z#5JNP*mUmDq4TywX6_8%+DKj@yYsN}P;F;x zs~Sy06X}*#uDQ7i4t1y4@e^&gBNN(#@|4_eym;lN^{dj7Q_?EUGMmj-qU3N8NR(vr zL5@U0AW!DyaDfW~n7L>qoU7ycb%~=uC}_($bO;~RAg|+gl_}Tm%SPM9pFM`C+p(U`f$Ogj39`p#D49F9Oe2B)Y(1=eW zw)bneg>cL|gV(T-@p*5{tE=Jcu_#{Qxp*GXIvt3kkYHpQ3rMZzl>31_u>s6-4t1k$ z+%4rq9}T342VUdi$!t^dQ!_JRmu7%?geCz#$k7y78#|!3og3_v;<;Rny}YW5!%{qk zYr=}g#4>emYj$g9vy8LVs?h8`L_|TiBLNz~6T}mIn`7Q#x%%eXmYM^ywlbt>Y*KQW ztPgGNM5|#@Lho##(bo(L9oRr~qe#cANDc%f=kjIw`MHHTDlBJG(mA{ekB4g&=UR+@ z#y>k2b08anAWukZCeRZa(ch0ofCOX(Es0wN+K`%qt+#QuZ7_-y0m}#2?n`dsD*wD% zU9TxGD=jNm!ZzETgs?z(%&2dH6S29assTs?*$2o*DW}7G$(=zkCn=n0K=g91j%PTP zO^O&KdH%vD8V)3XPz7L>;2B8w07~qv;%G|;IoyGV`0yOvTG|Z!pBsQ#a448*<@V{7 zdf2gEhBIedl9SbV5}wF0Z(rH8R)gfF3J%|GPxzE<#INuQA;=Fuj>54gr^1)E;a_nA zo)4mW8(@oc8NVA2@UCNk;D%})%w{#z2H@ok=K_g?v+@cKVge`%egi3pAfR$7s)V8% zDeAC@I!=iS?|Kv_iSmi9WFEB;;){P5Rf%dKM4(>OC~6j+5}g+P=`qz~g~xw9Zi~l? z6U67mcO<+dT5?YEC%uhsrC(z|gAE zO*vJ0Soy8esY(oZgqQLER6n4etX{4*s1K;GsNYi~jhAMuW{;*_b1QI4;QGKH$2>CT zA7i<(=f?Sr+dQskyn1}e_?r{PPpF*GHsRt#zlr~zR50n=$@LGNnX+igA5%|F+cqs@ z+S}6~n7(}aZ!^p@%4hsObLz||W*(ijYF6oN$QX$5KDr7zAHmywn^DlpJ_O|_m=Lh-A{Et-MyoGSNERokiok) zBnhB3NFqWKByj{Ii5OXtL=iv-I)VcRzH|jku>?yL&Y*4VU{JsS#rOmaeBcup%p(vg z?BW3W4M&OsA3!q@+*i8Vuj{V(uR|WXD@)op>iqEmJe@|bq0uaUO$x21Z|quaWJ_xUXAmZ_~hhx4bGFsw0wse^@d)0B zL-DjAP%gua%Yc&7*ptG~HMb>n%yYV^Ir+quNu8Y~X zOsAO}fxX6IZ{=QTe4}1~-O+ORpvERWcIMrGol^hUixhq6Nu^Kwy$j!Uz@hXT4-9Ss z-^eat$rCh}7lHN*%g%HL&}$Su8|+c)fPpL~YD3OWLx-U)QRDO)^r8pth-2Z11unc6 zgng%-ae6tu=(e_wW5-~S1W_f(E39}MY+<0HH}t}`?3|LK9Q9xyw$l+A#;7pmon0@m z&K*)1ESq+ndV%!`g!5xSUcduLyEub)22bZfY4K@?Qx%R1r~Nu#$Db%*0|u7If<;f- zZs~|Wl!(S*4>TT2kOs?S>p%Q{+3%`Sh&B5C`;XrEP=ho`23o%ajYA%X+By!lcghCs z(t*>G`3tf5iS25v9E+7>u>TlY=(eddSF1{x5@z+(?=Ec9VE;d`68_zm&3^yMUl5~Q z0Git}{%n4T8P1e5L>?Gep2ptkLk#cJzMcm|(|{by6<_nIywA5V(E)G8Gcom+3bm`G z563%p(Fbx;4q8>~c*j#Xi_WWWENE06tM5GgA^R;KAldIYrnu%>=<-IpTt0YLpJO5Z z7ka_5=ykNkF$!&QjdCo4<9+{Y{}-4YM?Pfn-Sr?2iLE?(P=OM*pd0w2DX66fl@N?-1iD^%I(}!F>Y{#DE3uA#DGd2hEe5<#MzbG*8eJ9rAVS*a7>X z{S`8p!61R*K0CV=3?EN|rl+Y>-AblM$u#nWsCFL|0B zfQG|)pZ4~I6JVA_-Cz?4mQ3W`hJitlTLhF*gLObK6@qDS+lA0x(4E2J0agpr&cu^; zCO{MD_+OBcSu~yntMX9y*I=$xBgAa|S3PuJ@wbLP?TrDFLn7oI!1w?W6b|fFfXJWR zs>T5*;3zvdesBW5jGjNr;s6}*4v+5OI|y>`@(7+gbxs`u84}+uPY@vw00iu76xufo z;xcky3)%Z&;>+Yhm+!$8%J?!scS9CB;mhtZ2z){+m9XdqJo!a-xeFw$i9EJ~O~`HB z##U^V3ifpbIY!5;!OjkR*D9R>68VYgd@_*MUtkE$$-fkUxcc07c}E{~7;XvDpX)Cb|1|XFuvZq>JsB#)PveQe{;jxBiN^8{5K0jUrRqVzDg~18#Ciz@>FQUv zymy! z&*Od810Fl&u{>a&NYRqnoKmjF>yBohOh1`&!vECeGZ#-?l2ulhSKE~}#We+0>ac&U zetlbytST=DEOI$HMPT2?V*?FMarLpa{zkN(ZYfS}NLFDp%px@Hdbg?*+HWKXULd8 zkEK16c|6zUdZ=x9l%!V#N--vs)1Y?7`7@ zUn0ko6}wEv0^s#bf$8Y;nt{g#G6c;O9Rxkp~37xp$cQT7Cj!TNVhT`^& zI&4Hw_&KKS_Q{rzgsVT3nbUxjS!=s=ByFFeTQM)>Kqhz5aopk1G=ntHm(bZMG8dQ$BhNn1}_Fh1}7Nti)0c zsT@ogRyZ#PtP12$h;{@IwrJG15JZTZim@zu2-s#H3a(^DF9b*f!~-`SXB4TWX_;v% zT*RcM)i;-FDx{sz1Pp>3(E_#;_tAw?r_B|uIG=Ss?X=o8Z{QexDBE<7`o%{7?Ua9oUL)qyK{_Ai_VIOP#S7N&Z?ckpe>SiZNU9u zm_q=i4bJZ5(sVGj!PB!f7mo=XL{82L5inMgk&7V{T*SK~8Nwgw=%`(Z+g00lwVjUA zU=<3WUD{k?Dq6tekKu^y$hJ1`S7AGt=)v}92iHh2woB0rmiQX{&w_)RM|6e?WpRxG1qwgX1Z!msyPF7Ub7d7P6Vlc}3fyKQX z{8za}`FR?A4PT@4^9plwl!99goGkcu9*=ILU}-~rO?{;X|K@0ah;2_8fQ@>SAE*Hu zm0Ehb1*Q3A1^#G9oZ@s=Z~7@U&T;h6C(|Pi z>r_B2x`_Sz(lt28)kCN2v$jPmT?xPQJ9rqtDh3Y{nDII?+Y{^5u5Q$qRByH=X89*( zW+qsbz#re{>&mNY!JH4q<+i%|_71QcjvmY20Be`s_Y9ba=Ca)^9*q@#$RFGQTd(6C zD%WBR767mVjOD@V9ovsqp^2K>2HSzmI?N+AtVd2c@Vk*_I(IXT8ZbX?y>VB zUjx`hNA3vvLF4-_R%7+suyd>U8$5c5_dOFpf9J3&TGE@)C^juSC%r(E5|OF3M9T2A z8F=ALyha5M-v?g!X1a!$w-VTSu>AxDq`vRwfu|HHXh4~0-SQeQgF!}1ZYz~VPn9c zflBaRv=`n3Qn*Usc#Ek45eF0^LSR7lb6Mh?HnDpSg`cyk1F(JR%Ob?7Vgyf{qpy_(zgvuS>Vj=cLo{pa z>7>`QufDBBFQFGv3;F@B7jX-I>9Oo}NgLE_GwF{*7W7V4osfp`C!~n`D{ zw)N2Ge`)&ziIhHfGEX#uH_&MpKf(LB?vesIuAl_mzgzL^#-FF3QCH;Vl;)~*24l45 z5hQEJ5XpdL?T;vL1Qt`RP}9%>a6BA^|X!|NjdB_-jxI_CZ_l=Idxa zYiv&H$kZH3Ka|;-Ec<2Ut6=@}QDUDhSUP#7+LCO}G^NX|nW;%eh5%56KxP0ZU4iv*KA7w1xTwa7;q_g#*D8$PI$hF$~8E;@fbZi2er?M%mste&UVe zXw>l^U;pv=3AlcEd7Zho235`~JX|gRb zKMD8VG5SSkg(gI)?#yI@*VMn7sL4H8YOkr6)!UoP8&pmwgM1I4LNhLF(2)Uk4S`SY@Fxs`Oc(;0h69>rvKnWwBS-<;xgEr(x6DibxmxA2GpmIW%yoQloTB&TirQB-&)3iy;JKCM^{C2fZQ!-8vmGcos@_>` zs?06jUahZ9ZjxoybQv>rMOIl>wlW*yIdawc z1=gI%9Q>fsugF}o-=uuC4DGI?OOHNR`nu}nH;VJ$(-gdSwdhq6NdZ#d`u?6~~Z{9B`t z1-wD7iVv{1TrJ$)^S%f-D(W5jPFReasvb;xyJU+{ge@XLF!sW1Y>t#pxHf&n1 zT#>nH|1Pz8XL!_BlgzYrRr(xN=QBka^;w~<(os*A)DqVV3{f`x~wu*<2rlCTY(;`{I>jL zIg(cYQuReK+EM8DP0?Fb7i+$1ey6Rcv#0a&>5I>wJl%P&@mbk{muvs|59Qaf*EhbW z_U+#I{v1%Pj(mLjABWnTWxgjboH*Xqepc3gw(i1Z<%PWN^t0;pv+-Sq_cH?QCUG% zdPQ{U<|=F`!^+a9%Ut<>^NXIy4^bDT=A~pM$7FvlUt%w-s(;S!0?Is#=3GHno8CWo>lpI)FKe$jT79zST+OkX zwj*_?YR}i6x1XsyQCHPo(E_mQ%IeFS(o1y3!G*H?$*YP&RM{3=S)>NP*O)ZkUffX9 zT;l&u;qy61(`3n|nI*aE+#T^)mAc-5XO|S1md4@P{+a8x;&v0(YMUovWmkUrJ&Pu zXoQi+mlzyVO8Y8*2502splvA@57<9pE;b(RGHHC@z@yN7Q&))11UB+fcs{K&H5xCf zKDlFG%!H&Hbw@N1lr{f|?xO7oSi+$#0O~rDel$eo146*S?V*`hq6(0H%NP%`pACJIXr6*_&%wUIKAOx$>g;p&(WnhH6fYKMq71sza*elGHFyzT zNPIVF5n6Pb9n8$&3wSgMoXv3B$C6Mh1fewGk~#e>zp;A#;b65xG}uIkv|TbiuX_H{ zk&Epb2jy&{55H9X#uX)4CZOX@#Zq2#rw<$&plbvIOi;aXCP=0bJUn3c-RxUQ+%1X* z{>fL~SNpafs_Cq6Q#Z8rzSI7;tgaj)tW-6%1zF{q_Q!hHHYCdG6KgDHrSE2tnfv2@ z*#3!n`zLrG>Rg06WEV2S+hbHQ5ecCgnnkz+d`6wy7t4G@cPx&bJ`uY72A&*2kiR() z6bXoV6U+i~@qib)t=M{V>dOo`ML-S4(`fXOqhDdqDM`!8!N1|({Bm;AN^(==Jist4j@u&|VHkfH@Du$@Qy2AQ$ zyS=B!4Apu-Qm z??=AR!Q1>cw5nx=g{6hW@|2gSS+|amKUv#qsXH{+_oKfB=iXcIlJfGBa)=elxEVFOi~iUHd&I=pcASXucdT%& zI1%%L?ZgRx=S$9)Xz&P5Vg--jbHH8UD3D7bnD#I%oeT0z8Q3~q@{90U0|W>Iq7TOh z1NXBNgAP&M96-(t7<7ax5CV`lsF`;0Kr{)mF%V-31dg>2)dn!v5Y0Px-e3)^bLR_u zAk-tD0EPi=Wb4oq5)tMOdh~ZfmOf-|vv(;;YY^!I0+^8?SJRo`dC@ukP#kZu9gS@X z7R zCS-&8Ac`H_`5nyExf3wSe-KjId?+zTryShb!;;qltDAkOl@Z$Z084;cCoF^bIV@Ee zi3{;N-Umb2864mq;zq|m6=t(Nu}cM>#x8r?A+v@+MLw**Gn*WdKniw(tq8euTdsi8Zq0W~rrMOat z%m0Qa9T0xxB&|C-8&94BV}cy@fj6lSv`8TpH^P5~fbH1MJPwr1O5YI>fq5L>0N%zO zpw)L380LDgt&xsGhe10dgc}3xt5^u(a<_ofE8Q_ik&>4J5mvKj)0vr&g(IvQf*&EM z=Wz@dRD$rSN=YG=v%iJN&b$_g?5u8v$WA1*LC~f?kA!H=1=V$Z2@4m*i z!)jf11|vI|n8CTKI0gr=6lqxSh(fRxsD;zUZFwYAz1w8iX;p%+pFb`A>8H=%KcT*I z^vK~Cl@~X6uZ!LX%cM?9PfXsuNtT-rdYCFNudJd#gZ+NZs4Z-@H~OP-Um>6O(8DSS zoDRl3UI$DI2g5tT@K!iGt*{MN6a;gygZes?bp@Y!A_yRcap%RV1Aj6_&7Kx;2d?wJhEtaB~olpbt#z|334}xAjCm}zo^*y)xKLutVI8W?{JDyFB1Q@ zZ_8I|ht9Q2;aCbEKK)ESZ-CDnes(Q&ErZV-ejfVF;b+G(wNC)OE>Uz9__G-Nz3=RO zZ6z2L7<36;qB{jz2UcO}R4@MkgsPa&d5c9es2Nn#RuU84VO2XdgMo>XE1Z^x!2y&xJLkH-3zbN3m%kH8KljihAJNb-ug>0nsnuBd*6X?d6;)zd+r*T zW2CS(mmnq)+H`6@{E%?I6J&tp0rb`DATh%L%b^w|O)E&6u#ND-5T68qh?oB|I~X|p z2@cFJ@H7ifZHSfthPe--wSjaqP6Yd#K)hyrfmUFjYbnTCJU^_5+x3N53hR# z%hh$(x|pT}S$1`GUZbk5zWG3NVQWdVrl`BPyIbklk4}H?SP7qr0PoF%gUtaaGMsqM zLWgx1?>y+dy%z!%qyh8|Q3L#d1ncPA3r`1b?*eB7@SU5^Ai{UTK*kTiV-(5hX({SM zd~#Y-s|GzOZEb1-=Sncs(wLU4DMm9C=_P4d;9uOpB&F3gYEqmc8a&F?73#_=d%0bO zOpM)LR8XaQxY8$jL6_Ykc&_$lHY{ri9Qr?lgOz-=rM)PkfMXZbcU8L&C61U zPD*?Y2U(X+x>f4h?fglZc;v8 z4XQz@C<#qQf2!cj1MkmH#g|cl&Gf^j-P?oJ;GFSuJ$4<3t(D<3({U9}#P2J0<+>`p zx+3xLwwx_^=b~}Sgz9{Iih9qH1F>&>{Td2=L3RG-`qbw&u{VB6y{SUe(A4wqAe9D; z`f9Wr?Y)Yw${Ma#zj>8d_#v(fJp@s(pg{&fWG{s1xT8FPC^iG04cu0s8#oI-dO3!C z)ukmxrS$QQT{BkW8dtF1<*URuP!?W^j$vPQNohq19dkwZ{d=g!5q!$w3*la{n*$Ow zUgQWyI(rdKs&+03P}IdMxon^wJ+EegJG^7B0Xxyc%CLKZ^bQ;6Uhr6Dl5U z*PMIqT+i`;$Qlk-w;v`8L*z602~b(lJVNvDvqSXW2=x9Z55$h2lomT!MMg4@`|!bbNtJ)t8(lGj!JyO57)!Bt(Pt>F0vKDH>o6MXX+Gi=;uJYQV7SX zDF7jBiywIBDywp93TsRJOKtE~7}!oUH*Z3GK79S*zYT3e^>CeVRgw<&V*iqIh%Zr9 zSC>^(g0^$Bwx+V7sNNq3IoG3kXx`16S5eTqtNx(10=0Et1*sM6Fn;`rt0#cl1;ImD zSRpS5K1Zw^3dHeOM zu@muwpA$d5brnd044QhC_)A~aod2Qw`&c>N|F)9h5%!0F8W~ zOX7qE><;<;HLE}y1wH9Hs3Sy80@-H}q@3Y{UXUS<^Hw5*49O3md?gc|=`UFU{A{4D zfsjB9Qhx~vM5zLGEd^u)kVD*p1(97&Lo5)Q4r>Qeb258EQC(D1Sf$265MffCpAA7} zu0Bx7gPCP)Q$bU99Yk<~t)Ve9xh6@Kl$@ImT2Y@%PG@Hoq@^K<+=iYnHXFSjIS=0spgd563i}N>f zk6XpVsBFQsxjg;O?JtUpi3k7a-Q)VbjFxT zvu)6pLrfF{lxH+gg0LQH5P-V>h`o9|_GVmVuA$1Ut2S;}6C%w{$x2C4(R#2LTireA zGXTz?AH*3;N=>Ee2jA~L^BMn|dECX&Z;-VqG#0AMi!9bMen9!STMt!W*k*AJ@r}uQ zOwxJ#0$W;D`|_L0>bXB)X}$J3c{4?dR8nb)ib(I>Bhm|}!`AHMjyMjLHP^%~-Mo6` zw)brZ^7oZWu@o)zM-Yj0asEV>kgepk&VHgHWG&VNHI`!fX8XTrvGZR*G;ak; z_W2{SfrA;dl|CgNoxWurPdk&P60(Nu^~V4|r@17&e~&0W^3bDNU~(%E9)-op%uY-c z!!*o*9Hxl@^o{X&85^7#&^;#N47#r>34Hv6m?MO%%Dp&A&K~$gK==z0Z!KOreIzYJ zA#wr=C8jcPn25upDggj}Cvm6@vF=Xfc`&lY418P3?p#c^TJ*y6+{M}Iawy-Ig>1DK zY~u>H*|&zM-k0?pe*4j*+qWO>+>w@4$0gOJ?bxYe?;qVB-jj3QZPzMy(gsqpp^5YA zFX&!-O}Fjd=*mbQYb6XH(N}FJ(GedN384c>e;Q10bUcFbZU6}(KwzBws*Q6FYaiCZ zZ#>h|a>fHt=4mJiy?OObZ6j8`8bz?L28{2 zw?jE)-rUJk=AOM;r}^|8;JYqI*Z+LN$?fbzkl5X$ltsyf3BcYCtWMdHv^{aV?~eVu z_U_y-&9MQ@s@g$iq|>$<&YF(d2q6oj0kB)y(C~t={B60uI#4%?j0yP(YC21tkd&N| z!6z;?Xbnq3Q^JzN5~<{SpB&GQAwU;D7aGMQZ2-R`&61Xr&NZyxwPDBF#4vqW>NfgX zxDR65@rf!rQ<9LESY+hLz;MUbg3zK+-;i~|8$#AgK|X~5LkN-i*M)PyeIgfQ&ov|Y zKxE(5B-QHcQhlqzLP;5J54mbj=OuLx1%qt?^bw&`B{My_)@>-2gp*gR(Pz9{PZ%WcbGeJfMYUJa}R{xq( z!4Wm+0@+>hv3$}5nLGtwdB2d)!dJ|$Z2BieX4oF0#rORpS2BDwoUT1t*y&<5l|L z6PbO#Ve63PCayBPXnBxIzSa7(#u8(Wjs~D}bToL~v?1%ZN$GZW z!(kqL9+nsmT)E>$aPm%m1+I3V)#N2Ly7HrVueeoKd$91>F;#VDO?nmAaHRC?IaN1U zZ&vTC^W|P??H8 zt(!nK+>8$!$*cVzZrvGPA673t_b$aqj8zAT<+D#>a3p8$?kzvX?;}qU@g5?BC5kU9 zNte%;U|{64t-UaPaW-@T5p?cToA-<*J~B<&ohWw)w!cW5@;|KTS&P zdM@^C&=Jm7WvQuF;Sk3XkA)rN%thJ7MXHv_mUYKCt3-bAB$=I!*|QU!uBKhZbP#=E z{Sx{zpByqec&nOX;AWqEGK|~B`?q~EWY@agEBCD0xAy$>Ep+Iw{iNP-%OAfs{d|!=I z%ex;^FJ#^vx*H}$k2uZ0HJ)?}>4_CsabMZA&Jc#Ys@R)F(Rw9Lnly(JKiTo73>MNq zq;8P#^nSs+0)*yGh>sxm?VNs(q>+3~)5-AR<@jg7zvM1>+fC`5PU709ONw3o%D0y+ z7|mswByTJ^_0cCMPF%l!bkVeIUby+#Unxi=_cmXCea8A#Yhts;gSNn2s#9Pz3USvXoF>* z1qz5+X8?tr|2n`1gQ*WEI3#r%uqSZ+d-PuzdxCevO7{WvelUFa4`d{OX2>D4?1)DchD@fD zkx%dkAp|kmQ5vKI{Ml#3kIgO2u;~m?lEMpM-UP%pX}gRT#qSnQ+qz-D6$q_np!we% z#v?kG2bBWvH=AG#w*FfNQ__W`u+YjV21KEFU3k~oQ%RRJQ(xlui|RfS2y{pT?e^Yl zoa-{#q3lO}fkjxdhI{XB1CWzLfSViu(}yU&meJ<>;tZL)HC{G=GR2dFGCGgM(hcOp zc<#XBrr@#!>B(h9OJ=BM1i{H1Fk=7*NWK%0{1(am0WAXt1hurZ6dgNxgexm*+I8T# zlzdnWQp*O$sKYg~>3mgubySt5{$3Fhd@G5fmb|miIhNGRb505zc}JO(V|1k3puUlv zVK8KvQ|##wWHRMgrSb{-)fbf+_Ed`@!;qN;Vuv*?H#5f~&5~GivT_Y}>8uM%b55o; z-2&{m$(U)(uo!Ha)=Zn(Y?0OnDswC*yTN9#rXh)#k(r%lO}85C#+)1}!T?>BW?Q-) z$N&gO7?C!&r8$gJd2c<)gch?+dfA|~r&?1?TuPcDJ&%jV_J>m7EhjX#&CG}$0P zV@ffmr)Q^Sg970&18-w9*`%(;t~pG_3l3q!?yMtxnd!T?G&{m;R=oLg7VQ$ITGp7= z0HX<~kKqLViyF`ZX25vy#L&qLUWauretq((&qI0l`2SD>mMinB4LhRCn7V~eVN$Fu zP8}EPK`3b5+K*vxxV7R}@zhr)XmR%Is!M9}cy4h%WV1ykvRAQnh@pe{fv& z4*p=(dxuqWYvqlw>o-&+{ZrCN-X*Vc=MP?M_+-0u_wDcZ{HT^2{IRNumXT-n?|1B1 z=UB5$IlSCH!4a1o75#4VyDL-+@C;qngg&E|n?r_%!H$Fxa>!;Y#Q zJ9