[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-07 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16389610#comment-16389610
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user asfgit closed the pull request at:

https://github.com/apache/metron/pull/940


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-06 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16388800#comment-16388800
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172711543
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/parallel/ParallelEnricher.java
 ---
@@ -0,0 +1,281 @@
+/**
+ * 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.parallel;
+
+import com.github.benmanes.caffeine.cache.stats.CacheStats;
+import org.apache.metron.common.Constants;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import 
org.apache.metron.common.configuration.enrichment.handler.ConfigHandler;
+import org.apache.metron.common.performance.PerformanceLogger;
+import org.apache.metron.common.utils.MessageUtils;
+import org.apache.metron.enrichment.bolt.CacheKey;
+import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
+import org.apache.metron.enrichment.utils.EnrichmentUtils;
+import org.json.simple.JSONObject;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BinaryOperator;
+import java.util.function.Supplier;
+
+/**
+ * This is an independent component which will accept a message and a set 
of enrichment adapters as well as a config which defines
+ * how those enrichments should be performed and fully enrich the message. 
 The result will be the enriched message
+ * unified together and a list of errors which happened.
+ */
+public class ParallelEnricher {
+
+  private Map> enrichmentsByType = new 
HashMap<>();
+  private EnumMap cacheStats = new 
EnumMap<>(EnrichmentStrategies.class);
+
+  /**
+   * The result of an enrichment.
+   */
+  public static class EnrichmentResult {
+private JSONObject result;
+private List> enrichmentErrors;
+
+public EnrichmentResult(JSONObject result, List> enrichmentErrors) {
+  this.result = result;
+  this.enrichmentErrors = enrichmentErrors;
+}
+
+/**
+ * The unified fully enriched result.
+ * @return
+ */
+public JSONObject getResult() {
+  return result;
+}
+
+/**
+ * The errors that happened in the course of enriching.
+ * @return
+ */
+public List> getEnrichmentErrors() {
+  return enrichmentErrors;
+}
+  }
+
+  private ConcurrencyContext concurrencyContext;
+
+  /**
+   * Construct a parallel enricher with a set of enrichment adapters 
associated with their enrichment types.
+   * @param enrichmentsByType
+   */
+  public ParallelEnricher( Map> 
enrichmentsByType
+ , ConcurrencyContext concurrencyContext
+ , boolean logStats
+ )
+  {
+this.enrichmentsByType = enrichmentsByType;
+this.concurrencyContext = concurrencyContext;
+if(logStats) {
+  for(EnrichmentStrategies s : EnrichmentStrategies.values()) {
+cacheStats.put(s, null);
+  }
+}
+  }
+
+  /**
+   * Fully enriches a message.  Each enrichment is done in parallel via a 
threadpool.
+   * Each enrichment is fronted with a LRU cache.
+   *
+   * @param message the message to enrich
+   * @param strategy The enrichment strategy to use (e.g. enrichment or 
threat intel)
+   * @param config The sensor enrichment confi

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-06 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16388692#comment-16388692
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user nickwallen commented on the issue:

https://github.com/apache/metron/pull/940
  
+1 The unified topology works great.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-06 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16388690#comment-16388690
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user nickwallen commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172694248
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/parallel/ParallelEnricher.java
 ---
@@ -0,0 +1,281 @@
+/**
+ * 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.parallel;
+
+import com.github.benmanes.caffeine.cache.stats.CacheStats;
+import org.apache.metron.common.Constants;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import 
org.apache.metron.common.configuration.enrichment.handler.ConfigHandler;
+import org.apache.metron.common.performance.PerformanceLogger;
+import org.apache.metron.common.utils.MessageUtils;
+import org.apache.metron.enrichment.bolt.CacheKey;
+import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
+import org.apache.metron.enrichment.utils.EnrichmentUtils;
+import org.json.simple.JSONObject;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BinaryOperator;
+import java.util.function.Supplier;
+
+/**
+ * This is an independent component which will accept a message and a set 
of enrichment adapters as well as a config which defines
+ * how those enrichments should be performed and fully enrich the message. 
 The result will be the enriched message
+ * unified together and a list of errors which happened.
+ */
+public class ParallelEnricher {
+
+  private Map> enrichmentsByType = new 
HashMap<>();
+  private EnumMap cacheStats = new 
EnumMap<>(EnrichmentStrategies.class);
+
+  /**
+   * The result of an enrichment.
+   */
+  public static class EnrichmentResult {
+private JSONObject result;
+private List> enrichmentErrors;
+
+public EnrichmentResult(JSONObject result, List> enrichmentErrors) {
+  this.result = result;
+  this.enrichmentErrors = enrichmentErrors;
+}
+
+/**
+ * The unified fully enriched result.
+ * @return
+ */
+public JSONObject getResult() {
+  return result;
+}
+
+/**
+ * The errors that happened in the course of enriching.
+ * @return
+ */
+public List> getEnrichmentErrors() {
+  return enrichmentErrors;
+}
+  }
+
+  private ConcurrencyContext concurrencyContext;
+
+  /**
+   * Construct a parallel enricher with a set of enrichment adapters 
associated with their enrichment types.
+   * @param enrichmentsByType
+   */
+  public ParallelEnricher( Map> 
enrichmentsByType
+ , ConcurrencyContext concurrencyContext
+ , boolean logStats
+ )
+  {
+this.enrichmentsByType = enrichmentsByType;
+this.concurrencyContext = concurrencyContext;
+if(logStats) {
+  for(EnrichmentStrategies s : EnrichmentStrategies.values()) {
+cacheStats.put(s, null);
+  }
+}
+  }
+
+  /**
+   * Fully enriches a message.  Each enrichment is done in parallel via a 
threadpool.
+   * Each enrichment is fronted with a LRU cache.
+   *
+   * @param message the message to enrich
+   * @param strategy The enrichment strategy to use (e.g. enrichment or 
threat intel)
+   * @param config The sensor enrichment con

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-06 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16388524#comment-16388524
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172656809
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/parallel/ParallelEnricher.java
 ---
@@ -0,0 +1,281 @@
+/**
+ * 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.parallel;
+
+import com.github.benmanes.caffeine.cache.stats.CacheStats;
+import org.apache.metron.common.Constants;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import 
org.apache.metron.common.configuration.enrichment.handler.ConfigHandler;
+import org.apache.metron.common.performance.PerformanceLogger;
+import org.apache.metron.common.utils.MessageUtils;
+import org.apache.metron.enrichment.bolt.CacheKey;
+import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
+import org.apache.metron.enrichment.utils.EnrichmentUtils;
+import org.json.simple.JSONObject;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BinaryOperator;
+import java.util.function.Supplier;
+
+/**
+ * This is an independent component which will accept a message and a set 
of enrichment adapters as well as a config which defines
+ * how those enrichments should be performed and fully enrich the message. 
 The result will be the enriched message
+ * unified together and a list of errors which happened.
+ */
+public class ParallelEnricher {
+
+  private Map> enrichmentsByType = new 
HashMap<>();
+  private EnumMap cacheStats = new 
EnumMap<>(EnrichmentStrategies.class);
+
+  /**
+   * The result of an enrichment.
+   */
+  public static class EnrichmentResult {
+private JSONObject result;
+private List> enrichmentErrors;
+
+public EnrichmentResult(JSONObject result, List> enrichmentErrors) {
+  this.result = result;
+  this.enrichmentErrors = enrichmentErrors;
+}
+
+/**
+ * The unified fully enriched result.
+ * @return
+ */
+public JSONObject getResult() {
+  return result;
+}
+
+/**
+ * The errors that happened in the course of enriching.
+ * @return
+ */
+public List> getEnrichmentErrors() {
+  return enrichmentErrors;
+}
+  }
+
+  private ConcurrencyContext concurrencyContext;
+
+  /**
+   * Construct a parallel enricher with a set of enrichment adapters 
associated with their enrichment types.
+   * @param enrichmentsByType
+   */
+  public ParallelEnricher( Map> 
enrichmentsByType
+ , ConcurrencyContext concurrencyContext
+ , boolean logStats
+ )
+  {
+this.enrichmentsByType = enrichmentsByType;
+this.concurrencyContext = concurrencyContext;
+if(logStats) {
+  for(EnrichmentStrategies s : EnrichmentStrategies.values()) {
+cacheStats.put(s, null);
+  }
+}
+  }
+
+  /**
+   * Fully enriches a message.  Each enrichment is done in parallel via a 
threadpool.
+   * Each enrichment is fronted with a LRU cache.
+   *
+   * @param message the message to enrich
+   * @param strategy The enrichment strategy to use (e.g. enrichment or 
threat intel)
+   * @param config The sensor enrichment confi

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-06 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16388459#comment-16388459
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user nickwallen commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172595029
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/parallel/ParallelEnricher.java
 ---
@@ -0,0 +1,281 @@
+/**
+ * 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.parallel;
+
+import com.github.benmanes.caffeine.cache.stats.CacheStats;
+import org.apache.metron.common.Constants;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import 
org.apache.metron.common.configuration.enrichment.handler.ConfigHandler;
+import org.apache.metron.common.performance.PerformanceLogger;
+import org.apache.metron.common.utils.MessageUtils;
+import org.apache.metron.enrichment.bolt.CacheKey;
+import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
+import org.apache.metron.enrichment.utils.EnrichmentUtils;
+import org.json.simple.JSONObject;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.BinaryOperator;
+import java.util.function.Supplier;
+
+/**
+ * This is an independent component which will accept a message and a set 
of enrichment adapters as well as a config which defines
+ * how those enrichments should be performed and fully enrich the message. 
 The result will be the enriched message
+ * unified together and a list of errors which happened.
+ */
+public class ParallelEnricher {
+
+  private Map> enrichmentsByType = new 
HashMap<>();
+  private EnumMap cacheStats = new 
EnumMap<>(EnrichmentStrategies.class);
+
+  /**
+   * The result of an enrichment.
+   */
+  public static class EnrichmentResult {
+private JSONObject result;
+private List> enrichmentErrors;
+
+public EnrichmentResult(JSONObject result, List> enrichmentErrors) {
+  this.result = result;
+  this.enrichmentErrors = enrichmentErrors;
+}
+
+/**
+ * The unified fully enriched result.
+ * @return
+ */
+public JSONObject getResult() {
+  return result;
+}
+
+/**
+ * The errors that happened in the course of enriching.
+ * @return
+ */
+public List> getEnrichmentErrors() {
+  return enrichmentErrors;
+}
+  }
+
+  private ConcurrencyContext concurrencyContext;
+
+  /**
+   * Construct a parallel enricher with a set of enrichment adapters 
associated with their enrichment types.
+   * @param enrichmentsByType
+   */
+  public ParallelEnricher( Map> 
enrichmentsByType
+ , ConcurrencyContext concurrencyContext
+ , boolean logStats
+ )
+  {
+this.enrichmentsByType = enrichmentsByType;
+this.concurrencyContext = concurrencyContext;
+if(logStats) {
+  for(EnrichmentStrategies s : EnrichmentStrategies.values()) {
+cacheStats.put(s, null);
+  }
+}
+  }
+
+  /**
+   * Fully enriches a message.  Each enrichment is done in parallel via a 
threadpool.
+   * Each enrichment is fronted with a LRU cache.
+   *
+   * @param message the message to enrich
+   * @param strategy The enrichment strategy to use (e.g. enrichment or 
threat intel)
+   * @param config The sensor enrichment con

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-06 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16388021#comment-16388021
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
Ok, README is updated with the new topology diagram.  Let me know if 
there's anything else.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-06 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16387684#comment-16387684
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user nickwallen commented on the issue:

https://github.com/apache/metron/pull/940
  
That's great @cestella .  Many thanks.  I will run it up in the lab. No 
problem.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16387173#comment-16387173
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user ottobackwards commented on the issue:

https://github.com/apache/metron/pull/940
  
Maybe the issue has to do with our keys, and their distribution as the size 
get's larger?  Maybe when we get larger sizes we get more collisions and end up 
calling equals() more or something.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16387170#comment-16387170
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user ottobackwards commented on the issue:

https://github.com/apache/metron/pull/940
  
This should have the equiv. diagram and documentation ( i believe as shown 
above ) to the original split join strategy.



> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16387100#comment-16387100
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172383791
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/parallel/Strategy.java
 ---
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.enrichment.parallel;
+
+import org.apache.metron.common.Constants;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import 
org.apache.metron.common.configuration.enrichment.handler.ConfigHandler;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+
+import java.util.Map;
+
+/**
+ * Enrichment strategy.  This interface provides a mechanism to interface 
with the enrichment config and any
+ * post processing steps that are needed to be done after-the-fact.
+ *
+ * The reasoning behind this is that the key difference between 
enrichments and threat intel is that they pull
+ * their configurations from different parts of the SensorEnrichmentConfig 
object and as a post-join step, they differ
+ * slightly.
+ *
+ */
+public interface Strategy {
+  Constants.ErrorType getErrorType();
--- End diff --

done


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> s

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16387101#comment-16387101
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172383810
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/UnifiedEnrichmentBolt.java
 ---
@@ -0,0 +1,415 @@
+/**
+ * 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.bolt;
+
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.bolt.ConfiguredEnrichmentBolt;
+import org.apache.metron.common.configuration.ConfigurationType;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.error.MetronError;
+import org.apache.metron.common.performance.PerformanceLogger;
+import org.apache.metron.common.utils.ErrorUtils;
+import org.apache.metron.common.utils.MessageUtils;
+import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase;
+import org.apache.metron.enrichment.configuration.Enrichment;
+import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
+import org.apache.metron.enrichment.parallel.EnrichmentContext;
+import org.apache.metron.enrichment.parallel.EnrichmentStrategies;
+import org.apache.metron.enrichment.parallel.ParallelEnricher;
+import org.apache.metron.enrichment.parallel.WorkerPoolStrategy;
+import org.apache.metron.stellar.dsl.Context;
+import org.apache.metron.stellar.dsl.StellarFunction;
+import org.apache.metron.stellar.dsl.StellarFunctions;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.UnsupportedEncodingException;
+import java.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * This bolt is a unified enrichment/threat intel bolt.  In contrast to 
the split/enrich/join
+ * bolts above, this handles the entire enrichment lifecycle in one bolt 
using a threadpool to
+ * enrich in parallel.
+ *
+ * From an architectural perspective, this is a divergence from the 
polymorphism based strategy we have
+ * used in the split/join bolts.  Rather, this bolt is provided a strategy 
to use, either enrichment or threat intel,
+ * through composition.  This allows us to move most of the implementation 
into components independent
+ * from Storm.  This will greater facilitate reuse.
+ */
+public class UnifiedEnrichmentBolt extends ConfiguredEnrichmentBolt {
+
+  public static class Perf {} // used for performance logging
+  private PerformanceLogger perfLog; // not static bc multiple bolts may 
exist in same worker
+
+  protected static final Logger LOG = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final String STELLAR_CONTEXT_CONF = "stellarContext";
+
+  /**
+   * The number of threads in the threadpool.  One threadpool is created 
per process.
+   * This is a topology-level configuration
+   */
+  public static final String THREADPOOL_NUM_THREADS_TOPOLOGY_CONF = 
"metron.threadpool.size";
+  /**
+   * The type of threadpool to create. This is a topology-level 
configuration.
+   */
+  public static final String THREADPOOL_TYPE_TOPOLOGY_CONF = 
"metron.threadpool.type";
+
+  /**
+   * The enricher implementation to use

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16387098#comment-16387098
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
@nickwallen Ok, I refactored the abstraction to separate some concerns, 
name things a bit, and collapse some of the more onerous abstractions.  Also 
updated javadocs.  

Can you give it another look and see what you think?  We probably should 
also give it another smoketest in the lab to make sure I didn't do something 
dumb.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16387099#comment-16387099
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172383754
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/parallel/EnrichmentStrategies.java
 ---
@@ -0,0 +1,79 @@
+/**
+ * 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.parallel;
+
+import org.apache.metron.common.Constants;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import 
org.apache.metron.common.configuration.enrichment.handler.ConfigHandler;
+import org.apache.metron.enrichment.bolt.CacheKey;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+public enum EnrichmentStrategies implements Strategy {
--- End diff --

I decided that this is too onerous of an abstraction and rethought it a 
bit.  Give it another look, please.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml i

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16387052#comment-16387052
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172377136
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/parallel/EnrichmentStrategies.java
 ---
@@ -0,0 +1,79 @@
+/**
+ * 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.parallel;
+
+import org.apache.metron.common.Constants;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import 
org.apache.metron.common.configuration.enrichment.handler.ConfigHandler;
+import org.apache.metron.enrichment.bolt.CacheKey;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+public enum EnrichmentStrategies implements Strategy {
--- End diff --

I might be answering a question that you're not asking, but this bit of 
awkwardness arises because we have merged the concepts of threat intel and 
enrichment, which differ really only in post-processing.  The approach 
presented here, in contrast to the inheritance-based approach in the bolts, 
allows for an abstraction through composition whereby we localize all the 
interactions with the sensor enrichment config in a strategy rather than bind 
the abstraction to Storm, our distributed processing engine.  That is the 
rationale behind this approach at least.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16387019#comment-16387019
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172373203
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/UnifiedEnrichmentBolt.java
 ---
@@ -0,0 +1,415 @@
+/**
+ * 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.bolt;
+
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.bolt.ConfiguredEnrichmentBolt;
+import org.apache.metron.common.configuration.ConfigurationType;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.error.MetronError;
+import org.apache.metron.common.performance.PerformanceLogger;
+import org.apache.metron.common.utils.ErrorUtils;
+import org.apache.metron.common.utils.MessageUtils;
+import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase;
+import org.apache.metron.enrichment.configuration.Enrichment;
+import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
+import org.apache.metron.enrichment.parallel.EnrichmentContext;
+import org.apache.metron.enrichment.parallel.EnrichmentStrategies;
+import org.apache.metron.enrichment.parallel.ParallelEnricher;
+import org.apache.metron.enrichment.parallel.WorkerPoolStrategy;
+import org.apache.metron.stellar.dsl.Context;
+import org.apache.metron.stellar.dsl.StellarFunction;
+import org.apache.metron.stellar.dsl.StellarFunctions;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.UnsupportedEncodingException;
+import java.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * This bolt is a unified enrichment/threat intel bolt.  In contrast to 
the split/enrich/join
+ * bolts above, this handles the entire enrichment lifecycle in one bolt 
using a threadpool to
+ * enrich in parallel.
+ *
+ * From an architectural perspective, this is a divergence from the 
polymorphism based strategy we have
+ * used in the split/join bolts.  Rather, this bolt is provided a strategy 
to use, either enrichment or threat intel,
+ * through composition.  This allows us to move most of the implementation 
into components independent
+ * from Storm.  This will greater facilitate reuse.
+ */
+public class UnifiedEnrichmentBolt extends ConfiguredEnrichmentBolt {
+
+  public static class Perf {} // used for performance logging
+  private PerformanceLogger perfLog; // not static bc multiple bolts may 
exist in same worker
+
+  protected static final Logger LOG = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final String STELLAR_CONTEXT_CONF = "stellarContext";
+
+  /**
+   * The number of threads in the threadpool.  One threadpool is created 
per process.
+   * This is a topology-level configuration
+   */
+  public static final String THREADPOOL_NUM_THREADS_TOPOLOGY_CONF = 
"metron.threadpool.size";
+  /**
+   * The type of threadpool to create. This is a topology-level 
configuration.
+   */
+  public static final String THREADPOOL_TYPE_TOPOLOGY_CONF = 
"metron.threadpool.type";
+
+  /**
+   * The enricher implementation to use

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386994#comment-16386994
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172369461
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/parallel/EnrichmentStrategies.java
 ---
@@ -0,0 +1,79 @@
+/**
+ * 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.parallel;
+
+import org.apache.metron.common.Constants;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import 
org.apache.metron.common.configuration.enrichment.handler.ConfigHandler;
+import org.apache.metron.enrichment.bolt.CacheKey;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+public enum EnrichmentStrategies implements Strategy {
--- End diff --

This is a strategy pattern using an enum.  The purpose of this class is to 
resolve the specific strategies possible.  It's broadly in line with other 
strategy patterns (e.g. Extractors). 


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by 

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386995#comment-16386995
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172369480
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/parallel/Strategy.java
 ---
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.enrichment.parallel;
+
+import org.apache.metron.common.Constants;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import 
org.apache.metron.common.configuration.enrichment.handler.ConfigHandler;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+
+import java.util.Map;
+
+/**
+ * Enrichment strategy.  This interface provides a mechanism to interface 
with the enrichment config and any
+ * post processing steps that are needed to be done after-the-fact.
+ *
+ * The reasoning behind this is that the key difference between 
enrichments and threat intel is that they pull
+ * their configurations from different parts of the SensorEnrichmentConfig 
object and as a post-join step, they differ
+ * slightly.
+ *
+ */
+public interface Strategy {
+  Constants.ErrorType getErrorType();
--- End diff --

Sure thing.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in t

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386973#comment-16386973
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user nickwallen commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172363362
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/parallel/EnrichmentStrategies.java
 ---
@@ -0,0 +1,79 @@
+/**
+ * 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.parallel;
+
+import org.apache.metron.common.Constants;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import 
org.apache.metron.common.configuration.enrichment.handler.ConfigHandler;
+import org.apache.metron.enrichment.bolt.CacheKey;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+public enum EnrichmentStrategies implements Strategy {
--- End diff --

I don't understand the purpose of this class.  Why have an 
`EnrichmentStrategy`, a `ThreatIntelStrategy`, and `EnrichmentStrategies`?


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script 

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386972#comment-16386972
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user nickwallen commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172359339
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/parallel/Strategy.java
 ---
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.metron.enrichment.parallel;
+
+import org.apache.metron.common.Constants;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import 
org.apache.metron.common.configuration.enrichment.handler.ConfigHandler;
+import org.json.simple.JSONObject;
+import org.slf4j.Logger;
+
+import java.util.Map;
+
+/**
+ * Enrichment strategy.  This interface provides a mechanism to interface 
with the enrichment config and any
+ * post processing steps that are needed to be done after-the-fact.
+ *
+ * The reasoning behind this is that the key difference between 
enrichments and threat intel is that they pull
+ * their configurations from different parts of the SensorEnrichmentConfig 
object and as a post-join step, they differ
+ * slightly.
+ *
+ */
+public interface Strategy {
+  Constants.ErrorType getErrorType();
--- End diff --

Can we javadoc each method?  This seems like an important interface.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386974#comment-16386974
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user nickwallen commented on a diff in the pull request:

https://github.com/apache/metron/pull/940#discussion_r172353404
  
--- Diff: 
metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/UnifiedEnrichmentBolt.java
 ---
@@ -0,0 +1,415 @@
+/**
+ * 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.bolt;
+
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.bolt.ConfiguredEnrichmentBolt;
+import org.apache.metron.common.configuration.ConfigurationType;
+import 
org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig;
+import org.apache.metron.common.error.MetronError;
+import org.apache.metron.common.performance.PerformanceLogger;
+import org.apache.metron.common.utils.ErrorUtils;
+import org.apache.metron.common.utils.MessageUtils;
+import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase;
+import org.apache.metron.enrichment.configuration.Enrichment;
+import org.apache.metron.enrichment.interfaces.EnrichmentAdapter;
+import org.apache.metron.enrichment.parallel.EnrichmentContext;
+import org.apache.metron.enrichment.parallel.EnrichmentStrategies;
+import org.apache.metron.enrichment.parallel.ParallelEnricher;
+import org.apache.metron.enrichment.parallel.WorkerPoolStrategy;
+import org.apache.metron.stellar.dsl.Context;
+import org.apache.metron.stellar.dsl.StellarFunction;
+import org.apache.metron.stellar.dsl.StellarFunctions;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.UnsupportedEncodingException;
+import java.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * This bolt is a unified enrichment/threat intel bolt.  In contrast to 
the split/enrich/join
+ * bolts above, this handles the entire enrichment lifecycle in one bolt 
using a threadpool to
+ * enrich in parallel.
+ *
+ * From an architectural perspective, this is a divergence from the 
polymorphism based strategy we have
+ * used in the split/join bolts.  Rather, this bolt is provided a strategy 
to use, either enrichment or threat intel,
+ * through composition.  This allows us to move most of the implementation 
into components independent
+ * from Storm.  This will greater facilitate reuse.
+ */
+public class UnifiedEnrichmentBolt extends ConfiguredEnrichmentBolt {
+
+  public static class Perf {} // used for performance logging
+  private PerformanceLogger perfLog; // not static bc multiple bolts may 
exist in same worker
+
+  protected static final Logger LOG = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final String STELLAR_CONTEXT_CONF = "stellarContext";
+
+  /**
+   * The number of threads in the threadpool.  One threadpool is created 
per process.
+   * This is a topology-level configuration
+   */
+  public static final String THREADPOOL_NUM_THREADS_TOPOLOGY_CONF = 
"metron.threadpool.size";
+  /**
+   * The type of threadpool to create. This is a topology-level 
configuration.
+   */
+  public static final String THREADPOOL_TYPE_TOPOLOGY_CONF = 
"metron.threadpool.type";
+
+  /**
+   * The enricher implementation to u

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386931#comment-16386931
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
Ahhh, that makes sense.  I bet we were getting killed by small allocations 
in the caching layer.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386928#comment-16386928
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user ben-manes commented on the issue:

https://github.com/apache/metron/pull/940
  
Caffeine doesn't allocate on read, so that would make sense. I saw a [25x 
boost](https://github.com/google/guava/issues/2063#issuecomment-107169736) 
(compared to 
[current](https://github.com/google/guava/issues/2063#issue-82444927)) when 
porting the buffers to Guava.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386923#comment-16386923
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
I actually suspect GC as well.  We adjusted the garbage collector to the 
G1GC and saw throughput gains, but not nearly the kinds of gains as we got with 
a drop-in of Caffeine to replace Guava.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386918#comment-16386918
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user ben-manes commented on the issue:

https://github.com/apache/metron/pull/940
  
Interesting. Then I guess the size must trigger the read bottleneck as 
larger than writes. Perhaps it is incurring a lot more GC overhead that causes 
more collections? The CLQ additions requires allocating a new queue node. That 
and the cache entry probably get promoted to old gen due to the high churn 
rate, causing everything to slow down. Probably isn't too interesting to 
investigate vs swapping libraries :)


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386903#comment-16386903
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
In this case, the loader isn't doing anything terribly expensive, though it 
may in the future (incur a hbase get or some more expensive computation).


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386880#comment-16386880
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user ben-manes commented on the issue:

https://github.com/apache/metron/pull/940
  
Internally Guava uses a `ConcurrentLinkedQueue` and an `AtomicInteger` to 
record its size, per segment. When a read occurs, it records that in the queue 
and then drains it under the segment's lock (via tryLock) to replay the events. 
This is similar to Caffeine, which uses optimized structures instead. I 
intended the CLQ & counter as baseline scaffolding for replacement, as it is an 
obvious bottleneck, but I could never get it replaced despite advocating for 
it. The penalty of draining the buffers is amortized, but unfortunately this 
buffer isn't capped.

Since there would be a higher hit rate with a larger cache, the reads would 
be recorded more often. Perhaps contention there and the penalty of draining 
the queue is more observable than a cache miss. That's still surprising since a 
cache miss is usually more expensive I/O. Is the loader doing expensive work in 
your case?

Caffeine gets around this problem by using more optimal buffers and being 
lossy (on reads only) if it can't keep up. By default it delegates the 
amortized maintenance work to a ForkJoinPool to avoid user-facing latencies, 
since you'll want those variances to be tight. Much of that can be back ported 
onto Guava for a nice boost.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386865#comment-16386865
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
We actually did increase the concurrency level for guava to 64; that is 
what confused us as well.  The hash code is mostly standard, should be evenly 
distributed (the key is pretty much a POJO).


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386861#comment-16386861
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user ben-manes commented on the issue:

https://github.com/apache/metron/pull/940
  
Guava defaults to a `concurrencyLevel` of 4, given its age and a desire to 
not abuse memory in low concurrent situations. You probably want to increase it 
to 64 in a heavy workload, which has a ~4x throughput gain on reads. It won't 
scale much higher, since it has internal bottlenecks and I could never get 
patches reviewed to fix those.

I've only noticed overall throughput be based on threads, and never 
realized there was a capacity constraint to its performance. One should expect 
some due to the older hash table design resulting in more collisions, whereas 
CHMv8 does much better there. Still, I would have expected it to even out 
enough unless have a bad hash code?


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386854#comment-16386854
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
The interesting thing that we found was that guava seems to be doing poorly 
when the # of items in the cache gets large.  When we scaled the test down (830 
distinct IP addresses chosen randomly and sent in at a rate of 200k events per 
second with a cache size of 100) kept up but scaling the test up (300k distinct 
ip addresses chosen randomly and sent in at a rate of 200k events per second 
with a cache size of 100k) didn't. 


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386851#comment-16386851
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user ben-manes commented on the issue:

https://github.com/apache/metron/pull/940
  
That makes sense. A uniform distribution will, of course, degrades all 
policies to random replacement so the test is then about how well the 
implementations handle concurrency. Most often caches exhibit a Zipfian 
distribution (80-20 rule), so our bias towards frequency is a net gain. We have 
observed a few rare cases where frequency is a poor signal and LRU is optimal, 
and we are exploring adaptive techniques to dynamically tune the cache based on 
the workload's characteristics. These cases don't seem to occur in many 
real-world scenarios that we know of, but it is always nice to know what users 
are experiencing and how much better (or worse) we perform than a standard LRU 
cache.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386842#comment-16386842
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
We were being purposefully unkind to the cache in the tests.  The load 
simulation chose a IP address at random to present, so each IP had an equal 
probability of being selected.  Whereas, in real traffic, we expect a coherent 
working set.  Not sure of the exact hit rates, though.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386815#comment-16386815
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user ben-manes commented on the issue:

https://github.com/apache/metron/pull/940
  
Do you know what the hit rates were, for the same data set, between Guava 
and Caffeine? The caches use different policies so it is always interesting to 
see how the handle given workloads. As we continue to refine our adaptive 
algorithm W-TinyLFU, its handy to know what types of workloads to investigate. 
(P.S. We have a simulator for re-running persisted traces if useful for your 
tuning)


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386809#comment-16386809
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user nickwallen commented on the issue:

https://github.com/apache/metron/pull/940
  
I completed some fairly extensive performance testing comparing this new 
Unified topology against the existing Split-Join implementation.  The 
difference was dramatic. 

- The Unified topology _performed roughly 3.4 times faster than Split-Join._

Both topologies in this side-by-side test included the same fixes, 
including the Guava cache problem fixed in #947. The tests included two 
enrichments:
* GeoIP enrichment; `geo := GEO_GET(ip_dst_addr)`
* Compute-only Stellar enrichment; `local := IN_SUBNET(ip_dst_addr, 
'192.168.0.0/24')`

The number one driver of performance is the cache hit rate, which is 
heavily dependent on what your data looks-like.  With these enrichments, that's 
driven by how varied the `ip_dst_addr` is in the data.  

I tested both of these topologies with different sets of data intended to 
either increase or decrease that cache hit rate.  The differences between the 
two topologies were fairly consistent across the different data sets. 

When running these topologies, reasonably well-tuned, on the same data, I 
was able to consistently maintain 70,000 events per second with the Split/Join 
topology.  In the same environment, I was able to maintain 312,000 events per 
second using the Unified topology.  

The raw throughput numbers are relative and depend on how much hardware you 
are willing to throw at the problem.  I was running on 3 nodes dedicated to 
running the Enrichment topology only.  But with the same data, on the same 
hardware, the difference was 3.4 times.  That's big.

Pushing as much as you can into a single executor and avoiding network hops 
is definitely the way to go here.



> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a w

[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16386542#comment-16386542
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
I ran this up with vagrant and ensured:
* Normal stellar works still in field transformations as well as enrichments
* swapped in and out new enrichments live
* swapped in and out new threat intel live

Are there any other pending issues here beyond a report of the performance 
impact?


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-02 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16384342#comment-16384342
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
Just FYI, as part of the performance experimentation in the lab here, we 
found that one major impediment to scale was the guava cache in this topology 
when the size of the cache becomes non-trivial in size (e.g. 10k+).  Swapping 
out [Caffeine](https://github.com/ben-manes/caffeine) immediately had a 
substantial affect.  I created #947 to migrate the split/join infrastructure to 
use caffeine as well and will look at the performance impact of that change.  I 
wanted to separate that work from here as it may be that guava performance is 
fine outside of an explicit threadpool like we have here.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-02 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16383995#comment-16383995
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
@arunmahadevan Thanks for chiming in Arun.   I would say that most of the 
enrichment work is I/O bound and we try to avoid it whenever possible with a a 
time-evicted LRU cache in front of the enrichments.  We don't always know a 
priori what enrichments users are doing, per se, as their individual 
enrichments may be expressed via stellar.  The threads here are entirely 
managed via the fixed threadpool service in storm and the threadpool is shared 
across all of the executors running in-process on the worker, so we try to 
minimize that.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-02 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16383872#comment-16383872
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user arunmahadevan commented on the issue:

https://github.com/apache/metron/pull/940
  
Managing threadpools within a bolt isn't fundamentally wrong, we have see 
some use cases where this is done. However, we have been putting efforts to 
reduce the overall number of threads created  internally within storm since the 
thread context switches were causing performance bottlenecks. I assume the 
threadpool threads are mostly IO/network bound so it should not cause too much 
harm.

Do you need multiple threads since the enrichments involve external DB look 
ups and are time consuming ?  Maybe you could compare the performance of 
maintaining a thread pool v/s increasing the bolt's parallelism to achieve a 
similar effect. 

Another option might be to prefetch the enrichment data and load it into 
each bolt so that you might not need separate threads to do the enrichment.

If you are able to manage without threads, that would be preferable. Even 
otherwise its not that bad as long as you don't create too many threads and 
they are cleaned up properly. (we have had some cases were the internal threads 
were causing workers to hang).


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-02 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16383660#comment-16383660
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
@ottobackwards I haven't sent an email to the storm team, but I did run the 
PR past a storm committer that I know and asked his opinion prior to submitting 
the PR.  The general answer was something to the effect of `The overall goal 
should be to reduce the network shuffle unless its really required.`  Also, the 
notion of using an external threadpool didn't seem to be fundamentally 
offensive.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-02 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16383655#comment-16383655
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user ottobackwards commented on the issue:

https://github.com/apache/metron/pull/940
  
have we thought to send a mail to the storm dev list and ask if anyone has 
done this?  potential issues?


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-02 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16383652#comment-16383652
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user ottobackwards commented on the issue:

https://github.com/apache/metron/pull/940
  
If we integrated storm with yarn this would also be a problem, as our 
resource management may be at odds with yarn's.  I think?

What would be nice is if storm could manage the pool and we could just use 
it.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-03-02 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16383634#comment-16383634
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
@mraliagha It's definitely a tradeoff.  This is why this is as a complement 
to the original split/join topology.  Keep in mind, also, that this 
architecture enables use-cases that the other would prevent or make extremely 
difficult and/or network intensive, such as multi-level stellar statements 
rather than the 2 levels we have now.  We are undergoing some preliminary 
testing in-lab right now, which @nickwallen alluded to, to compare the two 
approaches under at least synthetic load and will report back.

Ultimately this boils down to efficiencies gained by avoiding network hops 
and whether that's going to provide an outsized impact, I think.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-02-28 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16381373#comment-16381373
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user mraliagha commented on the issue:

https://github.com/apache/metron/pull/940
  
@cestella Thanks, Casey. Wouldn't be still hard to tune this solution? 
Still, thread pool tuning and probably the race condition between these threads 
and normal Strom workers makes the tuning hard for a production platform with 
tons of feeds/topologies. Storm resource management is very basic at this stage 
to absorb spikes, and having a separate thread pool transfers the complexity 
from one place to another place. 


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-02-28 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16380365#comment-16380365
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
@nickwallen Sounds good.  When scale tests are done, can we make sure that 
we also include #944 ?


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-02-27 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16378811#comment-16378811
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user nickwallen commented on the issue:

https://github.com/apache/metron/pull/940
  
I'd hold on merging this until we can get this tested at some decent scale. 
 Unless it already has been?  Otherwise, I don't see a need to merge this until 
we know it actually addresses a problem.


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-02-26 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16377266#comment-16377266
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user merrimanr commented on the issue:

https://github.com/apache/metron/pull/940
  
I tested this in full dev and worked as expected.  +1


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-02-23 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16374550#comment-16374550
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user cestella commented on the issue:

https://github.com/apache/metron/pull/940
  
The current architecture is described ![Image of 
Yaktocat](https://github.com/apache/metron/raw/master/metron-platform/metron-enrichment/enrichment_arch.png)

In short, for each message each splitter will
* inspect the configs for the sensor 
* For each sensor, extract the fields required for enrichment and send them 
to the appropriate enrichment bolt (e.g. hbase, geo, stellar)
  * If one enrichment enriches k fields, then k messages will be sent to 
the enrichment bolt
  * In the case of stellar, each stellar subgroup will be a separate message
  * the original message is sent directly to the join bolt
* The enrichment bolts do the enrichment and send the additional fields and 
values to the original message
* The join bolt will asynchronously collect the subresults and join them 
with the original message
  * The join bolt has a LRU cache to hold subresults until all results 
arrive
  * Tuning performance involves tuning this cache (max size and time until 
eviction)
  * Tuning this can be complex because it has to be large enough to handle 
spikes in traffic




> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (METRON-1460) Create a complementary non-split-join enrichment topology

2018-02-22 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/METRON-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16373969#comment-16373969
 ] 

ASF GitHub Bot commented on METRON-1460:


Github user mraliagha commented on the issue:

https://github.com/apache/metron/pull/940
  
Is there any document somewhere to show how the previous approach was 
implemented? I would like to understand the previous architecture in details. 
Becuase some of the pros/cons didn't make sense to me. Maybe I can help to 
predict what the impact will be. Thanks. 


> Create a complementary non-split-join enrichment topology
> -
>
> Key: METRON-1460
> URL: https://issues.apache.org/jira/browse/METRON-1460
> Project: Metron
>  Issue Type: New Feature
>Reporter: Casey Stella
>Priority: Major
>
> There are some deficiencies to the split/join topology.
>  * It's hard to reason about
>  * Understanding the latency of enriching a message requires looking at 
> multiple bolts that each give summary statistics
>  * The join bolt's cache is really hard to reason about when performance 
> tuning
>  * During spikes in traffic, you can overload the join bolt's cache and drop 
> messages if you aren't careful
>  * In general, it's hard to associate a cache size and a duration kept in 
> cache with throughput and latency
>  * There are a lot of network hops per message
>  * Right now we are stuck at 2 stages of transformations being done 
> (enrichment and threat intel).  It's very possible that you might want 
> stellar enrichments to depend on the output of other stellar enrichments.  In 
> order to implement this in split/join you'd have to create a cycle in the 
> storm topology
>  
> I propose that we move to a model where we do enrichments in a single bolt in 
> parallel using a static threadpool (e.g. multiple workers in the same process 
> would share the threadpool).  IN all other ways, this would be backwards 
> compatible.  A transparent drop-in for the existing enrichment topology.
> There are some pros/cons about this too:
>  * Pro
>  * Easier to reason about from an individual message perspective
>  * Architecturally decoupled from Storm
>  * This sets us up if we want to consider other streaming technologies
>  * Fewer bolts
>  * spout -> enrichment bolt -> threatintel bolt -> output bolt
>  * Way fewer network hops per message
>  * currently 2n+1 where n is the number of enrichments used (if using stellar 
> subgroups, each subgroup is a hop)
>  * Easier to reason about from a performance perspective
>  * We trade cache size and eviction timeout for threadpool size
>  * We set ourselves up to have stellar subgroups with dependencies
>  * i.e. stellar subgroups that depend on the output of other subgroups
>  * If we do this, we can shrink the topology to just spout -> 
> enrichment/threat intel -> output
>  * Con
>  * We can no longer tune stellar enrichments independent from HBase 
> enrichments
>  * To be fair, with enrichments moving to stellar, this is the case in the 
> split/join approach too
>  * No idea about performance
> What I propose is to submit a PR that will deliver an alternative, completely 
> backwards compatible topology for enrichment that you can use by adjusting 
> the start_enrichment_topology.sh script to use remote-unified.yaml instead of 
> remote.yaml.  If we live with it for a while and have some good experiences 
> with it, maybe we can consider retiring the old enrichment topology.
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)