[DISCUSS] Enrichment Split/Join issues
Hi All, Last week, I encountered some weirdness in the Enrichment topology. Doing some somewhat high-latency enrichment work, I noticed that at some point, data stopped flowing through the enrichment topology. I tracked down the problem to the join bolt. For those who aren't aware, we do a split/join pattern so that enrichments can be done in parallel. It works as follows: - A split bolt sends the appropriate subset of the message to each enrichment bolt as well as the whole message to the join bolt - The join bolt will receive each of the pieces of the message and then, when fully joined, it will send the message on. What is happening under load or high velocity, however, is that the cache is evicting the partially joined message before it can be fully joined due to the volume of traffic. This is obviously not ideal. As such, it is clear that adjusting the size of the cache and the characteristics of eviction is likely a good idea and a necessary part to tuning enrichments. The cache size is sensitive to: - The latency of the *slowest* enrichment - The number of tuples in flight at once As such, the knobs you have to tune are either the parallelism of the join bolt or the size of the cache. As it stands, I see a couple of things wrong here that we can correct with minimal issue: - We have no message of warning indicating that this is happening - Changing cache sizes means changing flux. We should promote this to the properties file. - We should document the knobs mentioned above clearly in the enrichment topology README Those small changes, I think, are table stakes, but what I wanted to discuss more in depth is the lingering questions: - Is this an architectural pattern that we can use as-is? - Should we consider a persistent cache a la HBase or Apache Ignite as a pluggable component to Metron? - Should we consider taking the performance hit and doing the enrichments serially? - When an eviction happens, what should we do? - Fail the tuple, thereby making congestion worse - Pass through the partially enriched results, thereby making enrichments "best effort" Anyway, I wanted to talk this through and inform of some of the things I'm seeing. Sorry for the novel. ;) Casey
[GitHub] metron issue #567: METRON-891: Changed Kafka API to Create a KafkaConsumer P...
Github user jjmeyer0 commented on the issue: https://github.com/apache/metron/pull/567 @merrimanr @justinleet is there anything else --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #567: METRON-891: Changed Kafka API to Create a KafkaCon...
GitHub user jjmeyer0 reopened a pull request: https://github.com/apache/metron/pull/567 METRON-891: Changed Kafka API to Create a KafkaConsumer Per Request ## Contributor Comments [Please place any comments here. A description of the problem/enhancement, how to reproduce the issue, your testing methodology, etc.] I stood up full-dev and made sure all kafka endpoints continue to work. I also created a postman [script](https://gist.github.com/jjmeyer0/49461a0665fd19a4cac5b2f31e524840) that uses [this](https://github.com/postmanlabs/newman/blob/develop/examples/parallel-collection-runs.js) to execute the postman scripts in parallel. I executed a few thousand requests against the API with no issue. ## Pull Request Checklist Thank you for submitting a contribution to Apache Metron. Please refer to our [Development Guidelines](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=61332235) for the complete guide to follow for contributions. Please refer also to our [Build Verification Guidelines](https://cwiki.apache.org/confluence/display/METRON/Verifying+Builds?show-miniview) for complete smoke testing guides. In order to streamline the review of the contribution we ask you follow these guidelines and ask you to double check the following: ### For all changes: - [x] Is there a JIRA ticket associated with this PR? If not one needs to be created at [Metron Jira](https://issues.apache.org/jira/browse/METRON/?selectedTab=com.atlassian.jira.jira-projects-plugin:summary-panel). - [x] Does your PR title start with METRON- where is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character. - [x] Has your PR been rebased against the latest commit within the target branch (typically master)? ### For code changes: - [ ] Have you included steps to reproduce the behavior or problem that is being changed or addressed? - [ ] Have you included steps or a guide to how the change may be verified and tested manually? - [x] Have you ensured that the full suite of tests and checks have been executed in the root incubating-metron folder via: ``` mvn -q clean integration-test install && build_utils/verify_licenses.sh ``` - [x] Have you written or updated unit tests and or integration tests to verify your changes? - [x] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? - [x] Have you verified the basic functionality of the build by building and running locally with Vagrant full-dev environment or the equivalent? ### For documentation related changes: - [ ] Have you ensured that format looks appropriate for the output in which it is rendered by building and verifying the site-book? If not then run the following commands and the verify changes via `site-book/target/site/index.html`: ``` cd site-book mvn site ``` Note: Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible. It is also recommended that [travis-ci](https://travis-ci.org) is set up for your personal repository such that your branches are built there before submitting a pull request. You can merge this pull request into a Git repository by running: $ git pull https://github.com/jjmeyer0/incubator-metron METRON-891 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/metron/pull/567.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #567 commit 0324c0b4ad303c8c1c3bd23731619a7e90739d90 Author: JJDate: 2017-04-27T16:40:59Z METRON-891: Updated kafka service to create a ConsumerFactory per thread to prevent possible concurrency issues. METRON-891: Updated kafka to use spring's ConsumerFactory functionality for Kafka. This will help with potential multi-threading issues. METRON-891: Fixed unit test issues and checkstyle issues. METRON-891: Really updating dependencies... METRON-891: Fixing license after autoformat add p tags around link. commit 7b875b17a8120f20a549570e9f57e30c8fb40d84 Author: JJ Date: 2017-05-11T11:49:36Z METRON-891: fixed license after autoformat. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #567: METRON-891: Changed Kafka API to Create a KafkaConsumer P...
Github user merrimanr commented on the issue: https://github.com/apache/metron/pull/567 Looks good to me. Thanks @jjmeyer0. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #574: METRON-934: Component and task id are missing in the inde...
Github user cestella commented on the issue: https://github.com/apache/metron/pull/574 Wow, great addition. We have definitely not tested the HDFS Writer sufficiently. ;) --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #584: METRON-950: Migrate storm-kafka-client to 1.1
Github user justinleet commented on the issue: https://github.com/apache/metron/pull/584 Same here, +1. Thanks a lot for the contribution, this makes a big difference. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #531: METRON-854 create dhcp dump parser
Github user ctramnitz commented on the issue: https://github.com/apache/metron/pull/531 dhcp also carries a client-id that is often (but not always and not reliably) the hostname. While not reliable, this is intersting information, especially since you don't have to perform (expensive) DNS lookups, but can consume whats on the wire or a dhcp server reports. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #585: METRON-936: Fixes to pcap for performance and test...
Github user justinleet commented on a diff in the pull request: https://github.com/apache/metron/pull/585#discussion_r116741150 --- Diff: metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/spout/pcap/PartitionHDFSWriter.java --- @@ -102,14 +106,43 @@ public void sync(FSDataOutputStream input) throws IOException { private SyncHandler syncHandler; private long batchStartTime; private long numWritten; + private Configuration fsConfig = new Configuration(); public PartitionHDFSWriter(String topic, int partition, String uuid, HDFSWriterConfig config) { this.topic = topic; this.partition = partition; this.uuid = uuid; this.config = config; + try { - this.fs = FileSystem.get(new Configuration()); + int replicationFactor = config.getReplicationFactor(); + if (replicationFactor != -1) { +fsConfig.set("dfs.replication", (String.valueOf(replicationFactor))); + } + if(config.getHDFSConfig() != null && !config.getHDFSConfig().isEmpty()) { +for(Map.Entryentry : config.getHDFSConfig().entrySet()) { + if(entry.getValue() instanceof Integer) { --- End diff -- Under the hood, don't all these fsConfig.setX calls just call String.valueOf()? I'm not sure if that relationship (being all String under the hood is guaranteed or just an implementation detail), but if it is, this if-else chain simplifies significantly. I'll dig in a bit to see if there is. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #585: METRON-936: Fixes to pcap for performance and test...
Github user justinleet commented on a diff in the pull request: https://github.com/apache/metron/pull/585#discussion_r116742092 --- Diff: metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/spout/pcap/deserializer/KeyValueDeserializer.java --- @@ -28,6 +26,17 @@ public abstract class KeyValueDeserializer implements Serializable { protected TimestampConverter converter; + public static class Result { --- End diff -- Can we call this something more informative than Result? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #585: METRON-936: Fixes to pcap for performance and test...
Github user justinleet commented on a diff in the pull request: https://github.com/apache/metron/pull/585#discussion_r116741758 --- Diff: metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/spout/pcap/PartitionHDFSWriter.java --- @@ -183,14 +219,14 @@ private void turnoverIfNecessary(long ts, boolean force) throws IOException { } } - writer = SequenceFile.createWriter(new Configuration() + writer = SequenceFile.createWriter(this.fsConfig , SequenceFile.Writer.keyClass(LongWritable.class) , SequenceFile.Writer.valueClass(BytesWritable.class) , SequenceFile.Writer.stream(outputStream) , SequenceFile.Writer.compression(SequenceFile.CompressionType.NONE) ); //reset state - LOG.info("Turning over and writing to " + path); + LOG.info(String.format("Turning over and writing to %s: [duration=%s NS, force=%s, initial=%s, overDuration=%s, tooManyPackets=%s]", path, duration, force, initial, overDuration, tooManyPackets)); --- End diff -- Couldn't this be formatted using the Logger's built-in lazy token replacement, rather than the String.format? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #585: METRON-936: Fixes to pcap for performance and test...
Github user justinleet commented on a diff in the pull request: https://github.com/apache/metron/pull/585#discussion_r116739444 --- Diff: metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/spout/pcap/PartitionHDFSWriter.java --- @@ -102,14 +106,43 @@ public void sync(FSDataOutputStream input) throws IOException { private SyncHandler syncHandler; private long batchStartTime; private long numWritten; + private Configuration fsConfig = new Configuration(); public PartitionHDFSWriter(String topic, int partition, String uuid, HDFSWriterConfig config) { this.topic = topic; this.partition = partition; this.uuid = uuid; this.config = config; + try { - this.fs = FileSystem.get(new Configuration()); + int replicationFactor = config.getReplicationFactor(); + if (replicationFactor != -1) { --- End diff -- Could we make this > 0? Nothing 0 or less makes sense to actually use. Might also make sense to log any replication factor actually used, for debugging. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #574: METRON-934: Component and task id are missing in t...
Github user asfgit closed the pull request at: https://github.com/apache/metron/pull/574 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
Re: [DISCUSS] Enrichment Split/Join issues
I do want to say here, that I don't mean to sound the alarm and say that everything is broken. I would not characterize the topology as "broken" architecturally, but rather the lack of reporting when things go pear-shaped is a bug in implementation. With logging and documentation about the knobs to tune, this architecture works, I believe. On Tue, May 16, 2017 at 12:09 PM, Casey Stellawrote: > We could definitely parallelize within the bolt, but you're right, it does > break the storm model. I also like making things other people's problems > (it's called working "smart" not "hard", right? not laziness, surely. ;), > but yeah, using windowing for this seems like it might introduce some > artificial latency. It's also not going to eliminate the problem, but > rather just make the knob to tweak things have a different characteristic. > Whereas before we have knobs around how many messages, now it's a knob > around how long an enrichment is going to take maximally (which, I think is > more natural, honestly). > > On Tue, May 16, 2017 at 12:05 PM, Simon Elliston Ball < > si...@simonellistonball.com> wrote: > >> Would you then parallelise within Stellar to handle things like multiple >> lookups? This feels like it would be breaking the storm model somewhat, and >> could lead to bad things with threads for example. Or would you think of >> doing something like the grouping Stellar uses today to parallelise across >> something like a pool of Stellar bolts and join? >> >> I like the idea of Otto’s solution (making it someone else's problem, >> storm’s specifically :) ) but that also assumes we insert the artificial >> latency of a time windowed join. If we’re going down that route, we might >> as well just use spark and run everything on yarn. At that point though we >> lose a lot of the benefits of low latency for time to detection, and >> real-time enrichment in things like the streaming enrichment writer. >> >> Simon >> >> > On 16 May 2017, at 16:59, Nick Allen wrote: >> > >> > I would like to see us just migrate wholly to Stellar enrichments and >> > remove the separate HBase and Geo enrichment bolts from the Enrichment >> > topology. Stellar provides a user with much greater flexibility than >> the >> > existing HBase and Geo enrichment bolts. >> > >> > A side effect of this would be to greatly simplify the Enrichment >> > topology. I don't think we would not need the split/join pattern if we >> did >> > this. No? >> > >> > On Tue, May 16, 2017 at 11:54 AM, Casey Stella >> wrote: >> > >> >> The problem is that an enrichment type won't necessarily have a fixed >> >> performance characteristic. Take stellar enrichments, for instance. >> Doing >> >> a HBase call for one sensor vs doing simple string munging will have >> vastly >> >> differing performance. Both of them are functioning within the stellar >> >> enrichment bolt. Also, some enrichments may call for multiple calls to >> >> HBase. Parallelizing those, would make some sense, I think. >> >> >> >> I do take your point, though, that it's not as though it's strictly >> serial, >> >> it's just that the unit of parallelism is the message, rather than the >> >> enrichment per message. >> >> >> >> On Tue, May 16, 2017 at 11:47 AM, Christian Tramnitz < >> tramn...@trasec.de> >> >> wrote: >> >> >> >>> I’m glad you bring this up. This is a huge architectural difference >> from >> >>> the original OpenSOC topology and one that we have been warned to take >> >> back >> >>> then. >> >>> To be perfectly honest, I don’t see the big perfomance improvement >> from >> >>> parallel processing. If a specific enrichment is a little more i/o >> >>> dependent than the other you can tweak parallelism to address this. >> Also >> >>> there can be dependencies that make parallel enrichment virtually >> >>> impossible or at least less efficient (i.e. first labeling, and >> >>> “completing” a message and then dependent of label and completeness do >> >>> different other enrichments). >> >>> >> >>> So you have a +1 from me for serial rather than parallel enrichment. >> >>> >> >>> >> >>> BR, >> >>> Christian >> >>> >> >>> On 16.05.17, 16:58, "Casey Stella" wrote: >> >>> >> >>>Hi All, >> >>> >> >>>Last week, I encountered some weirdness in the Enrichment topology. >> >>> Doing >> >>>some somewhat high-latency enrichment work, I noticed that at some >> >>> point, >> >>>data stopped flowing through the enrichment topology. I tracked >> down >> >>> the >> >>>problem to the join bolt. For those who aren't aware, we do a >> >>> split/join >> >>>pattern so that enrichments can be done in parallel. It works as >> >>> follows: >> >>> >> >>> - A split bolt sends the appropriate subset of the message to >> each >> >>> enrichment bolt as well as the whole message to the join bolt >> >>> - The join bolt will receive each of the pieces of the message >>
[GitHub] metron issue #531: METRON-854 create dhcp dump parser
Github user JonZeolla commented on the issue: https://github.com/apache/metron/pull/531 Is there enough interest for me to pursue support of this in #586? I could probably throw that together today. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
Re: we currently have 31 PR’s that are not landed
Assuming the unincubating process is almost completed (I don't know if that's true or not), I think there are some simple, obvious priorities based on our pending 0.4.0 release. Things like METRON-833, METRON-819, and METRON-953 should probably get finalized and merged in asap. Also, we have some big wins, like METRON-508, METRON-950, METRON-844, METRON-891, etc. (not an authoritative review) which I would love to see in the release. That said, doing a general call to review PRs without huge changes (METRON-777, I'm lookin' at you) and get them merged is probably a good idea. Jon On Tue, May 16, 2017 at 9:53 AM Otto Fowlerwrote: > https://github.com/apache/metron/pulls > > This seems a little large given that I *think* we have been at around 19 or > so consistently. > -- Jon
[GitHub] metron pull request #584: METRON-950: Migrate storm-kafka-client to 1.1
Github user asfgit closed the pull request at: https://github.com/apache/metron/pull/584 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
Re: [DISCUSS] Enrichment Split/Join issues
The problem is that an enrichment type won't necessarily have a fixed performance characteristic. Take stellar enrichments, for instance. Doing a HBase call for one sensor vs doing simple string munging will have vastly differing performance. Both of them are functioning within the stellar enrichment bolt. Also, some enrichments may call for multiple calls to HBase. Parallelizing those, would make some sense, I think. I do take your point, though, that it's not as though it's strictly serial, it's just that the unit of parallelism is the message, rather than the enrichment per message. On Tue, May 16, 2017 at 11:47 AM, Christian Tramnitzwrote: > I’m glad you bring this up. This is a huge architectural difference from > the original OpenSOC topology and one that we have been warned to take back > then. > To be perfectly honest, I don’t see the big perfomance improvement from > parallel processing. If a specific enrichment is a little more i/o > dependent than the other you can tweak parallelism to address this. Also > there can be dependencies that make parallel enrichment virtually > impossible or at least less efficient (i.e. first labeling, and > “completing” a message and then dependent of label and completeness do > different other enrichments). > > So you have a +1 from me for serial rather than parallel enrichment. > > > BR, >Christian > > On 16.05.17, 16:58, "Casey Stella" wrote: > > Hi All, > > Last week, I encountered some weirdness in the Enrichment topology. > Doing > some somewhat high-latency enrichment work, I noticed that at some > point, > data stopped flowing through the enrichment topology. I tracked down > the > problem to the join bolt. For those who aren't aware, we do a > split/join > pattern so that enrichments can be done in parallel. It works as > follows: > >- A split bolt sends the appropriate subset of the message to each >enrichment bolt as well as the whole message to the join bolt >- The join bolt will receive each of the pieces of the message and > then, >when fully joined, it will send the message on. > > > What is happening under load or high velocity, however, is that the > cache > is evicting the partially joined message before it can be fully joined > due > to the volume of traffic. This is obviously not ideal. As such, it is > clear that adjusting the size of the cache and the characteristics of > eviction is likely a good idea and a necessary part to tuning > enrichments. > The cache size is sensitive to: > >- The latency of the *slowest* enrichment >- The number of tuples in flight at once > > As such, the knobs you have to tune are either the parallelism of the > join > bolt or the size of the cache. > > As it stands, I see a couple of things wrong here that we can correct > with > minimal issue: > >- We have no message of warning indicating that this is happening >- Changing cache sizes means changing flux. We should promote this > to >the properties file. >- We should document the knobs mentioned above clearly in the > enrichment >topology README > > Those small changes, I think, are table stakes, but what I wanted to > discuss more in depth is the lingering questions: > >- Is this an architectural pattern that we can use as-is? > - Should we consider a persistent cache a la HBase or Apache > Ignite > as a pluggable component to Metron? > - Should we consider taking the performance hit and doing the > enrichments serially? >- When an eviction happens, what should we do? > - Fail the tuple, thereby making congestion worse > - Pass through the partially enriched results, thereby making > enrichments "best effort" > > Anyway, I wanted to talk this through and inform of some of the things > I'm > seeing. > > Sorry for the novel. ;) > > Casey > > >
[GitHub] metron pull request #584: METRON-950: Migrate storm-kafka-client to 1.1
GitHub user cestella reopened a pull request: https://github.com/apache/metron/pull/584 METRON-950: Migrate storm-kafka-client to 1.1 ## Contributor Comments There are MAJOR performance issues with the storm-kafka-client. Throughput is roughly an order of magnitude faster in the 1.1.0 version vs the 1.0.3. This also removes the requirement for the non-HDP profile to rely on the HDP repo. I will point out that the storm-kafka-client in 1.1 does rely on `Time.nanoTime()`, a method call that was introduced in 1.1. To get around this, I've created the `metron-storm-kafka-override` with a backported implementation of the `Time` class. This will be removed when we migrate to Storm 1.1. Normally, I would hold off this change until we take the effort to migrate to 1.1 properly, but the combination of empirical evidence (I saw it go an order of magnitude slower in 1.0.1 vs 1.1) and wide ranging change for Storm 1.1 makes me err on the side of getting this in faster. It should help make every topology faster, but is in particular absolutely required to get sensible throughput in the pcap topology. ## Pull Request Checklist Thank you for submitting a contribution to Apache Metron. Please refer to our [Development Guidelines](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=61332235) for the complete guide to follow for contributions. Please refer also to our [Build Verification Guidelines](https://cwiki.apache.org/confluence/display/METRON/Verifying+Builds?show-miniview) for complete smoke testing guides. In order to streamline the review of the contribution we ask you follow these guidelines and ask you to double check the following: ### For all changes: - [x] Is there a JIRA ticket associated with this PR? If not one needs to be created at [Metron Jira](https://issues.apache.org/jira/browse/METRON/?selectedTab=com.atlassian.jira.jira-projects-plugin:summary-panel). - [x] Does your PR title start with METRON- where is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character. - [x] Has your PR been rebased against the latest commit within the target branch (typically master)? ### For code changes: - [x] Have you included steps to reproduce the behavior or problem that is being changed or addressed? - [x] Have you included steps or a guide to how the change may be verified and tested manually? - [x] Have you ensured that the full suite of tests and checks have been executed in the root incubating-metron folder via: ``` mvn -q clean integration-test install && build_utils/verify_licenses.sh ``` - [x] Have you written or updated unit tests and or integration tests to verify your changes? - [x] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? - [ ] Have you verified the basic functionality of the build by building and running locally with Vagrant full-dev environment or the equivalent? ### For documentation related changes: - [x] Have you ensured that format looks appropriate for the output in which it is rendered by building and verifying the site-book? If not then run the following commands and the verify changes via `site-book/target/site/index.html`: ``` cd site-book mvn site ``` Note: Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible. It is also recommended that [travis-ci](https://travis-ci.org) is set up for your personal repository such that your branches are built there before submitting a pull request. You can merge this pull request into a Git repository by running: $ git pull https://github.com/cestella/incubator-metron storm_kafka_spout_11 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/metron/pull/584.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #584 commit a4cdfdcaaf15c8c60749a103d38b55e77703c2d5 Author: cstellaDate: 2017-05-12T15:59:18Z Upgrading spout to 1.1 for performance. commit 7ef320d3d80324d2322943a7b95354f0ef6c3852 Author: cstella Date: 2017-05-12T16:09:57Z Travis drama. commit 284c6452d08bc050fe8b1c66e5c85e4a95804cad Author: cstella Date: 2017-05-12T18:09:05Z Update commit 4246c36d7fdb94505d789c359c9115728f856e15 Author: cstella Date: 2017-05-12T18:48:21Z Remove ominous warning. commit 3387d420c3c348ea369d65c6404493483226f105 Author: cstella
[GitHub] metron pull request #584: METRON-950: Migrate storm-kafka-client to 1.1
Github user cestella closed the pull request at: https://github.com/apache/metron/pull/584 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #584: METRON-950: Migrate storm-kafka-client to 1.1
Github user cestella commented on the issue: https://github.com/apache/metron/pull/584 @ctramnitz While we don't really test on 1.1 yet, this shouldn't have an impact on the topologies running there. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
Re: [DISCUSS] Enrichment Split/Join issues
I would like to see us just migrate wholly to Stellar enrichments and remove the separate HBase and Geo enrichment bolts from the Enrichment topology. Stellar provides a user with much greater flexibility than the existing HBase and Geo enrichment bolts. A side effect of this would be to greatly simplify the Enrichment topology. I don't think we would not need the split/join pattern if we did this. No? On Tue, May 16, 2017 at 11:54 AM, Casey Stellawrote: > The problem is that an enrichment type won't necessarily have a fixed > performance characteristic. Take stellar enrichments, for instance. Doing > a HBase call for one sensor vs doing simple string munging will have vastly > differing performance. Both of them are functioning within the stellar > enrichment bolt. Also, some enrichments may call for multiple calls to > HBase. Parallelizing those, would make some sense, I think. > > I do take your point, though, that it's not as though it's strictly serial, > it's just that the unit of parallelism is the message, rather than the > enrichment per message. > > On Tue, May 16, 2017 at 11:47 AM, Christian Tramnitz > wrote: > > > I’m glad you bring this up. This is a huge architectural difference from > > the original OpenSOC topology and one that we have been warned to take > back > > then. > > To be perfectly honest, I don’t see the big perfomance improvement from > > parallel processing. If a specific enrichment is a little more i/o > > dependent than the other you can tweak parallelism to address this. Also > > there can be dependencies that make parallel enrichment virtually > > impossible or at least less efficient (i.e. first labeling, and > > “completing” a message and then dependent of label and completeness do > > different other enrichments). > > > > So you have a +1 from me for serial rather than parallel enrichment. > > > > > > BR, > >Christian > > > > On 16.05.17, 16:58, "Casey Stella" wrote: > > > > Hi All, > > > > Last week, I encountered some weirdness in the Enrichment topology. > > Doing > > some somewhat high-latency enrichment work, I noticed that at some > > point, > > data stopped flowing through the enrichment topology. I tracked down > > the > > problem to the join bolt. For those who aren't aware, we do a > > split/join > > pattern so that enrichments can be done in parallel. It works as > > follows: > > > >- A split bolt sends the appropriate subset of the message to each > >enrichment bolt as well as the whole message to the join bolt > >- The join bolt will receive each of the pieces of the message and > > then, > >when fully joined, it will send the message on. > > > > > > What is happening under load or high velocity, however, is that the > > cache > > is evicting the partially joined message before it can be fully > joined > > due > > to the volume of traffic. This is obviously not ideal. As such, it > is > > clear that adjusting the size of the cache and the characteristics of > > eviction is likely a good idea and a necessary part to tuning > > enrichments. > > The cache size is sensitive to: > > > >- The latency of the *slowest* enrichment > >- The number of tuples in flight at once > > > > As such, the knobs you have to tune are either the parallelism of the > > join > > bolt or the size of the cache. > > > > As it stands, I see a couple of things wrong here that we can correct > > with > > minimal issue: > > > >- We have no message of warning indicating that this is happening > >- Changing cache sizes means changing flux. We should promote > this > > to > >the properties file. > >- We should document the knobs mentioned above clearly in the > > enrichment > >topology README > > > > Those small changes, I think, are table stakes, but what I wanted to > > discuss more in depth is the lingering questions: > > > >- Is this an architectural pattern that we can use as-is? > > - Should we consider a persistent cache a la HBase or Apache > > Ignite > > as a pluggable component to Metron? > > - Should we consider taking the performance hit and doing the > > enrichments serially? > >- When an eviction happens, what should we do? > > - Fail the tuple, thereby making congestion worse > > - Pass through the partially enriched results, thereby making > > enrichments "best effort" > > > > Anyway, I wanted to talk this through and inform of some of the > things > > I'm > > seeing. > > > > Sorry for the novel. ;) > > > > Casey > > > > > > >
[GitHub] metron issue #531: METRON-854 create dhcp dump parser
Github user simonellistonball commented on the issue: https://github.com/apache/metron/pull/531 I'd love to see your bro PR expand for this @JonZeolla DHCP is a pretty key source, and Bro is a great way to extract it from taps. Let me know if there is anything I can do to help. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #585: METRON-936: Fixes to pcap for performance and test...
Github user cestella commented on a diff in the pull request: https://github.com/apache/metron/pull/585#discussion_r116791055 --- Diff: metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/spout/pcap/deserializer/KeyValueDeserializer.java --- @@ -28,6 +26,17 @@ public abstract class KeyValueDeserializer implements Serializable { protected TimestampConverter converter; + public static class Result { --- End diff -- Well, to be fair, it's `KeyValueDeserializer.Result` since it's a static inner class, not just `Result`. I'm inclined to think the naming is fine given the parent class, but I'm open to being convinced otherwise. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
Re: [DISCUSS] Enrichment Split/Join issues
Nick, I’d tend to agree with you there. How about: If an enrichment fails / effectively times out, the join bolt emits the message before cache eviction (as Nick’s point 2), but also adds a field stub to indicate failed enrichment. This is then an indicator to an operator or investigator as well that something is missing, and could drive things like replay of the message to retrospectively enrich when things have calmed down. Simon > On 16 May 2017, at 17:25, Nick Allenwrote: > > Ah, yes. Makes sense and I can see the value in the parallelism that the > split/join provides. Personally, I would like to see the code do the > following. > > (1) Scream and shout when something in the cache expires. We have to make > sure that it is blatantly obvious to a user what happened. We also need to > make it blatantly obvious to the user what knobs they can turn to correct > the problem. > > (2) Enrichments should be treated as best-effort. When the cache expires, > it should pass on the message without the enrichments that have not > completed. If I am relying on an external system for an enrichment, I > don't want an external system outage to fail all of my telemetry. > > > > > > On Tue, May 16, 2017 at 12:05 PM, Casey Stella wrote: > >> We still do use split/join even within stellar enrichments. Take for >> instance the following enrichment: >> { >> "enrichment" : { >>"fieldMap" : { >> "stellar" : { >> "config" : { >> "parallel-task-1" : { >> "my_field" : "PROFILE_GET()" >> }, >> "parallel-task-2" : { >> "my_field2" : "PROFILE_GET()" >> } >> } >> } >>} >> } >> >> Messages will get split between two tasks of the Stellar enrichment bolt >> and the stellar statements in "parallel-task-1" will be executed in >> parallel to those in "parallel-task-2". This is to enable people to >> separate computationally intensive or otherwise high latency tasks that are >> independent across nodes in the cluster. >> >> I will agree wholeheartedly, though, that my personal desire would be to >> have just stellar enrichments, though. You can do every one of the other >> enrichments in Stellar and it would greatly simplify that config above. >> >> >> >> On Tue, May 16, 2017 at 11:59 AM, Nick Allen wrote: >> >>> I would like to see us just migrate wholly to Stellar enrichments and >>> remove the separate HBase and Geo enrichment bolts from the Enrichment >>> topology. Stellar provides a user with much greater flexibility than the >>> existing HBase and Geo enrichment bolts. >>> >>> A side effect of this would be to greatly simplify the Enrichment >>> topology. I don't think we would not need the split/join pattern if we >> did >>> this. No? >>> >>> On Tue, May 16, 2017 at 11:54 AM, Casey Stella >> wrote: >>> The problem is that an enrichment type won't necessarily have a fixed performance characteristic. Take stellar enrichments, for instance. >>> Doing a HBase call for one sensor vs doing simple string munging will have >>> vastly differing performance. Both of them are functioning within the stellar enrichment bolt. Also, some enrichments may call for multiple calls to HBase. Parallelizing those, would make some sense, I think. I do take your point, though, that it's not as though it's strictly >>> serial, it's just that the unit of parallelism is the message, rather than the enrichment per message. On Tue, May 16, 2017 at 11:47 AM, Christian Tramnitz < >> tramn...@trasec.de wrote: > I’m glad you bring this up. This is a huge architectural difference >>> from > the original OpenSOC topology and one that we have been warned to >> take back > then. > To be perfectly honest, I don’t see the big perfomance improvement >> from > parallel processing. If a specific enrichment is a little more i/o > dependent than the other you can tweak parallelism to address this. >>> Also > there can be dependencies that make parallel enrichment virtually > impossible or at least less efficient (i.e. first labeling, and > “completing” a message and then dependent of label and completeness >> do > different other enrichments). > > So you have a +1 from me for serial rather than parallel enrichment. > > > BR, > Christian > > On 16.05.17, 16:58, "Casey Stella" wrote: > >Hi All, > >Last week, I encountered some weirdness in the Enrichment >> topology. > Doing >some somewhat high-latency enrichment work, I noticed that at >> some > point, >data stopped flowing through the enrichment topology. I tracked >>> down > the >problem to the join bolt. For those who aren't aware, we
[GitHub] metron issue #589: METRON-955: Make the default sync policy for HDFS Writer ...
Github user justinleet commented on the issue: https://github.com/apache/metron/pull/589 I'm +1 by inspection, thanks for the contribution. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #585: METRON-936: Fixes to pcap for performance and test...
Github user justinleet commented on a diff in the pull request: https://github.com/apache/metron/pull/585#discussion_r116795288 --- Diff: metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/spout/pcap/deserializer/KeyValueDeserializer.java --- @@ -28,6 +26,17 @@ public abstract class KeyValueDeserializer implements Serializable { protected TimestampConverter converter; + public static class Result { --- End diff -- Good call, I'm fine with it given the context. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
Re: [DISCUSS] Enrichment Split/Join issues
The field stub also gives something that can potentially be used in the error dashboard (or similar) to graph, allowing failed enrichments to "shout" louder to the end user. Jon On Tue, May 16, 2017 at 12:34 PM Nick Allenwrote: > > but also adds a field stub to indicate failed enrichment. This is then an > indicator to an operator or investigator as well that something is missing, > and could drive things like replay of the message to retrospectively enrich > when things have calmed down. > > Yes, I like the idea of a "field stub". You need some way to distinguish > "did I configure this wrong" versus "something bad happened outside of my > control". > > > > On Tue, May 16, 2017 at 12:27 PM, Simon Elliston Ball < > si...@simonellistonball.com> wrote: > > > Nick, I’d tend to agree with you there. > > > > How about: > > If an enrichment fails / effectively times out, the join bolt emits the > > message before cache eviction (as Nick’s point 2), but also adds a field > > stub to indicate failed enrichment. This is then an indicator to an > > operator or investigator as well that something is missing, and could > drive > > things like replay of the message to retrospectively enrich when things > > have calmed down. > > > > Simon > > > > > On 16 May 2017, at 17:25, Nick Allen wrote: > > > > > > Ah, yes. Makes sense and I can see the value in the parallelism that > the > > > split/join provides. Personally, I would like to see the code do the > > > following. > > > > > > (1) Scream and shout when something in the cache expires. We have to > > make > > > sure that it is blatantly obvious to a user what happened. We also > need > > to > > > make it blatantly obvious to the user what knobs they can turn to > correct > > > the problem. > > > > > > (2) Enrichments should be treated as best-effort. When the cache > > expires, > > > it should pass on the message without the enrichments that have not > > > completed. If I am relying on an external system for an enrichment, I > > > don't want an external system outage to fail all of my telemetry. > > > > > > > > > > > > > > > > > > On Tue, May 16, 2017 at 12:05 PM, Casey Stella > > wrote: > > > > > >> We still do use split/join even within stellar enrichments. Take for > > >> instance the following enrichment: > > >> { > > >> "enrichment" : { > > >>"fieldMap" : { > > >> "stellar" : { > > >> "config" : { > > >> "parallel-task-1" : { > > >> "my_field" : "PROFILE_GET()" > > >> }, > > >> "parallel-task-2" : { > > >> "my_field2" : "PROFILE_GET()" > > >> } > > >> } > > >> } > > >>} > > >> } > > >> > > >> Messages will get split between two tasks of the Stellar enrichment > bolt > > >> and the stellar statements in "parallel-task-1" will be executed in > > >> parallel to those in "parallel-task-2". This is to enable people to > > >> separate computationally intensive or otherwise high latency tasks > that > > are > > >> independent across nodes in the cluster. > > >> > > >> I will agree wholeheartedly, though, that my personal desire would be > to > > >> have just stellar enrichments, though. You can do every one of the > > other > > >> enrichments in Stellar and it would greatly simplify that config > above. > > >> > > >> > > >> > > >> On Tue, May 16, 2017 at 11:59 AM, Nick Allen > > wrote: > > >> > > >>> I would like to see us just migrate wholly to Stellar enrichments and > > >>> remove the separate HBase and Geo enrichment bolts from the > Enrichment > > >>> topology. Stellar provides a user with much greater flexibility than > > the > > >>> existing HBase and Geo enrichment bolts. > > >>> > > >>> A side effect of this would be to greatly simplify the Enrichment > > >>> topology. I don't think we would not need the split/join pattern if > we > > >> did > > >>> this. No? > > >>> > > >>> On Tue, May 16, 2017 at 11:54 AM, Casey Stella > > >> wrote: > > >>> > > The problem is that an enrichment type won't necessarily have a > fixed > > performance characteristic. Take stellar enrichments, for instance. > > >>> Doing > > a HBase call for one sensor vs doing simple string munging will have > > >>> vastly > > differing performance. Both of them are functioning within the > > stellar > > enrichment bolt. Also, some enrichments may call for multiple calls > > to > > HBase. Parallelizing those, would make some sense, I think. > > > > I do take your point, though, that it's not as though it's strictly > > >>> serial, > > it's just that the unit of parallelism is the message, rather than > the > > enrichment per message. > > > > On Tue, May 16, 2017 at 11:47 AM, Christian Tramnitz < > > >> tramn...@trasec.de > > > > wrote: > > > > > I’m glad you bring this up.
[GitHub] metron issue #585: METRON-936: Fixes to pcap for performance and testing
Github user mmiklavc commented on the issue: https://github.com/apache/metron/pull/585 I found some additional issues with error handling in the HDFSWriterCallback. So I fixed this to throw an IllegalArgumentException when the key is null, but that revealed further problems in our test infrastructure. PCAPTopologyIntegrationTest seems to be relying on data that does not provide a key. Was this by design? I get the following exception thrown, which is the one that I added as a null check on key: ``` Running org.apache.metron.pcap.integration.PcapTopologyIntegrationTest Formatting using clusterid: testClusterID 2017-05-16 11:05:39 ERROR util:0 - Async loop died! java.lang.IllegalArgumentException: Expected a key but none provided at org.apache.metron.spout.pcap.HDFSWriterCallback.apply(HDFSWriterCallback.java:121) at org.apache.storm.kafka.CallbackCollector.emit(CallbackCollector.java:59) at org.apache.storm.kafka.spout.KafkaSpoutStream.emit(KafkaSpoutStream.java:79) at org.apache.storm.kafka.spout.KafkaSpoutStreamsNamedTopics.emit(KafkaSpoutStreamsNamedTopics.java:101) at org.apache.storm.kafka.spout.KafkaSpout.emitTupleIfNotEmitted(KafkaSpout.java:280) at org.apache.storm.kafka.spout.KafkaSpout.emit(KafkaSpout.java:265) at org.apache.storm.kafka.spout.KafkaSpout.nextTuple(KafkaSpout.java:212) at org.apache.storm.daemon.executor$fn__6503$fn__6518$fn__6549.invoke(executor.clj:651) at org.apache.storm.util$async_loop$fn__554.invoke(util.clj:484) at clojure.lang.AFn.run(AFn.java:22) at java.lang.Thread.run(Thread.java:745) 2017-05-16 11:05:39 ERROR executor:0 - java.lang.IllegalArgumentException: Expected a key but none provided at org.apache.metron.spout.pcap.HDFSWriterCallback.apply(HDFSWriterCallback.java:121) at org.apache.storm.kafka.CallbackCollector.emit(CallbackCollector.java:59) at org.apache.storm.kafka.spout.KafkaSpoutStream.emit(KafkaSpoutStream.java:79) at org.apache.storm.kafka.spout.KafkaSpoutStreamsNamedTopics.emit(KafkaSpoutStreamsNamedTopics.java:101) at org.apache.storm.kafka.spout.KafkaSpout.emitTupleIfNotEmitted(KafkaSpout.java:280) at org.apache.storm.kafka.spout.KafkaSpout.emit(KafkaSpout.java:265) at org.apache.storm.kafka.spout.KafkaSpout.nextTuple(KafkaSpout.java:212) at org.apache.storm.daemon.executor$fn__6503$fn__6518$fn__6549.invoke(executor.clj:651) at org.apache.storm.util$async_loop$fn__554.invoke(util.clj:484) at clojure.lang.AFn.run(AFn.java:22) at java.lang.Thread.run(Thread.java:745) 2017-05-16 11:05:39 ERROR util:0 - Halting process: ("Worker died") java.lang.RuntimeException: ("Worker died") at org.apache.storm.util$exit_process_BANG_.doInvoke(util.clj:341) at clojure.lang.RestFn.invoke(RestFn.java:423) at org.apache.storm.daemon.worker$fn__7172$fn__7173.invoke(worker.clj:761) at org.apache.storm.daemon.executor$mk_executor_data$fn__6388$fn__6389.invoke(executor.clj:275) at org.apache.storm.util$async_loop$fn__554.invoke(util.clj:494) at clojure.lang.AFn.run(AFn.java:22) at java.lang.Thread.run(Thread.java:745) ``` When I attempt to view the PCAP file with the PcapInspector in the IDE, I get this exception. ``` Exception in thread "main" java.io.IOException: wrong key class: org.apache.hadoop.io.LongWritable is not class org.apache.hadoop.io.IntWritable at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:2254) at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:2306) at org.apache.metron.utils.PcapInspector.main(PcapInspector.java:142) Process finished with exit code 1 ``` --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #585: METRON-936: Fixes to pcap for performance and test...
Github user cestella commented on a diff in the pull request: https://github.com/apache/metron/pull/585#discussion_r116791165 --- Diff: metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/spout/pcap/PartitionHDFSWriter.java --- @@ -183,14 +219,14 @@ private void turnoverIfNecessary(long ts, boolean force) throws IOException { } } - writer = SequenceFile.createWriter(new Configuration() + writer = SequenceFile.createWriter(this.fsConfig , SequenceFile.Writer.keyClass(LongWritable.class) , SequenceFile.Writer.valueClass(BytesWritable.class) , SequenceFile.Writer.stream(outputStream) , SequenceFile.Writer.compression(SequenceFile.CompressionType.NONE) ); //reset state - LOG.info("Turning over and writing to " + path); + LOG.info(String.format("Turning over and writing to %s: [duration=%s NS, force=%s, initial=%s, overDuration=%s, tooManyPackets=%s]", path, duration, force, initial, overDuration, tooManyPackets)); --- End diff -- +1 to that --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
Re: [DISCUSS] Enrichment Split/Join issues
Ah, yes. Makes sense and I can see the value in the parallelism that the split/join provides. Personally, I would like to see the code do the following. (1) Scream and shout when something in the cache expires. We have to make sure that it is blatantly obvious to a user what happened. We also need to make it blatantly obvious to the user what knobs they can turn to correct the problem. (2) Enrichments should be treated as best-effort. When the cache expires, it should pass on the message without the enrichments that have not completed. If I am relying on an external system for an enrichment, I don't want an external system outage to fail all of my telemetry. On Tue, May 16, 2017 at 12:05 PM, Casey Stellawrote: > We still do use split/join even within stellar enrichments. Take for > instance the following enrichment: > { > "enrichment" : { > "fieldMap" : { > "stellar" : { > "config" : { > "parallel-task-1" : { > "my_field" : "PROFILE_GET()" > }, > "parallel-task-2" : { > "my_field2" : "PROFILE_GET()" > } > } > } > } > } > > Messages will get split between two tasks of the Stellar enrichment bolt > and the stellar statements in "parallel-task-1" will be executed in > parallel to those in "parallel-task-2". This is to enable people to > separate computationally intensive or otherwise high latency tasks that are > independent across nodes in the cluster. > > I will agree wholeheartedly, though, that my personal desire would be to > have just stellar enrichments, though. You can do every one of the other > enrichments in Stellar and it would greatly simplify that config above. > > > > On Tue, May 16, 2017 at 11:59 AM, Nick Allen wrote: > > > I would like to see us just migrate wholly to Stellar enrichments and > > remove the separate HBase and Geo enrichment bolts from the Enrichment > > topology. Stellar provides a user with much greater flexibility than the > > existing HBase and Geo enrichment bolts. > > > > A side effect of this would be to greatly simplify the Enrichment > > topology. I don't think we would not need the split/join pattern if we > did > > this. No? > > > > On Tue, May 16, 2017 at 11:54 AM, Casey Stella > wrote: > > > > > The problem is that an enrichment type won't necessarily have a fixed > > > performance characteristic. Take stellar enrichments, for instance. > > Doing > > > a HBase call for one sensor vs doing simple string munging will have > > vastly > > > differing performance. Both of them are functioning within the stellar > > > enrichment bolt. Also, some enrichments may call for multiple calls to > > > HBase. Parallelizing those, would make some sense, I think. > > > > > > I do take your point, though, that it's not as though it's strictly > > serial, > > > it's just that the unit of parallelism is the message, rather than the > > > enrichment per message. > > > > > > On Tue, May 16, 2017 at 11:47 AM, Christian Tramnitz < > tramn...@trasec.de > > > > > > wrote: > > > > > > > I’m glad you bring this up. This is a huge architectural difference > > from > > > > the original OpenSOC topology and one that we have been warned to > take > > > back > > > > then. > > > > To be perfectly honest, I don’t see the big perfomance improvement > from > > > > parallel processing. If a specific enrichment is a little more i/o > > > > dependent than the other you can tweak parallelism to address this. > > Also > > > > there can be dependencies that make parallel enrichment virtually > > > > impossible or at least less efficient (i.e. first labeling, and > > > > “completing” a message and then dependent of label and completeness > do > > > > different other enrichments). > > > > > > > > So you have a +1 from me for serial rather than parallel enrichment. > > > > > > > > > > > > BR, > > > >Christian > > > > > > > > On 16.05.17, 16:58, "Casey Stella" wrote: > > > > > > > > Hi All, > > > > > > > > Last week, I encountered some weirdness in the Enrichment > topology. > > > > Doing > > > > some somewhat high-latency enrichment work, I noticed that at > some > > > > point, > > > > data stopped flowing through the enrichment topology. I tracked > > down > > > > the > > > > problem to the join bolt. For those who aren't aware, we do a > > > > split/join > > > > pattern so that enrichments can be done in parallel. It works as > > > > follows: > > > > > > > >- A split bolt sends the appropriate subset of the message to > > each > > > >enrichment bolt as well as the whole message to the join bolt > > > >- The join bolt will receive each of the pieces of the message > > and > > > > then, > > > >when fully joined, it will send the message on. > > > > > > > > > > > > What is happening under load or high
Re: [DISCUSS] Enrichment Split/Join issues
If we are timing out things from the cache, we have that latency already On May 16, 2017 at 12:09:32, Casey Stella (ceste...@gmail.com) wrote: We could definitely parallelize within the bolt, but you're right, it does break the storm model. I also like making things other people's problems (it's called working "smart" not "hard", right? not laziness, surely. ;), but yeah, using windowing for this seems like it might introduce some artificial latency. It's also not going to eliminate the problem, but rather just make the knob to tweak things have a different characteristic. Whereas before we have knobs around how many messages, now it's a knob around how long an enrichment is going to take maximally (which, I think is more natural, honestly). On Tue, May 16, 2017 at 12:05 PM, Simon Elliston Ball < si...@simonellistonball.com> wrote: > Would you then parallelise within Stellar to handle things like multiple > lookups? This feels like it would be breaking the storm model somewhat, and > could lead to bad things with threads for example. Or would you think of > doing something like the grouping Stellar uses today to parallelise across > something like a pool of Stellar bolts and join? > > I like the idea of Otto’s solution (making it someone else's problem, > storm’s specifically :) ) but that also assumes we insert the artificial > latency of a time windowed join. If we’re going down that route, we might > as well just use spark and run everything on yarn. At that point though we > lose a lot of the benefits of low latency for time to detection, and > real-time enrichment in things like the streaming enrichment writer. > > Simon > > > On 16 May 2017, at 16:59, Nick Allenwrote: > > > > I would like to see us just migrate wholly to Stellar enrichments and > > remove the separate HBase and Geo enrichment bolts from the Enrichment > > topology. Stellar provides a user with much greater flexibility than the > > existing HBase and Geo enrichment bolts. > > > > A side effect of this would be to greatly simplify the Enrichment > > topology. I don't think we would not need the split/join pattern if we > did > > this. No? > > > > On Tue, May 16, 2017 at 11:54 AM, Casey Stella > wrote: > > > >> The problem is that an enrichment type won't necessarily have a fixed > >> performance characteristic. Take stellar enrichments, for instance. > Doing > >> a HBase call for one sensor vs doing simple string munging will have > vastly > >> differing performance. Both of them are functioning within the stellar > >> enrichment bolt. Also, some enrichments may call for multiple calls to > >> HBase. Parallelizing those, would make some sense, I think. > >> > >> I do take your point, though, that it's not as though it's strictly > serial, > >> it's just that the unit of parallelism is the message, rather than the > >> enrichment per message. > >> > >> On Tue, May 16, 2017 at 11:47 AM, Christian Tramnitz < > tramn...@trasec.de> > >> wrote: > >> > >>> I’m glad you bring this up. This is a huge architectural difference > from > >>> the original OpenSOC topology and one that we have been warned to take > >> back > >>> then. > >>> To be perfectly honest, I don’t see the big perfomance improvement from > >>> parallel processing. If a specific enrichment is a little more i/o > >>> dependent than the other you can tweak parallelism to address this. > Also > >>> there can be dependencies that make parallel enrichment virtually > >>> impossible or at least less efficient (i.e. first labeling, and > >>> “completing” a message and then dependent of label and completeness do > >>> different other enrichments). > >>> > >>> So you have a +1 from me for serial rather than parallel enrichment. > >>> > >>> > >>> BR, > >>> Christian > >>> > >>> On 16.05.17, 16:58, "Casey Stella" wrote: > >>> > >>> Hi All, > >>> > >>> Last week, I encountered some weirdness in the Enrichment topology. > >>> Doing > >>> some somewhat high-latency enrichment work, I noticed that at some > >>> point, > >>> data stopped flowing through the enrichment topology. I tracked > down > >>> the > >>> problem to the join bolt. For those who aren't aware, we do a > >>> split/join > >>> pattern so that enrichments can be done in parallel. It works as > >>> follows: > >>> > >>> - A split bolt sends the appropriate subset of the message to > each > >>> enrichment bolt as well as the whole message to the join bolt > >>> - The join bolt will receive each of the pieces of the message > and > >>> then, > >>> when fully joined, it will send the message on. > >>> > >>> > >>> What is happening under load or high velocity, however, is that the > >>> cache > >>> is evicting the partially joined message before it can be fully > >> joined > >>> due > >>> to the volume of traffic. This is obviously not ideal. As such, it > >> is > >>> clear that adjusting the size of the cache and the characteristics > of > >>> eviction is likely a
Re: [DISCUSS] Enrichment Split/Join issues
> but also adds a field stub to indicate failed enrichment. This is then an indicator to an operator or investigator as well that something is missing, and could drive things like replay of the message to retrospectively enrich when things have calmed down. Yes, I like the idea of a "field stub". You need some way to distinguish "did I configure this wrong" versus "something bad happened outside of my control". On Tue, May 16, 2017 at 12:27 PM, Simon Elliston Ball < si...@simonellistonball.com> wrote: > Nick, I’d tend to agree with you there. > > How about: > If an enrichment fails / effectively times out, the join bolt emits the > message before cache eviction (as Nick’s point 2), but also adds a field > stub to indicate failed enrichment. This is then an indicator to an > operator or investigator as well that something is missing, and could drive > things like replay of the message to retrospectively enrich when things > have calmed down. > > Simon > > > On 16 May 2017, at 17:25, Nick Allenwrote: > > > > Ah, yes. Makes sense and I can see the value in the parallelism that the > > split/join provides. Personally, I would like to see the code do the > > following. > > > > (1) Scream and shout when something in the cache expires. We have to > make > > sure that it is blatantly obvious to a user what happened. We also need > to > > make it blatantly obvious to the user what knobs they can turn to correct > > the problem. > > > > (2) Enrichments should be treated as best-effort. When the cache > expires, > > it should pass on the message without the enrichments that have not > > completed. If I am relying on an external system for an enrichment, I > > don't want an external system outage to fail all of my telemetry. > > > > > > > > > > > > On Tue, May 16, 2017 at 12:05 PM, Casey Stella > wrote: > > > >> We still do use split/join even within stellar enrichments. Take for > >> instance the following enrichment: > >> { > >> "enrichment" : { > >>"fieldMap" : { > >> "stellar" : { > >> "config" : { > >> "parallel-task-1" : { > >> "my_field" : "PROFILE_GET()" > >> }, > >> "parallel-task-2" : { > >> "my_field2" : "PROFILE_GET()" > >> } > >> } > >> } > >>} > >> } > >> > >> Messages will get split between two tasks of the Stellar enrichment bolt > >> and the stellar statements in "parallel-task-1" will be executed in > >> parallel to those in "parallel-task-2". This is to enable people to > >> separate computationally intensive or otherwise high latency tasks that > are > >> independent across nodes in the cluster. > >> > >> I will agree wholeheartedly, though, that my personal desire would be to > >> have just stellar enrichments, though. You can do every one of the > other > >> enrichments in Stellar and it would greatly simplify that config above. > >> > >> > >> > >> On Tue, May 16, 2017 at 11:59 AM, Nick Allen > wrote: > >> > >>> I would like to see us just migrate wholly to Stellar enrichments and > >>> remove the separate HBase and Geo enrichment bolts from the Enrichment > >>> topology. Stellar provides a user with much greater flexibility than > the > >>> existing HBase and Geo enrichment bolts. > >>> > >>> A side effect of this would be to greatly simplify the Enrichment > >>> topology. I don't think we would not need the split/join pattern if we > >> did > >>> this. No? > >>> > >>> On Tue, May 16, 2017 at 11:54 AM, Casey Stella > >> wrote: > >>> > The problem is that an enrichment type won't necessarily have a fixed > performance characteristic. Take stellar enrichments, for instance. > >>> Doing > a HBase call for one sensor vs doing simple string munging will have > >>> vastly > differing performance. Both of them are functioning within the > stellar > enrichment bolt. Also, some enrichments may call for multiple calls > to > HBase. Parallelizing those, would make some sense, I think. > > I do take your point, though, that it's not as though it's strictly > >>> serial, > it's just that the unit of parallelism is the message, rather than the > enrichment per message. > > On Tue, May 16, 2017 at 11:47 AM, Christian Tramnitz < > >> tramn...@trasec.de > > wrote: > > > I’m glad you bring this up. This is a huge architectural difference > >>> from > > the original OpenSOC topology and one that we have been warned to > >> take > back > > then. > > To be perfectly honest, I don’t see the big perfomance improvement > >> from > > parallel processing. If a specific enrichment is a little more i/o > > dependent than the other you can tweak parallelism to address this. > >>> Also > > there can be dependencies that make parallel enrichment virtually > > impossible or at least less
[GitHub] metron issue #567: METRON-891: Changed Kafka API to Create a KafkaConsumer P...
Github user justinleet commented on the issue: https://github.com/apache/metron/pull/567 I'm set, +1 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #588: METRON-954: Create ability to change output topic of pars...
Github user merrimanr commented on the issue: https://github.com/apache/metron/pull/588 Can we add this option to the README? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #588: METRON-954: Create ability to change output topic ...
Github user asfgit closed the pull request at: https://github.com/apache/metron/pull/588 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #531: METRON-854 create dhcp dump parser
Github user basvdl commented on the issue: https://github.com/apache/metron/pull/531 @nickwallen sometimes we are not able to grep DNS events from the customer server. In these cases we use DHCPDump. I've to admit, Bro is new to me, but it looks promising. If this can fulfill our requirement, It would be a good replacement since it can also grep other protocols as well. @JonZeolla lookups are also not always an option for us. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
Re: [DISCUSS] Enrichment Split/Join issues
We still do use split/join even within stellar enrichments. Take for instance the following enrichment: { "enrichment" : { "fieldMap" : { "stellar" : { "config" : { "parallel-task-1" : { "my_field" : "PROFILE_GET()" }, "parallel-task-2" : { "my_field2" : "PROFILE_GET()" } } } } } Messages will get split between two tasks of the Stellar enrichment bolt and the stellar statements in "parallel-task-1" will be executed in parallel to those in "parallel-task-2". This is to enable people to separate computationally intensive or otherwise high latency tasks that are independent across nodes in the cluster. I will agree wholeheartedly, though, that my personal desire would be to have just stellar enrichments, though. You can do every one of the other enrichments in Stellar and it would greatly simplify that config above. On Tue, May 16, 2017 at 11:59 AM, Nick Allenwrote: > I would like to see us just migrate wholly to Stellar enrichments and > remove the separate HBase and Geo enrichment bolts from the Enrichment > topology. Stellar provides a user with much greater flexibility than the > existing HBase and Geo enrichment bolts. > > A side effect of this would be to greatly simplify the Enrichment > topology. I don't think we would not need the split/join pattern if we did > this. No? > > On Tue, May 16, 2017 at 11:54 AM, Casey Stella wrote: > > > The problem is that an enrichment type won't necessarily have a fixed > > performance characteristic. Take stellar enrichments, for instance. > Doing > > a HBase call for one sensor vs doing simple string munging will have > vastly > > differing performance. Both of them are functioning within the stellar > > enrichment bolt. Also, some enrichments may call for multiple calls to > > HBase. Parallelizing those, would make some sense, I think. > > > > I do take your point, though, that it's not as though it's strictly > serial, > > it's just that the unit of parallelism is the message, rather than the > > enrichment per message. > > > > On Tue, May 16, 2017 at 11:47 AM, Christian Tramnitz > > > wrote: > > > > > I’m glad you bring this up. This is a huge architectural difference > from > > > the original OpenSOC topology and one that we have been warned to take > > back > > > then. > > > To be perfectly honest, I don’t see the big perfomance improvement from > > > parallel processing. If a specific enrichment is a little more i/o > > > dependent than the other you can tweak parallelism to address this. > Also > > > there can be dependencies that make parallel enrichment virtually > > > impossible or at least less efficient (i.e. first labeling, and > > > “completing” a message and then dependent of label and completeness do > > > different other enrichments). > > > > > > So you have a +1 from me for serial rather than parallel enrichment. > > > > > > > > > BR, > > >Christian > > > > > > On 16.05.17, 16:58, "Casey Stella" wrote: > > > > > > Hi All, > > > > > > Last week, I encountered some weirdness in the Enrichment topology. > > > Doing > > > some somewhat high-latency enrichment work, I noticed that at some > > > point, > > > data stopped flowing through the enrichment topology. I tracked > down > > > the > > > problem to the join bolt. For those who aren't aware, we do a > > > split/join > > > pattern so that enrichments can be done in parallel. It works as > > > follows: > > > > > >- A split bolt sends the appropriate subset of the message to > each > > >enrichment bolt as well as the whole message to the join bolt > > >- The join bolt will receive each of the pieces of the message > and > > > then, > > >when fully joined, it will send the message on. > > > > > > > > > What is happening under load or high velocity, however, is that the > > > cache > > > is evicting the partially joined message before it can be fully > > joined > > > due > > > to the volume of traffic. This is obviously not ideal. As such, > it > > is > > > clear that adjusting the size of the cache and the characteristics > of > > > eviction is likely a good idea and a necessary part to tuning > > > enrichments. > > > The cache size is sensitive to: > > > > > >- The latency of the *slowest* enrichment > > >- The number of tuples in flight at once > > > > > > As such, the knobs you have to tune are either the parallelism of > the > > > join > > > bolt or the size of the cache. > > > > > > As it stands, I see a couple of things wrong here that we can > correct > > > with > > > minimal issue: > > > > > >- We have no message of warning indicating that this is > happening > > >- Changing cache sizes means changing flux. We should promote >
Re: [DISCUSS] Enrichment Split/Join issues
I am not sure that you can say we wouldn’t ‘need’ it. But we would not ‘have’ it rather. On May 16, 2017 at 11:59:42, Nick Allen (n...@nickallen.org) wrote: I would like to see us just migrate wholly to Stellar enrichments and remove the separate HBase and Geo enrichment bolts from the Enrichment topology. Stellar provides a user with much greater flexibility than the existing HBase and Geo enrichment bolts. A side effect of this would be to greatly simplify the Enrichment topology. I don't think we would not need the split/join pattern if we did this. No? On Tue, May 16, 2017 at 11:54 AM, Casey Stellawrote: > The problem is that an enrichment type won't necessarily have a fixed > performance characteristic. Take stellar enrichments, for instance. Doing > a HBase call for one sensor vs doing simple string munging will have vastly > differing performance. Both of them are functioning within the stellar > enrichment bolt. Also, some enrichments may call for multiple calls to > HBase. Parallelizing those, would make some sense, I think. > > I do take your point, though, that it's not as though it's strictly serial, > it's just that the unit of parallelism is the message, rather than the > enrichment per message. > > On Tue, May 16, 2017 at 11:47 AM, Christian Tramnitz > wrote: > > > I’m glad you bring this up. This is a huge architectural difference from > > the original OpenSOC topology and one that we have been warned to take > back > > then. > > To be perfectly honest, I don’t see the big perfomance improvement from > > parallel processing. If a specific enrichment is a little more i/o > > dependent than the other you can tweak parallelism to address this. Also > > there can be dependencies that make parallel enrichment virtually > > impossible or at least less efficient (i.e. first labeling, and > > “completing” a message and then dependent of label and completeness do > > different other enrichments). > > > > So you have a +1 from me for serial rather than parallel enrichment. > > > > > > BR, > > Christian > > > > On 16.05.17, 16:58, "Casey Stella" wrote: > > > > Hi All, > > > > Last week, I encountered some weirdness in the Enrichment topology. > > Doing > > some somewhat high-latency enrichment work, I noticed that at some > > point, > > data stopped flowing through the enrichment topology. I tracked down > > the > > problem to the join bolt. For those who aren't aware, we do a > > split/join > > pattern so that enrichments can be done in parallel. It works as > > follows: > > > > - A split bolt sends the appropriate subset of the message to each > > enrichment bolt as well as the whole message to the join bolt > > - The join bolt will receive each of the pieces of the message and > > then, > > when fully joined, it will send the message on. > > > > > > What is happening under load or high velocity, however, is that the > > cache > > is evicting the partially joined message before it can be fully > joined > > due > > to the volume of traffic. This is obviously not ideal. As such, it > is > > clear that adjusting the size of the cache and the characteristics of > > eviction is likely a good idea and a necessary part to tuning > > enrichments. > > The cache size is sensitive to: > > > > - The latency of the *slowest* enrichment > > - The number of tuples in flight at once > > > > As such, the knobs you have to tune are either the parallelism of the > > join > > bolt or the size of the cache. > > > > As it stands, I see a couple of things wrong here that we can correct > > with > > minimal issue: > > > > - We have no message of warning indicating that this is happening > > - Changing cache sizes means changing flux. We should promote > this > > to > > the properties file. > > - We should document the knobs mentioned above clearly in the > > enrichment > > topology README > > > > Those small changes, I think, are table stakes, but what I wanted to > > discuss more in depth is the lingering questions: > > > > - Is this an architectural pattern that we can use as-is? > > - Should we consider a persistent cache a la HBase or Apache > > Ignite > > as a pluggable component to Metron? > > - Should we consider taking the performance hit and doing the > > enrichments serially? > > - When an eviction happens, what should we do? > > - Fail the tuple, thereby making congestion worse > > - Pass through the partially enriched results, thereby making > > enrichments "best effort" > > > > Anyway, I wanted to talk this through and inform of some of the > things > > I'm > > seeing. > > > > Sorry for the novel. ;) > > > > Casey > > > > > > >
Re: [DISCUSS] Enrichment Split/Join issues
Would you then parallelise within Stellar to handle things like multiple lookups? This feels like it would be breaking the storm model somewhat, and could lead to bad things with threads for example. Or would you think of doing something like the grouping Stellar uses today to parallelise across something like a pool of Stellar bolts and join? I like the idea of Otto’s solution (making it someone else's problem, storm’s specifically :) ) but that also assumes we insert the artificial latency of a time windowed join. If we’re going down that route, we might as well just use spark and run everything on yarn. At that point though we lose a lot of the benefits of low latency for time to detection, and real-time enrichment in things like the streaming enrichment writer. Simon > On 16 May 2017, at 16:59, Nick Allenwrote: > > I would like to see us just migrate wholly to Stellar enrichments and > remove the separate HBase and Geo enrichment bolts from the Enrichment > topology. Stellar provides a user with much greater flexibility than the > existing HBase and Geo enrichment bolts. > > A side effect of this would be to greatly simplify the Enrichment > topology. I don't think we would not need the split/join pattern if we did > this. No? > > On Tue, May 16, 2017 at 11:54 AM, Casey Stella wrote: > >> The problem is that an enrichment type won't necessarily have a fixed >> performance characteristic. Take stellar enrichments, for instance. Doing >> a HBase call for one sensor vs doing simple string munging will have vastly >> differing performance. Both of them are functioning within the stellar >> enrichment bolt. Also, some enrichments may call for multiple calls to >> HBase. Parallelizing those, would make some sense, I think. >> >> I do take your point, though, that it's not as though it's strictly serial, >> it's just that the unit of parallelism is the message, rather than the >> enrichment per message. >> >> On Tue, May 16, 2017 at 11:47 AM, Christian Tramnitz >> wrote: >> >>> I’m glad you bring this up. This is a huge architectural difference from >>> the original OpenSOC topology and one that we have been warned to take >> back >>> then. >>> To be perfectly honest, I don’t see the big perfomance improvement from >>> parallel processing. If a specific enrichment is a little more i/o >>> dependent than the other you can tweak parallelism to address this. Also >>> there can be dependencies that make parallel enrichment virtually >>> impossible or at least less efficient (i.e. first labeling, and >>> “completing” a message and then dependent of label and completeness do >>> different other enrichments). >>> >>> So you have a +1 from me for serial rather than parallel enrichment. >>> >>> >>> BR, >>> Christian >>> >>> On 16.05.17, 16:58, "Casey Stella" wrote: >>> >>>Hi All, >>> >>>Last week, I encountered some weirdness in the Enrichment topology. >>> Doing >>>some somewhat high-latency enrichment work, I noticed that at some >>> point, >>>data stopped flowing through the enrichment topology. I tracked down >>> the >>>problem to the join bolt. For those who aren't aware, we do a >>> split/join >>>pattern so that enrichments can be done in parallel. It works as >>> follows: >>> >>> - A split bolt sends the appropriate subset of the message to each >>> enrichment bolt as well as the whole message to the join bolt >>> - The join bolt will receive each of the pieces of the message and >>> then, >>> when fully joined, it will send the message on. >>> >>> >>>What is happening under load or high velocity, however, is that the >>> cache >>>is evicting the partially joined message before it can be fully >> joined >>> due >>>to the volume of traffic. This is obviously not ideal. As such, it >> is >>>clear that adjusting the size of the cache and the characteristics of >>>eviction is likely a good idea and a necessary part to tuning >>> enrichments. >>>The cache size is sensitive to: >>> >>> - The latency of the *slowest* enrichment >>> - The number of tuples in flight at once >>> >>>As such, the knobs you have to tune are either the parallelism of the >>> join >>>bolt or the size of the cache. >>> >>>As it stands, I see a couple of things wrong here that we can correct >>> with >>>minimal issue: >>> >>> - We have no message of warning indicating that this is happening >>> - Changing cache sizes means changing flux. We should promote >> this >>> to >>> the properties file. >>> - We should document the knobs mentioned above clearly in the >>> enrichment >>> topology README >>> >>>Those small changes, I think, are table stakes, but what I wanted to >>>discuss more in depth is the lingering questions: >>> >>> - Is this an architectural pattern that we
[GitHub] metron issue #584: METRON-950: Migrate storm-kafka-client to 1.1
Github user ctramnitz commented on the issue: https://github.com/apache/metron/pull/584 Does this change have any impact on using Storm 1.1 (i.e. from HDP 2.6)? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
we currently have 31 PR’s that are not landed
https://github.com/apache/metron/pulls This seems a little large given that I *think* we have been at around 19 or so consistently.
[GitHub] metron issue #574: METRON-934: Component and task id are missing in the inde...
Github user cestella commented on the issue: https://github.com/apache/metron/pull/574 +1 by inspection --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #584: METRON-950: Migrate storm-kafka-client to 1.1
Github user mmiklavc commented on the issue: https://github.com/apache/metron/pull/584 +1. Data flowed through to ES as expected, and I was able to spin up pcap. Along with @justinleet having tested the other topologies, I'm happy with the results. Great work @cestella! --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #567: METRON-891: Changed Kafka API to Create a KafkaConsumer P...
Github user jjmeyer0 commented on the issue: https://github.com/apache/metron/pull/567 @merrimanr @justinleet Accidentally clicked close/comment. Sorry about that. Anyway, does this look good to you all now that the licenses are fixed? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #585: METRON-936: Fixes to pcap for performance and test...
Github user mmiklavc commented on a diff in the pull request: https://github.com/apache/metron/pull/585#discussion_r116848618 --- Diff: metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/spout/pcap/PartitionHDFSWriter.java --- @@ -183,14 +219,14 @@ private void turnoverIfNecessary(long ts, boolean force) throws IOException { } } - writer = SequenceFile.createWriter(new Configuration() + writer = SequenceFile.createWriter(this.fsConfig , SequenceFile.Writer.keyClass(LongWritable.class) , SequenceFile.Writer.valueClass(BytesWritable.class) , SequenceFile.Writer.stream(outputStream) , SequenceFile.Writer.compression(SequenceFile.CompressionType.NONE) ); //reset state - LOG.info("Turning over and writing to " + path); + LOG.info(String.format("Turning over and writing to %s: [duration=%s NS, force=%s, initial=%s, overDuration=%s, tooManyPackets=%s]", path, duration, force, initial, overDuration, tooManyPackets)); --- End diff -- That sounds great. I wasn't planning to switch the logging implementation from Log4j in this PR, but I can change the deps if you think we need it now. `org.apache.log4j.Category#info(java.lang.Object, java.lang.Throwable)` --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #585: METRON-936: Fixes to pcap for performance and testing
Github user cestella commented on the issue: https://github.com/apache/metron/pull/585 Regarding the test data, it's not a sequence file in the format suitable for reading in PcapInspector. Depending on the test case, we construct the appropriate kafka representation. The value is what is being used, modified to be suitable for the test case (e.g. with headers and no key or without headers and with a key) and fed into kafka. This is being done in the `readPcaps` method of the Integration Test. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #588: METRON-954: Create ability to change output topic of pars...
Github user justinleet commented on the issue: https://github.com/apache/metron/pull/588 I'm +1. Thanks for adding this. @ottobackwards You need to see anything on this PR? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #585: METRON-936: Fixes to pcap for performance and testing
Github user mmiklavc commented on the issue: https://github.com/apache/metron/pull/585 I see that. re: keys and methods for retrieving and saving them. I'll save refactoring and cleaning that up to a separate PR. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #585: METRON-936: Fixes to pcap for performance and testing
Github user cestella commented on the issue: https://github.com/apache/metron/pull/585 @mmiklavc it depends on which test case you're talking about. We have two modes of operation in the pcap topology and 2 test cases in the integration test and these are defined by the flux property `kafka.pcap.ts_scheme`. These modes define the deserialization logic used in the topology to convert kafka key/values to bytes suitable for writing to HDFS: * `FROM_PACKET`: which expects a fully-formed packet (with headers) and parses the packet and extracts the timestamp from the value. This is a legacy mode, which functioned with pycapa prior to rewriting. We should eventually deprecate this and remove it. This is associated with the `FromPacketDeserializer` * `FROM_KEY` : which expects raw data and a timestamp from the key. This is by far the dominant mode of operation and the one you will see in `pycapa` or `fastcapa`. This is associated with the `FromKeyDeserializer` It appears that you are doing the null check in the `HDFSWriterCallback`. I would recommend doing this null check in `FromKeyDeserializer` as a null key is not an illegal state for the `FromPacketDeserializer`. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #585: METRON-936: Fixes to pcap for performance and test...
Github user justinleet commented on a diff in the pull request: https://github.com/apache/metron/pull/585#discussion_r116823693 --- Diff: metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/spout/pcap/HDFSWriterCallback.java --- @@ -116,7 +117,11 @@ public HDFSWriterCallback withConfig(HDFSWriterConfig config) { public List apply(List tuple, EmitContext context) { byte[] key = (byte[]) tuple.get(0); byte[] value = (byte[]) tuple.get(1); -if(!config.getDeserializer().deserializeKeyValue(key, value, KeyValue.key.get(), KeyValue.value.get())) { --- End diff -- KeyValue as a whole is unused now, right? Can we just delete the class entirely at this point? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #531: METRON-854 create dhcp dump parser
Github user basvdl commented on the issue: https://github.com/apache/metron/pull/531 @nickwallen I agree that relying on a modified source is not ideal. However with bro I'm not sure if you have all the functionality people wish for. If i'm correctly informed by the docs, bro will give you the IP and MAC relation, which differs from DHCPDump which captures IP and Hostname relations. Giving context to an IP by adding the Hostname looks more promising... So I would still like to discuss the opportunities of getting the original DHCPDump log format into Metron via NiFi. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #531: METRON-854 create dhcp dump parser
Github user nickwallen commented on the issue: https://github.com/apache/metron/pull/531 > So I would still like to discuss the opportunities of getting the original DHCPDump log format into Metron via NiFi. Sure, I think that sounds like another reasonable approach. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #581: METRON-844: Install Metron Management UI with Ambari MPac...
Github user merrimanr commented on the issue: https://github.com/apache/metron/pull/581 The Node.js repository setup has been moved outside of the MPack. In full dev this is now automated through the ambari-common Ansible task, which also handles other Ambari setup tasks. I documented this in the metron-deployment/README.md by simply adding Node.js as a prerequisite to installing the MPack. Is that enough or would more detailed documentation be helpful? Are there other places this should be documented? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #531: METRON-854 create dhcp dump parser
Github user nickwallen commented on the issue: https://github.com/apache/metron/pull/531 > If i'm correctly informed by the docs, bro will give you the IP and MAC relation, which differs from DHCPDump which captures IP and Hostname relations. Giving context to an IP by adding the Hostname looks more promising... I am a little confused by what you are looking for though. But maybe I am just misunderstanding and need more coffee. The purpose of DHCP is to hand out an IP that typically gets associated with a MAC address. That's the pairing I would be interested in from DHCP. [Bro can help with this.](https://www.bro.org/sphinx/scripts/base/protocols/dhcp/main.bro.html) If you are looking for IP to hostname, then I would think you would be interested in DNS. [Bro can also help with this.](https://www.bro.org/sphinx/scripts/base/protocols/dns/main.bro.html) --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #584: METRON-950: Migrate storm-kafka-client to 1.1
Github user justinleet commented on the issue: https://github.com/apache/metron/pull/584 I ran up the main topologies last night and everything seemed to correlate correctly and ran without issue. Hadn't run up pcap yet, but it look like @mmiklavc is working on validating it. Given that it's the same spout throughout, I'm not convinced it's necessary to retread it. Anybody disagree? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #531: METRON-854 create dhcp dump parser
Github user JonZeolla commented on the issue: https://github.com/apache/metron/pull/531 With bro there's also an option to [do a lookup](https://github.com/bro/bro/blob/master/src/bro.bif#L3431-L3458) and [add it](https://www.bro.org/sphinx-git/frameworks/logging.html#add-fields-to-a-log) into the DHCP log, although we shouldn't exist that will exist by default. That would also require a DNS lookup for each DHCP handshake. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
Re: [DISCUSS] Enrichment Split/Join issues
I’m glad you bring this up. This is a huge architectural difference from the original OpenSOC topology and one that we have been warned to take back then. To be perfectly honest, I don’t see the big perfomance improvement from parallel processing. If a specific enrichment is a little more i/o dependent than the other you can tweak parallelism to address this. Also there can be dependencies that make parallel enrichment virtually impossible or at least less efficient (i.e. first labeling, and “completing” a message and then dependent of label and completeness do different other enrichments). So you have a +1 from me for serial rather than parallel enrichment. BR, Christian On 16.05.17, 16:58, "Casey Stella"wrote: Hi All, Last week, I encountered some weirdness in the Enrichment topology. Doing some somewhat high-latency enrichment work, I noticed that at some point, data stopped flowing through the enrichment topology. I tracked down the problem to the join bolt. For those who aren't aware, we do a split/join pattern so that enrichments can be done in parallel. It works as follows: - A split bolt sends the appropriate subset of the message to each enrichment bolt as well as the whole message to the join bolt - The join bolt will receive each of the pieces of the message and then, when fully joined, it will send the message on. What is happening under load or high velocity, however, is that the cache is evicting the partially joined message before it can be fully joined due to the volume of traffic. This is obviously not ideal. As such, it is clear that adjusting the size of the cache and the characteristics of eviction is likely a good idea and a necessary part to tuning enrichments. The cache size is sensitive to: - The latency of the *slowest* enrichment - The number of tuples in flight at once As such, the knobs you have to tune are either the parallelism of the join bolt or the size of the cache. As it stands, I see a couple of things wrong here that we can correct with minimal issue: - We have no message of warning indicating that this is happening - Changing cache sizes means changing flux. We should promote this to the properties file. - We should document the knobs mentioned above clearly in the enrichment topology README Those small changes, I think, are table stakes, but what I wanted to discuss more in depth is the lingering questions: - Is this an architectural pattern that we can use as-is? - Should we consider a persistent cache a la HBase or Apache Ignite as a pluggable component to Metron? - Should we consider taking the performance hit and doing the enrichments serially? - When an eviction happens, what should we do? - Fail the tuple, thereby making congestion worse - Pass through the partially enriched results, thereby making enrichments "best effort" Anyway, I wanted to talk this through and inform of some of the things I'm seeing. Sorry for the novel. ;) Casey
Re: [DISCUSS] Enrichment Split/Join issues
We could definitely parallelize within the bolt, but you're right, it does break the storm model. I also like making things other people's problems (it's called working "smart" not "hard", right? not laziness, surely. ;), but yeah, using windowing for this seems like it might introduce some artificial latency. It's also not going to eliminate the problem, but rather just make the knob to tweak things have a different characteristic. Whereas before we have knobs around how many messages, now it's a knob around how long an enrichment is going to take maximally (which, I think is more natural, honestly). On Tue, May 16, 2017 at 12:05 PM, Simon Elliston Ball < si...@simonellistonball.com> wrote: > Would you then parallelise within Stellar to handle things like multiple > lookups? This feels like it would be breaking the storm model somewhat, and > could lead to bad things with threads for example. Or would you think of > doing something like the grouping Stellar uses today to parallelise across > something like a pool of Stellar bolts and join? > > I like the idea of Otto’s solution (making it someone else's problem, > storm’s specifically :) ) but that also assumes we insert the artificial > latency of a time windowed join. If we’re going down that route, we might > as well just use spark and run everything on yarn. At that point though we > lose a lot of the benefits of low latency for time to detection, and > real-time enrichment in things like the streaming enrichment writer. > > Simon > > > On 16 May 2017, at 16:59, Nick Allenwrote: > > > > I would like to see us just migrate wholly to Stellar enrichments and > > remove the separate HBase and Geo enrichment bolts from the Enrichment > > topology. Stellar provides a user with much greater flexibility than the > > existing HBase and Geo enrichment bolts. > > > > A side effect of this would be to greatly simplify the Enrichment > > topology. I don't think we would not need the split/join pattern if we > did > > this. No? > > > > On Tue, May 16, 2017 at 11:54 AM, Casey Stella > wrote: > > > >> The problem is that an enrichment type won't necessarily have a fixed > >> performance characteristic. Take stellar enrichments, for instance. > Doing > >> a HBase call for one sensor vs doing simple string munging will have > vastly > >> differing performance. Both of them are functioning within the stellar > >> enrichment bolt. Also, some enrichments may call for multiple calls to > >> HBase. Parallelizing those, would make some sense, I think. > >> > >> I do take your point, though, that it's not as though it's strictly > serial, > >> it's just that the unit of parallelism is the message, rather than the > >> enrichment per message. > >> > >> On Tue, May 16, 2017 at 11:47 AM, Christian Tramnitz < > tramn...@trasec.de> > >> wrote: > >> > >>> I’m glad you bring this up. This is a huge architectural difference > from > >>> the original OpenSOC topology and one that we have been warned to take > >> back > >>> then. > >>> To be perfectly honest, I don’t see the big perfomance improvement from > >>> parallel processing. If a specific enrichment is a little more i/o > >>> dependent than the other you can tweak parallelism to address this. > Also > >>> there can be dependencies that make parallel enrichment virtually > >>> impossible or at least less efficient (i.e. first labeling, and > >>> “completing” a message and then dependent of label and completeness do > >>> different other enrichments). > >>> > >>> So you have a +1 from me for serial rather than parallel enrichment. > >>> > >>> > >>> BR, > >>> Christian > >>> > >>> On 16.05.17, 16:58, "Casey Stella" wrote: > >>> > >>>Hi All, > >>> > >>>Last week, I encountered some weirdness in the Enrichment topology. > >>> Doing > >>>some somewhat high-latency enrichment work, I noticed that at some > >>> point, > >>>data stopped flowing through the enrichment topology. I tracked > down > >>> the > >>>problem to the join bolt. For those who aren't aware, we do a > >>> split/join > >>>pattern so that enrichments can be done in parallel. It works as > >>> follows: > >>> > >>> - A split bolt sends the appropriate subset of the message to > each > >>> enrichment bolt as well as the whole message to the join bolt > >>> - The join bolt will receive each of the pieces of the message > and > >>> then, > >>> when fully joined, it will send the message on. > >>> > >>> > >>>What is happening under load or high velocity, however, is that the > >>> cache > >>>is evicting the partially joined message before it can be fully > >> joined > >>> due > >>>to the volume of traffic. This is obviously not ideal. As such, it > >> is > >>>clear that adjusting the size of the cache and the characteristics > of > >>>eviction is likely a good idea and a necessary part to tuning > >>> enrichments. > >>>The
[GitHub] metron pull request #585: METRON-936: Fixes to pcap for performance and test...
Github user mmiklavc commented on a diff in the pull request: https://github.com/apache/metron/pull/585#discussion_r116865536 --- Diff: metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/spout/pcap/PartitionHDFSWriter.java --- @@ -102,14 +106,43 @@ public void sync(FSDataOutputStream input) throws IOException { private SyncHandler syncHandler; private long batchStartTime; private long numWritten; + private Configuration fsConfig = new Configuration(); public PartitionHDFSWriter(String topic, int partition, String uuid, HDFSWriterConfig config) { this.topic = topic; this.partition = partition; this.uuid = uuid; this.config = config; + try { - this.fs = FileSystem.get(new Configuration()); + int replicationFactor = config.getReplicationFactor(); + if (replicationFactor != -1) { +fsConfig.set("dfs.replication", (String.valueOf(replicationFactor))); + } + if(config.getHDFSConfig() != null && !config.getHDFSConfig().isEmpty()) { +for(Map.Entryentry : config.getHDFSConfig().entrySet()) { + if(entry.getValue() instanceof Integer) { --- End diff -- String.valueOf() takes a primitive and we don't specifically know the type of X in the incoming config map when calling fsConfig.setX (where X is a type, e.g. Boolean, Integer, etc.). --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #585: METRON-936: Fixes to pcap for performance and test...
Github user mmiklavc commented on a diff in the pull request: https://github.com/apache/metron/pull/585#discussion_r116866077 --- Diff: metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/spout/pcap/HDFSWriterCallback.java --- @@ -116,7 +117,11 @@ public HDFSWriterCallback withConfig(HDFSWriterConfig config) { public List apply(List tuple, EmitContext context) { byte[] key = (byte[]) tuple.get(0); byte[] value = (byte[]) tuple.get(1); -if(!config.getDeserializer().deserializeKeyValue(key, value, KeyValue.key.get(), KeyValue.value.get())) { --- End diff -- Looks that way, good catch. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #589: METRON-955: Make the default sync policy for HDFS ...
Github user asfgit closed the pull request at: https://github.com/apache/metron/pull/589 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron issue #586: METRON-508 Expand Elasticsearch templates to support the ...
Github user JonZeolla commented on the issue: https://github.com/apache/metron/pull/586 Per @simonellistonball 's comments in #531 I added initial support for the native way that Bro handles tracking DHCP's Client ID field and updated the above instructions appropriately. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] metron pull request #585: METRON-936: Fixes to pcap for performance and test...
Github user mmiklavc commented on a diff in the pull request: https://github.com/apache/metron/pull/585#discussion_r116883407 --- Diff: metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/spout/pcap/deserializer/KeyValueDeserializer.java --- @@ -28,6 +26,17 @@ public abstract class KeyValueDeserializer implements Serializable { protected TimestampConverter converter; + public static class Result { --- End diff -- I looked at this and decided to tweak the terms a bit in this inner class. After stepping away from it a couple days I couldn't immediately tell what Result.result actually meant :) --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---