-
Notifications
You must be signed in to change notification settings - Fork 505
METRON-174 Storm consumption of hbase enrichment reference data #127
Conversation
…d into HBase via storm
| writer.init(); | ||
|
|
||
| if(isBulk) { | ||
| writerTransformer = config -> new ParserWriterConfiguration(config); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Curious why a Function is used here. Why not just instantiate a WriterConfiguration object for each case and pass that to messageWriter.init? Is there a benefit to doing it this way?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The config object can change if zookeeper is updated, so we want an
indirection here.
On Wed, May 25, 2016 at 19:11 merrimanr notifications@github.com wrote:
In
metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java
#127 (comment)
:@@ -60,7 +93,25 @@ public void prepare(Map stormConf, TopologyContext context, OutputCollector coll
super.prepare(stormConf, context, collector);
this.collector = collector;
parser.init();
- writer.init();
- if(isBulk) {
writerTransformer = config -> new ParserWriterConfiguration(config);Curious why a Function is used here. Why not just instantiate a
WriterConfiguration object for each case and pass that to
messageWriter.init? Is there a benefit to doing it this way?—
You are receiving this because you authored the thread.
Reply to this email directly or view it on GitHub
https://github.com/apache/incubator-metron/pull/127/files/e448abd8cccf6f912095474f034668198e58f442#r64668872
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah I see. That is very defensive. I like it :)
|
I like it. +1 |
|
Where are the docs for validating this? I would like to stream in an enrichment |
|
In order to validate this, you can do the following:
|
|
FYI...for some reason the kafka topic does not always get auto created. I can't figure out what options cause it to not auto create. Also, some times kafka throws an error the first time you push into a topic. However, when you run the same command again it works just fine. Maybe something to keep in mind for the future is that we can't rely on the kafka producer to reliably create a topic |
|
On AWS the following did not work: /usr/metron/0.1BETA/bin/start_parser_topology.sh -s user -k xxx:9092 -z xxx:2181 I got a: 41 [main-EventThread] INFO o.a.c.f.s.ConnectionStateManager - State change: CONNECTED Looking at the file here: looks like it chokes on the sensor topic. However, listing the topics: /usr/hdp/current/kafka-broker/bin/kafka-topics.sh --list --zookeeper xxx:2181 looks like i have it correct. My parser topology config looks as follows: { And is located under /usr/metron/0.1BETA/config/zookeeper/parsers/user.json Any suggestions? |
|
@james-sirota Did you push the new parser config to zookeeper via |
|
If you did push the config before trying to start the parser, then please confirm that the |
|
/usr/metron/0.1BETA/bin/zk_load_configs.sh -m DUMP -z 1xxx:2181 PARSER Config: websphere PARSER Config: bluecoat PARSER Config: squid Exception in thread "main" java.lang.RuntimeException: Unable to load { Caused by: com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException: Unrecognized field "writerClassName" (class org.apache.metron.common.configuration.SensorParserConfig), not marked as ignorable (3 known properties: , "parserConfig", "parserClassName", "sensorTopic"]) So looks like it can't load it. Looking in the jar it seems to exist jar -tf metron-parsers-0.1BETA.jar | grep CSVParser jar -tf metron-parsers-0.1BETA.jar | grep SimpleHbaseEnrichmentWriter |
|
Looks like it can't find the writerClassname field. Are you sure you ran a
|
|
I built the jar from the branch and copied it out to an existing AWS cluster that I had. |
|
I was able to get past the previous error by uploading a new common jar. Now when the topology comes up it processes the CSV no problem. But, I only have the spout and the parser bolt come up. The writer does not come up. Also, I can't get the parser to ack anything. My file looks like this: { |
|
Tracing through the log i got this. Let me bounce hbase and see if that helps |
|
Another interesting thing that I think is a problem is that I sent it exactly 30 tuples. The spout acked 60 tuples (somehow doubled the count) and when it failed the number of failed acks was 100. Not sure where the other 40 tuples came from, but i think it may be replaying them. We need to check all our topologies for this behavior. We need to make sure that failed tuples are not being replayed |
|
Now getting the following error on the bro topology trying to enrich: 2016-05-31 04:24:05.212 o.a.k.c.n.Selector [WARN] Error in I/O with ip xxxx I am giving it the same kafka broker that i gave to the user topology, which worked |
|
You sure Kafka is still up? |
|
Try pulling data from that broker using the console consumer |
|
We should support streaming enrichment data into kafka and writing it out to HBase in a format suitable to be used in either the Simple HBase Enrichment Adapter or the Simple HBase Threat Intel Adapter.
This should be fully backwards compatible with how we did parser topologies before, so you should be able to spin up a vagrant image and see data go through the indices.