You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by ce...@apache.org on 2016/09/20 16:28:32 UTC

incubator-metron git commit: METRON-433: Documentation update closes apache/incubator-metron#260

Repository: incubator-metron
Updated Branches:
  refs/heads/master 5843421f5 -> 319e61e7a


METRON-433: Documentation update closes apache/incubator-metron#260


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

Branch: refs/heads/master
Commit: 319e61e7a4d6d6872a2889168dc9f8fdda28321d
Parents: 5843421
Author: cstella <ce...@gmail.com>
Authored: Tue Sep 20 12:27:57 2016 -0400
Committer: cstella <ce...@gmail.com>
Committed: Tue Sep 20 12:27:57 2016 -0400

----------------------------------------------------------------------
 README.md                                       |  14 +
 metron-analytics/metron-maas-service/README.md  | 192 +++++++
 .../metron-maas-service/maas_arch.png           | Bin 0 -> 106286 bytes
 metron-platform/metron-common/README.md         | 567 ++++++++++++-------
 metron-platform/metron-enrichment/README.md     | 337 ++++++++---
 .../metron-enrichment/enrichment_arch.png       | Bin 0 -> 224822 bytes
 .../stellar/DocumentationGenerator.java         |  52 ++
 metron-platform/metron-indexing/README.md       |  55 ++
 metron-platform/metron-parsers/README.md        |  78 ++-
 metron-platform/metron-parsers/parser_arch.png  | Bin 0 -> 95962 bytes
 10 files changed, 995 insertions(+), 300 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/319e61e7/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index d1e8f1f..0cfcf9f 100644
--- a/README.md
+++ b/README.md
@@ -61,4 +61,18 @@ Metron's code:
 
 Option 3 is more likely to have the latest code.
 
+# Navigating the Architecture
 
