You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by ma...@apache.org on 2014/06/09 22:31:06 UTC

[1/7] SAMZA-7: Rewrite Container section of docs to bring it up-to-date. Reviewed by Jakob Homan.

Repository: incubator-samza
Updated Branches:
  refs/heads/master a037d6f25 -> c72223f99


http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/jobs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/jobs/configuration.md b/docs/learn/documentation/0.7.0/jobs/configuration.md
index d4a516e..3bb80ef 100644
--- a/docs/learn/documentation/0.7.0/jobs/configuration.md
+++ b/docs/learn/documentation/0.7.0/jobs/configuration.md
@@ -15,7 +15,7 @@ task.class=samza.task.example.MyJavaStreamerTask
 task.inputs=example-system.example-stream
 
 # Serializers
-serializers.registry.json.class=samza.serializers.JsonSerdeFactory
+serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory
 serializers.registry.string.class=org.apache.samza.serializers.StringSerdeFactory
 
 # Systems
@@ -24,7 +24,12 @@ systems.example-system.samza.key.serde=string
 systems.example-system.samza.msg.serde=json
 ```
 
-There are four major sections to a configuration file. The job section defines things like the name of the job, and whether to use the YarnJobFactory or LocalJobFactory. The task section is where you specify the class name for your StreamTask. It's also where you define what the input streams are for your task. The serializers section defines the classes of the serdes used for serialization and deserialization of specific objects that are received and sent along different streams. The system section defines systems that your StreamTask can read from along with the types of serdes used for sending keys and messages from that system. Usually, you'll define a Kafka system, if you're reading from Kafka, although you can also specify your own self-implemented Samza-compatible systems. See the hello-samza example project's Wikipedia system for a good example of a self-implemented system.
+There are four major sections to a configuration file:
+
+1. The job section defines things like the name of the job, and whether to use the YarnJobFactory or LocalJobFactory.
+2. The task section is where you specify the class name for your [StreamTask](../api/overview.html). It's also where you define what the [input streams](../container/streams.html) are for your task.
+3. The serializers section defines the classes of the [serdes](../container/serialization.html) used for serialization and deserialization of specific objects that are received and sent along different streams.
+4. The system section defines systems that your StreamTask can read from along with the types of serdes used for sending keys and messages from that system. Usually, you'll define a Kafka system, if you're reading from Kafka, although you can also specify your own self-implemented Samza-compatible systems. See the [hello-samza example project](/startup/hello-samza/0.7.0)'s Wikipedia system for a good example of a self-implemented system.
 
 ### Required Configuration
 

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/jobs/job-runner.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/jobs/job-runner.md b/docs/learn/documentation/0.7.0/jobs/job-runner.md
index c73b234..b41a410 100644
--- a/docs/learn/documentation/0.7.0/jobs/job-runner.md
+++ b/docs/learn/documentation/0.7.0/jobs/job-runner.md
@@ -37,9 +37,7 @@ public interface StreamJob {
 }
 ```
 
-Once the JobRunner gets a job, it calls submit() on the job. This method is what tells the StreamJob implementation to start the TaskRunner. In the case of LocalJobRunner, it uses a run-container.sh script to execute the TaskRunner in a separate process, which will start one TaskRunner locally on the machine that you ran run-job.sh on.
-
-![diagram](/img/0.7.0/learn/documentation/container/job-flow.png)
+Once the JobRunner gets a job, it calls submit() on the job. This method is what tells the StreamJob implementation to start the SamzaContainer. In the case of LocalJobRunner, it uses a run-container.sh script to execute the SamzaContainer in a separate process, which will start one SamzaContainer locally on the machine that you ran run-job.sh on.
 
 This flow differs slightly when you use YARN, but we'll get to that later.
 

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/jobs/logging.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/jobs/logging.md b/docs/learn/documentation/0.7.0/jobs/logging.md
index 6bb6bf4..65a755c 100644
--- a/docs/learn/documentation/0.7.0/jobs/logging.md
+++ b/docs/learn/documentation/0.7.0/jobs/logging.md
@@ -7,7 +7,7 @@ Samza uses [SLF4J](http://www.slf4j.org/) for all of its logging. By default, Sa
 
 ### Log4j
 
-The [hello-samza](/startup/hello-samza/0.7.0) project shows how to use [log4j](http://logging.apache.org/log4j/1.2/) with Samza. To turn on log4j logging, you just need to make sure slf4j-log4j12 is in your Samza TaskRunner's classpath. In Maven, this can be done by adding the following dependency to your Samza package project.
+The [hello-samza](/startup/hello-samza/0.7.0) project shows how to use [log4j](http://logging.apache.org/log4j/1.2/) with Samza. To turn on log4j logging, you just need to make sure slf4j-log4j12 is in your SamzaContainer's classpath. In Maven, this can be done by adding the following dependency to your Samza package project.
 
     <dependency>
       <groupId>org.slf4j</groupId>
@@ -18,7 +18,7 @@ The [hello-samza](/startup/hello-samza/0.7.0) project shows how to use [log4j](h
 
 If you're not using Maven, just make sure that slf4j-log4j12 ends up in your Samza package's lib directory.
 
-#### log4j.xml
+#### Log4j configuration
 
 Samza's [run-class.sh](packaging.html) script will automatically set the following setting if log4j.xml exists in your [Samza package's](packaging.html) lib directory.
 
@@ -42,9 +42,7 @@ These settings are very useful if you're using a file-based appender. For exampl
 
 Setting up a file-based appender is recommended as a better alternative to using standard out. Standard out log files (see below) don't roll, and can get quite large if used for logging.
 
-<!-- TODO add notes showing how to use task.opts for gc logging
-#### task.opts
--->
+**NOTE:** If you use the task.opts configuration property, the log configuration is disrupted. This is a known bug; please see [SAMZA-109](https://issues.apache.org/jira/browse/SAMZA-109) for a workaround.
 
 ### Log Directory
 

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/jobs/packaging.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/jobs/packaging.md b/docs/learn/documentation/0.7.0/jobs/packaging.md
index 62c089a..4f06625 100644
--- a/docs/learn/documentation/0.7.0/jobs/packaging.md
+++ b/docs/learn/documentation/0.7.0/jobs/packaging.md
@@ -10,7 +10,7 @@ bin/run-am.sh
 bin/run-container.sh
 ```
 
-The run-container.sh script is responsible for starting the TaskRunner. The run-am.sh script is responsible for starting Samza's application master for YARN. Thus, the run-am.sh script is only used by the YarnJob, but both YarnJob and ProcessJob use run-container.sh.
+The run-container.sh script is responsible for starting the [SamzaContainer](../container/samza-container.html). The run-am.sh script is responsible for starting Samza's application master for YARN. Thus, the run-am.sh script is only used by the YarnJob, but both YarnJob and ProcessJob use run-container.sh.
 
 Typically, these two scripts are bundled into a tar.gz file that has a structure like this:
 

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/jobs/yarn-jobs.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/jobs/yarn-jobs.md b/docs/learn/documentation/0.7.0/jobs/yarn-jobs.md
index 3d971cd..5dbbe54 100644
--- a/docs/learn/documentation/0.7.0/jobs/yarn-jobs.md
+++ b/docs/learn/documentation/0.7.0/jobs/yarn-jobs.md
@@ -3,7 +3,7 @@ layout: page
 title: YARN Jobs
 ---
 
-When you define job.factory.class=samza.job.yarn.YarnJobFactory in your job's configuration, Samza will use YARN to execute your job. The YarnJobFactory will use the YARN_HOME environment variable on the machine that run-job.sh is executed on to get the appropriate YARN configuration, which will define where the YARN resource manager is. The YarnJob will work with the resource manager to get your job started on the YARN cluster.
+When you define job.factory.class=org.apache.samza.job.yarn.YarnJobFactory in your job's configuration, Samza will use YARN to execute your job. The YarnJobFactory will use the YARN_HOME environment variable on the machine that run-job.sh is executed on to get the appropriate YARN configuration, which will define where the YARN resource manager is. The YarnJob will work with the resource manager to get your job started on the YARN cluster.
 
 If you want to use YARN to run your Samza job, you'll also need to define the location of your Samza job's package. For example, you might say:
 
@@ -11,6 +11,8 @@ If you want to use YARN to run your Samza job, you'll also need to define the lo
 yarn.package.path=http://my.http.server/jobs/ingraphs-package-0.0.55.tgz
 ```
 
-This .tgz file follows the conventions outlined on the [Packaging](packaging.html) page (it has bin/run-am.sh and bin/run-container.sh). YARN NodeManagers will take responsibility for downloading this .tgz file on the appropriate machines, and untar'ing them. From there, YARN will execute run-am.sh or run-container.sh for the Samza Application Master, and TaskRunner, respectively.
+This .tgz file follows the conventions outlined on the [Packaging](packaging.html) page (it has bin/run-am.sh and bin/run-container.sh). YARN NodeManagers will take responsibility for downloading this .tgz file on the appropriate machines, and untar'ing them. From there, YARN will execute run-am.sh or run-container.sh for the Samza Application Master, and SamzaContainer, respectively.
+
+<!-- TODO document yarn.container.count and other key configs -->
 
 ## [Logging &raquo;](logging.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/yarn/application-master.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/yarn/application-master.md b/docs/learn/documentation/0.7.0/yarn/application-master.md
index 0da6dc0..92e1e18 100644
--- a/docs/learn/documentation/0.7.0/yarn/application-master.md
+++ b/docs/learn/documentation/0.7.0/yarn/application-master.md
@@ -7,7 +7,7 @@ YARN is Hadoop's next-generation cluster manager. It allows developers to deploy
 
 ### Integration
 
-Samza's main integration with YARN comes in the form of a Samza ApplicationMaster. This is the chunk of code responsible for managing a Samza job in a YARN grid. It decides what to do when a stream processor fails, which machines a Samza job's [TaskRunner](../container/task-runner.html) should run on, and so on.
+Samza's main integration with YARN comes in the form of a Samza ApplicationMaster. This is the chunk of code responsible for managing a Samza job in a YARN grid. It decides what to do when a stream processor fails, which machines a Samza job's [containers](../container/samza-container.html) should run on, and so on.
 
 When the Samza ApplicationMaster starts up, it does the following:
 
@@ -25,11 +25,11 @@ From this point on, the ApplicationMaster just reacts to events from the RM.
 
 ### Fault Tolerance
 
-Whenever a container is allocated, the AM will work with the YARN NM to start a TaskRunner (with appropriate partitions assigned to it) in the container. If a container fails with a non-zero return code, the AM will request a new container, and restart the TaskRunner. If a TaskRunner fails too many times, too quickly, the ApplicationMaster will fail the whole Samza job with a non-zero return code. See the yarn.countainer.retry.count and yarn.container.retry.window.ms [configuration](../jobs/configuration.html) parameters for details.
+Whenever a container is allocated, the AM will work with the YARN NM to start a SamzaContainer (with appropriate partitions assigned to it) in the container. If a container fails with a non-zero return code, the AM will request a new container, and restart the SamzaContainer. If a SamzaContainer fails too many times, too quickly, the ApplicationMaster will fail the whole Samza job with a non-zero return code. See the yarn.countainer.retry.count and yarn.container.retry.window.ms [configuration](../jobs/configuration.html) parameters for details.
 
 When the AM receives a reboot signal from YARN, it will throw a SamzaException. This will trigger a clean and successful shutdown of the AM (YARN won't think the AM failed).
 
-If the AM, itself, fails, YARN will handle restarting the AM. When the AM is restarted, all containers that were running will be killed, and the AM will start from scratch. The same list of operations, shown above, will be executed. The AM will request new containers for its TaskRunners, and proceed as though it has just started for the first time. YARN has a yarn.resourcemanager.am.max-retries configuration parameter that's defined in [yarn-site.xml](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-common/yarn-default.xml). This configuration defaults to 1, which means that, by default, a single AM failure will cause your Samza job to stop running.
+If the AM, itself, fails, YARN will handle restarting the AM. When the AM is restarted, all containers that were running will be killed, and the AM will start from scratch. The same list of operations, shown above, will be executed. The AM will request new containers for its SamzaContainers, and proceed as though it has just started for the first time. YARN has a yarn.resourcemanager.am.max-retries configuration parameter that's defined in [yarn-site.xml](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-common/yarn-default.xml). This configuration defaults to 1, which means that, by default, a single AM failure will cause your Samza job to stop running.
 
 ### Dashboard
 
@@ -42,7 +42,7 @@ Samza's ApplicationMaster comes with a dashboard to show useful information such
 
 You can find this dashboard by going to your YARN grid's ResourceManager page (usually something like [http://localhost:8088/cluster](http://localhost:8088/cluster)), and clicking on the "ApplicationMaster" link of a running Samza job.
 
-![diagram](/img/0.7.0/learn/documentation/yarn/samza-am-dashboard.png)
+<img src="/img/0.7.0/learn/documentation/yarn/samza-am-dashboard.png" alt="Screenshot of ApplicationMaster dashboard" class="diagram-large">
 
 ### Security
 

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/yarn/isolation.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/yarn/isolation.md b/docs/learn/documentation/0.7.0/yarn/isolation.md
index c685729..1a4f315 100644
--- a/docs/learn/documentation/0.7.0/yarn/isolation.md
+++ b/docs/learn/documentation/0.7.0/yarn/isolation.md
@@ -13,7 +13,7 @@ YARN currently supports resource management for memory and CPU.
 
 YARN will automatically enforce memory limits for all containers that it executes. All containers must have a max-memory size defined when they're created. If the sum of all memory usage for processes associated with a single YARN container exceeds this maximum, YARN will kill the container.
 
-Samza supports memory limits using the yarn.container.memory.mb and yarn.am.container.memory.mb configuration parameters. Keep in mind that this is simply the amount of memory YARN will allow a Samza [TaskRunner](../container/task-runner.html) or [ApplicationMaster](application-master.html) to have. You'll still need to configure your heap settings appropriately using task.opts, when using Java (the default is -Xmx160M). See the [Configuration](../jobs/configuration.html) and [Packaging](../jobs/packaging.html) pages for details.
+Samza supports memory limits using the yarn.container.memory.mb and yarn.am.container.memory.mb configuration parameters. Keep in mind that this is simply the amount of memory YARN will allow a [SamzaContainer](../container/samza-container.html) or [ApplicationMaster](application-master.html) to have. You'll still need to configure your heap settings appropriately using task.opts, when using Java (the default is -Xmx160M). See the [Configuration](../jobs/configuration.html) and [Packaging](../jobs/packaging.html) pages for details.
 
 ### CPU
 


[5/7] SAMZA-7: Rewrite Container section of docs to bring it up-to-date. Reviewed by Jakob Homan.

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/img/0.7.0/learn/documentation/container/metrics.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/container/metrics.png b/docs/img/0.7.0/learn/documentation/container/metrics.png
deleted file mode 100644
index 8be2a38..0000000
Binary files a/docs/img/0.7.0/learn/documentation/container/metrics.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/img/0.7.0/learn/documentation/container/tasks-and-partitions.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/container/tasks-and-partitions.png b/docs/img/0.7.0/learn/documentation/container/tasks-and-partitions.png
deleted file mode 100644
index 68b753f..0000000
Binary files a/docs/img/0.7.0/learn/documentation/container/tasks-and-partitions.png and /dev/null differ


[3/7] SAMZA-7: Rewrite Container section of docs to bring it up-to-date. Reviewed by Jakob Homan.

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/api/overview.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/api/overview.md b/docs/learn/documentation/0.7.0/api/overview.md
index 2d03ec3..489ba6d 100644
--- a/docs/learn/documentation/0.7.0/api/overview.md
+++ b/docs/learn/documentation/0.7.0/api/overview.md
@@ -3,100 +3,112 @@ layout: page
 title: API Overview
 ---
 
-When writing a stream processor for Samza, you must implement the StreamTask interface:
-
-```
-/** User processing tasks implement this. */
-public interface StreamTask {
-  void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) throws Exception;
-}
-```
-
-When Samza runs your task, the process method will be called once for each message that Samza receives from your task's input streams. The envelope contains three things of importance: the message, the key, and the stream that the message came from:
-
-```
-/** This class is given to a StreamTask once for each message that it receives. */
-public class IncomingMessageEnvelope {
-  /** A deserialized message. */
-  Object getMessage() { ... }
-
-  /** A deserialized key. */
-  Object getKey() { ... }
-
-  /** The stream and partition that this message came from. */
-  SystemStreamPartition getSystemStreamPartition() { ... }
-}
-```
-<!-- TODO This description and example needs to be updated to match SystemStreamPartition. -->
-Note that the getSystemStreamPartition() method returns a SystemStreamPartition object, not a String, as you might expect. This is because a Samza Stream actually consists of a name, a system, and a stream. The name is what you call the stream in your Samza configuration file. The system is the name of the cluster that the stream came from (e.g. kafka-aggreate-tracking, databus, etc). The system name is also defined in your Samza configuration file. Lastly, the actual stream is available. For Kafka, this would be the Kafka topic's name.
-
-```
-/** A name/system/stream tuple that represents a Samza stream. */
-public class SystemStreamPartition extends SystemStream {
-
-  /** The system name that this stream is associated with. This is also
-      defined in a Samza job's configuration. */
-  public String getSystem() { ... }
-
-  /** The stream name for the system. */
-  public String getStream() { ... }
-
-  /** The partition within the stream. */
-    public Partition getPartition() { ... }
-}
-```
-
-To make this a bit clearer, let me show you an example. A Samza job's configuration might have:
-
-```
-# the stream
-streams.page-view-event.stream=PageViewEvent
-streams.page-view-event.system=kafka
-streams.page-view-event.serde=json
-
-# the system
-systems.kafka.samza.partition.manager=samza.stream.kafka.KafkaPartitionManager
-systems.kafka.samza.consumer.factory=samza.stream.kafka.KafkaConsumerFactory
-systems.kafka.samza.producer.factory=samza.stream.kafka.KafkaProducerFactory
-...
-```
-
-In this example, getName would return page-view-event, getSystem would return kafka, and getStream would return PageViewEvent. If you've got more than one input stream feeding into your StreamTask, you can use the getStream() object to determine what kind of message you've received.
-
-What about sending messages? If you take a look at the process() method in StreamTask, you'll see that you get a MessageCollector.
-
-```
-/** When a task wishes to send a message, it uses this class. */
-public interface MessageCollector {
-  void send(OutgoingMessageEnvelope envelope);
-}
-```
-
-The collector takes OutgoingMessageEnvelope, which allows tasks to supply a partition key when sending the message. The partition key, if supplied, is used to determine which partition of a stream a message is destined for.
-
-Please only use the MessageCollector object within the process() method. If you hold onto a MessageCollector instance and use it again later, your messages may not be sent correctly.
-
-```
-/** A message envelope that has a key. */
-public class OutgoingMessageEnvelope {
-  ...
-  Object getKey() { ... }
-}
-```
-
-And, putting it all together:
-
-<!-- TODO Verify that this example actually works. -->
-
-```
-class MyStreamerTask extends StreamTask {
-  def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) {
-    val msg = envelope.getMessage.asInstanceOf[GenericRecord]
-    collector.send(new OutgoingMessageEnvelope(new SystemStream("kafka", "SomeTopicPartitionedByUserId"), msg.get("user_id"), msg))
-  }
-}
-```
-
-This is a simplistic example that just reads from a stream, and sends the messages to SomeTopicPartitionedByUserId, partitioned by the message's user ID.
-
-## [TaskRunner &raquo;](../container/task-runner.html)
+When writing a stream processor for Samza, you must implement the [StreamTask](javadocs/org/apache/samza/task/StreamTask.html) interface:
+
+    package com.example.samza;
+
+    public class MyTaskClass implements StreamTask {
+
+      public void process(IncomingMessageEnvelope envelope,
+                          MessageCollector collector,
+                          TaskCoordinator coordinator) {
+        // process message
+      }
+    }
+
+When you run your job, Samza will create several instances of your class (potentially on multiple machines). These task instances process the messages in the input streams.
+
+In your job's configuration you can tell Samza which streams you want to consume. An incomplete example could look like this (see the [configuration documentation](../jobs/configuration.html) for more detail):
+
+    # This is the class above, which Samza will instantiate when the job is run
+    task.class=com.example.samza.MyTaskClass
+
+    # Define a system called "kafka" (you can give it any name, and you can define
+    # multiple systems if you want to process messages from different sources)
+    systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory
+
+    # The job consumes a topic called "PageViewEvent" from the "kafka" system
+    task.inputs=kafka.PageViewEvent
+
+    # Define a serializer/deserializer called "json" which parses JSON messages
+    serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory
+
+    # Use the "json" serializer for messages in the "PageViewEvent" topic
+    systems.kafka.streams.PageViewEvent.samza.msg.serde=json
+
+For each message that Samza receives from the task's input streams, the *process* method is called. The [envelope](javadocs/org/apache/samza/system/IncomingMessageEnvelope.html) contains three things of importance: the message, the key, and the stream that the message came from.
+
+    /** Every message that is delivered to a StreamTask is wrapped
+     * in an IncomingMessageEnvelope, which contains metadata about
+     * the origin of the message. */
+    public class IncomingMessageEnvelope {
+      /** A deserialized message. */
+      Object getMessage() { ... }
+
+      /** A deserialized key. */
+      Object getKey() { ... }
+
+      /** The stream and partition that this message came from. */
+      SystemStreamPartition getSystemStreamPartition() { ... }
+    }
+
+The key and value are declared as Object, and need to be cast to the correct type. If you don't configure a [serializer/deserializer](../container/serialization.html), they are typically Java byte arrays. A deserializer can convert these bytes into any other type, for example the JSON deserializer mentioned above parses the byte array into java.util.Map, java.util.List and String objects.
+
+The getSystemStreamPartition() method returns a [SystemStreamPartition](javadocs/org/apache/samza/system/SystemStreamPartition.html) object, which tells you where the message came from. It consists of three parts:
+
+1. The *system*: the name of the system from which the message came, as defined in your job configuration. You can have multiple systems for input and/or output, each with a different name.
+2. The *stream name*: the name of the stream (topic, queue) within the source system. This is also defined in the job configuration.
+3. The [*partition*](javadocs/org/apache/samza/Partition.html): a stream is normally split into several partitions, and each partition is assigned to one StreamTask instance by Samza.
+
+The API looks like this:
+
+    /** A triple of system name, stream name and partition. */
+    public class SystemStreamPartition extends SystemStream {
+
+      /** The name of the system which provides this stream. It is
+          defined in the Samza job's configuration. */
+      public String getSystem() { ... }
+
+      /** The name of the stream/topic/queue within the system. */
+      public String getStream() { ... }
+
+      /** The partition within the stream. */
+      public Partition getPartition() { ... }
+    }
+
+In the example job configuration above, the system name is "kafka", the stream name is "PageViewEvent". (The name "kafka" isn't special &mdash; you can give your system any name you want.) If you have several input streams feeding into your StreamTask, you can use the SystemStreamPartition to determine what kind of message you've received.
+
+What about sending messages? If you take a look at the process() method in StreamTask, you'll see that you get a [MessageCollector](javadocs/org/apache/samza/task/MessageCollector.html).
+
+    /** When a task wishes to send a message, it uses this interface. */
+    public interface MessageCollector {
+      void send(OutgoingMessageEnvelope envelope);
+    }
+
+To send a message, you create an [OutgoingMessageEnvelope](javadocs/org/apache/samza/system/OutgoingMessageEnvelope.html) object and pass it to the message collector. At a minimum, the envelope specifies the message you want to send, and the system and stream name to send it to. Optionally you can specify the partitioning key and other parameters. See the [javadoc](javadocs/org/apache/samza/system/OutgoingMessageEnvelope.html) for details.
+
+**NOTE:** Please only use the MessageCollector object within the process() method. If you hold on to a MessageCollector instance and use it again later, your messages may not be sent correctly.
+
+For example, here's a simple task that splits each input message into words, and emits each word as a separate message:
+
+    public class SplitStringIntoWords implements StreamTask {
+
+      // Send outgoing messages to a stream called "words"
+      // in the "kafka" system.
+      private final SystemStream OUTPUT_STREAM =
+        new SystemStream("kafka", "words");
+
+      public void process(IncomingMessageEnvelope envelope,
+                          MessageCollector collector,
+                          TaskCoordinator coordinator) {
+        String message = (String) envelope.getMessage();
+
+        for (String word : message.split(" ")) {
+          // Use the word as the key, and 1 as the value.
+          // A second task can add the 1's to get the word count.
+          collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM, word, 1));
+        }
+      }
+    }
+
+## [SamzaContainer &raquo;](../container/samza-container.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/comparisons/introduction.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/comparisons/introduction.md b/docs/learn/documentation/0.7.0/comparisons/introduction.md
index 77c04ca..6ecffcf 100644
--- a/docs/learn/documentation/0.7.0/comparisons/introduction.md
+++ b/docs/learn/documentation/0.7.0/comparisons/introduction.md
@@ -43,7 +43,7 @@ example above, where you have a stream of page-view events including the ID of t
 
 Now you can write a Samza job that takes both the page-view event and the changelog as inputs. You make sure that they are partitioned on the same key (e.g. user ID). Every time a changelog event comes in, you write the updated user information to the task's local storage. Every time a page-view event comes in, you read the current information about that user from local storage. That way, you can keep all the state local to a task, and never need to query a remote database.
 
-![Stateful Processing](/img/0.7.0/learn/documentation/introduction/samza_state.png)
+<img src="/img/0.7.0/learn/documentation/introduction/samza_state.png" alt="Stateful Processing" class="diagram-large">
 
 In effect, you now have a replica of the main database, broken into small partitions that are on the same machines as the Samza tasks. Database writes still need to go to the main database, but when you need to read from the database in order to process a message from the input stream, you can just consult the task's local state.
 

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/comparisons/mupd8.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/comparisons/mupd8.md b/docs/learn/documentation/0.7.0/comparisons/mupd8.md
index 78e7b64..2cc8ee6 100644
--- a/docs/learn/documentation/0.7.0/comparisons/mupd8.md
+++ b/docs/learn/documentation/0.7.0/comparisons/mupd8.md
@@ -57,7 +57,7 @@ This was motivated by our experience with Hadoop, where the data flow between jo
 
 MUPD8 executes all of its map/update processors inside a single JVM, using threads. This is memory-efficient, as the JVM memory overhead is shared across the threads.
 
-Samza uses a separate JVM for each stream processor container ([TaskRunner](../container/task-runner.html)). This has the disadvantage of using more memory compared to running multiple stream processing threads within a single JVM. However, the advantage is improved isolation between tasks, which can make them more reliable.
+Samza uses a separate JVM for each [stream processor container](../container/samza-container.html). This has the disadvantage of using more memory compared to running multiple stream processing threads within a single JVM. However, the advantage is improved isolation between tasks, which can make them more reliable.
 
 ### Isolation
 

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/container/checkpointing.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/checkpointing.md b/docs/learn/documentation/0.7.0/container/checkpointing.md
index 42b2e8d..6a93d84 100644
--- a/docs/learn/documentation/0.7.0/container/checkpointing.md
+++ b/docs/learn/documentation/0.7.0/container/checkpointing.md
@@ -3,47 +3,79 @@ layout: page
 title: Checkpointing
 ---
 
-On the [Streams](streams.html) page, on important detail was glossed over. When a TaskRunner instantiates a SystemConsumer for an input stream/partition pair, how does the TaskRunner know where in the stream to start reading messages. If you recall, Kafka has the concept of an offset, which defines a specific location in a topic/partition pair. The idea is that an offset can be used to reference a specific point in a stream/partition pair. When you read messages from Kafka, you can supply an offset to specify at which point you'd like to read from. After you read, you increment your offset, and get the next message.
+Samza provides fault-tolerant processing of streams: Samza guarantees that messages won't be lost, even if your job crashes, if a machine dies, if there is a network fault, or something else goes wrong. In order to provide this guarantee, Samza expects the [input system](streams.html) to meet the following requirements:
 
-![diagram](/img/0.7.0/learn/documentation/container/checkpointing.png)
+* The stream may be sharded into one or more *partitions*. Each partition is independent from the others, and is replicated across multiple machines (the stream continues to be available, even if a machine fails).
+* Each partition consists of a sequence of messages in a fixed order. Each message has an *offset*, which indicates its position in that sequence. Messages are always consumed sequentially within each partition.
+* A Samza job can start consuming the sequence of messages from any starting offset.
 
-This diagram looks the same as on the [Streams](streams.html) page, except that there are black lines at different points in each input stream/partition pair. These lines represent the current offset for each stream consumer. As the stream consumer reads, the offset increases, and moves closer to the "head" of the stream. The diagram also illustrates that the offsets might be staggered, such that some offsets are farther along in their stream/partition than others.
+Kafka meets these requirements, but they can also be implemented with other message broker systems.
 
-If a SystemConsumer is reading messages for a TaskRunner, and the TaskRunner stops for some reason (due to hardware failure, re-deployment, or whatever), the SystemConsumer should start where it left off when the TaskRunner starts back up again. We're able to do this because the Kafka broker is buffering messages on a remote server (the broker). Since the messages are available when we come back, we can just start from our last offset, and continue moving forward, without losing data.
+As described in the [section on SamzaContainer](samza-container.html), each task instance of your job consumes one partition of an input stream. Each task has a *current offset* for each input stream: the offset of the next message to be read from that stream partition. Every time a message is read from the stream, the current offset moves forwards.
 
-The TaskRunner supports this ability using something called a CheckpointManager.
+If a Samza container fails, it needs to be restarted (potentially on another machine) and resume processing where the failed container left off. In order to enable this, a container periodically checkpoints the current offset for each task instance.
 
-```
-public interface CheckpointManager {
-  void start();
+<img src="/img/0.7.0/learn/documentation/container/checkpointing.svg" alt="Illustration of checkpointing" class="diagram-large">
 
-  void register(Partition partition);
+When a Samza container starts up, it looks for the most recent checkpoint and starts consuming messages from the checkpointed offsets. If the previous container failed unexpectedly, the most recent checkpoint may be slightly behind the current offsets (i.e. the job may have consumed some more messages since the last checkpoint was written), but we can't know for sure. In that case, the job may process a few messages again.
 
-  void writeCheckpoint(Partition partition, Checkpoint checkpoint);
+This guarantee is called *at-least-once processing*: Samza ensures that your job doesn't miss any messages, even if containers need to be restarted. However, it is possible for your job to see the same message more than once when a container is restarted. We are planning to address this in a future version of Samza, but for now it is just something to be aware of: for example, if you are counting page views, a forcefully killed container could cause events to be slightly over-counted. You can reduce duplication by checkpointing more frequently, at a slight performance cost.
 
-  Checkpoint readLastCheckpoint(Partition partition);
+For checkpoints to be effective, they need to be written somewhere where they will survive faults. Samza allows you to write checkpoints to the file system (using FileSystemCheckpointManager), but that doesn't help if the machine fails and the container needs to be restarted on another machine. The most common configuration is to use Kafka for checkpointing. You can enable this with the following job configuration:
 
-  void stop();
-}
+    # The name of your job determines the name under which checkpoints will be stored
+    job.name=example-job
 
-public class Checkpoint {
-  private final Map<SystemStream, String> offsets;
-  ...
-}
-```
+    # Define a system called "kafka" for consuming and producing to a Kafka cluster
+    systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory
 
-As you can see, the checkpoint manager provides a way to write out checkpoints for a given partition. Right now, the checkpoint contains a map. The map's keys are input stream names, and the map's values are each input stream's offset. Each checkpoint is managed per-partition. For example, if you have page-view-event and service-metric-event defined as streams in your Samza job's configuration file, the TaskRunner would supply a checkpoint with two keys in each checkpoint offset map (one for page-view-event and the other for service-metric-event).
+    # Declare that we want our job's checkpoints to be written to Kafka
+    task.checkpoint.factory=org.apache.samza.checkpoint.kafka.KafkaCheckpointManagerFactory
+    task.checkpoint.system=kafka
 
-Samza provides two checkpoint managers: FileSystemCheckpointManager and KafkaCheckpointManager. The KafkaCheckpointManager is what you generally want to use. The way that KafkaCheckpointManager works is as follows: it writes checkpoint messages for your Samza job to a special Kafka topic. This topic's name is \_\_samza\_checkpoint\_your-job-name. For example, if you had a Samza job called "my-first-job", the Kafka topic would be called \_\_samza\_checkpoint\_my-first-job. This Kafka topic is partitioned identically to your Samza job's partition count. If your Samza job has 10 partitions, the checkpoint topic for your Samza job will also have 10 partitions. Every time that the TaskRunner calls writeCheckpoint, a checkpoint message will be sent to the partition that corresponds with the partition for the checkpoint that the TaskRunner wishes to write.
+    # By default, a checkpoint is written every 60 seconds. You can change this if you like.
+    task.commit.ms=60000
 
-![diagram](/img/0.7.0/learn/documentation/container/checkpointing-2.png)
+In this configuration, Samza writes checkpoints to a separate Kafka topic called \_\_samza\_checkpoint\_&lt;job-name&gt;\_&lt;job-id&gt; (in the example configuration above, the topic would be called \_\_samza\_checkpoint\_example-job\_1). Once per minute, Samza automatically sends a message to this topic, in which the current offsets of the input streams are encoded. When a Samza container starts up, it looks for the most recent offset message in this topic, and loads that checkpoint.
 
-When the TaskRunner starts for the first time, the offset behavior of the SystemConsumers is undefined. If the system for the SystemConsumer is Kafka, we fall back to the auto.offset.reset setting. If the auto.offset.reset is set to "largest", we start reading messages from the head of the stream; if it's set to "smallest", we read from the tail. If it's undefined, the TaskRunner will fail.
+Sometimes it can be useful to use checkpoints only for some input streams, but not for others. In this case, you can tell Samza to ignore any checkpointed offsets for a particular stream name:
 
-The TaskRunner calls writeCheckpoint at a windowed interval (e.g. every 10 seconds). If the TaskRunner fails, and restarts, it simply calls readLastCheckpoint for each partition. In the case of the KafkaCheckpointManager, this readLastCheckpoint method will read the last message that was written to the checkpoint topic for each partition in the job. One edge case to consider is that SystemConsumers might have read messages from an offset that hasn't yet been checkpointed. In such a case, when the TaskRunner reads the last checkpoint for each partition, the offsets might be farther back in the stream. When this happens, your StreamTask could get duplicate messages (i.e. it saw message X, failed, restarted at an offset prior to message X, and then reads message X again). Thus, Samza currently provides at least once messaging. You might get duplicates. Caveat emptor.
+    # Ignore any checkpoints for the topic "my-special-topic"
+    systems.kafka.streams.my-special-topic.samza.reset.offset=true
 
-<!-- TODO Add a link to the fault tolerance SEP when one exists -->
+    # Always start consuming "my-special-topic" at the oldest available offset
+    systems.kafka.streams.my-special-topic.samza.offset.default=oldest
 
-*Note that there are design proposals in the works to give exactly once messaging.*
+The following table explains the meaning of these configuration parameters:
+
+<table class="documentation">
+  <tr>
+    <th>Parameter name</th>
+    <th>Value</th>
+    <th>Meaning</th>
+  </tr>
+  <tr>
+    <td rowspan="2" class="nowrap">systems.&lt;system&gt;.<br>streams.&lt;stream&gt;.<br>samza.reset.offset</td>
+    <td>false (default)</td>
+    <td>When container starts up, resume processing from last checkpoint</td>
+  </tr>
+  <tr>
+    <td>true</td>
+    <td>Ignore checkpoint (pretend that no checkpoint is present)</td>
+  </tr>
+  <tr>
+    <td rowspan="2" class="nowrap">systems.&lt;system&gt;.<br>streams.&lt;stream&gt;.<br>samza.offset.default</td>
+    <td>upcoming (default)</td>
+    <td>When container starts and there is no checkpoint (or the checkpoint is ignored), only process messages that are published after the job is started, but no old messages</td>
+  </tr>
+  <tr>
+    <td>oldest</td>
+    <td>When container starts and there is no checkpoint (or the checkpoint is ignored), jump back to the oldest available message in the system, and consume all messages from that point onwards (most likely this means repeated processing of messages already seen previously)</td>
+  </tr>
+</table>
+
+Note that the example configuration above causes your tasks to start consuming from the oldest offset *every time a container starts up*. This is useful in case you have some in-memory state in your tasks that you need to rebuild from source data in an input stream. If you are using streams in this way, you may also find [bootstrap streams](streams.html) useful.
+
+If you want to make a one-off change to a job's consumer offsets, for example to force old messages to be processed again with a new version of your code, you can use CheckpointTool to manipulate the job's checkpoint. The tool is included in Samza's [source repository](/contribute/code.html) and documented in the README.
 
 ## [State Management &raquo;](state-management.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/container/event-loop.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/event-loop.md b/docs/learn/documentation/0.7.0/container/event-loop.md
index 1f9c51e..903ef90 100644
--- a/docs/learn/documentation/0.7.0/container/event-loop.md
+++ b/docs/learn/documentation/0.7.0/container/event-loop.md
@@ -3,87 +3,40 @@ layout: page
 title: Event Loop
 ---
 
-The event loop is the [TaskRunner](task-runner.html)'s single thread that is in charge of [reading](streams.html), [writing](streams.html), [metrics flushing](metrics.html), [checkpointing](checkpointing.html), and [windowing](windowing.html). It's the code that puts all of this stuff together. Each SystemConsumer reads messages on its own thread, but writes messages into a centralized message queue. The TaskRunner uses this queue to funnel all of the messages into the event loop. Here's how the event loop works:
+The event loop is the [container](samza-container.html)'s single thread that is in charge of [reading and writing messages](streams.html), [flushing metrics](metrics.html), [checkpointing](checkpointing.html), and [windowing](windowing.html).
 
-1. Take a message from the incoming message queue (the queue that the SystemConsumers are putting their messages)
-2. Give the message to the appropriate StreamTask by calling process() on it
-3. Call window() on the StreamTask if it implements WindowableTask, and the window time has expired
-4. Send any StreamTask output from the process() and window() call to the appropriate SystemProducers
-5. Write checkpoints for any partitions that are past the defined checkpoint commit interval
+Samza uses a single thread because every container is designed to use a single CPU core; to get more parallelism, simply run more containers. This uses a bit more memory than multithreaded parallelism, because each JVM has some overhead, but it simplifies resource management and improves isolation between jobs. This helps Samza jobs run reliably on a multitenant cluster, where many different jobs written by different people are running at the same time.
 
-The TaskRunner does this, in a loop, until it is shutdown.
+You are strongly discouraged from using threads in your job's code. Samza uses multiple threads internally for communicating with input and output streams, but all message processing and user code runs on a single-threaded event loop. In general, Samza is not thread-safe.
 
-### Lifecycle Listeners
-
-Sometimes, it's useful to receive notifications when a specific event happens in the TaskRunner. For example, you might want to reset some context in the container whenever a new message arrives. To accomplish this, Samza provides a TaskLifecycleListener interface, that can be wired into the TaskRunner through configuration.
-
-```
-/**
- * Used to get before/after notifications before initializing/closing all tasks
- * in a given container (JVM/process).
- */
-public interface TaskLifecycleListener {
-  /**
-   * Called before all tasks in TaskRunner are initialized.
-   */
-  void beforeInit(Config config, TaskContext context);
-
-  /**
-   * Called after all tasks in TaskRunner are initialized.
-   */
-  void afterInit(Config config, TaskContext context);
-
-  /**
-   * Called before a message is processed by a task.
-   */
-  void beforeProcess(IncomingMessageEnvelope envelope, Config config, TaskContext context);
+### Event Loop Internals
 
-  /**
-   * Called after a message is processed by a task.
-   */
-  void afterProcess(IncomingMessageEnvelope envelope, Config config, TaskContext context);
+A container may have multiple [SystemConsumers](../api/javadocs/org/apache/samza/system/SystemConsumer.html) for consuming messages from different input systems. Each SystemConsumer reads messages on its own thread, but writes messages into a shared in-process message queue. The container uses this queue to funnel all of the messages into the event loop.
 
-  /**
-   * Called before all tasks in TaskRunner are closed.
-   */
-  void beforeClose(Config config, TaskContext context);
+The event loop works as follows:
 
-  /**
-   * Called after all tasks in TaskRunner are closed.
-   */
-  void afterClose(Config config, TaskContext context);
-}
-```
+1. Take a message from the incoming message queue;
+2. Give the message to the appropriate [task instance](samza-container.html) by calling process() on it;
+3. Call window() on the task instance if it implements [WindowableTask](../api/javadocs/org/apache/samza/task/WindowableTask.html), and the window time has expired;
+4. Send any output from the process() and window() calls to the appropriate [SystemProducers](../api/javadocs/org/apache/samza/system/SystemProducer.html);
+5. Write checkpoints for any tasks whose [commit interval](checkpointing.html) has elapsed.
 
-To use a TaskLifecycleListener, you must also create a factory for the listener.
+The container does this, in a loop, until it is shut down. Note that although there can be multiple task instances within a container (depending on the number of input stream partitions), their process() and window() methods are all called on the same thread, never concurrently on different threads.
 
-```
-public interface TaskLifecycleListenerFactory {
-  TaskLifecycleListener getLifecyleListener(String name, Config config);
-}
-```
-
-#### Configuring Lifecycle Listeners
-
-Once you have written a TaskLifecycleListener, and its factory, you can use the listener by configuring your Samza job with the following keys:
-
-* task.lifecycle.listeners: A CSV list of all defined listeners that should be used for the Samza job.
-* task.lifecycle.listener.&lt;listener name&gt;.class: A Java package and class name for a single listener factory.
+### Lifecycle Listeners
 
-For example, you might define a listener called "my-listener":
+Sometimes, you need to run your own code at specific points in a task's lifecycle. For example, you might want to set up some context in the container whenever a new message arrives, or perform some operations on startup or shutdown.
 
-```
-task.lifecycle.listener.my-listener.class=com.foo.bar.MyListenerFactory
-```
+To receive notifications when such events happen, you can implement the [TaskLifecycleListenerFactory](../api/javadocs/org/apache/samza/task/TaskLifecycleListenerFactory.html) interface. It returns a [TaskLifecycleListener](../api/javadocs/org/apache/samza/task/TaskLifecycleListener.html), whose methods are called by Samza at the appropriate times.
 
-And then enable it for your Samza job:
+You can then tell Samza to use your lifecycle listener with the following properties in your job configuration:
 
-```
-task.lifecycle.listeners=my-listener
-```
+    # Define a listener called "my-listener" by giving the factory class name
+    task.lifecycle.listener.my-listener.class=com.example.foo.MyListenerFactory
 
-Samza's container will create one instance of TaskLifecycleListener, and notify it whenever any of the events (shown in the API above) occur.
+    # Enable it in this job (multiple listeners can be separated by commas)
+    task.lifecycle.listeners=my-listener
 
-Borrowing from the example above, if we have a single Samza container processing partitions 0 and 2, and have defined a lifecycle listener called my-listener, then the Samza container will have a single instance of MyListener. MyListener's beforeInit, afterInit, beforeClose, and afterClose methods will all be called twice: one for each of the two partitions (e.g. beforeInit partition 0, before init partition 1, etc). The beforeProcess and afterProcess methods will simply be called once for each incoming message. The TaskContext is how the TaskLifecycleListener is able to tell which partition the event is for.
+The Samza container creates one instance of your [TaskLifecycleListener](../api/javadocs/org/apache/samza/task/TaskLifecycleListener.html). If the container has multiple task instances (processing different input stream partitions), the beforeInit, afterInit, beforeClose and afterClose methods are called for each task instance. The [TaskContext](../api/javadocs/org/apache/samza/task/TaskContext.html) argument of those methods gives you more information about the partitions.
 
 ## [JMX &raquo;](jmx.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/container/index.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/index.md b/docs/learn/documentation/0.7.0/container/index.md
deleted file mode 100644
index 17751de..0000000
--- a/docs/learn/documentation/0.7.0/container/index.md
+++ /dev/null
@@ -1,18 +0,0 @@
----
-layout: page
-title: Container
----
-
-The API section shows how a Samza StreamTask is written. To execute a StreamTask, Samza has a container that wraps around your StreamTask. The Samza container manages:
-
-* Metrics
-* Configuration
-* Lifecycle
-* Checkpointing
-* State management
-* Serialization
-* Data transport
-
-This container is called a TaskRunner. Read on to learn more about Samza's TaskRunner.
-
-## [JobRunner &raquo;](job-runner.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/container/jmx.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/jmx.md b/docs/learn/documentation/0.7.0/container/jmx.md
index a9fcc77..9fce867 100644
--- a/docs/learn/documentation/0.7.0/container/jmx.md
+++ b/docs/learn/documentation/0.7.0/container/jmx.md
@@ -3,11 +3,20 @@ layout: page
 title: JMX
 ---
 
-The Samza TaskRunner (and YARN Application Master) will turn on JMX using a randomly selected port, since Samza is meant to be run in a distributed environment, and it's unknown which ports will be available prior to runtime. The port will be output in the TaskRunner's logs with a line like this:
+Samza's containers and YARN ApplicationMaster enable [JMX](http://docs.oracle.com/javase/tutorial/jmx/) by default. JMX can be used for managing the JVM; for example, you can connect to it using [jconsole](http://docs.oracle.com/javase/7/docs/technotes/guides/management/jconsole.html), which is included in the JDK.
 
-    2013-07-05 20:42:36 JmxServer [INFO] According to InetAddress.getLocalHost.getHostName we are Chriss-MacBook-Pro.local
-    2013-07-05 20:42:36 JmxServer [INFO] Started JmxServer port=64905 url=service:jmx:rmi:///jndi/rmi://Chriss-MacBook-Pro.local:64905/jmxrmi
+You can tell Samza to publish its internal [metrics](metrics.html), and any custom metrics you define, as JMX MBeans. To enable this, set the following properties in your job configuration:
 
-Any metrics that are registered in the TaskRunner will be visible through JMX. To toggle JMX, see the [Configuration](../jobs/configuration.html) section.
+    # Define a Samza metrics reporter called "jmx", which publishes to JMX
+    metrics.reporter.jmx.class=org.apache.samza.metrics.reporter.JmxReporterFactory
+
+    # Use it (if you have multiple reporters defined, separate them with commas)
+    metrics.reporters=jmx
+
+JMX needs to be configured to use a specific port, but in a distributed environment, there is no way of knowing in advance which ports are available on the machines running your containers. Therefore Samza chooses the JMX port randomly. If you need to connect to it, you can find the port by looking in the container's logs, which report the JMX server details as follows:
+
+    2014-06-02 21:50:17 JmxServer [INFO] According to InetAddress.getLocalHost.getHostName we are samza-grid-1234.example.com
+    2014-06-02 21:50:17 JmxServer [INFO] Started JmxServer registry port=50214 server port=50215 url=service:jmx:rmi://localhost:50215/jndi/rmi://localhost:50214/jmxrmi
+    2014-06-02 21:50:17 JmxServer [INFO] If you are tunneling, you might want to try JmxServer registry port=50214 server port=50215 url=service:jmx:rmi://samza-grid-1234.example.com:50215/jndi/rmi://samza-grid-1234.example.com:50214/jmxrmi
 
 ## [JobRunner &raquo;](../jobs/job-runner.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/container/metrics.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/metrics.md b/docs/learn/documentation/0.7.0/container/metrics.md
index 078ce47..98acd81 100644
--- a/docs/learn/documentation/0.7.0/container/metrics.md
+++ b/docs/learn/documentation/0.7.0/container/metrics.md
@@ -3,52 +3,78 @@ layout: page
 title: Metrics
 ---
 
-Samza also provides a metrics library that the TaskRunner uses. It allows a StreamTask to create counters and gauges. The TaskRunner then writes those metrics to metrics infrastructure through a MetricsReporter implementation.
-
-```
-public class MyJavaStreamerTask implements StreamTask, InitableTask {
-  private static final Counter messageCount;
-
-  public void init(Config config, TaskContext context) {
-    this.messageCount = context.getMetricsRegistry().newCounter(MyJavaStreamerTask.class.toString(), "MessageCount");
-  }
-
-  @Override
-  public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) {
-    System.out.println(envelope.getMessage().toString());
-    messageCount.inc();
-  }
-}
-```
-
-Samza's metrics design is very similar to Coda Hale's [metrics](https://github.com/codahale/metrics) library. It has two important interfaces:
-
-```
-public interface MetricsRegistry {
-  Counter newCounter(String group, String name);
-
-  <T> Gauge<T> newGauge(String group, String name, T value);
-}
-
-public interface MetricsReporter {
-  void start();
-
-  void register(String source, ReadableMetricsRegistry registry);
-
-  void stop();
-}
-```
-
-### MetricsRegistry
-
-When the TaskRunner starts up, as with StreamTask instantiation, it creates a MetricsRegistry for every partition in the Samza job.
-
-![diagram](/img/0.7.0/learn/documentation/container/metrics.png)
-
-The TaskRunner, itself, also gets a MetricsRegistry that it can use to create counters and gauges. It uses this registry to measure a lot of relevant metrics for itself.
-
-### MetricsReporter
-
-The other important interface is the MetricsReporter. The TaskRunner uses MetricsReporter implementations to send its MetricsRegistry counters and gauges to whatever metrics infrastructure the reporter uses. A Samza job's configuration determines which MetricsReporters the TaskRunner will use. Out of the box, Samza comes with a MetricsSnapshotReporter that sends JSON metrics messages to a Kafka topic, and a JmxReporter that records metrics to be read via JMX.
+When you're running a stream process in production, it's important that you have good metrics to track the health of your job. In order to make this easy, Samza includes a metrics library. It is used by Samza itself to generate some standard metrics such as message throughput, but you can also use it in your task code to emit custom metrics.
+
+Metrics can be reported in various ways. You can expose them via [JMX](jmx.html), which is useful in development. In production, a common setup is for each Samza container to periodically publish its metrics to a "metrics" Kafka topic, in which the metrics from all Samza jobs are aggregated. You can then consume this stream in another Samza job, and send the metrics to your favorite graphing system such as [Graphite](http://graphite.wikidot.com/).
+
+To set up your job to publish metrics to Kafka, you can use the following configuration:
+
+    # Define a metrics reporter called "snapshot", which publishes metrics
+    # every 60 seconds.
+    metrics.reporters=snapshot
+    metrics.reporter.snapshot.class=org.apache.samza.metrics.reporter.MetricsSnapshotReporterFactory
+
+    # Tell the snapshot reporter to publish to a topic called "metrics"
+    # in the "kafka" system.
+    metrics.reporter.snapshot.stream=kafka.metrics
+
+    # Encode metrics data as JSON.
+    serializers.registry.metrics.class=org.apache.samza.serializers.MetricsSnapshotSerdeFactory
+    systems.kafka.streams.metrics.samza.msg.serde=metrics
+
+With this configuration, the job automatically sends several JSON-encoded messages to the "metrics" topic in Kafka every 60 seconds. The messages look something like this:
+
+    {
+      "header": {
+        "container-name": "samza-container-0",
+        "host": "samza-grid-1234.example.com",
+        "job-id": "1",
+        "job-name": "my-samza-job",
+        "reset-time": 1401729000347,
+        "samza-version": "0.0.1",
+        "source": "Partition-2",
+        "time": 1401729420566,
+        "version": "0.0.1"
+      },
+      "metrics": {
+        "org.apache.samza.container.TaskInstanceMetrics": {
+          "commit-calls": 7,
+          "commit-skipped": 77948,
+          "kafka-input-topic-offset": "1606",
+          "messages-sent": 985,
+          "process-calls": 1093,
+          "send-calls": 985,
+          "send-skipped": 76970,
+          "window-calls": 0,
+          "window-skipped": 77955
+        }
+      }
+    }
+
+There is a separate message for each task instance, and the header tells you the job name, job ID and partition of the task. The metrics allow you to see how many messages have been processed and sent, the current offset in the input stream partition, and other details. There are additional messages which give you metrics about the JVM (heap size, garbage collection information, threads etc.), internal metrics of the Kafka producers and consumers, and more.
+
+It's easy to generate custom metrics in your job, if there's some value you want to keep an eye on. You can use Samza's built-in metrics framework, which is similar in design to Coda Hale's [metrics](http://metrics.codahale.com/) library. 
+
+You can register your custom metrics through a [MetricsRegistry](../api/javadocs/org/apache/samza/metrics/MetricsRegistry.html). Your stream task needs to implement [InitableTask](../api/javadocs/org/apache/samza/task/InitableTask.html), so that you can get the metrics registry from the [TaskContext](../api/javadocs/org/apache/samza/task/TaskContext.html). This simple example shows how to count the number of messages processed by your task:
+
+    public class MyJavaStreamTask implements StreamTask, InitableTask {
+      private Counter messageCount;
+
+      public void init(Config config, TaskContext context) {
+        this.messageCount = context
+          .getMetricsRegistry()
+          .newCounter(getClass().getName(), "message-count");
+      }
+
+      public void process(IncomingMessageEnvelope envelope,
+                          MessageCollector collector,
+                          TaskCoordinator coordinator) {
+        messageCount.inc();
+      }
+    }
+
+Samza currently supports two kind of metrics: [counters](../api/javadocs/org/apache/samza/metrics/Counter.html) and [gauges](../api/javadocs/org/apache/samza/metrics/Gauge.html). Use a counter when you want to track how often something occurs, and a gauge when you want to report the level of something, such as the size of a buffer. Each task instance (for each input stream partition) gets its own set of metrics.
+
+If you want to report metrics in some other way, e.g. directly to a graphing system (without going via Kafka), you can implement a [MetricsReporterFactory](../api/javadocs/org/apache/samza/metrics/MetricsReporterFactory.html) and reference it in your job configuration.
 
 ## [Windowing &raquo;](windowing.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/container/samza-container.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/samza-container.md b/docs/learn/documentation/0.7.0/container/samza-container.md
new file mode 100644
index 0000000..5d259c4
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/container/samza-container.md
@@ -0,0 +1,66 @@
+---
+layout: page
+title: SamzaContainer
+---
+
+The SamzaContainer is responsible for managing the startup, execution, and shutdown of one or more [StreamTask](../api/overview.html) instances. Each SamzaContainer typically runs as an indepentent Java virtual machine. A Samza job can consist of several SamzaContainers, potentially running on different machines.
+
+When a SamzaContainer starts up, it does the following:
+
+1. Get last checkpointed offset for each input stream partition that it consumes
+2. Create a "reader" thread for every input stream partition that it consumes
+3. Start metrics reporters to report metrics
+4. Start a checkpoint timer to save your task's input stream offsets every so often
+5. Start a window timer to trigger your task's [window method](../api/javadocs/org/apache/samza/task/WindowableTask.html), if it is defined
+6. Instantiate and initialize your StreamTask once for each input stream partition
+7. Start an event loop that takes messages from the input stream reader threads, and gives them to your StreamTasks
+8. Notify lifecycle listeners during each one of these steps
+
+Let's start in the middle, with the instantiation of a StreamTask. The following sections of the documentation cover the other steps.
+
+### Tasks and Partitions
+
+When the container starts, it creates instances of the [task class](../api/overview.html) that you've written. If the task class implements the [InitableTask](../api/javadocs/org/apache/samza/task/InitableTask.html) interface, the SamzaContainer will also call the init() method.
+
+    /** Implement this if you want a callback when your task starts up. */
+    public interface InitableTask {
+      void init(Config config, TaskContext context);
+    }
+
+How many instances of your task class are created depends on the number of partitions in the job's input streams. If your Samza job has ten partitions, there will be ten instantiations of your task class: one for each partition. The first task instance will receive all messages for partition one, the second instance will receive all messages for partition two, and so on.
+
+<img src="/img/0.7.0/learn/documentation/container/tasks-and-partitions.svg" alt="Illustration of tasks consuming partitions" class="diagram-large">
+
+The number of partitions in the input streams is determined by the systems from which you are consuming. For example, if your input system is Kafka, you can specify the number of partitions when you create a topic.
+
+If a Samza job has more than one input stream, the number of task instances for the Samza job is the maximum number of partitions across all input streams. For example, if a Samza job is reading from PageViewEvent (12 partitions), and ServiceMetricEvent (14 partitions), then the Samza job would have 14 task instances (numbered 0 through 13). Task instances 12 and 13 only receive events from ServiceMetricEvent, because there is no corresponding PageViewEvent partition.
+
+There is [work underway](https://issues.apache.org/jira/browse/SAMZA-71) to make the assignment of partitions to tasks more flexible in future versions of Samza.
+
+### Containers and resource allocation
+
+Although the number of task instances is fixed &mdash; determined by the number of input partitions &mdash; you can configure how many containers you want to use for your job. If you are [using YARN](../jobs/yarn-jobs.html), the number of containers determines what CPU and memory resources are allocated to your job.
+
+If the data volume on your input streams is small, it might be sufficient to use just one SamzaContainer. In that case, Samza still creates one task instance per input partition, but all those tasks run within the same container. At the other extreme, you can create as many containers as you have partitions, and Samza will assign one task instance to each container.
+
+Each SamzaContainer is designed to use one CPU core, so it uses a [single-threaded event loop](event-loop.html) for execution. It's not advisable to create your own threads within a SamzaContainer. If you need more parallelism, please configure your job to use more containers.
+
+Any [state](state-management.html) in your job belongs to a task instance, not to a container. This is a key design decision for Samza's scalability: as your job's resource requirements grow and shrink, you can simply increase or decrease the number of containers, but the number of task instances remains unchanged. As you scale up or down, the same state remains attached to each task instance. Task instances may be moved from one container to another, and any persistent state managed by Samza will be moved with it. This allows the job's processing semantics to remain unchanged, even as you change the job's parallelism.
+
+### Joining multiple input streams
+
+If your job has multiple input streams, Samza provides a simple but powerful mechanism for joining data from different streams: each task instance receives messages from one partition of *each* of the input streams. For example, say you have two input streams, A and B, each with four partitions. Samza creates four task instances to process them, and assigns the partitions as follows:
+
+<table class="documentation">
+<tr><th>Task instance</th><th>Consumes stream partitions</th></tr>
+<tr><td>0</td><td>stream A partition 0, stream B partition 0</td></tr>
+<tr><td>1</td><td>stream A partition 1, stream B partition 1</td></tr>
+<tr><td>2</td><td>stream A partition 2, stream B partition 2</td></tr>
+<tr><td>3</td><td>stream A partition 3, stream B partition 3</td></tr>
+</table>
+
+Thus, if you want two events in different streams to be processed by the same task instance, you need to ensure they are sent to the same partition number. You can achieve this by using the same partitioning key when [sending the messages](../api/overview.html). Joining streams is discussed in detail in the [state management](state-management.html) section.
+
+There is one caveat in all of this: Samza currently assumes that a stream's partition count will never change. Partition splitting or repartitioning is not supported. If an input stream has N partitions, it is expected that it has always had, and will always have N partitions. If you want to re-partition a stream, you can write a job that reads messages from the stream, and writes them out to a new stream with the required number of partitions. For example, you could read messages from PageViewEvent, and write them to PageViewEventRepartition.
+
+## [Streams &raquo;](streams.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/container/serialization.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/serialization.md b/docs/learn/documentation/0.7.0/container/serialization.md
new file mode 100644
index 0000000..f6570c9
--- /dev/null
+++ b/docs/learn/documentation/0.7.0/container/serialization.md
@@ -0,0 +1,46 @@
+---
+layout: page
+title: Serialization
+---
+
+Every message that is read from or written to a [stream](streams.html) or a [persistent state store](state-management.html) needs to eventually be serialized to bytes (which are sent over the network or written to disk). There are various places where that serialization and deserialization can happen:
+
+1. In the client library: for example, the library for publishing to Kafka and consuming from Kafka supports pluggable serialization.
+2. In the task implementation: your [process method](../api/overview.html) can use raw byte arrays as inputs and outputs, and do any parsing and serialization itself.
+3. Between the two: Samza provides a layer of serializers and deserializers, or *serdes* for short.
+
+You can use whatever makes sense for your job; Samza doesn't impose any particular data model or serialization scheme on you. However, the cleanest solution is usually to use Samza's serde layer. The following configuration example shows how to use it.
+
+    # Define a system called "kafka"
+    systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory
+
+    # The job is going to consume a topic called "PageViewEvent" from the "kafka" system
+    task.inputs=kafka.PageViewEvent
+
+    # Define a serde called "json" which parses/serializes JSON objects
+    serializers.registry.json.class=org.apache.samza.serializers.JsonSerdeFactory
+
+    # Define a serde called "integer" which encodes an integer as 4 binary bytes (big-endian)
+    serializers.registry.integer.class=org.apache.samza.serializers.IntegerSerdeFactory
+
+    # For messages in the "PageViewEvent" topic, the key (the ID of the user viewing the page)
+    # is encoded as a binary integer, and the message is encoded as JSON.
+    systems.kafka.streams.PageViewEvent.samza.key.serde=integer
+    systems.kafka.streams.PageViewEvent.samza.msg.serde=json
+
+    # Define a key-value store which stores the most recent page view for each user ID.
+    # Again, the key is an integer user ID, and the value is JSON.
+    stores.LastPageViewPerUser.factory=org.apache.samza.storage.kv.KeyValueStorageEngineFactory
+    stores.LastPageViewPerUser.changelog=kafka.last-page-view-per-user
+    stores.LastPageViewPerUser.key.serde=integer
+    stores.LastPageViewPerUser.msg.serde=json
+
+Each serde is defined with a factory class. Samza comes with several builtin serdes for UTF-8 strings, binary-encoded integers, JSON (requires the samza-serializers dependency) and more. You can also create your own serializer by implementing the [SerdeFactory](../api/javadocs/org/apache/samza/serializers/SerdeFactory.html) interface.
+
+The name you give to a serde (such as "json" and "integer" in the example above) is only for convenience in your job configuration; you can choose whatever name you like. For each stream and each state store, you can use the serde name to declare how messages should be serialized and deserialized.
+
+If you don't declare a serde, Samza simply passes objects through between your task instance and the system stream. In that case your task needs to send and receive whatever type of object the underlying client library uses.
+
+All the Samza APIs for sending and receiving messages are typed as *Object*. This means that you have to cast messages to the correct type before you can use them. It's a little bit more code, but it has the advantage that Samza is not restricted to any particular data model.
+
+## [Checkpointing &raquo;](checkpointing.html)


[6/7] SAMZA-7: Rewrite Container section of docs to bring it up-to-date. Reviewed by Jakob Homan.

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/img/0.7.0/learn/documentation/container/checkpointing.svg
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/container/checkpointing.svg b/docs/img/0.7.0/learn/documentation/container/checkpointing.svg
new file mode 100644
index 0000000..676cd7c
--- /dev/null
+++ b/docs/img/0.7.0/learn/documentation/container/checkpointing.svg
@@ -0,0 +1,4 @@
+<?xml version="1.0" standalone="yes"?>
+
+<svg version="1.1" viewBox="0.0 0.0 643.0 236.0" fill="none" stroke="none" stroke-linecap="square" stroke-miterlimit="10" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink"><clipPath id="p.0"><path d="m0 0l643.0 0l0 236.0l-643.0 0l0 -236.0z" clip-rule="nonzero"></path></clipPath><g clip-path="url(#p.0)"><path fill="#000000" fill-opacity="0.0" d="m0 0l643.0525 0l0 236.86089l-643.0525 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m47.771652 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m47.771652 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m61.90129 57.33281l-1.140625 0l0 -7.28125q-0.421875 0.390625 -1.09375 0.796875q-0.65625 0.390625 -1.1875 0.578125l0 -1.109375q0.953125 -0.4375 1.671875 -1.078125q0.71875 -0.640625 1.015625 -1.25l0.734375 0l0 9.34375z" fil
 l-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m73.755905 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m73.755905 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m89.58867 56.23906l0 1.09375l-6.15625 0q-0.015625 -0.40625 0.140625 -0.796875q0.234375 -0.625 0.75 -1.234375q0.515625 -0.609375 1.5 -1.40625q1.515625 -1.25 2.046875 -1.96875q0.53125 -0.734375 0.53125 -1.375q0 -0.6875 -0.484375 -1.140625q-0.484375 -0.46875 -1.265625 -0.46875q-0.828125 0 -1.328125 0.5q-0.484375 0.484375 -0.5 1.359375l-1.171875 -0.125q0.125 -1.3125 0.90625 -2.0q0.78125 -0.6875 2.109375 -0.6875q1.34375 0 2.125 0.75q0.78125 0.734375 0.78125 1.828125q0 0.5625 -0.234375 1.109375q-0.21875 0.53125 -0.75 1.140625q-0.53125 0.59375 -1.765625 1.625q-1.03125 0.859375 -1.328125 1.171875q-0.28125 0.3125 -0.46875 0.625l4.5625 0z" fil
 l-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m99.74016 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m99.74016 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m109.57292 54.879684l1.140625 -0.15625q0.203125 0.96875 0.671875 1.40625q0.46875 0.421875 1.15625 0.421875q0.796875 0 1.34375 -0.546875q0.5625 -0.5625 0.5625 -1.390625q0 -0.796875 -0.515625 -1.296875q-0.5 -0.515625 -1.296875 -0.515625q-0.328125 0 -0.8125 0.125l0.125 -1.0q0.125 0.015625 0.1875 0.015625q0.734375 0 1.3125 -0.375q0.59375 -0.390625 0.59375 -1.1875q0 -0.625 -0.4375 -1.03125q-0.421875 -0.421875 -1.09375 -0.421875q-0.671875 0 -1.109375 0.421875q-0.4375 0.421875 -0.578125 1.25l-1.140625 -0.203125q0.21875 -1.140625 0.953125 -1.765625q0.75 -0.640625 1.84375 -0.640625q0.765625 0 1.40625 0.328125q0.640625 0.328125 0.984375 0.890625
 q0.34375 0.5625 0.34375 1.203125q0 0.59375 -0.328125 1.09375q-0.328125 0.5 -0.953125 0.78125q0.8125 0.203125 1.265625 0.796875q0.46875 0.59375 0.46875 1.5q0 1.21875 -0.890625 2.078125q-0.890625 0.84375 -2.25 0.84375q-1.21875 0 -2.03125 -0.734375q-0.8125 -0.734375 -0.921875 -1.890625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m125.72441 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m125.72441 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m139.21342 57.33281l0 -2.234375l-4.03125 0l0 -1.046875l4.234375 -6.03125l0.9375 0l0 6.03125l1.265625 0l0 1.046875l-1.265625 0l0 2.234375l-1.140625 0zm0 -3.28125l0 -4.1875l-2.921875 4.1875l2.921875 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m151.70866 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"><
 /path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m151.70866 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m161.54143 54.89531l1.1875 -0.109375q0.140625 0.890625 0.625 1.328125q0.484375 0.4375 1.171875 0.4375q0.828125 0 1.390625 -0.625q0.578125 -0.625 0.578125 -1.640625q0 -0.984375 -0.546875 -1.546875q-0.546875 -0.5625 -1.4375 -0.5625q-0.5625 0 -1.015625 0.25q-0.4375 0.25 -0.6875 0.640625l-1.0625 -0.140625l0.890625 -4.765625l4.625 0l0 1.078125l-3.703125 0l-0.5 2.5q0.828125 -0.578125 1.75 -0.578125q1.21875 0 2.046875 0.84375q0.84375 0.84375 0.84375 2.171875q0 1.265625 -0.734375 2.1875q-0.890625 1.125 -2.4375 1.125q-1.265625 0 -2.078125 -0.703125q-0.796875 -0.71875 -0.90625 -1.890625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m177.69292 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stro
 ke-linejoin="round" stroke-linecap="butt" d="m177.69292 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m193.44756 50.30156l-1.140625 0.09375q-0.140625 -0.671875 -0.421875 -0.984375q-0.46875 -0.484375 -1.140625 -0.484375q-0.546875 0 -0.96875 0.3125q-0.53125 0.390625 -0.84375 1.140625q-0.3125 0.75 -0.328125 2.15625q0.40625 -0.625 1.0 -0.921875q0.609375 -0.3125 1.265625 -0.3125q1.140625 0 1.9375 0.84375q0.8125 0.828125 0.8125 2.171875q0 0.875 -0.390625 1.625q-0.375 0.75 -1.03125 1.15625q-0.65625 0.390625 -1.5 0.390625q-1.421875 0 -2.328125 -1.046875q-0.90625 -1.046875 -0.90625 -3.46875q0 -2.6875 1.0 -3.921875q0.875 -1.0625 2.34375 -1.0625q1.09375 0 1.796875 0.625q0.703125 0.609375 0.84375 1.6875zm-4.671875 4.015625q0 0.59375 0.25 1.140625q0.25 0.53125 0.703125 0.8125q0.453125 0.28125 0.953125 0.28125q0.71875 0 1.234375 -0.578125q0.53125 -0.59375 0.53125 -1.59375q0 -0.96875 -0.515625 -1.515625q-0.515625 -0.5625 -1.296875 -0.5625q-0.78
 125 0 -1.328125 0.5625q-0.53125 0.546875 -0.53125 1.453125z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m203.67717 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m203.67717 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m213.57243 49.23906l0 -1.09375l6.03125 0l0 0.890625q-0.890625 0.953125 -1.765625 2.515625q-0.875 1.5625 -1.34375 3.21875q-0.34375 1.171875 -0.4375 2.5625l-1.171875 0q0.015625 -1.09375 0.421875 -2.640625q0.421875 -1.5625 1.1875 -3.0q0.765625 -1.453125 1.640625 -2.453125l-4.5625 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m229.66142 40.737534l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m229.66142 40.737534l25.984253 0l0 23.590553l-2
 5.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m241.24419 52.285934q-0.703125 -0.265625 -1.046875 -0.734375q-0.34375 -0.484375 -0.34375 -1.15625q0 -1.015625 0.71875 -1.703125q0.734375 -0.703125 1.953125 -0.703125q1.21875 0 1.953125 0.71875q0.75 0.703125 0.75 1.71875q0 0.640625 -0.34375 1.125q-0.34375 0.46875 -1.03125 0.734375q0.859375 0.28125 1.296875 0.90625q0.453125 0.625 0.453125 1.484375q0 1.1875 -0.84375 2.0q-0.84375 0.8125 -2.21875 0.8125q-1.375 0 -2.21875 -0.8125q-0.84375 -0.8125 -0.84375 -2.03125q0 -0.90625 0.453125 -1.515625q0.46875 -0.625 1.3125 -0.84375zm-0.234375 -1.9375q0 0.65625 0.421875 1.078125q0.4375 0.421875 1.109375 0.421875q0.671875 0 1.09375 -0.40625q0.421875 -0.421875 0.421875 -1.03125q0 -0.625 -0.4375 -1.046875q-0.4375 -0.4375 -1.078125 -0.4375q-0.65625 0 -1.09375 0.421875q-0.4375 0.421875 -0.4375 1.0zm-0.359375 4.296875q0 0.484375 0.234375 0.953125q0.234375 0.453125 0.6875 0.703125q0.453125 0.25 0.984375 0.25q0.8125 0 1.34375 -0.515625q0.5312
 5 -0.53125 0.53125 -1.34375q0 -0.828125 -0.546875 -1.359375q-0.546875 -0.546875 -1.375 -0.546875q-0.796875 0 -1.328125 0.53125q-0.53125 0.53125 -0.53125 1.328125z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m255.64568 40.737534l25.984238 0l0 23.590553l-25.984238 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m255.64568 40.737534l25.984238 0l0 23.590553l-25.984238 0z" fill-rule="nonzero"></path><path fill="#000000" d="m265.6503 55.17656l1.09375 -0.09375q0.140625 0.765625 0.53125 1.125q0.390625 0.34375 1.015625 0.34375q0.515625 0 0.90625 -0.234375q0.40625 -0.25 0.65625 -0.640625q0.265625 -0.40625 0.421875 -1.09375q0.171875 -0.6875 0.171875 -1.40625q0 -0.078125 0 -0.21875q-0.34375 0.546875 -0.9375 0.890625q-0.59375 0.328125 -1.28125 0.328125q-1.15625 0 -1.953125 -0.828125q-0.796875 -0.84375 -0.796875 -2.21875q0 -1.421875 0.828125 -2.28125q0.828125 -0.859375 2.09375 -0.859375q0.90625 0 
 1.65625 0.5q0.75 0.484375 1.140625 1.390625q0.390625 0.890625 0.390625 2.609375q0 1.78125 -0.390625 2.84375q-0.375 1.046875 -1.140625 1.609375q-0.765625 0.546875 -1.796875 0.546875q-1.09375 0 -1.796875 -0.59375q-0.6875 -0.609375 -0.8125 -1.71875zm4.671875 -4.109375q0 -0.984375 -0.53125 -1.546875q-0.515625 -0.578125 -1.25 -0.578125q-0.765625 0 -1.328125 0.625q-0.5625 0.609375 -0.5625 1.609375q0 0.875 0.53125 1.4375q0.53125 0.546875 1.328125 0.546875q0.796875 0 1.296875 -0.546875q0.515625 -0.5625 0.515625 -1.546875z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m47.771652 95.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m47.771652 95.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m61.90129 111.73438l-1.140625 0l0 -7.28125q-0.421875 0.390625 -1.09375 0.796875q-0.65625 0.390625 -1.1875 0.578125l0 -1
 .109375q0.953125 -0.4375 1.671875 -1.078125q0.71875 -0.640625 1.015625 -1.25l0.734375 0l0 9.34375z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m73.755905 95.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m73.755905 95.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m89.58867 110.64063l0 1.09375l-6.15625 0q-0.015625 -0.40625 0.140625 -0.796875q0.234375 -0.625 0.75 -1.234375q0.515625 -0.609375 1.5 -1.40625q1.515625 -1.25 2.046875 -1.96875q0.53125 -0.734375 0.53125 -1.375q0 -0.6875 -0.484375 -1.140625q-0.484375 -0.46875 -1.265625 -0.46875q-0.828125 0 -1.328125 0.5q-0.484375 0.484375 -0.5 1.359375l-1.171875 -0.125q0.125 -1.3125 0.90625 -2.0q0.78125 -0.6875 2.109375 -0.6875q1.34375 0 2.125 0.75q0.78125 0.734375 0.78125 1.828125q0 0.5625 -0.234375 1.109375q-0.21875 0.53125 -0.75 1.140625q-0.53125 0.59
 375 -1.765625 1.625q-1.03125 0.859375 -1.328125 1.171875q-0.28125 0.3125 -0.46875 0.625l4.5625 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m99.74016 95.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m99.74016 95.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m109.57292 109.28126l1.140625 -0.15625q0.203125 0.96875 0.671875 1.40625q0.46875 0.421875 1.15625 0.421875q0.796875 0 1.34375 -0.546875q0.5625 -0.5625 0.5625 -1.390625q0 -0.796875 -0.515625 -1.296875q-0.5 -0.515625 -1.296875 -0.515625q-0.328125 0 -0.8125 0.125l0.125 -1.0q0.125 0.015625 0.1875 0.015625q0.734375 0 1.3125 -0.375q0.59375 -0.390625 0.59375 -1.1875q0 -0.625 -0.4375 -1.03125q-0.421875 -0.421875 -1.09375 -0.421875q-0.671875 0 -1.109375 0.421875q-0.4375 0.421875 -0.578125 1.25l-1.140625 -0.203125q0.21875 -1.140625 0.953125 -1.765
 625q0.75 -0.640625 1.84375 -0.640625q0.765625 0 1.40625 0.328125q0.640625 0.328125 0.984375 0.890625q0.34375 0.5625 0.34375 1.203125q0 0.59375 -0.328125 1.09375q-0.328125 0.5 -0.953125 0.78125q0.8125 0.203125 1.265625 0.796875q0.46875 0.59375 0.46875 1.5q0 1.21875 -0.890625 2.078125q-0.890625 0.84375 -2.25 0.84375q-1.21875 0 -2.03125 -0.734375q-0.8125 -0.734375 -0.921875 -1.890625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m125.72441 95.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m125.72441 95.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m139.21342 111.73438l0 -2.234375l-4.03125 0l0 -1.046875l4.234375 -6.03125l0.9375 0l0 6.03125l1.265625 0l0 1.046875l-1.265625 0l0 2.234375l-1.140625 0zm0 -3.28125l0 -4.1875l-2.921875 4.1875l2.921875 0z" fill-rule="nonzero"></path><path fill="#000000" fill
 -opacity="0.0" d="m151.70866 95.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m151.70866 95.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m161.54143 109.29688l1.1875 -0.109375q0.140625 0.890625 0.625 1.328125q0.484375 0.4375 1.171875 0.4375q0.828125 0 1.390625 -0.625q0.578125 -0.625 0.578125 -1.640625q0 -0.984375 -0.546875 -1.546875q-0.546875 -0.5625 -1.4375 -0.5625q-0.5625 0 -1.015625 0.25q-0.4375 0.25 -0.6875 0.640625l-1.0625 -0.140625l0.890625 -4.765625l4.625 0l0 1.078125l-3.703125 0l-0.5 2.5q0.828125 -0.578125 1.75 -0.578125q1.21875 0 2.046875 0.84375q0.84375 0.84375 0.84375 2.171875q0 1.265625 -0.734375 2.1875q-0.890625 1.125 -2.4375 1.125q-1.265625 0 -2.078125 -0.703125q-0.796875 -0.71875 -0.90625 -1.890625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m177.69292 95.13911l25.984253 0l0 2
 3.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m177.69292 95.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m193.44756 104.70313l-1.140625 0.09375q-0.140625 -0.671875 -0.421875 -0.984375q-0.46875 -0.484375 -1.140625 -0.484375q-0.546875 0 -0.96875 0.3125q-0.53125 0.390625 -0.84375 1.140625q-0.3125 0.75 -0.328125 2.15625q0.40625 -0.625 1.0 -0.921875q0.609375 -0.3125 1.265625 -0.3125q1.140625 0 1.9375 0.84375q0.8125 0.828125 0.8125 2.171875q0 0.875 -0.390625 1.625q-0.375 0.75 -1.03125 1.15625q-0.65625 0.390625 -1.5 0.390625q-1.421875 0 -2.328125 -1.046875q-0.90625 -1.046875 -0.90625 -3.46875q0 -2.6875 1.0 -3.921875q0.875 -1.0625 2.34375 -1.0625q1.09375 0 1.796875 0.625q0.703125 0.609375 0.84375 1.6875zm-4.671875 4.015625q0 0.59375 0.25 1.140625q0.25 0.53125 0.703125 0.8125q0.453125 0.28125 0.953125 0.28125q0.71875 0 1.234375 -0.578125q0.53125 -
 0.59375 0.53125 -1.59375q0 -0.96875 -0.515625 -1.515625q-0.515625 -0.5625 -1.296875 -0.5625q-0.78125 0 -1.328125 0.5625q-0.53125 0.546875 -0.53125 1.453125z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m203.67717 95.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m203.67717 95.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m213.57243 103.64063l0 -1.09375l6.03125 0l0 0.890625q-0.890625 0.953125 -1.765625 2.515625q-0.875 1.5625 -1.34375 3.21875q-0.34375 1.171875 -0.4375 2.5625l-1.171875 0q0.015625 -1.09375 0.421875 -2.640625q0.421875 -1.5625 1.1875 -3.0q0.765625 -1.453125 1.640625 -2.453125l-4.5625 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m47.771652 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" s
 troke-linejoin="round" stroke-linecap="butt" d="m47.771652 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m61.90129 166.13596l-1.140625 0l0 -7.28125q-0.421875 0.390625 -1.09375 0.796875q-0.65625 0.390625 -1.1875 0.578125l0 -1.109375q0.953125 -0.4375 1.671875 -1.078125q0.71875 -0.640625 1.015625 -1.25l0.734375 0l0 9.34375z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m73.755905 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m73.755905 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m89.58867 165.0422l0 1.09375l-6.15625 0q-0.015625 -0.40625 0.140625 -0.796875q0.234375 -0.625 0.75 -1.234375q0.515625 -0.609375 1.5 -1.40625q1.515625 -1.25 2.046875 -1.96875q0.53125 -0.734375 0.53125 -1.375q0 -0.6875 -0.484375 -1.140625q-0.484375 -0.46875 -1.2656
 25 -0.46875q-0.828125 0 -1.328125 0.5q-0.484375 0.484375 -0.5 1.359375l-1.171875 -0.125q0.125 -1.3125 0.90625 -2.0q0.78125 -0.6875 2.109375 -0.6875q1.34375 0 2.125 0.75q0.78125 0.734375 0.78125 1.828125q0 0.5625 -0.234375 1.109375q-0.21875 0.53125 -0.75 1.140625q-0.53125 0.59375 -1.765625 1.625q-1.03125 0.859375 -1.328125 1.171875q-0.28125 0.3125 -0.46875 0.625l4.5625 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m99.74016 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m99.74016 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m109.57292 163.68283l1.140625 -0.15625q0.203125 0.96875 0.671875 1.40625q0.46875 0.421875 1.15625 0.421875q0.796875 0 1.34375 -0.546875q0.5625 -0.5625 0.5625 -1.390625q0 -0.796875 -0.515625 -1.296875q-0.5 -0.515625 -1.296875 -0.515625q-0.328125 0 -0.8125 0.125l0.125 
 -1.0q0.125 0.015625 0.1875 0.015625q0.734375 0 1.3125 -0.375q0.59375 -0.390625 0.59375 -1.1875q0 -0.625 -0.4375 -1.03125q-0.421875 -0.421875 -1.09375 -0.421875q-0.671875 0 -1.109375 0.421875q-0.4375 0.421875 -0.578125 1.25l-1.140625 -0.203125q0.21875 -1.140625 0.953125 -1.765625q0.75 -0.640625 1.84375 -0.640625q0.765625 0 1.40625 0.328125q0.640625 0.328125 0.984375 0.890625q0.34375 0.5625 0.34375 1.203125q0 0.59375 -0.328125 1.09375q-0.328125 0.5 -0.953125 0.78125q0.8125 0.203125 1.265625 0.796875q0.46875 0.59375 0.46875 1.5q0 1.21875 -0.890625 2.078125q-0.890625 0.84375 -2.25 0.84375q-1.21875 0 -2.03125 -0.734375q-0.8125 -0.734375 -0.921875 -1.890625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m125.72441 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m125.72441 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><p
 ath fill="#000000" d="m139.21342 166.13596l0 -2.234375l-4.03125 0l0 -1.046875l4.234375 -6.03125l0.9375 0l0 6.03125l1.265625 0l0 1.046875l-1.265625 0l0 2.234375l-1.140625 0zm0 -3.28125l0 -4.1875l-2.921875 4.1875l2.921875 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m151.70866 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m151.70866 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m161.54143 163.69846l1.1875 -0.109375q0.140625 0.890625 0.625 1.328125q0.484375 0.4375 1.171875 0.4375q0.828125 0 1.390625 -0.625q0.578125 -0.625 0.578125 -1.640625q0 -0.984375 -0.546875 -1.546875q-0.546875 -0.5625 -1.4375 -0.5625q-0.5625 0 -1.015625 0.25q-0.4375 0.25 -0.6875 0.640625l-1.0625 -0.140625l0.890625 -4.765625l4.625 0l0 1.078125l-3.703125 0l-0.5 2.5q0.828125 -0.578125 1.75 -0.578125q1.21875 0 2.046875 
 0.84375q0.84375 0.84375 0.84375 2.171875q0 1.265625 -0.734375 2.1875q-0.890625 1.125 -2.4375 1.125q-1.265625 0 -2.078125 -0.703125q-0.796875 -0.71875 -0.90625 -1.890625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m177.69292 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m177.69292 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m193.44756 159.1047l-1.140625 0.09375q-0.140625 -0.671875 -0.421875 -0.984375q-0.46875 -0.484375 -1.140625 -0.484375q-0.546875 0 -0.96875 0.3125q-0.53125 0.390625 -0.84375 1.140625q-0.3125 0.75 -0.328125 2.15625q0.40625 -0.625 1.0 -0.921875q0.609375 -0.3125 1.265625 -0.3125q1.140625 0 1.9375 0.84375q0.8125 0.828125 0.8125 2.171875q0 0.875 -0.390625 1.625q-0.375 0.75 -1.03125 1.15625q-0.65625 0.390625 -1.5 0.390625q-1.421875 0 -2.328125 -1.046875q-0.90625 -1.046875 
 -0.90625 -3.46875q0 -2.6875 1.0 -3.921875q0.875 -1.0625 2.34375 -1.0625q1.09375 0 1.796875 0.625q0.703125 0.609375 0.84375 1.6875zm-4.671875 4.015625q0 0.59375 0.25 1.140625q0.25 0.53125 0.703125 0.8125q0.453125 0.28125 0.953125 0.28125q0.71875 0 1.234375 -0.578125q0.53125 -0.59375 0.53125 -1.59375q0 -0.96875 -0.515625 -1.515625q-0.515625 -0.5625 -1.296875 -0.5625q-0.78125 0 -1.328125 0.5625q-0.53125 0.546875 -0.53125 1.453125z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m203.67717 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m203.67717 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m213.57243 158.0422l0 -1.09375l6.03125 0l0 0.890625q-0.890625 0.953125 -1.765625 2.515625q-0.875 1.5625 -1.34375 3.21875q-0.34375 1.171875 -0.4375 2.5625l-1.171875 0q0.015625 -1.09375 0.421875 -2.640625q0.42
 1875 -1.5625 1.1875 -3.0q0.765625 -1.453125 1.640625 -2.453125l-4.5625 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m229.66142 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m229.66142 149.54068l25.984253 0l0 23.59056l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m241.24419 161.08908q-0.703125 -0.265625 -1.046875 -0.734375q-0.34375 -0.484375 -0.34375 -1.15625q0 -1.015625 0.71875 -1.703125q0.734375 -0.703125 1.953125 -0.703125q1.21875 0 1.953125 0.71875q0.75 0.703125 0.75 1.71875q0 0.640625 -0.34375 1.125q-0.34375 0.46875 -1.03125 0.734375q0.859375 0.28125 1.296875 0.90625q0.453125 0.625 0.453125 1.484375q0 1.1875 -0.84375 2.0q-0.84375 0.8125 -2.21875 0.8125q-1.375 0 -2.21875 -0.8125q-0.84375 -0.8125 -0.84375 -2.03125q0 -0.90625 0.453125 -1.515625q0.46875 -0.625 1.3125 -0.84375zm-0.234375 -1.9375q0 0.65625 0.421875 1.0
 78125q0.4375 0.421875 1.109375 0.421875q0.671875 0 1.09375 -0.40625q0.421875 -0.421875 0.421875 -1.03125q0 -0.625 -0.4375 -1.046875q-0.4375 -0.4375 -1.078125 -0.4375q-0.65625 0 -1.09375 0.421875q-0.4375 0.421875 -0.4375 1.0zm-0.359375 4.296875q0 0.484375 0.234375 0.953125q0.234375 0.453125 0.6875 0.703125q0.453125 0.25 0.984375 0.25q0.8125 0 1.34375 -0.515625q0.53125 -0.53125 0.53125 -1.34375q0 -0.828125 -0.546875 -1.359375q-0.546875 -0.546875 -1.375 -0.546875q-0.796875 0 -1.328125 0.53125q-0.53125 0.53125 -0.53125 1.328125z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m307.61417 7.2598424l183.84253 0l0 218.83464l-183.84253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m307.61417 7.2598424l183.84253 0l0 218.83464l-183.84253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m317.47354 29.80484l1.6875 -0.140625q0.125 1.015625 0.5625 1.671875q0.4375 0.65625 1.359375 1.0625q0.9375 
 0.40625 2.09375 0.40625q1.03125 0 1.8125 -0.3125q0.796875 -0.3125 1.1875 -0.84375q0.390625 -0.53125 0.390625 -1.15625q0 -0.640625 -0.375 -1.109375q-0.375 -0.484375 -1.234375 -0.8125q-0.546875 -0.21875 -2.421875 -0.65625q-1.875 -0.453125 -2.625 -0.859375q-0.96875 -0.515625 -1.453125 -1.265625q-0.46875 -0.75 -0.46875 -1.6875q0 -1.03125 0.578125 -1.921875q0.59375 -0.90625 1.703125 -1.359375q1.125 -0.46875 2.5 -0.46875q1.515625 0 2.671875 0.484375q1.15625 0.484375 1.765625 1.4375q0.625 0.9375 0.671875 2.140625l-1.71875 0.125q-0.140625 -1.28125 -0.953125 -1.9375q-0.796875 -0.671875 -2.359375 -0.671875q-1.625 0 -2.375 0.609375q-0.75 0.59375 -0.75 1.4375q0 0.734375 0.53125 1.203125q0.515625 0.46875 2.703125 0.96875q2.203125 0.5 3.015625 0.875q1.1875 0.546875 1.75 1.390625q0.578125 0.828125 0.578125 1.921875q0 1.09375 -0.625 2.0625q-0.625 0.953125 -1.796875 1.484375q-1.15625 0.53125 -2.609375 0.53125q-1.84375 0 -3.09375 -0.53125q-1.25 -0.546875 -1.96875 -1.625q-0.703125 -1.078125 -0.734375 
 -2.453125zm19.271698 3.15625q-0.9375 0.796875 -1.796875 1.125q-0.859375 0.3125 -1.84375 0.3125q-1.609375 0 -2.484375 -0.78125q-0.875 -0.796875 -0.875 -2.03125q0 -0.734375 0.328125 -1.328125q0.328125 -0.59375 0.859375 -0.953125q0.53125 -0.359375 1.203125 -0.546875q0.5 -0.140625 1.484375 -0.25q2.03125 -0.25 2.984375 -0.578125q0 -0.34375 0 -0.4375q0 -1.015625 -0.46875 -1.4375q-0.640625 -0.5625 -1.90625 -0.5625q-1.171875 0 -1.734375 0.40625q-0.5625 0.40625 -0.828125 1.46875l-1.640625 -0.234375q0.234375 -1.046875 0.734375 -1.6875q0.515625 -0.640625 1.46875 -0.984375q0.96875 -0.359375 2.25 -0.359375q1.265625 0 2.046875 0.296875q0.78125 0.296875 1.15625 0.75q0.375 0.453125 0.515625 1.140625q0.09375 0.421875 0.09375 1.53125l0 2.234375q0 2.328125 0.09375 2.953125q0.109375 0.609375 0.4375 1.171875l-1.75 0q-0.265625 -0.515625 -0.328125 -1.21875zm-0.140625 -3.71875q-0.90625 0.359375 -2.734375 0.625q-1.03125 0.140625 -1.453125 0.328125q-0.421875 0.1875 -0.65625 0.546875q-0.234375 0.359375 -0.234
 375 0.796875q0 0.671875 0.5 1.125q0.515625 0.4375 1.484375 0.4375q0.96875 0 1.71875 -0.421875q0.75 -0.4375 1.109375 -1.15625q0.265625 -0.578125 0.265625 -1.671875l0 -0.609375zm4.0788574 4.9375l0 -9.859375l1.5 0l0 1.390625q0.453125 -0.71875 1.21875 -1.15625q0.78125 -0.453125 1.765625 -0.453125q1.09375 0 1.796875 0.453125q0.703125 0.453125 0.984375 1.28125q1.171875 -1.734375 3.046875 -1.734375q1.46875 0 2.25 0.8125q0.796875 0.8125 0.796875 2.5l0 6.765625l-1.671875 0l0 -6.203125q0 -1.0 -0.15625 -1.4375q-0.15625 -0.453125 -0.59375 -0.71875q-0.421875 -0.265625 -1.0 -0.265625q-1.03125 0 -1.71875 0.6875q-0.6875 0.6875 -0.6875 2.21875l0 5.71875l-1.671875 0l0 -6.40625q0 -1.109375 -0.40625 -1.65625q-0.40625 -0.5625 -1.34375 -0.5625q-0.703125 0 -1.3125 0.375q-0.59375 0.359375 -0.859375 1.078125q-0.265625 0.71875 -0.265625 2.0625l0 5.109375l-1.671875 0zm14.665802 0l0 -1.359375l6.265625 -7.1875q-1.0625 0.046875 -1.875 0.046875l-4.015625 0l0 -1.359375l8.046875 0l0 1.109375l-5.34375 6.25l-1.015625
  1.140625q1.109375 -0.078125 2.09375 -0.078125l4.5625 0l0 1.4375l-8.71875 0zm16.640625 -1.21875q-0.9375 0.796875 -1.796875 1.125q-0.859375 0.3125 -1.84375 0.3125q-1.609375 0 -2.484375 -0.78125q-0.875 -0.796875 -0.875 -2.03125q0 -0.734375 0.328125 -1.328125q0.328125 -0.59375 0.859375 -0.953125q0.53125 -0.359375 1.203125 -0.546875q0.5 -0.140625 1.484375 -0.25q2.03125 -0.25 2.984375 -0.578125q0 -0.34375 0 -0.4375q0 -1.015625 -0.46875 -1.4375q-0.640625 -0.5625 -1.90625 -0.5625q-1.171875 0 -1.734375 0.40625q-0.5625 0.40625 -0.828125 1.46875l-1.640625 -0.234375q0.234375 -1.046875 0.734375 -1.6875q0.515625 -0.640625 1.46875 -0.984375q0.96875 -0.359375 2.25 -0.359375q1.265625 0 2.046875 0.296875q0.78125 0.296875 1.15625 0.75q0.375 0.453125 0.515625 1.140625q0.09375 0.421875 0.09375 1.53125l0 2.234375q0 2.328125 0.09375 2.953125q0.109375 0.609375 0.4375 1.171875l-1.75 0q-0.265625 -0.515625 -0.328125 -1.21875zm-0.140625 -3.71875q-0.90625 0.359375 -2.734375 0.625q-1.03125 0.140625 -1.453125 0.
 328125q-0.421875 0.1875 -0.65625 0.546875q-0.234375 0.359375 -0.234375 0.796875q0 0.671875 0.5 1.125q0.515625 0.4375 1.484375 0.4375q0.96875 0 1.71875 -0.421875q0.75 -0.4375 1.109375 -1.15625q0.265625 -0.578125 0.265625 -1.671875l0 -0.609375zm14.000702 0.171875l1.796875 0.453125q-0.5625 2.21875 -2.03125 3.390625q-1.46875 1.15625 -3.59375 1.15625q-2.203125 0 -3.578125 -0.890625q-1.375 -0.90625 -2.09375 -2.59375q-0.71875 -1.703125 -0.71875 -3.65625q0 -2.125 0.796875 -3.703125q0.8125 -1.578125 2.3125 -2.390625q1.5 -0.828125 3.296875 -0.828125q2.046875 0 3.4375 1.046875q1.390625 1.03125 1.9375 2.90625l-1.765625 0.421875q-0.46875 -1.484375 -1.375 -2.15625q-0.90625 -0.6875 -2.265625 -0.6875q-1.5625 0 -2.625 0.75q-1.046875 0.75 -1.484375 2.03125q-0.421875 1.265625 -0.421875 2.609375q0 1.734375 0.5 3.03125q0.515625 1.28125 1.578125 1.921875q1.078125 0.640625 2.3125 0.640625q1.515625 0 2.5625 -0.859375q1.046875 -0.875 1.421875 -2.59375zm2.9260864 -0.15625q0 -2.734375 1.53125 -4.0625q1.265625
  -1.09375 3.09375 -1.09375q2.03125 0 3.3125 1.34375q1.296875 1.328125 1.296875 3.671875q0 1.90625 -0.578125 3.0q-0.5625 1.078125 -1.65625 1.6875q-1.078125 0.59375 -2.375 0.59375q-2.0625 0 -3.34375 -1.328125q-1.28125 -1.328125 -1.28125 -3.8125zm1.71875 0q0 1.890625 0.828125 2.828125q0.828125 0.9375 2.078125 0.9375q1.25 0 2.0625 -0.9375q0.828125 -0.953125 0.828125 -2.890625q0 -1.828125 -0.828125 -2.765625q-0.828125 -0.9375 -2.0625 -0.9375q-1.25 0 -2.078125 0.9375q-0.828125 0.9375 -0.828125 2.828125zm9.281952 4.921875l0 -9.859375l1.5 0l0 1.40625q1.09375 -1.625 3.140625 -1.625q0.890625 0 1.640625 0.328125q0.75 0.3125 1.109375 0.84375q0.375 0.515625 0.53125 1.21875q0.09375 0.46875 0.09375 1.625l0 6.0625l-1.671875 0l0 -6.0q0 -1.015625 -0.203125 -1.515625q-0.1875 -0.515625 -0.6875 -0.8125q-0.5 -0.296875 -1.171875 -0.296875q-1.0625 0 -1.84375 0.671875q-0.765625 0.671875 -0.765625 2.578125l0 5.375l-1.671875 0zm14.031982 -1.5l0.234375 1.484375q-0.703125 0.140625 -1.265625 0.140625q-0.90625 0 
 -1.40625 -0.28125q-0.5 -0.296875 -0.703125 -0.75q-0.203125 -0.46875 -0.203125 -1.984375l0 -5.65625l-1.234375 0l0 -1.3125l1.234375 0l0 -2.4375l1.65625 -1.0l0 3.4375l1.6875 0l0 1.3125l-1.6875 0l0 5.75q0 0.71875 0.078125 0.921875q0.09375 0.203125 0.296875 0.328125q0.203125 0.125 0.578125 0.125q0.265625 0 0.734375 -0.078125zm7.9645386 0.28125q-0.9375 0.796875 -1.796875 1.125q-0.859375 0.3125 -1.84375 0.3125q-1.609375 0 -2.484375 -0.78125q-0.875 -0.796875 -0.875 -2.03125q0 -0.734375 0.328125 -1.328125q0.328125 -0.59375 0.859375 -0.953125q0.53125 -0.359375 1.203125 -0.546875q0.5 -0.140625 1.484375 -0.25q2.03125 -0.25 2.984375 -0.578125q0 -0.34375 0 -0.4375q0 -1.015625 -0.46875 -1.4375q-0.640625 -0.5625 -1.90625 -0.5625q-1.171875 0 -1.734375 0.40625q-0.5625 0.40625 -0.828125 1.46875l-1.640625 -0.234375q0.234375 -1.046875 0.734375 -1.6875q0.515625 -0.640625 1.46875 -0.984375q0.96875 -0.359375 2.25 -0.359375q1.265625 0 2.046875 0.296875q0.78125 0.296875 1.15625 0.75q0.375 0.453125 0.515625 1
 .140625q0.09375 0.421875 0.09375 1.53125l0 2.234375q0 2.328125 0.09375 2.953125q0.109375 0.609375 0.4375 1.171875l-1.75 0q-0.265625 -0.515625 -0.328125 -1.21875zm-0.140625 -3.71875q-0.90625 0.359375 -2.734375 0.625q-1.03125 0.140625 -1.453125 0.328125q-0.421875 0.1875 -0.65625 0.546875q-0.234375 0.359375 -0.234375 0.796875q0 0.671875 0.5 1.125q0.515625 0.4375 1.484375 0.4375q0.96875 0 1.71875 -0.421875q0.75 -0.4375 1.109375 -1.15625q0.265625 -0.578125 0.265625 -1.671875l0 -0.609375zm4.0944824 -6.75l0 -1.90625l1.671875 0l0 1.90625l-1.671875 0zm0 11.6875l0 -9.859375l1.671875 0l0 9.859375l-1.671875 0zm4.129181 0l0 -9.859375l1.5 0l0 1.40625q1.09375 -1.625 3.140625 -1.625q0.890625 0 1.640625 0.328125q0.75 0.3125 1.109375 0.84375q0.375 0.515625 0.53125 1.21875q0.09375 0.46875 0.09375 1.625l0 6.0625l-1.671875 0l0 -6.0q0 -1.015625 -0.203125 -1.515625q-0.1875 -0.515625 -0.6875 -0.8125q-0.5 -0.296875 -1.171875 -0.296875q-1.0625 0 -1.84375 0.671875q-0.765625 0.671875 -0.765625 2.578125l0 5.375
 l-1.671875 0zm17.125732 -3.171875l1.71875 0.21875q-0.40625 1.5 -1.515625 2.34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 -3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 2.046875zm9.094452 5.875l0 -9.859375l1.5 0l0 1.5q0.578125 -1.046875 1.0625 -1.375q0.484375 -0.34375 1.078125 -0.34375q0.84375 0 1.71875 0.546875l-0.578125 1.546875q-0.609375 -0.359375 -1.234375 -0.359375q-0.546875 0 -0.984375 0.328125q-0.421875 0.328125 -0.609375 0.90625q-0.28125 0.890625 -0.28125 1.953125l0 5.15625l-1.671875 0z" fill-rule="nonzero"></path><path 
 fill="#000000" fill-opacity="0.0" d="m0 175.6693l0 -142.80316l39.149605 0l0 142.80316z" fill-rule="nonzero"></path><path fill="#000000" d="m15.232498 154.32956l-1.90625 0l0 -1.671875l1.90625 0l0 1.671875zm11.6875 0l-9.859375 0l0 -1.671875l9.859375 0l0 1.671875zm0 -4.129196l-9.859375 0l0 -1.5l1.40625 0q-1.625 -1.09375 -1.625 -3.140625q0 -0.890625 0.328125 -1.640625q0.3125 -0.75 0.84375 -1.109375q0.515625 -0.375 1.21875 -0.53125q0.46875 -0.09375 1.625 -0.09375l6.0625 0l0 1.671875l-6.0 0q-1.015625 0 -1.515625 0.203125q-0.515625 0.1875 -0.8125 0.6875q-0.296875 0.5 -0.296875 1.171875q0 1.0625 0.671875 1.84375q0.671875 0.765625 2.578125 0.765625l5.375 0l0 1.671875zm3.78125 -10.375717l-13.640625 0l0 -1.53125l1.28125 0q-0.75 -0.53125 -1.125 -1.203125q-0.375 -0.6875 -0.375 -1.640625q0 -1.265625 0.65625 -2.234375q0.640625 -0.96875 1.828125 -1.453125q1.1875 -0.5 2.59375 -0.5q1.515625 0 2.734375 0.546875q1.203125 0.546875 1.84375 1.578125q0.640625 1.03125 0.640625 2.171875q0 0.84375 -0.34375 1.
 515625q-0.359375 0.65625 -0.890625 1.078125l4.796875 0l0 1.671875zm-8.65625 -1.515625q1.90625 0 2.8125 -0.765625q0.90625 -0.78125 0.90625 -1.875q0 -1.109375 -0.9375 -1.890625q-0.9375 -0.796875 -2.921875 -0.796875q-1.875 0 -2.8125 0.78125q-0.9375 0.765625 -0.9375 1.84375q0 1.0625 1.0 1.890625q1.0 0.8125 2.890625 0.8125zm4.875 -15.313225l-1.453125 0q1.671875 1.140625 1.671875 3.125q0 0.859375 -0.328125 1.625q-0.34375 0.7500076 -0.84375 1.1250076q-0.5 0.359375 -1.234375 0.515625q-0.5 0.09375 -1.5625 0.09375l-6.109375 0l0 -1.6718826l5.46875 0q1.3125 0 1.765625 -0.09375q0.65625 -0.15625 1.03125 -0.671875q0.375 -0.515625 0.375 -1.265625q0 -0.75 -0.375 -1.40625q-0.390625 -0.65625 -1.046875 -0.921875q-0.671875 -0.28125 -1.9375 -0.28125l-5.28125 0l0 -1.671875l9.859375 0l0 1.5zm-1.5 -7.578842l1.484375 -0.234375q0.140625 0.703125 0.140625 1.265625q0 0.90625 -0.28125 1.40625q-0.296875 0.5 -0.75 0.703125q-0.46875 0.203125 -1.984375 0.203125l-5.65625 0l0 1.234375l-1.3125 0l0 -1.234375l-2.4375 0l-
 1.0 -1.65625l3.4375 0l0 -1.6875l1.3125 0l0 1.6875l5.75 0q0.71875 0 0.921875 -0.078125q0.203125 -0.09375 0.328125 -0.296875q0.125 -0.203125 0.125 -0.578125q0 -0.265625 -0.078125 -0.734375zm-1.4375 -6.0384827l-0.265625 -1.65625q1.0 -0.140625 1.53125 -0.765625q0.515625 -0.640625 0.515625 -1.78125q0 -1.15625 -0.46875 -1.703125q-0.46875 -0.5625 -1.09375 -0.5625q-0.5625 0 -0.890625 0.484375q-0.21875 0.34375 -0.5625 1.703125q-0.46875 1.84375 -0.796875 2.5625q-0.34375 0.703125 -0.9375 1.078125q-0.609375 0.359375 -1.328125 0.359375q-0.65625 0 -1.21875 -0.296875q-0.5625 -0.3125 -0.9375 -0.828125q-0.28125 -0.390625 -0.484375 -1.0625q-0.203125 -0.671875 -0.203125 -1.4375q0 -1.171875 0.34375 -2.046875q0.328125 -0.875 0.90625 -1.28125q0.5625 -0.421875 1.515625 -0.578125l0.21875 1.625q-0.75 0.109375 -1.171875 0.65625q-0.4375 0.53125 -0.4375 1.5q0 1.15625 0.390625 1.640625q0.375 0.484375 0.875 0.484375q0.328125 0 0.59375 -0.203125q0.265625 -0.203125 0.4375 -0.640625q0.09375 -0.25 0.4375 -1.46875q0.
 46875 -1.765625 0.765625 -2.46875q0.296875 -0.703125 0.875 -1.09375q0.578125 -0.40625 1.4375 -0.40625q0.828125 0 1.578125 0.484375q0.734375 0.484375 1.140625 1.40625q0.390625 0.921875 0.390625 2.078125q0 1.921875 -0.796875 2.9375q-0.796875 1.0 -2.359375 1.28125zm1.4375 -13.65625l1.484375 -0.234375q0.140625 0.703125 0.140625 1.265625q0 0.90625 -0.28125 1.40625q-0.296875 0.5 -0.75 0.703125q-0.46875 0.203125 -1.984375 0.203125l-5.65625 0l0 1.234375l-1.3125 0l0 -1.234375l-2.4375 0l-1.0 -1.65625l3.4375 0l0 -1.6875l1.3125 0l0 1.6875l5.75 0q0.71875 0 0.921875 -0.078125q0.203125 -0.09375 0.328125 -0.296875q0.125 -0.203125 0.125 -0.578125q0 -0.265625 -0.078125 -0.734375zm1.5 -1.5114288l-9.859375 0l0 -1.5l1.5 0q-1.046875 -0.578125 -1.375 -1.0625q-0.34375 -0.484375 -0.34375 -1.078125q0 -0.84375 0.546875 -1.71875l1.546875 0.578125q-0.359375 0.609375 -0.359375 1.234375q0 0.546875 0.328125 0.984375q0.328125 0.421875 0.90625 0.609375q0.890625 0.28125 1.953125 0.28125l5.15625 0l0 1.671875zm-3.17187
 5 -12.978302l0.21875 -1.71875q1.5 0.40625 2.34375 1.515625q0.828125 1.09375 0.828125 2.8125q0 2.15625 -1.328125 3.421875q-1.328125 1.265625 -3.734375 1.265625q-2.484375 0 -3.859375 -1.265625q-1.375 -1.28125 -1.375 -3.328125q0 -1.984375 1.34375 -3.234375q1.34375 -1.25 3.796875 -1.25q0.140625 0 0.4375 0.015625l0 7.34375q1.625 -0.09375 2.484375 -0.921875q0.859375 -0.828125 0.859375 -2.0625q0 -0.90625 -0.46875 -1.546875q-0.484375 -0.65625 -1.546875 -1.046875zm-2.703125 5.484375l0 -5.5q-1.234375 0.109375 -1.859375 0.625q-0.96875 0.796875 -0.96875 2.078125q0 1.140625 0.78125 1.9375q0.765625 0.78125 2.046875 0.859375zm4.65625 -15.547592q0.796875 0.9375 1.125 1.796875q0.3125 0.859375 0.3125 1.84375q0 1.609375 -0.78125 2.484375q-0.796875 0.875 -2.03125 0.875q-0.734375 0 -1.328125 -0.328125q-0.59375 -0.328125 -0.953125 -0.859375q-0.359375 -0.53125 -0.546875 -1.203125q-0.140625 -0.5 -0.25 -1.484375q-0.25 -2.03125 -0.578125 -2.984375q-0.34375 0 -0.4375 0q-1.015625 0 -1.4375 0.46875q-0.5625 0.64
 0625 -0.5625 1.90625q0 1.171875 0.40625 1.734375q0.40625 0.5625 1.46875 0.828125l-0.234375 1.640625q-1.046875 -0.234375 -1.6875 -0.734375q-0.640625 -0.515625 -0.984375 -1.46875q-0.359375 -0.96875 -0.359375 -2.25q0 -1.265625 0.296875 -2.046875q0.296875 -0.78125 0.75 -1.15625q0.453125 -0.375 1.140625 -0.515625q0.421875 -0.09375 1.53125 -0.09375l2.234375 0q2.328125 0 2.953125 -0.09375q0.609375 -0.109375 1.171875 -0.4375l0 1.75q-0.515625 0.265625 -1.21875 0.328125zm-3.71875 0.140625q0.359375 0.90625 0.625 2.734375q0.140625 1.03125 0.328125 1.453125q0.1875 0.421875 0.546875 0.65625q0.359375 0.234375 0.796875 0.234375q0.671875 0 1.125 -0.5q0.4375 -0.515625 0.4375 -1.484375q0 -0.96875 -0.421875 -1.71875q-0.4375 -0.75 -1.15625 -1.109375q-0.578125 -0.265625 -1.671875 -0.265625l-0.609375 0zm4.9375 -4.078842l-9.859375 0l0 -1.5l1.390625 0q-0.71875 -0.453125 -1.15625 -1.21875q-0.453125 -0.7812462 -0.453125 -1.7656212q0 -1.09375 0.453125 -1.796875q0.453125 -0.703125 1.28125 -0.984375q-1.734375 -1
 .171875 -1.734375 -3.046875q0 -1.46875 0.8125 -2.25q0.8125 -0.796875 2.5 -0.796875l6.765625 0l0 1.671875l-6.203125 0q-1.0 0 -1.4375 0.15625q-0.453125 0.15625 -0.71875 0.59375q-0.265625 0.421875 -0.265625 1.0q0 1.03125 0.6875 1.71875q0.6875 0.6875 2.21875 0.6875l5.71875 0l0 1.671875l-6.40625 0q-1.109375 0 -1.65625 0.40625q-0.5625 0.40625 -0.5625 1.34375q0 0.703125 0.375 1.3124962q0.359375 0.59375 1.078125 0.859375q0.71875 0.265625 2.0625 0.265625l5.109375 0l0 1.671875z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m39.149605 13.538536l90.960625 0l0 31.370079l-90.960625 0z" fill-rule="nonzero"></path><path fill="#000000" d="m49.149605 35.338535l0 -9.3125l3.515625 0q0.921875 0 1.40625 0.09375q0.6875 0.109375 1.15625 0.4375q0.46875 0.3125 0.75 0.890625q0.28125 0.578125 0.28125 1.28125q0 1.1875 -0.765625 2.015625q-0.75 0.8125 -2.71875 0.8125l-2.390625 0l0 3.78125l-1.234375 0zm1.234375 -4.875l2.40625 0q1.1875 0 1.6875 -0.4375q0.515625 -0.453125 0.515625 -1.265625q
 0 -0.578125 -0.296875 -0.984375q-0.296875 -0.421875 -0.78125 -0.5625q-0.3125 -0.078125 -1.15625 -0.078125l-2.375 0l0 3.328125zm11.90538 4.046875q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5
 625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.0999756 3.390625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm6.9539948 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0
 .078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029877 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm5.46109 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029877 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm2.539215 -3.375q0 -1.875 1.03125 -2.765625q0.875 -0.75 2.125 -0.75q1.390625 0 2.265625 0.90625q0.890625 0.90625 0.890625 2.515625q0 1.296875 -0.390625 2.046875q-0.390625 0.75 -1.140625 1.171875q-0.75 0.40625 -1.625 0.40625q-1.421875 0 -2.296875 -0.90625q-0.859375 -0.90625 -
 0.859375 -2.625zm1.171875 0q0 1.296875 0.5625 1.953125q0.5625 0.640625 1.421875 0.640625q0.84375 0 1.40625 -0.640625q0.578125 -0.65625 0.578125 -1.984375q0 -1.25 -0.578125 -1.890625q-0.5625 -0.65625 -1.40625 -0.65625q-0.859375 0 -1.421875 0.640625q-0.5625 0.640625 -0.5625 1.9375zm6.6624756 3.375l0 -6.734375l1.03125 0l0 0.953125q0.734375 -1.109375 2.140625 -1.109375q0.609375 0 1.109375 0.21875q0.515625 0.21875 0.765625 0.578125q0.265625 0.34375 0.359375 0.84375q0.0625 0.3125 0.0625 1.109375l0 4.140625l-1.140625 0l0 -4.09375q0 -0.703125 -0.140625 -1.046875q-0.125 -0.34375 -0.46875 -0.546875q-0.328125 -0.21875 -0.78125 -0.21875q-0.734375 0 -1.265625 0.46875q-0.53125 0.453125 -0.53125 1.75l0 3.6875l-1.140625 0zm10.802963 -4.59375q0 -1.640625 0.328125 -2.640625q0.34375 -1.015625 1.015625 -1.5625q0.671875 -0.546875 1.6875 -0.546875q0.75 0 1.3125 0.3125q0.5625 0.296875 0.921875 0.859375q0.375 0.5625 0.578125 1.390625q0.21875 0.8125 0.21875 2.1875q0 1.640625 -0.34375 2.65625q-0.328125 1.0 -
 1.0 1.546875q-0.671875 0.546875 -1.6875 0.546875q-1.34375 0 -2.125 -0.96875q-0.90625 -1.15625 -0.90625 -3.78125zm1.171875 0q0 2.296875 0.53125 3.0625q0.53125 0.75 1.328125 0.75q0.78125 0 1.3125 -0.75q0.546875 -0.765625 0.546875 -3.0625q0 -2.296875 -0.546875 -3.046875q-0.53125 -0.75 -1.328125 -0.75q-0.78125 0 -1.265625 0.65625q-0.578125 0.859375 -0.578125 3.140625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m39.149605 67.37405l90.960625 0l0 32.850395l-90.960625 0z" fill-rule="nonzero"></path><path fill="#000000" d="m49.149605 89.17404l0 -9.3125l3.515625 0q0.921875 0 1.40625 0.09375q0.6875 0.109375 1.15625 0.4375q0.46875 0.3125 0.75 0.890625q0.28125 0.578125 0.28125 1.28125q0 1.1875 -0.765625 2.015625q-0.75 0.8125 -2.71875 0.8125l-2.390625 0l0 3.78125l-1.234375 0zm1.234375 -4.875l2.40625 0q1.1875 0 1.6875 -0.4375q0.515625 -0.453125 0.515625 -1.265625q0 -0.578125 -0.296875 -0.984375q-0.296875 -0.421875 -0.78125 -0.5625q-0.3125 -0.078125 -1.15625 -0.078125l-2
 .375 0l0 3.328125zm11.90538 4.046875q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.156
 25 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.0999756 3.390625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm6.9539948 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029877 -6.96875l0 -1.328125l1.140625 0l0 1.328125
 l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm5.46109 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029877 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm2.539215 -3.375q0 -1.875 1.03125 -2.765625q0.875 -0.75 2.125 -0.75q1.390625 0 2.265625 0.90625q0.890625 0.90625 0.890625 2.515625q0 1.296875 -0.390625 2.046875q-0.390625 0.75 -1.140625 1.171875q-0.75 0.40625 -1.625 0.40625q-1.421875 0 -2.296875 -0.90625q-0.859375 -0.90625 -0.859375 -2.625zm1.171875 0q0 1.296875 0.5625 1.953125q0.5625 0.640625 1.421875 0.640625q0.84375 0 1.40625 -
 0.640625q0.578125 -0.65625 0.578125 -1.984375q0 -1.25 -0.578125 -1.890625q-0.5625 -0.65625 -1.40625 -0.65625q-0.859375 0 -1.421875 0.640625q-0.5625 0.640625 -0.5625 1.9375zm6.6624756 3.375l0 -6.734375l1.03125 0l0 0.953125q0.734375 -1.109375 2.140625 -1.109375q0.609375 0 1.109375 0.21875q0.515625 0.21875 0.765625 0.578125q0.265625 0.34375 0.359375 0.84375q0.0625 0.3125 0.0625 1.109375l0 4.140625l-1.140625 0l0 -4.09375q0 -0.703125 -0.140625 -1.046875q-0.125 -0.34375 -0.46875 -0.546875q-0.328125 -0.21875 -0.78125 -0.21875q-0.734375 0 -1.265625 0.46875q-0.53125 0.453125 -0.53125 1.75l0 3.6875l-1.140625 0zm15.099838 0l-1.140625 0l0 -7.28125q-0.421875 0.390625 -1.09375 0.796875q-0.65625 0.390625 -1.1875 0.578125l0 -1.109375q0.953125 -0.4375 1.671875 -1.078125q0.71875 -0.640625 1.015625 -1.25l0.734375 0l0 9.34375z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m39.149605 122.320206l90.960625 0l0 29.35434l-90.960625 0z" fill-rule="nonzero"></path><path fill="#000000"
  d="m49.149605 144.12021l0 -9.3125l3.515625 0q0.921875 0 1.40625 0.09375q0.6875 0.109375 1.15625 0.4375q0.46875 0.3125 0.75 0.890625q0.28125 0.578125 0.28125 1.28125q0 1.1875 -0.765625 2.015625q-0.75 0.8125 -2.71875 0.8125l-2.390625 0l0 3.78125l-1.234375 0zm1.234375 -4.875l2.40625 0q1.1875 0 1.6875 -0.4375q0.515625 -0.453125 0.515625 -1.265625q0 -0.578125 -0.296875 -0.984375q-0.296875 -0.421875 -0.78125 -0.5625q-0.3125 -0.078125 -1.15625 -0.078125l-2.375 0l0 3.328125zm11.90538 4.046875q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375
  -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.0999756 3.390625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm6.9539948 -1.015625
 l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029877 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm5.46109 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029877 -6.96875l0 -1.328125l1.140625 0l0 1.328125
 l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm2.539215 -3.375q0 -1.875 1.03125 -2.765625q0.875 -0.75 2.125 -0.75q1.390625 0 2.265625 0.90625q0.890625 0.90625 0.890625 2.515625q0 1.296875 -0.390625 2.046875q-0.390625 0.75 -1.140625 1.171875q-0.75 0.40625 -1.625 0.40625q-1.421875 0 -2.296875 -0.90625q-0.859375 -0.90625 -0.859375 -2.625zm1.171875 0q0 1.296875 0.5625 1.953125q0.5625 0.640625 1.421875 0.640625q0.84375 0 1.40625 -0.640625q0.578125 -0.65625 0.578125 -1.984375q0 -1.25 -0.578125 -1.890625q-0.5625 -0.65625 -1.40625 -0.65625q-0.859375 0 -1.421875 0.640625q-0.5625 0.640625 -0.5625 1.9375zm6.6624756 3.375l0 -6.734375l1.03125 0l0 0.953125q0.734375 -1.109375 2.140625 -1.109375q0.609375 0 1.109375 0.21875q0.515625 0.21875 0.765625 0.578125q0.265625 0.34375 0.359375 0.84375q0.0625 0.3125 0.0625 1.109375l0 4.140625l-1.140625 0l0 -4.09375q0 -0.703125 -0.140625 -1.046875q-0.125 -0.34375 -0.46875 -0.546875q-0.328125 -0.21875 -0.78125 -0.21875q-0.734375 0 -1.265
 625 0.46875q-0.53125 0.453125 -0.53125 1.75l0 3.6875l-1.140625 0zm16.802963 -1.09375l0 1.09375l-6.15625 0q-0.015625 -0.40625 0.140625 -0.796875q0.234375 -0.625 0.75 -1.234375q0.515625 -0.609375 1.5 -1.40625q1.515625 -1.25 2.046875 -1.96875q0.53125 -0.734375 0.53125 -1.375q0 -0.6875 -0.484375 -1.140625q-0.484375 -0.46875 -1.265625 -0.46875q-0.828125 0 -1.328125 0.5q-0.484375 0.484375 -0.5 1.359375l-1.171875 -0.125q0.125 -1.3125 0.90625 -2.0q0.78125 -0.6875 2.109375 -0.6875q1.34375 0 2.125 0.75q0.78125 0.734375 0.78125 1.828125q0 0.5625 -0.234375 1.109375q-0.21875 0.53125 -0.75 1.140625q-0.53125 0.59375 -1.765625 1.625q-1.03125 0.859375 -1.328125 1.171875q-0.28125 0.3125 -0.46875 0.625l4.5625 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m319.53543 56.850395l160.0 0l0 44.53543l-160.0 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m319.53543 56.850395l160.0 0l0 44.53543l-160.0 0z" fil
 l-rule="nonzero"></path><path fill="#000000" d="m333.0759 80.93373l1.171875 -0.109375q0.078125 0.703125 0.375 1.15625q0.3125 0.4375 0.9375 0.71875q0.640625 0.265625 1.4375 0.265625q0.703125 0 1.234375 -0.203125q0.546875 -0.203125 0.8125 -0.5625q0.265625 -0.375 0.265625 -0.8125q0 -0.4375 -0.265625 -0.765625q-0.25 -0.328125 -0.828125 -0.546875q-0.375 -0.140625 -1.65625 -0.453125q-1.28125 -0.3125 -1.796875 -0.578125q-0.671875 -0.34375 -1.0 -0.859375q-0.328125 -0.53125 -0.328125 -1.171875q0 -0.703125 0.390625 -1.3125q0.40625 -0.609375 1.171875 -0.921875q0.78125 -0.328125 1.71875 -0.328125q1.03125 0 1.8125 0.34375q0.796875 0.328125 1.21875 0.984375q0.4375 0.640625 0.46875 1.453125l-1.1875 0.09375q-0.09375 -0.890625 -0.640625 -1.328125q-0.546875 -0.453125 -1.625 -0.453125q-1.109375 0 -1.625 0.40625q-0.515625 0.40625 -0.515625 0.984375q0 0.5 0.359375 0.828125q0.359375 0.328125 1.859375 0.671875q1.5 0.328125 2.0625 0.578125q0.8125 0.375 1.1875 0.953125q0.390625 0.578125 0.390625 1.328125q0 
 0.734375 -0.421875 1.390625q-0.421875 0.65625 -1.21875 1.03125q-0.796875 0.359375 -1.796875 0.359375q-1.265625 0 -2.125 -0.359375q-0.84375 -0.375 -1.328125 -1.109375q-0.484375 -0.75 -0.515625 -1.671875zm11.67099 1.96875l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.1873779 1.015625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm9.0633545 -2.171875l1.1875 0.140625q-0.28125 1.046875 -1.0
 46875 1.625q-0.75 0.5625 -1.921875 0.5625q-1.484375 0 -2.359375 -0.90625q-0.859375 -0.921875 -0.859375 -2.5625q0 -1.703125 0.875 -2.640625q0.890625 -0.9375 2.28125 -0.9375q1.359375 0 2.203125 0.921875q0.859375 0.921875 0.859375 2.578125q0 0.109375 0 0.3125l-5.03125 0q0.0625 1.109375 0.625 1.703125q0.5625 0.59375 1.40625 0.59375q0.640625 0 1.078125 -0.328125q0.453125 -0.34375 0.703125 -1.0625zm-3.75 -1.84375l3.765625 0q-0.078125 -0.859375 -0.4375 -1.28125q-0.546875 -0.65625 -1.40625 -0.65625q-0.796875 0 -1.328125 0.53125q-0.53125 0.515625 -0.59375 1.40625zm10.943726 3.1875q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.
 203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.1156006 3.390625l0 -6.734375l1.015625 0l0 0.9375q0.328125 -0.5 0.84375 -0.796875q0.53125 -0.296875 1.203125 -0.296875q0.75 0 1.21875 0.3125q0.484375 0.3125 0.6875 0.859375q0.796875 -1.171875 2.078125 -1.171875q1.0 0 1.53125 0.5625q0.546875 0.54687
 5 0.546875 1.703125l0 4.625l-1.125 0l0 -4.25q0 -0.6875 -0.109375 -0.984375q-0.109375 -0.296875 -0.40625 -0.484375q-0.296875 -0.1875 -0.6875 -0.1875q-0.71875 0 -1.1875 0.484375q-0.46875 0.46875 -0.46875 1.5l0 3.921875l-1.140625 0l0 -4.375q0 -0.765625 -0.28125 -1.140625q-0.28125 -0.390625 -0.90625 -0.390625q-0.484375 0 -0.890625 0.265625q-0.40625 0.25 -0.59375 0.734375q-0.1875 0.484375 -0.1875 1.40625l0 3.5l-1.140625 0zm13.6180725 0l0 -8.203125l-3.0625 0l0 -1.109375l7.375 0l0 1.109375l-3.078125 0l0 8.203125l-1.234375 0zm10.016357 -0.828125q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.56
 25 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm2.6624756 1.375l1.125 -0.171875q0.09375 0.671875 0.53125 1.046875q0.4375 0.359375 1.21875 0.359375q0.78125 0 1.15625 -0.3125q0.390625 -0.328125 0.390625 -0.765625q0 -0.390625 -0.34375 -0.609375q-0.234375 -0.15625 -1.171875 -0.390625q-1.25 -0.3125 -1.734375 -0.546875q-0.484375 -0.2343
 75 -0.734375 -0.640625q-0.25 -0.40625 -0.25 -0.90625q0 -0.453125 0.203125 -0.828125q0.203125 -0.390625 0.5625 -0.640625q0.265625 -0.203125 0.71875 -0.328125q0.46875 -0.140625 1.0 -0.140625q0.78125 0 1.375 0.234375q0.609375 0.21875 0.890625 0.609375q0.296875 0.390625 0.40625 1.046875l-1.125 0.15625q-0.078125 -0.53125 -0.4375 -0.8125q-0.359375 -0.296875 -1.03125 -0.296875q-0.78125 0 -1.125 0.265625q-0.34375 0.25 -0.34375 0.609375q0 0.21875 0.140625 0.390625q0.140625 0.1875 0.4375 0.3125q0.171875 0.0625 1.015625 0.28125q1.21875 0.328125 1.6875 0.53125q0.484375 0.203125 0.75 0.609375q0.28125 0.390625 0.28125 0.96875q0 0.578125 -0.34375 1.078125q-0.328125 0.5 -0.953125 0.78125q-0.625 0.28125 -1.421875 0.28125q-1.3125 0 -2.0 -0.546875q-0.6875 -0.546875 -0.875 -1.625zm7.1171875 2.015625l0 -9.3125l1.140625 0l0 5.3125l2.703125 -2.734375l1.484375 0l-2.578125 2.5l2.84375 4.234375l-1.40625 0l-2.234375 -3.453125l-0.8125 0.78125l0 2.671875l-1.140625 0zm10.367035 2.578125l0 -9.3125l1.03125 0l0 0.8
 75q0.375 -0.515625 0.828125 -0.765625q0.46875 -0.265625 1.140625 -0.265625q0.859375 0 1.515625 0.453125q0.65625 0.4375 0.984375 1.25q0.34375 0.796875 0.34375 1.765625q0 1.03125 -0.375 1.859375q-0.359375 0.828125 -1.078125 1.28125q-0.703125 0.4375 -1.484375 0.4375q-0.5625 0 -1.015625 -0.234375q-0.453125 -0.25 -0.75 -0.625l0 3.28125l-1.140625 0zm1.03125 -5.90625q0 1.296875 0.53125 1.921875q0.53125 0.625 1.265625 0.625q0.765625 0 1.3125 -0.640625q0.546875 -0.65625 0.546875 -2.0q0 -1.296875 -0.53125 -1.9375q-0.53125 -0.640625 -1.265625 -0.640625q-0.734375 0 -1.296875 0.6875q-0.5625 0.671875 -0.5625 1.984375zm10.771851 2.5q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4
 375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.0999756 3.390625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-
 0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm6.95401 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029724 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm5.46109 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 
 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2030029 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm2.539215 -3.375q0 -1.875 1.03125 -2.765625q0.875 -0.75 2.125 -0.75q1.390625 0 2.265625 0.90625q0.890625 0.90625 0.890625 2.515625q0 1.296875 -0.390625 2.046875q-0.390625 0.75 -1.140625 1.171875q-0.75 0.40625 -1.625 0.40625q-1.421875 0 -2.296875 -0.90625q-0.859375 -0.90625 -0.859375 -2.625zm1.171875 0q0 1.296875 0.5625 1.953125q0.5625 0.640625 1.421875 0.640625q0.84375 0 1.40625 -0.640625q0.578125 -0.65625 0.578125 -1.984375q0 -1.25 -0.578125 -1.890625q-0.5625 -0.65625 -1.40625 -0.65625q-0.859375 0 -1.421875 0.640625q-0.5625 0.640625 -0.5625 1.9375zm6.6624756 3.375l0 -6.734375l1.03125 0l0 0.953125q0.734375 -1.109375 2.140625 -1.109375q0.609375 0 1.109375 0.21875q0.515625 0.21875 0.765625 0.578125q0.265625 0.34375 0.359375 0.84375q0.0625 0.3125 0.0625 1.109375l0 4.140625l-1.140625 0l
 0 -4.09375q0 -0.703125 -0.140625 -1.046875q-0.125 -0.34375 -0.46875 -0.546875q-0.328125 -0.21875 -0.78125 -0.21875q-0.734375 0 -1.265625 0.46875q-0.53125 0.453125 -0.53125 1.75l0 3.6875l-1.140625 0zm10.802948 -4.59375q0 -1.640625 0.328125 -2.640625q0.34375 -1.015625 1.015625 -1.5625q0.671875 -0.546875 1.6875 -0.546875q0.75 0 1.3125 0.3125q0.5625 0.296875 0.921875 0.859375q0.375 0.5625 0.578125 1.390625q0.21875 0.8125 0.21875 2.1875q0 1.640625 -0.34375 2.65625q-0.328125 1.0 -1.0 1.546875q-0.671875 0.546875 -1.6875 0.546875q-1.34375 0 -2.125 -0.96875q-0.90625 -1.15625 -0.90625 -3.78125zm1.171875 0q0 2.296875 0.53125 3.0625q0.53125 0.75 1.328125 0.75q0.78125 0 1.3125 -0.75q0.546875 -0.765625 0.546875 -3.0625q0 -2.296875 -0.546875 -3.046875q-0.53125 -0.75 -1.328125 -0.75q-0.78125 0 -1.265625 0.65625q-0.578125 0.859375 -0.578125 3.140625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m319.53543 111.25197l160.0 0l0 44.53543l-160.0 0z" fill-rule="nonzero"></path><p
 ath stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m319.53543 111.25197l160.0 0l0 44.53543l-160.0 0z" fill-rule="nonzero"></path><path fill="#000000" d="m333.0759 135.33531l1.171875 -0.109375q0.078125 0.703125 0.375 1.15625q0.3125 0.4375 0.9375 0.71875q0.640625 0.265625 1.4375 0.265625q0.703125 0 1.234375 -0.203125q0.546875 -0.203125 0.8125 -0.5625q0.265625 -0.375 0.265625 -0.8125q0 -0.4375 -0.265625 -0.765625q-0.25 -0.328125 -0.828125 -0.546875q-0.375 -0.140625 -1.65625 -0.453125q-1.28125 -0.3125 -1.796875 -0.578125q-0.671875 -0.34375 -1.0 -0.859375q-0.328125 -0.53125 -0.328125 -1.171875q0 -0.703125 0.390625 -1.3125q0.40625 -0.609375 1.171875 -0.921875q0.78125 -0.328125 1.71875 -0.328125q1.03125 0 1.8125 0.34375q0.796875 0.328125 1.21875 0.984375q0.4375 0.640625 0.46875 1.453125l-1.1875 0.09375q-0.09375 -0.890625 -0.640625 -1.328125q-0.546875 -0.453125 -1.625 -0.453125q-1.109375 0 -1.625 0.40625q-0.515625 0.40625 -0.515625 0.984375q0 0.5 0.3593
 75 0.828125q0.359375 0.328125 1.859375 0.671875q1.5 0.328125 2.0625 0.578125q0.8125 0.375 1.1875 0.953125q0.390625 0.578125 0.390625 1.328125q0 0.734375 -0.421875 1.390625q-0.421875 0.65625 -1.21875 1.03125q-0.796875 0.359375 -1.796875 0.359375q-1.265625 0 -2.125 -0.359375q-0.84375 -0.375 -1.328125 -1.109375q-0.484375 -0.75 -0.515625 -1.671875zm11.67099 1.96875l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.1873779 1.015625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.29
 6875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm9.0633545 -2.171875l1.1875 0.140625q-0.28125 1.046875 -1.046875 1.625q-0.75 0.5625 -1.921875 0.5625q-1.484375 0 -2.359375 -0.90625q-0.859375 -0.921875 -0.859375 -2.5625q0 -1.703125 0.875 -2.640625q0.890625 -0.9375 2.28125 -0.9375q1.359375 0 2.203125 0.921875q0.859375 0.921875 0.859375 2.578125q0 0.109375 0 0.3125l-5.03125 0q0.0625 1.109375 0.625 1.703125q0.5625 0.59375 1.40625 0.59375q0.640625 0 1.078125 -0.328125q0.453125 -0.34375 0.703125 -1.0625zm-3.75 -1.84375l3.765625 0q-0.078125 -0.859375 -0.4375 -1.28125q-0.546875 -0.65625 -1.40625 -0.65625q-0.796875 0 -1.328125 0.53125q-0.53125 0.515625 -0.59375 1.40625zm10.943726 3.1875q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.1
 5625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.1156006 3.390625l0 -6.734375l1.015625 0l0 0.9375q0.328125 -0.5 0.84375 -0.796875q0.53125 -0.296875 1.20
 3125 -0.296875q0.75 0 1.21875 0.3125q0.484375 0.3125 0.6875 0.859375q0.796875 -1.171875 2.078125 -1.171875q1.0 0 1.53125 0.5625q0.546875 0.546875 0.546875 1.703125l0 4.625l-1.125 0l0 -4.25q0 -0.6875 -0.109375 -0.984375q-0.109375 -0.296875 -0.40625 -0.484375q-0.296875 -0.1875 -0.6875 -0.1875q-0.71875 0 -1.1875 0.484375q-0.46875 0.46875 -0.46875 1.5l0 3.921875l-1.140625 0l0 -4.375q0 -0.765625 -0.28125 -1.140625q-0.28125 -0.390625 -0.90625 -0.390625q-0.484375 0 -0.890625 0.265625q-0.40625 0.25 -0.59375 0.734375q-0.1875 0.484375 -0.1875 1.40625l0 3.5l-1.140625 0zm13.6180725 0l0 -8.203125l-3.0625 0l0 -1.109375l7.375 0l0 1.109375l-3.078125 0l0 8.203125l-1.234375 0zm10.016357 -0.828125q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.
 234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm2.6624756 1.375l1.125 -0.171875q0.09375 0.671875 0.53125 1.046875q0.4375 0.359375 1.21875 0.359375q0.78125 0 1.15625 -0.3125q0.390625 -0.328
 125 0.390625 -0.765625q0 -0.390625 -0.34375 -0.609375q-0.234375 -0.15625 -1.171875 -0.390625q-1.25 -0.3125 -1.734375 -0.546875q-0.484375 -0.234375 -0.734375 -0.640625q-0.25 -0.40625 -0.25 -0.90625q0 -0.453125 0.203125 -0.828125q0.203125 -0.390625 0.5625 -0.640625q0.265625 -0.203125 0.71875 -0.328125q0.46875 -0.140625 1.0 -0.140625q0.78125 0 1.375 0.234375q0.609375 0.21875 0.890625 0.609375q0.296875 0.390625 0.40625 1.046875l-1.125 0.15625q-0.078125 -0.53125 -0.4375 -0.8125q-0.359375 -0.296875 -1.03125 -0.296875q-0.78125 0 -1.125 0.265625q-0.34375 0.25 -0.34375 0.609375q0 0.21875 0.140625 0.390625q0.140625 0.1875 0.4375 0.3125q0.171875 0.0625 1.015625 0.28125q1.21875 0.328125 1.6875 0.53125q0.484375 0.203125 0.75 0.609375q0.28125 0.390625 0.28125 0.96875q0 0.578125 -0.34375 1.078125q-0.328125 0.5 -0.953125 0.78125q-0.625 0.28125 -1.421875 0.28125q-1.3125 0 -2.0 -0.546875q-0.6875 -0.546875 -0.875 -1.625zm7.1171875 2.015625l0 -9.3125l1.140625 0l0 5.3125l2.703125 -2.734375l1.484375 0l-2
 .578125 2.5l2.84375 4.234375l-1.40625 0l-2.234375 -3.453125l-0.8125 0.78125l0 2.671875l-1.140625 0zm10.367035 2.578125l0 -9.3125l1.03125 0l0 0.875q0.375 -0.515625 0.828125 -0.765625q0.46875 -0.265625 1.140625 -0.265625q0.859375 0 1.515625 0.453125q0.65625 0.4375 0.984375 1.25q0.34375 0.796875 0.34375 1.765625q0 1.03125 -0.375 1.859375q-0.359375 0.828125 -1.078125 1.28125q-0.703125 0.4375 -1.484375 0.4375q-0.5625 0 -1.015625 -0.234375q-0.453125 -0.25 -0.75 -0.625l0 3.28125l-1.140625 0zm1.03125 -5.90625q0 1.296875 0.53125 1.921875q0.53125 0.625 1.265625 0.625q0.765625 0 1.3125 -0.640625q0.546875 -0.65625 0.546875 -2.0q0 -1.296875 -0.53125 -1.9375q-0.53125 -0.640625 -1.265625 -0.640625q-0.734375 0 -1.296875 0.6875q-0.5625 0.671875 -0.5625 1.984375zm10.771851 2.5q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0
 .328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.0999756 3.390625l0 -6.734375l1.03125 0l0 1.015625q0.3906
 25 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm6.95401 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029724 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm5.46109 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.82812
 5 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2030029 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm2.539215 -3.375q0 -1.875 1.03125 -2.765625q0.875 -0.75 2.125 -0.75q1.390625 0 2.265625 0.90625q0.890625 0.90625 0.890625 2.515625q0 1.296875 -0.390625 2.046875q-0.390625 0.75 -1.140625 1.171875q-0.75 0.40625 -1.625 0.40625q-1.421875 0 -2.296875 -0.90625q-0.859375 -0.90625 -0.859375 -2.625zm1.171875 0q0 1.296875 0.5625 1.953125q0.5625 0.640625 1.421875 0.640625q0.84375 0 1.40625 -0.640625q0.578125 -0.65625 0.578125 -1.984375q0 -1.25 -0.578125 -1.890625q-0.5625 -0.65625 -1.40625 -0.65625q-0.859375 0 -1.421875 0.640625q-0.5625 0.640625 -0.5625 1.9375zm6.6624756 3.375l0 -6.734375l1.03125 0l0 0.953125q0.734375 -1.109375 2.140625 -1.109375q0.6093
 75 0 1.109375 0.21875q0.515625 0.21875 0.765625 0.578125q0.265625 0.34375 0.359375 0.84375q0.0625 0.3125 0.0625 1.109375l0 4.140625l-1.140625 0l0 -4.09375q0 -0.703125 -0.140625 -1.046875q-0.125 -0.34375 -0.46875 -0.546875q-0.328125 -0.21875 -0.78125 -0.21875q-0.734375 0 -1.265625 0.46875q-0.53125 0.453125 -0.53125 1.75l0 3.6875l-1.140625 0zm15.099823 0l-1.140625 0l0 -7.28125q-0.421875 0.390625 -1.09375 0.796875q-0.65625 0.390625 -1.1875 0.578125l0 -1.109375q0.953125 -0.4375 1.671875 -1.078125q0.71875 -0.640625 1.015625 -1.25l0.734375 0l0 9.34375z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m319.53543 165.65355l160.0 0l0 44.53543l-160.0 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m319.53543 165.65355l160.0 0l0 44.53543l-160.0 0z" fill-rule="nonzero"></path><path fill="#000000" d="m333.0759 189.73688l1.171875 -0.109375q0.078125 0.703125 0.375 1.15625q0.3125 0.4375 0.9375 0.71875q0.
 640625 0.265625 1.4375 0.265625q0.703125 0 1.234375 -0.203125q0.546875 -0.203125 0.8125 -0.5625q0.265625 -0.375 0.265625 -0.8125q0 -0.4375 -0.265625 -0.765625q-0.25 -0.328125 -0.828125 -0.546875q-0.375 -0.140625 -1.65625 -0.453125q-1.28125 -0.3125 -1.796875 -0.578125q-0.671875 -0.34375 -1.0 -0.859375q-0.328125 -0.53125 -0.328125 -1.171875q0 -0.703125 0.390625 -1.3125q0.40625 -0.609375 1.171875 -0.921875q0.78125 -0.328125 1.71875 -0.328125q1.03125 0 1.8125 0.34375q0.796875 0.328125 1.21875 0.984375q0.4375 0.640625 0.46875 1.453125l-1.1875 0.09375q-0.09375 -0.890625 -0.640625 -1.328125q-0.546875 -0.453125 -1.625 -0.453125q-1.109375 0 -1.625 0.40625q-0.515625 0.40625 -0.515625 0.984375q0 0.5 0.359375 0.828125q0.359375 0.328125 1.859375 0.671875q1.5 0.328125 2.0625 0.578125q0.8125 0.375 1.1875 0.953125q0.390625 0.578125 0.390625 1.328125q0 0.734375 -0.421875 1.390625q-0.421875 0.65625 -1.21875 1.03125q-0.796875 0.359375 -1.796875 0.359375q-1.265625 0 -2.125 -0.359375q-0.84375 -0.375 -1.
 328125 -1.109375q-0.484375 -0.75 -0.515625 -1.671875zm11.67099 1.96875l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.1873779 1.015625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm9.0633545 -2.171875l1.1875 0.140625q-0.28125 1.046875 -1.046875 1.625q-0.75 0.5625 -1.921875 0.5625q-1.484375 0 -2.359375 -0.90625q-0.859375 -0.921875 -0.859375 -2.5625q0 -1.703125 0.875 -2.640625q0.890625 -
 0.9375 2.28125 -0.9375q1.359375 0 2.203125 0.921875q0.859375 0.921875 0.859375 2.578125q0 0.109375 0 0.3125l-5.03125 0q0.0625 1.109375 0.625 1.703125q0.5625 0.59375 1.40625 0.59375q0.640625 0 1.078125 -0.328125q0.453125 -0.34375 0.703125 -1.0625zm-3.75 -1.84375l3.765625 0q-0.078125 -0.859375 -0.4375 -1.28125q-0.546875 -0.65625 -1.40625 -0.65625q-0.796875 0 -1.328125 0.53125q-0.53125 0.515625 -0.59375 1.40625zm10.943726 3.1875q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234
 375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.1156006 3.390625l0 -6.734375l1.015625 0l0 0.9375q0.328125 -0.5 0.84375 -0.796875q0.53125 -0.296875 1.203125 -0.296875q0.75 0 1.21875 0.3125q0.484375 0.3125 0.6875 0.859375q0.796875 -1.171875 2.078125 -1.171875q1.0 0 1.53125 0.5625q0.546875 0.546875 0.546875 1.703125l0 4.625l-1.125 0l0 -4.25q0 -0.6875 -0.109375 -0.984375q-0.109375 -0.296875 -0.40625 -0.484375q-0.296875 -0.1875 -0.6875 -0.1875q-
 0.71875 0 -1.1875 0.484375q-0.46875 0.46875 -0.46875 1.5l0 3.921875l-1.140625 0l0 -4.375q0 -0.765625 -0.28125 -1.140625q-0.28125 -0.390625 -0.90625 -0.390625q-0.484375 0 -0.890625 0.265625q-0.40625 0.25 -0.59375 0.734375q-0.1875 0.484375 -0.1875 1.40625l0 3.5l-1.140625 0zm13.6180725 0l0 -8.203125l-3.0625 0l0 -1.109375l7.375 0l0 1.109375l-3.078125 0l0 8.203125l-1.234375 0zm10.016357 -0.828125q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.406
 25 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm2.6624756 1.375l1.125 -0.171875q0.09375 0.671875 0.53125 1.046875q0.4375 0.359375 1.21875 0.359375q0.78125 0 1.15625 -0.3125q0.390625 -0.328125 0.390625 -0.765625q0 -0.390625 -0.34375 -0.609375q-0.234375 -0.15625 -1.171875 -0.390625q-1.25 -0.3125 -1.734375 -0.546875q-0.484375 -0.234375 -0.734375 -0.640625q-0.25 -0.40625 -0.25 -0.90625q0 -0.453125 0.203125 -0.828125q0.203125 -0.390625 0.5625 -0.640625q0.265625 -0.203125 0.71875 -0
 .328125q0.46875 -0.140625 1.0 -0.140625q0.78125 0 1.375 0.234375q0.609375 0.21875 0.890625 0.609375q0.296875 0.390625 0.40625 1.046875l-1.125 0.15625q-0.078125 -0.53125 -0.4375 -0.8125q-0.359375 -0.296875 -1.03125 -0.296875q-0.78125 0 -1.125 0.265625q-0.34375 0.25 -0.34375 0.609375q0 0.21875 0.140625 0.390625q0.140625 0.1875 0.4375 0.3125q0.171875 0.0625 1.015625 0.28125q1.21875 0.328125 1.6875 0.53125q0.484375 0.203125 0.75 0.609375q0.28125 0.390625 0.28125 0.96875q0 0.578125 -0.34375 1.078125q-0.328125 0.5 -0.953125 0.78125q-0.625 0.28125 -1.421875 0.28125q-1.3125 0 -2.0 -0.546875q-0.6875 -0.546875 -0.875 -1.625zm7.1171875 2.015625l0 -9.3125l1.140625 0l0 5.3125l2.703125 -2.734375l1.484375 0l-2.578125 2.5l2.84375 4.234375l-1.40625 0l-2.234375 -3.453125l-0.8125 0.78125l0 2.671875l-1.140625 0zm10.367035 2.578125l0 -9.3125l1.03125 0l0 0.875q0.375 -0.515625 0.828125 -0.765625q0.46875 -0.265625 1.140625 -0.265625q0.859375 0 1.515625 0.453125q0.65625 0.4375 0.984375 1.25q0.34375 0.796875
  0.34375 1.765625q0 1.03125 -0.375 1.859375q-0.359375 0.828125 -1.078125 1.28125q-0.703125 0.4375 -1.484375 0.4375q-0.5625 0 -1.015625 -0.234375q-0.453125 -0.25 -0.75 -0.625l0 3.28125l-1.140625 0zm1.03125 -5.90625q0 1.296875 0.53125 1.921875q0.53125 0.625 1.265625 0.625q0.765625 0 1.3125 -0.640625q0.546875 -0.65625 0.546875 -2.0q0 -1.296875 -0.53125 -1.9375q-0.53125 -0.640625 -1.265625 -0.640625q-0.734375 0 -1.296875 0.6875q-0.5625 0.671875 -0.5625 1.984375zm10.771851 2.5q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.35
 9375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.0999756 3.390625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm6.95401 -1.015625l0.15625 1.0q-0.
 484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029724 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm5.46109 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2030029 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 
 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm2.539215 -3.375q0 -1.875 1.03125 -2.765625q0.875 -0.75 2.125 -0.75q1.390625 0 2.265625 0.90625q0.890625 0.90625 0.890625 2.515625q0 1.296875 -0.390625 2.046875q-0.390625 0.75 -1.140625 1.171875q-0.75 0.40625 -1.625 0.40625q-1.421875 0 -2.296875 -0.90625q-0.859375 -0.90625 -0.859375 -2.625zm1.171875 0q0 1.296875 0.5625 1.953125q0.5625 0.640625 1.421875 0.640625q0.84375 0 1.40625 -0.640625q0.578125 -0.65625 0.578125 -1.984375q0 -1.25 -0.578125 -1.890625q-0.5625 -0.65625 -1.40625 -0.65625q-0.859375 0 -1.421875 0.640625q-0.5625 0.640625 -0.5625 1.9375zm6.6624756 3.375l0 -6.734375l1.03125 0l0 0.953125q0.734375 -1.109375 2.140625 -1.109375q0.609375 0 1.109375 0.21875q0.515625 0.21875 0.765625 0.578125q0.265625 0.34375 0.359375 0.84375q0.0625 0.3125 0.0625 1.109375l0 4.140625l-1.140625 0l0 -4.09375q0 -0.703125 -0.140625 -1.046875q-0.125 -0.34375 -0.46875 -0.546875q-0.328125 -0.21875 -0.78125 -0.21875q-0.734375 0 -1.265625 0.46875q-0.5
 3125 0.453125 -0.53125 1.75l0 3.6875l-1.140625 0zm16.802948 -1.09375l0 1.09375l-6.15625 0q-0.015625 -0.40625 0.140625 -0.796875q0.234375 -0.625 0.75 -1.234375q0.515625 -0.609375 1.5 -1.40625q1.515625 -1.25 2.046875 -1.96875q0

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/img/0.7.0/learn/documentation/container/job-flow.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/container/job-flow.png b/docs/img/0.7.0/learn/documentation/container/job-flow.png
deleted file mode 100644
index 046f9e3..0000000
Binary files a/docs/img/0.7.0/learn/documentation/container/job-flow.png and /dev/null differ


[2/7] SAMZA-7: Rewrite Container section of docs to bring it up-to-date. Reviewed by Jakob Homan.

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/container/state-management.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/state-management.md b/docs/learn/documentation/0.7.0/container/state-management.md
index aa418ff..11a9f87 100644
--- a/docs/learn/documentation/0.7.0/container/state-management.md
+++ b/docs/learn/documentation/0.7.0/container/state-management.md
@@ -3,196 +3,148 @@ layout: page
 title: State Management
 ---
 
-One of the more interesting aspects of Samza is the ability for tasks to store data locally and execute rich queries on this data.
+One of the more interesting features of Samza is stateful stream processing. Tasks can store and query data through APIs provided by Samza. That data is stored on the same machine as the stream task; compared to connecting over the network to a remote database, Samza's local state allows you to read and write large amounts of data with better performance. Samza replicates this state across multiple machines for fault-tolerance (described in detail below).
 
-Of course simple filtering or single-row transformations can be done without any need for collecting state. A simple analogy to SQL may make this more obvious. The select- and where-clauses of a SQL query don't usually require state: these can be executed a row at a time on input data and maintain state between rows. The rest of SQL, multi-row aggregations and joins, require accumulating state between rows. Samza doesn't provide a high-level language like SQL but it does provide lower-level primitives that make streaming aggregation and joins and other stateful processing easy to implement.
+Some stream processing jobs don't require state: if you only need to transform one message at a time, or filter out messages based on some condition, your job can be simple. Every call to your task's [process method](../api/overview.html) handles one incoming message, and each message is independent of all the other messages.
 
-Let's dive into how this works and why it is useful.
+However, being able to maintain state opens up many possibilities for sophisticated stream processing jobs: joining input streams, grouping messages and aggregating groups of messages. By analogy to SQL, the *select* and *where* clauses of a query are usually stateless, but *join*, *group by* and aggregation functions like *sum* and *count* require state. Samza doesn't yet provide a higher-level SQL-like language, but it does provide lower-level primitives that you can use to implement streaming aggregation and joins.
 
 ### Common use cases for stateful processing
 
-First, let's look at some simplistic examples of stateful stream processing that might be seen on a consumer website. Later in this document we'll go through specific details of using Samza's built-in key-value storage capabilities to implement each of these applications, but for now it is enough just to see some examples of the kind of applications that tend to need to manage state.
+First, let's look at some simple examples of stateful stream processing that might be seen in the backend of a consumer website. Later in this page we'll discuss how to implement these applications using Samza's built-in key-value storage capabilities.
 
-##### Windowed aggregation
+#### Windowed aggregation
 
-Example: Counting the number of page views for each user per hour
+*Example: Counting the number of page views for each user per hour*
 
-This kind of windowed processing is common for ranking and relevance, detecting "trending topics", as well as simple real-time reporting and monitoring. For small windows one can just maintain the aggregate in memory and manually commit the task position only at window boundaries. However this means we have to recover up to a full window on fail-over, which will be very slow for large windows due to the amount of reprocessing. For large (or infinite!) windows it is better to make the in-process aggregation fault-tolerant rather than try to recompute it.
+In this case, your state typically consists of a number of counters which are incremented when a message is processed. The aggregation is typically limited to a time window (e.g. 1 minute, 1 hour, 1 day) so that you can observe changes of activity over time. This kind of windowed processing is common for ranking and relevance, detecting "trending topics", as well as real-time reporting and monitoring.
 
-##### Table-table join
+The simplest implementation keeps this state in memory (e.g. a hash map in the task instances), and writes it to a database or output stream at the end of every time window. However, you need to consider what happens when a container fails and your in-memory state is lost. You might be able to restore it by processing all the messages in the current window again, but that might take a long time if the window covers a long period of time. Samza can speed up this recovery by making the state fault-tolerant rather than trying to recompute it.
 
-Example: Join a table of user profiles to a table of user\_settings by user\_id and emit a "materialized view" of the joined stream
+#### Table-table join
 
-This example is somewhat simplistic: one might wonder why you would want to join two tables in a stream processing system. However real-life examples are often far more complex then what would normally be considered the domain of materialized views over tables. Consider a few examples of real-time data normalization:
+*Example: Join a table of user profiles to a table of user settings by user\_id and emit the joined stream*
+
+You might wonder: does it make sense to join two tables in a stream processing system? It does if your database can supply a log of all the changes in the database. There is a [duality between a database and a changelog stream](http://engineering.linkedin.com/distributed-systems/log-what-every-software-engineer-should-know-about-real-time-datas-unifying): you can publish every data change to a stream, and if you consume the entire stream from beginning to end, you can reconstruct the entire contents of the database. Samza is designed for data processing jobs that follow this philosophy.
+
+If you have changelog streams for several database tables, you can write a stream processing job which keeps the latest state of each table in a local key-value store, where you can access it much faster than by making queries to the original database. Now, whenever data in one table changes, you can join it with the latest data for the same key in the other table, and output the joined result.
+
+There are several real-life examples of data normalization which essentially work in this way:
 
 * E-commerce companies like Amazon and EBay need to import feeds of merchandise from merchants, normalize them by product, and present products with all the associated merchants and pricing information.
-* Web search requires building a crawler which creates essentially a [table of web page contents](http://labs.yahoo.com/files/YahooWebmap.pdf) and joins on all the relevance attributes such as page CTR or pagerank.
+* Web search requires building a crawler which creates essentially a [table of web page contents](http://labs.yahoo.com/files/YahooWebmap.pdf) and joins on all the relevance attributes such as click-through ratio or pagerank.
 * Social networks take feeds of user-entered text and need to normalize out entities such as companies, schools, and skills.
 
-Each of these use cases is a massively complex data normalization problem that can be thought of as constructing a very complex materialized view over many input tables.
+Each of these use cases is a massively complex data normalization problem that can be thought of as constructing a materialized view over many input tables. Samza can help implement such data processing pipelines robustly.
 
-##### Stream-table join
+#### Stream-table join
 
-Example: Join user region information on to a stream of page views to create an augmented stream of page view with region.
+*Example: Augment a stream of page view events with the user's ZIP code (perhaps to allow aggregation by zip code in a later stage)*
 
-Joining side-information to a real-time feed is a classic use for stream processing. It's particularly common in advertising, relevance ranking, fraud detection and other domains. Activity data such as page views are generally captured with only a few primary keys, the additional attributes about the viewer and viewed items that are needed for processing need to joined on after-the-fact.
+Joining side-information to a real-time feed is a classic use for stream processing. It's particularly common in advertising, relevance ranking, fraud detection and other domains. Activity events such as page views generally only include a small number of attributes, such as the ID of the viewer and the viewed items, but not detailed attributes of the viewer and the viewed items, such as the ZIP code of the user. If you want to aggregate the stream by attributes of the viewer or the viewed items, you need to join with the users table or the items table respectively.
 
-##### Stream-stream join
+In data warehouse terminology, you can think of the raw event stream as rows in the central fact table, which needs to be joined with dimension tables so that you can use attributes of the dimensions in your analysis.
 
-Example: Join a stream of ad clicks to a stream of ad views to link the ad view that lead to the click
+#### Stream-stream join
 
-This is the classic stream join for "nearly aligned" streams. If the events that need to be joined arrive in a limited window it may be possible to buffer unjoined events in memory. Obviously this will be only approximate: any in-flight items will be lost if a machine crashes. However for more exact results, or to handle a very large window of misalignment, stateful processing is needed.
+*Example: Join a stream of ad clicks to a stream of ad impressions (to link the information on when the ad was shown to the information on when it was clicked)*
 
-##### More
+A stream join is useful for "nearly aligned" streams, where you expect to receive related events on several input streams, and you want to combine them into a single output event. You cannot rely on the events arriving at the stream processor at the same time, but you can set a maximum period of time over which you allow the events to be spread out.
 
-Of course there are infinite variations on joins and aggregations, but most amount to essentially variations and combinations of the above patterns.
+In order to perform a join between streams, your job needs to buffer events for the time window over which you want to join. For short time windows, you can do this in memory (at the risk of losing events if the machine fails). You can also use Samza's state store to buffer events, which supports buffering more messages than you can fit in memory.
+
+#### More
+
+There are many variations of joins and aggregations, but most are essentially variations and combinations of the above patterns.
 
 ### Approaches to managing task state
 
-So how do systems support this kind of stateful processing? We'll lead in by describing what we have seen in other systems and then describe what Samza does.
+So how do systems support this kind of stateful processing? We'll lead in by describing what we have seen in other stream processing systems, and then describe what Samza does.
 
 #### In-memory state with checkpointing
 
-A simple approach, common in academic stream processing systems, is to periodically save out the state of the task's in-memory data. S4's [state management](http://incubator.apache.org/s4/doc/0.6.0/fault_tolerance) implements this approach&mdash;tasks implement Java's serializable interface and are periodically serialized using java serialization to save out copies of the processor state.
+A simple approach, common in academic stream processing systems, is to periodically save the task's entire in-memory data to durable storage. This approach works well if the in-memory state consists of only a few values. However, you have to store the complete task state on each checkpoint, which becomes increasingly expensive as task state grows. Unfortunately, many non-trivial use cases for joins and aggregation have large amounts of state &mdash; often many gigabytes. This makes full dumps of the state impractical.
 
-This approach works well enough if the in-memory state consists of only a few values. However since you have to save out the complete task state on each save this will become increasingly expensive as task state grows. Unfortunately most use cases we have seen revolve around joins and aggregation and so have large amounts of state&mdash;often many gigabytes. This makes periodic full dumps extremely impractical. Some academic systems handle this case by having the tasks produce "diffs" in addition to full checkpoints. However this requires a great deal of complexity in the task to track what has changed and efficiently produce a compact diff of changes.
+Some academic systems produce *diffs* in addition to full checkpoints, which are smaller if only some of the state has changed since the last checkpoint. [Storm's Trident abstraction](../comparisons/storm.html) similarly keeps an in-memory cache of state, and periodically writes any changes to a remote store such as Cassandra. However, this optimization only helps if most of the state remains unchanged. In some use cases, such as stream joins, it is normal to have a lot of churn in the state, so this technique essentially degrades to making a remote database request for every message (see below).
 
 #### Using an external store
 
-In the absence of built-in support a common pattern for stateful processing is to push any state that would be accumulated between rows into an external database or key-value store. The database holds aggregates or the dataset being queried to enrich the incoming stream. You get something that looks like this:
+Another common pattern for stateful processing is to store the state in an external database or key-value store. Conventional database replication can be used to make that database fault-tolerant. The architecture looks something like this:
 
 ![state-kv-store](/img/0.7.0/learn/documentation/container/stream_job_and_db.png)
 
-Samza allows this style of processing (nothing will stop you from querying a remote database or service from your job) but also supports stateful processing natively in a way we think is often superior.
-
-#### The problems of remote stores
+Samza allows this style of processing &mdash; there is nothing to stop you querying a remote database or service within your job. However, there are a few reasons why a remote database can be problematic for stateful stream processing:
 
-To understand why this is useful let's first understand some of the drawbacks of making remote queries in a stream processing job:
-
-1. **Performance**: The first major drawback of making remote queries is that they are slow and expensive. For example, a Kafka stream can deliver hundreds of thousands or even millions of messages per second per CPU core because it transfers large chunks of data at a time. But a remote database query is a more expensive proposition. Though the database may be partitioned and scalable this partitioning doesn't match the partitioning of the job into tasks so batching becomes much less effective. As a result you would expect to get a few thousand queries per second per core for remote requests. This means that adding a processing stage that uses an external database will often reduce the throughput by several orders of magnitude.
-1. **Isolation**: If your database or service is also running live processing, mixing in asynchronous processing can be quite dangerous. A scalable stream processing system can run with very high parallelism. If such a job comes down (say for a code push) it queues up data for processing, when it restarts it will potentially have a large backlog of data to process. Since the job may actually have very high parallelism this can result in huge load spikes, many orders of magnitude higher than steady state load. If this load is mixed with live queries (i.e. the queries used to build web pages or render mobile UI or anything else that has a user waiting on the other end) then you may end up causing a denial-of-service attack on your live service.
-1. **Query Capabilities**: Many scalable databases expose very limited query interfaces--only supporting simple key-value lookups. Doing the equivalent of a "full table scan" or rich traversal may not be practical in this model.
-1. **Correctness**: If your task keeps counts or otherwise modifies state in a remote store how is this rolled back if the task fails? 
-
-Where these issues become particularly problematic is when you need to reprocess data. Your output, after all, is a combination of your code and your input&mdash;when you change your code you often want to reprocess input to recreate the output state with the new improved code. This is generally quite reasonable for pure stream processing jobs, but generally impractical for performance and isolation reasons for jobs that make external queries.
+1. **Performance**: Making database queries over a network is slow and expensive. A Kafka stream can deliver hundreds of thousands or even millions of messages per second per CPU core to a stream processor, but if you need to make a remote request for every message you process, your throughput is likely to drop by 2-3 orders of magnitude. You can somewhat mitigate this with careful caching of reads and batching of writes, but then you're back to the problems of checkpointing, discussed above.
+2. **Isolation**: If your database or service also serves requests to users, it can be dangerous to use the same database with a stream processor. A scalable stream processing system can run with very high throughput, and easily generates a huge amount of load (for example when catching up on a queue backlog). If you're not very careful, you may cause a denial-of-service attack on your own database, and cause problems for interactive requests from users.
+3. **Query Capabilities**: Many scalable databases expose very limited query interfaces (e.g. only supporting simple key-value lookups), because the equivalent of a "full table scan" or rich traversal would be too expensive. Stream processes are often less latency-sensitive, so richer query capabilities would be more feasible.
+4. **Correctness**: When a stream processor fails and needs to be restarted, how is the database state made consistent with the processing task? For this purpose, some frameworks such as [Storm](../comparisons/storm.html) attach metadata to database entries, but it needs to be handled carefully, otherwise the stream process generates incorrect output.
+5. **Reprocessing**: Sometimes it can be useful to re-run a stream process on a large amount of historical data, e.g. after updating your processing task's code. However, the issues above make this impractical for jobs that make external queries.
 
 ### Local state in Samza
 
-Samza allows tasks to maintain persistent, mutable, queryable state that is physically co-located with each task. The state is highly available: in the event of a task failure it will be restored when the task fails over to another machine.
-
-You can think of this as taking the remote table out of the remote database and physically partitioning it up and co-locating these partitions with the tasks. This looks something like this:
-
-![state-local](/img/0.7.0/learn/documentation/container/stateful_job.png)
+Samza allows tasks to maintain state in a way that is different from the approaches described above:
 
-Note that now the state is physically on the same machine as the tasks, and each task has access only to its local partition. However the combination of stateful tasks with the normal partitioning capabilities Samza offers makes this a very general feature: you just repartition on the key by which you want to split your processing and then you have full local access to the data within storage in that partition.
+* The state is stored on disk, so the job can maintain more state than would fit in memory.
+* It is stored on the same machine as the processing task, to avoid the performance problems of making database queries over the network.
+* Each job has its own datastore, to avoid the isolation problems of a shared database (if you make an expensive query, it affects only the current task, nobody else).
+* Different storage engines can be plugged in, enabling rich query capabilities.
+* The state is continuously replicated, enabling fault tolerance without the problems of checkpointing large amounts of state.
 
-Let's look at how this addresses the problems of the remote store:
+Imagine you take a remote database, partition it to match the number of tasks in the stream processing job, and co-locate each partition with its task. The result looks like this:
 
-1. This fixes the performance issues of remote queries because the data is now local, what would otherwise be a remote query may now just be a lookup against local memory or disk (we ship a [LevelDB](https://code.google.com/p/leveldb)-based store which is described in detail below).
-1. The isolation issue goes away as well as the queries are executed against the same servers the job runs against and this computation is not intermixed with live service calls.
-1. Data is now local so any kind of data-intensive processing, scans, and filtering is now possible.
-1. Since the state changes are themselves modeled as a stream the store can abide by the same delivery and fault-tolerance guarantees that Samza gives tasks.
+![state-local](/img/0.7.0/learn/documentation/container/stateful_job.png)
 
-This isn't always the right pattern to follow, it has a few drawbacks too.
-1. If the data is very large then storing it with each task that uses the data may use more space.
-1. As the per-container data size grows so too will the restore time for a failed task (50Mb/sec is a reasonable restore time to expect).
+If a machine fails, all the tasks running on that machine and their database partitions are lost. In order to make them highly available, all writes to the database partition are replicated to a durable changelog (typically Kafka). Now, when a machine fails, we can restart the tasks on another machine, and consume this changelog in order to restore the contents of the database partition.
 
-However we find that the local state approach is the best more often than not, and, of course, nothing prevents the use of external storage when needed.
+Note that each task only has access to its own database partition, not to any other task's partition. This is important: when you scale out your job by giving it more computing resources, Samza needs to move tasks from one machine to another. By giving each task its own state, tasks can be relocated without affecting the job's operation. If necessary, you can repartition your streams so that all messages for a particular database partition are routed to the same task instance.
 
-### Databases as input streams
+[Log compaction](http://kafka.apache.org/documentation.html#compaction) runs in the background on the changelog topic, and ensures that the changelog does not grow indefinitely. If you overwrite the same value in the store many times, log compaction keeps only the most recent value, and throws away any old values in the log. If you delete an item from the store, log compaction also removes it from the log. With the right tuning, the changelog is not much bigger than the database itself.
 
-In cases where we were querying the external database on each input message we can transform this to local processing by instead transforming the database into a stream of row changes. These changes can be taken as input by a task, stored, and queried against just as the remote database would be.
+With this architecture, Samza allows tasks to maintain large amounts of fault-tolerant state, at a performance that is almost as good as a pure in-memory implementation. There are just a few limitations:
 
-But how can you get such a stream? Many databases such Oracle, HBase, MySQL, and MongoDB offer built-in support for directly capturing changes. If not this can be done by publishing a stream of changes to Kafka or by implementing our [pluggable stream interface](streams.html) to directly poll the database for changes (say by some last_modified timestamp). You want this to be done in a way that you can reset the offset or timestamp back to zero to replay the current state of the database as changes if you need to reprocess. If this stream capture is efficient enough you can often avoid having changelogs for your tasks and simply replay them from the source when they fail.
+* If you have some data that you want to share between tasks (across partition boundaries), you need to go to some additional effort to repartition and distribute the data. Each task will need its own copy of the data, so this may use more space overall.
+* When a container is restarted, it can take some time to restore the data in all of its state partitions. The time depends on the amount of data, the storage engine, your access patterns, and other factors. As a rule of thumb, 50&nbsp;MB/sec is a reasonable restore time to expect.
 
-A wonderful contribution would be a generic jdbc-based stream implementation for extracting changes from relational databases by modified date.
+Nothing prevents you from using an external database if you want to, but for many use cases, Samza's local state is a powerful tool for enabling stateful stream processing.
 
 ### Key-value storage
 
-Though the storage format is pluggable, we provide a key-value store implementation to tasks out-of-the-box that gives the usual put/get/delete/range queries. This is backed by a highly available "changelog" stream that provides fault-tolerance by acting as a kind of [redo log](http://en.wikipedia.org/wiki/Redo_log) for the task's state (we describe this more in the next section).
-
-This key-value storage engine is built on top of [LevelDB](https://code.google.com/p/leveldb) using a [LevelDB JNI API](https://github.com/fusesource/leveldbjni). LevelDB has several nice properties. First it maintains data outside the java heap which means it is immediately preferable to any simple approach using a hash table both because of memory-efficiency and to avoid GC. It will use an off-heap memory cache and when that is exhausted go to disk for lookups&mdash;so small data sets can be [very fast](https://code.google.com/p/leveldb) and non-memory-resident datasets, though slower, are still possible. It is [log-structured](http://www.igvita.com/2012/02/06/sstable-and-log-structured-storage-leveldb/) and writes can be performed at close to disk speeds. It also does built-in block compression which helps to reduce both I/O and memory usage.
-
-The nature of Samza's usage allows us to optimize this further. We add an optional "L1" LRU cache which is in-heap and holds deserialized rows. This cache is meant to be very small and lets us introduce several optimizations for both reads and writes.
-
-The cache is an "object" or "row" cache&mdash;that is it maintains the java objects stored with no transformation or serialization. This complements LevelDB's own block level caching well. Reads and writes both populate the cache, and reads on keys in the cache avoid the cost of deserialization for these very common objects.
-
-For writes the cache provides two benefits. Since LevelDB is itself really only a persistent "cache" in our architecture we do not immediately need to apply every write to the filesystem. We can batch together a few hundred writes and apply them all at once. LevelDB heavily optimizes this kind of batch write. This does not impact consistency&mdash;a task always reads what it wrote (since it checks the cache first and is the only writer to its store). Secondly the cache effectively deduplicates updates so that if multiple updates to the same key occur close together we can optimize away all but the final write to leveldb and the changelog. For example, an important use case is maintaining a small number of counters that are incremented on every input. A naive implementation would need to write out each new value to LevelDB as well as perhaps logging the change out to the changelog for the task. In the extreme case where you had only a single variable, x, incremented on each input, an
  uncached implementation would produce writes in the form "x=1", "x=2", "x=3", etc which is quite inefficient. This is overkill, we only need to flush to the changelog at [commit points](checkpointing.html) not on every write. This allows us to "deduplicate" the writes that go to leveldb and the changelog to just the final value before the commit point ("x=3" or whatever it happened to be).
-
-The combination of these features makes it possible to provide highly available processing that performs very close to memory speeds for small datasets yet still scales up to TBs of data (partitioned up across all the tasks).
-
-### Fault-tolerance
-
-As mentioned the actual local storage (i.e. LevelDB for key-value storage) is really just a cache. How can we ensure that this data is not lost when a machine fails and the tasks running on that machine have to be brought up on another machine (which, of course, doesn't yet have the local persistent state)?
-
-The answer is that Samza handles state as just another stream. There are two mechanisms for accomplishing this.
-
-The first approach is just to allow the task to replay one or more of its input streams to populate its store when it restarts. This works well if the input stream maintains the complete data (as a stream fed by a database table might) and if the input stream is fast enough to make this practical. This requires no framework support.
-
-However often the state that is stored is much smaller than the input stream (because it is an aggregation or projection of the original input streams). Or the input stream may not maintain a complete, replayable set of inputs (say for event logs). To support these cases we provide the ability to back the state of the store with a changelog stream. A changelog is just a stream to which the task logs each change to its state&mdash;i.e. the sequence of key-value pairs applied to the local store. Changelogs are co-partitioned with their tasks (so each task has its own stream partition for which it is the only writer).
+Any storage engine can be plugged into Samza, as described below. Out of the box, Samza ships with a key-value store implementation that is built on [LevelDB](https://code.google.com/p/leveldb) using a [JNI API](https://github.com/fusesource/leveldbjni).
 
-The changelogs are just normal streams&mdash;other downstream tasks can subscribe to this state and use it. And it turns out that very often the most natural way to represent the output of a job is as the changelog of its task (we'll show some examples in a bit).
+LevelDB has several nice properties. Its memory allocation is outside of the Java heap, which makes it more memory-efficient and less prone to garbage collection pauses than a Java-based storage engine. It is very fast for small datasets that fit in memory; datasets larger than memory are slower but still possible. It is [log-structured](http://www.igvita.com/2012/02/06/sstable-and-log-structured-storage-leveldb/), allowing very fast writes. It also includes support for block compression, which helps to reduce I/O and memory usage.
 
-Of course a log of changes only grows over time so this would soon become impractical. Kafka has [log compaction](http://kafka.apache.org/documentation#compaction) which provides special support for this kind of use case, though. This feature allows Kafka to compact duplicate entries (i.e. multiple updates with the same key) in the log rather than just deleting old log segments. This feature is available since Kafka 0.8.1.
+Samza includes an additional in-memory caching layer in front of LevelDB, which avoids the cost of deserialization for frequently-accessed objects and batches writes. If the same key is updated multiple times in quick succession, the batching coalesces those updates into a single write. The writes are flushed to the changelog when a task [commits](checkpointing.html).
 
-The Kafka brokers scale well up to terabytes of data per machine for changelogs as for other topics. Log compaction proceeds at about 50MB/sec/core or whatever the I/O limits of the broker are.
-
-### Other storage engines
-
-One interesting aspect of this design is that the fault-tolerance mechanism is completely decoupled from the query apis the storage engine provides to the task or the way it stores data on disk or in memory. We have provided a key-value index with Samza, but you can easily implement and plug-in storage engines that are optimized for other types of queries and plug them in to our fault tolerance mechanism to provide different query capabilities to your tasks.
-
-Here are a few examples of storage engine types we think would be interesting to pursue in the future (patches accepted!):
-
-##### Persistent heap
-
-A common operation in stream processing is to maintain a running top-N. There are two primary applications of this. The first is ranking items over some window. The second is performing a "bounded sort" operation to transform a nearly sorted input stream into a totally sorted output stream. This occurs when dealing with a data stream where the order is by arrival and doesn't exactly match the source timestamp (for example log events collected across many machines).
-
-##### Sketches
-
-Many applications don't require exact results and for these [approximate algorithms](http://infolab.stanford.edu/~ullman/mmds/ch4.pdf) such as [bloom filters](http://en.wikipedia.org/wiki/Bloom_filter) for set membership, [hyperloglog](http://research.google.com/pubs/pub40671.html) for counting distinct keys, and a multitude of algorithms for quantile and histogram approximation.
-
-These algorithms are inherently approximate but good algorithms give a strong bound on the accuracy of the approximation. This obviously doesn't carry over well to the case where the task can crash and lose all state. By logging out the changes to the structure we can ensure it is restored on fail-over. The nature of sketch algorithms allows significant opportunity for optimization in the form of logging. 
-
-##### Inverted index
-
-Inverted indexes such as is provided by [Lucene](http://lucene.apache.org) are common for text matching and other applications that do matching and ranking with selective queries and large result sets. 
-
-##### More
-
-There are a variety of other storage engines that could be useful:
-
-* For small datasets logged, in-memory collections may be ideal.
-* Specialized data structures for graph traversal are common.
-* Many applications are doing OLAP-like aggregations on their input. It might be possible to optimize these kinds of dimensional summary queries.
-
-### Using the key-value store
-
-In this section we will give a quick tutorial on configuring and using the key-value store.
-
-To declare a new store for usage you add the following to your job config:
+To use a key-value store in your job, add the following to your job config:
 
     # Use the key-value store implementation for a store called "my-store"
     stores.my-store.factory=org.apache.samza.storage.kv.KeyValueStorageEngineFactory
 
-    # Log changes to the store to an output stream for restore
-    # If no changelog is specified the store will not be logged (but you can still rebuild off your input streams)
-    stores.my-store.changelog=kafka.my-stream-name
+    # Use the Kafka topic "my-store-changelog" as the changelog stream for this store.
+    # This enables automatic recovery of the store after a failure. If you don't
+    # configure this, no changelog stream will be generated.
+    stores.my-store.changelog=kafka.my-store-changelog
 
-    # The serialization format to use
+    # Encode keys and values in the store as UTF-8 strings.
+    serializers.registry.string.class=org.apache.samza.serializers.StringSerdeFactory
     stores.my-store.key.serde=string
     stores.my-store.msg.serde=string
 
-Here is some simple example code that only writes to the store:
+See the [serialization section](serialization.html) for more information on the *serde* options.
+
+Here is a simple example that writes every incoming message to the store:
 
     public class MyStatefulTask implements StreamTask, InitableTask {
       private KeyValueStore<String, String> store;
       
       public void init(Config config, TaskContext context) {
-        this.store = (KeyValueStore<String, String>) context.getStore("store");
+        this.store = (KeyValueStore<String, String>) context.getStore("my-store");
       }
 
-      public void process(IncomingMessageEnvelope envelope, MessageCollector collector, TaskCoordinator coordinator) {
-        System.out.println("Adding " + envelope.getKey() + " => " + envelope.getMessage() + " to the store.");
+      public void process(IncomingMessageEnvelope envelope,
+                          MessageCollector collector,
+                          TaskCoordinator coordinator) {
         store.put((String) envelope.getKey(), (String) envelope.getMessage());
       }
     }
@@ -208,79 +160,84 @@ Here is the complete key-value store API:
       KeyValueIterator<K,V> all();
     }
 
-Here is a list of additional configurations accepted by the key-value store along with their default values:
+Here is a list of additional configurations accepted by the key-value store, along with their default values:
 
     # The number of writes to batch together
     stores.my-store.write.batch.size=500
 
-    # The total number of objects to cache in the "L1" object cache. This must be at least as large as the batch.size.
+    # The number of objects to keep in Samza's cache (in front of LevelDB).
+    # This must be at least as large as write.batch.size.
     # A cache size of 0 disables all caching and batching.
     stores.my-store.object.cache.size=1000
 
-    # The size of the off-heap leveldb block cache in bytes, per container. If you have multiple tasks within
-    # one container, each task is given a proportional share of this cache.
+    # The size of the off-heap leveldb block cache in bytes, per container.
+    # If you have multiple tasks within one container, each task is given a
+    # proportional share of this cache.
     stores.my-store.container.cache.size.bytes=104857600
 
-    # The amount of memory leveldb uses for buffering writes before they are written to disk, per container.
-    # If you have multiple tasks within one container, each task is given a proportional share of this buffer.
+    # The amount of memory leveldb uses for buffering writes before they are
+    # written to disk, per container. If you have multiple tasks within one
+    # container, each task is given a proportional share of this buffer.
     # This setting also determines the size of leveldb's segment files.
     stores.my-store.container.write.buffer.size.bytes=33554432
 
     # Enable block compression? (set compression=none to disable)
     stores.my-store.leveldb.compression=snappy
 
-    # If compression is enabled, leveldb groups approximately this many uncompressed bytes into one compressed block.
-    # You probably don't need to change this unless you are a compulsive fiddler.
+    # If compression is enabled, leveldb groups approximately this many
+    # uncompressed bytes into one compressed block. You probably don't need
+    # to change this unless you are a compulsive fiddler.
     stores.my-store.leveldb.block.size.bytes=4096
 
 ### Implementing common use cases with the key-value store
 
-Let's look at how you can address some of the common use-cases we discussed before using the key-value storage engine.
+Earlier in this section we discussed some example use cases for stateful stream processing. Let's look at how each of these could be implemented using a key-value storage engine such as Samza's LevelDB.
 
-##### Windowed aggregation
+#### Windowed aggregation
 
-Example: Counting the number of page views for each user per hour
+*Example: Counting the number of page views for each user per hour*
 
-Implementation: We have two processing stages. The first partitions the input data by user id (if it's already partitioned by user id, which would be reasonable, you can skip this), and the second stage does the counting. The job has a single store containing the mapping of user_id to the running count. Each new input record would cause the job to retrieve the current running count, increment it and write back the count. When the window is complete (i.e. the hour is over), we iterate over the contents of our store and emit the aggregates.
+Implementation: You need two processing stages.
 
-One thing to note is that this job effectively pauses at the hour mark to output its results. This is unusual for stream processing, but totally fine for Samza&mdash;and we have specifically designed for this case. Scans over the contents of the key-value store will be quite fast and input data will buffer while the job is doing this scanning and emitting aggregates.
+1. The first one re-partitions the input data by user ID, so that all the events for a particular user are routed to the same stream task. If the input stream is already partitioned by user ID, you can skip this.
+2. The second stage does the counting, using a key-value store that maps a user ID to the running count. For each new event, the job reads the current count for the appropriate user from the store, increments it, and writes it back. When the window is complete (e.g. at the end of an hour), the job iterates over the contents of the store and emits the aggregates to an output stream.
 
-##### Table-table join
+Note that this job effectively pauses at the hour mark to output its results. This is totally fine for Samza, as scanning over the contents of the key-value store is quite fast. The input stream is buffered while the job is doing this hourly work.
 
-Example: Join a table of user profiles to a table of user settings by user\_id and emit the joined stream
+#### Table-table join
 
-Implementation: The job subscribes to the change stream for user profiles and for user settings databases, both partitioned by user\_id. The job keeps a single key-value store keyed by user\_id containing the joined contents of profiles and settings. When a new record comes in from either stream it looks up the current value in its store and writes back the record with the appropriate fields updated (i.e. new profile fields if it was a profile update, and new settings fields if it was a settings update). The changelog of the store doubles as the output stream of the task.
+*Example: Join a table of user profiles to a table of user settings by user\_id and emit the joined stream*
 
-##### Table-stream join
+Implementation: The job subscribes to the change streams for the user profiles database and the user settings database, both partitioned by user\_id. The job keeps a key-value store keyed by user\_id, which contains the latest profile record and the latest settings record for each user\_id. When a new event comes in from either stream, the job looks up the current value in its store, updates the appropriate fields (depending on whether it was a profile update or a settings update), and writes back the new joined record to the store. The changelog of the store doubles as the output stream of the task.
 
-Example: Join user zip code to page view data (perhaps to allow aggregation by zip code in a later stage)
+#### Table-stream join
 
-Implementation: The job subscribes to the user profile stream and page view stream. Each time it gets a profile update it stores the zipcode keyed by user\_id. Each time a page view arrives it looks up the zip code for the user and emits the enriched page view + zipcode event.
+*Example: Augment a stream of page view events with the user's ZIP code (perhaps to allow aggregation by zip code in a later stage)*
 
-##### Stream-stream join
+Implementation: The job subscribes to the stream of user profile updates and the stream of page view events. Both streams must be partitioned by user\_id. The job maintains a key-value store where the key is the user\_id and the value is the user's ZIP code. Every time the job receives a profile update, it extracts the user's new ZIP code from the profile update and writes it to the store. Every time it receives a page view event, it reads the zip code for that user from the store, and emits the page view event with an added ZIP code field.
 
-Example: Join ad clicks to ad impressions by impression id (an impression is advertising terminology for the event that records the display of an ad)
+If the next stage needs to aggregate by ZIP code, the ZIP code can be used as the partitioning key of the job's output stream. That ensures that all the events for the same ZIP code are sent to the same stream partition.
 
-Note: In this example we are assuming that impressions are assigned a unique guid and this is present in both the original impression event and any subsequent click. In the absence of this the business logic for choosing the join could be substituted for the simple lookup.
+#### Stream-stream join
 
-Implementation: Partition the ad click and ad impression streams by the impression id or user id. The task keeps a store of unmatched clicks and unmatched impressions. When a click comes in we try to find its matching impression in the impression store, and vice versa. If a match is found emit the joined pair and delete the entry. If no match is found store the event to wait for a match. Since this is presumably a left outer join (i.e. every click has a corresponding impression but not vice versa) we will periodically scan the impression table and delete old impressions for which no click arrived.
+*Example: Join a stream of ad clicks to a stream of ad impressions (to link the information on when the ad was shown to the information on when it was clicked)*
 
-### Implementing storage engines
+In this example we assume that each impression of an ad has a unique identifier, e.g. a UUID, and that the same identifier is included in both the impression and the click events. This identifier is used as the join key.
 
-We mentioned that the storage engine interface was pluggable. Of course you can use any data structure you like in your task provided you can repopulate it off your inputs on failure. However to plug into our changelog infrastructure you need to implement a generic StorageEngine interface that handles restoring your state on failure and ensures that data is flushed prior to commiting the task position.
+Implementation: Partition the ad click and ad impression streams by the impression ID or user ID (assuming that two events with the same impression ID always have the same user ID). The task keeps two stores, one containing click events and one containing impression events, using the impression ID as key for both stores. When the job receives a click event, it looks for the corresponding impression in the impression store, and vice versa. If a match is found, the joined pair is emitted and the entry is deleted. If no match is found, the event is written to the appropriate store. Periodically the job scans over both stores and deletes any old events that were not matched within the time window of the join.
 
-The above code shows usage of the key-value storage engine, but it is not too hard to implement an alternate storage engine. To do so, you implement methods to restore the contents of the store from a stream, flush any cached content on commit, and close the store:
+### Other storage engines
 
-    public interface StorageEngine {
-      void restore(Iterator<IncomingMessageEnvelope> envelopes);
-      void flush();
-      void stop();
-    }
+Samza's fault-tolerance mechanism (sending a local store's writes to a replicated changelog) is completely decoupled from the storage engine's data structures and query APIs. While a key-value storage engine is good for general-purpose processing, you can easily add your own storage engines for other types of queries by implementing the [StorageEngine](../api/javadocs/org/apache/samza/storage/StorageEngine.html) interface. Samza's model is especially amenable to embedded storage engines, which run as a library in the same process as the stream task. 
 
-The user specifies the type of storage engine they want by passing in a factory for that store in their configuration.
+Some ideas for other storage engines that could be useful: a persistent heap (for running top-N queries), [approximate algorithms](http://infolab.stanford.edu/~ullman/mmds/ch4.pdf) such as [bloom filters](http://en.wikipedia.org/wiki/Bloom_filter) and [hyperloglog](http://research.google.com/pubs/pub40671.html), or full-text indexes such as [Lucene](http://lucene.apache.org). (Patches accepted!)
 
 ### Fault tolerance semantics with state
 
-Samza currently only supports at-least-once delivery guarantees in the presence of failure (this is sometimes referred to as "guaranteed delivery"). This means messages are not lost but if a task fails some messages may be redelivered. The guarantee holds from the commit point of the task which records the position from which the task will restart on failure (the user can either force a commit at convenient points or the framework will by default do this at regular intervals). This is true for both input, output, and changelog streams. This is a fairly weak guarantee&mdash;duplicates can give incorrect results in counts, for example. We have a plan to extend this to exact semantics in the presence of failure which we will include in a future release.
+As discussed in the section on [checkpointing](checkpointing.html), Samza currently only supports at-least-once delivery guarantees in the presence of failure (this is sometimes referred to as "guaranteed delivery"). This means that if a task fails, no messages are lost, but some messages may be redelivered.
+
+For many of the stateful processing use cases discussed above, this is not a problem: if the effect of a message on state is idempotent, it is safe for the same message to be processed more than once. For example, if the store contains the ZIP code for each user, then processing the same profile update twice has no effect, because the duplicate update does not change the ZIP code.
+
+However, for non-idempotent operations such as counting, at-least-once delivery guarantees can give incorrect results. If a Samza task fails and is restarted, it may double-count some messages that were processed shortly before the failure. We are planning to address this limitation in a future release of Samza.
 
 ## [Metrics &raquo;](metrics.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/container/streams.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/streams.md b/docs/learn/documentation/0.7.0/container/streams.md
index 0f18b34..4e0f065 100644
--- a/docs/learn/documentation/0.7.0/container/streams.md
+++ b/docs/learn/documentation/0.7.0/container/streams.md
@@ -3,152 +3,111 @@ layout: page
 title: Streams
 ---
 
-The [TaskRunner](task-runner.html) reads and writes messages using the SystemConsumer and SystemProducer interfaces.
+The [samza container](samza-container.html) reads and writes messages using the [SystemConsumer](../api/javadocs/org/apache/samza/system/SystemConsumer.html) and [SystemProducer](../api/javadocs/org/apache/samza/system/SystemProducer.html) interfaces. You can integrate any message broker with Samza by implementing these two interfaces.
 
-```
-public interface SystemConsumer {
+    public interface SystemConsumer {
+      void start();
 
-  void start();
+      void stop();
 
-  void stop();
+      void register(
+          SystemStreamPartition systemStreamPartition,
+          String lastReadOffset);
 
-  void register(SystemStreamPartition systemStreamPartition, String lastReadOffset);
+      List<IncomingMessageEnvelope> poll(
+          Map<SystemStreamPartition, Integer> systemStreamPartitions,
+          long timeout)
+        throws InterruptedException;
+    }
 
-  List<IncomingMessageEnvelope> poll(Map<SystemStreamPartition, Integer> systemStreamPartitions, long timeout) throws InterruptedException;
-}
+    public class IncomingMessageEnvelope {
+      public Object getMessage() { ... }
 
-public class IncomingMessageEnvelope {
-  public Object getMessage() { ... }
+      public Object getKey() { ... }
 
-  public Object getKey() { ... }
+      public SystemStreamPartition getSystemStreamPartition() { ... }
+    }
 
-  public SystemStreamPartition getSystemStreamPartition() { ... }
-}
+    public interface SystemProducer {
+      void start();
 
-public interface SystemProducer {
-  void start();
+      void stop();
 
-  void stop();
+      void register(String source);
 
-  void register(String source);
+      void send(String source, OutgoingMessageEnvelope envelope);
 
-  void send(String source, OutgoingMessageEnvelope envelope);
+      void flush(String source);
+    }
 
-  void flush(String source);
-}
+    public class OutgoingMessageEnvelope {
+      ...
+      public Object getKey() { ... }
 
-public class OutgoingMessageEnvelope {
-  ...
-  public Object getKey() { ... }
+      public Object getMessage() { ... }
+    }
 
-  public Object getMessage() { ... }
-}
-```
+Out of the box, Samza supports Kafka (KafkaSystemConsumer and KafkaSystemProducer). However, any message bus system can be plugged in, as long as it can provide the semantics required by Samza, as described in the [javadoc](../api/javadocs/org/apache/samza/system/SystemConsumer.html).
 
-Out of the box, Samza supports reads and writes to Kafka (i.e. it has a KafkaSystemConsumer/KafkaSystemProducer), but the interfaces are pluggable, and most message bus systems can be plugged in, with some degree of support.
+SystemConsumers and SystemProducers may read and write messages of any data type. It's ok if they only support byte arrays &mdash; Samza has a separate [serialization layer](serialization.html) which converts to and from objects that application code can use. Samza does not prescribe any particular data model or serialization format.
 
-A number of stream-related properties should be defined in your Samza job's configuration file. These properties define systems that Samza can read from, the streams on these systems, and how to serialize and deserialize the messages from the streams. For example, you might wish to read PageViewEvent from a specific Kafka cluster. The system properties in the configuration file would define how to connect to the Kafka cluster. The stream section would define PageViewEvent as an input stream. The serializer in the configuration would define the serde to use to decode PageViewEvent messages.
+The job configuration file can include properties that are specific to a particular consumer and producer implementation. For example, the configuration would typically indicate the hostname and port of the message broker to use, and perhaps connection options.
 
-When the TaskRunner starts up, it will use the stream-related properties in your configuration to instantiate consumers for each stream partition. For example, if your input stream is PageViewEvent, which has 12 partitions, then the TaskRunner would create 12 KafkaSystemConsumers. Each consumer will read ByteBuffers from one partition, deserialize the ByteBuffer to an object, and put them into a queue. This queue is what the [event loop](event-loop.html) will use to feed messages to your StreamTask instances.
+### How streams are processed
 
-In the process method in StreamTask, there is a MessageCollector parameter given to use. When the TaskRunner calls process() on one of your StreamTask instances, it provides the collector. After the process() method completes, the TaskRunner takes any output messages that your StreamTask wrote to the collector, serializes the messages, and calls the send() method on the appropriate SystemProducer.
+If a job is consuming messages from more than one input stream, and all input streams have messages available, messages are processed in a round robin fashion by default. For example, if a job is consuming AdImpressionEvent and AdClickEvent, the task instance's process() method is called with a message from AdImpressionEvent, then a message from AdClickEvent, then another message from AdImpressionEvent, ... and continues to alternate between the two.
 
-### Message Ordering
-
-If a job is consuming messages from more than one system/stream/partition combination, by default, messages will be processed in a round robin fashion. For example, if a job is reading partitions 1 and 2 of page-view-events from a Kafka system, and there are messages available to be processed from both partitions, your StreamTask will get messages in round robin order (partition 1, partition 2, partition 1, partition 2, etc). If a message is not available for a given partition, it will be skipped, until a message becomes available.
+If one of the input streams has no new messages available (the most recent message has already been consumed), that stream is skipped, and the job continues to consume from the other inputs. It continues to check for new messages becoming available.
 
 #### MessageChooser
 
-The default round robin behavior can be overridden by implementing a custom MessageChooser. A MessageChooser's job is to answer the question, "Given a set of incoming messages, which one should a Samza container process next?".  To write a custom MessageChooser, take a look at the [Javadocs](../api/javadocs/org/apache/samza/system/MessageChooser.html), and then configure your task with the "task.chooser.class" configuration, which should point to your MessageChooserFactory.
-
-Out of the box, Samza ships with a RoundRobinChooser, which is the default. You can use the StreamChooser by adding the following configuration to your job.
+When a Samza container has several incoming messages on different stream partitions, how does it decide which to process first? The behavior is determined by a [MessageChooser](../api/javadocs/org/apache/samza/system/chooser/MessageChooser.html). The default chooser is RoundRobinChooser, but you can override it by implementing a custom chooser.
 
-```
-task.chooser.class=org.apache.samza.system.YourStreamChooserFactory
-```
+To plug in your own message chooser, you need to implement the [MessageChooserFactory](../api/javadocs/org/apache/samza/system/chooser/MessageChooserFactory.html) interface, and set the "task.chooser.class" configuration to the fully-qualified class name of your implementation:
 
-#### Prioritizing
+    task.chooser.class=com.example.samza.YourMessageChooserFactory
 
-There are certain times when messages from a stream should be favored over messages from any other stream. For example, some Samza jobs consume two streams: one stream is fed by a real-time system and the other stream is fed by a batch system. A typical pattern is to have a Samza processor with a statistical model that is ranking a real-time feed of data. Periodically, this model needs to be retrained and updated. The Samza processor can be re-deployed with the new model, but how do you re-process all of the old data that the processor has already seen? This can be accomplished by having a batch system send messages to the Samza processor for any data that needs to be re-processed. In this example, you'd like to favor the real-time system over the batch system, when messages are available for the real-time system. This prevents latency from being introduced into the real-time feed even when the batch system is sending messages by always processing the real-time messages first.
+#### Prioritizing input streams
 
-Samza provides a mechanism to prioritize one stream over another by setting this value: systems.&lt;system&gt;.streams.&lt;stream&gt;.samza.priority=2. A config snippet illustrates the settings:
+There are certain times when messages from one stream should be processed with higher priority than messages from another stream. For example, some Samza jobs consume two streams: one stream is fed by a real-time system and the other stream is fed by a batch system. In this case, it's useful to prioritize the real-time stream over the batch stream, so that the real-time processing doesn't slow down if there is a sudden burst of data on the batch stream.
 
-```
-systems.kafka.streams.my-stream.samza.priority=2
-systems.kafka.streams.my-other-stream.samza.priority=1
-```
+Samza provides a mechanism to prioritize one stream over another by setting this configuration parameter: systems.&lt;system&gt;.streams.&lt;stream&gt;.samza.priority=&lt;number&gt;. For example:
 
-This declares that my-stream's messages will be processed before my-other-stream's. If my-stream has no messages available at the moment (because more are still being read in, for instance), then my-other-stream's messages will get processed.
+    systems.kafka.streams.my-real-time-stream.samza.priority=2
+    systems.kafka.streams.my-batch-stream.samza.priority=1
 
-Each priority level gets its own MessageChooser. In the example above, one MessageChooser is used for my-stream, and another is used for my-other-stream. The MessageChooser for my-other-stream will only be used when my-stream's MessageChooser doesn't return a message to process. 
+This declares that my-real-time-stream's messages should be processed with higher priority than my-batch-stream's messages. If my-real-time-stream has any messages available, they are processed first. Only if there are no messages currently waiting on my-real-time-stream, the Samza job continues processing my-batch-stream.
 
-It is also valid to define two streams with the same priority. If messages are available from two streams at the same priority level, it's up to the MessageChooser for that priority level to decide which message should be processed first.
+Each priority level gets its own MessageChooser. It is valid to define two streams with the same priority. If messages are available from two streams at the same priority level, it's up to the MessageChooser for that priority level to decide which message should be processed first.
 
-It's also valid to only define priorities for some streams. All non-prioritized streams will be treated as the lowest priority, and will share a single MessageChooser. If you had my-third-stream, as a third input stream in the example above, it would be prioritized as the lowest stream, and also get its own MessageChooser.
+It's also valid to only define priorities for some streams. All non-prioritized streams are treated as the lowest priority, and share a MessageChooser.
 
 #### Bootstrapping
 
-Some Samza jobs wish to fully consume a stream from offset 0 all the way through to the last message in the stream before they process messages from any other stream. This is useful for streams that have some key-value data that a Samza job wishes to use when processing messages from another stream. This is 
-
-Consider a case where you want to read a currency-code stream, which has mappings of country code (e.g. USD) to symbols (e.g. $), and is partitioned by country code. You might want to join these symbols to a stream called transactions which is also partitioned by currency, and has a schema like {"country": "USD", "amount": 1234}. You could then have your StreamTask join the currency symbol to each transaction message, and emit messages like {"amount": "$1234"}.
-
-To bootstrap the currency-code stream, you need to read it from offset 0 all the way to the last message in the stream (what I'm calling head). It is not desirable to read any message from the transactions stream until the currency-code stream has been fully read, or else you might try to join a transaction message to a country code that hasn't yet been read.
-
-Samza supports this style of processing with the systems.&lt;system&gt;.streams.&lt;stream&gt;.samza.bootstrap property.
-
-```
-systems.kafka.streams.currency-code.samza.bootstrap=true
-```
-
-This configuration tells Samza that currency-code's messages should be read from the last checkpointed offset all the way until the stream is caught up to "head", before any other message is processed. If you wish to process all messages in currency-code from offset 0 to head, you can define:
+Sometimes, a Samza job needs to fully consume a stream (from offset 0 up to the most recent message) before it processes messages from any other stream. This is useful in situations where the stream contains some prerequisite data that the job needs, and it doesn't make sense to process messages from other streams until the job has loaded that prerequisite data. Samza supports this use case with *bootstrap streams*.
 
-```
-systems.kafka.streams.currency-code.samza.bootstrap=true
-systems.kafka.streams.currency-code.samza.reset.offset=true
-```
+A bootstrap stream seems similar to a stream with a high priority, but is subtly different. Before allowing any other stream to be processed, a bootstrap stream waits for the consumer to explicitly confirm that the stream has been fully consumed. Until then, the bootstrap stream is the exclusive input to the job: even if a network issue or some other factor causes the bootstrap stream consumer to slow down, other inputs can't sneak their messages in.
 
-This tells Samza to start from beginning of the currency-code stream, and read all the way to head.
+Another difference between a bootstrap stream and a high-priority stream is that the bootstrap stream's special treatment is temporary: when it has been fully consumed (we say it has "caught up"), its priority drops to be the same as all the other input streams.
 
-The difference between prioritizing a stream and bootstrapping a stream, is a high priority stream will still allow lower priority stream messages to be processed when no messages are available for the high priority stream. In the case of bootstrapping, no streams will be allowed to be processed until all messages in the bootstrap stream have been read up to the last message.
+To configure a stream called "my-bootstrap-stream" to be a fully-consumed bootstrap stream, use the following settings:
 
-Once a bootstrap stream has been fully consumed ("caught up"), it is treated like a normal stream, and no bootstrapping logic happens.
+    systems.kafka.streams.my-bootstrap-stream.samza.bootstrap=true
+    systems.kafka.streams.my-bootstrap-stream.samza.reset.offset=true
+    systems.kafka.streams.my-bootstrap-stream.samza.offset.default=oldest
 
-It is valid to define multiple bootstrap streams.
+The bootstrap=true parameter enables the bootstrap behavior (prioritization over other streams). The combination of reset.offset=true and offset.default=oldest tells Samza to always start reading the stream from the oldest offset, every time a container starts up (rather than starting to read from the most recent checkpoint).
 
-```
-systems.kafka.streams.currency-code.samza.bootstrap=true
-systems.kafka.streams.other-bootstrap-stream.samza.bootstrap=true
-```
-
-In this case, currency-code and other-bootstrap-stream will both be processed before any other stream is processed. The order of message processing (the bootstrap order) between currency-code and other-bootstrap-stream is up to the MessageChooser. If you want to fully process one bootstrap stream before another, you can use priorities:
-
-```
-systems.kafka.streams.currency-code.samza.bootstrap=true
-systems.kafka.streams.currency-code.samza.priority=2
-systems.kafka.streams.other-bootstrap-stream.samza.bootstrap=true
-systems.kafka.streams.other-bootstrap-stream.samza.priority=1
-```
-
-This defines a specific bootstrap ordering: fully bootstrap currency-code before bootstrapping other-bootstrap-stream.
-
-Lastly, bootstrap and non-bootstrap prioritized streams can be mixed:
-
-```
-systems.kafka.streams.currency-code.samza.bootstrap=true
-systems.kafka.streams.non-bootstrap-stream.samza.priority=2
-systems.kafka.streams.other-non-bootstrap-stream.samza.priority=1
-```
-
-Bootstrap streams are assigned a priority of Int.MaxInt by default, so they will always be prioritized over any other prioritized stream. In this case, currency-code will be fully bootstrapped, and then treated as the highest priority stream (Int.IntMax). The next highest priority stream will be non-bootstrap-stream (priority 2), followed by other-non-bootstrap-stream (priority 1), and then any non-bootstrap/non-prioritized streams.
+It is valid to define multiple bootstrap streams. In this case, the order in which they are bootstrapped is determined by the priority.
 
 #### Batching
 
-There are cases where consuming from the same SystemStreamPartition repeatedly leads to better performance. Samza allows for consumer batching to satisfy this use case. For example, if you had two SystemStreamPartitions, SSP1 and SSP2, you might wish to read 100 messages from SSP1 and then one from SSP2, regardless of the MessageChooser that's used. This can be accomplished with:
+In some cases, you can improve performance by consuming several messages from the same stream partition in sequence. Samza supports this mode of operation, called *batching*.
+
+For example, if you want to read 100 messages in a row from each stream partition (regardless of the MessageChooser), you can use this configuration parameter:
 
-```
-task.consumer.batch.size=100
-```
+    task.consumer.batch.size=100
 
-With this setting, Samza will always try and read a message from the last SystemStreamPartition that was read. This behavior will continue until no message is available for the SystemStreamPartition, or the batch size has been reached. In either of these cases, Samza will defer to the MessageChooser to determine the next message to process. It will then try and stick to the new message's SystemStreamPartition again.
+With this setting, Samza tries to read a message from the most recently used [SystemStreamPartition](../api/javadocs/org/apache/samza/system/SystemStreamPartition.html). This behavior continues either until no more messages are available for that SystemStreamPartition, or until the batch size has been reached. When that happens, Samza defers to the MessageChooser to determine the next message to process. It then again tries to continue consume from the chosen message's SystemStreamPartition until the batch size is reached.
 
-## [Checkpointing &raquo;](checkpointing.html)
+## [Serialization &raquo;](serialization.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/container/task-runner.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/task-runner.md b/docs/learn/documentation/0.7.0/container/task-runner.md
deleted file mode 100644
index bdc5ab6..0000000
--- a/docs/learn/documentation/0.7.0/container/task-runner.md
+++ /dev/null
@@ -1,44 +0,0 @@
----
-layout: page
-title: TaskRunner
----
-<!-- TODO: Is TaskRunner still appropriate terminology to use (appears to be a combo of SamzaContainer and TaskInstance in the code)? -->
-
-The TaskRunner is Samza's stream processing container. It is responsible for managing the startup, execution, and shutdown of one or more StreamTask instances.
-
-When the a TaskRunner starts up, it does the following:
-
-1. Get last checkpointed offset for each input stream/partition pair
-2. Create a "reader" thread for every input stream/partition pair
-3. Start metrics reporters to report metrics
-4. Start a checkpoint timer to save your task's input stream offsets every so often
-5. Start a window timer to trigger your StreamTask's window method, if it is defined
-6. Instantiate and initialize your StreamTask once for each input stream partition
-7. Start an event loop that takes messages from the input stream reader threads, and gives them to your StreamTasks
-8. Notify lifecycle listeners during each one of these steps
-
-Let's go over each of these items, starting in the middle, with the instantiation of a StreamTask.
-
-### Tasks and Partitions
-
-When the TaskRunner starts, it creates an instance of the StreamTask that you've written. If the StreamTask implements the InitableTask interface, the TaskRunner will also call the init() method.
-
-```
-public interface InitableTask {
-  void init(Config config, TaskContext context);
-}
-```
-
-It doesn't just do this once, though. It creates the StreamTask once for each partition in your Samza job. If your Samza job has ten partitions, there will be ten instantiations of your StreamTask: one for each partition. The StreamTask instance for partition one will receive all messages for partition one, the instance for partition two will receive all messages for partition two, and so on.
-
-The number of partitions that a Samza job has is determined by the number of partitions in its input streams. If a Samza job is set up to read from a topic called PageViewEvent, which has 12 partitions, then the Samza job will have 12 partitions when it executes.
-
-![diagram](/img/0.7.0/learn/documentation/container/tasks-and-partitions.png)
-
-If a Samza job has more than one input stream, then the number of partitions for the Samza job will be the maximum number of partitions across all input streams. For example, if a Samza job is reading from PageView event, which has 12 partitions, and ServiceMetricEvent, which has 14 partitions, then the Samza job would have 14 partitions (0 through 13).
-
-When the TaskRunner's StreamConsumer threads are reading messages from each input stream partition, the messages that it receives are tagged with the partition number that it came from. Each message is fed to the StreamTask instance that corresponds to the message's partition. This design has two important properties. When a Samza job has more than one input stream, and those streams have an imbalanced number of partitions (e.g. one has 12 partitions and the other has 14), then some of your StreamTask instances will not receive messages from all streams. In the PageViewEvent/ServiceMetricEvent example, the last two StreamTask instances would only receive messages from the ServiceMetricEvent topic (partitions 12 and 13). The lower 12 instances would receive messages from both streams. If your Samza job is reading more than one input stream, you probably want all input streams to have the same number of partitions, especially if you're trying to join streams together. The second impor
 tant property is that Samza assumes that a stream's partition count will never change. No partition splitting is supported. If an input stream has N partitions, it is expected that it has always had, and will always have N partitions. If you want to re-partition, you must read messages from the stream, and write them out to a new stream that has the number of partitions that you want. For example you could read messages from PageViewEvent, and write them to PageViewEventRepartition, which could have 14 partitions. If you did this, then you would achieve balance between PageViewEventRepartition and ServiceMetricEvent.
-
-This design is important because it guarantees that any state that your StreamTask keeps in memory will be isolated on a per-partition basis. For example, if you refer back to the page-view counting job we used as an example in the [Architecture](../introduction/architecture.html) section, we might have a Map&lt;Integer, Integer&gt; map that keeps track of page view counts per-user ID. If we were to have just one StreamTask per Samza job, for instance, then the user ID counts from different partitions would be inter-mingled into the same map. This inter-mingling would prevent us from moving partitions between processes or machines, which is something that we want to do with YARN. You can imagine a case where you started with one TaskRunner in a single YARN container. Your Samza job might be unable to keep up with only one container, so you ask for a second YARN container to put some of the StreamTask partitions. In such a case, how would we split the counts such that one container g
 ets only user ID counts for the partitions in charge of? This is effectively impossible if we've inter-mingled the StreamTask's state together. This is why we isolate StreamTask instances on a per-partition basis: to make partition migration possible.
-
-## [Streams &raquo;](streams.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/container/windowing.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/container/windowing.md b/docs/learn/documentation/0.7.0/container/windowing.md
index 6a24378..ebe2e86 100644
--- a/docs/learn/documentation/0.7.0/container/windowing.md
+++ b/docs/learn/documentation/0.7.0/container/windowing.md
@@ -3,18 +3,39 @@ layout: page
 title: Windowing
 ---
 
-Referring back to the "count PageViewEvent by user ID" example in the [Architecture](../introduction/architecture.html) section, one thing that we left out was what we do with the counts. Let's say that the Samza job wants to update the user ID counts in a database once every minute. Here's how it would work. The Samza job that does the counting would keep a Map&lt;Integer, Integer&gt; in memory, which maps user IDs to page view counts. Every time a message arrives, the job would take the user ID in the PageViewEvent, and use it to increment the user ID's count in the in-memory map. Then, once a minute, the StreamTask would update the database (total_count += current_count) for every user ID in the map, and then reset the count map.
+Sometimes a stream processing job needs to do something in regular time intervals, regardless of how many incoming messages the job is processing. For example, say you want to report the number of page views per minute. To do this, you increment a counter every time you see a page view event. Once per minute, you send the current counter value to an output stream and reset the counter to zero.
 
-Windowing is how we achieve this. If a StreamTask implements the WindowableTask interface, the TaskRunner will call the window() method on the task over a configured interval.
+Samza's *windowing* feature provides a way for tasks to do something in regular time intervals, for example once per minute. To enable windowing, you just need to set one property in your job configuration:
 
-```
-public interface WindowableTask {
-  void window(MessageCollector collector, TaskCoordinator coordinator);
-}
-```
+    # Call the window() method every 60 seconds
+    task.window.ms=60000
 
-If you choose to implement the WindowableTask interface, you can use the Samza job's configuration to define how often the TaskRunner should call your window() method. In the PageViewEvent example (above), you would define it to flush every 60000 milliseconds (60 seconds).
+Next, your stream task needs to implement the [WindowableTask](../api/javadocs/org/apache/samza/task/WindowableTask.html) interface. This interface defines a window() method which is called by Samza in the regular interval that you configured.
 
-If you need to send messages to output streams, you can use the MessageCollector object passed to the window() method. Please only use that MessageCollector object for sending messages, and don't use it outside of the call to window().
+For example, this is how you would implement a basic per-minute event counter:
+
+    public class EventCounterTask implements StreamTask, WindowableTask {
+
+      public static final SystemStream OUTPUT_STREAM =
+        new SystemStream("kafka", "events-per-minute");
+
+      private int eventsSeen = 0;
+
+      public void process(IncomingMessageEnvelope envelope,
+                          MessageCollector collector,
+                          TaskCoordinator coordinator) {
+        eventsSeen++;
+      }
+
+      public void window(MessageCollector collector,
+                         TaskCoordinator coordinator) {
+        collector.send(new OutgoingMessageEnvelope(OUTPUT_STREAM, eventsSeen));
+        eventsSeen = 0;
+      }
+    }
+
+If you need to send messages to output streams, you can use the [MessageCollector](../api/javadocs/org/apache/samza/task/MessageCollector.html) object passed to the window() method. Please only use that MessageCollector object for sending messages, and don't use it outside of the call to window().
+
+Note that Samza uses [single-threaded execution](event-loop.html), so the window() call can never happen concurrently with a process() call. This has the advantage that you don't need to worry about thread safety in your code (no need to synchronize anything), but the downside that the window() call may be delayed if your process() method takes a long time to return.
 
 ## [Event Loop &raquo;](event-loop.html)

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/index.html
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/index.html b/docs/learn/documentation/0.7.0/index.html
index a3f0c7f..f992f37 100644
--- a/docs/learn/documentation/0.7.0/index.html
+++ b/docs/learn/documentation/0.7.0/index.html
@@ -34,14 +34,15 @@ title: Documentation
 <h4>Container</h4>
 
 <ul class="documentation-list">
-  <li><a href="container/task-runner.html">TaskRunner</a></li>
+  <li><a href="container/samza-container.html">SamzaContainer</a></li>
   <li><a href="container/streams.html">Streams</a></li>
+  <li><a href="container/serialization.html">Serialization</a></li>
   <li><a href="container/checkpointing.html">Checkpointing</a></li>
   <li><a href="container/state-management.html">State Management</a></li>
   <li><a href="container/metrics.html">Metrics</a></li>
   <li><a href="container/windowing.html">Windowing</a></li>
   <li><a href="container/event-loop.html">Event Loop</a></li>
-  <li><a href="container/jmx.html">JMX</a>
+  <li><a href="container/jmx.html">JMX</a></li>
 </ul>
 
 <h4>Jobs</h4>

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/learn/documentation/0.7.0/introduction/architecture.md
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/0.7.0/introduction/architecture.md b/docs/learn/documentation/0.7.0/introduction/architecture.md
index c4e0fc9..d451273 100644
--- a/docs/learn/documentation/0.7.0/introduction/architecture.md
+++ b/docs/learn/documentation/0.7.0/introduction/architecture.md
@@ -61,7 +61,7 @@ Samza provides a YARN ApplicationMaster and a YARN job runner out of the box. Th
 
 ![diagram-small](/img/0.7.0/learn/documentation/introduction/samza-yarn-integration.png)
 
-The Samza client talks to the YARN RM when it wants to start a new Samza job. The YARN RM talks to a YARN NM to allocate space on the cluster for Samza's ApplicationMaster. Once the NM allocates space, it starts the Samza AM. After the Samza AM starts, it asks the YARN RM for one or more YARN containers to run Samza [TaskRunners](../container/task-runner.html). Again, the RM works with NMs to allocate space for the containers. Once the space has been allocated, the NMs start the Samza containers.
+The Samza client talks to the YARN RM when it wants to start a new Samza job. The YARN RM talks to a YARN NM to allocate space on the cluster for Samza's ApplicationMaster. Once the NM allocates space, it starts the Samza AM. After the Samza AM starts, it asks the YARN RM for one or more YARN containers to run [SamzaContainers](../container/samza-container.html). Again, the RM works with NMs to allocate space for the containers. Once the space has been allocated, the NMs start the Samza containers.
 
 ### Samza
 
@@ -69,7 +69,7 @@ Samza uses YARN and Kafka to provide a framework for stage-wise stream processin
 
 ![diagram-small](/img/0.7.0/learn/documentation/introduction/samza-yarn-kafka-integration.png)
 
-The Samza client uses YARN to run a Samza job. The Samza [TaskRunners](../container/task-runner.html) run in one or more YARN containers, and execute user-written Samza [StreamTasks](../api/overview.html). The input and output for the Samza StreamTasks come from Kafka brokers that are (usually) co-located on the same machines as the YARN NMs.
+The Samza client uses YARN to run a Samza job: YARN starts and supervises one or more [SamzaContainers](../container/samza-container.html), and your processing code (using the [StreamTask](../api/overview.html) API) runs inside those containers. The input and output for the Samza StreamTasks come from Kafka brokers that are (usually) co-located on the same machines as the YARN NMs.
 
 ### Example
 
@@ -81,7 +81,7 @@ Although Samza doesn't support SQL right now, the idea is the same. Two jobs are
 
 In the first job, the grouping is done by sending all messages with the same user ID to the same partition of an intermediate topic. You can do this by using the user ID as key of the messages that are emitted by the first job, and this key is mapped to one of the intermediate topic's partitions (usually by taking a hash of the key mod the number of partitions). The second job consumes the intermediate topic. Each task in the second job consumes one partition of the intermediate topic, i.e. all the messages for a subset of user IDs. The task has a counter for each user ID in its partition, and the appropriate counter is incremented every time the task receives a message with a particular user ID.
 
-![diagram-large](/img/0.7.0/learn/documentation/introduction/group-by-example.png)
+<img src="/img/0.7.0/learn/documentation/introduction/group-by-example.png" alt="Repartitioning for a GROUP BY" class="diagram-large">
 
 If you are familiar with Hadoop, you may recognize this as a Map/Reduce operation, where each record is associated with a particular key in the mappers, records with the same key are grouped together by the framework, and then counted in the reduce step. The difference between Hadoop and Samza is that Hadoop operates on a fixed input, whereas Samza works with unbounded streams of data.
 


[4/7] SAMZA-7: Rewrite Container section of docs to bring it up-to-date. Reviewed by Jakob Homan.

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/img/0.7.0/learn/documentation/container/tasks-and-partitions.svg
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/container/tasks-and-partitions.svg b/docs/img/0.7.0/learn/documentation/container/tasks-and-partitions.svg
new file mode 100644
index 0000000..f73b14d
--- /dev/null
+++ b/docs/img/0.7.0/learn/documentation/container/tasks-and-partitions.svg
@@ -0,0 +1,4 @@
+<?xml version="1.0" standalone="yes"?>
+
+<svg version="1.1" viewBox="0.0 0.0 501.0 320.0" fill="none" stroke="none" stroke-linecap="square" stroke-miterlimit="10" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink"><clipPath id="p.0"><path d="m0 0l501.0 0l0 320.0l-501.0 0l0 -320.0z" clip-rule="nonzero"></path></clipPath><g clip-path="url(#p.0)"><path fill="#000000" fill-opacity="0.0" d="m0 0l501.10498 0l0 320.95013l-501.10498 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m307.61417 160.80052l183.84253 0l0 150.07875l-183.84253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m307.61417 160.80052l183.84253 0l0 150.07875l-183.84253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m317.47354 183.34552l1.6875 -0.140625q0.125 1.015625 0.5625 1.671875q0.4375 0.65625 1.359375 1.0625q0.9375 0.40625 2.09375 0.40625q1.03125 0 1.8125 -0.3125q0.796875 -0.3125 1.1875 -0.84375q0.390625 -0.53125 0.390625 -1.
 15625q0 -0.640625 -0.375 -1.109375q-0.375 -0.484375 -1.234375 -0.8125q-0.546875 -0.21875 -2.421875 -0.65625q-1.875 -0.453125 -2.625 -0.859375q-0.96875 -0.515625 -1.453125 -1.265625q-0.46875 -0.75 -0.46875 -1.6875q0 -1.03125 0.578125 -1.921875q0.59375 -0.90625 1.703125 -1.359375q1.125 -0.46875 2.5 -0.46875q1.515625 0 2.671875 0.484375q1.15625 0.484375 1.765625 1.4375q0.625 0.9375 0.671875 2.140625l-1.71875 0.125q-0.140625 -1.28125 -0.953125 -1.9375q-0.796875 -0.671875 -2.359375 -0.671875q-1.625 0 -2.375 0.609375q-0.75 0.59375 -0.75 1.4375q0 0.734375 0.53125 1.203125q0.515625 0.46875 2.703125 0.96875q2.203125 0.5 3.015625 0.875q1.1875 0.546875 1.75 1.390625q0.578125 0.828125 0.578125 1.921875q0 1.09375 -0.625 2.0625q-0.625 0.953125 -1.796875 1.484375q-1.15625 0.53125 -2.609375 0.53125q-1.84375 0 -3.09375 -0.53125q-1.25 -0.546875 -1.96875 -1.625q-0.703125 -1.078125 -0.734375 -2.453125zm19.271698 3.15625q-0.9375 0.796875 -1.796875 1.125q-0.859375 0.3125 -1.84375 0.3125q-1.609375 0 -2.48
 4375 -0.78125q-0.875 -0.796875 -0.875 -2.03125q0 -0.734375 0.328125 -1.328125q0.328125 -0.59375 0.859375 -0.953125q0.53125 -0.359375 1.203125 -0.546875q0.5 -0.140625 1.484375 -0.25q2.03125 -0.25 2.984375 -0.578125q0 -0.34375 0 -0.4375q0 -1.015625 -0.46875 -1.4375q-0.640625 -0.5625 -1.90625 -0.5625q-1.171875 0 -1.734375 0.40625q-0.5625 0.40625 -0.828125 1.46875l-1.640625 -0.234375q0.234375 -1.046875 0.734375 -1.6875q0.515625 -0.640625 1.46875 -0.984375q0.96875 -0.359375 2.25 -0.359375q1.265625 0 2.046875 0.296875q0.78125 0.296875 1.15625 0.75q0.375 0.453125 0.515625 1.140625q0.09375 0.421875 0.09375 1.53125l0 2.234375q0 2.328125 0.09375 2.953125q0.109375 0.609375 0.4375 1.171875l-1.75 0q-0.265625 -0.515625 -0.328125 -1.21875zm-0.140625 -3.71875q-0.90625 0.359375 -2.734375 0.625q-1.03125 0.140625 -1.453125 0.328125q-0.421875 0.1875 -0.65625 0.546875q-0.234375 0.359375 -0.234375 0.796875q0 0.671875 0.5 1.125q0.515625 0.4375 1.484375 0.4375q0.96875 0 1.71875 -0.421875q0.75 -0.4375 1.109
 375 -1.15625q0.265625 -0.578125 0.265625 -1.671875l0 -0.609375zm4.0788574 4.9375l0 -9.859375l1.5 0l0 1.390625q0.453125 -0.71875 1.21875 -1.15625q0.78125 -0.453125 1.765625 -0.453125q1.09375 0 1.796875 0.453125q0.703125 0.453125 0.984375 1.28125q1.171875 -1.734375 3.046875 -1.734375q1.46875 0 2.25 0.8125q0.796875 0.8125 0.796875 2.5l0 6.765625l-1.671875 0l0 -6.203125q0 -1.0 -0.15625 -1.4375q-0.15625 -0.453125 -0.59375 -0.71875q-0.421875 -0.265625 -1.0 -0.265625q-1.03125 0 -1.71875 0.6875q-0.6875 0.6875 -0.6875 2.21875l0 5.71875l-1.671875 0l0 -6.40625q0 -1.109375 -0.40625 -1.65625q-0.40625 -0.5625 -1.34375 -0.5625q-0.703125 0 -1.3125 0.375q-0.59375 0.359375 -0.859375 1.078125q-0.265625 0.71875 -0.265625 2.0625l0 5.109375l-1.671875 0zm14.665802 0l0 -1.359375l6.265625 -7.1875q-1.0625 0.046875 -1.875 0.046875l-4.015625 0l0 -1.359375l8.046875 0l0 1.109375l-5.34375 6.25l-1.015625 1.140625q1.109375 -0.078125 2.09375 -0.078125l4.5625 0l0 1.4375l-8.71875 0zm16.640625 -1.21875q-0.9375 0.796875
  -1.796875 1.125q-0.859375 0.3125 -1.84375 0.3125q-1.609375 0 -2.484375 -0.78125q-0.875 -0.796875 -0.875 -2.03125q0 -0.734375 0.328125 -1.328125q0.328125 -0.59375 0.859375 -0.953125q0.53125 -0.359375 1.203125 -0.546875q0.5 -0.140625 1.484375 -0.25q2.03125 -0.25 2.984375 -0.578125q0 -0.34375 0 -0.4375q0 -1.015625 -0.46875 -1.4375q-0.640625 -0.5625 -1.90625 -0.5625q-1.171875 0 -1.734375 0.40625q-0.5625 0.40625 -0.828125 1.46875l-1.640625 -0.234375q0.234375 -1.046875 0.734375 -1.6875q0.515625 -0.640625 1.46875 -0.984375q0.96875 -0.359375 2.25 -0.359375q1.265625 0 2.046875 0.296875q0.78125 0.296875 1.15625 0.75q0.375 0.453125 0.515625 1.140625q0.09375 0.421875 0.09375 1.53125l0 2.234375q0 2.328125 0.09375 2.953125q0.109375 0.609375 0.4375 1.171875l-1.75 0q-0.265625 -0.515625 -0.328125 -1.21875zm-0.140625 -3.71875q-0.90625 0.359375 -2.734375 0.625q-1.03125 0.140625 -1.453125 0.328125q-0.421875 0.1875 -0.65625 0.546875q-0.234375 0.359375 -0.234375 0.796875q0 0.671875 0.5 1.125q0.515625 0.
 4375 1.484375 0.4375q0.96875 0 1.71875 -0.421875q0.75 -0.4375 1.109375 -1.15625q0.265625 -0.578125 0.265625 -1.671875l0 -0.609375zm14.000702 0.171875l1.796875 0.453125q-0.5625 2.21875 -2.03125 3.390625q-1.46875 1.15625 -3.59375 1.15625q-2.203125 0 -3.578125 -0.890625q-1.375 -0.90625 -2.09375 -2.59375q-0.71875 -1.703125 -0.71875 -3.65625q0 -2.125 0.796875 -3.703125q0.8125 -1.578125 2.3125 -2.390625q1.5 -0.828125 3.296875 -0.828125q2.046875 0 3.4375 1.046875q1.390625 1.03125 1.9375 2.90625l-1.765625 0.421875q-0.46875 -1.484375 -1.375 -2.15625q-0.90625 -0.6875 -2.265625 -0.6875q-1.5625 0 -2.625 0.75q-1.046875 0.75 -1.484375 2.03125q-0.421875 1.265625 -0.421875 2.609375q0 1.734375 0.5 3.03125q0.515625 1.28125 1.578125 1.921875q1.078125 0.640625 2.3125 0.640625q1.515625 0 2.5625 -0.859375q1.046875 -0.875 1.421875 -2.59375zm2.9260864 -0.15625q0 -2.734375 1.53125 -4.0625q1.265625 -1.09375 3.09375 -1.09375q2.03125 0 3.3125 1.34375q1.296875 1.328125 1.296875 3.671875q0 1.90625 -0.578125 3.0q
 -0.5625 1.078125 -1.65625 1.6875q-1.078125 0.59375 -2.375 0.59375q-2.0625 0 -3.34375 -1.328125q-1.28125 -1.328125 -1.28125 -3.8125zm1.71875 0q0 1.890625 0.828125 2.828125q0.828125 0.9375 2.078125 0.9375q1.25 0 2.0625 -0.9375q0.828125 -0.953125 0.828125 -2.890625q0 -1.828125 -0.828125 -2.765625q-0.828125 -0.9375 -2.0625 -0.9375q-1.25 0 -2.078125 0.9375q-0.828125 0.9375 -0.828125 2.828125zm9.281952 4.921875l0 -9.859375l1.5 0l0 1.40625q1.09375 -1.625 3.140625 -1.625q0.890625 0 1.640625 0.328125q0.75 0.3125 1.109375 0.84375q0.375 0.515625 0.53125 1.21875q0.09375 0.46875 0.09375 1.625l0 6.0625l-1.671875 0l0 -6.0q0 -1.015625 -0.203125 -1.515625q-0.1875 -0.515625 -0.6875 -0.8125q-0.5 -0.296875 -1.171875 -0.296875q-1.0625 0 -1.84375 0.671875q-0.765625 0.671875 -0.765625 2.578125l0 5.375l-1.671875 0zm14.031982 -1.5l0.234375 1.484375q-0.703125 0.140625 -1.265625 0.140625q-0.90625 0 -1.40625 -0.28125q-0.5 -0.296875 -0.703125 -0.75q-0.203125 -0.46875 -0.203125 -1.984375l0 -5.65625l-1.234375 0l0
  -1.3125l1.234375 0l0 -2.4375l1.65625 -1.0l0 3.4375l1.6875 0l0 1.3125l-1.6875 0l0 5.75q0 0.71875 0.078125 0.921875q0.09375 0.203125 0.296875 0.328125q0.203125 0.125 0.578125 0.125q0.265625 0 0.734375 -0.078125zm7.9645386 0.28125q-0.9375 0.796875 -1.796875 1.125q-0.859375 0.3125 -1.84375 0.3125q-1.609375 0 -2.484375 -0.78125q-0.875 -0.796875 -0.875 -2.03125q0 -0.734375 0.328125 -1.328125q0.328125 -0.59375 0.859375 -0.953125q0.53125 -0.359375 1.203125 -0.546875q0.5 -0.140625 1.484375 -0.25q2.03125 -0.25 2.984375 -0.578125q0 -0.34375 0 -0.4375q0 -1.015625 -0.46875 -1.4375q-0.640625 -0.5625 -1.90625 -0.5625q-1.171875 0 -1.734375 0.40625q-0.5625 0.40625 -0.828125 1.46875l-1.640625 -0.234375q0.234375 -1.046875 0.734375 -1.6875q0.515625 -0.640625 1.46875 -0.984375q0.96875 -0.359375 2.25 -0.359375q1.265625 0 2.046875 0.296875q0.78125 0.296875 1.15625 0.75q0.375 0.453125 0.515625 1.140625q0.09375 0.421875 0.09375 1.53125l0 2.234375q0 2.328125 0.09375 2.953125q0.109375 0.609375 0.4375 1.17187
 5l-1.75 0q-0.265625 -0.515625 -0.328125 -1.21875zm-0.140625 -3.71875q-0.90625 0.359375 -2.734375 0.625q-1.03125 0.140625 -1.453125 0.328125q-0.421875 0.1875 -0.65625 0.546875q-0.234375 0.359375 -0.234375 0.796875q0 0.671875 0.5 1.125q0.515625 0.4375 1.484375 0.4375q0.96875 0 1.71875 -0.421875q0.75 -0.4375 1.109375 -1.15625q0.265625 -0.578125 0.265625 -1.671875l0 -0.609375zm4.0944824 -6.75l0 -1.90625l1.671875 0l0 1.90625l-1.671875 0zm0 11.6875l0 -9.859375l1.671875 0l0 9.859375l-1.671875 0zm4.129181 0l0 -9.859375l1.5 0l0 1.40625q1.09375 -1.625 3.140625 -1.625q0.890625 0 1.640625 0.328125q0.75 0.3125 1.109375 0.84375q0.375 0.515625 0.53125 1.21875q0.09375 0.46875 0.09375 1.625l0 6.0625l-1.671875 0l0 -6.0q0 -1.015625 -0.203125 -1.515625q-0.1875 -0.515625 -0.6875 -0.8125q-0.5 -0.296875 -1.171875 -0.296875q-1.0625 0 -1.84375 0.671875q-0.765625 0.671875 -0.765625 2.578125l0 5.375l-1.671875 0zm17.125732 -3.171875l1.71875 0.21875q-0.40625 1.5 -1.515625 2.34375q-1.09375 0.828125 -2.8125 0.828
 125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 -3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.078125 -0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 2.046875zm9.094452 5.875l0 -9.859375l1.5 0l0 1.5q0.578125 -1.046875 1.0625 -1.375q0.484375 -0.34375 1.078125 -0.34375q0.84375 0 1.71875 0.546875l-0.578125 1.546875q-0.609375 -0.359375 -1.234375 -0.359375q-0.546875 0 -0.984375 0.328125q-0.421875 0.328125 -0.609375 0.90625q-0.28125 0.890625 -0.28125 1.953125l0 5.15625l-1.671875 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m47.771652 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonze
 ro"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m47.771652 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m61.90129 49.33281l-1.140625 0l0 -7.28125q-0.421875 0.390625 -1.09375 0.796875q-0.65625 0.390625 -1.1875 0.578125l0 -1.109375q0.953125 -0.4375 1.671875 -1.078125q0.71875 -0.640625 1.015625 -1.25l0.734375 0l0 9.34375z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m73.755905 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m73.755905 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m89.58867 48.23906l0 1.09375l-6.15625 0q-0.015625 -0.40625 0.140625 -0.796875q0.234375 -0.625 0.75 -1.234375q0.515625 -0.609375 1.5 -1.40625q1.515625 -1.25 2.046875 -1.96875q0.53125 -0.734375 0.53125 -1.375q0 -
 0.6875 -0.484375 -1.140625q-0.484375 -0.46875 -1.265625 -0.46875q-0.828125 0 -1.328125 0.5q-0.484375 0.484375 -0.5 1.359375l-1.171875 -0.125q0.125 -1.3125 0.90625 -2.0q0.78125 -0.6875 2.109375 -0.6875q1.34375 0 2.125 0.75q0.78125 0.734375 0.78125 1.828125q0 0.5625 -0.234375 1.109375q-0.21875 0.53125 -0.75 1.140625q-0.53125 0.59375 -1.765625 1.625q-1.03125 0.859375 -1.328125 1.171875q-0.28125 0.3125 -0.46875 0.625l4.5625 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m99.74016 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m99.74016 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m109.57292 46.879684l1.140625 -0.15625q0.203125 0.96875 0.671875 1.40625q0.46875 0.421875 1.15625 0.421875q0.796875 0 1.34375 -0.546875q0.5625 -0.5625 0.5625 -1.390625q0 -0.796875 -0.515625 -1.296875q-0.5 -0.515625
  -1.296875 -0.515625q-0.328125 0 -0.8125 0.125l0.125 -1.0q0.125 0.015625 0.1875 0.015625q0.734375 0 1.3125 -0.375q0.59375 -0.390625 0.59375 -1.1875q0 -0.625 -0.4375 -1.03125q-0.421875 -0.421875 -1.09375 -0.421875q-0.671875 0 -1.109375 0.421875q-0.4375 0.421875 -0.578125 1.25l-1.140625 -0.203125q0.21875 -1.140625 0.953125 -1.765625q0.75 -0.640625 1.84375 -0.640625q0.765625 0 1.40625 0.328125q0.640625 0.328125 0.984375 0.890625q0.34375 0.5625 0.34375 1.203125q0 0.59375 -0.328125 1.09375q-0.328125 0.5 -0.953125 0.78125q0.8125 0.203125 1.265625 0.796875q0.46875 0.59375 0.46875 1.5q0 1.21875 -0.890625 2.078125q-0.890625 0.84375 -2.25 0.84375q-1.21875 0 -2.03125 -0.734375q-0.8125 -0.734375 -0.921875 -1.890625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m125.72441 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m125.72441 32.737534l25.984253 0l0 
 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m139.21342 49.33281l0 -2.234375l-4.03125 0l0 -1.046875l4.234375 -6.03125l0.9375 0l0 6.03125l1.265625 0l0 1.046875l-1.265625 0l0 2.234375l-1.140625 0zm0 -3.28125l0 -4.1875l-2.921875 4.1875l2.921875 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m151.70866 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m151.70866 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m161.54143 46.89531l1.1875 -0.109375q0.140625 0.890625 0.625 1.328125q0.484375 0.4375 1.171875 0.4375q0.828125 0 1.390625 -0.625q0.578125 -0.625 0.578125 -1.640625q0 -0.984375 -0.546875 -1.546875q-0.546875 -0.5625 -1.4375 -0.5625q-0.5625 0 -1.015625 0.25q-0.4375 0.25 -0.6875 0.640625l-1.0625 -0.140625l0.890625 -4.765625l4.625 0l0 1.078125l-3.703125 0l-0.5 2.5q0.
 828125 -0.578125 1.75 -0.578125q1.21875 0 2.046875 0.84375q0.84375 0.84375 0.84375 2.171875q0 1.265625 -0.734375 2.1875q-0.890625 1.125 -2.4375 1.125q-1.265625 0 -2.078125 -0.703125q-0.796875 -0.71875 -0.90625 -1.890625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m177.69292 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m177.69292 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m193.44756 42.30156l-1.140625 0.09375q-0.140625 -0.671875 -0.421875 -0.984375q-0.46875 -0.484375 -1.140625 -0.484375q-0.546875 0 -0.96875 0.3125q-0.53125 0.390625 -0.84375 1.140625q-0.3125 0.75 -0.328125 2.15625q0.40625 -0.625 1.0 -0.921875q0.609375 -0.3125 1.265625 -0.3125q1.140625 0 1.9375 0.84375q0.8125 0.828125 0.8125 2.171875q0 0.875 -0.390625 1.625q-0.375 0.75 -1.03125 1.15625q-0.65625 0.390625 -1.5 0.390625q
 -1.421875 0 -2.328125 -1.046875q-0.90625 -1.046875 -0.90625 -3.46875q0 -2.6875 1.0 -3.921875q0.875 -1.0625 2.34375 -1.0625q1.09375 0 1.796875 0.625q0.703125 0.609375 0.84375 1.6875zm-4.671875 4.015625q0 0.59375 0.25 1.140625q0.25 0.53125 0.703125 0.8125q0.453125 0.28125 0.953125 0.28125q0.71875 0 1.234375 -0.578125q0.53125 -0.59375 0.53125 -1.59375q0 -0.96875 -0.515625 -1.515625q-0.515625 -0.5625 -1.296875 -0.5625q-0.78125 0 -1.328125 0.5625q-0.53125 0.546875 -0.53125 1.453125z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m203.67717 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m203.67717 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m213.57243 41.23906l0 -1.09375l6.03125 0l0 0.890625q-0.890625 0.953125 -1.765625 2.515625q-0.875 1.5625 -1.34375 3.21875q-0.34375 1.171875 -0.4375 2.5625l-1
 .171875 0q0.015625 -1.09375 0.421875 -2.640625q0.421875 -1.5625 1.1875 -3.0q0.765625 -1.453125 1.640625 -2.453125l-4.5625 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m229.66142 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m229.66142 32.737534l25.984253 0l0 23.59055l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m241.24419 44.285934q-0.703125 -0.265625 -1.046875 -0.734375q-0.34375 -0.484375 -0.34375 -1.15625q0 -1.015625 0.71875 -1.703125q0.734375 -0.703125 1.953125 -0.703125q1.21875 0 1.953125 0.71875q0.75 0.703125 0.75 1.71875q0 0.640625 -0.34375 1.125q-0.34375 0.46875 -1.03125 0.734375q0.859375 0.28125 1.296875 0.90625q0.453125 0.625 0.453125 1.484375q0 1.1875 -0.84375 2.0q-0.84375 0.8125 -2.21875 0.8125q-1.375 0 -2.21875 -0.8125q-0.84375 -0.8125 -0.84375 -2.03125q0 -0.90625 0.453125 -1.515625q0.46875 -0.625 1.3125
  -0.84375zm-0.234375 -1.9375q0 0.65625 0.421875 1.078125q0.4375 0.421875 1.109375 0.421875q0.671875 0 1.09375 -0.40625q0.421875 -0.421875 0.421875 -1.03125q0 -0.625 -0.4375 -1.046875q-0.4375 -0.4375 -1.078125 -0.4375q-0.65625 0 -1.09375 0.421875q-0.4375 0.421875 -0.4375 1.0zm-0.359375 4.296875q0 0.484375 0.234375 0.953125q0.234375 0.453125 0.6875 0.703125q0.453125 0.25 0.984375 0.25q0.8125 0 1.34375 -0.515625q0.53125 -0.53125 0.53125 -1.34375q0 -0.828125 -0.546875 -1.359375q-0.546875 -0.546875 -1.375 -0.546875q-0.796875 0 -1.328125 0.53125q-0.53125 0.53125 -0.53125 1.328125z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m255.64568 32.737534l25.984238 0l0 23.59055l-25.984238 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m255.64568 32.737534l25.984238 0l0 23.59055l-25.984238 0z" fill-rule="nonzero"></path><path fill="#000000" d="m265.6503 47.17656l1.09375 -0.09375q0.140625 0.765625 0.5
 3125 1.125q0.390625 0.34375 1.015625 0.34375q0.515625 0 0.90625 -0.234375q0.40625 -0.25 0.65625 -0.640625q0.265625 -0.40625 0.421875 -1.09375q0.171875 -0.6875 0.171875 -1.40625q0 -0.078125 0 -0.21875q-0.34375 0.546875 -0.9375 0.890625q-0.59375 0.328125 -1.28125 0.328125q-1.15625 0 -1.953125 -0.828125q-0.796875 -0.84375 -0.796875 -2.21875q0 -1.421875 0.828125 -2.28125q0.828125 -0.859375 2.09375 -0.859375q0.90625 0 1.65625 0.5q0.75 0.484375 1.140625 1.390625q0.390625 0.890625 0.390625 2.609375q0 1.78125 -0.390625 2.84375q-0.375 1.046875 -1.140625 1.609375q-0.765625 0.546875 -1.796875 0.546875q-1.09375 0 -1.796875 -0.59375q-0.6875 -0.609375 -0.8125 -1.71875zm4.671875 -4.109375q0 -0.984375 -0.53125 -1.546875q-0.515625 -0.578125 -1.25 -0.578125q-0.765625 0 -1.328125 0.625q-0.5625 0.609375 -0.5625 1.609375q0 0.875 0.53125 1.4375q0.53125 0.546875 1.328125 0.546875q0.796875 0 1.296875 -0.546875q0.515625 -0.5625 0.515625 -1.546875z" fill-rule="nonzero"></path><path fill="#000000" fill-opacit
 y="0.0" d="m47.771652 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m47.771652 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m61.90129 103.73438l-1.140625 0l0 -7.28125q-0.421875 0.390625 -1.09375 0.796875q-0.65625 0.390625 -1.1875 0.578125l0 -1.109375q0.953125 -0.4375 1.671875 -1.078125q0.71875 -0.640625 1.015625 -1.25l0.734375 0l0 9.34375z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m73.755905 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m73.755905 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m89.58867 102.64063l0 1.09375l-6.15625 0q-0.015625 -0.40625 0.140625 -0.796875q0.234375 -0.625 0.75 -1.234375q0.515625 -0.6
 09375 1.5 -1.40625q1.515625 -1.25 2.046875 -1.96875q0.53125 -0.734375 0.53125 -1.375q0 -0.6875 -0.484375 -1.140625q-0.484375 -0.46875 -1.265625 -0.46875q-0.828125 0 -1.328125 0.5q-0.484375 0.484375 -0.5 1.359375l-1.171875 -0.125q0.125 -1.3125 0.90625 -2.0q0.78125 -0.6875 2.109375 -0.6875q1.34375 0 2.125 0.75q0.78125 0.734375 0.78125 1.828125q0 0.5625 -0.234375 1.109375q-0.21875 0.53125 -0.75 1.140625q-0.53125 0.59375 -1.765625 1.625q-1.03125 0.859375 -1.328125 1.171875q-0.28125 0.3125 -0.46875 0.625l4.5625 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m99.74016 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m99.74016 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m109.57292 101.28126l1.140625 -0.15625q0.203125 0.96875 0.671875 1.40625q0.46875 0.421875 1.15625 0.421875q0.796875 0 1.34375 
 -0.546875q0.5625 -0.5625 0.5625 -1.390625q0 -0.796875 -0.515625 -1.296875q-0.5 -0.515625 -1.296875 -0.515625q-0.328125 0 -0.8125 0.125l0.125 -1.0q0.125 0.015625 0.1875 0.015625q0.734375 0 1.3125 -0.375q0.59375 -0.390625 0.59375 -1.1875q0 -0.625 -0.4375 -1.03125q-0.421875 -0.421875 -1.09375 -0.421875q-0.671875 0 -1.109375 0.421875q-0.4375 0.421875 -0.578125 1.25l-1.140625 -0.203125q0.21875 -1.140625 0.953125 -1.765625q0.75 -0.640625 1.84375 -0.640625q0.765625 0 1.40625 0.328125q0.640625 0.328125 0.984375 0.890625q0.34375 0.5625 0.34375 1.203125q0 0.59375 -0.328125 1.09375q-0.328125 0.5 -0.953125 0.78125q0.8125 0.203125 1.265625 0.796875q0.46875 0.59375 0.46875 1.5q0 1.21875 -0.890625 2.078125q-0.890625 0.84375 -2.25 0.84375q-1.21875 0 -2.03125 -0.734375q-0.8125 -0.734375 -0.921875 -1.890625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m125.72441 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1
 .0" stroke-linejoin="round" stroke-linecap="butt" d="m125.72441 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m139.21342 103.73438l0 -2.234375l-4.03125 0l0 -1.046875l4.234375 -6.03125l0.9375 0l0 6.03125l1.265625 0l0 1.046875l-1.265625 0l0 2.234375l-1.140625 0zm0 -3.28125l0 -4.1875l-2.921875 4.1875l2.921875 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m151.70866 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m151.70866 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m161.54143 101.29688l1.1875 -0.109375q0.140625 0.890625 0.625 1.328125q0.484375 0.4375 1.171875 0.4375q0.828125 0 1.390625 -0.625q0.578125 -0.625 0.578125 -1.640625q0 -0.984375 -0.546875 -1.546875q-0.546875 -0.5625 -1.4375 -0.5625q-0.5625 0 -1.015625 0.25q-0.4375 0.25 -0.687
 5 0.640625l-1.0625 -0.140625l0.890625 -4.765625l4.625 0l0 1.078125l-3.703125 0l-0.5 2.5q0.828125 -0.578125 1.75 -0.578125q1.21875 0 2.046875 0.84375q0.84375 0.84375 0.84375 2.171875q0 1.265625 -0.734375 2.1875q-0.890625 1.125 -2.4375 1.125q-1.265625 0 -2.078125 -0.703125q-0.796875 -0.71875 -0.90625 -1.890625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m177.69292 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m177.69292 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m193.44756 96.70313l-1.140625 0.09375q-0.140625 -0.671875 -0.421875 -0.984375q-0.46875 -0.484375 -1.140625 -0.484375q-0.546875 0 -0.96875 0.3125q-0.53125 0.390625 -0.84375 1.140625q-0.3125 0.75 -0.328125 2.15625q0.40625 -0.625 1.0 -0.921875q0.609375 -0.3125 1.265625 -0.3125q1.140625 0 1.9375 0.84375q0.8125 0.828125 0.8125 2.17
 1875q0 0.875 -0.390625 1.625q-0.375 0.75 -1.03125 1.15625q-0.65625 0.390625 -1.5 0.390625q-1.421875 0 -2.328125 -1.046875q-0.90625 -1.046875 -0.90625 -3.46875q0 -2.6875 1.0 -3.921875q0.875 -1.0625 2.34375 -1.0625q1.09375 0 1.796875 0.625q0.703125 0.609375 0.84375 1.6875zm-4.671875 4.015625q0 0.59375 0.25 1.140625q0.25 0.53125 0.703125 0.8125q0.453125 0.28125 0.953125 0.28125q0.71875 0 1.234375 -0.578125q0.53125 -0.59375 0.53125 -1.59375q0 -0.96875 -0.515625 -1.515625q-0.515625 -0.5625 -1.296875 -0.5625q-0.78125 0 -1.328125 0.5625q-0.53125 0.546875 -0.53125 1.453125z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m203.67717 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m203.67717 87.13911l25.984253 0l0 23.590553l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m213.57243 95.64063l0 -1.09375l6.03125 0l0 0.890625q-0.890625 0.95
 3125 -1.765625 2.515625q-0.875 1.5625 -1.34375 3.21875q-0.34375 1.171875 -0.4375 2.5625l-1.171875 0q0.015625 -1.09375 0.421875 -2.640625q0.421875 -1.5625 1.1875 -3.0q0.765625 -1.453125 1.640625 -2.453125l-4.5625 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m47.771652 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m47.771652 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m61.90129 200.12546l-1.140625 0l0 -7.28125q-0.421875 0.390625 -1.09375 0.796875q-0.65625 0.390625 -1.1875 0.578125l0 -1.109375q0.953125 -0.4375 1.671875 -1.078125q0.71875 -0.640625 1.015625 -1.25l0.734375 0l0 9.34375z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m73.755905 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stro
 ke-linejoin="round" stroke-linecap="butt" d="m73.755905 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m89.58867 199.03171l0 1.09375l-6.15625 0q-0.015625 -0.40625 0.140625 -0.796875q0.234375 -0.625 0.75 -1.234375q0.515625 -0.609375 1.5 -1.40625q1.515625 -1.25 2.046875 -1.96875q0.53125 -0.734375 0.53125 -1.375q0 -0.6875 -0.484375 -1.140625q-0.484375 -0.46875 -1.265625 -0.46875q-0.828125 0 -1.328125 0.5q-0.484375 0.484375 -0.5 1.359375l-1.171875 -0.125q0.125 -1.3125 0.90625 -2.0q0.78125 -0.6875 2.109375 -0.6875q1.34375 0 2.125 0.75q0.78125 0.734375 0.78125 1.828125q0 0.5625 -0.234375 1.109375q-0.21875 0.53125 -0.75 1.140625q-0.53125 0.59375 -1.765625 1.625q-1.03125 0.859375 -1.328125 1.171875q-0.28125 0.3125 -0.46875 0.625l4.5625 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m99.74016 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stro
 ke-linejoin="round" stroke-linecap="butt" d="m99.74016 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m109.57292 197.67233l1.140625 -0.15625q0.203125 0.96875 0.671875 1.40625q0.46875 0.421875 1.15625 0.421875q0.796875 0 1.34375 -0.546875q0.5625 -0.5625 0.5625 -1.390625q0 -0.796875 -0.515625 -1.296875q-0.5 -0.515625 -1.296875 -0.515625q-0.328125 0 -0.8125 0.125l0.125 -1.0q0.125 0.015625 0.1875 0.015625q0.734375 0 1.3125 -0.375q0.59375 -0.390625 0.59375 -1.1875q0 -0.625 -0.4375 -1.03125q-0.421875 -0.421875 -1.09375 -0.421875q-0.671875 0 -1.109375 0.421875q-0.4375 0.421875 -0.578125 1.25l-1.140625 -0.203125q0.21875 -1.140625 0.953125 -1.765625q0.75 -0.640625 1.84375 -0.640625q0.765625 0 1.40625 0.328125q0.640625 0.328125 0.984375 0.890625q0.34375 0.5625 0.34375 1.203125q0 0.59375 -0.328125 1.09375q-0.328125 0.5 -0.953125 0.78125q0.8125 0.203125 1.265625 0.796875q0.46875 0.59375 0.46875 1.5q0 1.21875 -0.890625 2.078125q-0.890625 0.843
 75 -2.25 0.84375q-1.21875 0 -2.03125 -0.734375q-0.8125 -0.734375 -0.921875 -1.890625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m125.72441 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m125.72441 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m139.21342 200.12546l0 -2.234375l-4.03125 0l0 -1.046875l4.234375 -6.03125l0.9375 0l0 6.03125l1.265625 0l0 1.046875l-1.265625 0l0 2.234375l-1.140625 0zm0 -3.28125l0 -4.1875l-2.921875 4.1875l2.921875 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m151.70866 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m151.70866 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path fill="#0
 00000" d="m161.54143 197.68796l1.1875 -0.109375q0.140625 0.890625 0.625 1.328125q0.484375 0.4375 1.171875 0.4375q0.828125 0 1.390625 -0.625q0.578125 -0.625 0.578125 -1.640625q0 -0.984375 -0.546875 -1.546875q-0.546875 -0.5625 -1.4375 -0.5625q-0.5625 0 -1.015625 0.25q-0.4375 0.25 -0.6875 0.640625l-1.0625 -0.140625l0.890625 -4.765625l4.625 0l0 1.078125l-3.703125 0l-0.5 2.5q0.828125 -0.578125 1.75 -0.578125q1.21875 0 2.046875 0.84375q0.84375 0.84375 0.84375 2.171875q0 1.265625 -0.734375 2.1875q-0.890625 1.125 -2.4375 1.125q-1.265625 0 -2.078125 -0.703125q-0.796875 -0.71875 -0.90625 -1.890625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m177.69292 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m177.69292 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m193.44756 193.09421l-1.140625 0.09375q-0
 .140625 -0.671875 -0.421875 -0.984375q-0.46875 -0.484375 -1.140625 -0.484375q-0.546875 0 -0.96875 0.3125q-0.53125 0.390625 -0.84375 1.140625q-0.3125 0.75 -0.328125 2.15625q0.40625 -0.625 1.0 -0.921875q0.609375 -0.3125 1.265625 -0.3125q1.140625 0 1.9375 0.84375q0.8125 0.828125 0.8125 2.171875q0 0.875 -0.390625 1.625q-0.375 0.75 -1.03125 1.15625q-0.65625 0.390625 -1.5 0.390625q-1.421875 0 -2.328125 -1.046875q-0.90625 -1.046875 -0.90625 -3.46875q0 -2.6875 1.0 -3.921875q0.875 -1.0625 2.34375 -1.0625q1.09375 0 1.796875 0.625q0.703125 0.609375 0.84375 1.6875zm-4.671875 4.015625q0 0.59375 0.25 1.140625q0.25 0.53125 0.703125 0.8125q0.453125 0.28125 0.953125 0.28125q0.71875 0 1.234375 -0.578125q0.53125 -0.59375 0.53125 -1.59375q0 -0.96875 -0.515625 -1.515625q-0.515625 -0.5625 -1.296875 -0.5625q-0.78125 0 -1.328125 0.5625q-0.53125 0.546875 -0.53125 1.453125z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m203.67717 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-
 rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m203.67717 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m213.57243 192.03171l0 -1.09375l6.03125 0l0 0.890625q-0.890625 0.953125 -1.765625 2.515625q-0.875 1.5625 -1.34375 3.21875q-0.34375 1.171875 -0.4375 2.5625l-1.171875 0q0.015625 -1.09375 0.421875 -2.640625q0.421875 -1.5625 1.1875 -3.0q0.765625 -1.453125 1.640625 -2.453125l-4.5625 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m229.66142 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m229.66142 183.53018l25.984253 0l0 23.590546l-25.984253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m241.24419 195.07858q-0.703125 -0.265625 -1.046875 -0.734375q-0.34375 -0.484375 -0.34375 -1.15625q0 -1.015625 0.71875 -1.703125q0.7
 34375 -0.703125 1.953125 -0.703125q1.21875 0 1.953125 0.71875q0.75 0.703125 0.75 1.71875q0 0.640625 -0.34375 1.125q-0.34375 0.46875 -1.03125 0.734375q0.859375 0.28125 1.296875 0.90625q0.453125 0.625 0.453125 1.484375q0 1.1875 -0.84375 2.0q-0.84375 0.8125 -2.21875 0.8125q-1.375 0 -2.21875 -0.8125q-0.84375 -0.8125 -0.84375 -2.03125q0 -0.90625 0.453125 -1.515625q0.46875 -0.625 1.3125 -0.84375zm-0.234375 -1.9375q0 0.65625 0.421875 1.078125q0.4375 0.421875 1.109375 0.421875q0.671875 0 1.09375 -0.40625q0.421875 -0.421875 0.421875 -1.03125q0 -0.625 -0.4375 -1.046875q-0.4375 -0.4375 -1.078125 -0.4375q-0.65625 0 -1.09375 0.421875q-0.4375 0.421875 -0.4375 1.0zm-0.359375 4.296875q0 0.484375 0.234375 0.953125q0.234375 0.453125 0.6875 0.703125q0.453125 0.25 0.984375 0.25q0.8125 0 1.34375 -0.515625q0.53125 -0.53125 0.53125 -1.34375q0 -0.828125 -0.546875 -1.359375q-0.546875 -0.546875 -1.375 -0.546875q-0.796875 0 -1.328125 0.53125q-0.53125 0.53125 -0.53125 1.328125z" fill-rule="nonzero"></path><pat
 h fill="#000000" fill-opacity="0.0" d="m307.61417 9.6850395l183.84253 0l0 146.61417l-183.84253 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m307.61417 9.6850395l183.84253 0l0 146.61417l-183.84253 0z" fill-rule="nonzero"></path><path fill="#000000" d="m317.47354 32.230038l1.6875 -0.140625q0.125 1.015625 0.5625 1.671875q0.4375 0.65625 1.359375 1.0625q0.9375 0.40625 2.09375 0.40625q1.03125 0 1.8125 -0.3125q0.796875 -0.3125 1.1875 -0.84375q0.390625 -0.53125 0.390625 -1.15625q0 -0.640625 -0.375 -1.109375q-0.375 -0.484375 -1.234375 -0.8125q-0.546875 -0.21875 -2.421875 -0.65625q-1.875 -0.453125 -2.625 -0.859375q-0.96875 -0.515625 -1.453125 -1.265625q-0.46875 -0.75 -0.46875 -1.6875q0 -1.03125 0.578125 -1.921875q0.59375 -0.90625 1.703125 -1.359375q1.125 -0.46875 2.5 -0.46875q1.515625 0 2.671875 0.484375q1.15625 0.484375 1.765625 1.4375q0.625 0.9375 0.671875 2.140625l-1.71875 0.125q-0.140625 -1.28125 -0.953125 -1.937
 5q-0.796875 -0.671875 -2.359375 -0.671875q-1.625 0 -2.375 0.609375q-0.75 0.59375 -0.75 1.4375q0 0.734375 0.53125 1.203125q0.515625 0.46875 2.703125 0.96875q2.203125 0.5 3.015625 0.875q1.1875 0.546875 1.75 1.390625q0.578125 0.828125 0.578125 1.921875q0 1.09375 -0.625 2.0625q-0.625 0.953125 -1.796875 1.484375q-1.15625 0.53125 -2.609375 0.53125q-1.84375 0 -3.09375 -0.53125q-1.25 -0.546875 -1.96875 -1.625q-0.703125 -1.078125 -0.734375 -2.453125zm19.271698 3.15625q-0.9375 0.796875 -1.796875 1.125q-0.859375 0.3125 -1.84375 0.3125q-1.609375 0 -2.484375 -0.78125q-0.875 -0.796875 -0.875 -2.03125q0 -0.734375 0.328125 -1.328125q0.328125 -0.59375 0.859375 -0.953125q0.53125 -0.359375 1.203125 -0.546875q0.5 -0.140625 1.484375 -0.25q2.03125 -0.25 2.984375 -0.578125q0 -0.34375 0 -0.4375q0 -1.015625 -0.46875 -1.4375q-0.640625 -0.5625 -1.90625 -0.5625q-1.171875 0 -1.734375 0.40625q-0.5625 0.40625 -0.828125 1.46875l-1.640625 -0.234375q0.234375 -1.046875 0.734375 -1.6875q0.515625 -0.640625 1.46875 -0.9
 84375q0.96875 -0.359375 2.25 -0.359375q1.265625 0 2.046875 0.296875q0.78125 0.296875 1.15625 0.75q0.375 0.453125 0.515625 1.140625q0.09375 0.421875 0.09375 1.53125l0 2.234375q0 2.328125 0.09375 2.953125q0.109375 0.609375 0.4375 1.171875l-1.75 0q-0.265625 -0.515625 -0.328125 -1.21875zm-0.140625 -3.71875q-0.90625 0.359375 -2.734375 0.625q-1.03125 0.140625 -1.453125 0.328125q-0.421875 0.1875 -0.65625 0.546875q-0.234375 0.359375 -0.234375 0.796875q0 0.671875 0.5 1.125q0.515625 0.4375 1.484375 0.4375q0.96875 0 1.71875 -0.421875q0.75 -0.4375 1.109375 -1.15625q0.265625 -0.578125 0.265625 -1.671875l0 -0.609375zm4.0788574 4.9375l0 -9.859375l1.5 0l0 1.390625q0.453125 -0.71875 1.21875 -1.15625q0.78125 -0.453125 1.765625 -0.453125q1.09375 0 1.796875 0.453125q0.703125 0.453125 0.984375 1.28125q1.171875 -1.734375 3.046875 -1.734375q1.46875 0 2.25 0.8125q0.796875 0.8125 0.796875 2.5l0 6.765625l-1.671875 0l0 -6.203125q0 -1.0 -0.15625 -1.4375q-0.15625 -0.453125 -0.59375 -0.71875q-0.421875 -0.265625 
 -1.0 -0.265625q-1.03125 0 -1.71875 0.6875q-0.6875 0.6875 -0.6875 2.21875l0 5.71875l-1.671875 0l0 -6.40625q0 -1.109375 -0.40625 -1.65625q-0.40625 -0.5625 -1.34375 -0.5625q-0.703125 0 -1.3125 0.375q-0.59375 0.359375 -0.859375 1.078125q-0.265625 0.71875 -0.265625 2.0625l0 5.109375l-1.671875 0zm14.665802 0l0 -1.359375l6.265625 -7.1875q-1.0625 0.046875 -1.875 0.046875l-4.015625 0l0 -1.359375l8.046875 0l0 1.109375l-5.34375 6.25l-1.015625 1.140625q1.109375 -0.078125 2.09375 -0.078125l4.5625 0l0 1.4375l-8.71875 0zm16.640625 -1.21875q-0.9375 0.796875 -1.796875 1.125q-0.859375 0.3125 -1.84375 0.3125q-1.609375 0 -2.484375 -0.78125q-0.875 -0.796875 -0.875 -2.03125q0 -0.734375 0.328125 -1.328125q0.328125 -0.59375 0.859375 -0.953125q0.53125 -0.359375 1.203125 -0.546875q0.5 -0.140625 1.484375 -0.25q2.03125 -0.25 2.984375 -0.578125q0 -0.34375 0 -0.4375q0 -1.015625 -0.46875 -1.4375q-0.640625 -0.5625 -1.90625 -0.5625q-1.171875 0 -1.734375 0.40625q-0.5625 0.40625 -0.828125 1.46875l-1.640625 -0.234375q
 0.234375 -1.046875 0.734375 -1.6875q0.515625 -0.640625 1.46875 -0.984375q0.96875 -0.359375 2.25 -0.359375q1.265625 0 2.046875 0.296875q0.78125 0.296875 1.15625 0.75q0.375 0.453125 0.515625 1.140625q0.09375 0.421875 0.09375 1.53125l0 2.234375q0 2.328125 0.09375 2.953125q0.109375 0.609375 0.4375 1.171875l-1.75 0q-0.265625 -0.515625 -0.328125 -1.21875zm-0.140625 -3.71875q-0.90625 0.359375 -2.734375 0.625q-1.03125 0.140625 -1.453125 0.328125q-0.421875 0.1875 -0.65625 0.546875q-0.234375 0.359375 -0.234375 0.796875q0 0.671875 0.5 1.125q0.515625 0.4375 1.484375 0.4375q0.96875 0 1.71875 -0.421875q0.75 -0.4375 1.109375 -1.15625q0.265625 -0.578125 0.265625 -1.671875l0 -0.609375zm14.000702 0.171875l1.796875 0.453125q-0.5625 2.21875 -2.03125 3.390625q-1.46875 1.15625 -3.59375 1.15625q-2.203125 0 -3.578125 -0.890625q-1.375 -0.90625 -2.09375 -2.59375q-0.71875 -1.703125 -0.71875 -3.65625q0 -2.125 0.796875 -3.703125q0.8125 -1.578125 2.3125 -2.390625q1.5 -0.828125 3.296875 -0.828125q2.046875 0 3.437
 5 1.046875q1.390625 1.03125 1.9375 2.90625l-1.765625 0.421875q-0.46875 -1.484375 -1.375 -2.15625q-0.90625 -0.6875 -2.265625 -0.6875q-1.5625 0 -2.625 0.75q-1.046875 0.75 -1.484375 2.03125q-0.421875 1.265625 -0.421875 2.609375q0 1.734375 0.5 3.03125q0.515625 1.28125 1.578125 1.921875q1.078125 0.640625 2.3125 0.640625q1.515625 0 2.5625 -0.859375q1.046875 -0.875 1.421875 -2.59375zm2.9260864 -0.15625q0 -2.734375 1.53125 -4.0625q1.265625 -1.09375 3.09375 -1.09375q2.03125 0 3.3125 1.34375q1.296875 1.328125 1.296875 3.671875q0 1.90625 -0.578125 3.0q-0.5625 1.078125 -1.65625 1.6875q-1.078125 0.59375 -2.375 0.59375q-2.0625 0 -3.34375 -1.328125q-1.28125 -1.328125 -1.28125 -3.8125zm1.71875 0q0 1.890625 0.828125 2.828125q0.828125 0.9375 2.078125 0.9375q1.25 0 2.0625 -0.9375q0.828125 -0.953125 0.828125 -2.890625q0 -1.828125 -0.828125 -2.765625q-0.828125 -0.9375 -2.0625 -0.9375q-1.25 0 -2.078125 0.9375q-0.828125 0.9375 -0.828125 2.828125zm9.281952 4.921875l0 -9.859375l1.5 0l0 1.40625q1.09375 -1.62
 5 3.140625 -1.625q0.890625 0 1.640625 0.328125q0.75 0.3125 1.109375 0.84375q0.375 0.515625 0.53125 1.21875q0.09375 0.46875 0.09375 1.625l0 6.0625l-1.671875 0l0 -6.0q0 -1.015625 -0.203125 -1.515625q-0.1875 -0.515625 -0.6875 -0.8125q-0.5 -0.296875 -1.171875 -0.296875q-1.0625 0 -1.84375 0.671875q-0.765625 0.671875 -0.765625 2.578125l0 5.375l-1.671875 0zm14.031982 -1.5l0.234375 1.484375q-0.703125 0.140625 -1.265625 0.140625q-0.90625 0 -1.40625 -0.28125q-0.5 -0.296875 -0.703125 -0.75q-0.203125 -0.46875 -0.203125 -1.984375l0 -5.65625l-1.234375 0l0 -1.3125l1.234375 0l0 -2.4375l1.65625 -1.0l0 3.4375l1.6875 0l0 1.3125l-1.6875 0l0 5.75q0 0.71875 0.078125 0.921875q0.09375 0.203125 0.296875 0.328125q0.203125 0.125 0.578125 0.125q0.265625 0 0.734375 -0.078125zm7.9645386 0.28125q-0.9375 0.796875 -1.796875 1.125q-0.859375 0.3125 -1.84375 0.3125q-1.609375 0 -2.484375 -0.78125q-0.875 -0.796875 -0.875 -2.03125q0 -0.734375 0.328125 -1.328125q0.328125 -0.59375 0.859375 -0.953125q0.53125 -0.359375 1.203
 125 -0.546875q0.5 -0.140625 1.484375 -0.25q2.03125 -0.25 2.984375 -0.578125q0 -0.34375 0 -0.4375q0 -1.015625 -0.46875 -1.4375q-0.640625 -0.5625 -1.90625 -0.5625q-1.171875 0 -1.734375 0.40625q-0.5625 0.40625 -0.828125 1.46875l-1.640625 -0.234375q0.234375 -1.046875 0.734375 -1.6875q0.515625 -0.640625 1.46875 -0.984375q0.96875 -0.359375 2.25 -0.359375q1.265625 0 2.046875 0.296875q0.78125 0.296875 1.15625 0.75q0.375 0.453125 0.515625 1.140625q0.09375 0.421875 0.09375 1.53125l0 2.234375q0 2.328125 0.09375 2.953125q0.109375 0.609375 0.4375 1.171875l-1.75 0q-0.265625 -0.515625 -0.328125 -1.21875zm-0.140625 -3.71875q-0.90625 0.359375 -2.734375 0.625q-1.03125 0.140625 -1.453125 0.328125q-0.421875 0.1875 -0.65625 0.546875q-0.234375 0.359375 -0.234375 0.796875q0 0.671875 0.5 1.125q0.515625 0.4375 1.484375 0.4375q0.96875 0 1.71875 -0.421875q0.75 -0.4375 1.109375 -1.15625q0.265625 -0.578125 0.265625 -1.671875l0 -0.609375zm4.0944824 -6.75l0 -1.90625l1.671875 0l0 1.90625l-1.671875 0zm0 11.6875l0 -
 9.859375l1.671875 0l0 9.859375l-1.671875 0zm4.129181 0l0 -9.859375l1.5 0l0 1.40625q1.09375 -1.625 3.140625 -1.625q0.890625 0 1.640625 0.328125q0.75 0.3125 1.109375 0.84375q0.375 0.515625 0.53125 1.21875q0.09375 0.46875 0.09375 1.625l0 6.0625l-1.671875 0l0 -6.0q0 -1.015625 -0.203125 -1.515625q-0.1875 -0.515625 -0.6875 -0.8125q-0.5 -0.296875 -1.171875 -0.296875q-1.0625 0 -1.84375 0.671875q-0.765625 0.671875 -0.765625 2.578125l0 5.375l-1.671875 0zm17.125732 -3.171875l1.71875 0.21875q-0.40625 1.5 -1.515625 2.34375q-1.09375 0.828125 -2.8125 0.828125q-2.15625 0 -3.421875 -1.328125q-1.265625 -1.328125 -1.265625 -3.734375q0 -2.484375 1.265625 -3.859375q1.28125 -1.375 3.328125 -1.375q1.984375 0 3.234375 1.34375q1.25 1.34375 1.25 3.796875q0 0.140625 -0.015625 0.4375l-7.34375 0q0.09375 1.625 0.921875 2.484375q0.828125 0.859375 2.0625 0.859375q0.90625 0 1.546875 -0.46875q0.65625 -0.484375 1.046875 -1.546875zm-5.484375 -2.703125l5.5 0q-0.109375 -1.234375 -0.625 -1.859375q-0.796875 -0.96875 -2.07
 8125 -0.96875q-1.140625 0 -1.9375 0.78125q-0.78125 0.765625 -0.859375 2.046875zm9.094452 5.875l0 -9.859375l1.5 0l0 1.5q0.578125 -1.046875 1.0625 -1.375q0.484375 -0.34375 1.078125 -0.34375q0.84375 0 1.71875 0.546875l-0.578125 1.546875q-0.609375 -0.359375 -1.234375 -0.359375q-0.546875 0 -0.984375 0.328125q-0.421875 0.328125 -0.609375 0.90625q-0.28125 0.890625 -0.28125 1.953125l0 5.15625l-1.671875 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m0 220.60104l0 -147.37007l39.149605 0l0 147.37007z" fill-rule="nonzero"></path><path fill="#000000" d="m15.232498 196.97784l-1.90625 0l0 -1.671875l1.90625 0l0 1.671875zm11.6875 0l-9.859375 0l0 -1.671875l9.859375 0l0 1.671875zm0 -4.129196l-9.859375 0l0 -1.5l1.40625 0q-1.625 -1.09375 -1.625 -3.140625q0 -0.890625 0.328125 -1.640625q0.3125 -0.75 0.84375 -1.109375q0.515625 -0.375 1.21875 -0.53125q0.46875 -0.09375 1.625 -0.09375l6.0625 0l0 1.671875l-6.0 0q-1.015625 0 -1.515625 0.203125q-0.515625 0.1875 -0.8125 0.6875q-0.29687
 5 0.5 -0.296875 1.171875q0 1.0625 0.671875 1.84375q0.671875 0.765625 2.578125 0.765625l5.375 0l0 1.671875zm3.78125 -10.375717l-13.640625 0l0 -1.53125l1.28125 0q-0.75 -0.53125 -1.125 -1.203125q-0.375 -0.6875 -0.375 -1.640625q0 -1.265625 0.65625 -2.234375q0.640625 -0.96875 1.828125 -1.453125q1.1875 -0.5 2.59375 -0.5q1.515625 0 2.734375 0.546875q1.203125 0.546875 1.84375 1.578125q0.640625 1.03125 0.640625 2.171875q0 0.84375 -0.34375 1.515625q-0.359375 0.65625 -0.890625 1.078125l4.796875 0l0 1.671875zm-8.65625 -1.515625q1.90625 0 2.8125 -0.765625q0.90625 -0.78125 0.90625 -1.875q0 -1.109375 -0.9375 -1.890625q-0.9375 -0.796875 -2.921875 -0.796875q-1.875 0 -2.8125 0.78125q-0.9375 0.765625 -0.9375 1.84375q0 1.0625 1.0 1.890625q1.0 0.8125 2.890625 0.8125zm4.875 -15.313217l-1.453125 0q1.671875 1.140625 1.671875 3.125q0 0.859375 -0.328125 1.625q-0.34375 0.75 -0.84375 1.125q-0.5 0.359375 -1.234375 0.515625q-0.5 0.09375 -1.5625 0.09375l-6.109375 0l0 -1.671875l5.46875 0q1.3125 0 1.765625 -0.09375
 q0.65625 -0.15625 1.03125 -0.671875q0.375 -0.515625 0.375 -1.265625q0 -0.75 -0.375 -1.40625q-0.390625 -0.65625 -1.046875 -0.921875q-0.671875 -0.28125 -1.9375 -0.28125l-5.28125 0l0 -1.671875l9.859375 0l0 1.5zm-1.5 -7.578842l1.484375 -0.234375q0.140625 0.703125 0.140625 1.265625q0 0.90625 -0.28125 1.40625q-0.296875 0.5 -0.75 0.703125q-0.46875 0.203125 -1.984375 0.203125l-5.65625 0l0 1.234375l-1.3125 0l0 -1.234375l-2.4375 0l-1.0 -1.65625l3.4375 0l0 -1.6875l1.3125 0l0 1.6875l5.75 0q0.71875 0 0.921875 -0.078125q0.203125 -0.09375 0.328125 -0.296875q0.125 -0.203125 0.125 -0.578125q0 -0.265625 -0.078125 -0.734375zm-1.4375 -6.0384827l-0.265625 -1.65625q1.0 -0.140625 1.53125 -0.765625q0.515625 -0.640625 0.515625 -1.78125q0 -1.15625 -0.46875 -1.703125q-0.46875 -0.5625 -1.09375 -0.5625q-0.5625 0 -0.890625 0.484375q-0.21875 0.34375 -0.5625 1.703125q-0.46875 1.84375 -0.796875 2.5625q-0.34375 0.703125 -0.9375 1.078125q-0.609375 0.359375 -1.328125 0.359375q-0.65625 0 -1.21875 -0.296875q-0.5625 -0.3
 125 -0.9375 -0.828125q-0.28125 -0.390625 -0.484375 -1.0625q-0.203125 -0.671875 -0.203125 -1.4375q0 -1.171875 0.34375 -2.046875q0.328125 -0.875 0.90625 -1.28125q0.5625 -0.421875 1.515625 -0.578125l0.21875 1.625q-0.75 0.109375 -1.171875 0.65625q-0.4375 0.53125 -0.4375 1.5q0 1.15625 0.390625 1.640625q0.375 0.484375 0.875 0.484375q0.328125 0 0.59375 -0.203125q0.265625 -0.203125 0.4375 -0.640625q0.09375 -0.25 0.4375 -1.46875q0.46875 -1.765625 0.765625 -2.46875q0.296875 -0.703125 0.875 -1.09375q0.578125 -0.40625 1.4375 -0.40625q0.828125 0 1.578125 0.484375q0.734375 0.484375 1.140625 1.40625q0.390625 0.921875 0.390625 2.078125q0 1.921875 -0.796875 2.9375q-0.796875 1.0 -2.359375 1.28125zm1.4375 -13.65625l1.484375 -0.234375q0.140625 0.703125 0.140625 1.265625q0 0.90625 -0.28125 1.40625q-0.296875 0.5 -0.75 0.703125q-0.46875 0.203125 -1.984375 0.203125l-5.65625 0l0 1.234375l-1.3125 0l0 -1.234375l-2.4375 0l-1.0 -1.65625l3.4375 0l0 -1.6875l1.3125 0l0 1.6875l5.75 0q0.71875 0 0.921875 -0.078125q0.
 203125 -0.09375 0.328125 -0.296875q0.125 -0.203125 0.125 -0.578125q0 -0.265625 -0.078125 -0.734375zm1.5 -1.5114288l-9.859375 0l0 -1.5l1.5 0q-1.046875 -0.578125 -1.375 -1.0625q-0.34375 -0.484375 -0.34375 -1.078125q0 -0.84375 0.546875 -1.71875l1.546875 0.578125q-0.359375 0.609375 -0.359375 1.234375q0 0.546875 0.328125 0.984375q0.328125 0.421875 0.90625 0.609375q0.890625 0.28125 1.953125 0.28125l5.15625 0l0 1.671875zm-3.171875 -12.978294l0.21875 -1.71875q1.5 0.40625 2.34375 1.515625q0.828125 1.09375 0.828125 2.8125q0 2.1562424 -1.328125 3.4218674q-1.328125 1.265625 -3.734375 1.265625q-2.484375 0 -3.859375 -1.265625q-1.375 -1.28125 -1.375 -3.3281174q0 -1.984375 1.34375 -3.234375q1.34375 -1.25 3.796875 -1.25q0.140625 0 0.4375 0.015625l0 7.3437424q1.625 -0.09375 2.484375 -0.921875q0.859375 -0.8281174 0.859375 -2.0624924q0 -0.90625 -0.46875 -1.546875q-0.484375 -0.65625 -1.546875 -1.046875zm-2.703125 5.4843674l0 -5.4999924q-1.234375 0.109375 -1.859375 0.625q-0.96875 0.796875 -0.96875 2.0781
 25q0 1.140625 0.78125 1.9374924q0.765625 0.78125 2.046875 0.859375zm4.65625 -15.547585q0.796875 0.9375 1.125 1.796875q0.3125 0.859375 0.3125 1.84375q0 1.609375 -0.78125 2.484375q-0.796875 0.875 -2.03125 0.875q-0.734375 0 -1.328125 -0.328125q-0.59375 -0.328125 -0.953125 -0.859375q-0.359375 -0.53125 -0.546875 -1.203125q-0.140625 -0.5 -0.25 -1.484375q-0.25 -2.03125 -0.578125 -2.984375q-0.34375 0 -0.4375 0q-1.015625 0 -1.4375 0.46875q-0.5625 0.640625 -0.5625 1.90625q0 1.171875 0.40625 1.734375q0.40625 0.5625 1.46875 0.828125l-0.234375 1.640625q-1.046875 -0.234375 -1.6875 -0.734375q-0.640625 -0.515625 -0.984375 -1.46875q-0.359375 -0.96875 -0.359375 -2.25q0 -1.265625 0.296875 -2.046875q0.296875 -0.78125 0.75 -1.15625q0.453125 -0.375 1.140625 -0.515625q0.421875 -0.09375 1.53125 -0.09375l2.234375 0q2.328125 0 2.953125 -0.09375q0.609375 -0.109375 1.171875 -0.4375l0 1.75q-0.515625 0.265625 -1.21875 0.328125zm-3.71875 0.140625q0.359375 0.90625 0.625 2.734375q0.140625 1.03125 0.328125 1.453125q
 0.1875 0.421875 0.546875 0.65625q0.359375 0.234375 0.796875 0.234375q0.671875 0 1.125 -0.5q0.4375 -0.515625 0.4375 -1.484375q0 -0.96875 -0.421875 -1.71875q-0.4375 -0.75 -1.15625 -1.109375q-0.578125 -0.265625 -1.671875 -0.265625l-0.609375 0zm4.9375 -4.078842l-9.859375 0l0 -1.5l1.390625 0q-0.71875 -0.453125 -1.15625 -1.21875q-0.453125 -0.78125 -0.453125 -1.765625q0 -1.09375 0.453125 -1.796875q0.453125 -0.703125 1.28125 -0.984375q-1.734375 -1.171875 -1.734375 -3.046875q0 -1.46875 0.8125 -2.25q0.8125 -0.796875 2.5 -0.796875l6.765625 0l0 1.671875l-6.203125 0q-1.0 0 -1.4375 0.15625q-0.453125 0.15625 -0.71875 0.59375q-0.265625 0.421875 -0.265625 1.0q0 1.03125 0.6875 1.71875q0.6875 0.6875 2.21875 0.6875l5.71875 0l0 1.671875l-6.40625 0q-1.109375 0 -1.65625 0.40625q-0.5625 0.40625 -0.5625 1.34375q0 0.703125 0.375 1.3125q0.359375 0.59375 1.078125 0.859375q0.71875 0.265625 2.0625 0.265625l5.109375 0l0 1.671875z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m39.149605 5.
 5385365l90.960625 0l0 31.370079l-90.960625 0z" fill-rule="nonzero"></path><path fill="#000000" d="m49.149605 27.338535l0 -9.3125l3.515625 0q0.921875 0 1.40625 0.09375q0.6875 0.109375 1.15625 0.4375q0.46875 0.3125 0.75 0.890625q0.28125 0.578125 0.28125 1.28125q0 1.1875 -0.765625 2.015625q-0.75 0.8125 -2.71875 0.8125l-2.390625 0l0 3.78125l-1.234375 0zm1.234375 -4.875l2.40625 0q1.1875 0 1.6875 -0.4375q0.515625 -0.453125 0.515625 -1.265625q0 -0.578125 -0.296875 -0.984375q-0.296875 -0.421875 -0.78125 -0.5625q-0.3125 -0.078125 -1.15625 -0.078125l-2.375 0l0 3.328125zm11.90538 4.046875q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125
  0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.0999756 3.390625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.2187
 5 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm6.9539948 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029877 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm5.46109 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.3906
 25 0.078125q0.203125 0 0.515625 -0.03125zm1.2029877 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm2.539215 -3.375q0 -1.875 1.03125 -2.765625q0.875 -0.75 2.125 -0.75q1.390625 0 2.265625 0.90625q0.890625 0.90625 0.890625 2.515625q0 1.296875 -0.390625 2.046875q-0.390625 0.75 -1.140625 1.171875q-0.75 0.40625 -1.625 0.40625q-1.421875 0 -2.296875 -0.90625q-0.859375 -0.90625 -0.859375 -2.625zm1.171875 0q0 1.296875 0.5625 1.953125q0.5625 0.640625 1.421875 0.640625q0.84375 0 1.40625 -0.640625q0.578125 -0.65625 0.578125 -1.984375q0 -1.25 -0.578125 -1.890625q-0.5625 -0.65625 -1.40625 -0.65625q-0.859375 0 -1.421875 0.640625q-0.5625 0.640625 -0.5625 1.9375zm6.6624756 3.375l0 -6.734375l1.03125 0l0 0.953125q0.734375 -1.109375 2.140625 -1.109375q0.609375 0 1.109375 0.21875q0.515625 0.21875 0.765625 0.578125q0.265625 0.34375 0.359375 0.84375q0.0625 0.3125 0.0625 1.109375l0 4.140625l-1.140625 0l0 -4.09375q0 -0.703125 -0.140625 -1.046
 875q-0.125 -0.34375 -0.46875 -0.546875q-0.328125 -0.21875 -0.78125 -0.21875q-0.734375 0 -1.265625 0.46875q-0.53125 0.453125 -0.53125 1.75l0 3.6875l-1.140625 0zm10.802963 -4.59375q0 -1.640625 0.328125 -2.640625q0.34375 -1.015625 1.015625 -1.5625q0.671875 -0.546875 1.6875 -0.546875q0.75 0 1.3125 0.3125q0.5625 0.296875 0.921875 0.859375q0.375 0.5625 0.578125 1.390625q0.21875 0.8125 0.21875 2.1875q0 1.640625 -0.34375 2.65625q-0.328125 1.0 -1.0 1.546875q-0.671875 0.546875 -1.6875 0.546875q-1.34375 0 -2.125 -0.96875q-0.90625 -1.15625 -0.90625 -3.78125zm1.171875 0q0 2.296875 0.53125 3.0625q0.53125 0.75 1.328125 0.75q0.78125 0 1.3125 -0.75q0.546875 -0.765625 0.546875 -3.0625q0 -2.296875 -0.546875 -3.046875q-0.53125 -0.75 -1.328125 -0.75q-0.78125 0 -1.265625 0.65625q-0.578125 0.859375 -0.578125 3.140625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m39.149605 59.374046l90.960625 0l0 32.850395l-90.960625 0z" fill-rule="nonzero"></path><path fill="#000000" d="m49.1496
 05 81.17404l0 -9.3125l3.515625 0q0.921875 0 1.40625 0.09375q0.6875 0.109375 1.15625 0.4375q0.46875 0.3125 0.75 0.890625q0.28125 0.578125 0.28125 1.28125q0 1.1875 -0.765625 2.015625q-0.75 0.8125 -2.71875 0.8125l-2.390625 0l0 3.78125l-1.234375 0zm1.234375 -4.875l2.40625 0q1.1875 0 1.6875 -0.4375q0.515625 -0.453125 0.515625 -1.265625q0 -0.578125 -0.296875 -0.984375q-0.296875 -0.421875 -0.78125 -0.5625q-0.3125 -0.078125 -1.15625 -0.078125l-2.375 0l0 3.328125zm11.90538 4.046875q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.3
 59375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.0999756 3.390625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm6.9539948 -1.015625l0.15625 1.0q
 -0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029877 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm5.46109 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029877 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0z
 m0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm2.539215 -3.375q0 -1.875 1.03125 -2.765625q0.875 -0.75 2.125 -0.75q1.390625 0 2.265625 0.90625q0.890625 0.90625 0.890625 2.515625q0 1.296875 -0.390625 2.046875q-0.390625 0.75 -1.140625 1.171875q-0.75 0.40625 -1.625 0.40625q-1.421875 0 -2.296875 -0.90625q-0.859375 -0.90625 -0.859375 -2.625zm1.171875 0q0 1.296875 0.5625 1.953125q0.5625 0.640625 1.421875 0.640625q0.84375 0 1.40625 -0.640625q0.578125 -0.65625 0.578125 -1.984375q0 -1.25 -0.578125 -1.890625q-0.5625 -0.65625 -1.40625 -0.65625q-0.859375 0 -1.421875 0.640625q-0.5625 0.640625 -0.5625 1.9375zm6.6624756 3.375l0 -6.734375l1.03125 0l0 0.953125q0.734375 -1.109375 2.140625 -1.109375q0.609375 0 1.109375 0.21875q0.515625 0.21875 0.765625 0.578125q0.265625 0.34375 0.359375 0.84375q0.0625 0.3125 0.0625 1.109375l0 4.140625l-1.140625 0l0 -4.09375q0 -0.703125 -0.140625 -1.046875q-0.125 -0.34375 -0.46875 -0.546875q-0.328125 -0.21875 -0.78125 -0.21875q-0.734375 0 -1.265625 0.46875q-
 0.53125 0.453125 -0.53125 1.75l0 3.6875l-1.140625 0zm15.099838 0l-1.140625 0l0 -7.28125q-0.421875 0.390625 -1.09375 0.796875q-0.65625 0.390625 -1.1875 0.578125l0 -1.109375q0.953125 -0.4375 1.671875 -1.078125q0.71875 -0.640625 1.015625 -1.25l0.734375 0l0 9.34375z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m39.149605 156.30971l90.960625 0l0 29.35434l-90.960625 0z" fill-rule="nonzero"></path><path fill="#000000" d="m49.149605 178.10971l0 -9.3125l3.515625 0q0.921875 0 1.40625 0.09375q0.6875 0.109375 1.15625 0.4375q0.46875 0.3125 0.75 0.890625q0.28125 0.578125 0.28125 1.28125q0 1.1875 -0.765625 2.015625q-0.75 0.8125 -2.71875 0.8125l-2.390625 0l0 3.78125l-1.234375 0zm1.234375 -4.875l2.40625 0q1.1875 0 1.6875 -0.4375q0.515625 -0.453125 0.515625 -1.265625q0 -0.578125 -0.296875 -0.984375q-0.296875 -0.421875 -0.78125 -0.5625q-0.3125 -0.078125 -1.15625 -0.078125l-2.375 0l0 3.328125zm11.90538 4.046875q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-
 1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1
 .171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.0999756 3.390625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm6.9539948 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029877 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm5.46109 -1.015625l0.15625 1.0q-
 0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029877 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm2.539215 -3.375q0 -1.875 1.03125 -2.765625q0.875 -0.75 2.125 -0.75q1.390625 0 2.265625 0.90625q0.890625 0.90625 0.890625 2.515625q0 1.296875 -0.390625 2.046875q-0.390625 0.75 -1.140625 1.171875q-0.75 0.40625 -1.625 0.40625q-1.421875 0 -2.296875 -0.90625q-0.859375 -0.90625 -0.859375 -2.625zm1.171875 0q0 1.296875 0.5625 1.953125q0.5625 0.640625 1.421875 0.640625q0.84375 0 1.40625 -0.640625q0.578125 -0.65625 0.578125 -1.984375q0 -1.25 -0.578125 -1.890625q-0.5625 -0.65625 -1.40625 -0.
 65625q-0.859375 0 -1.421875 0.640625q-0.5625 0.640625 -0.5625 1.9375zm6.6624756 3.375l0 -6.734375l1.03125 0l0 0.953125q0.734375 -1.109375 2.140625 -1.109375q0.609375 0 1.109375 0.21875q0.515625 0.21875 0.765625 0.578125q0.265625 0.34375 0.359375 0.84375q0.0625 0.3125 0.0625 1.109375l0 4.140625l-1.140625 0l0 -4.09375q0 -0.703125 -0.140625 -1.046875q-0.125 -0.34375 -0.46875 -0.546875q-0.328125 -0.21875 -0.78125 -0.21875q-0.734375 0 -1.265625 0.46875q-0.53125 0.453125 -0.53125 1.75l0 3.6875l-1.140625 0zm16.802963 -1.09375l0 1.09375l-6.15625 0q-0.015625 -0.40625 0.140625 -0.796875q0.234375 -0.625 0.75 -1.234375q0.515625 -0.609375 1.5 -1.40625q1.515625 -1.25 2.046875 -1.96875q0.53125 -0.734375 0.53125 -1.375q0 -0.6875 -0.484375 -1.140625q-0.484375 -0.46875 -1.265625 -0.46875q-0.828125 0 -1.328125 0.5q-0.484375 0.484375 -0.5 1.359375l-1.171875 -0.125q0.125 -1.3125 0.90625 -2.0q0.78125 -0.6875 2.109375 -0.6875q1.34375 0 2.125 0.75q0.78125 0.734375 0.78125 1.828125q0 0.5625 -0.234375 1.1093
 75q-0.21875 0.53125 -0.75 1.140625q-0.53125 0.59375 -1.765625 1.625q-1.03125 0.859375 -1.328125 1.171875q-0.28125 0.3125 -0.46875 0.625l4.5625 0z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m319.53543 48.850395l160.0 0l0 44.53543l-160.0 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m319.53543 48.850395l160.0 0l0 44.53543l-160.0 0z" fill-rule="nonzero"></path><path fill="#000000" d="m333.0759 72.93373l1.171875 -0.109375q0.078125 0.703125 0.375 1.15625q0.3125 0.4375 0.9375 0.71875q0.640625 0.265625 1.4375 0.265625q0.703125 0 1.234375 -0.203125q0.546875 -0.203125 0.8125 -0.5625q0.265625 -0.375 0.265625 -0.8125q0 -0.4375 -0.265625 -0.765625q-0.25 -0.328125 -0.828125 -0.546875q-0.375 -0.140625 -1.65625 -0.453125q-1.28125 -0.3125 -1.796875 -0.578125q-0.671875 -0.34375 -1.0 -0.859375q-0.328125 -0.53125 -0.328125 -1.171875q0 -0.703125 0.390625 -1.3125q0.40625 -0.609375 1.171875 -0.921875q0
 .78125 -0.328125 1.71875 -0.328125q1.03125 0 1.8125 0.34375q0.796875 0.328125 1.21875 0.984375q0.4375 0.640625 0.46875 1.453125l-1.1875 0.09375q-0.09375 -0.890625 -0.640625 -1.328125q-0.546875 -0.453125 -1.625 -0.453125q-1.109375 0 -1.625 0.40625q-0.515625 0.40625 -0.515625 0.984375q0 0.5 0.359375 0.828125q0.359375 0.328125 1.859375 0.671875q1.5 0.328125 2.0625 0.578125q0.8125 0.375 1.1875 0.953125q0.390625 0.578125 0.390625 1.328125q0 0.734375 -0.421875 1.390625q-0.421875 0.65625 -1.21875 1.03125q-0.796875 0.359375 -1.796875 0.359375q-1.265625 0 -2.125 -0.359375q-0.84375 -0.375 -1.328125 -1.109375q-0.484375 -0.75 -0.515625 -1.671875zm11.67099 1.96875l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.
 140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.1873779 1.015625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm9.0633545 -2.171875l1.1875 0.140625q-0.28125 1.046875 -1.046875 1.625q-0.75 0.5625 -1.921875 0.5625q-1.484375 0 -2.359375 -0.90625q-0.859375 -0.921875 -0.859375 -2.5625q0 -1.703125 0.875 -2.640625q0.890625 -0.9375 2.28125 -0.9375q1.359375 0 2.203125 0.921875q0.859375 0.921875 0.859375 2.578125q0 0.109375 0 0.3125l-5.03125 0q0.0625 1.109375 0.625 1.703125q0.5625 0.59375 1.40625 0.59375q0.640625 0 1.078125 -0.328125q0.453125 -0.34375 0.703125 -1.0625zm-3.75 -1.84375l3.765625 0q-0.078125 -0.859375 -0.4375 -1.28125q-0.546875 -0.65625 -1.40625 -0.65625q-0.796875 0 -1.328125 0.53125q-0.53125 0.515625 -0.59375 1.40
 625zm10.943726 3.1875q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.
 15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.1156006 3.390625l0 -6.734375l1.015625 0l0 0.9375q0.328125 -0.5 0.84375 -0.796875q0.53125 -0.296875 1.203125 -0.296875q0.75 0 1.21875 0.3125q0.484375 0.3125 0.6875 0.859375q0.796875 -1.171875 2.078125 -1.171875q1.0 0 1.53125 0.5625q0.546875 0.546875 0.546875 1.703125l0 4.625l-1.125 0l0 -4.25q0 -0.6875 -0.109375 -0.984375q-0.109375 -0.296875 -0.40625 -0.484375q-0.296875 -0.1875 -0.6875 -0.1875q-0.71875 0 -1.1875 0.484375q-0.46875 0.46875 -0.46875 1.5l0 3.921875l-1.140625 0l0 -4.375q0 -0.765625 -0.28125 -1.140625q-0.28125 -0.390625 -0.90625 -0.390625q-0.484375 0 -0.890625 0.265625q-0.40625 0.25 -0.59375 0.734375q-0.1875 0.484375 -0.1875 1.40625l0 3.5l-1.140625 0zm13.6180725 0l0 -8.203125l-3.0625 0l0 -1.109375l7.375 0l0 1.109375l-3.078125 0l0 8.203125l-1.234375 0zm10.016357 -0.828125q-0.625 0.5312
 5 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.
 765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm2.6624756 1.375l1.125 -0.171875q0.09375 0.671875 0.53125 1.046875q0.4375 0.359375 1.21875 0.359375q0.78125 0 1.15625 -0.3125q0.390625 -0.328125 0.390625 -0.765625q0 -0.390625 -0.34375 -0.609375q-0.234375 -0.15625 -1.171875 -0.390625q-1.25 -0.3125 -1.734375 -0.546875q-0.484375 -0.234375 -0.734375 -0.640625q-0.25 -0.40625 -0.25 -0.90625q0 -0.453125 0.203125 -0.828125q0.203125 -0.390625 0.5625 -0.640625q0.265625 -0.203125 0.71875 -0.328125q0.46875 -0.140625 1.0 -0.140625q0.78125 0 1.375 0.234375q0.609375 0.21875 0.890625 0.609375q0.296875 0.390625 0.40625 1.046875l-1.125 0.15625q-0.078125 -0.53125 -0.4375 -0.8125q-0.359375 -0.296875 -1.03125 -0.296875q-0.78125 0 -1.125 0.265625q-0.34375 0.25 -0.34375 0.609375q0 0.21875 0.140625 0.390625q0.140625 0.1875 0.4375 0.3125q0.171875 0.0625 1.015625 0.28125q1.21875 0.328125 1.6875 0.53125q0.
 484375 0.203125 0.75 0.609375q0.28125 0.390625 0.28125 0.96875q0 0.578125 -0.34375 1.078125q-0.328125 0.5 -0.953125 0.78125q-0.625 0.28125 -1.421875 0.28125q-1.3125 0 -2.0 -0.546875q-0.6875 -0.546875 -0.875 -1.625zm7.1171875 2.015625l0 -9.3125l1.140625 0l0 5.3125l2.703125 -2.734375l1.484375 0l-2.578125 2.5l2.84375 4.234375l-1.40625 0l-2.234375 -3.453125l-0.8125 0.78125l0 2.671875l-1.140625 0zm10.367035 2.578125l0 -9.3125l1.03125 0l0 0.875q0.375 -0.515625 0.828125 -0.765625q0.46875 -0.265625 1.140625 -0.265625q0.859375 0 1.515625 0.453125q0.65625 0.4375 0.984375 1.25q0.34375 0.796875 0.34375 1.765625q0 1.03125 -0.375 1.859375q-0.359375 0.828125 -1.078125 1.28125q-0.703125 0.4375 -1.484375 0.4375q-0.5625 0 -1.015625 -0.234375q-0.453125 -0.25 -0.75 -0.625l0 3.28125l-1.140625 0zm1.03125 -5.90625q0 1.296875 0.53125 1.921875q0.53125 0.625 1.265625 0.625q0.765625 0 1.3125 -0.640625q0.546875 -0.65625 0.546875 -2.0q0 -1.296875 -0.53125 -1.9375q-0.53125 -0.640625 -1.265625 -0.640625q-0.734375
  0 -1.296875 0.6875q-0.5625 0.671875 -0.5625 1.984375zm10.771851 2.5q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296
 875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.0999756 3.390625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm6.95401 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029724 -6.96875l0 -
 1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm5.46109 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2030029 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm2.539215 -3.375q0 -1.875 1.03125 -2.765625q0.875 -0.75 2.125 -0.75q1.390625 0 2.265625 0.90625q0.890625 0.90625 0.890625 2.515625q0 1.296875 -0.390625 2.046875q-0.390625 0.75 -1.140625 1.171875q-0.75 0.40625 -1.625 0.40625q-1.421875 0 -2.296875 -0.90625q-0.859375 -0.90625 -0.859375 -2.625zm1.171875 0q0 1.296875 0.5625 1.953125q0.5625 0.640625 1.42187
 5 0.640625q0.84375 0 1.40625 -0.640625q0.578125 -0.65625 0.578125 -1.984375q0 -1.25 -0.578125 -1.890625q-0.5625 -0.65625 -1.40625 -0.65625q-0.859375 0 -1.421875 0.640625q-0.5625 0.640625 -0.5625 1.9375zm6.6624756 3.375l0 -6.734375l1.03125 0l0 0.953125q0.734375 -1.109375 2.140625 -1.109375q0.609375 0 1.109375 0.21875q0.515625 0.21875 0.765625 0.578125q0.265625 0.34375 0.359375 0.84375q0.0625 0.3125 0.0625 1.109375l0 4.140625l-1.140625 0l0 -4.09375q0 -0.703125 -0.140625 -1.046875q-0.125 -0.34375 -0.46875 -0.546875q-0.328125 -0.21875 -0.78125 -0.21875q-0.734375 0 -1.265625 0.46875q-0.53125 0.453125 -0.53125 1.75l0 3.6875l-1.140625 0zm10.802948 -4.59375q0 -1.640625 0.328125 -2.640625q0.34375 -1.015625 1.015625 -1.5625q0.671875 -0.546875 1.6875 -0.546875q0.75 0 1.3125 0.3125q0.5625 0.296875 0.921875 0.859375q0.375 0.5625 0.578125 1.390625q0.21875 0.8125 0.21875 2.1875q0 1.640625 -0.34375 2.65625q-0.328125 1.0 -1.0 1.546875q-0.671875 0.546875 -1.6875 0.546875q-1.34375 0 -2.125 -0.96875q-0
 .90625 -1.15625 -0.90625 -3.78125zm1.171875 0q0 2.296875 0.53125 3.0625q0.53125 0.75 1.328125 0.75q0.78125 0 1.3125 -0.75q0.546875 -0.765625 0.546875 -3.0625q0 -2.296875 -0.546875 -3.046875q-0.53125 -0.75 -1.328125 -0.75q-0.78125 0 -1.265625 0.65625q-0.578125 0.859375 -0.578125 3.140625z" fill-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m319.53543 103.25197l160.0 0l0 44.53543l-160.0 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m319.53543 103.25197l160.0 0l0 44.53543l-160.0 0z" fill-rule="nonzero"></path><path fill="#000000" d="m333.0759 127.33531l1.171875 -0.109375q0.078125 0.703125 0.375 1.15625q0.3125 0.4375 0.9375 0.71875q0.640625 0.265625 1.4375 0.265625q0.703125 0 1.234375 -0.203125q0.546875 -0.203125 0.8125 -0.5625q0.265625 -0.375 0.265625 -0.8125q0 -0.4375 -0.265625 -0.765625q-0.25 -0.328125 -0.828125 -0.546875q-0.375 -0.140625 -1.65625 -0.453125q-1.28125 -0.3125 -1.796875 -0.578
 125q-0.671875 -0.34375 -1.0 -0.859375q-0.328125 -0.53125 -0.328125 -1.171875q0 -0.703125 0.390625 -1.3125q0.40625 -0.609375 1.171875 -0.921875q0.78125 -0.328125 1.71875 -0.328125q1.03125 0 1.8125 0.34375q0.796875 0.328125 1.21875 0.984375q0.4375 0.640625 0.46875 1.453125l-1.1875 0.09375q-0.09375 -0.890625 -0.640625 -1.328125q-0.546875 -0.453125 -1.625 -0.453125q-1.109375 0 -1.625 0.40625q-0.515625 0.40625 -0.515625 0.984375q0 0.5 0.359375 0.828125q0.359375 0.328125 1.859375 0.671875q1.5 0.328125 2.0625 0.578125q0.8125 0.375 1.1875 0.953125q0.390625 0.578125 0.390625 1.328125q0 0.734375 -0.421875 1.390625q-0.421875 0.65625 -1.21875 1.03125q-0.796875 0.359375 -1.796875 0.359375q-1.265625 0 -2.125 -0.359375q-0.84375 -0.375 -1.328125 -1.109375q-0.484375 -0.75 -0.515625 -1.671875zm11.67099 1.96875l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.82
 8125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.1873779 1.015625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm9.0633545 -2.171875l1.1875 0.140625q-0.28125 1.046875 -1.046875 1.625q-0.75 0.5625 -1.921875 0.5625q-1.484375 0 -2.359375 -0.90625q-0.859375 -0.921875 -0.859375 -2.5625q0 -1.703125 0.875 -2.640625q0.890625 -0.9375 2.28125 -0.9375q1.359375 0 2.203125 0.921875q0.859375 0.921875 0.859375 2.578125q0 0.109375 0 0.3125l-5.03125 0q0.0625 1.109375 0.625 1.703125q0.5625 0.59375 1.40625 0.59375q0.640625 0 1.078125 -0.328125q0.453125 -0.34375 0.703125 -1.0625zm-3.75 -1.84375l3.
 765625 0q-0.078125 -0.859375 -0.4375 -1.28125q-0.546875 -0.65625 -1.40625 -0.65625q-0.796875 0 -1.328125 0.53125q-0.53125 0.515625 -0.59375 1.40625zm10.943726 3.1875q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.
 828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.1156006 3.390625l0 -6.734375l1.015625 0l0 0.9375q0.328125 -0.5 0.84375 -0.796875q0.53125 -0.296875 1.203125 -0.296875q0.75 0 1.21875 0.3125q0.484375 0.3125 0.6875 0.859375q0.796875 -1.171875 2.078125 -1.171875q1.0 0 1.53125 0.5625q0.546875 0.546875 0.546875 1.703125l0 4.625l-1.125 0l0 -4.25q0 -0.6875 -0.109375 -0.984375q-0.109375 -0.296875 -0.40625 -0.484375q-0.296875 -0.1875 -0.6875 -0.1875q-0.71875 0 -1.1875 0.484375q-0.46875 0.46875 -0.46875 1.5l0 3.921875l-1.140625 0l0 -4.375q0 -0.765625 -0.28125 -1.140625q-0.28125 -0.390625 -0.90625 -0.390625q-0.484375 0 -0.890625 0.265625q-0.40625 0.25 -0.59375 0.734375q-0.1875 0.484375 -0.1875 1.40625l0 3.5l-1.1
 40625 0zm13.6180725 0l0 -8.203125l-3.0625 0l0 -1.109375l7.375 0l0 1.109375l-3.078125 0l0 8.203125l-1.234375 0zm10.016357 -0.828125q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.2
 65625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm2.6624756 1.375l1.125 -0.171875q0.09375 0.671875 0.53125 1.046875q0.4375 0.359375 1.21875 0.359375q0.78125 0 1.15625 -0.3125q0.390625 -0.328125 0.390625 -0.765625q0 -0.390625 -0.34375 -0.609375q-0.234375 -0.15625 -1.171875 -0.390625q-1.25 -0.3125 -1.734375 -0.546875q-0.484375 -0.234375 -0.734375 -0.640625q-0.25 -0.40625 -0.25 -0.90625q0 -0.453125 0.203125 -0.828125q0.203125 -0.390625 0.5625 -0.640625q0.265625 -0.203125 0.71875 -0.328125q0.46875 -0.140625 1.0 -0.140625q0.78125 0 1.375 0.234375q0.609375 0.21875 0.890625 0.609375q0.296875 0.390625 0.40625 1.046875l-1.125 0.15625q-0.078125 -0.53125 -0.4375 -0.8125q-0.359375 -0.296875 -1.03125 -0.296875q-0.78125 0 -1.125 0.265625q-0.34375 0.25
  -0.34375 0.609375q0 0.21875 0.140625 0.390625q0.140625 0.1875 0.4375 0.3125q0.171875 0.0625 1.015625 0.28125q1.21875 0.328125 1.6875 0.53125q0.484375 0.203125 0.75 0.609375q0.28125 0.390625 0.28125 0.96875q0 0.578125 -0.34375 1.078125q-0.328125 0.5 -0.953125 0.78125q-0.625 0.28125 -1.421875 0.28125q-1.3125 0 -2.0 -0.546875q-0.6875 -0.546875 -0.875 -1.625zm7.1171875 2.015625l0 -9.3125l1.140625 0l0 5.3125l2.703125 -2.734375l1.484375 0l-2.578125 2.5l2.84375 4.234375l-1.40625 0l-2.234375 -3.453125l-0.8125 0.78125l0 2.671875l-1.140625 0zm10.367035 2.578125l0 -9.3125l1.03125 0l0 0.875q0.375 -0.515625 0.828125 -0.765625q0.46875 -0.265625 1.140625 -0.265625q0.859375 0 1.515625 0.453125q0.65625 0.4375 0.984375 1.25q0.34375 0.796875 0.34375 1.765625q0 1.03125 -0.375 1.859375q-0.359375 0.828125 -1.078125 1.28125q-0.703125 0.4375 -1.484375 0.4375q-0.5625 0 -1.015625 -0.234375q-0.453125 -0.25 -0.75 -0.625l0 3.28125l-1.140625 0zm1.03125 -5.90625q0 1.296875 0.53125 1.921875q0.53125 0.625 1.265625
  0.625q0.765625 0 1.3125 -0.640625q0.546875 -0.65625 0.546875 -2.0q0 -1.296875 -0.53125 -1.9375q-0.53125 -0.640625 -1.265625 -0.640625q-0.734375 0 -1.296875 0.6875q-0.5625 0.671875 -0.5625 1.984375zm10.771851 2.5q-0.625 0.53125 -1.21875 0.765625q-0.578125 0.21875 -1.25 0.21875q-1.125 0 -1.71875 -0.546875q-0.59375 -0.546875 -0.59375 -1.390625q0 -0.484375 0.21875 -0.890625q0.234375 -0.421875 0.59375 -0.671875q0.375 -0.25 0.828125 -0.375q0.328125 -0.078125 1.015625 -0.171875q1.375 -0.15625 2.03125 -0.390625q0.015625 -0.234375 0.015625 -0.296875q0 -0.703125 -0.328125 -0.984375q-0.4375 -0.390625 -1.296875 -0.390625q-0.8125 0 -1.203125 0.28125q-0.375 0.28125 -0.5625 1.0l-1.109375 -0.140625q0.140625 -0.71875 0.484375 -1.15625q0.359375 -0.453125 1.015625 -0.6875q0.671875 -0.234375 1.53125 -0.234375q0.875 0 1.40625 0.203125q0.546875 0.203125 0.796875 0.515625q0.25 0.296875 0.359375 0.765625q0.046875 0.296875 0.046875 1.0625l0 1.515625q0 1.59375 0.078125 2.015625q0.078125 0.421875 0.28125 0.8
 125l-1.1875 0q-0.171875 -0.359375 -0.234375 -0.828125zm-0.09375 -2.5625q-0.625 0.265625 -1.859375 0.4375q-0.703125 0.109375 -1.0 0.234375q-0.296875 0.125 -0.453125 0.375q-0.15625 0.234375 -0.15625 0.53125q0 0.453125 0.34375 0.765625q0.34375 0.296875 1.015625 0.296875q0.65625 0 1.171875 -0.28125q0.515625 -0.296875 0.765625 -0.796875q0.171875 -0.375 0.171875 -1.140625l0 -0.421875zm3.0999756 3.390625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1875 1.328125l0 3.53125l-1.140625 0zm6.95401 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125
 q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2029724 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm5.46109 -1.015625l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.2030029 -6.96875l0 -1.328125l1.140625 0l0 1.328125l-1.140625 0zm0 7.984375l0 -6.734375l1.140625 0l0 6.734375l-1.140625 0zm2.539215 -3.375q0 -1.875 1.03125 -2.765625q0.875 -0.75 2.125 -0.75q1.390625 0 2.265625 0.90625q0.890625 0.90625 0.890625 2.515625q0 1.296875 -0.390625 2.046875q-0.390625 0.75 -1.140625 1.171875q-0.75 0.40625 
 -1.625 0.40625q-1.421875 0 -2.296875 -0.90625q-0.859375 -0.90625 -0.859375 -2.625zm1.171875 0q0 1.296875 0.5625 1.953125q0.5625 0.640625 1.421875 0.640625q0.84375 0 1.40625 -0.640625q0.578125 -0.65625 0.578125 -1.984375q0 -1.25 -0.578125 -1.890625q-0.5625 -0.65625 -1.40625 -0.65625q-0.859375 0 -1.421875 0.640625q-0.5625 0.640625 -0.5625 1.9375zm6.6624756 3.375l0 -6.734375l1.03125 0l0 0.953125q0.734375 -1.109375 2.140625 -1.109375q0.609375 0 1.109375 0.21875q0.515625 0.21875 0.765625 0.578125q0.265625 0.34375 0.359375 0.84375q0.0625 0.3125 0.0625 1.109375l0 4.140625l-1.140625 0l0 -4.09375q0 -0.703125 -0.140625 -1.046875q-0.125 -0.34375 -0.46875 -0.546875q-0.328125 -0.21875 -0.78125 -0.21875q-0.734375 0 -1.265625 0.46875q-0.53125 0.453125 -0.53125 1.75l0 3.6875l-1.140625 0zm15.099823 0l-1.140625 0l0 -7.28125q-0.421875 0.390625 -1.09375 0.796875q-0.65625 0.390625 -1.1875 0.578125l0 -1.109375q0.953125 -0.4375 1.671875 -1.078125q0.71875 -0.640625 1.015625 -1.25l0.734375 0l0 9.34375z" fil
 l-rule="nonzero"></path><path fill="#000000" fill-opacity="0.0" d="m319.53543 199.64305l160.0 0l0 44.53543l-160.0 0z" fill-rule="nonzero"></path><path stroke="#000000" stroke-width="1.0" stroke-linejoin="round" stroke-linecap="butt" d="m319.53543 199.64305l160.0 0l0 44.53543l-160.0 0z" fill-rule="nonzero"></path><path fill="#000000" d="m333.0759 223.72638l1.171875 -0.109375q0.078125 0.703125 0.375 1.15625q0.3125 0.4375 0.9375 0.71875q0.640625 0.265625 1.4375 0.265625q0.703125 0 1.234375 -0.203125q0.546875 -0.203125 0.8125 -0.5625q0.265625 -0.375 0.265625 -0.8125q0 -0.4375 -0.265625 -0.765625q-0.25 -0.328125 -0.828125 -0.546875q-0.375 -0.140625 -1.65625 -0.453125q-1.28125 -0.3125 -1.796875 -0.578125q-0.671875 -0.34375 -1.0 -0.859375q-0.328125 -0.53125 -0.328125 -1.171875q0 -0.703125 0.390625 -1.3125q0.40625 -0.609375 1.171875 -0.921875q0.78125 -0.328125 1.71875 -0.328125q1.03125 0 1.8125 0.34375q0.796875 0.328125 1.21875 0.984375q0.4375 0.640625 0.46875 1.453125l-1.1875 0.09375q-0.09
 375 -0.890625 -0.640625 -1.328125q-0.546875 -0.453125 -1.625 -0.453125q-1.109375 0 -1.625 0.40625q-0.515625 0.40625 -0.515625 0.984375q0 0.5 0.359375 0.828125q0.359375 0.328125 1.859375 0.671875q1.5 0.328125 2.0625 0.578125q0.8125 0.375 1.1875 0.953125q0.390625 0.578125 0.390625 1.328125q0 0.734375 -0.421875 1.390625q-0.421875 0.65625 -1.21875 1.03125q-0.796875 0.359375 -1.796875 0.359375q-1.265625 0 -2.125 -0.359375q-0.84375 -0.375 -1.328125 -1.109375q-0.484375 -0.75 -0.515625 -1.671875zm11.67099 1.96875l0.15625 1.0q-0.484375 0.109375 -0.859375 0.109375q-0.625 0 -0.96875 -0.203125q-0.34375 -0.203125 -0.484375 -0.515625q-0.140625 -0.328125 -0.140625 -1.34375l0 -3.890625l-0.828125 0l0 -0.875l0.828125 0l0 -1.671875l1.140625 -0.6875l0 2.359375l1.15625 0l0 0.875l-1.15625 0l0 3.953125q0 0.484375 0.0625 0.625q0.0625 0.140625 0.1875 0.21875q0.140625 0.078125 0.390625 0.078125q0.203125 0 0.515625 -0.03125zm1.1873779 1.015625l0 -6.734375l1.03125 0l0 1.015625q0.390625 -0.71875 0.71875 -0.9375
 q0.34375 -0.234375 0.734375 -0.234375q0.578125 0 1.171875 0.359375l-0.390625 1.0625q-0.421875 -0.25 -0.828125 -0.25q-0.375 0 -0.6875 0.234375q-0.296875 0.21875 -0.421875 0.625q-0.1875 0.609375 -0.1

<TRUNCATED>

[7/7] git commit: SAMZA-7: Rewrite Container section of docs to bring it up-to-date. Reviewed by Jakob Homan.

Posted by ma...@apache.org.
SAMZA-7: Rewrite Container section of docs to bring it up-to-date. Reviewed by Jakob Homan.


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

Branch: refs/heads/master
Commit: c72223f994bd73523049238a9e2347bca7bd9f18
Parents: a037d6f
Author: Martin Kleppmann <mk...@linkedin.com>
Authored: Mon Jun 9 19:06:05 2014 +0100
Committer: Martin Kleppmann <mk...@linkedin.com>
Committed: Mon Jun 9 21:14:23 2014 +0100

----------------------------------------------------------------------
 docs/css/main.css                               |  27 +-
 .../documentation/container/checkpointing-2.png | Bin 18449 -> 0 bytes
 .../documentation/container/checkpointing.png   | Bin 14613 -> 0 bytes
 .../documentation/container/checkpointing.svg   |   4 +
 .../learn/documentation/container/job-flow.png  | Bin 31104 -> 0 bytes
 .../learn/documentation/container/metrics.png   | Bin 20542 -> 0 bytes
 .../container/tasks-and-partitions.png          | Bin 14362 -> 0 bytes
 .../container/tasks-and-partitions.svg          |   4 +
 docs/learn/documentation/0.7.0/api/overview.md  | 206 ++++++++-------
 .../0.7.0/comparisons/introduction.md           |   2 +-
 .../documentation/0.7.0/comparisons/mupd8.md    |   2 +-
 .../0.7.0/container/checkpointing.md            |  82 ++++--
 .../documentation/0.7.0/container/event-loop.md |  89 ++-----
 .../documentation/0.7.0/container/index.md      |  18 --
 docs/learn/documentation/0.7.0/container/jmx.md |  17 +-
 .../documentation/0.7.0/container/metrics.md    | 120 +++++----
 .../0.7.0/container/samza-container.md          |  66 +++++
 .../0.7.0/container/serialization.md            |  46 ++++
 .../0.7.0/container/state-management.md         | 263 ++++++++-----------
 .../documentation/0.7.0/container/streams.md    | 163 +++++-------
 .../0.7.0/container/task-runner.md              |  44 ----
 .../documentation/0.7.0/container/windowing.md  |  39 ++-
 docs/learn/documentation/0.7.0/index.html       |   5 +-
 .../0.7.0/introduction/architecture.md          |   6 +-
 .../documentation/0.7.0/jobs/configuration.md   |   9 +-
 .../documentation/0.7.0/jobs/job-runner.md      |   4 +-
 docs/learn/documentation/0.7.0/jobs/logging.md  |   8 +-
 .../learn/documentation/0.7.0/jobs/packaging.md |   2 +-
 .../learn/documentation/0.7.0/jobs/yarn-jobs.md |   6 +-
 .../0.7.0/yarn/application-master.md            |   8 +-
 .../learn/documentation/0.7.0/yarn/isolation.md |   2 +-
 31 files changed, 648 insertions(+), 594 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/css/main.css
----------------------------------------------------------------------
diff --git a/docs/css/main.css b/docs/css/main.css
index 4dc014c..44d4888 100755
--- a/docs/css/main.css
+++ b/docs/css/main.css
@@ -136,11 +136,14 @@ h4 {
 pre {
   border: 0px !important;
   border-radius: 0px !important;
-  overflow-x: auto;
+  overflow: scroll !important;
+  white-space: pre;
+  overflow-wrap: normal;
+  word-wrap: normal !important;
 }
 pre code {
-  overflow-wrap: normal;
   white-space: pre;
+  font-size: 12px;
 }
 th.header {
   cursor: pointer;
@@ -194,6 +197,26 @@ ul.documentation-list {
   list-style: none;
   padding-left: 20px;
 }
+img.diagram-large {
+  width: 100%;
+}
+table.documentation {
+  border-collapse: collapse;
+  font-size: 12px;
+  margin: 1em 0;
+}
+table.documentation th, table.documentation td {
+  text-align: left;
+  vertical-align: top;
+  border: 1px solid #888;
+  padding: 5px;
+}
+table.documentation th.nowrap, table.documentation td.nowrap {
+  white-space: nowrap;
+}
+table.documentation th {
+  background-color: #eee;
+}
 .footer {
   clear: both;
   position: relative;

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/img/0.7.0/learn/documentation/container/checkpointing-2.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/container/checkpointing-2.png b/docs/img/0.7.0/learn/documentation/container/checkpointing-2.png
deleted file mode 100644
index cfd85e1..0000000
Binary files a/docs/img/0.7.0/learn/documentation/container/checkpointing-2.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/c72223f9/docs/img/0.7.0/learn/documentation/container/checkpointing.png
----------------------------------------------------------------------
diff --git a/docs/img/0.7.0/learn/documentation/container/checkpointing.png b/docs/img/0.7.0/learn/documentation/container/checkpointing.png
deleted file mode 100644
index aa36240..0000000
Binary files a/docs/img/0.7.0/learn/documentation/container/checkpointing.png and /dev/null differ