+Metron is at its core a Kappa architecture with Apache Storm as the processing
+component and Apache Kafka as the unified data bus.
+
+Some high level links to the relevant subparts of the architecture, for
+more information:
+* [Parsers](metron-platform/metron-parsers) : Parsing data from kafka into the Metron data model and passing it downstream to Enrichment.  
+* [Enrichment](metron-platform/metron-enrichment) : Enriching data post-parsing and providing the ability to tag a message as an alert and assign a risk triage level via a custom rule language.
+* [Indexing](metron-platform/metron-indexing) : Indexing the data post-enrichment into HDFS, Elasticsearch or Solr.
+
+Some useful utilities that cross all of these parts of the architecture:
+* [Stellar](metron-platform/metron-common) : A custom data transformation language that is used throughout metron from simple field transformation to expressing triage rules.
+* [Model as a Service](metron-analytics/metron-maas-service) : A Yarn application which can deploy machine learning and statistical models onto the cluster along with the associated Stellar functions to be able to call out to them in a scalable manner.
+* [Data management](metron-platform/metron-data-management) : A set of data management utilities aimed at getting data into HBase in a format which will allow data flowing through metron to be enriched with the results.  Contains integrations with threat intelligence feeds exposed via TAXII as well as simple flat file structures.

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/319e61e7/metron-analytics/metron-maas-service/README.md
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-maas-service/README.md b/metron-analytics/metron-maas-service/README.md
new file mode 100644
index 0000000..d1172b1
--- /dev/null
+++ b/metron-analytics/metron-maas-service/README.md
@@ -0,0 +1,192 @@
+# Model Management Infrastructure
+
+##Introduction
+
+One of the main features envisioned and requested is the ability to augment the threat intelligence and enrichment processes with insights derived from machine learning or statistical models.  The challenges with this sort of infrastructure are
+* Applying the model may be sufficiently computationally/resource intensive that we need to support scaling via load balancing, which will require service discovery and management.
+* Models require out of band and frequent training to react to growing threats and new patterns that emerge.
+* Models should be language/environment agnostic as much as possible.  These should include small-data and big-data libraries and languages.
+
+To support a high throughput environment that is manageable, it is evident that 
+* Multiple versions of models will need to be exposed
+* Deployment should happen using Yarn to manage resources
+* Clients should have new model endpoints pushed to them
+
+##Architecture
+
+![Architecture](maas_arch.png)
+
+To support these requirements, the following components have been created:
+* A Yarn application which will listen for model deployment requests and upon execution, register their endpoints in zookeeper:
+  * Operation type: ADD, REMOVE, LIST
+  * Model Name
+  * Model Version
+  * Memory requirements (in megabytes)
+  * Number of instances
+* A command line deployment client which will localize the model payload onto HDFS and submit a model request
+* A Java client which will interact with zookeeper and receive updates about model state changes (new deployments, removals, etc.)
+* A series of Stellar functions for interacting with models deployed via the Model as a Service infrastructure.
+
+## `maas_service.sh`
+
+The `maas_service.sh` script starts the Yarn application which will listen for requests.  Right now the queue for the requests is a distributed queue stored in [zookeeper](http://curator.apache.org/curator-recipes/distributed-queue.html) for convenience.
+
+```
+./maas_service.sh
+usage: MaaSClient
+ -c,--create                          Flag to indicate whether to create
+                                      the domain specified with -domain.
+ -d,--domain <arg>                    ID of the timeline domain where the
+                                      timeline entities will be put
+ -e,--shell_env <arg>                 Environment for shell script.
+                                      Specified as env_key=env_val pairs
+ -h,--help                            This screen
+ -j,--jar <arg>                       Jar file containing the application
+                                      master
+ -l4j,--log_properties <arg>          log4j.properties file
+ -ma,--modify_acls <arg>              Users and groups that allowed to
+                                      modify the timeline entities in the
+                                      given domain
+ -mc,--master_vcores <arg>            Amount of virtual cores to be
+                                      requested to run the application
+                                      master
+ -mm,--master_memory <arg>            Amount of memory in MB to be
+                                      requested to run the application
+                                      master
+ -nle,--node_label_expression <arg>   Node label expression to determine
+                                      the nodes where all the containers
+                                      of this application will be
+                                      allocated, "" means containers can
+                                      be allocated anywhere, if you don't
+                                      specify the option, default
+                                      node_label_expression of queue will
+                                      be used.
+ -q,--queue <arg>                     RM Queue in which this application
+                                      is to be submitted
+ -t,--timeout <arg>                   Application timeout in milliseconds
+ -va,--view_acls <arg>                Users and groups that allowed to
+                                      view the timeline entities in the
+                                      given domain
+ -zq,--zk_quorum <arg>                Zookeeper Quorum
+ -zr,--zk_root <arg>                  Zookeeper Root
+```
+
+## `maas_deploy.sh`
+
+The `maas_deploy.sh` script allows users to deploy models and their collateral from their local disk to the cluster.
+It is assumed that the 
+* Collateral has exactly one `.sh` script capable of starting the endpoint
+* The model service executable will expose itself as a URL endpoint (e.g. as a REST interface, but not necessarily)
+* The model service executable will write out to local disk a JSON blob indicating the endpoint (see [here](https://gist.github.com/cestella/cba10aff0f970078a4c2c8cade3a4d1a#file-dga-py-L21) for an example mock service using Python and Flask).
+
+```
+./maas_deploy.sh
+usage: ModelSubmission
+ -h,--help                       This screen
+ -hmp,--hdfs_model_path <arg>    Model Path (HDFS)
+ -lmp,--local_model_path <arg>   Model Path (local)
+ -m,--memory <arg>               Memory for container
+ -mo,--mode <arg>                ADD, LIST or REMOVE
+ -n,--name <arg>                 Model Name
+ -ni,--num_instances <arg>       Number of model instances
+ -v,--version <arg>              Model version
+ -zq,--zk_quorum <arg>           Zookeeper Quorum
+ -zr,--zk_root <arg>             Zookeeper Root
+```
+
+##Stellar Integration
+
+Two Stellar functions have been added to provide the ability to call out to models deployed via Model as a Service.
+One aimed at recovering a load balanced endpoint of a deployed model given the name and, optionally, the version.
+The second is aimed at calling that endpoint assuming that it is exposed as a REST endpoint.
+
+* `MAAS_MODEL_APPLY(endpoint, function?, model_args)` : Returns the output of a model deployed via model which is deployed at endpoint.  `endpoint` is a map containing `name`, `version`, `url` for the REST endpoint, `function` is the endpoint path and is optional, and `model_args` is a dictionary of arguments for the model (these become request params).
+* `MAAS_GET_ENDPOINT(model_name, model_version?)` : Inspects zookeeper and returns a map containing the `name`, `version` and `url` for the model referred to by `model_name` and `model_version`.  If `model_version` is not specified, the most current model associated with `model_name` is returned.  In the instance where more than one model is deployed, a random one is selected with uniform probability.
+
+# Example
+
+Let's augment the `squid` proxy sensor to use a model that will determine if the destination host is a domain generating algorithm.  For the purposes of demonstration, this algorithm is super simple and is implemented using Python with a REST interface exposed via the Flask python library.
+
+## Install Prerequisites and Mock DGA Service
+Now let's install some prerequisites:
+* Flask via `yum install python-flask`
+* Jinja2 via `yum install python-jinja2`
+* Squid client via `yum install squid`
+* ES Head plugin via `/usr/share/elasticsearch/bin/plugin install mobz/elasticsearch-head`
+
+Start Squid via `service squid start`
+
+Now that we have flask and jinja, we can create a mock DGA service to deploy with MaaS:
+* Download the files in [this](https://gist.github.com/cestella/cba10aff0f970078a4c2c8cade3a4d1a) gist into the `/root/mock_dga` directory
+* Make `rest.sh` executable via `chmod +x /root/mock_dga/rest.sh`
+
+This service will treat `yahoo.com` and `amazon.com` as legit and everything else as malicious.  The contract is that the REST service exposes an endpoint `/apply` and returns back JSON maps with a single key `is_malicious` which can be `malicious` or `legit`.
+
+## Deploy Mock DGA Service via MaaS
+
+Now let's start MaaS and deploy the Mock DGA Service:
+* Start MaaS via `$METRON_HOME/bin/maas_service.sh -zq node1:2181`
+* Start one instance of the mock DGA model with 512M of memory via `$METRON_HOME/bin/maas_deploy.sh -zq node1:2181 -lmp /root/mock_dga -hmp /user/root/models -mo ADD -m 512 -n dga -v 1.0 -ni 1`
+* As a sanity check:
+  * Ensure that the model is running via `$METRON_HOME/bin/maas_deploy.sh -zq node1:2181 -mo LIST`.  You should see `Model dga @ 1.0` be displayed and under that a url such as (but not exactly) `http://node1:36161`
+  * Try to hit the model via curl: `curl 'http://localhost:36161/apply?host=caseystella.com'` and ensure that it returns a JSON map indicating the domain is malicious.
+
+## Adjust Configurations for Squid to Call Model
+Now that we have a deployed model, let's adjust the configurations for the Squid topology to annotate the messages with the output of the model.
+
+* Edit the squid parser configuration at `$METRON_HOME/config/zookeeper/parsers/squid.json` in your favorite text editor and add a new FieldTransformation to indicate a threat alert based on the model (note the addition of `is_malicious` and `is_alert`):
+```
+{
+  "parserClassName": "org.apache.metron.parsers.GrokParser",
+  "sensorTopic": "squid",
+  "parserConfig": {
+    "grokPath": "/patterns/squid",
+    "patternLabel": "SQUID_DELIMITED",
+    "timestampField": "timestamp"
+  },
+  "fieldTransformations" : [
+    {
+      "transformation" : "STELLAR"
+    ,"output" : [ "full_hostname", "domain_without_subdomains", "is_malicious", "is_alert" ]
+    ,"config" : {
+      "full_hostname" : "URL_TO_HOST(url)"
+      ,"domain_without_subdomains" : "DOMAIN_REMOVE_SUBDOMAINS(full_hostname)"
+      ,"is_malicious" : "MAP_GET('is_malicious', MAAS_MODEL_APPLY(MAAS_GET_ENDPOINT('dga'), {'host' : domain_without_subdomains}))"
+      ,"is_alert" : "if is_malicious == 'malicious' then 'true' else null"
+                }
+    }
+                           ]
+}
+```
+* Edit the squid enrichment configuration at `$METRON_HOME/config/zookeeper/enrichments/squid.json` (this file will not exist, so create a new one) to make the threat triage adjust the level of risk based on the model output:
+```
+{
+  "index": "squid",
+  "batchSize": 1,
+  "enrichment" : {
+    "fieldMap": {}
+  },
+  "threatIntel" : {
+    "fieldMap":{},
+    "triageConfig" : {
+      "riskLevelRules" : {
+        "is_malicious == 'malicious'" : 100
+      },
+      "aggregator" : "MAX"
+    }
+  }
+}
+```
+* Upload new configs via `$METRON_HOME/bin/zk_load_configs.sh --mode PUSH -i $METRON_HOME/config/zookeeper -z node1:2181`
+* Make the Squid topic in kafka via `/usr/hdp/current/kafka-broker/bin/kafka-topics.sh --zookeeper node1:2181 --create --topic squid --partitions 1 --replication-factor 1`
+
+## Start Topologies and Send Data
+Now we need to start the topologies and send some data:
+* Start the squid topology via `$METRON_HOME/bin/start_parser_topology.sh -k node1:6667 -z node1:2181 -s squid`
+* Generate some data via the squid client:
+  * Generate a legit example: `squidclient http://yahoo.com`
+  * Generate a malicious example: `squidclient http://cnn.com`
+* Send the data to kafka via `cat /var/log/squid/access.log | /usr/hdp/current/kafka-broker/bin/kafka-console-producer.sh --broker-list node1:6667 --topic squid`
+* Browse the data in elasticsearch via the ES Head plugin @ [http://node1:9200/_plugin/head/](http://node1:9200/_plugin/head/) and verify that in the squid index you have two documents
+  * One from `yahoo.com` which does not have `is_alert` set and does have `is_malicious` set to `legit`
+  * One from `cnn.com` which does have `is_alert` set to `true`, `is_malicious` set to `malicious` and `threat:triage:level` set to 100

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/319e61e7/metron-analytics/metron-maas-service/maas_arch.png
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-maas-service/maas_arch.png b/metron-analytics/metron-maas-service/maas_arch.png
new file mode 100644
index 0000000..8409f70
Binary files /dev/null and b/metron-analytics/metron-maas-service/maas_arch.png differ

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/319e61e7/metron-platform/metron-common/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/README.md b/metron-platform/metron-common/README.md
index 2d3350e..367af7a 100644
--- a/metron-platform/metron-common/README.md
+++ b/metron-platform/metron-common/README.md
@@ -14,41 +14,368 @@ The query language supports the following:
 * if/then/else comparisons (i.e. `if var1 < 10 then 'less than 10' else '10 or more'`)
 * Determining whether a field exists (via `exists`)
 * The ability to have parenthesis to make order of operations explicit
-* A fixed set of functions which take strings and return boolean.  Currently:
-    * `IN_SUBNET(ip, cidr1, cidr2, ...)`
-    * `IS_EMPTY(str)`
-    * `STARTS_WITH(str, prefix)`
-    * `ENDS_WITH(str, suffix)`
-    * `REGEXP_MATCH(str, pattern)`
-    * `IS_IP` : Validates that the input fields are an IP address.  By default, if no second arg is set, it assumes `IPV4`, but you can specify the type by passing in either `IPV6` or `IPV4` to the second argument.
-    * `IS_DOMAIN` 
-    * `IS_EMAIL`
-    * `IS_URL`
-    * `IS_DATE`
-    * `IS_INTEGER`
-* A  fixed set of transformation functions:
-    * `TO_LOWER(string)` : Transforms the first argument to a lowercase string
-    * `TO_UPPER(string)` : Transforms the first argument to an uppercase string
-    * `TO_STRING(string)` : Transforms the first argument to a string
-    * `TO_INTEGER(x)` : Transforms the first argument to an integer 
-    * `TO_DOUBLE(x)` : Transforms the first argument to a double
-    * `TRIM(string)` : Trims whitespace from both sides of a string.
-    * `JOIN(list, delim)` : Joins the components of the list with the specified delimiter
-    * `SPLIT(string, delim)` : Splits the string by the delimiter.  Returns a list.
-    * `GET_FIRST(list)` : Returns the first element of the list
-    * `GET_LAST(list)` : Returns the last element of the list
-    * `GET(list, i)` : Returns the i'th element of the list (i is 0-based).
-    * `MAP_GET(key, map, default)` : Returns the value associated with the key in the map.  If the key does not exist, the default will be returned.  If the default is unspecified, then null will be returned.
-    * `DOMAIN_TO_TLD(domain)` : Returns the TLD of the domain.
-    * `DOMAIN_REMOVE_TLD(domain)` : Remove the TLD of the domain.
-    * `REMOVE_TLD(domain)` : Removes the TLD from the domain.
-    * `URL_TO_HOST(url)` : Returns the host from a URL
-    * `URL_TO_PROTOCOL(url)` : Returns the protocol from a URL
-    * `URL_TO_PORT(url)` : Returns the port from a URL
-    * `URL_TO_PATH(url)` : Returns the path from a URL
-    * `TO_EPOCH_TIMESTAMP(dateTime, format, timezone)` : Returns the epoch timestamp of the `dateTime` given the `format`.  If the format does not have a timestamp and you wish to assume a given timestamp, you may specify the `timezone` optionally.
-    * `MODEL_APPLY(endpoint, function?, model_args)` : Returns the output of a model deployed via model which is deployed at endpoint.  `endpoint` is a map containing `name`, `version`, `url` for the REST endpoint, `function` is the endpoint path and is optional, and `model_args` is a dictionary of arguments for the model (these become request params).
-    * `MAAS_GET_ENDPOINT(model_name, model_version?)` : Inspects zookeeper and returns a map containing the `name`, `version` and `url` for the model referred to by `model_name` and `model_version`.  If `model_version` is not specified, the most current model associated with `model_name` is returned.  In the instance where more than one model is deployed, a random one is selected with uniform probability.
+* User defined functions
+
+The following functions are supported:
+
+* `BLOOM_ADD`
+  * Description: Adds an element to the bloom filter passed in
+  * Input:
+    * bloom - The bloom filter
+    * value* - The values to add
+  * Returns: Bloom Filter
+* `BLOOM_EXISTS`
+  * Description: If the bloom filter contains the value
+  * Input:
+    * bloom - The bloom filter
+    * value - The value to check
+  * Returns: True if the filter might contain the value and false otherwise
+* `BLOOM_INIT`
+  * Description: Returns an empty bloom filter
+  * Input:
+    * expectedInsertions - The expected insertions
+    * falsePositiveRate - The false positive rate you are willing to tolerate
+  * Returns: Bloom Filter
+* `BLOOM_MERGE`
+  * Description: Returns a merged bloom filter
+  * Input:
+    * bloomfilters - A list of bloom filters to merge
+  * Returns: Bloom Filter or null if the list is empty
+* `DAY_OF_MONTH`
+  * Description: The numbered day within the month.  The first day within the month has a value of 1.
+  * Input:
+    * dateTime - The datetime as a long representing the milliseconds since unix epoch
+  * Returns: The numbered day within the month.
+* `DAY_OF_WEEK`
+  * Description: The numbered day within the week.  The first day of the week, Sunday, has a value of 1.
+  * Input:
+    * dateTime - The datetime as a long representing the milliseconds since unix epoch
+  * Returns: The numbered day within the week.
+* `DAY_OF_YEAR`
+  * Description: The day number within the year.  The first day of the year has value of 1.
+  * Input:
+    * dateTime - The datetime as a long representing the milliseconds since unix epoch
+  * Returns: The day number within the year.
+* `DOMAIN_REMOVE_SUBDOMAINS`
+  * Description: Remove subdomains from a domain.
+  * Input:
+    * domain - fully qualified domain name
+  * Returns: The domain without the subdomains.  e.g. DOMAIN_REMOVE_SUBDOMAINS('mail.yahoo.com') yields 'yahoo.com'
+* `DOMAIN_REMOVE_TLD`
+  * Description: Remove top level domain suffix from a domain.
+  * Input:
+    * domain - fully qualified domain name
+  * Returns: The domain without the TLD.  e.g. DOMAIN_REMOVE_TLD('mail.yahoo.co.uk') yields 'mail.yahoo'
+* `DOMAIN_TO_TLD`
+  * Description: Extract the top level domain from a domain
+  * Input:
+    * domain - fully qualified domain name
+  * Returns: The TLD of the domain.  e.g. DOMAIN_TO_TLD('mail.yahoo.co.uk') yields 'co.uk'
+* `ENDS_WITH`
+  * Description: Determines whether a string ends with a prefix
+  * Input:
+    * string - The string to test
+    * suffix - The proposed suffix
+  * Returns: True if the string ends with the specified suffix and false otherwise.
+* `ENRICHMENT_EXISTS`
+  * Description: Interrogates the HBase table holding the simple hbase enrichment data and returns whether the enrichment type and indicator are in the table.
+  * Input:
+    * enrichment_type - The enrichment type
+    * indicator - The string indicator to look up
+    * nosql_table - The NoSQL Table to use
+    * column_family - The Column Family to use
+  * Returns: True if the enrichment indicator exists and false otherwise
+* `ENRICHMENT_GET`
+  * Description: Interrogates the HBase table holding the simple hbase enrichment data and retrieves the tabular value associated with the enrichment type and indicator.
+  * Input:
+    * enrichment_type - The enrichment type
+    * indicator - The string indicator to look up
+    * nosql_table - The NoSQL Table to use
+    * column_family - The Column Family to use
+  * Returns: A Map associated with the indicator and enrichment type.  Empty otherwise.
+* `GET`
+  * Description: Returns the i'th element of the list 
+  * Input:
+    * input - List
+    * i - the index (0-based)
+  * Returns: First element of the list
+* `GET_FIRST`
+  * Description: Returns the first element of the list
+  * Input:
+    * input - List
+  * Returns: First element of the list
+* `GET_LAST`
+  * Description: Returns the last element of the list
+  * Input:
+    * input - List
+  * Returns: Last element of the list
+* `IN_SUBNET`
+  * Description: Returns if an IP is within a subnet range.
+  * Input:
+    * ip - the IP address in String form
+    * cidr+ - one or more IP ranges specified in CIDR notation (e.g. 192.168.0.0/24)
+  * Returns: True if the IP address is within at least one of the network ranges and false otherwise
+* `IS_DATE`
+  * Description: Determines if a string passed is a date of a given format.
+  * Input:
+    * date - The date in string form.
+    * format - The format of the date.
+  * Returns: True if the date is of the specified format and false otherwise.
+* `IS_DOMAIN`
+  * Description: Tests if a string is a valid domain.  Domain names are evaluated according to the standards RFC1034 section 3, and RFC1123 section 2.1.
+  * Input:
+    * address - The String to test
+  * Returns: True if the string is a valid domain and false otherwise.
+* `IS_EMAIL`
+  * Description: Tests if a string is a valid email address
+  * Input:
+    * address - The String to test
+  * Returns: True if the string is a valid email address and false otherwise.
+* `IS_EMPTY`
+  * Description: Returns true if string or collection is empty and false otherwise
+  * Input:
+    * input - Object of string or collection type (e.g. list)
+  * Returns: Boolean
+* `IS_INTEGER`
+  * Description: Determine if an object is an integer or not.
+  * Input:
+    * x - An object which we wish to test is an integer
+  * Returns: True if the object can be converted to an integer and false otherwise.
+* `IS_IP`
+  * Description: Determine if an string is an IP or not.
+  * Input:
+    * ip - An object which we wish to test is an ip
+    * type (optional) - one of IPV4 or IPV6.  The default is IPV4.
+  * Returns: True if the string is an IP and false otherwise.
+* `IS_URL`
+  * Description: Tests if a string is a valid URL
+  * Input:
+    * url - The String to test
+  * Returns: True if the string is a valid URL and false otherwise.
+* `JOIN`
+  * Description: Joins the components of the list with the specified delimiter.
+  * Input:
+    * list - List of Strings
+    * delim - String delimiter
+  * Returns: String
+* `MAAS_GET_ENDPOINT`
+  * Description: Inspects zookeeper and returns a map containing the name, version and url for the model referred to by the input params
+  * Input:
+    * model_name - the name of the model
+    * model_version - the optional version of the model.  If it is not specified, the most current version is used.
+  * Returns: A map containing the name, version, url for the REST endpoint (fields named name, version and url).  Note that the output of this function is suitable for input into the first argument of MAAS_MODEL_APPLY.
+* `MAAS_MODEL_APPLY`
+  * Description: Returns the output of a model deployed via model which is deployed at endpoint. NOTE: Results are cached at the client for 10 minutes.
+  * Input:
+    * endpoint - a map containing name, version, url for the REST endpoint
+    * function - the optional endpoint path, default is 'apply'
+    * model_args - dictionary of arguments for the model (these become request params).
+  * Returns: The output of the model deployed as a REST endpoint in Map form.  Assumes REST endpoint returns a JSON Map.
+* `MAP_EXISTS`
+  * Description: Checks for existence of a key in a map.
+  * Input:
+    * key - The key to check for existence
+    * map - The map to check for existence of the key
+  * Returns: True if the key is found in the map and false otherwise.
+* `MAP_GET`
+  * Description: Gets the value associated with a key from a map
+  * Input:
+    * key - The key
+    * map - The map
+    * default - Optionally the default value to return if the key is not in the map.
+  * Returns: The object associated with key in the map.  If there is no value associated, then default if specified and null if a default is not specified.
+* `MONTH`
+  * Description: The number representing the month.  The first month, January, has a value of 0.
+  * Input:
+    * dateTime - The datetime as a long representing the milliseconds since unix epoch
+  * Returns: The current month (0-based).
+* `PROTOCOL_TO_NAME`
+  * Description: Convert the IANA protocol number to the protocol name
+  * Input:
+    * IANA Number
+  * Returns: The protocol name associated with the IANA number.
+* `REGEXP_MATCH`
+  * Description: Determines whether a regex matches a string
+  * Input:
+    * string - The string to test
+    * pattern - The proposed regex pattern
+  * Returns: True if the regex pattern matches the string and false otherwise.
+* `SPLIT`
+  * Description: Splits the string by the delimiter.
+  * Input:
+    * input - String to split
+    * delim - String delimiter
+  * Returns: List of Strings
+* `STARTS_WITH`
+  * Description: Determines whether a string starts with a prefix
+  * Input:
+    * string - The string to test
+    * prefix - The proposed prefix
+  * Returns: True if the string starts with the specified prefix and false otherwise.
+* `STATS_ADD`
+  * Description: Add one or more input values to those that are used to calculate the summary statistics.
+  * Input:
+    * stats - The Stellar statistics object.  If null, then a new one is initialized.
+    * value+ - one or more numbers to add 
+  * Returns: A StatisticsProvider object
+* `STATS_COUNT`
+  * Description: Calculates the count of the values accumulated (or in the window if a window is used).
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The count of the values in the window or NaN if the statistics object is null.
+* `STATS_GEOMETRIC_MEAN`
+  * Description: Calculates the geometric mean of the values accumulated (or in the window if a window is used). See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics 
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The geometric mean of the values in the window or NaN if the statistics object is null.
+* `STATS_INIT`
+  * Description: Initialize a Statistics object
+  * Input:
+    * window_size - The number of input data values to maintain in a rolling window in memory.  If equal to 0, then no rolling window is maintained. Using no rolling window is less memory intensive, but cannot calculate certain statistics like percentiles and kurtosis.
+  * Returns: A StatisticsProvider object
+* `STATS_KURTOSIS`
+  * Description: Calculates the kurtosis of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics 
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The kurtosis of the values in the window or NaN if the statistics object is null.
+* `STATS_MAX`
+  * Description: Calculates the max of the values accumulated (or in the window if a window is used).
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The max of the values in the window or NaN if the statistics object is null.
+* `STATS_MEAN`
+  * Description: Calculates the mean of the values accumulated (or in the window if a window is used).
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The mean of the values in the window or NaN if the statistics object is null.
+* `STATS_MERGE`
+  * Description: Merge statistic providers
+  * Input:
+    * statisticsProviders - A list of statistics providers
+  * Returns: A StatisticsProvider object
+* `STATS_MIN`
+  * Description: Calculates the min of the values accumulated (or in the window if a window is used).
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The min of the values in the window or NaN if the statistics object is null.
+* `STATS_PERCENTILE`
+  * Description: Computes the p'th percentile of the values accumulated (or in the window if a window is used).
+  * Input:
+    * stats - The Stellar statistics object.
+    * p - a double where 0 <= p < 1 representing the percentile
+  * Returns: The p'th percentile of the data or NaN if the statistics object is null
+* `STATS_POPULATION_VARIANCE`
+  * Description: Calculates the population variance of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics 
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The population variance of the values in the window or NaN if the statistics object is null.
+* `STATS_QUADRATIC_MEAN`
+  * Description: Calculates the quadratic mean of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics 
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The quadratic mean of the values in the window or NaN if the statistics object is null.
+* `STATS_SD`
+  * Description: Calculates the standard deviation of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics 
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The standard deviation of the values in the window or NaN if the statistics object is null.
+* `STATS_SKEWNESS`
+  * Description: Calculates the skewness of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics 
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The skewness of the values in the window or NaN if the statistics object is null.
+* `STATS_SUM`
+  * Description: Calculates the sum of the values accumulated (or in the window if a window is used).
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The sum of the values in the window or NaN if the statistics object is null.
+* `STATS_SUM_LOGS`
+  * Description: Calculates the sum of the (natural) log of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics 
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The sum of the (natural) log of the values in the window or NaN if the statistics object is null.
+* `STATS_SUM_SQUARES`
+  * Description: Calculates the sum of the squares of the values accumulated (or in the window if a window is used).
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The sum of the squares of the values in the window or NaN if the statistics object is null.
+* `STATS_VARIANCE`
+  * Description: Calculates the variance of the values accumulated (or in the window if a window is used).  See http://commons.apache.org/proper/commons-math/userguide/stat.html#a1.2_Descriptive_statistics 
+  * Input:
+    * stats - The Stellar statistics object.
+  * Returns: The variance of the values in the window or NaN if the statistics object is null.
+* `TO_DOUBLE`
+  * Description: Transforms the first argument to a double precision number
+  * Input:
+    * input - Object of string or numeric type
+  * Returns: Double
+* `TO_EPOCH_TIMESTAMP`
+  * Description: Returns the epoch timestamp of the dateTime given the format. If the format does not have a timestamp and you wish to assume a given timestamp, you may specify the timezone optionally.
+  * Input:
+    * dateTime - DateTime in String format
+    * format - DateTime format as a String
+    * timezone - Optional timezone in String format
+  * Returns: Boolean
+* `TO_INTEGER`
+  * Description: Transforms the first argument to an integer
+  * Input:
+    * input - Object of string or numeric type
+  * Returns: Integer
+* `TO_LOWER`
+  * Description: Transforms the first argument to a lowercase string
+  * Input:
+    * input - String
+  * Returns: String
+* `TO_STRING`
+  * Description: Transforms the first argument to a string
+  * Input:
+    * input - Object
+  * Returns: String
+* `TO_UPPER`
+  * Description: Transforms the first argument to an uppercase string
+  * Input:
+    * input - String
+  * Returns: String
+* `TRIM`
+  * Description: Trims whitespace from both sides of a string.
+  * Input:
+    * input - String
+  * Returns: String
+* `URL_TO_HOST`
+  * Description: Extract the hostname from a URL.
+  * Input:
+    * url - URL in String form
+  * Returns: The hostname from the URL as a String.  e.g. URL_TO_HOST('http://www.yahoo.com/foo') would yield 'www.yahoo.com'
+* `URL_TO_PATH`
+  * Description: Extract the path from a URL.
+  * Input:
+    * url - URL in String form
+  * Returns: The path from the URL as a String.  e.g. URL_TO_PATH('http://www.yahoo.com/foo') would yield 'foo'
+* `URL_TO_PORT`
+  * Description: Extract the port from a URL.  If the port is not explicitly stated in the URL, then an implicit port is inferred based on the protocol.
+  * Input:
+    * url - URL in String form
+  * Returns: The port used in the URL as an Integer.  e.g. URL_TO_PORT('http://www.yahoo.com/foo') would yield 80
+* `URL_TO_PROTOCOL`
+  * Description: Extract the protocol from a URL.
+  * Input:
+    * url - URL in String form
+  * Returns: The protocol from the URL as a String. e.g. URL_TO_PROTOCOL('http://www.yahoo.com/foo') would yield 'http'
+* `WEEK_OF_MONTH`
+  * Description: The numbered week within the month.  The first week within the month has a value of 1.
+  * Input:
+    * dateTime - The datetime as a long representing the milliseconds since unix epoch
+  * Returns: The numbered week within the month.
+* `WEEK_OF_YEAR`
+  * Description: The numbered week within the year.  The first week in the year has a value of 1.
+  * Input:
+    * dateTime - The datetime as a long representing the milliseconds since unix epoch
+  * Returns: The numbered week within the year.
+* `YEAR`
+  * Description: The number representing the year. 
+  * Input:
+    * dateTime - The datetime as a long representing the milliseconds since unix epoch
+  * Returns: The current year
 
 The following is an example query (i.e. a function which returns a
 boolean) which would be seen possibly in threat triage:
@@ -75,15 +402,6 @@ This will convert the timestamp field to an epoch timestamp based on the
   `dc`, defaulting to `UTC`
 
 
-#Enrichment Configuration
-
-The configuration for the `enrichment` topology, the topology primarily
-responsible for enrichment and threat intelligence enrichment, is
-defined by JSON documents stored in zookeeper.
-
-There are two types of configurations at the moment, `global` and
-`sensor` specific.  
-
 ##Global Configuration
 The format of the global enrichment is a JSON String to Object map.  This is intended for
 configuration which is non sensor specific configuration.
@@ -131,167 +449,6 @@ structured like so:
    * `REGEX_MATCH` : Validates that the fields match a regex.  Expects `pattern` in the config.
    * `NOT_EMPTY` : Validates that the fields exist and are not empty (after trimming.)
 
-##Sensor Enrichment Configuration
-
-The sensor specific configuration is intended to configure the
-individual enrichments and threat intelligence enrichments for a given
-sensor type (e.g. `snort`).
-
-Just like the global config, the format is a JSON stored in zookeeper.
-The configuration is a complex JSON object with the following top level fields:
-
-* `index` : The name of the sensor
-* `batchSize` : The size of the batch that is written to the indices at once.
-* `enrichment` : A complex JSON object representing the configuration of the enrichments
-* `threatIntel` : A complex JSON object representing the configuration of the threat intelligence enrichments
-
-###The `enrichment` Configuration 
-
-
-| Field            | Description                                                                                                                                                                                                                   | Example                                                          |
-|------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------------------|
-| `fieldToTypeMap` | In the case of a simple HBase enrichment (i.e. a key/value lookup), the mapping between fields and the enrichment types associated with those fields must be known.  This enrichment type is used as part of the HBase key. | `"fieldToTypeMap" : { "ip_src_addr" : [ "asset_enrichment" ] }`  |
-| `fieldMap`       | The map of enrichment bolts names to configuration handlers which know how to split the message up.  The simplest of which is just a list of fields.  More complex examples would be the stellar enrichment which provides stellar statements.  Each field is sent to the enrichment referenced in the key.                                                                                                 | `"fieldMap": {"hbaseEnrichment": ["ip_src_addr","ip_dst_addr"]}` |
-| `config`         | The general configuration for the enrichment                                                                                                                                                                                  | `"config": {"typeToColumnFamily": { "asset_enrichment" : "cf" } }` |
-
-The `config` map is intended to house enrichment specific configuration.
-For instance, for the `hbaseEnrichment`, the mappings between the
-enrichment types to the column families is specified.
-
-The `fieldMap`contents are of interest because they contain the routing and configuration information for the enrichments.  When we say 'routing', we mean how the messages get split up and sent to the enrichment adapter bolts.  The simplest, by far, is just providing a simple list as in
-```
-    "fieldMap": {
-      "geo": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ],
-      "host": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ],
-      "hbaseEnrichment": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ]
-      }
-```
-For the `geo`, `host` and `hbaseEnrichment`, this is sufficient.  However, more complex enrichments may contain their own configuration.  Currently, the `stellar` enrichment requires a more complex configuration, such as:
-```
-    "fieldMap": {
-       ...
-      "stellar" : {
-        "config" : {
-          "numeric" : {
-                      "foo": "1 + 1"
-                      }
-          ,"ALL_CAPS" : "TO_UPPER(source.type)"
-        }
-      }
-    }
-```
-
-Whereas the simpler enrichments just need a set of fields explicitly stated so they can be separated from the message and sent to the enrichment adapter bolt for enrichment and ultimately joined back in the join bolt, the stellar enrichment has its set of required fields implicitly stated through usage.  For instance, if your stellar statement references a field, it should be included and if not, then it should not be included.  We did not want to require users to make explicit the implicit.
-
-The other way in which the stellar enrichment is somewhat more complex is in how the statements are executed.  In the general purpose case for a list of fields, those fields are used to create a message to send to the enrichment adapter bolt and that bolt's worker will handle the fields one by one in serial for a given message.  For stellar enrichment, we wanted to have a more complex design so that users could specify the groups of stellar statements sent to the same worker in the same message (and thus executed sequentially).  Consider the following configuration:
-```
-    "fieldMap": {
-      "stellar" : {
-        "config" : {
-          "numeric" : {
-                      "foo": "1 + 1"
-                      "bar" : TO_LOWER(source.type)"
-                      }
-         ,"text" : {
-                   "ALL_CAPS" : "TO_UPPER(source.type)"
-                   }
-        }
-      }
-    }
-```
-We have a group called `numeric` whose stellar statements will be executed sequentially.  In parallel to that, we have the group of stellar statements under the group `text` executing.  The intent here is to allow you to not force higher latency operations to be done sequentially.
-
-###The `threatIntel` Configuration 
-
-| Field            | Description                                                                                                                                                                                                                                   | Example                                                                  |
-|------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------|
-| `fieldToTypeMap` | In the case of a simple HBase threat intel enrichment (i.e. a key/value lookup), the mapping between fields and the enrichment types associated with those fields must be known.  This enrichment type is used as part of the HBase key. | `"fieldToTypeMap" : { "ip_src_addr" : [ "malicious_ips" ] }`             |
-| `fieldMap`       | The map of threat intel enrichment bolts names to fields in the JSON messages. Each field is sent to the threat intel enrichment bolt referenced in the key.                                                                              | `"fieldMap": {"hbaseThreatIntel": ["ip_src_addr","ip_dst_addr"]}`        |
-| `triageConfig`   | The configuration of the threat triage scorer.  In the situation where a threat is detected, a score is assigned to the message and embedded in the indexed message.                                                                    | `"riskLevelRules" : { "IN_SUBNET(ip_dst_addr, '192.168.0.0/24')" : 10 }` |
-| `config`         | The general configuration for the Threat Intel                                                                                                                                                                                                | `"config": {"typeToColumnFamily": { "malicious_ips","cf" } }`            |
-
-The `config` map is intended to house threat intel specific configuration.
-For instance, for the `hbaseThreatIntel` threat intel adapter, the mappings between the
-enrichment types to the column families is specified.
-
-The `triageConfig` field is also a complex field and it bears some description:
-
-| Field            | Description                                                                                                                                             | Example                                                                  |
-|------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------|
-| `riskLevelRules` | The mapping of Stellar (see above) queries to a score.                                                                                                  | `"riskLevelRules" : { "IN_SUBNET(ip_dst_addr, '192.168.0.0/24')" : 10 }` |
-| `aggregator`     | An aggregation function that takes all non-zero scores representing the matching queries from `riskLevelRules` and aggregates them into a single score. | `"MAX"`                                                                  |
-
-The supported aggregation functions are:
-* `MAX` : The max of all of the associated values for matching queries
-* `MIN` : The min of all of the associated values for matching queries
-* `MEAN` : The mean of all of the associated values for matching queries
-* `POSITIVE_MEAN` : The mean of the positive associated values for the matching queries.
-
-###Example
-
-An example configuration for the YAF sensor is as follows:
-```json
-{
-  "index": "yaf",
-  "batchSize": 5,
-  "enrichment": {
-    "fieldMap": {
-      "geo": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ],
-      "host": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ],
-      "hbaseEnrichment": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ]
-    }
-  ,"fieldToTypeMap": {
-      "ip_src_addr": [
-        "playful_classification"
-      ],
-      "ip_dst_addr": [
-        "playful_classification"
-      ]
-    }
-  },
-  "threatIntel": {
-    "fieldMap": {
-      "hbaseThreatIntel": [
-        "ip_src_addr",
-        "ip_dst_addr"
-      ]
-    },
-    "fieldToTypeMap": {
-      "ip_src_addr": [
-        "malicious_ip"
-      ],
-      "ip_dst_addr": [
-        "malicious_ip"
-      ]
-    },
-    "triageConfig" : {
-      "riskLevelRules" : {
-        "ip_src_addr == '10.0.2.3' or ip_dst_addr == '10.0.2.3'" : 10
-      },
-      "aggregator" : "MAX"
-    }
-  }
-}
-```
-
 
 ##Management Utility
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/319e61e7/metron-platform/metron-enrichment/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/README.md b/metron-platform/metron-enrichment/README.md
index 83d2926..9527c8e 100644
--- a/metron-platform/metron-enrichment/README.md
+++ b/metron-platform/metron-enrichment/README.md
@@ -1,125 +1,286 @@
-#metron-enrichment
+#Enrichment
 
-##Module Description
+## Introduction
 
-This module enables enrichment of message metafields with additional information from various enrichment sources.  Currently there is only a limited number of enrichments available, but this is an extensible framework that can be extended with additional enrichments.  Enrichments currently available are geo, whois, hosts, and CIF.
+The `enrichment` topology is a topology dedicated to taking the data
+from the parsing topologies that have been normalized into the Metron
+data format (e.g. a JSON Map structure with `original_message` and
+`timestamp`) and 
+* Enriching messages with external data from data stores (e.g. hbase) by
+  adding new fields based on existing fields in the messages.
+* Marking messages as threats based on data in external data stores
+* Marking threat alerts with a numeric triage level based on a set of
+  Stellar rules.
 
-##Message Format
+## Enrichment Architecture
 
-Enrichment bolts are designed to go after the parser bolts.  Parser bolts will parse the telemetry, taking it from its native format and producing a standard JSON that would look like so:
+![Architecture](enrichment_arch.png)
 
-```json
-{
-"message":
-{"ip_src_addr": xxxx,
-"ip_dst_addr": xxxx,
-"ip_src_port": xxxx,
-"ip_dst_port": xxxx,
-"protocol": xxxx,
-"additional-field 1": xxx,
-}
+## Enrichment Configuration
 
-}
-```
+The configuration for the `enrichment` topology, the topology primarily
+responsible for enrichment and threat intelligence enrichment, is
+defined by JSON documents stored in zookeeper.
 
-A single enrichment bolt would enrich the message and produce a JSON enrichment and attach it to the message.  Enrichments are stackable so multiple enrichments can be attached sequentially after a single parser bolt.  Stacked enrichments would produce messages under the "enrichment" tag and attach it to the message like so:
+There are two types of configurations at the moment, `global` and
+`sensor` specific.  
 
-```json
-{
-"message":
-{"ip_src_addr": xxxx,
-"ip_dst_addr": xxxx,
-"ip_src_port": xxxx,
-"ip_dst_port": xxxx,
-"protocol": xxxx,
-"additional-field 1": xxxx,
-},
-"enrichment" : {"geo": xxxx, "whois": xxxx, "hosts": xxxxx, "CIF": "xxxxx"}
+## Global Configuration 
 
-}
-```
+See the "[Global Configuration](../metron-common)" section.
 
-##Enrichment Sources
+##Sensor Enrichment Configuration
 
-Each enrichment has to have an anrichment source which can serve as a lookup table for enriching relevant message fields.  In order to minimize the use of additional platforms and tools we primarily try to rely on HBase as much as possible to store the enrichment information for lookup by key.  In order to use Hbase we have to pre-process the enrichment feeds for bulk-loading into HBase with specific key format optimized for retrieval as well as utilize caches within the enrichment bolts to be able to provide enrichments real-time.  Our wiki contains information on how to setup the environment, pre-process feeds, and plug in the enrichment sources.
+The sensor specific configuration is intended to configure the
+individual enrichments and threat intelligence enrichments for a given
+sensor type (e.g. `snort`).
 
-##Enrichment Bolt
+Just like the global config, the format is a JSON stored in zookeeper.
+The configuration is a complex JSON object with the following top level fields:
 
-The enrichment bolt is designed to be extensible to be re-used for all kinds of enrichment processes.  The bolt signature for declaration in a storm topology is as follows:
+* `index` : The name of the sensor
+* `batchSize` : The size of the batch that is written to the indices at once.
+* `enrichment` : A complex JSON object representing the configuration of the enrichments
+* `threatIntel` : A complex JSON object representing the configuration of the threat intelligence enrichments
 
+###The `enrichment` Configuration 
 
 
-```
-GenericEnrichmentBolt geo_enrichment = new GenericEnrichmentBolt()
-.withEnrichmentTag(
-config.getString("bolt.enrichment.geo.enrichment_tag"))
-.withAdapter(geo_adapter)
-.withMaxTimeRetain(
-config.getInt("bolt.enrichment.geo.MAX_TIME_RETAIN_MINUTES"))
-.withMaxCacheSize(
-config.getInt("bolt.enrichment.geo.MAX_CACHE_SIZE_OBJECTS_NUM"))
-.withKeys(geo_keys).withMetricConfiguration(config);
+| Field            | Description                                                                                                                                                                                                                   | Example                                                          |
+|------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------------------|
+| `fieldToTypeMap` | In the case of a simple HBase enrichment (i.e. a key/value lookup), the mapping between fields and the enrichment types associated with those fields must be known.  This enrichment type is used as part of the HBase key. | `"fieldToTypeMap" : { "ip_src_addr" : [ "asset_enrichment" ] }`  |
+| `fieldMap`       | The map of enrichment bolts names to configuration handlers which know how to split the message up.  The simplest of which is just a list of fields.  More complex examples would be the stellar enrichment which provides stellar statements.  Each field is sent to the enrichment referenced in the key.                                                                                                 | `"fieldMap": {"hbaseEnrichment": ["ip_src_addr","ip_dst_addr"]}` |
+| `config`         | The general configuration for the enrichment                                                                                                                                                                                  | `"config": {"typeToColumnFamily": { "asset_enrichment" : "cf" } }` |
+
+The `config` map is intended to house enrichment specific configuration.
+For instance, for the `hbaseEnrichment`, the mappings between the
+enrichment types to the column families is specified.
 
+The `fieldMap`contents are of interest because they contain the routing and configuration information for the enrichments.  When we say 'routing', we mean how the messages get split up and sent to the enrichment adapter bolts.  The simplest, by far, is just providing a simple list as in
+```
+    "fieldMap": {
+      "geo": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ],
+      "host": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ],
+      "hbaseEnrichment": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ]
+      }
+```
+For the `geo`, `host` and `hbaseEnrichment`, this is sufficient.  However, more complex enrichments may contain their own configuration.  Currently, the `stellar` enrichment requires a more complex configuration, such as:
+```
+    "fieldMap": {
+       ...
+      "stellar" : {
+        "config" : {
+          "numeric" : {
+                      "foo": "1 + 1"
+                      }
+          ,"ALL_CAPS" : "TO_UPPER(source.type)"
+        }
+      }
+    }
 ```
 
-EnrichmentTag - Name of the enrichment (geo, whois, hosts, etc)
-Keys - Keys which this enrichment is able to enrich (hosts field for hosts enrichment, source_ip, dest_ip, for geo enrichment, etc)
-MaxTimeToRetain & MaxCacheSize - define the caching policy of the enrichment bolt
-Adapter - which adapter to use with the enrichment bolt instance
+Whereas the simpler enrichments just need a set of fields explicitly stated so they can be separated from the message and sent to the enrichment adapter bolt for enrichment and ultimately joined back in the join bolt, the stellar enrichment has its set of required fields implicitly stated through usage.  For instance, if your stellar statement references a field, it should be included and if not, then it should not be included.  We did not want to require users to make explicit the implicit.
 
-###Geo Adapter
-Geo adapter is able to do geo enrichment on hosts and destination IPs.  The open source verison of the geo adapter uses the free Geo feeds from MaxMind.  The format of these feeds does not easily lend itself to a no-sql DB so this adapter is designed to work with mySql.  But it is extensible enough to be made work with a variety of other back ends.
+The other way in which the stellar enrichment is somewhat more complex is in how the statements are executed.  In the general purpose case for a list of fields, those fields are used to create a message to send to the enrichment adapter bolt and that bolt's worker will handle the fields one by one in serial for a given message.  For stellar enrichment, we wanted to have a more complex design so that users could specify the groups of stellar statements sent to the same worker in the same message (and thus executed sequentially).  Consider the following configuration:
+```
+    "fieldMap": {
+      "stellar" : {
+        "config" : {
+          "numeric" : {
+                      "foo": "1 + 1"
+                      "bar" : TO_LOWER(source.type)"
+                      }
+         ,"text" : {
+                   "ALL_CAPS" : "TO_UPPER(source.type)"
+                   }
+        }
+      }
+    }
+```
+We have a group called `numeric` whose stellar statements will be executed sequentially.  In parallel to that, we have the group of stellar statements under the group `text` executing.  The intent here is to allow you to not force higher latency operations to be done sequentially.
 
-The signature of a geo adapter is as follows;
+###The `threatIntel` Configuration 
 
-```
-GeoMysqlAdapter geo_adapter = new GeoMysqlAdapter(
-config.getString("mysql.ip"), config.getInt("mysql.port"),
-config.getString("mysql.username"),
-config.getString("mysql.password"),
-config.getString("bolt.enrichment.geo.adapter.table"));
+| Field            | Description                                                                                                                                                                                                                                   | Example                                                                  |
+|------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------|
+| `fieldToTypeMap` | In the case of a simple HBase threat intel enrichment (i.e. a key/value lookup), the mapping between fields and the enrichment types associated with those fields must be known.  This enrichment type is used as part of the HBase key. | `"fieldToTypeMap" : { "ip_src_addr" : [ "malicious_ips" ] }`             |
+| `fieldMap`       | The map of threat intel enrichment bolts names to fields in the JSON messages. Each field is sent to the threat intel enrichment bolt referenced in the key.                                                                              | `"fieldMap": {"hbaseThreatIntel": ["ip_src_addr","ip_dst_addr"]}`        |
+| `triageConfig`   | The configuration of the threat triage scorer.  In the situation where a threat is detected, a score is assigned to the message and embedded in the indexed message.                                                                    | `"riskLevelRules" : { "IN_SUBNET(ip_dst_addr, '192.168.0.0/24')" : 10 }` |
+| `config`         | The general configuration for the Threat Intel                                                                                                                                                                                                | `"config": {"typeToColumnFamily": { "malicious_ips","cf" } }`            |
 
-```
+The `config` map is intended to house threat intel specific configuration.
+For instance, for the `hbaseThreatIntel` threat intel adapter, the mappings between the
+enrichment types to the column families is specified.
 
-###Hosts Adapter
-The hosts adapter is designed to enrich message format with the static host information that can be read from a standard text file.  This adapter is intended for use with a network crawling script that can identify all customer assets and place them in a text file.  For example, this script would identify all workstations, printers, appliantces, etc.  Then if any of these assets are seen in the telemetry messages flowing through the adapter this enrichment would fire and the relevant known information about a host would be attached.  We are currently working on porting this adapter to work with HBase, but this work is not ready yet.  The known hosts file is located under the /etc/whitelists config directory of Metron.
+The `triageConfig` field is also a complex field and it bears some description:
 
-The signature of the hosts adapter is as follows:
+| Field            | Description                                                                                                                                             | Example                                                                  |
+|------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------|
+| `riskLevelRules` | The mapping of Stellar (see above) queries to a score.                                                                                                  | `"riskLevelRules" : { "IN_SUBNET(ip_dst_addr, '192.168.0.0/24')" : 10 }` |
+| `aggregator`     | An aggregation function that takes all non-zero scores representing the matching queries from `riskLevelRules` and aggregates them into a single score. | `"MAX"`                                                                  |
 
-```
-Map<String, JSONObject> known_hosts = SettingsLoader
-.loadKnownHosts(hosts_path);
+The supported aggregation functions are:
+* `MAX` : The max of all of the associated values for matching queries
+* `MIN` : The min of all of the associated values for matching queries
+* `MEAN` : The mean of all of the associated values for matching queries
+* `POSITIVE_MEAN` : The mean of the positive associated values for the matching queries.
 
-HostFromPropertiesFileAdapter host_adapter = new HostFromPropertiesFileAdapter(
-known_hosts);
+###Example Configuration
 
+An example configuration for the YAF sensor is as follows:
+```json
+{
+  "index": "yaf",
+  "batchSize": 5,
+  "enrichment": {
+    "fieldMap": {
+      "geo": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ],
+      "host": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ],
+      "hbaseEnrichment": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ]
+    }
+  ,"fieldToTypeMap": {
+      "ip_src_addr": [
+        "playful_classification"
+      ],
+      "ip_dst_addr": [
+        "playful_classification"
+      ]
+    }
+  },
+  "threatIntel": {
+    "fieldMap": {
+      "hbaseThreatIntel": [
+        "ip_src_addr",
+        "ip_dst_addr"
+      ]
+    },
+    "fieldToTypeMap": {
+      "ip_src_addr": [
+        "malicious_ip"
+      ],
+      "ip_dst_addr": [
+        "malicious_ip"
+      ]
+    },
+    "triageConfig" : {
+      "riskLevelRules" : {
+        "ip_src_addr == '10.0.2.3' or ip_dst_addr == '10.0.2.3'" : 10
+      },
+      "aggregator" : "MAX"
+    }
+  }
+}
 ```
-* The source and dest ips refer to the name of the message JSON key where the host information is located
 
-###Whois Adapter
-Whois adapter enriches the host name with additional whois information obtained from our proprietary Cisco feed.  The enricher itself is provided in this open source distribution, but the feed is not.  You have to have your own feed in order to use it.  Alternatively, you can contact us for providing you with this feed, but we would have to charge you a fee (we can't distribute it for free). The implemetation of the whois enrichment we provide works with HBase
+# Example Enrichment via Stellar
 
-The signature of the whois adapter is as follows:
+Let's walk through doing a simple enrichment using Stellar on your cluster using the Squid topology.
 
-```
+## Install Prerequisites
+Now let's install some prerequisites:
+* Squid client via `yum install squid`
+* ES Head plugin via `/usr/share/elasticsearch/bin/plugin install mobz/elasticsearch-head`
 
-EnrichmentAdapter whois_adapter = new WhoisHBaseAdapter(
-config.getString("bolt.enrichment.whois.hbase.table.name"),
-config.getString("kafka.zk.list"),
-config.getString("kafka.zk.port"));
-```
+Start Squid via `service squid start`
 
-###CIF Adapter
-CIF adapter is designed to take in CIF feeds and cross-reference them against every message processed by Storm.  If there is a hit then the relevant information is attached to the message.  
+## Adjust Enrichment Configurations for Squid to Call Stellar
+Let's adjust the configurations for the Squid topology to annotate the messages using some Stellar functions.
 
-The signature of the CIF adapter is as follows:
+* Edit the squid enrichment configuration at `$METRON_HOME/config/zookeeper/enrichments/squid.json` (this file will not exist, so create a new one) to add some new fields based on stellar queries: 
 
+ ```
+{
+  "index": "squid",
+  "batchSize": 1,
+  "enrichment" : {
+    "fieldMap": {
+      "stellar" : {
+        "config" : {
+          "numeric" : {
+                      "foo": "1 + 1"
+                      }
+          ,"ALL_CAPS" : "TO_UPPER(source.type)"
+        }
+      }
+     }
+  },
+  "threatIntel" : {
+    "fieldMap":{
+     "stellar" : {
+        "config" : {
+          "bar" : "TO_UPPER(source.type)"
+        }
+      } 
+    },
+    "triageConfig" : {
+    }
+  }
+}
 ```
-CIFHbaseAdapter = new CIFHbaseAdapter(config
-.getString("kafka.zk.list"), config
-.getString("kafka.zk.port"), config
-.getString("bolt.enrichment.cif.tablename")))
-```
-
-##Stacking Enrichments
-Enrichments can be stacked.  By default each enrichment bolt listens on the "message" stream.  In order to create and stack enrichment bolts create a new bolt and instantiate the appropariate adapter.  You can look at our sample topologies to see how enrichments can be stacked
+We have added the following fields as part of the enrichment phase of the enrichment topology:
+* `foo` ==  2
+* `ALL_CAPS` == SQUID 
+
+We have added the following as part of the threat intel:
+* ` bar` == SQUID
+
+Please note that foo and ALL_CAPS will be applied in separate workers due to them being in separate groups.
+
+* Upload new configs via `$METRON_HOME/bin/zk_load_configs.sh --mode PUSH -i $METRON_HOME/config/zookeeper -z node1:2181`
+* Make the Squid topic in kafka via `/usr/hdp/current/kafka-broker/bin/kafka-topics.sh --zookeeper node1:2181 --create --topic squid --partitions 1 --replication-factor 1`
+
+## Start Topologies and Send Data
+Now we need to start the topologies and send some data:
+* Start the squid topology via `$METRON_HOME/bin/start_parser_topology.sh -k node1:6667 -z node1:2181 -s squid`
+* Generate some data via the squid client:
+  * `squidclient http://yahoo.com`
+  * `squidclient http://cnn.com`
+* Send the data to kafka via `cat /var/log/squid/access.log | /usr/hdp/current/kafka-broker/bin/kafka-console-producer.sh --broker-list node1:6667 --topic squid`
+* Browse the data in elasticsearch via the ES Head plugin @ [http://node1:9200/_plugin/head/](http://node1:9200/_plugin/head/) and verify that in the squid index you have two documents
+* Ensure that the documents have new fields `foo`, `bar` and `ALL_CAPS` with values as described above.
+
+Note that we could have used any Stellar statements here, including calling out to HBase via `ENRICHMENT_GET` and `ENRICHMENT_EXISTS` or even calling a machine learning model via [Model as a Service](../../metron-analytics/metron-maas-service).
+
+# Notes on Performance Tuning
+
+Default installed Metron is untuned for production deployment.  There
+are a few knobs to tune to get the most out of your system.
+
+## Kafka Queue
+The `enrichments` kafka queue is a collection point from all of the
+parser topologies.  As such, make sure that the number of partitions in
+the kafka topic is sufficient to handle the throughput that you expect
+from your parser topologies.
+
+## Enrichment Topology
+The enrichment topology as started by the `$METRON_HOME/bin/start_enrichment_topology.sh` 
+script uses a default of one executor per bolt.  In a real production system, this should 
+be customized by modifying the flux file in
+`$METRON_HOME/flux/enrichment/remote.yaml`. 
+* Add a `parallelism` field to the bolts to give Storm a parallelism hint for the various components.  Give bolts which appear to be bottlenecks (e.g. stellar enrichment bolt, hbase enrichment and threat intel bolts) a larger hint.
+* Add a `parallelism` field to the kafka spout which matches the number of partitions for the enrichment kafka queue.
+* Adjust the number of workers for the topology by adjusting the 
+  `topology.workers` field for the topology. 
+
+Finally, if workers and executors are new to you or you don't know where
+to modify the flux file, the following might be of use to you:
+* [Understanding the Parallelism of a Storm Topology](http://www.michael-noll.com/blog/2012/10/16/understanding-the-parallelism-of-a-storm-topology/)
+* [Flux Docs](http://storm.apache.org/releases/current/flux.html)

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/319e61e7/metron-platform/metron-enrichment/enrichment_arch.png
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/enrichment_arch.png b/metron-platform/metron-enrichment/enrichment_arch.png
new file mode 100644
index 0000000..32d7236
Binary files /dev/null and b/metron-platform/metron-enrichment/enrichment_arch.png differ

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/319e61e7/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/stellar/DocumentationGenerator.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/stellar/DocumentationGenerator.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/stellar/DocumentationGenerator.java
new file mode 100644
index 0000000..a2d6f52
--- /dev/null
+++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/stellar/DocumentationGenerator.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.enrichment.stellar;
+
+import com.google.common.collect.Lists;
+import org.apache.metron.common.dsl.FunctionResolverSingleton;
+import org.apache.metron.common.dsl.Stellar;
+import org.apache.metron.common.dsl.StellarFunction;
+import org.apache.metron.common.dsl.StellarFunctionInfo;
+import org.reflections.Reflections;
+import org.reflections.util.ConfigurationBuilder;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import static org.apache.metron.common.dsl.FunctionResolverSingleton.effectiveClassPathUrls;
+
+public class DocumentationGenerator {
+
+  public static void main(String... argv) {
+    List<StellarFunctionInfo> functions = Lists.newArrayList(FunctionResolverSingleton.getInstance().getFunctionInfo());
+    Collections.sort(functions, (o1, o2) -> o1.getName().compareTo(o2.getName()));
+    for(StellarFunctionInfo info: functions) {
+      System.out.println( "* `" + info.getName() + "`");
+      System.out.println( "  * Description: " + info.getDescription() );
+      System.out.println( "  * Input:");
+      for(String param :info.getParams()) {
+        System.out.println( "    * " + param );
+      }
+      System.out.println( "  * Returns: " + info.getReturns() );
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/319e61e7/metron-platform/metron-indexing/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/README.md b/metron-platform/metron-indexing/README.md
new file mode 100644
index 0000000..9f58ff8
--- /dev/null
+++ b/metron-platform/metron-indexing/README.md
@@ -0,0 +1,55 @@
+#Indexing
+
+## Introduction
+
+The `indexing` topology is a topology dedicated to taking the data
+from the enrichment topology that have been enriched and storing the data in one or more supported indices
+* HDFS as rolled text files, one JSON blob per line
+* Elasticsearch
+* Solr
+
+By default, this topology writes out to both HDFS and one of
+Elasticsearch and Solr.
+
+Indices are written in batch and the batch size is specified in the
+[Enrichment Config](../metron-enrichment) via the `batchSize` parameter.
+This config is variable by sensor type.
+
+## Indexing Architecture
+
+The indexing topology is extremely simple.  Data is ingested into kafka
+and sent to 
+* An indexing bolt configured to write to either elasticsearch or Solr
+* An indexing bolt configured to write to HDFS under `/apps/metron/enrichment/indexed`
+
+Errors during indexing are sent to a kafka queue called `index_errors`
+
+# Notes on Performance Tuning
+
+Default installed Metron is untuned for production deployment.  By far
+and wide, the most likely piece to require TLC from a performance
+perspective is the indexing layer.  An index that does not keep up will
+back up and you will see errors in the kafka bolt.  There
+are a few knobs to tune to get the most out of your system.
+
+## Kafka Queue
+The `indexing` kafka queue is a collection point from the enrichment
+topology.  As such, make sure that the number of partitions in
+the kafka topic is sufficient to handle the throughput that you expect.
+
+## Indexing Topology
+The enrichment topology as started by the `$METRON_HOME/bin/start_elasticsearch_topology.sh` 
+or `$METRON_HOME/bin/start_solr_topology.sh`
+script uses a default of one executor per bolt.  In a real production system, this should 
+be customized by modifying the flux file in
+`$METRON_HOME/flux/indexing/remote.yaml`. 
+* Add a `parallelism` field to the bolts to give Storm a parallelism
+  hint for the various components.  Give bolts which appear to be bottlenecks (e.g. the indexing bolt) a larger hint.
+* Add a `parallelism` field to the kafka spout which matches the number of partitions for the enrichment kafka queue.
+* Adjust the number of workers for the topology by adjusting the 
+  `topology.workers` field for the topology. 
+
+Finally, if workers and executors are new to you or you don't know where
+to modify the flux file, the following might be of use to you:
+* [Understanding the Parallelism of a Storm Topology](http://www.michael-noll.com/blog/2012/10/16/understanding-the-parallelism-of-a-storm-topology/)
+* [Flux Docs](http://storm.apache.org/releases/current/flux.html)

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/319e61e7/metron-platform/metron-parsers/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/README.md b/metron-platform/metron-parsers/README.md
index f94651c..0be3da4 100644
--- a/metron-platform/metron-parsers/README.md
+++ b/metron-platform/metron-parsers/README.md
@@ -1,14 +1,37 @@
 #Parsers
 
+## Introduction
+
 Parsers are pluggable components which are used to transform raw data
 (textual or raw bytes) into JSON messages suitable for downstream
 enrichment and indexing.  
 
-There are two types of parsers:
-*  A parser written in Java which conforms to the `MessageParser` interface.  This kind of parser is optimized for speed and performance and
-is built for use with higher velocity topologies.  These parsers are not easily modifiable and in order to make changes to them the entire topology need to be recompiled.  
-* A Grok parser.  This type of parser is primarily designed for lower-velocity topologies or for quickly standing up a parser for a new telemetry before a permanent Java parser can be written for it.
-
+There are two general types types of parsers:
+*  A parser written in Java which conforms to the `MessageParser` interface.  This kind of parser is optimized for speed and performance and is built for use with higher velocity topologies.  These parsers are not easily modifiable and in order to make changes to them the entire topology need to be recompiled.  
+* A general purpose parser.  This type of parser is primarily designed for lower-velocity topologies or for quickly standing up a parser for a new telemetry before a permanent Java parser can be written for it.  As of the time of this writing, we have:
+  * Grok parser: `org.apache.metron.parsers.GrokParser` with possible `parserConfig` entries of 
+    * `grokPath` : The path in HDFS (or in the Jar) to the grok statement
+    * `patternLabel` : The pattern label to use from the grok statement
+    * `timestampField` : The field to use for timestamp
+    * `timeFields` : A list of fields to be treated as time
+    * `dateFormat` : The date format to use to parse the time fields
+    * `timezone` : The timezone to use. `UTC` is default.
+  * CSV Parser: `org.apache.metron.parsers.csv.CSVParser` with possible `parserConfig` entries of
+    * `timestampFormat` : The date format of the timestamp to use.  If unspecified, the parser assumes the timestamp is ms since unix epoch.
+    * `columns` : A map of column names you wish to extract from the CSV to their offsets (e.g. `{ 'name' : 1, 'profession' : 3}`  would be a column map for extracting the 2nd and 4th columns from a CSV)
+    * `separator` : The column separator, `,` by default.
+just
+
+## Parser Architecture
+
+![Architecture](parser_arch.png)
+
+Data flows through the parser bolt via kafka and into the `enrichments`
+topology in kafka.  Errors are collected with the context of the error
+(e.g. stacktrace) and original message causing the error and sent to an
+`error` queue.  Invalid messages as determined by global validation
+functions are sent to an `invalid` queue. 
+ 
 ##Message Format
 
 All Metron messages follow a specific format in order to ingest a message.  If a message does not conform to this format it will be dropped and put onto an error queue for further examination.  The message must be of a JSON format and must have a JSON tag message like so:
@@ -47,6 +70,10 @@ So putting it all together a typical Metron message with all 5-tuple fields pres
 }
 ```
 
+##Global Configuration 
+
+See the "[Global Configuration](../metron-common)" section.
+
 ##Parser Configuration
 
 The configuration for the various parser topologies is defined by JSON
@@ -126,7 +153,8 @@ to a textual representation of the protocol:
 This transformation would transform `{ "protocol" : 6, "source.type" : "bro", ... }` 
 into `{ "protocol" : "TCP", "source.type" : "bro", ...}`
 
-* `MTL` : This transformation executes a set of transformations expressed as [Metron Transformation Language](../metron-common) statements.
+* `STELLAR` : This transformation executes a set of transformations
+  expressed as [Stellar Language](../metron-common) statements.
 
 Consider the following sensor parser config to add three new fields to a
 message:
@@ -139,7 +167,7 @@ message:
 ...
     "fieldTransformations" : [
           {
-           "transformation" : "MTL"
+           "transformation" : "STELLAR"
           ,"output" : [ "utc_timestamp", "url_host", "url_protocol" ]
           ,"config" : {
             "utc_timestamp" : "TO_EPOCH_TIMESTAMP(timestamp, 'yyyy-MM-dd
@@ -313,3 +341,39 @@ you could create a file called `custom_config.json` containing
 }
 ```
 and pass `--extra_topology_options custom_config.json` to `start_parser_topology.sh`.
+
+# Notes on Performance Tuning
+
+Default installed Metron is untuned for production deployment.  There
+are a few knobs to tune to get the most out of your system.
+
+## Kafka Queue
+The kafka queue associated with your parser is a collection point for
+all of the data sent to your parser.  As such, make sure that the number of partitions in
+the kafka topic is sufficient to handle the throughput that you expect
+from your parser topology.
+
+## Parser Topology
+The enrichment topology as started by the `$METRON_HOME/bin/start_parser_topology.sh` 
+script uses a default of one executor per bolt.  In a real production system, this should 
+be customized by modifying the arguments sent to this utility.
+* Topology Wide
+  * `--num_workers` : The number of workers for the topology
+  * `--num_ackers` : The number of ackers for the topology
+* The Kafka Spout
+  * `--spout_num_tasks` : The number of tasks for the spout
+  * `--spout_p` : The parallelism hint for the spout
+  * Ensure that the spout has enough parallelism so that it can dedicate a worker per partition in your kafka topic.
+* The Parser Bolt
+  * `--parser_num_tasks` : The number of tasks for the parser bolt
+  * `--parser_p` : The parallelism hint for the spout
+  * This is bolt that gets the most processing, so ensure that it is configured with sufficient parallelism to match your throughput expectations.
+* The Error Message Writer Bolt
+  * `--error_writer_num_tasks` : The number of tasks for the error writer bolt
+  * `--error_writer_p` : The parallelism hint for the error writer bolt
+* The Invalid Message Writer Bolt
+  * `--invalid_writer_num_tasks` : The number of tasks for the error writer bolt
+  * `--invalid_writer_p` : The parallelism hint for the error writer bolt
+ 
+Finally, if workers and executors are new to you, the following might be of use to you:
+* [Understanding the Parallelism of a Storm Topology](http://www.michael-noll.com/blog/2012/10/16/understanding-the-parallelism-of-a-storm-topology/)

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/319e61e7/metron-platform/metron-parsers/parser_arch.png
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/parser_arch.png b/metron-platform/metron-parsers/parser_arch.png
new file mode 100644
index 0000000..d96be37
Binary files /dev/null and b/metron-platform/metron-parsers/parser_arch.png differ