From 4fb9201670acf9960087f004f8858e99c0a87981 Mon Sep 17 00:00:00 2001 From: nickwallen Date: Wed, 15 Aug 2018 09:26:42 -0400 Subject: [PATCH 01/25] METRON-1703 Make Core Profiler Components Serializable (nickwallen) closes apache/metron#1145 --- .../profiler/DefaultMessageDistributor.java | 28 ++++++++++--------- .../metron/profiler/DefaultMessageRouter.java | 3 +- .../apache/metron/profiler/MessageRoute.java | 4 ++- .../profiler/clock/DefaultClockFactory.java | 4 ++- .../metron/profiler/clock/EventTimeClock.java | 3 +- 5 files changed, 25 insertions(+), 17 deletions(-) diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageDistributor.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageDistributor.java index 82f7174fbe..c926a708c6 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageDistributor.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageDistributor.java @@ -20,19 +20,11 @@ package org.apache.metron.profiler; -import static java.lang.String.format; - import com.google.common.base.Ticker; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; import com.google.common.cache.RemovalListener; import com.google.common.cache.RemovalNotification; -import java.lang.invoke.MethodHandles; -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; import org.apache.commons.lang.builder.HashCodeBuilder; import org.apache.metron.common.configuration.profiler.ProfileConfig; import org.apache.metron.stellar.dsl.Context; @@ -40,6 +32,16 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Serializable; +import java.lang.invoke.MethodHandles; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import static java.lang.String.format; + /** * The default implementation of a {@link MessageDistributor}. * @@ -57,7 +59,7 @@ * lost. * */ -public class DefaultMessageDistributor implements MessageDistributor { +public class DefaultMessageDistributor implements MessageDistributor, Serializable { protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); @@ -73,7 +75,7 @@ public class DefaultMessageDistributor implements MessageDistributor { * messages. Once it has not received messages for a period of time, it is * moved to the expired cache. */ - private transient Cache activeCache; + private Cache activeCache; /** * A cache of expired profiles. @@ -84,7 +86,7 @@ public class DefaultMessageDistributor implements MessageDistributor { * can flush the state of the expired profile. If the client does not flush * the expired profiles, this state will be lost forever. */ - private transient Cache expiredCache; + private Cache expiredCache; /** * Create a new message distributor. @@ -287,7 +289,7 @@ public DefaultMessageDistributor withPeriodDuration(int duration, TimeUnit units /** * A listener that is notified when profiles expire from the active cache. */ - private class ActiveCacheRemovalListener implements RemovalListener { + private class ActiveCacheRemovalListener implements RemovalListener, Serializable { @Override public void onRemoval(RemovalNotification notification) { @@ -305,7 +307,7 @@ public void onRemoval(RemovalNotification notification) /** * A listener that is notified when profiles expire from the active cache. */ - private class ExpiredCacheRemovalListener implements RemovalListener { + private class ExpiredCacheRemovalListener implements RemovalListener, Serializable { @Override public void onRemoval(RemovalNotification notification) { diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageRouter.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageRouter.java index d1a1a3b597..19bfa8ce9f 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageRouter.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageRouter.java @@ -30,6 +30,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Serializable; import java.lang.invoke.MethodHandles; import java.util.ArrayList; import java.util.List; @@ -44,7 +45,7 @@ * A single telemetry message may need to take multiple routes. This is the case * when a message is needed by more than one profile. */ -public class DefaultMessageRouter implements MessageRouter { +public class DefaultMessageRouter implements MessageRouter, Serializable { protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRoute.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRoute.java index 7288f03468..680e4e84a1 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRoute.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRoute.java @@ -22,6 +22,8 @@ import org.apache.metron.common.configuration.profiler.ProfileConfig; +import java.io.Serializable; + /** * Defines the 'route' a message must take through the Profiler. * @@ -33,7 +35,7 @@ * * @see MessageRouter */ -public class MessageRoute { +public class MessageRoute implements Serializable { /** * The definition of the profile on this route. diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/DefaultClockFactory.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/DefaultClockFactory.java index d62e62bd49..8a574f6c4e 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/DefaultClockFactory.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/DefaultClockFactory.java @@ -21,6 +21,8 @@ import org.apache.metron.common.configuration.profiler.ProfilerConfig; +import java.io.Serializable; + /** * Creates a {@link Clock} based on the profiler configuration. * @@ -29,7 +31,7 @@ * *

The default implementation of a {@link ClockFactory}. */ -public class DefaultClockFactory implements ClockFactory { +public class DefaultClockFactory implements ClockFactory, Serializable { /** * @param config The profiler configuration. diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/EventTimeClock.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/EventTimeClock.java index 5cd574eff4..c094b7d4b7 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/EventTimeClock.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/EventTimeClock.java @@ -26,6 +26,7 @@ import java.lang.invoke.MethodHandles; import java.util.Optional; +import java.io.Serializable; /** * A {@link Clock} that advances based on event time. @@ -33,7 +34,7 @@ * Event time is advanced by the timestamps contained within telemetry messages, rather * than the system clock. */ -public class EventTimeClock implements Clock { +public class EventTimeClock implements Clock, Serializable { protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); From 5eff97fbe4a99b4d9fdec1010cfa3358cf182ddd Mon Sep 17 00:00:00 2001 From: nickwallen Date: Thu, 16 Aug 2018 08:26:03 -0400 Subject: [PATCH 02/25] METRON-1704 Message Timestamp Logic Should be Shared (nickwallen) closes apache/metron#1146 --- .../profiler/DefaultMessageDistributor.java | 6 +- .../metron/profiler/DefaultMessageRouter.java | 34 ++++++++-- .../metron/profiler/MessageDistributor.java | 4 +- .../apache/metron/profiler/MessageRoute.java | 64 ++++++++++++++++++- .../metron/profiler/StandAloneProfiler.java | 26 ++------ .../DefaultMessageDistributorTest.java | 32 +++++----- .../profiler/DefaultMessageRouterTest.java | 55 ++++++++++++++++ .../profiler/bolt/ProfileBuilderBolt.java | 4 +- .../profiler/bolt/ProfileSplitterBolt.java | 37 ++++------- .../profiler/bolt/ProfileBuilderBoltTest.java | 2 +- .../bolt/ProfileSplitterBoltTest.java | 5 +- 11 files changed, 189 insertions(+), 80 deletions(-) diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageDistributor.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageDistributor.java index c926a708c6..d950b07e5b 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageDistributor.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageDistributor.java @@ -148,16 +148,14 @@ public DefaultMessageDistributor( /** * Distribute a message along a MessageRoute. * - * @param message The message that needs distributed. - * @param timestamp The timestamp of the message. * @param route The message route. * @param context The Stellar execution context. */ @Override - public void distribute(JSONObject message, long timestamp, MessageRoute route, Context context) { + public void distribute(MessageRoute route, Context context) { try { ProfileBuilder builder = getBuilder(route, context); - builder.apply(message, timestamp); + builder.apply(route.getMessage(), route.getTimestamp()); } catch(ExecutionException e) { LOG.error("Unexpected error", e); diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageRouter.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageRouter.java index 19bfa8ce9f..21ff2b1e79 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageRouter.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageRouter.java @@ -22,6 +22,9 @@ import org.apache.metron.common.configuration.profiler.ProfileConfig; import org.apache.metron.common.configuration.profiler.ProfilerConfig; +import org.apache.metron.profiler.clock.Clock; +import org.apache.metron.profiler.clock.ClockFactory; +import org.apache.metron.profiler.clock.DefaultClockFactory; import org.apache.metron.stellar.common.DefaultStellarStatefulExecutor; import org.apache.metron.stellar.common.StellarStatefulExecutor; import org.apache.metron.stellar.dsl.Context; @@ -54,10 +57,16 @@ public class DefaultMessageRouter implements MessageRouter, Serializable { */ private StellarStatefulExecutor executor; + /** + * Responsible for creating the {@link Clock}. + */ + private ClockFactory clockFactory; + public DefaultMessageRouter(Context context) { this.executor = new DefaultStellarStatefulExecutor(); StellarFunctions.initialize(context); executor.setContext(context); + clockFactory = new DefaultClockFactory(); } /** @@ -74,7 +83,8 @@ public List route(JSONObject message, ProfilerConfig config, Conte // attempt to route the message to each of the profiles for (ProfileConfig profile: config.getProfiles()) { - Optional route = routeToProfile(message, profile); + Clock clock = clockFactory.createClock(config); + Optional route = routeToProfile(message, profile, clock); route.ifPresent(routes::add); } @@ -87,20 +97,24 @@ public List route(JSONObject message, ProfilerConfig config, Conte * @param profile The profile that may need the message. * @return A MessageRoute if the message is needed by the profile. */ - private Optional routeToProfile(JSONObject message, ProfileConfig profile) { + private Optional routeToProfile(JSONObject message, ProfileConfig profile, Clock clock) { Optional route = Optional.empty(); // allow the profile to access the fields defined within the message @SuppressWarnings("unchecked") final Map state = (Map) message; - try { // is this message needed by this profile? if (executor.execute(profile.getOnlyif(), state, Boolean.class)) { - // what is the name of the entity in this message? - String entity = executor.execute(profile.getForeach(), state, String.class); - route = Optional.of(new MessageRoute(profile, entity)); + // what time is is? could be either system or event time + Optional timestamp = clock.currentTimeMillis(message); + if(timestamp.isPresent()) { + + // what is the name of the entity in this message? + String entity = executor.execute(profile.getForeach(), state, String.class); + route = Optional.of(new MessageRoute(profile, entity, message, timestamp.get())); + } } } catch(Throwable e) { @@ -111,4 +125,12 @@ private Optional routeToProfile(JSONObject message, ProfileConfig return route; } + + public void setExecutor(StellarStatefulExecutor executor) { + this.executor = executor; + } + + public void setClockFactory(ClockFactory clockFactory) { + this.clockFactory = clockFactory; + } } diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageDistributor.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageDistributor.java index ea5be0f9c6..b164207f87 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageDistributor.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageDistributor.java @@ -43,12 +43,10 @@ public interface MessageDistributor { /** * Distribute a message along a {@link MessageRoute}. * - * @param message The message that needs distributed. - * @param timestamp The timestamp of the message. * @param route The message route. * @param context The Stellar execution context. */ - void distribute(JSONObject message, long timestamp, MessageRoute route, Context context); + void distribute(MessageRoute route, Context context); /** * Flush all active profiles. diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRoute.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRoute.java index 680e4e84a1..e76b897e10 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRoute.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRoute.java @@ -20,7 +20,11 @@ package org.apache.metron.profiler; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.metron.common.configuration.profiler.ProfileConfig; +import org.json.simple.JSONObject; import java.io.Serializable; @@ -47,15 +51,27 @@ public class MessageRoute implements Serializable { */ private String entity; + /** + * The message taking this route. + */ + private JSONObject message; + + /** + * The timestamp of the message. + */ + private Long timestamp; + /** * Create a {@link MessageRoute}. * * @param profileDefinition The profile definition. - * @param entity The entity. + * @param entity The entity. */ - public MessageRoute(ProfileConfig profileDefinition, String entity) { + public MessageRoute(ProfileConfig profileDefinition, String entity, JSONObject message, Long timestamp) { this.entity = entity; this.profileDefinition = profileDefinition; + this.message = message; + this.timestamp = timestamp; } public String getEntity() { @@ -73,4 +89,48 @@ public ProfileConfig getProfileDefinition() { public void setProfileDefinition(ProfileConfig profileDefinition) { this.profileDefinition = profileDefinition; } + + public JSONObject getMessage() { + return message; + } + + public void setMessage(JSONObject message) { + this.message = message; + } + + public Long getTimestamp() { + return timestamp; + } + + public void setTimestamp(Long timestamp) { + this.timestamp = timestamp; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + MessageRoute that = (MessageRoute) o; + return new EqualsBuilder() + .append(profileDefinition, that.profileDefinition) + .append(entity, that.entity) + .append(message, that.message) + .append(timestamp, that.timestamp) + .isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(17, 37) + .append(profileDefinition) + .append(entity) + .append(message) + .append(timestamp) + .toHashCode(); + } } diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/StandAloneProfiler.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/StandAloneProfiler.java index f79efe6568..befa29685e 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/StandAloneProfiler.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/StandAloneProfiler.java @@ -112,26 +112,14 @@ public StandAloneProfiler(ProfilerConfig config, * @param message The message to apply. */ public void apply(JSONObject message) { - - // what time is it? - Clock clock = clockFactory.createClock(config); - Optional timestamp = clock.currentTimeMillis(message); - - // can only route the message, if we have a timestamp - if(timestamp.isPresent()) { - - // route the message to the correct profile builders - List routes = router.route(message, config, context); - for (MessageRoute route : routes) { - distributor.distribute(message, timestamp.get(), route, context); - } - - routeCount += routes.size(); - messageCount += 1; - - } else { - LOG.warn("No timestamp available for the message. The message will be ignored."); + // route the message to the correct profile builders + List routes = router.route(message, config, context); + for (MessageRoute route : routes) { + distributor.distribute(route, context); } + + routeCount += routes.size(); + messageCount += 1; } /** diff --git a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultMessageDistributorTest.java b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultMessageDistributorTest.java index ea9c5c66f9..48161e25a1 100644 --- a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultMessageDistributorTest.java +++ b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultMessageDistributorTest.java @@ -123,10 +123,10 @@ public void testDistribute() throws Exception { long timestamp = 100; ProfileConfig definition = createDefinition(profileOne); String entity = (String) messageOne.get("ip_src_addr"); - MessageRoute route = new MessageRoute(definition, entity); + MessageRoute route = new MessageRoute(definition, entity, messageOne, timestamp); // distribute one message and flush - distributor.distribute(messageOne, timestamp, route, context); + distributor.distribute(route, context); List measurements = distributor.flush(); // expect one measurement coming from one profile @@ -144,12 +144,12 @@ public void testDistributeWithTwoProfiles() throws Exception { String entity = (String) messageOne.get("ip_src_addr"); // distribute one message to the first profile - MessageRoute routeOne = new MessageRoute(createDefinition(profileOne), entity); - distributor.distribute(messageOne, timestamp, routeOne, context); + MessageRoute routeOne = new MessageRoute(createDefinition(profileOne), entity, messageOne, timestamp); + distributor.distribute(routeOne, context); // distribute another message to the second profile, but same entity - MessageRoute routeTwo = new MessageRoute(createDefinition(profileTwo), entity); - distributor.distribute(messageOne, timestamp, routeTwo, context); + MessageRoute routeTwo = new MessageRoute(createDefinition(profileTwo), entity, messageOne, timestamp); + distributor.distribute(routeTwo, context); // expect 2 measurements; 1 for each profile List measurements = distributor.flush(); @@ -164,13 +164,13 @@ public void testDistributeWithTwoEntities() throws Exception { // distribute one message String entityOne = (String) messageOne.get("ip_src_addr"); - MessageRoute routeOne = new MessageRoute(createDefinition(profileOne), entityOne); - distributor.distribute(messageOne, timestamp, routeOne, context); + MessageRoute routeOne = new MessageRoute(createDefinition(profileOne), entityOne, messageOne, timestamp); + distributor.distribute(routeOne, context); // distribute another message with a different entity String entityTwo = (String) messageTwo.get("ip_src_addr"); - MessageRoute routeTwo = new MessageRoute(createDefinition(profileTwo), entityTwo); - distributor.distribute(messageTwo, timestamp, routeTwo, context); + MessageRoute routeTwo = new MessageRoute(createDefinition(profileTwo), entityTwo, messageTwo, timestamp); + distributor.distribute(routeTwo, context); // expect 2 measurements; 1 for each entity List measurements = distributor.flush(); @@ -190,7 +190,7 @@ public void testNotYetTimeToExpireProfiles() throws Exception { // setup ProfileConfig definition = createDefinition(profileOne); String entity = (String) messageOne.get("ip_src_addr"); - MessageRoute route = new MessageRoute(definition, entity); + MessageRoute route = new MessageRoute(definition, entity, messageOne, System.currentTimeMillis()); distributor = new DefaultMessageDistributor( periodDurationMillis, profileTimeToLiveMillis, @@ -198,7 +198,7 @@ public void testNotYetTimeToExpireProfiles() throws Exception { ticker); // distribute one message - distributor.distribute(messageOne, 1000000, route, context); + distributor.distribute(route, context); // advance time to just shy of the profile TTL ticker.advanceTime(profileTimeToLiveMillis - 1000, MILLISECONDS); @@ -220,7 +220,7 @@ public void testProfilesShouldExpire() throws Exception { // setup ProfileConfig definition = createDefinition(profileOne); String entity = (String) messageOne.get("ip_src_addr"); - MessageRoute route = new MessageRoute(definition, entity); + MessageRoute route = new MessageRoute(definition, entity, messageOne, System.currentTimeMillis()); distributor = new DefaultMessageDistributor( periodDurationMillis, profileTimeToLiveMillis, @@ -228,7 +228,7 @@ public void testProfilesShouldExpire() throws Exception { ticker); // distribute one message - distributor.distribute(messageOne, 100000, route, context); + distributor.distribute(route, context); // advance time to just beyond the period duration ticker.advanceTime(profileTimeToLiveMillis + 1000, MILLISECONDS); @@ -251,7 +251,7 @@ public void testExpiredProfilesShouldBeRemoved() throws Exception { // setup ProfileConfig definition = createDefinition(profileOne); String entity = (String) messageOne.get("ip_src_addr"); - MessageRoute route = new MessageRoute(definition, entity); + MessageRoute route = new MessageRoute(definition, entity, messageOne, System.currentTimeMillis()); distributor = new DefaultMessageDistributor( periodDurationMillis, profileTimeToLiveMillis, @@ -259,7 +259,7 @@ public void testExpiredProfilesShouldBeRemoved() throws Exception { ticker); // distribute one message - distributor.distribute(messageOne, 1000000, route, context); + distributor.distribute(route, context); // advance time a couple of hours ticker.advanceTime(2, HOURS); diff --git a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultMessageRouterTest.java b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultMessageRouterTest.java index 534f155f1d..f583c30ab7 100644 --- a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultMessageRouterTest.java +++ b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultMessageRouterTest.java @@ -55,6 +55,17 @@ public class DefaultMessageRouterTest { private String inputTwo; private JSONObject messageTwo; + /** + * { + * "ip_src_addr": "10.0.0.1", + * "value": "22", + * "timestamp": 1531250226659 + * } + */ + @Multiline + private String inputWithTimestamp; + private JSONObject messageWithTimestamp; + /** * { * "profiles": [ ] @@ -175,6 +186,23 @@ public class DefaultMessageRouterTest { @Multiline private String goodAndBad; + /** + * { + * "profiles": [ + * { + * "profile": "profile-one", + * "foreach": "ip_src_addr", + * "init": { "x": "0" }, + * "update": { "x": "x + 1" }, + * "result": "x" + * } + * ], + * "timestampField": "timestamp" + * } + */ + @Multiline + private String profileWithEventTime; + private DefaultMessageRouter router; private Context context; @@ -193,6 +221,7 @@ public void setup() throws Exception { JSONParser parser = new JSONParser(); this.messageOne = (JSONObject) parser.parse(inputOne); this.messageTwo = (JSONObject) parser.parse(inputTwo); + this.messageWithTimestamp = (JSONObject) parser.parse(inputWithTimestamp); } @Test @@ -268,4 +297,30 @@ public void testWithGoodAndBad() throws Exception { assertEquals("good-profile", route1.getProfileDefinition().getProfile()); assertEquals(messageOne.get("ip_src_addr"), route1.getEntity()); } + + /** + * + */ + @Test + public void testMessageWithTimestamp() throws Exception { + List routes = router.route(messageWithTimestamp, createConfig(profileWithEventTime), context);; + + assertEquals(1, routes.size()); + MessageRoute route1 = routes.get(0); + assertEquals("profile-one", route1.getProfileDefinition().getProfile()); + assertEquals(messageWithTimestamp.get("ip_src_addr"), route1.getEntity()); + assertEquals(messageWithTimestamp.get("timestamp"), route1.getTimestamp()); + } + + /** + * If the timestamp of a message cannot be determined, it should not be routed. + * + *

This might happen when using event time and the message is missing the timestamp field. + */ + @Test + public void testMessageWithMissingTimestamp() throws Exception { + // messageOne does not contain a timestamp + List routes = router.route(messageOne, createConfig(profileWithEventTime), context); + assertEquals(0, routes.size()); + } } diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java index 0d1f27ea72..f9c0edd72d 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileBuilderBolt.java @@ -363,9 +363,9 @@ private void handleMessage(Tuple input) { activeFlushSignal.update(timestamp); // distribute the message - MessageRoute route = new MessageRoute(definition, entity); + MessageRoute route = new MessageRoute(definition, entity, message, timestamp); synchronized (messageDistributor) { - messageDistributor.distribute(message, timestamp, route, getStellarContext()); + messageDistributor.distribute(route, getStellarContext()); } LOG.debug("Message distributed: profile={}, entity={}, timestamp={}", definition.getProfile(), entity, timestamp); diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java index f28411f647..f57deb7eea 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileSplitterBolt.java @@ -96,11 +96,6 @@ public class ProfileSplitterBolt extends ConfiguredProfilerBolt { */ private transient MessageRouter router; - /** - * Responsible for creating the {@link Clock}. - */ - private transient ClockFactory clockFactory; - /** * @param zookeeperUrl The Zookeeper URL that contains the configuration for this bolt. */ @@ -114,10 +109,9 @@ public void prepare(Map stormConf, TopologyContext context, OutputCollector coll this.collector = collector; this.parser = new JSONParser(); this.router = new DefaultMessageRouter(getStellarContext()); - this.clockFactory = new DefaultClockFactory(); } - private Context getStellarContext() { + public Context getStellarContext() { Map global = getConfigurations().getGlobalConfig(); return new Context.Builder() .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client) @@ -162,15 +156,9 @@ private void doExecute(Tuple input) throws ParseException, UnsupportedEncodingEx // ensure there is a valid profiler configuration ProfilerConfig config = getProfilerConfig(); if(config != null && config.getProfiles().size() > 0) { + routeMessage(input, message, config); - // what time is it? - Clock clock = clockFactory.createClock(config); - Optional timestamp = clock.currentTimeMillis(message); - - // route the message. if a message does not contain the timestamp field, it cannot be routed. - timestamp.ifPresent(ts -> routeMessage(input, message, config, ts)); - - } else { + } else if(LOG.isDebugEnabled()) { LOG.debug("No Profiler configuration found. Nothing to do."); } } @@ -180,24 +168,23 @@ private void doExecute(Tuple input) throws ParseException, UnsupportedEncodingEx * @param input The input tuple on which to anchor. * @param message The telemetry message. * @param config The Profiler configuration. - * @param timestamp The timestamp of the telemetry message. */ - private void routeMessage(Tuple input, JSONObject message, ProfilerConfig config, Long timestamp) { + private void routeMessage(Tuple input, JSONObject message, ProfilerConfig config) { // emit a tuple for each 'route' List routes = router.route(message, config, getStellarContext()); for (MessageRoute route : routes) { - Values values = createValues(message, timestamp, route); + Values values = createValues(route); collector.emit(input, values); LOG.debug("Found route for message; profile={}, entity={}, timestamp={}", route.getProfileDefinition().getProfile(), route.getEntity(), - timestamp); + route.getTimestamp()); } - LOG.debug("Found {} route(s) for message with timestamp={}", routes.size(), timestamp); + LOG.debug("Found {} route(s) for message", routes.size()); } /** @@ -222,22 +209,20 @@ public void declareOutputFields(OutputFieldsDeclarer declarer) { /** * Creates the {@link Values} attached to the outgoing tuple. * - * @param message The telemetry message. - * @param timestamp The timestamp of the message. * @param route The route the message must take. * @return */ - private Values createValues(JSONObject message, Long timestamp, MessageRoute route) { + private Values createValues(MessageRoute route) { // the order here must match `declareOutputFields` - return new Values(message, timestamp, route.getEntity(), route.getProfileDefinition()); + return new Values(route.getMessage(), route.getTimestamp(), route.getEntity(), route.getProfileDefinition()); } protected MessageRouter getMessageRouter() { return router; } - public void setClockFactory(ClockFactory clockFactory) { - this.clockFactory = clockFactory; + public void setRouter(MessageRouter router) { + this.router = router; } } diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java index 3d009fb46b..3132ae62df 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileBuilderBoltTest.java @@ -127,7 +127,7 @@ public void testExtractMessage() throws Exception { bolt.execute(tupleWindow); // the message should have been extracted from the tuple and passed to the MessageDistributor - verify(distributor).distribute(eq(message1), eq(timestamp1), any(MessageRoute.class), any()); + verify(distributor).distribute(any(MessageRoute.class), any()); } diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileSplitterBoltTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileSplitterBoltTest.java index bf819239ef..d57e8257b9 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileSplitterBoltTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileSplitterBoltTest.java @@ -23,6 +23,7 @@ import org.adrianwalker.multilinestring.Multiline; import org.apache.metron.common.configuration.profiler.ProfileConfig; import org.apache.metron.common.configuration.profiler.ProfilerConfig; +import org.apache.metron.profiler.DefaultMessageRouter; import org.apache.metron.profiler.clock.FixedClockFactory; import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.test.bolt.BaseBoltTest; @@ -428,7 +429,9 @@ private ProfileSplitterBolt createBolt(ProfilerConfig config) throws Exception { bolt.prepare(new HashMap<>(), topologyContext, outputCollector); // set the clock factory AFTER calling prepare to use the fixed clock factory - bolt.setClockFactory(new FixedClockFactory(timestamp)); + DefaultMessageRouter router = new DefaultMessageRouter(bolt.getStellarContext()); + router.setClockFactory(new FixedClockFactory(timestamp)); + bolt.setRouter(router); return bolt; } From 401bd7a180dad4ff805fd6e5840ab826ee4e94b7 Mon Sep 17 00:00:00 2001 From: nickwallen Date: Mon, 20 Aug 2018 15:35:57 -0400 Subject: [PATCH 03/25] METRON-1706 HbaseClient.mutate should return the number of mutations (nickwallen) closes apache/metron#1147 --- .../metron/hbase/client/HBaseClient.java | 8 +++-- .../metron/hbase/client/HBaseClientTest.java | 30 ++++++++++++++++++- 2 files changed, 35 insertions(+), 3 deletions(-) diff --git a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java index c027c5c6f9..f0a0a106f8 100644 --- a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java +++ b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java @@ -133,9 +133,11 @@ public void clearMutations() { /** * Submits all queued Mutations. + * @return The number of mutation submitted. */ - public void mutate() { - Object[] result = new Object[mutations.size()]; + public int mutate() { + int mutationCount = mutations.size(); + Object[] result = new Object[mutationCount]; try { table.batch(mutations, result); mutations.clear(); @@ -144,6 +146,8 @@ public void mutate() { LOG.warn("Error performing a mutation to HBase.", e); throw new RuntimeException(e); } + + return mutationCount; } /** diff --git a/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java index 1849745f7d..ca1f5a686e 100644 --- a/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java +++ b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java @@ -160,7 +160,10 @@ public void testBatchWrite() throws Exception { // add two mutations to the queue client.addMutation(rowKey1, cols1, Durability.SYNC_WAL); client.addMutation(rowKey2, cols2, Durability.SYNC_WAL); - client.mutate(); + int count = client.mutate(); + + // there were two mutations + Assert.assertEquals(2, count); HBaseProjectionCriteria criteria = new HBaseProjectionCriteria(); criteria.addColumnFamily(WidgetMapper.CF_STRING); @@ -179,6 +182,31 @@ public void testBatchWrite() throws Exception { } } + /** + * What happens when there is nothing in the batch to write? + */ + @Test + public void testEmptyBatch() throws Exception { + + // do not add any mutations before attempting to write + int count = client.mutate(); + Assert.assertEquals(0, count); + + HBaseProjectionCriteria criteria = new HBaseProjectionCriteria(); + criteria.addColumnFamily(WidgetMapper.CF_STRING); + + // read back both + client.addGet(rowKey1, criteria); + client.addGet(rowKey2, criteria); + Result[] results = client.getAll(); + + // validate - there should be nothing to find + assertEquals(2, results.length); + for(Result result : results) { + Assert.assertTrue(result.isEmpty()); + } + } + /** * Should be able to read back widgets that were written with a TTL 30 days out. */ From 6fb50a18d75c3694c75596ab45bcc7531711eec7 Mon Sep 17 00:00:00 2001 From: nickwallen Date: Mon, 20 Aug 2018 17:53:58 -0400 Subject: [PATCH 04/25] METRON-1705 Create ProfilePeriod Using Period ID (nickwallen) closes apache/metron#1148 --- .../profiler/DefaultProfileBuilder.java | 4 +- .../metron/profiler/ProfileMeasurement.java | 2 +- .../apache/metron/profiler/ProfilePeriod.java | 43 +++++++++++---- .../profiler/DefaultProfileBuilderTest.java | 6 +-- .../metron/profiler/ProfilePeriodTest.java | 52 ++++++++++++++----- 5 files changed, 79 insertions(+), 28 deletions(-) diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultProfileBuilder.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultProfileBuilder.java index 66034ac7ac..ced1e8bc7e 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultProfileBuilder.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultProfileBuilder.java @@ -160,10 +160,8 @@ public void apply(JSONObject message, long timestamp) { */ @Override public Optional flush() { - Optional result; - ProfilePeriod period = new ProfilePeriod(maxTimestamp, periodDurationMillis, TimeUnit.MILLISECONDS); - + ProfilePeriod period = ProfilePeriod.fromTimestamp(maxTimestamp, periodDurationMillis, TimeUnit.MILLISECONDS); try { // execute the 'profile' expression String profileExpression = definition diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java index 4737c3d9dc..4d24fac325 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileMeasurement.java @@ -95,7 +95,7 @@ public ProfileMeasurement withGroups(List groups) { } public ProfileMeasurement withPeriod(long whenMillis, long periodDuration, TimeUnit periodUnits) { - this.withPeriod(new ProfilePeriod(whenMillis, periodDuration, periodUnits)); + this.withPeriod(ProfilePeriod.fromTimestamp(whenMillis, periodDuration, periodUnits)); return this; } diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfilePeriod.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfilePeriod.java index cbb827506c..78aa796f1b 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfilePeriod.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfilePeriod.java @@ -47,19 +47,46 @@ public class ProfilePeriod implements Serializable { */ private long durationMillis; + /** + * @param periodId A monotonically increasing number identifying the period. + * @param duration The duration of each profile period. + * @param units The units of the duration; hours, minutes, etc. + */ + private ProfilePeriod(long periodId, long duration, TimeUnit units) { + this.durationMillis = units.toMillis(duration); + this.period = periodId; + } /** + * Creates a {@link ProfilePeriod} given a timestamp defined in milliseconds + * from the epoch. + * * @param epochMillis A timestamp contained somewhere within the profile period. * @param duration The duration of each profile period. * @param units The units of the duration; hours, minutes, etc. */ - public ProfilePeriod(long epochMillis, long duration, TimeUnit units) { + public static ProfilePeriod fromTimestamp(long epochMillis, long duration, TimeUnit units) { if(duration <= 0) { - throw new IllegalArgumentException(format( - "period duration must be greater than 0; got '%d %s'", duration, units)); + throw new IllegalArgumentException(format("period duration must be > 0; got '%d %s'", duration, units)); } - this.durationMillis = units.toMillis(duration); - this.period = epochMillis / durationMillis; + long durationMillis = units.toMillis(duration); + long periodId = epochMillis / durationMillis; + return new ProfilePeriod(periodId, duration, units); + } + + /** + * Creates a {@link ProfilePeriod} given a timestamp defined in milliseconds + * from the epoch. + * + * @param periodId A monotonically increasing number identifying the period. + * @param duration The duration of each profile period. + * @param units The units of the duration; hours, minutes, etc. + */ + public static ProfilePeriod fromPeriodId(long periodId, long duration, TimeUnit units) { + if(periodId < 0) { + throw new IllegalArgumentException(format("period id must be >= 0; got '%d'", periodId)); + } + return new ProfilePeriod(periodId, duration, units); } /** @@ -80,15 +107,13 @@ public long getEndTimeMillis() { * Returns the next ProfilePeriod in time. */ public ProfilePeriod next() { - long nextStart = getStartTimeMillis() + durationMillis; - return new ProfilePeriod(nextStart, durationMillis, TimeUnit.MILLISECONDS); + return fromPeriodId(period + 1, durationMillis, TimeUnit.MILLISECONDS); } public long getPeriod() { return period; } - public long getDurationMillis() { return durationMillis; } @@ -126,7 +151,7 @@ public static List visitPeriods(long startEpochMillis , Function transformation ) { - ProfilePeriod period = new ProfilePeriod(startEpochMillis, duration, units); + ProfilePeriod period = ProfilePeriod.fromTimestamp(startEpochMillis, duration, units); List ret = new ArrayList<>(); while(period.getStartTimeMillis() <= endEpochMillis) { if(!inclusionPredicate.isPresent() || inclusionPredicate.get().test(period)) { diff --git a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultProfileBuilderTest.java b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultProfileBuilderTest.java index 24eb5f8fb7..3d0b4cca79 100644 --- a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultProfileBuilderTest.java +++ b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/DefaultProfileBuilderTest.java @@ -235,7 +235,7 @@ public void testProfilePeriodOnFlush() throws Exception { assertTrue(m.isPresent()); // validate the profile period - ProfilePeriod expected = new ProfilePeriod(timestamp, 10, TimeUnit.MINUTES); + ProfilePeriod expected = ProfilePeriod.fromTimestamp(timestamp, 10, TimeUnit.MINUTES); assertEquals(expected, m.get().getPeriod()); } { @@ -248,7 +248,7 @@ public void testProfilePeriodOnFlush() throws Exception { assertTrue(m.isPresent()); // validate the profile period - ProfilePeriod expected = new ProfilePeriod(timestamp, 10, TimeUnit.MINUTES); + ProfilePeriod expected = ProfilePeriod.fromTimestamp(timestamp, 10, TimeUnit.MINUTES); assertEquals(expected, m.get().getPeriod()); } } @@ -314,7 +314,7 @@ public void testStateAvailableToGroupBy() throws Exception { // setup long timestamp = 1503081070340L; - ProfilePeriod period = new ProfilePeriod(timestamp, 10, TimeUnit.MINUTES); + ProfilePeriod period = ProfilePeriod.fromTimestamp(timestamp, 10, TimeUnit.MINUTES); definition = JSONUtils.INSTANCE.load(testStateAvailableToGroupBy, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) diff --git a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java index f52bd09941..295117add1 100644 --- a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java +++ b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/ProfilePeriodTest.java @@ -48,7 +48,7 @@ public void testFirstPeriodAtEpoch() { long duration = 1; TimeUnit units = TimeUnit.HOURS; - ProfilePeriod period = new ProfilePeriod(0, duration, units); + ProfilePeriod period = ProfilePeriod.fromTimestamp(0, duration, units); assertEquals(0, period.getPeriod()); assertEquals(0, period.getStartTimeMillis()); assertEquals(units.toMillis(duration), period.getDurationMillis()); @@ -59,7 +59,7 @@ public void testOneMinutePeriods() { long duration = 1; TimeUnit units = TimeUnit.MINUTES; - ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units); + ProfilePeriod period = ProfilePeriod.fromTimestamp(AUG2016, duration, units); assertEquals(24535527, period.getPeriod()); assertEquals(1472131620000L, period.getStartTimeMillis()); // Thu, 25 Aug 2016 13:27:00 GMT assertEquals(units.toMillis(duration), period.getDurationMillis()); @@ -70,7 +70,7 @@ public void testFifteenMinutePeriods() { long duration = 15; TimeUnit units = TimeUnit.MINUTES; - ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units); + ProfilePeriod period = ProfilePeriod.fromTimestamp(AUG2016, duration, units); assertEquals(1635701, period.getPeriod()); assertEquals(1472130900000L, period.getStartTimeMillis()); // Thu, 25 Aug 2016 13:15:00 GMT assertEquals(units.toMillis(duration), period.getDurationMillis()); @@ -81,7 +81,7 @@ public void testOneHourPeriods() { long duration = 1; TimeUnit units = TimeUnit.HOURS; - ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units); + ProfilePeriod period = ProfilePeriod.fromTimestamp(AUG2016, duration, units); assertEquals(408925, period.getPeriod()); assertEquals(1472130000000L, period.getStartTimeMillis()); // Thu, 25 Aug 2016 13:00:00 GMT assertEquals(units.toMillis(duration), period.getDurationMillis()); @@ -92,7 +92,7 @@ public void testTwoHourPeriods() { long duration = 2; TimeUnit units = TimeUnit.HOURS; - ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units); + ProfilePeriod period = ProfilePeriod.fromTimestamp(AUG2016, duration, units); assertEquals(204462, period.getPeriod()); assertEquals(1472126400000L, period.getStartTimeMillis()); // Thu, 25 Aug 2016 12:00:00 GMT assertEquals(units.toMillis(duration), period.getDurationMillis()); @@ -103,7 +103,7 @@ public void testEightHourPeriods() { long duration = 8; TimeUnit units = TimeUnit.HOURS; - ProfilePeriod period = new ProfilePeriod(AUG2016, duration, units); + ProfilePeriod period = ProfilePeriod.fromTimestamp(AUG2016, duration, units); assertEquals(51115, period.getPeriod()); assertEquals(1472112000000L, period.getStartTimeMillis()); // Thu, 25 Aug 2016 08:00:00 GMT assertEquals(units.toMillis(duration), period.getDurationMillis()); @@ -114,7 +114,7 @@ public void testNextWithFifteenMinutePeriods() { long duration = 15; TimeUnit units = TimeUnit.MINUTES; - ProfilePeriod previous = new ProfilePeriod(AUG2016, duration, units); + ProfilePeriod previous = ProfilePeriod.fromTimestamp(AUG2016, duration, units); IntStream.range(0, 100).forEach(i -> { ProfilePeriod next = previous.next(); @@ -128,7 +128,7 @@ public void testNextWithFifteenMinutePeriods() { public void testPeriodDurationOfZero() { long duration = 0; TimeUnit units = TimeUnit.HOURS; - new ProfilePeriod(0, duration, units); + ProfilePeriod.fromTimestamp(0, duration, units); } /** @@ -137,8 +137,7 @@ public void testPeriodDurationOfZero() { */ @Test public void testKryoSerialization() throws Exception { - - ProfilePeriod expected = new ProfilePeriod(AUG2016, 1, TimeUnit.HOURS); + ProfilePeriod expected = ProfilePeriod.fromTimestamp(AUG2016, 1, TimeUnit.HOURS); // round-trip java serialization byte[] raw = SerDeUtils.toBytes(expected); @@ -154,8 +153,7 @@ public void testKryoSerialization() throws Exception { */ @Test public void testJavaSerialization() throws Exception { - - ProfilePeriod expected = new ProfilePeriod(AUG2016, 1, TimeUnit.HOURS); + ProfilePeriod expected = ProfilePeriod.fromTimestamp(AUG2016, 1, TimeUnit.HOURS); // serialize using java ByteArrayOutputStream bytes = new ByteArrayOutputStream(); @@ -173,4 +171,34 @@ public void testJavaSerialization() throws Exception { // ensure that the round-trip was successful assertEquals(expected, actual); } + + /** + * A {@link ProfilePeriod} can also be created from the period identifier and duration. + */ + @Test + public void testFromPeriodId() { + ProfilePeriod expected = ProfilePeriod.fromTimestamp(AUG2016, 1, TimeUnit.HOURS); + + // create the same period, but use the period identifier and duration + long periodId = expected.getPeriod(); + long duration = expected.getDurationMillis(); + ProfilePeriod actual = ProfilePeriod.fromPeriodId(periodId, duration, TimeUnit.MILLISECONDS); + + assertEquals(expected, actual); + } + + @Test(expected = IllegalArgumentException.class) + public void testWithNegativePeriodId() { + ProfilePeriod.fromPeriodId(-1, 1, TimeUnit.HOURS); + } + + /** + * The first period identifier 0 should start at the epoch. + */ + @Test + public void testFromPeriodIdAtEpoch() { + assertEquals( + ProfilePeriod.fromTimestamp(0, 1, TimeUnit.HOURS), + ProfilePeriod.fromPeriodId(0, 1, TimeUnit.HOURS)); + } } From 3bfbf018a9c3e1c74dc934901446b5111a0ada03 Mon Sep 17 00:00:00 2001 From: nickwallen Date: Thu, 23 Aug 2018 17:58:18 -0400 Subject: [PATCH 05/25] METRON-1707 Port Profiler to Spark (nickwallen) closes apache/metron#1150 --- dependencies_with_url.csv | 64 +++++- .../profiler/DefaultMessageDistributor.java | 1 - .../apache/metron/profiler/MessageRoute.java | 10 +- .../metron-profiler-spark/pom.xml | 195 ++++++++++++++++++ .../metron/profiler/spark/BatchProfiler.java | 102 +++++++++ .../profiler/spark/BatchProfilerConfig.java | 190 +++++++++++++++++ .../spark/ProfileMeasurementAdapter.java | 132 ++++++++++++ .../spark/function/GroupByPeriodFunction.java | 60 ++++++ .../spark/function/HBaseWriterFunction.java | 171 +++++++++++++++ .../spark/function/MessageRouterFunction.java | 113 ++++++++++ .../function/ProfileBuilderFunction.java | 107 ++++++++++ .../profiler/spark/function/TaskUtils.java | 41 ++++ .../spark/BatchProfilerIntegrationTest.java | 111 ++++++++++ .../function/HBaseWriterFunctionTest.java | 176 ++++++++++++++++ .../function/MessageRouterFunctionTest.java | 114 ++++++++++ .../function/ProfileBuilderFunctionTest.java | 98 +++++++++ .../src/test/resources/log4j.properties | 31 +++ .../src/test/resources/telemetry.json | 100 +++++++++ metron-analytics/pom.xml | 1 + .../configuration/profiler/ProfileResult.java | 4 + .../profiler/ProfileResultExpressions.java | 4 + pom.xml | 1 + 22 files changed, 1822 insertions(+), 4 deletions(-) create mode 100644 metron-analytics/metron-profiler-spark/pom.xml create mode 100644 metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfiler.java create mode 100644 metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfilerConfig.java create mode 100644 metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/ProfileMeasurementAdapter.java create mode 100644 metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/GroupByPeriodFunction.java create mode 100644 metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/HBaseWriterFunction.java create mode 100644 metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/MessageRouterFunction.java create mode 100644 metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/ProfileBuilderFunction.java create mode 100644 metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/TaskUtils.java create mode 100644 metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/BatchProfilerIntegrationTest.java create mode 100644 metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/function/HBaseWriterFunctionTest.java create mode 100644 metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/function/MessageRouterFunctionTest.java create mode 100644 metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/function/ProfileBuilderFunctionTest.java create mode 100644 metron-analytics/metron-profiler-spark/src/test/resources/log4j.properties create mode 100644 metron-analytics/metron-profiler-spark/src/test/resources/telemetry.json diff --git a/dependencies_with_url.csv b/dependencies_with_url.csv index 6ac1f23f90..6b4385b679 100644 --- a/dependencies_with_url.csv +++ b/dependencies_with_url.csv @@ -33,13 +33,18 @@ com.maxmind.geoip2:geoip2:jar:2.8.0:compile,Apache v2,https://github.com/maxmind com.sun.xml.bind:jaxb-impl:jar:2.2.3-1:compile,CDDL,http://jaxb.java.net/ com.sun.xml.bind:jaxb-impl:jar:2.2.5-2:compile,CDDL,http://jaxb.java.net/ com.twitter:jsr166e:jar:1.1.0:compile,CC0 1.0 Universal,http://github.com/twitter/jsr166e +com.twitter:chill-java:jar:0.8.4:compile,ASLv2,https://github.com/twitter/chill +com.twitter:chill_2.11:jar:0.8.4:compile,ASLv2,https://github.com/twitter/chill it.unimi.dsi:fastutil:jar:7.0.6:compile,ASLv2,https://github.com/vigna/fastutil javassist:javassist:jar:3.12.1.GA:compile,Apache v2,http://www.javassist.org/ javax.activation:activation:jar:1.1:compile,Common Development and Distribution License (CDDL) v1.0,http://java.sun.com/products/javabeans/jaf/index.jsp +javax.activation:activation:jar:1.1.1:compile,Common Development and Distribution License (CDDL) v1.0,http://java.sun.com/products/javabeans/jaf/index.jsp javax.annotation:jsr250-api:jar:1.0:compile,COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0,http://jcp.org/aboutJava/communityprocess/final/jsr250/index.html javax.annotation:javax.annotation-api:jar:1.3.2:compile,CDDL 1.1,https://github.com/javaee/javax.annotation/ +javax.annotation:javax.annotation-api:jar:1.2:compile,CDDL 1.1,https://github.com/javaee/javax.annotation/ javax.mail:mail:jar:1.4:compile,Common Development and Distribution License (CDDL) v1.0,https://glassfish.dev.java.net/javaee5/mail/ javax.servlet:javax.servlet-api:jar:3.1.0:compile,CDDL,http://servlet-spec.java.net +javax.ws.rs:javax.ws.rs-api:jar:2.0.1:compile,CDDL 1.1,https://github.com/jax-rs/api javax.xml.bind:jaxb-api:jar:2.2.11:compile,CDDL,http://jaxb.java.net/ javax.xml.bind:jaxb-api:jar:2.2.2:compile,CDDL,https://jaxb.dev.java.net/ javax.xml.bind:jaxb-api:jar:2.3.0:compile,CDDL,https://jaxb.dev.java.net/ @@ -47,25 +52,41 @@ javax.xml.stream:stax-api:jar:1.0-2:compile,COMMON DEVELOPMENT AND DISTRIBUTION jline:jline:jar:0.9.94:compile,BSD,http://jline.sourceforge.net junit:junit:jar:4.12:compile,Eclipse Public License 1.0,http://junit.org junit:junit:jar:4.4:compile,Common Public License Version 1.0,http://junit.org +net.razorvine:pyrolite:jar:4.13:compile,MIT,https://github.com/irmen/Pyrolite net.sf.jopt-simple:jopt-simple:jar:3.2:compile,The MIT License,http://jopt-simple.sourceforge.net net.sf.jopt-simple:jopt-simple:jar:4.9:compile,The MIT License,http://jopt-simple.sourceforge.net net.sf.saxon:Saxon-HE:jar:9.5.1-5:compile,Mozilla Public License Version 2.0,http://www.saxonica.com/ org.abego.treelayout:org.abego.treelayout.core:jar:1.0.1:compile,BSD 3-Clause "New" or "Revised" License (BSD-3-Clause),http://code.google.com/p/treelayout/ org.adrianwalker:multiline-string:jar:0.1.2:compile,Common Public License Version 1.0,https://github.com/benelog/multiline org.antlr:antlr4-runtime:jar:4.5:compile,BSD 3-Clause License,http://www.antlr.org +org.bouncycastle:bcprov-jdk15on:jar:1.52:compile,MIT,https://www.bouncycastle.org/license.html org.clojure:clojure:jar:1.6.0:compile,Eclipse Public License 1.0,http://clojure.org/ org.clojure:clojure:jar:1.7.0:compile,Eclipse Public License 1.0,http://clojure.org/ org.codehaus.jackson:jackson-jaxrs:jar:1.8.3:compile,Apache v2,http://jackson.codehaus.org org.codehaus.jackson:jackson-jaxrs:jar:1.9.13:compile,Apache v2,http://jackson.codehaus.org org.codehaus.jackson:jackson-xc:jar:1.8.3:compile,Apache v2,http://jackson.codehaus.org org.codehaus.jackson:jackson-xc:jar:1.9.13:compile,Apache v2,http://jackson.codehaus.org +org.codehaus.janino:commons-compiler:jar:3.0.8:compile,New BSD,https://github.com/janino-compiler/janino +org.codehaus.janino:janino:jar:3.0.8:compile,New BSD,https://github.com/janino-compiler/janino org.codehaus.woodstox:stax2-api:jar:3.1.4:compile,The BSD License,http://wiki.fasterxml.com/WoodstoxStax2 +org.json4s:json4s-ast_2.11:jar:3.2.11:compile,ASLv2,https://github.com/json4s/json4s +org.json4s:json4s-core_2.11:jar:3.2.11:compile,ASLv2,https://github.com/json4s/json4s +org.json4s:json4s-jackson_2.11:jar:3.2.11:compile,ASLv2,https://github.com/json4s/json4s org.jruby.jcodings:jcodings:jar:1.0.8:compile,MIT License,https://github.com/jruby/jcodings org.jruby.joni:joni:jar:2.1.2:compile,MIT License,https://github.com/jruby/joni +org.lz4:lz4-java:jar:1.4.0:compile,ASLv2,https://github.com/lz4/lz4-java org.mitre.taxii:taxii:jar:1.1.0.1:compile,The BSD 3-Clause License,https://github.com/TAXIIProject/java-taxii org.mitre:stix:jar:1.2.0.2:compile,The BSD 3-Clause License,https://github.com/STIXProject/java-stix org.mockito:mockito-core:jar:1.10.19:compile,The MIT License,http://www.mockito.org +org.roaringbitmap:RoaringBitmap:jar:0.5.11:compile,ASLv2,https://github.com/RoaringBitmap/RoaringBitmap org.scala-lang:scala-library:jar:2.10.6:compile,BSD-like,http://www.scala-lang.org/ +org.scala-lang.modules:scala-parser-combinators_2.11:jar:1.0.4:compile,BSD-like,http://www.scala-lang.org/ +org.scala-lang.modules:scala-xml_2.11:jar:1.0.1:compile,BSD-like,http://www.scala-lang.org/ +org.scala-lang:scala-compiler:jar:2.11.0:compile,BSD-like,http://www.scala-lang.org/ +org.scala-lang:scala-library:jar:2.11.8:compile,BSD-like,http://www.scala-lang.org/ +org.scala-lang:scala-reflect:jar:2.11.8:compile,BSD-like,http://www.scala-lang.org/ +org.scala-lang:scalap:jar:2.11.0:compile,BSD-like,http://www.scala-lang.org/ +oro:oro:jar:2.0.8:compile,ASLv2,http://attic.apache.org/projects/jakarta-oro.html xmlenc:xmlenc:jar:0.52:compile,The BSD License,http://xmlenc.sourceforge.net asm:asm:jar:3.1:compile,BSD,http://asm.ow2.org/ com.sun.jersey.contribs:jersey-guice:jar:1.9:compile,CDDL 1.1,https://jersey.java.net/ @@ -103,7 +124,10 @@ org.slf4j:slf4j-log4j12:jar:1.7.5:compile,MIT,http://www.slf4j.org org.slf4j:slf4j-log4j12:jar:1.7.7:compile,MIT,http://www.slf4j.org org.slf4j:slf4j-simple:jar:1.7.7:compile,MIT,http://www.slf4j.org org.slf4j:jcl-over-slf4j:jar:1.7.7:compile,MIT,http://www.slf4j.org +org.slf4j:jcl-over-slf4j:jar:1.7.16:compile,MIT,http://www.slf4j.org org.slf4j:jcl-over-slf4j:jar:1.7.21:compile,MIT,http://www.slf4j.org +org.slf4j:jcl-over-slf4j:jar:1.7.21:compile,MIT,http://www.slf4j.org +org.slf4j:jul-to-slf4j:jar:1.7.16:compile,MIT,http://www.slf4j.org org.slf4j:jul-to-slf4j:jar:1.7.21:compile,MIT,http://www.slf4j.org org.slf4j:jul-to-slf4j:jar:1.7.25:compile,MIT,http://www.slf4j.org aopalliance:aopalliance:jar:1.0:compile,Public Domain,http://aopalliance.sourceforge.net @@ -113,7 +137,9 @@ com.github.tony19:named-regexp:jar:0.2.3:compile,Apache License, Version 2.0, com.google.code.findbugs:jsr305:jar:1.3.9:compile,The Apache Software License, Version 2.0,http://findbugs.sourceforge.net/ com.google.code.findbugs:jsr305:jar:3.0.0:compile,The Apache Software License, Version 2.0,http://findbugs.sourceforge.net/ com.google.code.findbugs:annotations:jar:2.0.1:compile,The Apache Software License, Version 2.0,http://findbugs.sourceforge.net/ -com.carrotsearch:hppc:jar:0.7.1:compile,ASLv2, +com.carrotsearch:hppc:jar:0.7.1:compile,ASLv2,https://github.com/carrotsearch/hppc +com.carrotsearch:hppc:jar:0.7.2:compile,ASLv2,https://github.com/carrotsearch/hppc +com.clearspring.analytics:stream:jar:2.7.0:compile,ASLv2,https://github.com/addthis/stream-lib com.clearspring.analytics:stream:jar:2.9.5:compile,ASLv2,https://github.com/addthis/stream-lib com.codahale.metrics:metrics-core:jar:3.0.2:compile,MIT,https://github.com/codahale/metrics com.codahale.metrics:metrics-graphite:jar:3.0.2:compile,MIT,https://github.com/codahale/metrics @@ -132,6 +158,7 @@ com.fasterxml.jackson.core:jackson-core:jar:2.9.4:compile,ASLv2,https://github.c com.fasterxml.jackson.core:jackson-core:jar:2.9.5:compile,ASLv2,https://github.com/FasterXML/jackson-core com.fasterxml.jackson.core:jackson-databind:jar:2.2.3:compile,ASLv2,http://wiki.fasterxml.com/JacksonHome com.fasterxml.jackson.core:jackson-databind:jar:2.4.3:compile,ASLv2,http://github.com/FasterXML/jackson +com.fasterxml.jackson.core:jackson-databind:jar:2.6.7.1:compile,ASLv2,http://github.com/FasterXML/jackson com.fasterxml.jackson.core:jackson-databind:jar:2.7.4:compile,ASLv2,http://github.com/FasterXML/jackson com.fasterxml.jackson.core:jackson-databind:jar:2.8.3:compile,ASLv2,http://github.com/FasterXML/jackson com.fasterxml.jackson.core:jackson-databind:jar:2.9.4:compile,ASLv2,http://github.com/FasterXML/jackson @@ -147,6 +174,8 @@ com.fasterxml.jackson.datatype:jackson-datatype-joda:jar:2.9.5:compile,ASLv2,htt com.fasterxml.jackson.datatype:jackson-datatype-jdk8:jar:2.9.5:compile,ASLv2,https://github.com/FasterXML/jackson-modules-java8 com.fasterxml.jackson.datatype:jackson-datatype-jsr310:jar:2.9.5:compile,ASLv2,https://github.com/FasterXML/jackson-modules-java8 com.fasterxml.jackson.module:jackson-module-parameter-names:jar:2.9.5:compile,ASLv2,https://github.com/FasterXML/jackson-modules-java8 +com.fasterxml.jackson.module:jackson-module-paranamer:jar:2.7.9:compile,ASLv2,https://github.com/FasterXML/jackson-modules-base +com.fasterxml.jackson.module:jackson-module-scala_2.11:jar:2.6.7.1:compile,ASLv2,https://github.com/FasterXML/jackson-module-scala com.fasterxml:classmate:jar:1.3.1:compile,ASLv2,http://github.com/cowtowncoder/java-classmate com.fasterxml:classmate:jar:1.3.4:compile,ASLv2,http://github.com/cowtowncoder/java-classmate com.google.code.gson:gson:jar:2.2.4:compile,The Apache Software License, Version 2.0,http://code.google.com/p/google-gson/ @@ -164,10 +193,14 @@ com.lmax:disruptor:jar:3.3.2:compile,The Apache Software License, Version 2.0,ht com.googlecode.json-simple:json-simple:jar:1.1:compile,The Apache Software License, Version 2.0,http://code.google.com/p/json-simple/ com.googlecode.json-simple:json-simple:jar:1.1.1:compile,The Apache Software License, Version 2.0,http://code.google.com/p/json-simple/ com.jamesmurty.utils:java-xmlbuilder:jar:0.4:compile,Apache License, Version 2.0,http://code.google.com/p/java-xmlbuilder/ +com.jamesmurty.utils:java-xmlbuilder:jar:1.1:compile,Apache License, Version 2.0,http://code.google.com/p/java-xmlbuilder/ com.ning:compress-lzf:jar:1.0.2:compile,Apache License 2.0,http://github.com/ning/compress +com.ning:compress-lzf:jar:1.0.3:compile,Apache License 2.0,http://github.com/ning/compress com.opencsv:opencsv:jar:3.7:compile,Apache 2,http://opencsv.sf.net com.spatial4j:spatial4j:jar:0.5:compile,The Apache Software License, Version 2.0, com.tdunning:t-digest:jar:3.0:compile,The Apache Software License, Version 2.0,https://github.com/tdunning/t-digest +com.univocity:univocity-parsers:jar:2.5.9:compile,ASLv2,https://github.com/uniVocity/univocity-parsers +com.vlkan:flatbuffers:jar:1.2.0-3f79e055:compile,ASLv2,https://github.com/vy/flatbuffers com.yammer.metrics:metrics-core:jar:2.2.0:compile,ASLv2, commons-beanutils:commons-beanutils-core:jar:1.8.0:compile,ASLv2,http://commons.apache.org/beanutils/ commons-beanutils:commons-beanutils-core:jar:1.8.0:provided,ASLv2,http://commons.apache.org/beanutils/ @@ -203,19 +236,28 @@ commons-lang:commons-lang:jar:2.6:provided,ASLv2,http://commons.apache.org/lang/ commons-logging:commons-logging:jar:1.1.1:compile,ASLv2,http://commons.apache.org/logging commons-logging:commons-logging:jar:1.1.3:compile,ASLv2,http://commons.apache.org/proper/commons-logging/ commons-logging:commons-logging:jar:1.2:compile,ASLv2,http://commons.apache.org/proper/commons-logging/ +commons-net:commons-net:jar:2.2:compile,ASLv2,http://commons.apache.org/net/ commons-net:commons-net:jar:3.1:compile,ASLv2,http://commons.apache.org/net/ commons-net:commons-net:jar:3.1:provided,ASLv2,http://commons.apache.org/net/ commons-text:commons-text:jar:1.1:compile,ASLv2,http://commons.apache.org/proper/commons-text/ commons-validator:commons-validator:jar:1.4.0:compile,ASLv2,http://commons.apache.org/validator/ commons-validator:commons-validator:jar:1.5.1:compile,ASLv2,http://commons.apache.org/proper/commons-validator/ commons-validator:commons-validator:jar:1.6:compile,ASLv2,http://commons.apache.org/proper/commons-validator/ +et.razorvine:pyrolite:jar:4.13:compile,MIT,https://github.com/irmen/Pyrolite +io.airlift:aircompressor:jar:0.8:compile,ASLv2,https://github.com/airlift/aircompressor io.confluent:kafka-avro-serializer:jar:1.0:compile,ASLv2,https://github.com/confluentinc/schema-registry/ io.confluent:kafka-schema-registry-client:jar:1.0:compile,ASLv2,https://github.com/confluentinc/schema-registry/ +io.dropwizard.metrics:metrics-core:jar:3.1.5:compile,ASLv2,https://github.com/dropwizard/metrics +io.dropwizard.metrics:metrics-graphite:jar:3.1.5:compile,ASLv2,https://github.com/dropwizard/metrics +io.dropwizard.metrics:metrics-json:jar:3.1.5:compile,ASLv2,https://github.com/dropwizard/metrics +io.dropwizard.metrics:metrics-jvm:jar:3.1.5:compile,ASLv2,https://github.com/dropwizard/metrics io.netty:netty-all:jar:4.0.23.Final:compile,ASLv2, io.netty:netty-all:jar:4.0.23.Final:provided,ASLv2, -io.netty:netty:jar:3.10.5.Final:compile,Apache License, Version 2.0,http://netty.io/ +io.netty:netty-all:jar:4.1.17.Final:compile,ASLv2, io.netty:netty:jar:3.6.2.Final:compile,Apache License, Version 2.0,http://netty.io/ io.netty:netty:jar:3.7.0.Final:compile,Apache License, Version 2.0,http://netty.io/ +io.netty:netty:jar:3.9.9.Final:compile,Apache License, Version 2.0,http://netty.io/ +io.netty:netty:jar:3.10.5.Final:compile,Apache License, Version 2.0,http://netty.io/ io.thekraken:grok:jar:0.1.0:compile,Apache License, Version 2.0,http://maven.apache.org javax.inject:javax.inject:jar:1:compile,The Apache Software License, Version 2.0,http://code.google.com/p/atinject/ joda-time:joda-time:jar:2.3:compile,Apache 2,http://www.joda.org/joda-time/ @@ -224,9 +266,12 @@ joda-time:joda-time:jar:2.9.9:compile,Apache 2,http://www.joda.org/joda-time/ log4j:log4j:jar:1.2.15:compile,The Apache Software License, Version 2.0,http://logging.apache.org:80/log4j/1.2/ log4j:log4j:jar:1.2.16:compile,The Apache Software License, Version 2.0,http://logging.apache.org/log4j/1.2/ log4j:log4j:jar:1.2.17:compile,The Apache Software License, Version 2.0,http://logging.apache.org/log4j/1.2/ +net.iharder:base64:jar:2.3.8:compile,Public Domain,http://iharder.sourceforge.net/current/java/base64/ net.java.dev.jets3t:jets3t:jar:0.9.0:compile,Apache License, Version 2.0,http://www.jets3t.org +net.java.dev.jets3t:jets3t:jar:0.9.4:compile,Apache License, Version 2.0,http://www.jets3t.org net.jpountz.lz4:lz4:jar:1.2.0:compile,The Apache Software License, Version 2.0,https://github.com/jpountz/lz4-java net.jpountz.lz4:lz4:jar:1.3.0:compile,The Apache Software License, Version 2.0,https://github.com/jpountz/lz4-java +net.sf.py4j:py4j:jar:0.10.7:compile,, nl.jqno.equalsverifier:equalsverifier:jar:2.0.2:compile,The Apache Software License, Version 2.0,http://www.jqno.nl/equalsverifier org.codehaus.jackson:jackson-core-asl:jar:1.9.13:compile,The Apache Software License, Version 2.0,http://jackson.codehaus.org org.codehaus.jackson:jackson-mapper-asl:jar:1.9.13:compile,The Apache Software License, Version 2.0,http://jackson.codehaus.org @@ -354,6 +399,7 @@ org.springframework.security:spring-security-core:jar:4.1.3.RELEASE:compile,ASLv org.springframework.security:spring-security-core:jar:5.0.4.RELEASE:compile,ASLv2,https://github.com/spring-projects/spring-security org.springframework.security:spring-security-web:jar:4.1.3.RELEASE:compile,ASLv2,https://github.com/spring-projects/spring-security org.springframework.security:spring-security-web:jar:5.0.4.RELEASE:compile,ASLv2,https://github.com/spring-projects/spring-security +org.spark-project.spark:unused:jar:1.0.0:compile,ASLv2,https://spark.apache.org antlr:antlr:jar:2.7.7:compile,BSD 3-Clause License,http://www.antlr2.org com.h2database:h2:jar:1.4.192:compile,EPL 1.0,http://www.h2database.com/html/license.html com.h2database:h2:jar:1.4.197:compile,EPL 1.0,http://www.h2database.com/html/license.html @@ -370,6 +416,7 @@ org.springframework.kafka:spring-kafka:jar:1.1.1.RELEASE:compile,ASLv2,https://g org.springframework.kafka:spring-kafka:jar:2.0.4.RELEASE:compile,ASLv2,https://github.com/spring-projects/spring-kafka ch.hsr:geohash:jar:1.3.0:compile,ASLv2,https://github.com/kungfoo/geohash-java org.locationtech.spatial4j:spatial4j:jar:0.6:compile,ASLv2,https://github.com/locationtech/spatial4j +com.github.luben:zstd-jni:jar:1.3.2-2:compile,BSD,https://github.com/luben/zstd-jni com.github.spullara.mustache.java:compiler:jar:0.9.3:compile,ASLv2,https://github.com/spullara/mustache.java/blob/master/LICENSE io.netty:netty-buffer:jar:4.1.13.Final:compile,ASLv2,http://netty.io/ io.netty:netty-codec-http:jar:4.1.13.Final:compile,ASLv2,http://netty.io/ @@ -395,6 +442,19 @@ org.elasticsearch:securesm:jar:1.1:compile,ASLv2,https://github.com/elastic/elas org.hdrhistogram:HdrHistogram:jar:2.1.9:compile,BSD,https://github.com/HdrHistogram/HdrHistogram/blob/master/LICENSE.txt com.trendmicro:tlsh:jar:3.7.1:compile,ASLv2,https://github.com/trendmicro/tlsh org.glassfish:javax.json:jar:1.0.4:compile,Common Development and Distribution License (CDDL) v1.0,https://github.com/javaee/jsonp +org.glassfish.hk2.external:aopalliance-repackaged:jar:2.4.0-b34:compile,Common Development and Distribution License (CDDL) v1.0,https://github.com/javaee/hk2 +org.glassfish.hk2.external:javax.inject:jar:2.4.0-b34:compile,Common Development and Distribution License (CDDL) v1.0,https://github.com/javaee/hk2 +org.glassfish.hk2:hk2-api:jar:2.4.0-b34:compile,Common Development and Distribution License (CDDL) v1.0,https://github.com/javaee/hk2 +org.glassfish.hk2:hk2-locator:jar:2.4.0-b34:compile,Common Development and Distribution License (CDDL) v1.0,https://github.com/javaee/hk2 +org.glassfish.hk2:hk2-utils:jar:2.4.0-b34:compile,Common Development and Distribution License (CDDL) v1.0,https://github.com/javaee/hk2 +org.glassfish.hk2:osgi-resource-locator:jar:1.0.1:compile,Common Development and Distribution License (CDDL) v1.0,https://github.com/javaee/hk2 +org.glassfish.jersey.bundles.repackaged:jersey-guava:jar:2.22.2:compile +org.glassfish.jersey.containers:jersey-container-servlet-core:jar:2.22.2:compile +org.glassfish.jersey.containers:jersey-container-servlet:jar:2.22.2:compile +org.glassfish.jersey.core:jersey-client:jar:2.22.2:compile,EPL 2.0,https://github.com/eclipse-ee4j/jersey +org.glassfish.jersey.core:jersey-common:jar:2.22.2:compile,EPL 2.0,https://github.com/eclipse-ee4j/jersey +org.glassfish.jersey.core:jersey-server:jar:2.22.2:compile,EPL 2.0,https://github.com/eclipse-ee4j/jersey +org.glassfish.jersey.media:jersey-media-jaxb:jar:2.22.2:compile,EPL 2.0,https://github.com/eclipse-ee4j/jersey org.eclipse.persistence:javax.persistence:jar:2.1.1:compile,EPL 1.0,http://www.eclipse.org/eclipselink org.eclipse.persistence:org.eclipse.persistence.antlr:jar:2.6.4:compile,EPL 1.0,http://www.eclipse.org/eclipselink org.eclipse.persistence:org.eclipse.persistence.asm:jar:2.6.4:compile,EPL 1.0,http://www.eclipse.org/eclipselink diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageDistributor.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageDistributor.java index d950b07e5b..673072bbf7 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageDistributor.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/DefaultMessageDistributor.java @@ -28,7 +28,6 @@ import org.apache.commons.lang.builder.HashCodeBuilder; import org.apache.metron.common.configuration.profiler.ProfileConfig; import org.apache.metron.stellar.dsl.Context; -import org.json.simple.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRoute.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRoute.java index e76b897e10..7cdb6077a0 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRoute.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRoute.java @@ -22,11 +22,11 @@ import org.apache.commons.lang3.builder.EqualsBuilder; import org.apache.commons.lang3.builder.HashCodeBuilder; -import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.metron.common.configuration.profiler.ProfileConfig; import org.json.simple.JSONObject; import java.io.Serializable; +import java.util.Map; /** * Defines the 'route' a message must take through the Profiler. @@ -74,6 +74,10 @@ public MessageRoute(ProfileConfig profileDefinition, String entity, JSONObject m this.timestamp = timestamp; } + public MessageRoute() { + // necessary for serialization + } + public String getEntity() { return entity; } @@ -98,6 +102,10 @@ public void setMessage(JSONObject message) { this.message = message; } + public void setMessage(Map message) { + this.message = new JSONObject(message); + } + public Long getTimestamp() { return timestamp; } diff --git a/metron-analytics/metron-profiler-spark/pom.xml b/metron-analytics/metron-profiler-spark/pom.xml new file mode 100644 index 0000000000..93ce08af30 --- /dev/null +++ b/metron-analytics/metron-profiler-spark/pom.xml @@ -0,0 +1,195 @@ + + + + + 4.0.0 + + org.apache.metron + metron-analytics + 0.5.1 + + metron-profiler-spark + https://metron.apache.org/ + + UTF-8 + UTF-8 + + + + org.apache.spark + spark-core_2.11 + ${global_spark_version} + + + org.apache.spark + spark-sql_2.11 + ${global_spark_version} + + + org.antlr + antlr-runtime + + + + + org.apache.metron + metron-profiler-common + ${project.parent.version} + + + org.apache.metron + metron-common + ${project.parent.version} + + + com.google.guava + guava + + + org.slf4j + slf4j-log4j12 + + + + + org.apache.metron + metron-hbase + ${project.parent.version} + + + org.slf4j + slf4j-log4j12 + + + org.apache.storm + storm-hbase + + + + + org.apache.metron + metron-hbase + ${project.parent.version} + test + test-jar + + + org.apache.hbase + hbase-client + ${global_hbase_version} + + + org.apache.hadoop + hadoop-auth + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + + + + org.apache.metron + metron-statistics + ${project.parent.version} + + + kryo + com.esotericsoftware + + + + + org.apache.logging.log4j + log4j-api + ${global_log4j_core_version} + test + + + org.apache.logging.log4j + log4j-core + ${global_log4j_core_version} + test + + + + + + org.apache.maven.plugins + maven-shade-plugin + ${global_shade_version} + + true + + + + package + + shade + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.tdunning + org.apache.metron.tdunning + + + + + storm:storm-core:* + storm:storm-lib:* + org.slf4j.impl* + org.slf4j:slf4j-log4j* + + + + + + .yaml + LICENSE.txt + ASL2.0 + NOTICE.txt + + + + + + + + + + + + + + diff --git a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfiler.java b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfiler.java new file mode 100644 index 0000000000..f9996135d8 --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfiler.java @@ -0,0 +1,102 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.metron.profiler.spark; + +import com.google.common.collect.Maps; +import org.apache.metron.common.configuration.profiler.ProfilerConfig; +import org.apache.metron.profiler.MessageRoute; +import org.apache.metron.profiler.spark.function.GroupByPeriodFunction; +import org.apache.metron.profiler.spark.function.HBaseWriterFunction; +import org.apache.metron.profiler.spark.function.MessageRouterFunction; +import org.apache.metron.profiler.spark.function.ProfileBuilderFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.lang.invoke.MethodHandles; +import java.util.Map; +import java.util.Properties; + +import static org.apache.metron.profiler.spark.BatchProfilerConfig.TELEMETRY_INPUT_FORMAT; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.TELEMETRY_INPUT_PATH; +import static org.apache.spark.sql.functions.sum; + +/** + * The 'Batch Profiler' that generates profiles by consuming data in batch from archived telemetry. + * + *

The Batch Profiler is executed in Spark. + */ +public class BatchProfiler implements Serializable { + + protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + /** + * Execute the Batch Profiler. + * + * @param spark The spark session. + * @param properties The profiler configuration properties. + * @param profiles The profile definitions. + * @return The number of profile measurements produced. + */ + public long run(SparkSession spark, + Properties properties, + Properties globalProperties, + ProfilerConfig profiles) { + + LOG.debug("Building {} profile(s)", profiles.getProfiles().size()); + Map globals = Maps.fromProperties(globalProperties); + + String inputFormat = TELEMETRY_INPUT_FORMAT.get(properties, String.class); + String inputPath = TELEMETRY_INPUT_PATH.get(properties, String.class); + LOG.debug("Loading telemetry from '{}'", inputPath); + + // fetch the archived telemetry + Dataset telemetry = spark + .read() + .format(inputFormat) + .load(inputPath) + .as(Encoders.STRING()); + LOG.debug("Found {} telemetry record(s)", telemetry.cache().count()); + + // find all routes for each message + Dataset routes = telemetry + .flatMap(new MessageRouterFunction(profiles, globals), Encoders.bean(MessageRoute.class)); + LOG.debug("Generated {} message route(s)", routes.cache().count()); + + // build the profiles + Dataset measurements = routes + .groupByKey(new GroupByPeriodFunction(properties), Encoders.STRING()) + .mapGroups(new ProfileBuilderFunction(properties, globals), Encoders.bean(ProfileMeasurementAdapter.class)); + LOG.debug("Produced {} profile measurement(s)", measurements.cache().count()); + + // write the profile measurements to HBase + long count = measurements + .mapPartitions(new HBaseWriterFunction(properties), Encoders.INT()) + .agg(sum("value")) + .head() + .getLong(0); + LOG.debug("{} profile measurement(s) written to HBase", count); + + return count; + } +} \ No newline at end of file diff --git a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfilerConfig.java b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfilerConfig.java new file mode 100644 index 0000000000..054806ed32 --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/BatchProfilerConfig.java @@ -0,0 +1,190 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.metron.profiler.spark; + +import org.apache.hadoop.hbase.client.Durability; +import org.apache.metron.stellar.common.utils.ConversionUtils; + +import java.util.Map; +import java.util.Properties; + +/** + * Defines the configuration values recognized by the Batch Profiler. + */ +public enum BatchProfilerConfig { + + PERIOD_DURATION_UNITS("profiler.period.duration.units", "MINUTES", String.class), + + PERIOD_DURATION("profiler.period.duration", 15, Integer.class), + + HBASE_SALT_DIVISOR("profiler.hbase.salt.divisor", 1000, Integer.class), + + HBASE_TABLE_PROVIDER("profiler.hbase.table.provider", "org.apache.metron.hbase.HTableProvider", String.class), + + HBASE_TABLE_NAME("profiler.hbase.table", "profiler", String.class), + + HBASE_COLUMN_FAMILY("profiler.hbase.column.family", "P", String.class), + + HBASE_WRITE_DURABILITY("profiler.hbase.durability", Durability.USE_DEFAULT, Durability.class), + + TELEMETRY_INPUT_FORMAT("profiler.batch.input.format", "text", String.class), + + TELEMETRY_INPUT_PATH("profiler.batch.input.path", "hdfs://localhost:9000/apps/metron/indexing/indexed/*/*", String.class); + + /** + * The key for the configuration value. + */ + private String key; + + /** + * The default value of the configuration, if none other is specified. + */ + private Object defaultValue; + + /** + * The type of the configuration value. + */ + private Class valueType; + + BatchProfilerConfig(String key, Object defaultValue, Class valueType) { + this.key = key; + this.defaultValue = defaultValue; + this.valueType = valueType; + } + + /** + * Returns the key of the configuration value. + */ + public String getKey() { + return key; + } + + /** + * Returns the default value of the configuration. + */ + public Object getDefault() { + return getDefault(valueType); + } + + /** + * Returns the default value of the configuration, cast to the expected type. + * + * @param clazz The class of the expected type of the configuration value. + * @param The expected type of the configuration value. + */ + public T getDefault(Class clazz) { + return defaultValue == null ? null: ConversionUtils.convert(defaultValue, clazz); + } + + /** + * Returns the configuration value from a map of configuration values. + * + * @param config A map containing configuration values. + */ + public Object get(Map config) { + return getOrDefault(config, defaultValue); + } + + /** + * Returns the configuration value from a map of configuration values. + * + * @param properties Configuration properties. + */ + public Object get(Properties properties) { + return getOrDefault(properties, defaultValue); + } + + /** + * Returns the configuration value from a map of configuration values, cast to the expected type. + * + * @param config A map containing configuration values. + */ + public T get(Map config, Class clazz) { + return getOrDefault(config, defaultValue, clazz); + } + + /** + * Returns the configuration value from a map of configuration values, cast to the expected type. + * + * @param properties Configuration properties. + */ + public T get(Properties properties, Class clazz) { + return getOrDefault(properties, defaultValue, clazz); + } + + /** + * Returns the configuration value from a map of configuration values. If the value is not specified, + * the default value is returned. + * + * @param config A map containing configuration values. + * @param defaultValue The default value to return, if one is not defined. + * @return The configuration value or the specified default, if one is not defined. + */ + private Object getOrDefault(Map config, Object defaultValue) { + return getOrDefault(config, defaultValue, valueType); + } + + /** + * Returns the configuration value from a map of configuration values. If the value is not specified, + * the default value is returned. + * + * @param properties A map containing configuration values. + * @param defaultValue The default value to return, if one is not defined. + * @return The configuration value or the specified default, if one is not defined. + */ + private Object getOrDefault(Properties properties, Object defaultValue) { + return getOrDefault(properties, defaultValue, valueType); + } + + /** + * Returns the configuration value, cast to the expected type, from a map of configuration values. + * If the value is not specified, the default value is returned. + * + * @param config A map containing configuration values. + * @param defaultValue The default value to return, if one is not defined. + * @param clazz The class of the expected type of the configuration value. + * @param The expected type of the configuration value. + * @return The configuration value or the specified default, if one is not defined. + */ + private T getOrDefault(Map config, Object defaultValue, Class clazz) { + Object value = config.getOrDefault(key, defaultValue.toString()); + return value == null ? null : ConversionUtils.convert(value, clazz); + } + + /** + * Returns the configuration value, cast to the expected type, from a map of configuration values. + * If the value is not specified, the default value is returned. + * + * @param properties Configuration properties. + * @param defaultValue The default value to return, if one is not defined. + * @param clazz The class of the expected type of the configuration value. + * @param The expected type of the configuration value. + * @return The configuration value or the specified default, if one is not defined. + */ + private T getOrDefault(Properties properties, Object defaultValue, Class clazz) { + Object value = properties.getOrDefault(key, defaultValue); + return value == null ? null : ConversionUtils.convert(value, clazz); + } + + @Override + public String toString() { + return key; + } +} diff --git a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/ProfileMeasurementAdapter.java b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/ProfileMeasurementAdapter.java new file mode 100644 index 0000000000..5da7d04d2f --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/ProfileMeasurementAdapter.java @@ -0,0 +1,132 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.metron.profiler.spark; + +import org.apache.metron.common.utils.SerDeUtils; +import org.apache.metron.profiler.ProfileMeasurement; +import org.apache.metron.profiler.ProfilePeriod; + +import java.io.Serializable; +import java.util.concurrent.TimeUnit; + +/** + * An adapter for the {@link ProfileMeasurement} class so that the data + * can be serialized as required by Spark. + * + *

The `Encoders.bean(Class)` encoder does not handle serialization of type `Object` well. This + * adapter encodes the profile's result as byte[] rather than an Object to work around this. + */ +public class ProfileMeasurementAdapter implements Serializable { + + /** + * The name of the profile that this measurement is associated with. + */ + private String profileName; + + /** + * The name of the entity being profiled. + */ + private String entity; + + /** + * A monotonically increasing number identifying the period. The first period is 0 + * and began at the epoch. + */ + private Long periodId; + + /** + * The duration of each period in milliseconds. + */ + private Long durationMillis; + + /** + * The result of evaluating the profile expression. + * + * The `Encoders.bean(Class)` encoder does not handle serialization of type `Object`. This + * adapter encodes the profile's result as `byte[]` rather than an `Object` to work around this. + */ + private byte[] profileValue; + + public ProfileMeasurementAdapter() { + // default constructor required for serialization in Spark + } + + public ProfileMeasurementAdapter(ProfileMeasurement measurement) { + this.profileName = measurement.getProfileName(); + this.entity = measurement.getEntity(); + this.periodId = measurement.getPeriod().getPeriod(); + this.durationMillis = measurement.getPeriod().getDurationMillis(); + this.profileValue = SerDeUtils.toBytes(measurement.getProfileValue()); + } + + public ProfileMeasurement toProfileMeasurement() { + ProfilePeriod period = ProfilePeriod.fromPeriodId(periodId, durationMillis, TimeUnit.MILLISECONDS); + ProfileMeasurement measurement = new ProfileMeasurement() + .withProfileName(profileName) + .withEntity(entity) + .withPeriod(period) + .withProfileValue(SerDeUtils.fromBytes(profileValue, Object.class)); + return measurement; + } + + public String getProfileName() { + return profileName; + } + + public void setProfileName(String profileName) { + this.profileName = profileName; + } + + public String getEntity() { + return entity; + } + + public void setEntity(String entity) { + this.entity = entity; + } + + public Long getPeriodId() { + return periodId; + } + + public void setPeriodId(Long periodId) { + this.periodId = periodId; + } + + public Long getDurationMillis() { + return durationMillis; + } + + public void setDurationMillis(Long durationMillis) { + this.durationMillis = durationMillis; + } + + public byte[] getProfileValue() { + return profileValue; + } + + public void setProfileValue(byte[] profileValue) { + this.profileValue = profileValue; + } + + public void setProfileValue(Object profileValue) { + this.profileValue = SerDeUtils.toBytes(profileValue); + } +} diff --git a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/GroupByPeriodFunction.java b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/GroupByPeriodFunction.java new file mode 100644 index 0000000000..1b602f4b47 --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/GroupByPeriodFunction.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.metron.profiler.spark.function; + +import org.apache.metron.profiler.MessageRoute; +import org.apache.metron.profiler.ProfilePeriod; +import org.apache.spark.api.java.function.MapFunction; + +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +import static org.apache.metron.profiler.spark.BatchProfilerConfig.PERIOD_DURATION; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.PERIOD_DURATION_UNITS; + +/** + * Defines how {@link MessageRoute} are grouped. + * + * The routes are grouped by (profile, entity, periodId) so that all of the required + * messages are available to produce a {@link org.apache.metron.profiler.ProfileMeasurement}. + */ +public class GroupByPeriodFunction implements MapFunction { + + /** + * The duration of each profile period. + */ + private int periodDuration; + + /** + * The units of the period duration. + */ + private TimeUnit periodDurationUnits; + + public GroupByPeriodFunction(Properties profilerProperties) { + periodDurationUnits = TimeUnit.valueOf(PERIOD_DURATION_UNITS.get(profilerProperties, String.class)); + periodDuration = PERIOD_DURATION.get(profilerProperties, Integer.class); + } + + @Override + public String call(MessageRoute route) { + ProfilePeriod period = ProfilePeriod.fromTimestamp(route.getTimestamp(), periodDuration, periodDurationUnits); + return route.getProfileDefinition().getProfile() + "-" + route.getEntity() + "-" + period.getPeriod(); + } +} diff --git a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/HBaseWriterFunction.java b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/HBaseWriterFunction.java new file mode 100644 index 0000000000..cfabd94a15 --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/HBaseWriterFunction.java @@ -0,0 +1,171 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.metron.profiler.spark.function; + +import org.apache.commons.collections4.IteratorUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.client.Durability; +import org.apache.metron.hbase.HTableProvider; +import org.apache.metron.hbase.TableProvider; +import org.apache.metron.hbase.client.HBaseClient; +import org.apache.metron.profiler.ProfileMeasurement; +import org.apache.metron.profiler.hbase.ColumnBuilder; +import org.apache.metron.profiler.hbase.RowKeyBuilder; +import org.apache.metron.profiler.hbase.SaltyRowKeyBuilder; +import org.apache.metron.profiler.hbase.ValueOnlyColumnBuilder; +import org.apache.metron.profiler.spark.ProfileMeasurementAdapter; +import org.apache.spark.api.java.function.MapPartitionsFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.lang.reflect.InvocationTargetException; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_COLUMN_FAMILY; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_SALT_DIVISOR; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_TABLE_NAME; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_TABLE_PROVIDER; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_WRITE_DURABILITY; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.PERIOD_DURATION; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.PERIOD_DURATION_UNITS; + +/** + * Writes the profile measurements to HBase in Spark. + */ +public class HBaseWriterFunction implements MapPartitionsFunction { + + protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private TableProvider tableProvider; + + /** + * The name of the HBase table to write to. + */ + private String tableName; + + /** + * The durability guarantee when writing to HBase. + */ + private Durability durability; + + /** + * Builds the HBase row key. + */ + private RowKeyBuilder rowKeyBuilder; + + /** + * Assembles the columns for HBase. + */ + private ColumnBuilder columnBuilder; + + public HBaseWriterFunction(Properties properties) { + tableName = HBASE_TABLE_NAME.get(properties, String.class); + durability = HBASE_WRITE_DURABILITY.get(properties, Durability.class); + + // row key builder + int saltDivisor = HBASE_SALT_DIVISOR.get(properties, Integer.class); + int periodDuration = PERIOD_DURATION.get(properties, Integer.class); + TimeUnit periodDurationUnits = TimeUnit.valueOf(PERIOD_DURATION_UNITS.get(properties, String.class)); + rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDuration, periodDurationUnits); + + // column builder + String columnFamily = HBASE_COLUMN_FAMILY.get(properties, String.class); + columnBuilder = new ValueOnlyColumnBuilder(columnFamily); + + // hbase table provider + String providerImpl = HBASE_TABLE_PROVIDER.get(properties, String.class); + tableProvider = createTableProvider(providerImpl); + } + + /** + * Writes a set of measurements to HBase. + * + * @param iterator The measurements to write. + * @return The number of measurements written to HBase. + */ + @Override + public Iterator call(Iterator iterator) throws Exception { + int count = 0; + LOG.debug("About to write profile measurement(s) to HBase"); + + // do not open hbase connection, if nothing to write + List measurements = IteratorUtils.toList(iterator); + if(measurements.size() > 0) { + + // open an HBase connection + Configuration config = HBaseConfiguration.create(); + try (HBaseClient client = new HBaseClient(tableProvider, config, tableName)) { + + for (ProfileMeasurementAdapter adapter : measurements) { + ProfileMeasurement m = adapter.toProfileMeasurement(); + client.addMutation(rowKeyBuilder.rowKey(m), columnBuilder.columns(m), durability); + } + count = client.mutate(); + + } catch (IOException e) { + LOG.error("Unable to open connection to HBase", e); + throw new RuntimeException(e); + } + } + + LOG.debug("{} profile measurement(s) written to HBase", count); + return IteratorUtils.singletonIterator(count); + } + + /** + * Set the {@link TableProvider} using the class name of the provider. + * @param providerImpl The name of the class. + * @return + */ + public HBaseWriterFunction withTableProviderImpl(String providerImpl) { + this.tableProvider = createTableProvider(providerImpl); + return this; + } + + /** + * Creates a TableProvider based on a class name. + * @param providerImpl The class name of a TableProvider + */ + private static TableProvider createTableProvider(String providerImpl) { + LOG.trace("Creating table provider; className={}", providerImpl); + + // if class name not defined, use a reasonable default + if(StringUtils.isEmpty(providerImpl) || providerImpl.charAt(0) == '$') { + return new HTableProvider(); + } + + // instantiate the table provider + try { + Class clazz = (Class) Class.forName(providerImpl); + return clazz.getConstructor().newInstance(); + + } catch (InstantiationException | IllegalAccessException | IllegalStateException | + InvocationTargetException | NoSuchMethodException | ClassNotFoundException e) { + throw new IllegalStateException("Unable to instantiate connector", e); + } + } +} diff --git a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/MessageRouterFunction.java b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/MessageRouterFunction.java new file mode 100644 index 0000000000..cf8029f0ec --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/MessageRouterFunction.java @@ -0,0 +1,113 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.metron.profiler.spark.function; + +import org.apache.metron.common.configuration.profiler.ProfilerConfig; +import org.apache.metron.profiler.DefaultMessageRouter; +import org.apache.metron.profiler.MessageRoute; +import org.apache.metron.profiler.MessageRouter; +import org.apache.metron.stellar.dsl.Context; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.invoke.MethodHandles; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * The function responsible for finding routes for a given message in Spark. + */ +public class MessageRouterFunction implements FlatMapFunction { + + protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + /** + * The global configuration used for the execution of Stellar. + */ + private Map globals; + + /** + * The profile definitions. + */ + private ProfilerConfig profilerConfig; + + public MessageRouterFunction(ProfilerConfig profilerConfig, Map globals) { + this.profilerConfig = profilerConfig; + this.globals = globals; + } + + /** + * Find all routes for a given telemetry message. + * + *

A message may need routed to multiple profiles should it be needed by more than one. A + * message may also not be routed should it not be needed by any profiles. + * + * @param jsonMessage The raw JSON message. + * @return A list of message routes. + */ + @Override + public Iterator call(String jsonMessage) throws Exception { + List routes; + + JSONParser parser = new JSONParser(); + Context context = TaskUtils.getContext(globals); + MessageRouter router = new DefaultMessageRouter(context); + + // parse the raw message + Optional message = toMessage(jsonMessage, parser); + if(message.isPresent()) { + + // find all routes + routes = router.route(message.get(), profilerConfig, context); + LOG.trace("Found {} route(s) for a message", routes.size()); + + } else { + // the message is not valid and must be ignored + routes = Collections.emptyList(); + LOG.trace("No route possible. Unable to parse message."); + } + + return routes.iterator(); + } + + /** + * Parses the raw JSON of a message. + * + * @param json The raw JSON to parse. + * @param parser The parser to use. + * @return The parsed telemetry message. + */ + private static Optional toMessage(String json, JSONParser parser) { + try { + JSONObject message = (JSONObject) parser.parse(json); + return Optional.of(message); + + } catch(Throwable e) { + LOG.warn(String.format("Unable to parse message, message will be ignored; message='%s'", json), e); + return Optional.empty(); + } + } +} diff --git a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/ProfileBuilderFunction.java b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/ProfileBuilderFunction.java new file mode 100644 index 0000000000..273695b42d --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/ProfileBuilderFunction.java @@ -0,0 +1,107 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.metron.profiler.spark.function; + +import org.apache.metron.profiler.DefaultMessageDistributor; +import org.apache.metron.profiler.MessageDistributor; +import org.apache.metron.profiler.MessageRoute; +import org.apache.metron.profiler.ProfileMeasurement; +import org.apache.metron.profiler.spark.ProfileMeasurementAdapter; +import org.apache.metron.stellar.dsl.Context; +import org.apache.spark.api.java.function.MapGroupsFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.invoke.MethodHandles; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import static java.util.Comparator.comparing; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.PERIOD_DURATION; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.PERIOD_DURATION_UNITS; + +/** + * The function responsible for building profiles in Spark. + */ +public class ProfileBuilderFunction implements MapGroupsFunction { + + protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private long periodDurationMillis; + private Map globals; + + public ProfileBuilderFunction(Properties properties, Map globals) { + TimeUnit periodDurationUnits = TimeUnit.valueOf(PERIOD_DURATION_UNITS.get(properties, String.class)); + int periodDuration = PERIOD_DURATION.get(properties, Integer.class); + this.periodDurationMillis = periodDurationUnits.toMillis(periodDuration); + this.globals = globals; + } + + /** + * Build a profile from a set of message routes. + * + *

This assumes that all of the necessary routes have been provided + * + * @param group The group identifier. + * @param iterator The message routes. + * @return + */ + @Override + public ProfileMeasurementAdapter call(String group, Iterator iterator) throws Exception { + // create the distributor; some settings are unnecessary because it is cleaned-up immediately after processing the batch + int maxRoutes = Integer.MAX_VALUE; + long profileTTLMillis = Long.MAX_VALUE; + MessageDistributor distributor = new DefaultMessageDistributor(periodDurationMillis, profileTTLMillis, maxRoutes); + Context context = TaskUtils.getContext(globals); + + // sort the messages/routes + List routes = toStream(iterator) + .sorted(comparing(rt -> rt.getTimestamp())) + .collect(Collectors.toList()); + LOG.debug("Building a profile for group '{}' from {} message(s)", group, routes.size()); + + // apply each message/route to build the profile + for(MessageRoute route: routes) { + distributor.distribute(route, context); + } + + // flush the profile + List measurements = distributor.flush(); + if(measurements.size() > 1) { + throw new IllegalStateException("No more than 1 profile measurement is expected"); + } + + ProfileMeasurement m = measurements.get(0); + LOG.debug("Profile measurement created; profile={}, entity={}, period={}, value={}", + m.getProfileName(), m.getEntity(), m.getPeriod().getPeriod(), m.getProfileValue()); + return new ProfileMeasurementAdapter(m); + } + + private static Stream toStream(Iterator iterator) { + Iterable iterable = () -> iterator; + return StreamSupport.stream(iterable.spliterator(), false); + } +} diff --git a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/TaskUtils.java b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/TaskUtils.java new file mode 100644 index 0000000000..d401f1251f --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/function/TaskUtils.java @@ -0,0 +1,41 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.metron.profiler.spark.function; + +import org.apache.metron.stellar.dsl.Context; +import org.apache.metron.stellar.dsl.StellarFunctions; + +import java.io.Serializable; +import java.util.Map; + +public class TaskUtils implements Serializable { + + /** + * Create the execution context for running Stellar. + */ + public static Context getContext(Map globals) { + Context context = new Context.Builder() + .with(Context.Capabilities.GLOBAL_CONFIG, () -> globals) + .with(Context.Capabilities.STELLAR_CONFIG, () -> globals) + .build(); + StellarFunctions.initialize(context); + return context; + } +} diff --git a/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/BatchProfilerIntegrationTest.java b/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/BatchProfilerIntegrationTest.java new file mode 100644 index 0000000000..f56074097b --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/BatchProfilerIntegrationTest.java @@ -0,0 +1,111 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.metron.profiler.spark; + +import org.apache.hadoop.hbase.client.Put; +import org.apache.metron.common.configuration.profiler.ProfileConfig; +import org.apache.metron.common.configuration.profiler.ProfilerConfig; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; +import org.apache.metron.hbase.mock.MockHTable; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.SparkSession; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.List; +import java.util.Properties; + +import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_COLUMN_FAMILY; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_TABLE_NAME; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_TABLE_PROVIDER; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.TELEMETRY_INPUT_FORMAT; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.TELEMETRY_INPUT_PATH; +import static org.junit.Assert.assertEquals; + +public class BatchProfilerIntegrationTest { + + private static SparkSession spark; + private MockHTable profilerTable; + private Properties profilerProperties; + + @BeforeClass + public static void setupSpark() { + SparkConf conf = new SparkConf() + .setMaster("local") + .setAppName("BatchProfilerIntegrationTest") + .set("spark.sql.shuffle.partitions", "8"); + spark = SparkSession + .builder() + .config(conf) + .getOrCreate(); + } + + @AfterClass + public static void tearDownSpark() { + if(spark != null) { + spark.close(); + } + } + + @Before + public void setup() { + profilerProperties = new Properties(); + + // define the source of the input telemetry + profilerProperties.put(TELEMETRY_INPUT_PATH.getKey(), "src/test/resources/telemetry.json"); + profilerProperties.put(TELEMETRY_INPUT_FORMAT.getKey(), "text"); + + // define where the output will go + String tableName = HBASE_TABLE_NAME.get(profilerProperties, String.class); + String columnFamily = HBASE_COLUMN_FAMILY.get(profilerProperties, String.class); + profilerProperties.put(HBASE_TABLE_PROVIDER.getKey(), MockHBaseTableProvider.class.getName()); + + // create the mock hbase table + profilerTable = (MockHTable) MockHBaseTableProvider.addToCache(tableName, columnFamily); + } + + @Test + public void testBatchProfiler() { + + // run the batch profiler + BatchProfiler profiler = new BatchProfiler(); + profiler.run(spark, profilerProperties, getGlobals(), getProfile()); + + List puts = profilerTable.getPutLog(); + assertEquals(2, puts.size()); + } + + + private ProfilerConfig getProfile() { + ProfileConfig profile = new ProfileConfig() + .withProfile("profile1") + .withForeach("ip_src_addr") + .withUpdate("count", "count + 1") + .withResult("count"); + return new ProfilerConfig() + .withProfile(profile); + } + + private Properties getGlobals() { + return new Properties(); + } +} diff --git a/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/function/HBaseWriterFunctionTest.java b/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/function/HBaseWriterFunctionTest.java new file mode 100644 index 0000000000..55f3e21c54 --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/function/HBaseWriterFunctionTest.java @@ -0,0 +1,176 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.metron.profiler.spark.function; + +import org.apache.commons.collections4.IteratorUtils; +import org.apache.metron.common.configuration.profiler.ProfileConfig; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; +import org.apache.metron.profiler.ProfileMeasurement; +import org.apache.metron.profiler.spark.ProfileMeasurementAdapter; +import org.json.simple.JSONObject; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_COLUMN_FAMILY; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.HBASE_TABLE_NAME; + +public class HBaseWriterFunctionTest { + + Properties profilerProperties; + + @Before + public void setup() { + profilerProperties = getProfilerProperties(); + + // create a mock table for HBase + String tableName = HBASE_TABLE_NAME.get(profilerProperties, String.class); + String columnFamily = HBASE_COLUMN_FAMILY.get(profilerProperties, String.class); + MockHBaseTableProvider.addToCache(tableName, columnFamily); + } + + @Test + public void testWrite() throws Exception { + + JSONObject message = getMessage(); + String entity = (String) message.get("ip_src_addr"); + long timestamp = (Long) message.get("timestamp"); + ProfileConfig profile = getProfile(); + + // setup the profile measurements that will be written + List measurements = createMeasurements(1, entity, timestamp, profile); + + // setup the function to test + HBaseWriterFunction function = new HBaseWriterFunction(profilerProperties); + function.withTableProviderImpl(MockHBaseTableProvider.class.getName()); + + // write the measurements + Iterator results = function.call(measurements.iterator()); + + // validate the result + List counts = IteratorUtils.toList(results); + Assert.assertEquals(1, counts.size()); + Assert.assertEquals(1, counts.get(0).intValue()); + } + + @Test + public void testWriteMany() throws Exception { + + JSONObject message = getMessage(); + String entity = (String) message.get("ip_src_addr"); + long timestamp = (Long) message.get("timestamp"); + ProfileConfig profile = getProfile(); + + // setup the profile measurements that will be written + List measurements = createMeasurements(10, entity, timestamp, profile); + + // setup the function to test + HBaseWriterFunction function = new HBaseWriterFunction(profilerProperties); + function.withTableProviderImpl(MockHBaseTableProvider.class.getName()); + + // write the measurements + Iterator results = function.call(measurements.iterator()); + + // validate the result + List counts = IteratorUtils.toList(results); + Assert.assertEquals(1, counts.size()); + Assert.assertEquals(10, counts.get(0).intValue()); + } + + @Test + public void testWriteNone() throws Exception { + + // there are no profile measurements to write + List measurements = new ArrayList<>(); + + // setup the function to test + HBaseWriterFunction function = new HBaseWriterFunction(profilerProperties); + function.withTableProviderImpl(MockHBaseTableProvider.class.getName()); + + // write the measurements + Iterator results = function.call(measurements.iterator()); + + // validate the result + List counts = IteratorUtils.toList(results); + Assert.assertEquals(1, counts.size()); + Assert.assertEquals(0, counts.get(0).intValue()); + } + + /** + * Create a list of measurements for testing. + * + * @param count The number of messages to create. + * @param entity The entity. + * @param timestamp The timestamp. + * @param profile The profile definition. + * @return + */ + private List createMeasurements(int count, String entity, long timestamp, ProfileConfig profile) { + List measurements = new ArrayList<>(); + + for(int i=0; i iter = function.call(goodMessage); + + List routes = Lists.newArrayList(iter); + Assert.assertEquals(1, routes.size()); + Assert.assertEquals("profile1", routes.get(0).getProfileDefinition().getProfile()); + } + + /** + * A bad or invalid message should return no routes. + */ + @Test + public void testWithBadMessage() throws Exception { + MessageRouterFunction function = new MessageRouterFunction(oneProfile(), getGlobals()); + Iterator iter = function.call(badMessage); + + List routes = Lists.newArrayList(iter); + Assert.assertEquals(0, routes.size()); + } + + @Test + public void testFindMultipleRoutes() throws Exception { + MessageRouterFunction function = new MessageRouterFunction(twoProfiles(), getGlobals()); + Iterator iter = function.call(goodMessage); + + List routes = Lists.newArrayList(iter); + Assert.assertEquals(2, routes.size()); + Assert.assertEquals("profile1", routes.get(0).getProfileDefinition().getProfile()); + Assert.assertEquals("profile2", routes.get(1).getProfileDefinition().getProfile()); + } + + private ProfilerConfig oneProfile() { + ProfileConfig profile = new ProfileConfig() + .withProfile("profile1") + .withForeach("ip_src_addr") + .withUpdate("count", "count + 1") + .withResult("count"); + + return new ProfilerConfig() + .withProfile(profile); + } + + private ProfilerConfig twoProfiles() { + ProfileConfig profile1 = new ProfileConfig() + .withProfile("profile1") + .withForeach("ip_src_addr") + .withUpdate("count", "count + 1") + .withResult("count"); + ProfileConfig profile2 = new ProfileConfig() + .withProfile("profile2") + .withForeach("ip_src_addr") + .withUpdate("count", "count + 1") + .withResult("count"); + return new ProfilerConfig() + .withProfile(profile1) + .withProfile(profile2); + } + + private Map getGlobals() { + return Collections.emptyMap(); + } +} diff --git a/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/function/ProfileBuilderFunctionTest.java b/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/function/ProfileBuilderFunctionTest.java new file mode 100644 index 0000000000..d5a4dba2f5 --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/function/ProfileBuilderFunctionTest.java @@ -0,0 +1,98 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.metron.profiler.spark.function; + +import org.apache.metron.common.configuration.profiler.ProfileConfig; +import org.apache.metron.profiler.MessageRoute; +import org.apache.metron.profiler.ProfilePeriod; +import org.apache.metron.profiler.spark.ProfileMeasurementAdapter; +import org.json.simple.JSONObject; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +import static org.apache.metron.profiler.spark.BatchProfilerConfig.PERIOD_DURATION; +import static org.apache.metron.profiler.spark.BatchProfilerConfig.PERIOD_DURATION_UNITS; + +public class ProfileBuilderFunctionTest { + + @Test + public void testBuildProfile() throws Exception { + // setup the message and profile + JSONObject message = getMessage(); + String entity = "192.168.1.1"; + long timestamp = (Long) message.get("timestamp"); + ProfileConfig profile = getProfile(); + + // setup the route + MessageRoute route = new MessageRoute(profile, entity, message, timestamp); + List routes = new ArrayList(); + routes.add(route); + routes.add(route); + routes.add(route); + Properties profilerProperties = getProfilerProperties(); + + // setup the period + int periodDuration = PERIOD_DURATION.get(profilerProperties, Integer.class); + TimeUnit periodDurationUnits = TimeUnit.valueOf(PERIOD_DURATION_UNITS.get(profilerProperties, String.class)); + ProfilePeriod expectedPeriod = ProfilePeriod.fromTimestamp(timestamp, periodDuration, periodDurationUnits); + + // build the profile + ProfileBuilderFunction function = new ProfileBuilderFunction(profilerProperties, getGlobals()); + ProfileMeasurementAdapter measurement = function.call("profile1-192.168.1.1-0", routes.iterator()); + + // validate the measurement + Assert.assertEquals(entity, measurement.getEntity()); + Assert.assertEquals(profile.getProfile(), measurement.getProfileName()); + Assert.assertEquals(routes.size(), measurement.toProfileMeasurement().getProfileValue()); + Assert.assertEquals(expectedPeriod.getPeriod(), (long) measurement.getPeriodId()); + } + + private JSONObject getMessage() { + JSONObject message = new JSONObject(); + message.put("ip_src_addr", "192.168.1.1"); + message.put("status", "red"); + message.put("timestamp", System.currentTimeMillis()); + return message; + } + + private Properties getProfilerProperties() { + return new Properties(); + } + + private Map getGlobals() { + return Collections.emptyMap(); + } + + private ProfileConfig getProfile() { + return new ProfileConfig() + .withProfile("profile1") + .withForeach("ip_src_addr") + .withUpdate("count", "count + 1") + .withResult("count"); + + } +} diff --git a/metron-analytics/metron-profiler-spark/src/test/resources/log4j.properties b/metron-analytics/metron-profiler-spark/src/test/resources/log4j.properties new file mode 100644 index 0000000000..c3db0affdc --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/test/resources/log4j.properties @@ -0,0 +1,31 @@ +# +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# + +# Root logger option +log4j.rootLogger=ERROR, stdout + +# Direct log messages to stdout +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.Target=System.out +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n + +log4j.logger.org.apache.metron.profiler=ERROR +log4j.logger.org.apache.metron.profiler.spark=ERROR \ No newline at end of file diff --git a/metron-analytics/metron-profiler-spark/src/test/resources/telemetry.json b/metron-analytics/metron-profiler-spark/src/test/resources/telemetry.json new file mode 100644 index 0000000000..4a324cf434 --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/test/resources/telemetry.json @@ -0,0 +1,100 @@ +{"adapter.threatinteladapter.end.ts":"1530978697769","qclass_name":"qclass-32769","bro_timestamp":"1530978687.836793","qtype_name":"PTR","ip_dst_port":5353,"enrichmentsplitterbolt.splitter.end.ts":"1530978696551","qtype":12,"rejected":false,"enrichmentsplitterbolt.splitter.begin.ts":"1530978696550","adapter.hostfromjsonlistadapter.end.ts":"1530978696606","trans_id":0,"adapter.geoadapter.begin.ts":"1530978696857","uid":"CGs8rS1rqhyXRRgA64","protocol":"dns","original_string":"DNS | AA:false qclass_name:qclass-32769 id.orig_p:5353 qtype_name:PTR qtype:12 rejected:false id.resp_p:5353 query:_googlecast._tcp.local trans_id:0 TC:false RA:false uid:CGs8rS1rqhyXRRgA64 RD:false proto:udp id.orig_h:192.168.66.1 Z:0 qclass:32769 ts:1530978687.836793 id.resp_h:224.0.0.251","ip_dst_addr":"224.0.0.251","threatinteljoinbolt.joiner.ts":"1530978697808","enrichmentjoinbolt.joiner.ts":"1530978696932","adapter.hostfromjsonlistadapter.begin.ts":"1530978696606","threatintelsplitterbolt.splitter.begin.ts":"1530978696949","Z":0,"ip_src_addr":"192.168.66.1","qclass":32769,"timestamp":1530978687836,"AA":false,"query":"_googlecast._tcp.local","TC":false,"RA":false,"source.type":"bro","adapter.geoadapter.end.ts":"1530978696857","RD":false,"threatintelsplitterbolt.splitter.end.ts":"1530978696952","adapter.threatinteladapter.begin.ts":"1530978697764","ip_src_port":5353,"proto":"udp","guid":"90751ce5-703d-4b9f-8c2d-8e5c42e72262"} +{"adapter.threatinteladapter.end.ts":"1530978697772","bro_timestamp":"1530978687.77394","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978696605","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978696605","adapter.hostfromjsonlistadapter.end.ts":"1530978696649","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978696857","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"CBJatv2DcsW8fow3Dg","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49186 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:CBJatv2DcsW8fow3Dg referrer:http:\/\/va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in\/?285a4d4e4e5a4d4d4649584c5d43064b4745 resp_mime_types:[\"text\\\/html\"] trans_depth:1 host:r03afd2.c3008e.xc07r.b0f.a39.h7f0fa5eu.vb8fbl.e8mfzdgrf7g0.groupprograms.in status_msg:OK id.orig_h:192.168.138.158 response_body_len:121635 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978687.77394 id.resp_h:62.75.195.236 resp_fuids:[\"F77a061yn9H0cUBGVa\"]","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978697808","host":"r03afd2.c3008e.xc07r.b0f.a39.h7f0fa5eu.vb8fbl.e8mfzdgrf7g0.groupprograms.in","enrichmentjoinbolt.joiner.ts":"1530978696943","adapter.hostfromjsonlistadapter.begin.ts":"1530978696607","threatintelsplitterbolt.splitter.begin.ts":"1530978696952","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["F77a061yn9H0cUBGVa"],"timestamp":1530978687773,"method":"GET","request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978696858","referrer":"http:\/\/va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in\/?285a4d4e4e5a4d4d4649584c5d43064b4745","threatintelsplitterbolt.splitter.end.ts":"1530978696952","adapter.threatinteladapter.begin.ts":"1530978697769","ip_src_port":49186,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"f5b315b0-e776-481a-9f28-765fdb19e6e8","response_body_len":121635} +{"adapter.threatinteladapter.end.ts":"1530978697776","bro_timestamp":"1530978687.916811","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978696606","enrichmentsplitterbolt.splitter.begin.ts":"1530978696606","adapter.hostfromjsonlistadapter.end.ts":"1530978696650","adapter.geoadapter.begin.ts":"1530978696858","uid":"CUrRne3iLIxXavQtci","trans_depth":6,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/clusters\/metron_cluster\/components\/?ServiceComponentInfo\/component_name=APP_TIMELINE_SERVER|ServiceComponentInfo\/category=MASTER&fields=ServiceComponentInfo\/service_name,host_components\/HostRoles\/display_name,host_components\/HostRoles\/host_name,host_components\/HostRoles\/state,host_components\/HostRoles\/maintenance_state,host_components\/HostRoles\/stale_configs,host_components\/HostRoles\/ha_state,host_components\/HostRoles\/desired_admin_state,,host_components\/metrics\/jvm\/memHeapUsedM,host_components\/metrics\/jvm\/HeapMemoryMax,host_components\/metrics\/jvm\/HeapMemoryUsed,host_components\/metrics\/jvm\/memHeapCommittedM,host_components\/metrics\/mapred\/jobtracker\/trackers_decommissioned,host_components\/metrics\/cpu\/cpu_wio,host_components\/metrics\/rpc\/client\/RpcQueueTime_avg_time,host_components\/metrics\/dfs\/FSNamesystem\/*,host_components\/metrics\/dfs\/namenode\/Version,host_components\/metrics\/dfs\/namenode\/LiveNodes,host_components\/metrics\/dfs\/namenode\/DeadNodes,host_components\/metrics\/dfs\/namenode\/DecomNodes,host_components\/metrics\/dfs\/namenode\/TotalFiles,host_components\/metrics\/dfs\/namenode\/UpgradeFinalized,host_components\/metrics\/dfs\/namenode\/Safemode,host_components\/metrics\/runtime\/StartTime,host_components\/metrics\/hbase\/master\/IsActiveMaster,host_components\/metrics\/hbase\/master\/MasterStartTime,host_components\/metrics\/hbase\/master\/MasterActiveTime,host_components\/metrics\/hbase\/master\/AverageLoad,host_components\/metrics\/master\/AssignmentManger\/ritCount,metrics\/api\/v1\/cluster\/summary,metrics\/api\/v1\/topology\/summary,metrics\/api\/v1\/nimbus\/summary,host_components\/metrics\/yarn\/Queue,host_components\/metrics\/yarn\/ClusterMetrics\/NumActiveNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumLostNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumUnhealthyNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumRebootedNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumDecommissionedNMs&minimal_response=true&_=1484168361295 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:6 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978687.916811 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978697808","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978696948","adapter.hostfromjsonlistadapter.begin.ts":"1530978696649","threatintelsplitterbolt.splitter.begin.ts":"1530978696953","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978687916,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/clusters\/metron_cluster\/components\/?ServiceComponentInfo\/component_name=APP_TIMELINE_SERVER|ServiceComponentInfo\/category=MASTER&fields=ServiceComponentInfo\/service_name,host_components\/HostRoles\/display_name,host_components\/HostRoles\/host_name,host_components\/HostRoles\/state,host_components\/HostRoles\/maintenance_state,host_components\/HostRoles\/stale_configs,host_components\/HostRoles\/ha_state,host_components\/HostRoles\/desired_admin_state,,host_components\/metrics\/jvm\/memHeapUsedM,host_components\/metrics\/jvm\/HeapMemoryMax,host_components\/metrics\/jvm\/HeapMemoryUsed,host_components\/metrics\/jvm\/memHeapCommittedM,host_components\/metrics\/mapred\/jobtracker\/trackers_decommissioned,host_components\/metrics\/cpu\/cpu_wio,host_components\/metrics\/rpc\/client\/RpcQueueTime_avg_time,host_components\/metrics\/dfs\/FSNamesystem\/*,host_components\/metrics\/dfs\/namenode\/Version,host_components\/metrics\/dfs\/namenode\/LiveNodes,host_components\/metrics\/dfs\/namenode\/DeadNodes,host_components\/metrics\/dfs\/namenode\/DecomNodes,host_components\/metrics\/dfs\/namenode\/TotalFiles,host_components\/metrics\/dfs\/namenode\/UpgradeFinalized,host_components\/metrics\/dfs\/namenode\/Safemode,host_components\/metrics\/runtime\/StartTime,host_components\/metrics\/hbase\/master\/IsActiveMaster,host_components\/metrics\/hbase\/master\/MasterStartTime,host_components\/metrics\/hbase\/master\/MasterActiveTime,host_components\/metrics\/hbase\/master\/AverageLoad,host_components\/metrics\/master\/AssignmentManger\/ritCount,metrics\/api\/v1\/cluster\/summary,metrics\/api\/v1\/topology\/summary,metrics\/api\/v1\/nimbus\/summary,host_components\/metrics\/yarn\/Queue,host_components\/metrics\/yarn\/ClusterMetrics\/NumActiveNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumLostNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumUnhealthyNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumRebootedNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumDecommissionedNMs&minimal_response=true&_=1484168361295","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978696858","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978696953","adapter.threatinteladapter.begin.ts":"1530978697772","ip_src_port":50451,"guid":"db5e7329-9439-4a8a-972b-05d22d08e1fa","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978697777","bro_timestamp":"1530978687.073175","status_code":404,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978696609","enrichments.geo.ip_dst_addr.city":"Phoenix","enrichments.geo.ip_dst_addr.latitude":"33.4499","enrichmentsplitterbolt.splitter.begin.ts":"1530978696609","adapter.hostfromjsonlistadapter.end.ts":"1530978696650","enrichments.geo.ip_dst_addr.country":"US","enrichments.geo.ip_dst_addr.locID":"5308655","adapter.geoadapter.begin.ts":"1530978696858","enrichments.geo.ip_dst_addr.postalCode":"85004","uid":"CxQY13LFLIWBK5kw6","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49203 status_code:404 method:POST request_body_len:110 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/wp-content\/themes\/twentyfifteen\/img5.php?f=ka6nnuvccqlw9 tags:[] uid:CxQY13LFLIWBK5kw6 resp_mime_types:[\"text\\\/html\"] trans_depth:1 orig_fuids:[\"FUF7cQ2NWtIJObUXFf\"] host:runlove.us status_msg:Not Found id.orig_h:192.168.138.158 response_body_len:357 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978687.073175 id.resp_h:204.152.254.221 resp_fuids:[\"FNXPE1PFFrR89EeJa\"]","ip_dst_addr":"204.152.254.221","threatinteljoinbolt.joiner.ts":"1530978697808","enrichments.geo.ip_dst_addr.dmaCode":"753","host":"runlove.us","enrichmentjoinbolt.joiner.ts":"1530978696948","adapter.hostfromjsonlistadapter.begin.ts":"1530978696650","threatintelsplitterbolt.splitter.begin.ts":"1530978696953","enrichments.geo.ip_dst_addr.longitude":"-112.0712","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FNXPE1PFFrR89EeJa"],"timestamp":1530978687073,"method":"POST","request_body_len":110,"orig_mime_types":["text\/plain"],"uri":"\/wp-content\/themes\/twentyfifteen\/img5.php?f=ka6nnuvccqlw9","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978696870","threatintelsplitterbolt.splitter.end.ts":"1530978696953","adapter.threatinteladapter.begin.ts":"1530978697776","orig_fuids":["FUF7cQ2NWtIJObUXFf"],"ip_src_port":49203,"enrichments.geo.ip_dst_addr.location_point":"33.4499,-112.0712","status_msg":"Not Found","guid":"1d9eefeb-832b-4262-a800-5b67da9f7277","response_body_len":357} +{"adapter.threatinteladapter.end.ts":"1530978697780","bro_timestamp":"1530978687.027914","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978696609","enrichments.geo.ip_dst_addr.city":"Los Angeles","enrichments.geo.ip_dst_addr.latitude":"34.0494","enrichmentsplitterbolt.splitter.begin.ts":"1530978696609","adapter.hostfromjsonlistadapter.end.ts":"1530978696651","enrichments.geo.ip_dst_addr.country":"US","enrichments.geo.ip_dst_addr.locID":"5368361","adapter.geoadapter.begin.ts":"1530978696870","enrichments.geo.ip_dst_addr.postalCode":"90014","uid":"CxZIVD4f5vBwpXUjwf","resp_mime_types":["text\/plain"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49198 status_code:200 method:POST request_body_len:134 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/wp-content\/themes\/grizzly\/img5.php?c=cdcnw7cfz43rmtg tags:[] uid:CxZIVD4f5vBwpXUjwf resp_mime_types:[\"text\\\/plain\"] trans_depth:1 orig_fuids:[\"FiPZ8g4gdpjEyHuez2\"] host:comarksecurity.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:14 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978687.027914 id.resp_h:72.34.49.86 resp_fuids:[\"FM8l2i6ib3vOd45ob\"]","ip_dst_addr":"72.34.49.86","threatinteljoinbolt.joiner.ts":"1530978697808","enrichments.geo.ip_dst_addr.dmaCode":"803","host":"comarksecurity.com","enrichmentjoinbolt.joiner.ts":"1530978696949","adapter.hostfromjsonlistadapter.begin.ts":"1530978696650","threatintelsplitterbolt.splitter.begin.ts":"1530978696953","enrichments.geo.ip_dst_addr.longitude":"-118.2641","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FM8l2i6ib3vOd45ob"],"timestamp":1530978687027,"method":"POST","request_body_len":134,"orig_mime_types":["text\/plain"],"uri":"\/wp-content\/themes\/grizzly\/img5.php?c=cdcnw7cfz43rmtg","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978696875","threatintelsplitterbolt.splitter.end.ts":"1530978696953","adapter.threatinteladapter.begin.ts":"1530978697778","orig_fuids":["FiPZ8g4gdpjEyHuez2"],"ip_src_port":49198,"enrichments.geo.ip_dst_addr.location_point":"34.0494,-118.2641","status_msg":"OK","guid":"0c21f313-5cb7-46de-b62a-b429c565bfb0","response_body_len":14} +{"adapter.threatinteladapter.end.ts":"1530978697782","bro_timestamp":"1530978687.58302","ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978696609","enrichments.geo.ip_dst_addr.latitude":"48.8582","enrichmentsplitterbolt.splitter.begin.ts":"1530978696609","adapter.hostfromjsonlistadapter.end.ts":"1530978696651","enrichments.geo.ip_dst_addr.country":"FR","adapter.geoadapter.begin.ts":"1530978696875","uid":"CT2ax04BCxPW20AlGc","trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49195 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:CT2ax04BCxPW20AlGc trans_depth:1 host:ip-addr.es id.orig_h:192.168.138.158 response_body_len:0 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978687.58302 id.resp_h:188.165.164.184","ip_dst_addr":"188.165.164.184","threatinteljoinbolt.joiner.ts":"1530978697809","host":"ip-addr.es","enrichmentjoinbolt.joiner.ts":"1530978696949","adapter.hostfromjsonlistadapter.begin.ts":"1530978696651","threatintelsplitterbolt.splitter.begin.ts":"1530978696953","enrichments.geo.ip_dst_addr.longitude":"2.3387000000000002","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","timestamp":1530978687583,"method":"GET","request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978696879","threatintelsplitterbolt.splitter.end.ts":"1530978696953","adapter.threatinteladapter.begin.ts":"1530978697780","ip_src_port":49195,"enrichments.geo.ip_dst_addr.location_point":"48.8582,2.3387000000000002","guid":"ed0d58c1-88cb-4f4e-ab7b-ee1a36e7cdcb","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978697783","qclass_name":"C_INTERNET","bro_timestamp":"1530978687.445971","qtype_name":"PTR","ip_dst_port":5353,"enrichmentsplitterbolt.splitter.end.ts":"1530978696610","qtype":12,"rejected":false,"enrichmentsplitterbolt.splitter.begin.ts":"1530978696610","adapter.hostfromjsonlistadapter.end.ts":"1530978696651","trans_id":0,"adapter.geoadapter.begin.ts":"1530978696879","uid":"ChMDrL20pLP4UzCncj","protocol":"dns","original_string":"DNS | AA:false qclass_name:C_INTERNET id.orig_p:5353 qtype_name:PTR qtype:12 rejected:false id.resp_p:5353 query:_googlecast._tcp.local trans_id:0 TC:false RA:false uid:ChMDrL20pLP4UzCncj RD:false proto:udp id.orig_h:192.168.66.1 Z:0 qclass:1 ts:1530978687.445971 id.resp_h:224.0.0.251","ip_dst_addr":"224.0.0.251","threatinteljoinbolt.joiner.ts":"1530978697809","enrichmentjoinbolt.joiner.ts":"1530978696949","adapter.hostfromjsonlistadapter.begin.ts":"1530978696651","threatintelsplitterbolt.splitter.begin.ts":"1530978696953","Z":0,"ip_src_addr":"192.168.66.1","qclass":1,"timestamp":1530978687445,"AA":false,"query":"_googlecast._tcp.local","TC":false,"RA":false,"source.type":"bro","adapter.geoadapter.end.ts":"1530978696879","RD":false,"threatintelsplitterbolt.splitter.end.ts":"1530978696953","adapter.threatinteladapter.begin.ts":"1530978697783","ip_src_port":5353,"proto":"udp","guid":"a6f4fe3a-c485-4521-bcfe-b2600746885e"} +{"TTLs":[21599.0],"adapter.threatinteladapter.end.ts":"1530978697784","qclass_name":"C_INTERNET","bro_timestamp":"1530978687.053752","qtype_name":"A","ip_dst_port":53,"enrichmentsplitterbolt.splitter.end.ts":"1530978696610","qtype":1,"rejected":false,"answers":["188.165.164.184"],"enrichmentsplitterbolt.splitter.begin.ts":"1530978696610","adapter.hostfromjsonlistadapter.end.ts":"1530978696652","trans_id":15553,"adapter.geoadapter.begin.ts":"1530978696879","uid":"CoiTkw2sb9stNr10zg","protocol":"dns","original_string":"DNS | AA:false TTLs:[21599.0] qclass_name:C_INTERNET id.orig_p:53571 qtype_name:A qtype:1 rejected:false id.resp_p:53 query:ip-addr.es answers:[\"188.165.164.184\"] trans_id:15553 rcode:0 rcode_name:NOERROR TC:false RA:true uid:CoiTkw2sb9stNr10zg RD:true proto:udp id.orig_h:192.168.138.158 Z:0 qclass:1 ts:1530978687.053752 id.resp_h:192.168.138.2","ip_dst_addr":"192.168.138.2","threatinteljoinbolt.joiner.ts":"1530978697809","enrichmentjoinbolt.joiner.ts":"1530978696953","adapter.hostfromjsonlistadapter.begin.ts":"1530978696652","threatintelsplitterbolt.splitter.begin.ts":"1530978696961","Z":0,"ip_src_addr":"192.168.138.158","qclass":1,"timestamp":1530978687053,"AA":false,"query":"ip-addr.es","rcode":0,"rcode_name":"NOERROR","TC":false,"RA":true,"source.type":"bro","adapter.geoadapter.end.ts":"1530978696879","RD":true,"threatintelsplitterbolt.splitter.end.ts":"1530978696961","adapter.threatinteladapter.begin.ts":"1530978697783","ip_src_port":53571,"proto":"udp","guid":"bbfd5e54-db09-455e-b01f-b6cbbd444e88"} +{"adapter.threatinteladapter.end.ts":"1530978697784","bro_timestamp":"1530978687.267256","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978696610","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978696610","adapter.hostfromjsonlistadapter.end.ts":"1530978696652","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978696880","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"CID7qb45BoqLfAMHic","trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49193 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/?34eaf8bd50d85d8c6baacb45f0a7b22e tags:[] uid:CID7qb45BoqLfAMHic trans_depth:1 host:62.75.195.236 status_msg:OK id.orig_h:192.168.138.158 response_body_len:0 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978687.267256 id.resp_h:62.75.195.236","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978697809","host":"62.75.195.236","enrichmentjoinbolt.joiner.ts":"1530978696953","adapter.hostfromjsonlistadapter.begin.ts":"1530978696652","threatintelsplitterbolt.splitter.begin.ts":"1530978696961","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","timestamp":1530978687267,"method":"GET","request_body_len":0,"uri":"\/?34eaf8bd50d85d8c6baacb45f0a7b22e","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978696880","threatintelsplitterbolt.splitter.end.ts":"1530978696961","adapter.threatinteladapter.begin.ts":"1530978697784","ip_src_port":49193,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"ad2f6714-2a4a-4262-8ce0-1940f3e8f340","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978697786","bro_timestamp":"1530978687.417086","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978696610","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978696610","adapter.hostfromjsonlistadapter.end.ts":"1530978696652","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978696880","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CEkDUW1JYqnTIkYzc1","resp_mime_types":["image\/png"],"trans_depth":2,"protocol":"http","original_string":"HTTP | id.orig_p:49210 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/lb.png tags:[] uid:CEkDUW1JYqnTIkYzc1 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:2 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:239 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978687.417086 id.resp_h:95.163.121.204 resp_fuids:[\"FZy6Lx4RGFmQ1AZeU8\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978697809","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978696953","adapter.hostfromjsonlistadapter.begin.ts":"1530978696652","threatintelsplitterbolt.splitter.begin.ts":"1530978696961","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FZy6Lx4RGFmQ1AZeU8"],"timestamp":1530978687417,"method":"GET","request_body_len":0,"uri":"\/img\/lb.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978696887","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978696961","adapter.threatinteladapter.begin.ts":"1530978697784","ip_src_port":49210,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"9711e04e-9926-4606-a7e8-e719dac535e6","response_body_len":239} +{"adapter.threatinteladapter.end.ts":"1530978697786","qclass_name":"C_INTERNET","bro_timestamp":"1530978694.884106","qtype_name":"PTR","ip_dst_port":5353,"enrichmentsplitterbolt.splitter.end.ts":"1530978696611","qtype":12,"rejected":false,"enrichmentsplitterbolt.splitter.begin.ts":"1530978696611","adapter.hostfromjsonlistadapter.end.ts":"1530978696652","trans_id":0,"adapter.geoadapter.begin.ts":"1530978696887","uid":"CkwtUK1ANyyZwj0PW1","protocol":"dns","original_string":"DNS | AA:false qclass_name:C_INTERNET id.orig_p:5353 qtype_name:PTR qtype:12 rejected:false id.resp_p:5353 query:_googlecast._tcp.local trans_id:0 TC:false RA:false uid:CkwtUK1ANyyZwj0PW1 RD:false proto:udp id.orig_h:192.168.66.1 Z:0 qclass:1 ts:1530978694.884106 id.resp_h:224.0.0.251","ip_dst_addr":"224.0.0.251","threatinteljoinbolt.joiner.ts":"1530978697809","enrichmentjoinbolt.joiner.ts":"1530978696953","adapter.hostfromjsonlistadapter.begin.ts":"1530978696652","threatintelsplitterbolt.splitter.begin.ts":"1530978696961","Z":0,"ip_src_addr":"192.168.66.1","qclass":1,"timestamp":1530978694884,"AA":false,"query":"_googlecast._tcp.local","TC":false,"RA":false,"source.type":"bro","adapter.geoadapter.end.ts":"1530978696887","RD":false,"threatintelsplitterbolt.splitter.end.ts":"1530978696961","adapter.threatinteladapter.begin.ts":"1530978697786","ip_src_port":5353,"proto":"udp","guid":"f9c14d84-59c5-4598-97b7-5d6e95aba4e6"} +{"adapter.threatinteladapter.end.ts":"1530978697786","bro_timestamp":"1530978694.621046","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978696611","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978696611","adapter.hostfromjsonlistadapter.end.ts":"1530978696653","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978696887","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"C1ia4w3K4ngOWPmAsi","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49186 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:C1ia4w3K4ngOWPmAsi referrer:http:\/\/va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in\/?285a4d4e4e5a4d4d4649584c5d43064b4745 resp_mime_types:[\"text\\\/html\"] trans_depth:1 host:r03afd2.c3008e.xc07r.b0f.a39.h7f0fa5eu.vb8fbl.e8mfzdgrf7g0.groupprograms.in status_msg:OK id.orig_h:192.168.138.158 response_body_len:121635 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978694.621046 id.resp_h:62.75.195.236 resp_fuids:[\"FKGK2W1X8Bfk7D7XD9\"]","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978697809","host":"r03afd2.c3008e.xc07r.b0f.a39.h7f0fa5eu.vb8fbl.e8mfzdgrf7g0.groupprograms.in","enrichmentjoinbolt.joiner.ts":"1530978696953","adapter.hostfromjsonlistadapter.begin.ts":"1530978696652","threatintelsplitterbolt.splitter.begin.ts":"1530978696962","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FKGK2W1X8Bfk7D7XD9"],"timestamp":1530978694621,"method":"GET","request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978696887","referrer":"http:\/\/va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in\/?285a4d4e4e5a4d4d4649584c5d43064b4745","threatintelsplitterbolt.splitter.end.ts":"1530978696962","adapter.threatinteladapter.begin.ts":"1530978697786","ip_src_port":49186,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"98769db4-ee20-4f69-bb04-2e7005de9c6d","response_body_len":121635} +{"adapter.threatinteladapter.end.ts":"1530978697787","bro_timestamp":"1530978694.641679","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978696611","enrichmentsplitterbolt.splitter.begin.ts":"1530978696611","adapter.hostfromjsonlistadapter.end.ts":"1530978696653","adapter.geoadapter.begin.ts":"1530978696887","uid":"CUrRne3iLIxXavQtci","trans_depth":254,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/persist\/wizard-data?_=1484169473684 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:254 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978694.641679 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978697810","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978696954","adapter.hostfromjsonlistadapter.begin.ts":"1530978696653","threatintelsplitterbolt.splitter.begin.ts":"1530978696962","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978694641,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/persist\/wizard-data?_=1484169473684","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978696887","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978696962","adapter.threatinteladapter.begin.ts":"1530978697787","ip_src_port":50451,"guid":"ffda80d8-44e3-42db-b72a-d2fa7cf38042","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978697788","bro_timestamp":"1530978694.388009","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978696611","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978696611","adapter.hostfromjsonlistadapter.end.ts":"1530978696653","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978696887","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CJyQ1119VSEe7SGiTa","resp_mime_types":["image\/x-icon"],"trans_depth":2,"protocol":"http","original_string":"HTTP | id.orig_p:49207 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/favicon.ico tags:[] uid:CJyQ1119VSEe7SGiTa resp_mime_types:[\"image\\\/x-icon\"] trans_depth:2 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:318 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978694.388009 id.resp_h:95.163.121.204 resp_fuids:[\"F0Pvmv1dj2gRa9c7v\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978697810","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978696954","adapter.hostfromjsonlistadapter.begin.ts":"1530978696653","threatintelsplitterbolt.splitter.begin.ts":"1530978696962","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["F0Pvmv1dj2gRa9c7v"],"timestamp":1530978694388,"method":"GET","request_body_len":0,"uri":"\/favicon.ico","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978696887","threatintelsplitterbolt.splitter.end.ts":"1530978696962","adapter.threatinteladapter.begin.ts":"1530978697787","ip_src_port":49207,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"66eda80b-7f24-4aec-85b9-e381e128dfc7","response_body_len":318} +{"adapter.threatinteladapter.end.ts":"1530978697788","bro_timestamp":"1530978694.979947","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978696611","enrichments.geo.ip_dst_addr.city":"Los Angeles","enrichments.geo.ip_dst_addr.latitude":"34.0494","enrichmentsplitterbolt.splitter.begin.ts":"1530978696611","adapter.hostfromjsonlistadapter.end.ts":"1530978696653","enrichments.geo.ip_dst_addr.country":"US","enrichments.geo.ip_dst_addr.locID":"5368361","adapter.geoadapter.begin.ts":"1530978696887","enrichments.geo.ip_dst_addr.postalCode":"90014","uid":"COZAhy4ljJ4lBc5bgf","resp_mime_types":["text\/plain"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49204 status_code:200 method:POST request_body_len:110 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/wp-content\/themes\/grizzly\/img5.php?u=ka6nnuvccqlw9 tags:[] uid:COZAhy4ljJ4lBc5bgf resp_mime_types:[\"text\\\/plain\"] trans_depth:1 orig_fuids:[\"FgncKy2eauwZjDL6h9\"] host:comarksecurity.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:14 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978694.979947 id.resp_h:72.34.49.86 resp_fuids:[\"FDVxtiyWLP0KeNRg8\"]","ip_dst_addr":"72.34.49.86","threatinteljoinbolt.joiner.ts":"1530978697810","enrichments.geo.ip_dst_addr.dmaCode":"803","host":"comarksecurity.com","enrichmentjoinbolt.joiner.ts":"1530978696954","adapter.hostfromjsonlistadapter.begin.ts":"1530978696653","threatintelsplitterbolt.splitter.begin.ts":"1530978696962","enrichments.geo.ip_dst_addr.longitude":"-118.2641","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FDVxtiyWLP0KeNRg8"],"timestamp":1530978694979,"method":"POST","request_body_len":110,"orig_mime_types":["text\/plain"],"uri":"\/wp-content\/themes\/grizzly\/img5.php?u=ka6nnuvccqlw9","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978696887","threatintelsplitterbolt.splitter.end.ts":"1530978696962","adapter.threatinteladapter.begin.ts":"1530978697788","orig_fuids":["FgncKy2eauwZjDL6h9"],"ip_src_port":49204,"enrichments.geo.ip_dst_addr.location_point":"34.0494,-118.2641","status_msg":"OK","guid":"a5bc5b67-a861-43e2-9232-fb902239cea3","response_body_len":14} +{"adapter.threatinteladapter.end.ts":"1530978702605","bro_timestamp":"1530978694.045879","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702588","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978702588","adapter.hostfromjsonlistadapter.end.ts":"1530978702593","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978702593","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CmNSa535EEM4iN5uwh","resp_mime_types":["image\/png"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49209 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/flags\/de.png tags:[] uid:CmNSa535EEM4iN5uwh referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:1 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:534 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978694.045879 id.resp_h:95.163.121.204 resp_fuids:[\"FZgahz2hSOfoAP9y1l\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978702609","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978702598","adapter.hostfromjsonlistadapter.begin.ts":"1530978702593","threatintelsplitterbolt.splitter.begin.ts":"1530978702601","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FZgahz2hSOfoAP9y1l"],"timestamp":1530978694045,"method":"GET","request_body_len":0,"uri":"\/img\/flags\/de.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702593","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978702601","adapter.threatinteladapter.begin.ts":"1530978702605","ip_src_port":49209,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"9e19e186-6aba-45ad-8b70-9e696ef02448","response_body_len":534} +{"adapter.threatinteladapter.end.ts":"1530978702605","bro_timestamp":"1530978694.98983","status_code":404,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702589","enrichments.geo.ip_dst_addr.city":"Phoenix","enrichments.geo.ip_dst_addr.latitude":"33.4499","enrichmentsplitterbolt.splitter.begin.ts":"1530978702589","adapter.hostfromjsonlistadapter.end.ts":"1530978702593","enrichments.geo.ip_dst_addr.country":"US","enrichments.geo.ip_dst_addr.locID":"5308655","adapter.geoadapter.begin.ts":"1530978702593","enrichments.geo.ip_dst_addr.postalCode":"85004","uid":"CPbKPD2f2Vg9rvtXXk","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49199 status_code:404 method:POST request_body_len:96 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/wp-content\/themes\/twentyfifteen\/img5.php?l=8r1gf1b2t1kuq42 tags:[] uid:CPbKPD2f2Vg9rvtXXk resp_mime_types:[\"text\\\/html\"] trans_depth:1 orig_fuids:[\"FVYpPq1KmqTn8vOfT\"] host:runlove.us status_msg:Not Found id.orig_h:192.168.138.158 response_body_len:357 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978694.98983 id.resp_h:204.152.254.221 resp_fuids:[\"FnmQJXPDEKwZZ8TMf\"]","ip_dst_addr":"204.152.254.221","threatinteljoinbolt.joiner.ts":"1530978702609","enrichments.geo.ip_dst_addr.dmaCode":"753","host":"runlove.us","enrichmentjoinbolt.joiner.ts":"1530978702598","adapter.hostfromjsonlistadapter.begin.ts":"1530978702593","threatintelsplitterbolt.splitter.begin.ts":"1530978702601","enrichments.geo.ip_dst_addr.longitude":"-112.0712","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FnmQJXPDEKwZZ8TMf"],"timestamp":1530978694989,"method":"POST","request_body_len":96,"orig_mime_types":["text\/plain"],"uri":"\/wp-content\/themes\/twentyfifteen\/img5.php?l=8r1gf1b2t1kuq42","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702593","threatintelsplitterbolt.splitter.end.ts":"1530978702601","adapter.threatinteladapter.begin.ts":"1530978702605","orig_fuids":["FVYpPq1KmqTn8vOfT"],"ip_src_port":49199,"enrichments.geo.ip_dst_addr.location_point":"33.4499,-112.0712","status_msg":"Not Found","guid":"23070f86-2358-4f4c-9bf4-a612afc8c3e3","response_body_len":357} +{"adapter.threatinteladapter.end.ts":"1530978702605","bro_timestamp":"1530978694.665931","status_code":404,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702589","enrichments.geo.ip_dst_addr.city":"Phoenix","enrichments.geo.ip_dst_addr.latitude":"33.4499","enrichmentsplitterbolt.splitter.begin.ts":"1530978702589","adapter.hostfromjsonlistadapter.end.ts":"1530978702593","enrichments.geo.ip_dst_addr.country":"US","enrichments.geo.ip_dst_addr.locID":"5308655","adapter.geoadapter.begin.ts":"1530978702593","enrichments.geo.ip_dst_addr.postalCode":"85004","uid":"CQPUy829Fo1TwbqZh5","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49203 status_code:404 method:POST request_body_len:110 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/wp-content\/themes\/twentyfifteen\/img5.php?f=ka6nnuvccqlw9 tags:[] uid:CQPUy829Fo1TwbqZh5 resp_mime_types:[\"text\\\/html\"] trans_depth:1 orig_fuids:[\"FHf5Gv2fxGeTgj5aLk\"] host:runlove.us status_msg:Not Found id.orig_h:192.168.138.158 response_body_len:357 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978694.665931 id.resp_h:204.152.254.221 resp_fuids:[\"FuBgoE4Ro7nr1s5NO8\"]","ip_dst_addr":"204.152.254.221","threatinteljoinbolt.joiner.ts":"1530978702609","enrichments.geo.ip_dst_addr.dmaCode":"753","host":"runlove.us","enrichmentjoinbolt.joiner.ts":"1530978702599","adapter.hostfromjsonlistadapter.begin.ts":"1530978702593","threatintelsplitterbolt.splitter.begin.ts":"1530978702601","enrichments.geo.ip_dst_addr.longitude":"-112.0712","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FuBgoE4Ro7nr1s5NO8"],"timestamp":1530978694665,"method":"POST","request_body_len":110,"orig_mime_types":["text\/plain"],"uri":"\/wp-content\/themes\/twentyfifteen\/img5.php?f=ka6nnuvccqlw9","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702593","threatintelsplitterbolt.splitter.end.ts":"1530978702601","adapter.threatinteladapter.begin.ts":"1530978702605","orig_fuids":["FHf5Gv2fxGeTgj5aLk"],"ip_src_port":49203,"enrichments.geo.ip_dst_addr.location_point":"33.4499,-112.0712","status_msg":"Not Found","guid":"41e087a9-84a3-41a2-af03-b0ade87ffa76","response_body_len":357} +{"adapter.threatinteladapter.end.ts":"1530978702605","bro_timestamp":"1530978694.939958","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702590","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978702589","adapter.hostfromjsonlistadapter.end.ts":"1530978702593","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978702593","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CA0G2ASkF1efFirs7","resp_mime_types":["image\/png"],"trans_depth":2,"protocol":"http","original_string":"HTTP | id.orig_p:49210 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/lb.png tags:[] uid:CA0G2ASkF1efFirs7 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:2 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:239 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978694.939958 id.resp_h:95.163.121.204 resp_fuids:[\"FXqalu3YBvkNyrelff\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978702610","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978702599","adapter.hostfromjsonlistadapter.begin.ts":"1530978702593","threatintelsplitterbolt.splitter.begin.ts":"1530978702601","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FXqalu3YBvkNyrelff"],"timestamp":1530978694939,"method":"GET","request_body_len":0,"uri":"\/img\/lb.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702593","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978702601","adapter.threatinteladapter.begin.ts":"1530978702605","ip_src_port":49210,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"b4a27884-579e-4266-b1d5-0c12f941924a","response_body_len":239} +{"adapter.threatinteladapter.end.ts":"1530978702605","bro_timestamp":"1530978694.291127","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702590","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978702590","adapter.hostfromjsonlistadapter.end.ts":"1530978702593","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978702593","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CodIOCgeqZXqVSCg6","resp_mime_types":["image\/png"],"trans_depth":4,"protocol":"http","original_string":"HTTP | id.orig_p:49205 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/bitcoin.png tags:[] uid:CodIOCgeqZXqVSCg6 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:4 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:5523 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978694.291127 id.resp_h:95.163.121.204 resp_fuids:[\"Ft8inr3vk76ny20gZ2\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978702610","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978702599","adapter.hostfromjsonlistadapter.begin.ts":"1530978702593","threatintelsplitterbolt.splitter.begin.ts":"1530978702601","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["Ft8inr3vk76ny20gZ2"],"timestamp":1530978694291,"method":"GET","request_body_len":0,"uri":"\/img\/bitcoin.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702593","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978702601","adapter.threatinteladapter.begin.ts":"1530978702605","ip_src_port":49205,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"105bf657-f1ec-4276-bfcd-091905599296","response_body_len":5523} +{"adapter.threatinteladapter.end.ts":"1530978702609","bro_timestamp":"1530978698.168044","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702590","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978702590","adapter.hostfromjsonlistadapter.end.ts":"1530978702597","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978702597","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"ChNCWL3i4gNIYPkoDe","trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49194 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/?60dbe33b908e0086292196ef001816bc tags:[] uid:ChNCWL3i4gNIYPkoDe trans_depth:1 host:62.75.195.236 status_msg:OK id.orig_h:192.168.138.158 response_body_len:0 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978698.168044 id.resp_h:62.75.195.236","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978702612","host":"62.75.195.236","enrichmentjoinbolt.joiner.ts":"1530978702601","adapter.hostfromjsonlistadapter.begin.ts":"1530978702597","threatintelsplitterbolt.splitter.begin.ts":"1530978702605","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","timestamp":1530978698168,"method":"GET","request_body_len":0,"uri":"\/?60dbe33b908e0086292196ef001816bc","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702597","threatintelsplitterbolt.splitter.end.ts":"1530978702605","adapter.threatinteladapter.begin.ts":"1530978702609","ip_src_port":49194,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"5bd73342-6081-4de8-af0d-b68efab3bf95","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978702609","bro_timestamp":"1530978698.840044","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702590","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978702590","adapter.hostfromjsonlistadapter.end.ts":"1530978702597","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978702597","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CX5zuR35fzQMB5VJmd","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49205 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/11iQmfg tags:[] uid:CX5zuR35fzQMB5VJmd resp_mime_types:[\"text\\\/html\"] trans_depth:1 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:3289 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978698.840044 id.resp_h:95.163.121.204 resp_fuids:[\"FjwIxc3D3tcVPcqmGc\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978702612","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978702601","adapter.hostfromjsonlistadapter.begin.ts":"1530978702597","threatintelsplitterbolt.splitter.begin.ts":"1530978702605","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FjwIxc3D3tcVPcqmGc"],"timestamp":1530978698840,"method":"GET","request_body_len":0,"uri":"\/11iQmfg","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702597","threatintelsplitterbolt.splitter.end.ts":"1530978702605","adapter.threatinteladapter.begin.ts":"1530978702609","ip_src_port":49205,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"f9dbf04b-bcc8-48a9-b858-8ca45b6f8274","response_body_len":3289} +{"adapter.threatinteladapter.end.ts":"1530978702609","bro_timestamp":"1530978698.949395","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702590","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978702590","adapter.hostfromjsonlistadapter.end.ts":"1530978702597","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978702598","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"C8Ljn32fwV1v4G45R8","trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49188 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/aa25f5fe2875e3d0a244e6969e589cc4 tags:[] uid:C8Ljn32fwV1v4G45R8 trans_depth:1 host:62.75.195.236 status_msg:OK id.orig_h:192.168.138.158 response_body_len:861 ts:1530978698.949395 id.resp_h:62.75.195.236 resp_fuids:[\"FfQLue1qc3s7ZfGzH5\"]","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978702612","host":"62.75.195.236","enrichmentjoinbolt.joiner.ts":"1530978702601","adapter.hostfromjsonlistadapter.begin.ts":"1530978702597","threatintelsplitterbolt.splitter.begin.ts":"1530978702605","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","resp_fuids":["FfQLue1qc3s7ZfGzH5"],"timestamp":1530978698949,"method":"GET","request_body_len":0,"uri":"\/aa25f5fe2875e3d0a244e6969e589cc4","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702598","threatintelsplitterbolt.splitter.end.ts":"1530978702605","adapter.threatinteladapter.begin.ts":"1530978702609","ip_src_port":49188,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"e8cff10e-d3ab-4578-8aeb-f94c614b5bd6","response_body_len":861} +{"adapter.threatinteladapter.end.ts":"1530978702609","qclass_name":"qclass-32769","bro_timestamp":"1530978698.075525","qtype_name":"PTR","ip_dst_port":5353,"enrichmentsplitterbolt.splitter.end.ts":"1530978702590","qtype":12,"rejected":false,"enrichmentsplitterbolt.splitter.begin.ts":"1530978702590","adapter.hostfromjsonlistadapter.end.ts":"1530978702597","trans_id":0,"adapter.geoadapter.begin.ts":"1530978702598","uid":"C8DgGj1pj2jXyhi9g1","protocol":"dns","original_string":"DNS | AA:false qclass_name:qclass-32769 id.orig_p:5353 qtype_name:PTR qtype:12 rejected:false id.resp_p:5353 query:_googlecast._tcp.local trans_id:0 TC:false RA:false uid:C8DgGj1pj2jXyhi9g1 RD:false proto:udp id.orig_h:192.168.66.1 Z:0 qclass:32769 ts:1530978698.075525 id.resp_h:224.0.0.251","ip_dst_addr":"224.0.0.251","threatinteljoinbolt.joiner.ts":"1530978702612","enrichmentjoinbolt.joiner.ts":"1530978702602","adapter.hostfromjsonlistadapter.begin.ts":"1530978702597","threatintelsplitterbolt.splitter.begin.ts":"1530978702605","Z":0,"ip_src_addr":"192.168.66.1","qclass":32769,"timestamp":1530978698075,"AA":false,"query":"_googlecast._tcp.local","TC":false,"RA":false,"source.type":"bro","adapter.geoadapter.end.ts":"1530978702598","RD":false,"threatintelsplitterbolt.splitter.end.ts":"1530978702606","adapter.threatinteladapter.begin.ts":"1530978702609","ip_src_port":5353,"proto":"udp","guid":"f88c60ba-4062-411f-ae82-c9a86e0a0d1b"} +{"adapter.threatinteladapter.end.ts":"1530978702609","bro_timestamp":"1530978698.312623","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702590","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978702590","adapter.hostfromjsonlistadapter.end.ts":"1530978702597","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978702598","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"CCTaln3ggV4dOqGETi","trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49194 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/?60dbe33b908e0086292196ef001816bc tags:[] uid:CCTaln3ggV4dOqGETi trans_depth:1 host:62.75.195.236 status_msg:OK id.orig_h:192.168.138.158 response_body_len:0 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978698.312623 id.resp_h:62.75.195.236","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978702612","host":"62.75.195.236","enrichmentjoinbolt.joiner.ts":"1530978702602","adapter.hostfromjsonlistadapter.begin.ts":"1530978702597","threatintelsplitterbolt.splitter.begin.ts":"1530978702606","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","timestamp":1530978698312,"method":"GET","request_body_len":0,"uri":"\/?60dbe33b908e0086292196ef001816bc","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702598","threatintelsplitterbolt.splitter.end.ts":"1530978702606","adapter.threatinteladapter.begin.ts":"1530978702609","ip_src_port":49194,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"c0049477-bdc9-42ab-88fe-c088a7d9e76d","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978702611","bro_timestamp":"1530978698.907146","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702591","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978702591","adapter.hostfromjsonlistadapter.end.ts":"1530978702597","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978702598","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"Cnd9EM1uTP3PbJ0BS","resp_mime_types":["image\/png"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49209 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/flags\/de.png tags:[] uid:Cnd9EM1uTP3PbJ0BS referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:1 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:534 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978698.907146 id.resp_h:95.163.121.204 resp_fuids:[\"Fck5px3MJLpCrDeCZ3\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978702613","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978702602","adapter.hostfromjsonlistadapter.begin.ts":"1530978702597","threatintelsplitterbolt.splitter.begin.ts":"1530978702607","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["Fck5px3MJLpCrDeCZ3"],"timestamp":1530978698907,"method":"GET","request_body_len":0,"uri":"\/img\/flags\/de.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702598","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978702607","adapter.threatinteladapter.begin.ts":"1530978702611","ip_src_port":49209,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"358e4eca-0f08-4c10-a307-881009c223b0","response_body_len":534} +{"adapter.threatinteladapter.end.ts":"1530978702611","bro_timestamp":"1530978698.884865","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702591","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978702591","adapter.hostfromjsonlistadapter.end.ts":"1530978702597","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978702598","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CJY1nx4uy46hVP4kmg","resp_mime_types":["text\/plain"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49206 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/style.css tags:[] uid:CJY1nx4uy46hVP4kmg referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"text\\\/plain\"] trans_depth:1 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:4492 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978698.884865 id.resp_h:95.163.121.204 resp_fuids:[\"FindPO2TsX283BvQw3\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978702613","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978702602","adapter.hostfromjsonlistadapter.begin.ts":"1530978702597","threatintelsplitterbolt.splitter.begin.ts":"1530978702607","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FindPO2TsX283BvQw3"],"timestamp":1530978698884,"method":"GET","request_body_len":0,"uri":"\/img\/style.css","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702598","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978702607","adapter.threatinteladapter.begin.ts":"1530978702611","ip_src_port":49206,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"94df9cde-8877-43bf-97a6-d2e0bbc840c4","response_body_len":4492} +{"adapter.threatinteladapter.end.ts":"1530978702611","bro_timestamp":"1530978698.521985","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702591","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978702591","adapter.hostfromjsonlistadapter.end.ts":"1530978702597","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978702598","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"C1qlzE2SalKbpWSJGi","resp_mime_types":["image\/png"],"trans_depth":3,"protocol":"http","original_string":"HTTP | id.orig_p:49210 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/button_pay.png tags:[] uid:C1qlzE2SalKbpWSJGi referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:3 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:727 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978698.521985 id.resp_h:95.163.121.204 resp_fuids:[\"Fd2ecB4nK7EKV7lLA1\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978702613","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978702602","adapter.hostfromjsonlistadapter.begin.ts":"1530978702597","threatintelsplitterbolt.splitter.begin.ts":"1530978702607","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["Fd2ecB4nK7EKV7lLA1"],"timestamp":1530978698521,"method":"GET","request_body_len":0,"uri":"\/img\/button_pay.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702598","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978702607","adapter.threatinteladapter.begin.ts":"1530978702611","ip_src_port":49210,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"024a7ece-fce2-4ec2-86ee-e5e7d0dc2a5d","response_body_len":727} +{"adapter.threatinteladapter.end.ts":"1530978702611","bro_timestamp":"1530978698.077529","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702591","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978702591","adapter.hostfromjsonlistadapter.end.ts":"1530978702597","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978702598","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"C5UfKV32U65H7ojqJd","resp_mime_types":["image\/png"],"trans_depth":4,"protocol":"http","original_string":"HTTP | id.orig_p:49205 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/bitcoin.png tags:[] uid:C5UfKV32U65H7ojqJd referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:4 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:5523 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978698.077529 id.resp_h:95.163.121.204 resp_fuids:[\"Fy6w2R347d11rin2hg\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978702613","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978702602","adapter.hostfromjsonlistadapter.begin.ts":"1530978702597","threatintelsplitterbolt.splitter.begin.ts":"1530978702607","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["Fy6w2R347d11rin2hg"],"timestamp":1530978698077,"method":"GET","request_body_len":0,"uri":"\/img\/bitcoin.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702598","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978702607","adapter.threatinteladapter.begin.ts":"1530978702611","ip_src_port":49205,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"626f9f8b-2af5-4c9f-a36a-3cfedea5614e","response_body_len":5523} +{"adapter.threatinteladapter.end.ts":"1530978702611","bro_timestamp":"1530978698.241724","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978702591","enrichments.geo.ip_dst_addr.city":"Los Angeles","enrichments.geo.ip_dst_addr.latitude":"34.0494","enrichmentsplitterbolt.splitter.begin.ts":"1530978702591","adapter.hostfromjsonlistadapter.end.ts":"1530978702597","enrichments.geo.ip_dst_addr.country":"US","enrichments.geo.ip_dst_addr.locID":"5368361","adapter.geoadapter.begin.ts":"1530978702598","enrichments.geo.ip_dst_addr.postalCode":"90014","uid":"CJNiGM3zcyXHHORzFb","resp_mime_types":["text\/plain"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49198 status_code:200 method:POST request_body_len:134 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/wp-content\/themes\/grizzly\/img5.php?c=cdcnw7cfz43rmtg tags:[] uid:CJNiGM3zcyXHHORzFb resp_mime_types:[\"text\\\/plain\"] trans_depth:1 orig_fuids:[\"FJWjcF3Z0qYg56Pw65\"] host:comarksecurity.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:14 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978698.241724 id.resp_h:72.34.49.86 resp_fuids:[\"FrVBRXxij8xG1u239\"]","ip_dst_addr":"72.34.49.86","threatinteljoinbolt.joiner.ts":"1530978702614","enrichments.geo.ip_dst_addr.dmaCode":"803","host":"comarksecurity.com","enrichmentjoinbolt.joiner.ts":"1530978702603","adapter.hostfromjsonlistadapter.begin.ts":"1530978702597","threatintelsplitterbolt.splitter.begin.ts":"1530978702608","enrichments.geo.ip_dst_addr.longitude":"-118.2641","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FrVBRXxij8xG1u239"],"timestamp":1530978698241,"method":"POST","request_body_len":134,"orig_mime_types":["text\/plain"],"uri":"\/wp-content\/themes\/grizzly\/img5.php?c=cdcnw7cfz43rmtg","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978702598","threatintelsplitterbolt.splitter.end.ts":"1530978702608","adapter.threatinteladapter.begin.ts":"1530978702611","orig_fuids":["FJWjcF3Z0qYg56Pw65"],"ip_src_port":49198,"enrichments.geo.ip_dst_addr.location_point":"34.0494,-118.2641","status_msg":"OK","guid":"40b1b0b6-a51c-41a2-9d97-ef26badb79fa","response_body_len":14} +{"adapter.threatinteladapter.end.ts":"1530978710497","bro_timestamp":"1530978704.958145","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978710473","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978710473","adapter.hostfromjsonlistadapter.end.ts":"1530978710476","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978710476","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"C7KeXZ1jvzj9qkSqt7","resp_mime_types":["application\/x-shockwave-flash"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49185 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:C7KeXZ1jvzj9qkSqt7 referrer:http:\/\/va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in\/?285a4d4e4e5a4d4d4649584c5d43064b4745 resp_mime_types:[\"application\\\/x-shockwave-flash\"] trans_depth:1 host:ubb67.3c147o.u806a4.w07d919.o5f.f1.b80w.r0faf9.e8mfzdgrf7g0.groupprograms.in status_msg:OK id.orig_h:192.168.138.158 response_body_len:8973 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978704.958145 id.resp_h:62.75.195.236 resp_fuids:[\"Ft6fqj1vE6fmJBYPx6\"]","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978710501","host":"ubb67.3c147o.u806a4.w07d919.o5f.f1.b80w.r0faf9.e8mfzdgrf7g0.groupprograms.in","enrichmentjoinbolt.joiner.ts":"1530978710487","adapter.hostfromjsonlistadapter.begin.ts":"1530978710475","threatintelsplitterbolt.splitter.begin.ts":"1530978710493","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["Ft6fqj1vE6fmJBYPx6"],"timestamp":1530978704958,"method":"GET","request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978710476","referrer":"http:\/\/va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in\/?285a4d4e4e5a4d4d4649584c5d43064b4745","threatintelsplitterbolt.splitter.end.ts":"1530978710493","adapter.threatinteladapter.begin.ts":"1530978710497","ip_src_port":49185,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"f1d5ef09-d2e5-4cdd-a26b-fc23df82c385","response_body_len":8973} +{"adapter.threatinteladapter.end.ts":"1530978710497","bro_timestamp":"1530978704.608287","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978710477","enrichmentsplitterbolt.splitter.begin.ts":"1530978710477","adapter.hostfromjsonlistadapter.end.ts":"1530978710486","adapter.geoadapter.begin.ts":"1530978710486","uid":"CUrRne3iLIxXavQtci","trans_depth":40,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/persist\/wizard-data?_=1484168498643 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:40 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978704.608287 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978710501","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978710491","adapter.hostfromjsonlistadapter.begin.ts":"1530978710486","threatintelsplitterbolt.splitter.begin.ts":"1530978710493","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978704608,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/persist\/wizard-data?_=1484168498643","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978710486","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978710493","adapter.threatinteladapter.begin.ts":"1530978710497","ip_src_port":50451,"guid":"438b5c9d-522a-4611-9d70-c3723645611b","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978710499","bro_timestamp":"1530978704.063932","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978710477","enrichmentsplitterbolt.splitter.begin.ts":"1530978710477","adapter.hostfromjsonlistadapter.end.ts":"1530978710489","adapter.geoadapter.begin.ts":"1530978710490","uid":"CUrRne3iLIxXavQtci","trans_depth":178,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/clusters\/metron_cluster\/components\/?ServiceComponentInfo\/component_name=APP_TIMELINE_SERVER|ServiceComponentInfo\/category=MASTER&fields=ServiceComponentInfo\/service_name,host_components\/HostRoles\/display_name,host_components\/HostRoles\/host_name,host_components\/HostRoles\/state,host_components\/HostRoles\/maintenance_state,host_components\/HostRoles\/stale_configs,host_components\/HostRoles\/ha_state,host_components\/HostRoles\/desired_admin_state,,host_components\/metrics\/jvm\/memHeapUsedM,host_components\/metrics\/jvm\/HeapMemoryMax,host_components\/metrics\/jvm\/HeapMemoryUsed,host_components\/metrics\/jvm\/memHeapCommittedM,host_components\/metrics\/mapred\/jobtracker\/trackers_decommissioned,host_components\/metrics\/cpu\/cpu_wio,host_components\/metrics\/rpc\/client\/RpcQueueTime_avg_time,host_components\/metrics\/dfs\/FSNamesystem\/*,host_components\/metrics\/dfs\/namenode\/Version,host_components\/metrics\/dfs\/namenode\/LiveNodes,host_components\/metrics\/dfs\/namenode\/DeadNodes,host_components\/metrics\/dfs\/namenode\/DecomNodes,host_components\/metrics\/dfs\/namenode\/TotalFiles,host_components\/metrics\/dfs\/namenode\/UpgradeFinalized,host_components\/metrics\/dfs\/namenode\/Safemode,host_components\/metrics\/runtime\/StartTime,host_components\/metrics\/hbase\/master\/IsActiveMaster,host_components\/metrics\/hbase\/master\/MasterStartTime,host_components\/metrics\/hbase\/master\/MasterActiveTime,host_components\/metrics\/hbase\/master\/AverageLoad,host_components\/metrics\/master\/AssignmentManger\/ritCount,metrics\/api\/v1\/cluster\/summary,metrics\/api\/v1\/topology\/summary,metrics\/api\/v1\/nimbus\/summary,host_components\/metrics\/yarn\/Queue,host_components\/metrics\/yarn\/ClusterMetrics\/NumActiveNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumLostNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumUnhealthyNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumRebootedNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumDecommissionedNMs&minimal_response=true&_=1484169119448 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:178 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978704.063932 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978710502","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978710494","adapter.hostfromjsonlistadapter.begin.ts":"1530978710489","threatintelsplitterbolt.splitter.begin.ts":"1530978710497","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978704063,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/clusters\/metron_cluster\/components\/?ServiceComponentInfo\/component_name=APP_TIMELINE_SERVER|ServiceComponentInfo\/category=MASTER&fields=ServiceComponentInfo\/service_name,host_components\/HostRoles\/display_name,host_components\/HostRoles\/host_name,host_components\/HostRoles\/state,host_components\/HostRoles\/maintenance_state,host_components\/HostRoles\/stale_configs,host_components\/HostRoles\/ha_state,host_components\/HostRoles\/desired_admin_state,,host_components\/metrics\/jvm\/memHeapUsedM,host_components\/metrics\/jvm\/HeapMemoryMax,host_components\/metrics\/jvm\/HeapMemoryUsed,host_components\/metrics\/jvm\/memHeapCommittedM,host_components\/metrics\/mapred\/jobtracker\/trackers_decommissioned,host_components\/metrics\/cpu\/cpu_wio,host_components\/metrics\/rpc\/client\/RpcQueueTime_avg_time,host_components\/metrics\/dfs\/FSNamesystem\/*,host_components\/metrics\/dfs\/namenode\/Version,host_components\/metrics\/dfs\/namenode\/LiveNodes,host_components\/metrics\/dfs\/namenode\/DeadNodes,host_components\/metrics\/dfs\/namenode\/DecomNodes,host_components\/metrics\/dfs\/namenode\/TotalFiles,host_components\/metrics\/dfs\/namenode\/UpgradeFinalized,host_components\/metrics\/dfs\/namenode\/Safemode,host_components\/metrics\/runtime\/StartTime,host_components\/metrics\/hbase\/master\/IsActiveMaster,host_components\/metrics\/hbase\/master\/MasterStartTime,host_components\/metrics\/hbase\/master\/MasterActiveTime,host_components\/metrics\/hbase\/master\/AverageLoad,host_components\/metrics\/master\/AssignmentManger\/ritCount,metrics\/api\/v1\/cluster\/summary,metrics\/api\/v1\/topology\/summary,metrics\/api\/v1\/nimbus\/summary,host_components\/metrics\/yarn\/Queue,host_components\/metrics\/yarn\/ClusterMetrics\/NumActiveNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumLostNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumUnhealthyNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumRebootedNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumDecommissionedNMs&minimal_response=true&_=1484169119448","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978710490","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978710497","adapter.threatinteladapter.begin.ts":"1530978710499","ip_src_port":50451,"guid":"03546910-68c3-4aa4-90a3-6983bc23324e","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978710499","bro_timestamp":"1530978704.137918","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978710478","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978710478","adapter.hostfromjsonlistadapter.end.ts":"1530978710489","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978710490","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"Cx8Ucg1r67RywyWab1","resp_mime_types":["image\/png"],"trans_depth":2,"protocol":"http","original_string":"HTTP | id.orig_p:49205 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/flags\/us.png tags:[] uid:Cx8Ucg1r67RywyWab1 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:2 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:825 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978704.137918 id.resp_h:95.163.121.204 resp_fuids:[\"FCr63p4t8M7SUAumi3\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978710502","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978710494","adapter.hostfromjsonlistadapter.begin.ts":"1530978710489","threatintelsplitterbolt.splitter.begin.ts":"1530978710497","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FCr63p4t8M7SUAumi3"],"timestamp":1530978704137,"method":"GET","request_body_len":0,"uri":"\/img\/flags\/us.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978710490","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978710497","adapter.threatinteladapter.begin.ts":"1530978710499","ip_src_port":49205,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"5ea15274-bf38-423e-9c3e-6fb0f3bf0270","response_body_len":825} +{"adapter.threatinteladapter.end.ts":"1530978710499","bro_timestamp":"1530978704.973595","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978710478","enrichmentsplitterbolt.splitter.begin.ts":"1530978710478","adapter.hostfromjsonlistadapter.end.ts":"1530978710489","adapter.geoadapter.begin.ts":"1530978710490","uid":"CUrRne3iLIxXavQtci","trans_depth":251,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/clusters\/metron_cluster?fields=Clusters\/desired_configs\/cluster-env&_=1484169429016 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:251 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978704.973595 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978710502","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978710494","adapter.hostfromjsonlistadapter.begin.ts":"1530978710489","threatintelsplitterbolt.splitter.begin.ts":"1530978710497","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978704973,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/clusters\/metron_cluster?fields=Clusters\/desired_configs\/cluster-env&_=1484169429016","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978710490","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978710497","adapter.threatinteladapter.begin.ts":"1530978710499","ip_src_port":50451,"guid":"273e3d59-b616-424e-8c30-add81bd671b9","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978710499","bro_timestamp":"1530978704.973117","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978710478","enrichmentsplitterbolt.splitter.begin.ts":"1530978710478","adapter.hostfromjsonlistadapter.end.ts":"1530978710489","adapter.geoadapter.begin.ts":"1530978710490","uid":"CUrRne3iLIxXavQtci","trans_depth":247,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/clusters?fields=Clusters\/provisioning_state&_=1484169420015 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:247 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978704.973117 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978710502","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978710494","adapter.hostfromjsonlistadapter.begin.ts":"1530978710489","threatintelsplitterbolt.splitter.begin.ts":"1530978710497","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978704973,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/clusters?fields=Clusters\/provisioning_state&_=1484169420015","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978710490","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978710497","adapter.threatinteladapter.begin.ts":"1530978710499","ip_src_port":50451,"guid":"9ba63d37-9e6b-4ba8-8504-cc418d9ce8aa","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978710499","qclass_name":"C_INTERNET","bro_timestamp":"1530978704.094553","qtype_name":"PTR","ip_dst_port":5353,"enrichmentsplitterbolt.splitter.end.ts":"1530978710479","qtype":12,"rejected":false,"enrichmentsplitterbolt.splitter.begin.ts":"1530978710479","adapter.hostfromjsonlistadapter.end.ts":"1530978710489","trans_id":0,"adapter.geoadapter.begin.ts":"1530978710490","uid":"C03Lir2lgO0AxyDctk","protocol":"dns","original_string":"DNS | AA:false qclass_name:C_INTERNET id.orig_p:5353 qtype_name:PTR qtype:12 rejected:false id.resp_p:5353 query:_googlecast._tcp.local trans_id:0 TC:false RA:false uid:C03Lir2lgO0AxyDctk RD:false proto:udp id.orig_h:192.168.66.1 Z:0 qclass:1 ts:1530978704.094553 id.resp_h:224.0.0.251","ip_dst_addr":"224.0.0.251","threatinteljoinbolt.joiner.ts":"1530978710502","enrichmentjoinbolt.joiner.ts":"1530978710494","adapter.hostfromjsonlistadapter.begin.ts":"1530978710489","threatintelsplitterbolt.splitter.begin.ts":"1530978710497","Z":0,"ip_src_addr":"192.168.66.1","qclass":1,"timestamp":1530978704094,"AA":false,"query":"_googlecast._tcp.local","TC":false,"RA":false,"source.type":"bro","adapter.geoadapter.end.ts":"1530978710490","RD":false,"threatintelsplitterbolt.splitter.end.ts":"1530978710497","adapter.threatinteladapter.begin.ts":"1530978710499","ip_src_port":5353,"proto":"udp","guid":"b5849fa0-3b1f-44a8-8b89-0eb3e823ba6f"} +{"adapter.threatinteladapter.end.ts":"1530978710499","bro_timestamp":"1530978704.896579","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978710479","enrichmentsplitterbolt.splitter.begin.ts":"1530978710479","adapter.hostfromjsonlistadapter.end.ts":"1530978710489","adapter.geoadapter.begin.ts":"1530978710490","uid":"CUrRne3iLIxXavQtci","trans_depth":132,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/clusters\/metron_cluster\/components\/?fields=ServiceComponentInfo\/service_name,ServiceComponentInfo\/category,ServiceComponentInfo\/installed_count,ServiceComponentInfo\/started_count,ServiceComponentInfo\/init_count,ServiceComponentInfo\/install_failed_count,ServiceComponentInfo\/unknown_count,ServiceComponentInfo\/total_count,ServiceComponentInfo\/display_name,host_components\/HostRoles\/host_name&minimal_response=true&_=1484168884281 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:132 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978704.896579 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978710502","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978710495","adapter.hostfromjsonlistadapter.begin.ts":"1530978710489","threatintelsplitterbolt.splitter.begin.ts":"1530978710497","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978704896,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/clusters\/metron_cluster\/components\/?fields=ServiceComponentInfo\/service_name,ServiceComponentInfo\/category,ServiceComponentInfo\/installed_count,ServiceComponentInfo\/started_count,ServiceComponentInfo\/init_count,ServiceComponentInfo\/install_failed_count,ServiceComponentInfo\/unknown_count,ServiceComponentInfo\/total_count,ServiceComponentInfo\/display_name,host_components\/HostRoles\/host_name&minimal_response=true&_=1484168884281","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978710490","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978710497","adapter.threatinteladapter.begin.ts":"1530978710499","ip_src_port":50451,"guid":"bc257399-d461-4dd2-b6ac-d18c26af2dd2","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978710499","bro_timestamp":"1530978704.015832","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978710479","enrichments.geo.ip_dst_addr.city":"Los Angeles","enrichments.geo.ip_dst_addr.latitude":"34.0494","enrichmentsplitterbolt.splitter.begin.ts":"1530978710479","adapter.hostfromjsonlistadapter.end.ts":"1530978710489","enrichments.geo.ip_dst_addr.country":"US","enrichments.geo.ip_dst_addr.locID":"5368361","adapter.geoadapter.begin.ts":"1530978710490","enrichments.geo.ip_dst_addr.postalCode":"90014","uid":"CpBTZB1XlDvW4TC9o4","resp_mime_types":["image\/png"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49202 status_code:200 method:POST request_body_len:162 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/wp-content\/themes\/grizzly\/img5.php?u=mfymi71rapdzk tags:[] uid:CpBTZB1XlDvW4TC9o4 resp_mime_types:[\"image\\\/png\"] trans_depth:1 orig_fuids:[\"F2Jzoe3KKRC5QVAA72\"] host:comarksecurity.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:45662 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978704.015832 id.resp_h:72.34.49.86 resp_fuids:[\"FUTx241N7osQc8nSH1\"]","ip_dst_addr":"72.34.49.86","threatinteljoinbolt.joiner.ts":"1530978710503","enrichments.geo.ip_dst_addr.dmaCode":"803","host":"comarksecurity.com","enrichmentjoinbolt.joiner.ts":"1530978710495","adapter.hostfromjsonlistadapter.begin.ts":"1530978710489","threatintelsplitterbolt.splitter.begin.ts":"1530978710497","enrichments.geo.ip_dst_addr.longitude":"-118.2641","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FUTx241N7osQc8nSH1"],"timestamp":1530978704015,"method":"POST","request_body_len":162,"orig_mime_types":["text\/plain"],"uri":"\/wp-content\/themes\/grizzly\/img5.php?u=mfymi71rapdzk","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978710490","threatintelsplitterbolt.splitter.end.ts":"1530978710497","adapter.threatinteladapter.begin.ts":"1530978710499","orig_fuids":["F2Jzoe3KKRC5QVAA72"],"ip_src_port":49202,"enrichments.geo.ip_dst_addr.location_point":"34.0494,-118.2641","status_msg":"OK","guid":"48802e94-0e14-43b6-aa8b-14e3b59e72a6","response_body_len":45662} +{"adapter.threatinteladapter.end.ts":"1530978710499","bro_timestamp":"1530978704.264989","status_code":304,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978710480","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978710480","adapter.hostfromjsonlistadapter.end.ts":"1530978710489","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978710490","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CX9L2c29ZYGsLN10n5","trans_depth":4,"protocol":"http","original_string":"HTTP | id.orig_p:49206 status_code:304 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/style.css tags:[] uid:CX9L2c29ZYGsLN10n5 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg trans_depth:4 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:Not Modified id.orig_h:192.168.138.158 response_body_len:0 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978704.264989 id.resp_h:95.163.121.204","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978710503","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978710495","adapter.hostfromjsonlistadapter.begin.ts":"1530978710489","threatintelsplitterbolt.splitter.begin.ts":"1530978710497","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","timestamp":1530978704264,"method":"GET","request_body_len":0,"uri":"\/img\/style.css","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978710490","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978710497","adapter.threatinteladapter.begin.ts":"1530978710499","ip_src_port":49206,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"Not Modified","guid":"f627bf99-57d1-4844-94e8-f44f10530bd8","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978710499","bro_timestamp":"1530978708.760542","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978710480","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978710480","adapter.hostfromjsonlistadapter.end.ts":"1530978710489","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978710490","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"Cx7JE83AWXe9eyppvg","resp_mime_types":["image\/x-icon"],"trans_depth":2,"protocol":"http","original_string":"HTTP | id.orig_p:49207 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/favicon.ico tags:[] uid:Cx7JE83AWXe9eyppvg resp_mime_types:[\"image\\\/x-icon\"] trans_depth:2 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:318 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978708.760542 id.resp_h:95.163.121.204 resp_fuids:[\"FABIhF7C2XrovcYCi\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978710503","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978710495","adapter.hostfromjsonlistadapter.begin.ts":"1530978710489","threatintelsplitterbolt.splitter.begin.ts":"1530978710497","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FABIhF7C2XrovcYCi"],"timestamp":1530978708760,"method":"GET","request_body_len":0,"uri":"\/favicon.ico","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978710490","threatintelsplitterbolt.splitter.end.ts":"1530978710497","adapter.threatinteladapter.begin.ts":"1530978710499","ip_src_port":49207,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"3aae739b-f036-4e3f-8a7c-fd4bd2924471","response_body_len":318} +{"TTLs":[14069.0],"adapter.threatinteladapter.end.ts":"1530978710499","qclass_name":"C_INTERNET","bro_timestamp":"1530978708.288392","qtype_name":"A","ip_dst_port":53,"enrichmentsplitterbolt.splitter.end.ts":"1530978710481","qtype":1,"rejected":false,"answers":["204.152.254.221"],"enrichmentsplitterbolt.splitter.begin.ts":"1530978710480","adapter.hostfromjsonlistadapter.end.ts":"1530978710490","trans_id":23625,"adapter.geoadapter.begin.ts":"1530978710490","uid":"CAS65s1sNWsjjDoLH3","protocol":"dns","original_string":"DNS | AA:false TTLs:[14069.0] qclass_name:C_INTERNET id.orig_p:61720 qtype_name:A qtype:1 rejected:false id.resp_p:53 query:runlove.us answers:[\"204.152.254.221\"] trans_id:23625 rcode:0 rcode_name:NOERROR TC:false RA:true uid:CAS65s1sNWsjjDoLH3 RD:true proto:udp id.orig_h:192.168.138.158 Z:0 qclass:1 ts:1530978708.288392 id.resp_h:192.168.138.2","ip_dst_addr":"192.168.138.2","threatinteljoinbolt.joiner.ts":"1530978710503","enrichmentjoinbolt.joiner.ts":"1530978710495","adapter.hostfromjsonlistadapter.begin.ts":"1530978710490","threatintelsplitterbolt.splitter.begin.ts":"1530978710497","Z":0,"ip_src_addr":"192.168.138.158","qclass":1,"timestamp":1530978708288,"AA":false,"query":"runlove.us","rcode":0,"rcode_name":"NOERROR","TC":false,"RA":true,"source.type":"bro","adapter.geoadapter.end.ts":"1530978710490","RD":true,"threatintelsplitterbolt.splitter.end.ts":"1530978710497","adapter.threatinteladapter.begin.ts":"1530978710499","ip_src_port":61720,"proto":"udp","guid":"5e2bca3e-47d7-45f3-8442-08c2a2768f2b"} +{"adapter.threatinteladapter.end.ts":"1530978710500","bro_timestamp":"1530978708.670419","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978710481","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978710481","adapter.hostfromjsonlistadapter.end.ts":"1530978710490","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978710490","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"CYQq8E1GZOt01YVHq1","trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49192 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/?d71e0bd86db9587158745a986a4b3606 tags:[] uid:CYQq8E1GZOt01YVHq1 trans_depth:1 host:62.75.195.236 status_msg:OK id.orig_h:192.168.138.158 response_body_len:0 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978708.670419 id.resp_h:62.75.195.236","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978710503","host":"62.75.195.236","enrichmentjoinbolt.joiner.ts":"1530978710495","adapter.hostfromjsonlistadapter.begin.ts":"1530978710490","threatintelsplitterbolt.splitter.begin.ts":"1530978710497","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","timestamp":1530978708670,"method":"GET","request_body_len":0,"uri":"\/?d71e0bd86db9587158745a986a4b3606","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978710490","threatintelsplitterbolt.splitter.end.ts":"1530978710497","adapter.threatinteladapter.begin.ts":"1530978710500","ip_src_port":49192,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"cef73766-56af-4c44-a595-e5118795a708","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978710500","bro_timestamp":"1530978708.883522","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978710482","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978710482","adapter.hostfromjsonlistadapter.end.ts":"1530978710490","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978710490","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"ClNvrm11cIpvatxVR2","resp_mime_types":["image\/x-icon"],"trans_depth":2,"protocol":"http","original_string":"HTTP | id.orig_p:49207 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/favicon.ico tags:[] uid:ClNvrm11cIpvatxVR2 resp_mime_types:[\"image\\\/x-icon\"] trans_depth:2 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:318 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978708.883522 id.resp_h:95.163.121.204 resp_fuids:[\"FlIiHFeqqHoJP0GH4\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978710503","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978710495","adapter.hostfromjsonlistadapter.begin.ts":"1530978710490","threatintelsplitterbolt.splitter.begin.ts":"1530978710497","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FlIiHFeqqHoJP0GH4"],"timestamp":1530978708883,"method":"GET","request_body_len":0,"uri":"\/favicon.ico","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978710490","threatintelsplitterbolt.splitter.end.ts":"1530978710497","adapter.threatinteladapter.begin.ts":"1530978710500","ip_src_port":49207,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"eb723658-e320-472d-9d19-8c832f93dd37","response_body_len":318} +{"adapter.threatinteladapter.end.ts":"1530978710500","qclass_name":"C_INTERNET","bro_timestamp":"1530978708.430175","qtype_name":"PTR","ip_dst_port":5353,"enrichmentsplitterbolt.splitter.end.ts":"1530978710482","qtype":12,"rejected":false,"enrichmentsplitterbolt.splitter.begin.ts":"1530978710482","adapter.hostfromjsonlistadapter.end.ts":"1530978710490","trans_id":0,"adapter.geoadapter.begin.ts":"1530978710490","uid":"Cx7bil4EcuyIC1pVvb","protocol":"dns","original_string":"DNS | AA:false qclass_name:C_INTERNET id.orig_p:5353 qtype_name:PTR qtype:12 rejected:false id.resp_p:5353 query:_googlecast._tcp.local trans_id:0 TC:false RA:false uid:Cx7bil4EcuyIC1pVvb RD:false proto:udp id.orig_h:192.168.66.1 Z:0 qclass:1 ts:1530978708.430175 id.resp_h:224.0.0.251","ip_dst_addr":"224.0.0.251","threatinteljoinbolt.joiner.ts":"1530978710504","enrichmentjoinbolt.joiner.ts":"1530978710495","adapter.hostfromjsonlistadapter.begin.ts":"1530978710490","threatintelsplitterbolt.splitter.begin.ts":"1530978710497","Z":0,"ip_src_addr":"192.168.66.1","qclass":1,"timestamp":1530978708430,"AA":false,"query":"_googlecast._tcp.local","TC":false,"RA":false,"source.type":"bro","adapter.geoadapter.end.ts":"1530978710490","RD":false,"threatintelsplitterbolt.splitter.end.ts":"1530978710497","adapter.threatinteladapter.begin.ts":"1530978710500","ip_src_port":5353,"proto":"udp","guid":"be730290-a5d9-4a4e-b251-014a2694b9d2"} +{"adapter.threatinteladapter.end.ts":"1530978714364","bro_timestamp":"1530978708.970988","status_code":304,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978714345","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978714345","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978714353","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CFYsPe4XJH9BV5pQ2c","trans_depth":4,"protocol":"http","original_string":"HTTP | id.orig_p:49206 status_code:304 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/style.css tags:[] uid:CFYsPe4XJH9BV5pQ2c referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg trans_depth:4 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:Not Modified id.orig_h:192.168.138.158 response_body_len:0 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978708.970988 id.resp_h:95.163.121.204","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978714420","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978714358","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","timestamp":1530978708970,"method":"GET","request_body_len":0,"uri":"\/img\/style.css","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714364","ip_src_port":49206,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"Not Modified","guid":"a0844630-4535-402b-b4c7-b271bc55ea3c","response_body_len":0} +{"TTLs":[14069.0],"adapter.threatinteladapter.end.ts":"1530978714364","qclass_name":"C_INTERNET","bro_timestamp":"1530978708.667408","qtype_name":"A","ip_dst_port":53,"enrichmentsplitterbolt.splitter.end.ts":"1530978714346","qtype":1,"rejected":false,"answers":["204.152.254.221"],"enrichmentsplitterbolt.splitter.begin.ts":"1530978714346","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","trans_id":23625,"adapter.geoadapter.begin.ts":"1530978714353","uid":"C8vTE82vZNVedhnxh5","protocol":"dns","original_string":"DNS | AA:false TTLs:[14069.0] qclass_name:C_INTERNET id.orig_p:61720 qtype_name:A qtype:1 rejected:false id.resp_p:53 query:runlove.us answers:[\"204.152.254.221\"] trans_id:23625 rcode:0 rcode_name:NOERROR TC:false RA:true uid:C8vTE82vZNVedhnxh5 RD:true proto:udp id.orig_h:192.168.138.158 Z:0 qclass:1 ts:1530978708.667408 id.resp_h:192.168.138.2","ip_dst_addr":"192.168.138.2","threatinteljoinbolt.joiner.ts":"1530978714421","enrichmentjoinbolt.joiner.ts":"1530978714358","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","Z":0,"ip_src_addr":"192.168.138.158","qclass":1,"timestamp":1530978708667,"AA":false,"query":"runlove.us","rcode":0,"rcode_name":"NOERROR","TC":false,"RA":true,"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","RD":true,"threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714364","ip_src_port":61720,"proto":"udp","guid":"ef958b8c-d356-4f81-ba1a-5b4e5050c7da"} +{"adapter.threatinteladapter.end.ts":"1530978714364","bro_timestamp":"1530978708.709189","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978714346","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978714346","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978714353","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"CqpQlxu995hhX7Ooa","trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49188 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/aa25f5fe2875e3d0a244e6969e589cc4 tags:[] uid:CqpQlxu995hhX7Ooa trans_depth:1 host:62.75.195.236 status_msg:OK id.orig_h:192.168.138.158 response_body_len:861 ts:1530978708.709189 id.resp_h:62.75.195.236 resp_fuids:[\"FqjRj7tQ2u0nCuYKi\"]","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978714421","host":"62.75.195.236","enrichmentjoinbolt.joiner.ts":"1530978714359","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","resp_fuids":["FqjRj7tQ2u0nCuYKi"],"timestamp":1530978708709,"method":"GET","request_body_len":0,"uri":"\/aa25f5fe2875e3d0a244e6969e589cc4","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714364","ip_src_port":49188,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"a9c0251b-50db-4ec4-a923-9aab8ebc40eb","response_body_len":861} +{"TTLs":[14277.0],"adapter.threatinteladapter.end.ts":"1530978714364","qclass_name":"C_INTERNET","bro_timestamp":"1530978708.469342","qtype_name":"A","ip_dst_port":53,"enrichmentsplitterbolt.splitter.end.ts":"1530978714346","qtype":1,"rejected":false,"answers":["95.163.121.204"],"enrichmentsplitterbolt.splitter.begin.ts":"1530978714346","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","trans_id":5810,"adapter.geoadapter.begin.ts":"1530978714353","uid":"CU5LbB1StA2bxIOFb5","protocol":"dns","original_string":"DNS | AA:false TTLs:[14277.0] qclass_name:C_INTERNET id.orig_p:50329 qtype_name:A qtype:1 rejected:false id.resp_p:53 query:7oqnsnzwwnm6zb7y.gigapaysun.com answers:[\"95.163.121.204\"] trans_id:5810 rcode:0 rcode_name:NOERROR TC:false RA:true uid:CU5LbB1StA2bxIOFb5 RD:true proto:udp id.orig_h:192.168.138.158 Z:0 qclass:1 ts:1530978708.469342 id.resp_h:192.168.138.2","ip_dst_addr":"192.168.138.2","threatinteljoinbolt.joiner.ts":"1530978714421","enrichmentjoinbolt.joiner.ts":"1530978714359","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","Z":0,"ip_src_addr":"192.168.138.158","qclass":1,"timestamp":1530978708469,"AA":false,"query":"7oqnsnzwwnm6zb7y.gigapaysun.com","rcode":0,"rcode_name":"NOERROR","TC":false,"RA":true,"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","RD":true,"threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714364","ip_src_port":50329,"proto":"udp","guid":"84d667e2-e710-4766-a3fa-9bbc86301e36"} +{"adapter.threatinteladapter.end.ts":"1530978714364","bro_timestamp":"1530978708.870464","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978714346","enrichmentsplitterbolt.splitter.begin.ts":"1530978714346","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","adapter.geoadapter.begin.ts":"1530978714353","uid":"CUrRne3iLIxXavQtci","trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/clusters\/metron_cluster\/requests?to=end&page_size=10&fields=Requests&_=1484168316902 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:1 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978708.870464 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978714421","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978714359","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978708870,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/clusters\/metron_cluster\/requests?to=end&page_size=10&fields=Requests&_=1484168316902","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714364","ip_src_port":50451,"guid":"1c06bdf1-c93a-4a25-afb4-38706fe66dad","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978714364","bro_timestamp":"1530978712.058367","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978714346","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978714346","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978714353","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"CuIYie338ayQ4QvzPi","trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49193 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/?34eaf8bd50d85d8c6baacb45f0a7b22e tags:[] uid:CuIYie338ayQ4QvzPi trans_depth:1 host:62.75.195.236 status_msg:OK id.orig_h:192.168.138.158 response_body_len:0 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978712.058367 id.resp_h:62.75.195.236","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978714421","host":"62.75.195.236","enrichmentjoinbolt.joiner.ts":"1530978714359","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","timestamp":1530978712058,"method":"GET","request_body_len":0,"uri":"\/?34eaf8bd50d85d8c6baacb45f0a7b22e","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714364","ip_src_port":49193,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"f51a85af-13fb-4876-8dc5-a37510654832","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978714364","bro_timestamp":"1530978712.722503","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978714346","enrichments.geo.ip_dst_addr.city":"Los Angeles","enrichments.geo.ip_dst_addr.latitude":"34.0494","enrichmentsplitterbolt.splitter.begin.ts":"1530978714346","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","enrichments.geo.ip_dst_addr.country":"US","enrichments.geo.ip_dst_addr.locID":"5368361","adapter.geoadapter.begin.ts":"1530978714353","enrichments.geo.ip_dst_addr.postalCode":"90014","uid":"CsyRyi4LZvVfzdaS2e","resp_mime_types":["image\/png"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49202 status_code:200 method:POST request_body_len:162 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/wp-content\/themes\/grizzly\/img5.php?u=mfymi71rapdzk tags:[] uid:CsyRyi4LZvVfzdaS2e resp_mime_types:[\"image\\\/png\"] trans_depth:1 orig_fuids:[\"Fl9usz3US821agwyb\"] host:comarksecurity.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:45662 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978712.722503 id.resp_h:72.34.49.86 resp_fuids:[\"Fgjkrn1QODy4ZEaGf7\"]","ip_dst_addr":"72.34.49.86","threatinteljoinbolt.joiner.ts":"1530978714421","enrichments.geo.ip_dst_addr.dmaCode":"803","host":"comarksecurity.com","enrichmentjoinbolt.joiner.ts":"1530978714359","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","enrichments.geo.ip_dst_addr.longitude":"-118.2641","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["Fgjkrn1QODy4ZEaGf7"],"timestamp":1530978712722,"method":"POST","request_body_len":162,"orig_mime_types":["text\/plain"],"uri":"\/wp-content\/themes\/grizzly\/img5.php?u=mfymi71rapdzk","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714364","orig_fuids":["Fl9usz3US821agwyb"],"ip_src_port":49202,"enrichments.geo.ip_dst_addr.location_point":"34.0494,-118.2641","status_msg":"OK","guid":"cb82531d-9a62-47a9-ac17-c9cbe841be64","response_body_len":45662} +{"adapter.threatinteladapter.end.ts":"1530978714364","bro_timestamp":"1530978712.051866","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978714346","enrichmentsplitterbolt.splitter.begin.ts":"1530978714346","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","adapter.geoadapter.begin.ts":"1530978714353","uid":"CUrRne3iLIxXavQtci","trans_depth":129,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/clusters\/metron_cluster\/alerts?format=groupedSummary&_=1484168877436 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:129 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978712.051866 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978714421","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978714359","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978712051,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/clusters\/metron_cluster\/alerts?format=groupedSummary&_=1484168877436","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714364","ip_src_port":50451,"guid":"24cd0763-288e-4353-ba28-5af2da4363de","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978714364","bro_timestamp":"1530978712.535296","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978714347","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978714347","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978714353","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CX9L2c29ZYGsLN10n5","resp_mime_types":["text\/plain"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49206 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/style.css tags:[] uid:CX9L2c29ZYGsLN10n5 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"text\\\/plain\"] trans_depth:1 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:4492 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978712.535296 id.resp_h:95.163.121.204 resp_fuids:[\"Fzfuja3jgtxmJaOSLl\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978714421","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978714359","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["Fzfuja3jgtxmJaOSLl"],"timestamp":1530978712535,"method":"GET","request_body_len":0,"uri":"\/img\/style.css","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714364","ip_src_port":49206,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"c7e9c878-0a2e-4312-b273-fa956293e614","response_body_len":4492} +{"adapter.threatinteladapter.end.ts":"1530978714364","bro_timestamp":"1530978712.06239","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978714348","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978714348","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978714353","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"Cmwl0u2GEZFDTv6OLh","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49184 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/?285a4d4e4e5a4d4d4649584c5d43064b4745 tags:[] uid:Cmwl0u2GEZFDTv6OLh resp_mime_types:[\"text\\\/html\"] trans_depth:1 host:va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in status_msg:OK id.orig_h:192.168.138.158 response_body_len:560 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978712.06239 id.resp_h:62.75.195.236 resp_fuids:[\"FJRfjlMKNsEu63NAh\"]","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978714421","host":"va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in","enrichmentjoinbolt.joiner.ts":"1530978714359","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FJRfjlMKNsEu63NAh"],"timestamp":1530978712062,"method":"GET","request_body_len":0,"uri":"\/?285a4d4e4e5a4d4d4649584c5d43064b4745","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714364","ip_src_port":49184,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"5a67d7b6-a747-4cf2-bcda-7d8d855b8454","response_body_len":560} +{"adapter.threatinteladapter.end.ts":"1530978714364","bro_timestamp":"1530978712.296445","status_code":404,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978714349","enrichments.geo.ip_dst_addr.city":"Phoenix","enrichments.geo.ip_dst_addr.latitude":"33.4499","enrichmentsplitterbolt.splitter.begin.ts":"1530978714348","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","enrichments.geo.ip_dst_addr.country":"US","enrichments.geo.ip_dst_addr.locID":"5308655","adapter.geoadapter.begin.ts":"1530978714353","enrichments.geo.ip_dst_addr.postalCode":"85004","uid":"CDW8Tf2Tcs6fh21wG2","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49199 status_code:404 method:POST request_body_len:96 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/wp-content\/themes\/twentyfifteen\/img5.php?l=8r1gf1b2t1kuq42 tags:[] uid:CDW8Tf2Tcs6fh21wG2 resp_mime_types:[\"text\\\/html\"] trans_depth:1 orig_fuids:[\"FkHP9x3nWdAJWySSUf\"] host:runlove.us status_msg:Not Found id.orig_h:192.168.138.158 response_body_len:357 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978712.296445 id.resp_h:204.152.254.221 resp_fuids:[\"FJOPgf2GhG5SBYMCo7\"]","ip_dst_addr":"204.152.254.221","threatinteljoinbolt.joiner.ts":"1530978714421","enrichments.geo.ip_dst_addr.dmaCode":"753","host":"runlove.us","enrichmentjoinbolt.joiner.ts":"1530978714360","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","enrichments.geo.ip_dst_addr.longitude":"-112.0712","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FJOPgf2GhG5SBYMCo7"],"timestamp":1530978712296,"method":"POST","request_body_len":96,"orig_mime_types":["text\/plain"],"uri":"\/wp-content\/themes\/twentyfifteen\/img5.php?l=8r1gf1b2t1kuq42","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714364","orig_fuids":["FkHP9x3nWdAJWySSUf"],"ip_src_port":49199,"enrichments.geo.ip_dst_addr.location_point":"33.4499,-112.0712","status_msg":"Not Found","guid":"6a265634-59a6-4983-a9c1-051bc7b8a19d","response_body_len":357} +{"adapter.threatinteladapter.end.ts":"1530978714364","bro_timestamp":"1530978712.630998","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978714349","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978714349","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978714353","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"Cm8nbh1mEqDSWqLB61","resp_mime_types":["image\/png"],"trans_depth":3,"protocol":"http","original_string":"HTTP | id.orig_p:49210 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/button_pay.png tags:[] uid:Cm8nbh1mEqDSWqLB61 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:3 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:727 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978712.630998 id.resp_h:95.163.121.204 resp_fuids:[\"F4UU9y2L5THk5eQzNl\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978714422","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978714360","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["F4UU9y2L5THk5eQzNl"],"timestamp":1530978712630,"method":"GET","request_body_len":0,"uri":"\/img\/button_pay.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714364","ip_src_port":49210,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"310ef43e-7a93-4111-9035-f6a96aa51054","response_body_len":727} +{"TTLs":[21599.0],"adapter.threatinteladapter.end.ts":"1530978714365","qclass_name":"C_INTERNET","bro_timestamp":"1530978712.462277","qtype_name":"A","ip_dst_port":53,"enrichmentsplitterbolt.splitter.end.ts":"1530978714349","qtype":1,"rejected":false,"answers":["188.165.164.184"],"enrichmentsplitterbolt.splitter.begin.ts":"1530978714349","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","trans_id":15553,"adapter.geoadapter.begin.ts":"1530978714353","uid":"CbGIVF37GpnLr9Yl9b","protocol":"dns","original_string":"DNS | AA:false TTLs:[21599.0] qclass_name:C_INTERNET id.orig_p:53571 qtype_name:A qtype:1 rejected:false id.resp_p:53 query:ip-addr.es answers:[\"188.165.164.184\"] trans_id:15553 rcode:0 rcode_name:NOERROR TC:false RA:true uid:CbGIVF37GpnLr9Yl9b RD:true proto:udp id.orig_h:192.168.138.158 Z:0 qclass:1 ts:1530978712.462277 id.resp_h:192.168.138.2","ip_dst_addr":"192.168.138.2","threatinteljoinbolt.joiner.ts":"1530978714422","enrichmentjoinbolt.joiner.ts":"1530978714360","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","Z":0,"ip_src_addr":"192.168.138.158","qclass":1,"timestamp":1530978712462,"AA":false,"query":"ip-addr.es","rcode":0,"rcode_name":"NOERROR","TC":false,"RA":true,"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","RD":true,"threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714364","ip_src_port":53571,"proto":"udp","guid":"9cea11f4-a8fa-4d01-a50e-0cf67e593b2b"} +{"adapter.threatinteladapter.end.ts":"1530978714365","bro_timestamp":"1530978712.979947","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978714349","enrichments.geo.ip_dst_addr.city":"Los Angeles","enrichments.geo.ip_dst_addr.latitude":"34.0494","enrichmentsplitterbolt.splitter.begin.ts":"1530978714349","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","enrichments.geo.ip_dst_addr.country":"US","enrichments.geo.ip_dst_addr.locID":"5368361","adapter.geoadapter.begin.ts":"1530978714353","enrichments.geo.ip_dst_addr.postalCode":"90014","uid":"COZAhy4ljJ4lBc5bgf","resp_mime_types":["text\/plain"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49204 status_code:200 method:POST request_body_len:110 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/wp-content\/themes\/grizzly\/img5.php?u=ka6nnuvccqlw9 tags:[] uid:COZAhy4ljJ4lBc5bgf resp_mime_types:[\"text\\\/plain\"] trans_depth:1 orig_fuids:[\"FgncKy2eauwZjDL6h9\"] host:comarksecurity.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:14 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978712.979947 id.resp_h:72.34.49.86 resp_fuids:[\"FDVxtiyWLP0KeNRg8\"]","ip_dst_addr":"72.34.49.86","threatinteljoinbolt.joiner.ts":"1530978714422","enrichments.geo.ip_dst_addr.dmaCode":"803","host":"comarksecurity.com","enrichmentjoinbolt.joiner.ts":"1530978714360","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","enrichments.geo.ip_dst_addr.longitude":"-118.2641","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FDVxtiyWLP0KeNRg8"],"timestamp":1530978712979,"method":"POST","request_body_len":110,"orig_mime_types":["text\/plain"],"uri":"\/wp-content\/themes\/grizzly\/img5.php?u=ka6nnuvccqlw9","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714365","orig_fuids":["FgncKy2eauwZjDL6h9"],"ip_src_port":49204,"enrichments.geo.ip_dst_addr.location_point":"34.0494,-118.2641","status_msg":"OK","guid":"363f87f6-73aa-4737-9090-0ffb8568fe6e","response_body_len":14} +{"adapter.threatinteladapter.end.ts":"1530978714365","bro_timestamp":"1530978712.964196","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978714349","enrichmentsplitterbolt.splitter.begin.ts":"1530978714349","adapter.hostfromjsonlistadapter.end.ts":"1530978714351","adapter.geoadapter.begin.ts":"1530978714353","uid":"CUrRne3iLIxXavQtci","trans_depth":234,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/clusters\/metron_cluster\/components\/?ServiceComponentInfo\/component_name=APP_TIMELINE_SERVER|ServiceComponentInfo\/category=MASTER&fields=ServiceComponentInfo\/service_name,host_components\/HostRoles\/display_name,host_components\/HostRoles\/host_name,host_components\/HostRoles\/state,host_components\/HostRoles\/maintenance_state,host_components\/HostRoles\/stale_configs,host_components\/HostRoles\/ha_state,host_components\/HostRoles\/desired_admin_state,,host_components\/metrics\/jvm\/memHeapUsedM,host_components\/metrics\/jvm\/HeapMemoryMax,host_components\/metrics\/jvm\/HeapMemoryUsed,host_components\/metrics\/jvm\/memHeapCommittedM,host_components\/metrics\/mapred\/jobtracker\/trackers_decommissioned,host_components\/metrics\/cpu\/cpu_wio,host_components\/metrics\/rpc\/client\/RpcQueueTime_avg_time,host_components\/metrics\/dfs\/FSNamesystem\/*,host_components\/metrics\/dfs\/namenode\/Version,host_components\/metrics\/dfs\/namenode\/LiveNodes,host_components\/metrics\/dfs\/namenode\/DeadNodes,host_components\/metrics\/dfs\/namenode\/DecomNodes,host_components\/metrics\/dfs\/namenode\/TotalFiles,host_components\/metrics\/dfs\/namenode\/UpgradeFinalized,host_components\/metrics\/dfs\/namenode\/Safemode,host_components\/metrics\/runtime\/StartTime,host_components\/metrics\/hbase\/master\/IsActiveMaster,host_components\/metrics\/hbase\/master\/MasterStartTime,host_components\/metrics\/hbase\/master\/MasterActiveTime,host_components\/metrics\/hbase\/master\/AverageLoad,host_components\/metrics\/master\/AssignmentManger\/ritCount,metrics\/api\/v1\/cluster\/summary,metrics\/api\/v1\/topology\/summary,metrics\/api\/v1\/nimbus\/summary,host_components\/metrics\/yarn\/Queue,host_components\/metrics\/yarn\/ClusterMetrics\/NumActiveNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumLostNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumUnhealthyNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumRebootedNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumDecommissionedNMs&minimal_response=true&_=1484169361350 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:234 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978712.964196 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978714422","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978714360","adapter.hostfromjsonlistadapter.begin.ts":"1530978714351","threatintelsplitterbolt.splitter.begin.ts":"1530978714362","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978712964,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/clusters\/metron_cluster\/components\/?ServiceComponentInfo\/component_name=APP_TIMELINE_SERVER|ServiceComponentInfo\/category=MASTER&fields=ServiceComponentInfo\/service_name,host_components\/HostRoles\/display_name,host_components\/HostRoles\/host_name,host_components\/HostRoles\/state,host_components\/HostRoles\/maintenance_state,host_components\/HostRoles\/stale_configs,host_components\/HostRoles\/ha_state,host_components\/HostRoles\/desired_admin_state,,host_components\/metrics\/jvm\/memHeapUsedM,host_components\/metrics\/jvm\/HeapMemoryMax,host_components\/metrics\/jvm\/HeapMemoryUsed,host_components\/metrics\/jvm\/memHeapCommittedM,host_components\/metrics\/mapred\/jobtracker\/trackers_decommissioned,host_components\/metrics\/cpu\/cpu_wio,host_components\/metrics\/rpc\/client\/RpcQueueTime_avg_time,host_components\/metrics\/dfs\/FSNamesystem\/*,host_components\/metrics\/dfs\/namenode\/Version,host_components\/metrics\/dfs\/namenode\/LiveNodes,host_components\/metrics\/dfs\/namenode\/DeadNodes,host_components\/metrics\/dfs\/namenode\/DecomNodes,host_components\/metrics\/dfs\/namenode\/TotalFiles,host_components\/metrics\/dfs\/namenode\/UpgradeFinalized,host_components\/metrics\/dfs\/namenode\/Safemode,host_components\/metrics\/runtime\/StartTime,host_components\/metrics\/hbase\/master\/IsActiveMaster,host_components\/metrics\/hbase\/master\/MasterStartTime,host_components\/metrics\/hbase\/master\/MasterActiveTime,host_components\/metrics\/hbase\/master\/AverageLoad,host_components\/metrics\/master\/AssignmentManger\/ritCount,metrics\/api\/v1\/cluster\/summary,metrics\/api\/v1\/topology\/summary,metrics\/api\/v1\/nimbus\/summary,host_components\/metrics\/yarn\/Queue,host_components\/metrics\/yarn\/ClusterMetrics\/NumActiveNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumLostNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumUnhealthyNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumRebootedNMs,host_components\/metrics\/yarn\/ClusterMetrics\/NumDecommissionedNMs&minimal_response=true&_=1484169361350","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978714353","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978714362","adapter.threatinteladapter.begin.ts":"1530978714365","ip_src_port":50451,"guid":"cb259950-845c-4fd3-bf57-e5da6a024244","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978721981","bro_timestamp":"1530978716.980649","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978721955","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978721954","adapter.hostfromjsonlistadapter.end.ts":"1530978721960","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978721960","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"CeScgBTiBLSNBBT39","trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49191 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/?3a08b0be8322c244f5a1cb9c1057d941 tags:[] uid:CeScgBTiBLSNBBT39 trans_depth:1 host:62.75.195.236 status_msg:OK id.orig_h:192.168.138.158 response_body_len:0 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978716.980649 id.resp_h:62.75.195.236","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978721985","host":"62.75.195.236","enrichmentjoinbolt.joiner.ts":"1530978721967","adapter.hostfromjsonlistadapter.begin.ts":"1530978721960","threatintelsplitterbolt.splitter.begin.ts":"1530978721974","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","timestamp":1530978716980,"method":"GET","request_body_len":0,"uri":"\/?3a08b0be8322c244f5a1cb9c1057d941","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978721960","threatintelsplitterbolt.splitter.end.ts":"1530978721974","adapter.threatinteladapter.begin.ts":"1530978721981","ip_src_port":49191,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"22f56903-ea89-4207-8385-ac2de659da35","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978721984","bro_timestamp":"1530978716.395041","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978721955","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978721955","adapter.hostfromjsonlistadapter.end.ts":"1530978721960","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978721960","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CdmrWIO9csTm3yLv9","resp_mime_types":["image\/png"],"trans_depth":2,"protocol":"http","original_string":"HTTP | id.orig_p:49205 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/flags\/us.png tags:[] uid:CdmrWIO9csTm3yLv9 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:2 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:825 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978716.395041 id.resp_h:95.163.121.204 resp_fuids:[\"FvyeyH3W8TkTNUHi88\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978721990","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978721969","adapter.hostfromjsonlistadapter.begin.ts":"1530978721960","threatintelsplitterbolt.splitter.begin.ts":"1530978721979","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FvyeyH3W8TkTNUHi88"],"timestamp":1530978716395,"method":"GET","request_body_len":0,"uri":"\/img\/flags\/us.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978721960","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978721980","adapter.threatinteladapter.begin.ts":"1530978721984","ip_src_port":49205,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"4079606b-b5b2-414c-b8d1-b635ab7feba2","response_body_len":825} +{"TTLs":[13888.0],"adapter.threatinteladapter.end.ts":"1530978721984","qclass_name":"C_INTERNET","bro_timestamp":"1530978716.362076","qtype_name":"A","ip_dst_port":53,"enrichmentsplitterbolt.splitter.end.ts":"1530978721957","qtype":1,"rejected":false,"answers":["72.34.49.86"],"enrichmentsplitterbolt.splitter.begin.ts":"1530978721956","adapter.hostfromjsonlistadapter.end.ts":"1530978721960","trans_id":41589,"adapter.geoadapter.begin.ts":"1530978721960","uid":"Cw3zPc1Lrwkxsn4oK","protocol":"dns","original_string":"DNS | AA:false TTLs:[13888.0] qclass_name:C_INTERNET id.orig_p:56753 qtype_name:A qtype:1 rejected:false id.resp_p:53 query:comarksecurity.com answers:[\"72.34.49.86\"] trans_id:41589 rcode:0 rcode_name:NOERROR TC:false RA:true uid:Cw3zPc1Lrwkxsn4oK RD:true proto:udp id.orig_h:192.168.138.158 Z:0 qclass:1 ts:1530978716.362076 id.resp_h:192.168.138.2","ip_dst_addr":"192.168.138.2","threatinteljoinbolt.joiner.ts":"1530978721990","enrichmentjoinbolt.joiner.ts":"1530978721970","adapter.hostfromjsonlistadapter.begin.ts":"1530978721960","threatintelsplitterbolt.splitter.begin.ts":"1530978721980","Z":0,"ip_src_addr":"192.168.138.158","qclass":1,"timestamp":1530978716362,"AA":false,"query":"comarksecurity.com","rcode":0,"rcode_name":"NOERROR","TC":false,"RA":true,"source.type":"bro","adapter.geoadapter.end.ts":"1530978721960","RD":true,"threatintelsplitterbolt.splitter.end.ts":"1530978721980","adapter.threatinteladapter.begin.ts":"1530978721984","ip_src_port":56753,"proto":"udp","guid":"277c57ce-577c-441d-b66a-592b1440750f"} +{"adapter.threatinteladapter.end.ts":"1530978721984","bro_timestamp":"1530978716.529291","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978721957","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978721957","adapter.hostfromjsonlistadapter.end.ts":"1530978721960","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978721960","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CU2iXc2cpNDX4bSQg4","resp_mime_types":["image\/png"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49210 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/lt.png tags:[] uid:CU2iXc2cpNDX4bSQg4 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:1 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:240 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978716.529291 id.resp_h:95.163.121.204 resp_fuids:[\"FJVM34yr7CRJiObpb\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978721990","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978721970","adapter.hostfromjsonlistadapter.begin.ts":"1530978721960","threatintelsplitterbolt.splitter.begin.ts":"1530978721980","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FJVM34yr7CRJiObpb"],"timestamp":1530978716529,"method":"GET","request_body_len":0,"uri":"\/img\/lt.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978721960","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978721980","adapter.threatinteladapter.begin.ts":"1530978721984","ip_src_port":49210,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"af5d8842-4e9a-4a90-ac6d-feb4357168ba","response_body_len":240} +{"adapter.threatinteladapter.end.ts":"1530978721984","bro_timestamp":"1530978716.267981","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978721957","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978721957","adapter.hostfromjsonlistadapter.end.ts":"1530978721960","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978721960","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CAMU6hVkiTheUH4z4","resp_mime_types":["image\/png"],"trans_depth":2,"protocol":"http","original_string":"HTTP | id.orig_p:49210 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/lb.png tags:[] uid:CAMU6hVkiTheUH4z4 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:2 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:239 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978716.267981 id.resp_h:95.163.121.204 resp_fuids:[\"FAleJd1zw6L9UPfNii\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978721990","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978721970","adapter.hostfromjsonlistadapter.begin.ts":"1530978721960","threatintelsplitterbolt.splitter.begin.ts":"1530978721980","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FAleJd1zw6L9UPfNii"],"timestamp":1530978716267,"method":"GET","request_body_len":0,"uri":"\/img\/lb.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978721960","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978721980","adapter.threatinteladapter.begin.ts":"1530978721984","ip_src_port":49210,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"e1e321c7-f04c-46c3-a09a-5c9aba818f82","response_body_len":239} +{"adapter.threatinteladapter.end.ts":"1530978721984","bro_timestamp":"1530978716.421926","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978721957","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978721957","adapter.hostfromjsonlistadapter.end.ts":"1530978721960","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978721960","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"Ci6H8p2J9hTYmaR6Xj","trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49191 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/?3a08b0be8322c244f5a1cb9c1057d941 tags:[] uid:Ci6H8p2J9hTYmaR6Xj trans_depth:1 host:62.75.195.236 status_msg:OK id.orig_h:192.168.138.158 response_body_len:0 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978716.421926 id.resp_h:62.75.195.236","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978721991","host":"62.75.195.236","enrichmentjoinbolt.joiner.ts":"1530978721970","adapter.hostfromjsonlistadapter.begin.ts":"1530978721960","threatintelsplitterbolt.splitter.begin.ts":"1530978721980","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","timestamp":1530978716421,"method":"GET","request_body_len":0,"uri":"\/?3a08b0be8322c244f5a1cb9c1057d941","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978721960","threatintelsplitterbolt.splitter.end.ts":"1530978721980","adapter.threatinteladapter.begin.ts":"1530978721984","ip_src_port":49191,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"2e1f0eb2-76de-4064-8acc-7f001f1459fa","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978721984","bro_timestamp":"1530978716.788839","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978721958","enrichmentsplitterbolt.splitter.begin.ts":"1530978721957","adapter.hostfromjsonlistadapter.end.ts":"1530978721960","adapter.geoadapter.begin.ts":"1530978721960","uid":"CUrRne3iLIxXavQtci","trans_depth":200,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/clusters\/metron_cluster\/services?fields=ServiceInfo\/state,ServiceInfo\/maintenance_state,components\/ServiceComponentInfo\/component_name&minimal_response=true&_=1484169228168 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:200 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978716.788839 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978721991","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978721971","adapter.hostfromjsonlistadapter.begin.ts":"1530978721960","threatintelsplitterbolt.splitter.begin.ts":"1530978721980","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978716788,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/clusters\/metron_cluster\/services?fields=ServiceInfo\/state,ServiceInfo\/maintenance_state,components\/ServiceComponentInfo\/component_name&minimal_response=true&_=1484169228168","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978721960","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978721980","adapter.threatinteladapter.begin.ts":"1530978721984","ip_src_port":50451,"guid":"8c559118-72a7-4df9-a876-34c0e2fb82e5","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978721984","bro_timestamp":"1530978716.778337","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978721958","enrichments.geo.ip_dst_addr.city":"Los Angeles","enrichments.geo.ip_dst_addr.latitude":"34.0494","enrichmentsplitterbolt.splitter.begin.ts":"1530978721958","adapter.hostfromjsonlistadapter.end.ts":"1530978721960","enrichments.geo.ip_dst_addr.country":"US","enrichments.geo.ip_dst_addr.locID":"5368361","adapter.geoadapter.begin.ts":"1530978721960","enrichments.geo.ip_dst_addr.postalCode":"90014","uid":"Cbe8Jk2tJb38gjFUJ1","resp_mime_types":["image\/png"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49202 status_code:200 method:POST request_body_len:162 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/wp-content\/themes\/grizzly\/img5.php?u=mfymi71rapdzk tags:[] uid:Cbe8Jk2tJb38gjFUJ1 resp_mime_types:[\"image\\\/png\"] trans_depth:1 orig_fuids:[\"F0mGwV142T4ZO12UIe\"] host:comarksecurity.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:45662 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978716.778337 id.resp_h:72.34.49.86 resp_fuids:[\"FCYJUSdhsQQ0aRjQc\"]","ip_dst_addr":"72.34.49.86","threatinteljoinbolt.joiner.ts":"1530978721991","enrichments.geo.ip_dst_addr.dmaCode":"803","host":"comarksecurity.com","enrichmentjoinbolt.joiner.ts":"1530978721971","adapter.hostfromjsonlistadapter.begin.ts":"1530978721960","threatintelsplitterbolt.splitter.begin.ts":"1530978721980","enrichments.geo.ip_dst_addr.longitude":"-118.2641","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FCYJUSdhsQQ0aRjQc"],"timestamp":1530978716778,"method":"POST","request_body_len":162,"orig_mime_types":["text\/plain"],"uri":"\/wp-content\/themes\/grizzly\/img5.php?u=mfymi71rapdzk","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978721960","threatintelsplitterbolt.splitter.end.ts":"1530978721980","adapter.threatinteladapter.begin.ts":"1530978721984","orig_fuids":["F0mGwV142T4ZO12UIe"],"ip_src_port":49202,"enrichments.geo.ip_dst_addr.location_point":"34.0494,-118.2641","status_msg":"OK","guid":"598faabb-e1ae-42f3-9858-9ff491849ce4","response_body_len":45662} +{"adapter.threatinteladapter.end.ts":"1530978721985","bro_timestamp":"1530978716.814225","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978721966","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978721966","adapter.hostfromjsonlistadapter.end.ts":"1530978721970","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978721970","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"CW5RvsMStnenkVMN9","resp_mime_types":["application\/x-dosexec"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49189 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/?b514ee6f0fe486009a6d83b035a4c0bd tags:[] uid:CW5RvsMStnenkVMN9 resp_mime_types:[\"application\\\/x-dosexec\"] trans_depth:1 host:62.75.195.236 status_msg:OK id.orig_h:192.168.138.158 response_body_len:221184 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978716.814225 id.resp_h:62.75.195.236 resp_fuids:[\"FJbBkl1yTXU8JMGR4l\"]","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978721991","host":"62.75.195.236","enrichmentjoinbolt.joiner.ts":"1530978721973","adapter.hostfromjsonlistadapter.begin.ts":"1530978721970","threatintelsplitterbolt.splitter.begin.ts":"1530978721981","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FJbBkl1yTXU8JMGR4l"],"timestamp":1530978716814,"method":"GET","request_body_len":0,"uri":"\/?b514ee6f0fe486009a6d83b035a4c0bd","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978721970","threatintelsplitterbolt.splitter.end.ts":"1530978721982","adapter.threatinteladapter.begin.ts":"1530978721985","ip_src_port":49189,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"2edc3fb8-bc67-4134-af08-45a45ef9b8f6","response_body_len":221184} +{"adapter.threatinteladapter.end.ts":"1530978721997","bro_timestamp":"1530978716.588208","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978721967","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978721967","adapter.hostfromjsonlistadapter.end.ts":"1530978721970","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978721970","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CDRWth1RkZQBuVOyX2","resp_mime_types":["image\/x-icon"],"trans_depth":2,"protocol":"http","original_string":"HTTP | id.orig_p:49207 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/favicon.ico tags:[] uid:CDRWth1RkZQBuVOyX2 resp_mime_types:[\"image\\\/x-icon\"] trans_depth:2 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:318 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978716.588208 id.resp_h:95.163.121.204 resp_fuids:[\"FJygKu2sv9kbVQLbhh\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978722000","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978721980","adapter.hostfromjsonlistadapter.begin.ts":"1530978721970","threatintelsplitterbolt.splitter.begin.ts":"1530978721986","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FJygKu2sv9kbVQLbhh"],"timestamp":1530978716588,"method":"GET","request_body_len":0,"uri":"\/favicon.ico","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978721970","threatintelsplitterbolt.splitter.end.ts":"1530978721986","adapter.threatinteladapter.begin.ts":"1530978721997","ip_src_port":49207,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"37680d59-d404-4034-afcc-58e3ab40ef13","response_body_len":318} +{"adapter.threatinteladapter.end.ts":"1530978721997","bro_timestamp":"1530978720.836714","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978721968","enrichmentsplitterbolt.splitter.begin.ts":"1530978721967","adapter.hostfromjsonlistadapter.end.ts":"1530978721970","adapter.geoadapter.begin.ts":"1530978721970","uid":"CUrRne3iLIxXavQtci","trans_depth":194,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/clusters\/metron_cluster\/requests?to=end&page_size=10&fields=Requests&_=1484169178878 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:194 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978720.836714 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978722000","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978721980","adapter.hostfromjsonlistadapter.begin.ts":"1530978721970","threatintelsplitterbolt.splitter.begin.ts":"1530978721986","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978720836,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/clusters\/metron_cluster\/requests?to=end&page_size=10&fields=Requests&_=1484169178878","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978721970","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978721986","adapter.threatinteladapter.begin.ts":"1530978721997","ip_src_port":50451,"guid":"a18186b3-c18a-43d6-b963-4984871d7e34","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978722005","bro_timestamp":"1530978720.52134","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978721980","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978721979","adapter.hostfromjsonlistadapter.end.ts":"1530978721993","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978721994","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"Cniyw631RU6tL0SDij","trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49188 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/aa25f5fe2875e3d0a244e6969e589cc4 tags:[] uid:Cniyw631RU6tL0SDij trans_depth:1 host:62.75.195.236 status_msg:OK id.orig_h:192.168.138.158 response_body_len:861 ts:1530978720.52134 id.resp_h:62.75.195.236 resp_fuids:[\"FBQW2X3B0fup5s5Ln9\"]","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978722008","host":"62.75.195.236","enrichmentjoinbolt.joiner.ts":"1530978721998","adapter.hostfromjsonlistadapter.begin.ts":"1530978721993","threatintelsplitterbolt.splitter.begin.ts":"1530978722002","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","resp_fuids":["FBQW2X3B0fup5s5Ln9"],"timestamp":1530978720521,"method":"GET","request_body_len":0,"uri":"\/aa25f5fe2875e3d0a244e6969e589cc4","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978721994","threatintelsplitterbolt.splitter.end.ts":"1530978722002","adapter.threatinteladapter.begin.ts":"1530978722005","ip_src_port":49188,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"045cbfab-748d-4bd0-adf5-24bf2a016a8b","response_body_len":861} +{"adapter.threatinteladapter.end.ts":"1530978722005","bro_timestamp":"1530978720.080146","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978721980","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978721980","adapter.hostfromjsonlistadapter.end.ts":"1530978721993","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978721994","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"CitfgJ3XKMUwMApZeb","trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49193 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/?34eaf8bd50d85d8c6baacb45f0a7b22e tags:[] uid:CitfgJ3XKMUwMApZeb trans_depth:1 host:62.75.195.236 status_msg:OK id.orig_h:192.168.138.158 response_body_len:0 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978720.080146 id.resp_h:62.75.195.236","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978722008","host":"62.75.195.236","enrichmentjoinbolt.joiner.ts":"1530978721998","adapter.hostfromjsonlistadapter.begin.ts":"1530978721993","threatintelsplitterbolt.splitter.begin.ts":"1530978722002","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","timestamp":1530978720080,"method":"GET","request_body_len":0,"uri":"\/?34eaf8bd50d85d8c6baacb45f0a7b22e","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978721994","threatintelsplitterbolt.splitter.end.ts":"1530978722002","adapter.threatinteladapter.begin.ts":"1530978722005","ip_src_port":49193,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"bc61632d-5795-41ca-a0a9-e2530807be33","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978722005","qclass_name":"C_INTERNET","bro_timestamp":"1530978720.819805","qtype_name":"PTR","ip_dst_port":5353,"enrichmentsplitterbolt.splitter.end.ts":"1530978721980","qtype":12,"rejected":false,"enrichmentsplitterbolt.splitter.begin.ts":"1530978721980","adapter.hostfromjsonlistadapter.end.ts":"1530978721994","trans_id":0,"adapter.geoadapter.begin.ts":"1530978721994","uid":"CHXOlu44YwhzMgdZpj","protocol":"dns","original_string":"DNS | AA:false qclass_name:C_INTERNET id.orig_p:5353 qtype_name:PTR qtype:12 rejected:false id.resp_p:5353 query:_googlecast._tcp.local trans_id:0 TC:false RA:false uid:CHXOlu44YwhzMgdZpj RD:false proto:udp id.orig_h:192.168.66.1 Z:0 qclass:1 ts:1530978720.819805 id.resp_h:224.0.0.251","ip_dst_addr":"224.0.0.251","threatinteljoinbolt.joiner.ts":"1530978722008","enrichmentjoinbolt.joiner.ts":"1530978721998","adapter.hostfromjsonlistadapter.begin.ts":"1530978721994","threatintelsplitterbolt.splitter.begin.ts":"1530978722002","Z":0,"ip_src_addr":"192.168.66.1","qclass":1,"timestamp":1530978720819,"AA":false,"query":"_googlecast._tcp.local","TC":false,"RA":false,"source.type":"bro","adapter.geoadapter.end.ts":"1530978721994","RD":false,"threatintelsplitterbolt.splitter.end.ts":"1530978722002","adapter.threatinteladapter.begin.ts":"1530978722005","ip_src_port":5353,"proto":"udp","guid":"614f12d4-2ec4-4616-8b0a-95296278bf93"} +{"adapter.threatinteladapter.end.ts":"1530978722005","bro_timestamp":"1530978720.992674","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978721980","enrichmentsplitterbolt.splitter.begin.ts":"1530978721980","adapter.hostfromjsonlistadapter.end.ts":"1530978721994","adapter.geoadapter.begin.ts":"1530978721994","uid":"CUrRne3iLIxXavQtci","trans_depth":78,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/clusters\/metron_cluster?fields=Clusters\/health_report,Clusters\/total_hosts,alerts_summary_hosts&minimal_response=true&_=1484168593029 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:78 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978720.992674 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978722008","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978721998","adapter.hostfromjsonlistadapter.begin.ts":"1530978721994","threatintelsplitterbolt.splitter.begin.ts":"1530978722002","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978720992,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/clusters\/metron_cluster?fields=Clusters\/health_report,Clusters\/total_hosts,alerts_summary_hosts&minimal_response=true&_=1484168593029","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978721994","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978722002","adapter.threatinteladapter.begin.ts":"1530978722005","ip_src_port":50451,"guid":"30e3043b-84be-48d9-9862-ab09ad073b6d","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978725883","bro_timestamp":"1530978720.600685","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978725865","enrichmentsplitterbolt.splitter.begin.ts":"1530978725865","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","adapter.geoadapter.begin.ts":"1530978725868","uid":"CUrRne3iLIxXavQtci","trans_depth":66,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/clusters\/metron_cluster\/alerts?format=groupedSummary&_=1484168566981 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:66 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978720.600685 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978725886","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978725877","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725881","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978720600,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/clusters\/metron_cluster\/alerts?format=groupedSummary&_=1484168566981","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978725868","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978725881","adapter.threatinteladapter.begin.ts":"1530978725883","ip_src_port":50451,"guid":"60abfcb4-fed1-4867-9974-c3c02326e6a7","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978725883","bro_timestamp":"1530978720.419558","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978725866","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978725866","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978725869","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CnlAPI2zWmgFfLxi3j","resp_mime_types":["image\/png"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49207 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/flags\/es.png tags:[] uid:CnlAPI2zWmgFfLxi3j referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:1 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:634 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978720.419558 id.resp_h:95.163.121.204 resp_fuids:[\"FQZuDx2tatMj8BVT2a\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978725887","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978725877","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725881","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FQZuDx2tatMj8BVT2a"],"timestamp":1530978720419,"method":"GET","request_body_len":0,"uri":"\/img\/flags\/es.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978725881","adapter.threatinteladapter.begin.ts":"1530978725883","ip_src_port":49207,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"3ac5455c-5dae-412a-925c-c15e647b1554","response_body_len":634} +{"adapter.threatinteladapter.end.ts":"1530978725883","bro_timestamp":"1530978720.723386","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978725866","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978725866","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978725869","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"Cc9Ml62nY1dMHMe475","resp_mime_types":["image\/png"],"trans_depth":2,"protocol":"http","original_string":"HTTP | id.orig_p:49210 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/lb.png tags:[] uid:Cc9Ml62nY1dMHMe475 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:2 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:239 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978720.723386 id.resp_h:95.163.121.204 resp_fuids:[\"FAjI4U27ebPkwNwsCe\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978725887","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978725877","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725881","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FAjI4U27ebPkwNwsCe"],"timestamp":1530978720723,"method":"GET","request_body_len":0,"uri":"\/img\/lb.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978725881","adapter.threatinteladapter.begin.ts":"1530978725883","ip_src_port":49210,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"9b9046f8-0a15-4144-8b34-c3ea6fa8ba6d","response_body_len":239} +{"adapter.threatinteladapter.end.ts":"1530978725883","bro_timestamp":"1530978720.511788","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978725866","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978725866","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978725869","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"CRGLdEasAJUDL8Tu4","resp_mime_types":["application\/x-shockwave-flash"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49185 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:CRGLdEasAJUDL8Tu4 referrer:http:\/\/va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in\/?285a4d4e4e5a4d4d4649584c5d43064b4745 resp_mime_types:[\"application\\\/x-shockwave-flash\"] trans_depth:1 host:ubb67.3c147o.u806a4.w07d919.o5f.f1.b80w.r0faf9.e8mfzdgrf7g0.groupprograms.in status_msg:OK id.orig_h:192.168.138.158 response_body_len:8973 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978720.511788 id.resp_h:62.75.195.236 resp_fuids:[\"FHMpUl2B1lUkpzZoQi\"]","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978725887","host":"ubb67.3c147o.u806a4.w07d919.o5f.f1.b80w.r0faf9.e8mfzdgrf7g0.groupprograms.in","enrichmentjoinbolt.joiner.ts":"1530978725877","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725881","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FHMpUl2B1lUkpzZoQi"],"timestamp":1530978720511,"method":"GET","request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","referrer":"http:\/\/va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in\/?285a4d4e4e5a4d4d4649584c5d43064b4745","threatintelsplitterbolt.splitter.end.ts":"1530978725881","adapter.threatinteladapter.begin.ts":"1530978725883","ip_src_port":49185,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"9ee8ee16-daf7-4115-9e8f-e0c022eb8217","response_body_len":8973} +{"adapter.threatinteladapter.end.ts":"1530978725883","qclass_name":"C_INTERNET","bro_timestamp":"1530978720.863134","qtype_name":"PTR","ip_dst_port":5353,"enrichmentsplitterbolt.splitter.end.ts":"1530978725866","qtype":12,"rejected":false,"enrichmentsplitterbolt.splitter.begin.ts":"1530978725866","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","trans_id":0,"adapter.geoadapter.begin.ts":"1530978725869","uid":"CeViBZ1CapumWgfFd3","protocol":"dns","original_string":"DNS | AA:false qclass_name:C_INTERNET id.orig_p:5353 qtype_name:PTR qtype:12 rejected:false id.resp_p:5353 query:_googlecast._tcp.local trans_id:0 TC:false RA:false uid:CeViBZ1CapumWgfFd3 RD:false proto:udp id.orig_h:192.168.66.1 Z:0 qclass:1 ts:1530978720.863134 id.resp_h:224.0.0.251","ip_dst_addr":"224.0.0.251","threatinteljoinbolt.joiner.ts":"1530978725887","enrichmentjoinbolt.joiner.ts":"1530978725877","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725881","Z":0,"ip_src_addr":"192.168.66.1","qclass":1,"timestamp":1530978720863,"AA":false,"query":"_googlecast._tcp.local","TC":false,"RA":false,"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","RD":false,"threatintelsplitterbolt.splitter.end.ts":"1530978725881","adapter.threatinteladapter.begin.ts":"1530978725883","ip_src_port":5353,"proto":"udp","guid":"eabdac1d-b441-4db5-a623-9a672c0ed23b"} +{"adapter.threatinteladapter.end.ts":"1530978725883","bro_timestamp":"1530978724.628751","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978725866","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978725866","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978725869","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CMVPGXJHk7JvyZCNa","resp_mime_types":["text\/html"],"trans_depth":2,"protocol":"http","original_string":"HTTP | id.orig_p:49209 status_code:200 method:POST request_body_len:14 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/11iQmfg tags:[] uid:CMVPGXJHk7JvyZCNa referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"text\\\/html\"] trans_depth:2 orig_fuids:[\"FF6cSD3gsoO9FhYLkh\"] host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:14641 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978724.628751 id.resp_h:95.163.121.204 resp_fuids:[\"F63yZr4e1MYOvR4Mi9\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978725887","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978725878","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725881","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["F63yZr4e1MYOvR4Mi9"],"timestamp":1530978724628,"method":"POST","request_body_len":14,"orig_mime_types":["text\/plain"],"uri":"\/11iQmfg","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978725881","adapter.threatinteladapter.begin.ts":"1530978725883","orig_fuids":["FF6cSD3gsoO9FhYLkh"],"ip_src_port":49209,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"e5e335df-d201-4bd5-8900-84ae22d4b479","response_body_len":14641} +{"adapter.threatinteladapter.end.ts":"1530978725883","bro_timestamp":"1530978724.057812","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978725866","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978725866","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978725869","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"CR2Y0926uadDbgt7A2","resp_mime_types":["application\/x-shockwave-flash"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49185 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:CR2Y0926uadDbgt7A2 referrer:http:\/\/va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in\/?285a4d4e4e5a4d4d4649584c5d43064b4745 resp_mime_types:[\"application\\\/x-shockwave-flash\"] trans_depth:1 host:ubb67.3c147o.u806a4.w07d919.o5f.f1.b80w.r0faf9.e8mfzdgrf7g0.groupprograms.in status_msg:OK id.orig_h:192.168.138.158 response_body_len:8973 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978724.057812 id.resp_h:62.75.195.236 resp_fuids:[\"FlWuOV1SeCMAoAtlc\"]","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978725887","host":"ubb67.3c147o.u806a4.w07d919.o5f.f1.b80w.r0faf9.e8mfzdgrf7g0.groupprograms.in","enrichmentjoinbolt.joiner.ts":"1530978725878","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725881","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FlWuOV1SeCMAoAtlc"],"timestamp":1530978724057,"method":"GET","request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","referrer":"http:\/\/va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in\/?285a4d4e4e5a4d4d4649584c5d43064b4745","threatintelsplitterbolt.splitter.end.ts":"1530978725881","adapter.threatinteladapter.begin.ts":"1530978725883","ip_src_port":49185,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"1368c61b-b97e-4a08-8250-11261e0a2853","response_body_len":8973} +{"adapter.threatinteladapter.end.ts":"1530978725883","bro_timestamp":"1530978724.468395","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978725866","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978725866","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978725869","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"CKC27s27NkdWd5dlzh","resp_mime_types":["application\/x-dosexec"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49189 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/?b514ee6f0fe486009a6d83b035a4c0bd tags:[] uid:CKC27s27NkdWd5dlzh resp_mime_types:[\"application\\\/x-dosexec\"] trans_depth:1 host:62.75.195.236 status_msg:OK id.orig_h:192.168.138.158 response_body_len:221184 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978724.468395 id.resp_h:62.75.195.236 resp_fuids:[\"FwC0pj2qXLNlZWorPe\"]","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978725887","host":"62.75.195.236","enrichmentjoinbolt.joiner.ts":"1530978725878","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725881","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FwC0pj2qXLNlZWorPe"],"timestamp":1530978724468,"method":"GET","request_body_len":0,"uri":"\/?b514ee6f0fe486009a6d83b035a4c0bd","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","threatintelsplitterbolt.splitter.end.ts":"1530978725881","adapter.threatinteladapter.begin.ts":"1530978725883","ip_src_port":49189,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"4e0c6951-54f8-49f9-b1c4-7986c020f4d3","response_body_len":221184} +{"adapter.threatinteladapter.end.ts":"1530978725884","bro_timestamp":"1530978724.348984","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978725866","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978725866","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978725869","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CvtyM13PVeCo2WyZzg","resp_mime_types":["image\/png"],"trans_depth":2,"protocol":"http","original_string":"HTTP | id.orig_p:49208 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/rb.png tags:[] uid:CvtyM13PVeCo2WyZzg referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:2 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:237 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978724.348984 id.resp_h:95.163.121.204 resp_fuids:[\"FPTipF484ZUebA4Q8k\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978725887","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978725880","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725882","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FPTipF484ZUebA4Q8k"],"timestamp":1530978724348,"method":"GET","request_body_len":0,"uri":"\/img\/rb.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978725882","adapter.threatinteladapter.begin.ts":"1530978725884","ip_src_port":49208,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"831bc8b8-aeb5-4cb7-b76b-577c7ae94fc9","response_body_len":237} +{"adapter.threatinteladapter.end.ts":"1530978725884","bro_timestamp":"1530978724.56314","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978725866","enrichments.geo.ip_dst_addr.city":"Los Angeles","enrichments.geo.ip_dst_addr.latitude":"34.0494","enrichmentsplitterbolt.splitter.begin.ts":"1530978725866","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","enrichments.geo.ip_dst_addr.country":"US","enrichments.geo.ip_dst_addr.locID":"5368361","adapter.geoadapter.begin.ts":"1530978725869","enrichments.geo.ip_dst_addr.postalCode":"90014","uid":"C56tVGV3yQ6s1VSj9","resp_mime_types":["text\/plain"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49200 status_code:200 method:POST request_body_len:96 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/wp-content\/themes\/grizzly\/img5.php?t=8r1gf1b2t1kuq42 tags:[] uid:C56tVGV3yQ6s1VSj9 resp_mime_types:[\"text\\\/plain\"] trans_depth:1 orig_fuids:[\"Fz3UYt2bSuP0vAIV08\"] host:comarksecurity.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:996 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978724.56314 id.resp_h:72.34.49.86 resp_fuids:[\"FEXpeXHD91rXhhyoc\"]","ip_dst_addr":"72.34.49.86","threatinteljoinbolt.joiner.ts":"1530978725887","enrichments.geo.ip_dst_addr.dmaCode":"803","host":"comarksecurity.com","enrichmentjoinbolt.joiner.ts":"1530978725880","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725882","enrichments.geo.ip_dst_addr.longitude":"-118.2641","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FEXpeXHD91rXhhyoc"],"timestamp":1530978724563,"method":"POST","request_body_len":96,"orig_mime_types":["text\/plain"],"uri":"\/wp-content\/themes\/grizzly\/img5.php?t=8r1gf1b2t1kuq42","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","threatintelsplitterbolt.splitter.end.ts":"1530978725882","adapter.threatinteladapter.begin.ts":"1530978725884","orig_fuids":["Fz3UYt2bSuP0vAIV08"],"ip_src_port":49200,"enrichments.geo.ip_dst_addr.location_point":"34.0494,-118.2641","status_msg":"OK","guid":"442494bc-df16-450e-9909-c0b65c5f4f52","response_body_len":996} +{"TTLs":[14277.0],"adapter.threatinteladapter.end.ts":"1530978725884","qclass_name":"C_INTERNET","bro_timestamp":"1530978724.929184","qtype_name":"A","ip_dst_port":53,"enrichmentsplitterbolt.splitter.end.ts":"1530978725866","qtype":1,"rejected":false,"answers":["95.163.121.204"],"enrichmentsplitterbolt.splitter.begin.ts":"1530978725866","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","trans_id":5810,"adapter.geoadapter.begin.ts":"1530978725869","uid":"CnArm31VD2mmBoGuG9","protocol":"dns","original_string":"DNS | AA:false TTLs:[14277.0] qclass_name:C_INTERNET id.orig_p:50329 qtype_name:A qtype:1 rejected:false id.resp_p:53 query:7oqnsnzwwnm6zb7y.gigapaysun.com answers:[\"95.163.121.204\"] trans_id:5810 rcode:0 rcode_name:NOERROR TC:false RA:true uid:CnArm31VD2mmBoGuG9 RD:true proto:udp id.orig_h:192.168.138.158 Z:0 qclass:1 ts:1530978724.929184 id.resp_h:192.168.138.2","ip_dst_addr":"192.168.138.2","threatinteljoinbolt.joiner.ts":"1530978725888","enrichmentjoinbolt.joiner.ts":"1530978725880","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725882","Z":0,"ip_src_addr":"192.168.138.158","qclass":1,"timestamp":1530978724929,"AA":false,"query":"7oqnsnzwwnm6zb7y.gigapaysun.com","rcode":0,"rcode_name":"NOERROR","TC":false,"RA":true,"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","RD":true,"threatintelsplitterbolt.splitter.end.ts":"1530978725882","adapter.threatinteladapter.begin.ts":"1530978725884","ip_src_port":50329,"proto":"udp","guid":"9f0c4dbc-fdfe-4ee0-9029-871d1664dd9e"} +{"adapter.threatinteladapter.end.ts":"1530978725884","bro_timestamp":"1530978724.442684","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978725866","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978725866","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978725869","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CXVtpNU35nZ84YA8","resp_mime_types":["text\/plain"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49206 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/style.css tags:[] uid:CXVtpNU35nZ84YA8 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"text\\\/plain\"] trans_depth:1 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:4492 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978724.442684 id.resp_h:95.163.121.204 resp_fuids:[\"FPdNKp4VPGRjd0dvRd\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978725888","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978725880","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725882","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FPdNKp4VPGRjd0dvRd"],"timestamp":1530978724442,"method":"GET","request_body_len":0,"uri":"\/img\/style.css","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978725882","adapter.threatinteladapter.begin.ts":"1530978725884","ip_src_port":49206,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"7bae6dbf-55c2-4668-9662-1cc1bf9e7485","response_body_len":4492} +{"adapter.threatinteladapter.end.ts":"1530978725884","bro_timestamp":"1530978724.584019","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978725867","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978725867","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978725869","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CVxPm9xkzN80U39i9","resp_mime_types":["image\/png"],"trans_depth":3,"protocol":"http","original_string":"HTTP | id.orig_p:49205 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/rt.png tags:[] uid:CVxPm9xkzN80U39i9 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:3 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:242 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978724.584019 id.resp_h:95.163.121.204 resp_fuids:[\"FM9IXt063nGYHO8F7\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978725888","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978725880","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725882","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FM9IXt063nGYHO8F7"],"timestamp":1530978724584,"method":"GET","request_body_len":0,"uri":"\/img\/rt.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978725882","adapter.threatinteladapter.begin.ts":"1530978725884","ip_src_port":49205,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"467db5b0-35d4-483b-ab1f-1640a8af971c","response_body_len":242} +{"adapter.threatinteladapter.end.ts":"1530978725884","qclass_name":"C_INTERNET","bro_timestamp":"1530978724.85904","qtype_name":"PTR","ip_dst_port":5353,"enrichmentsplitterbolt.splitter.end.ts":"1530978725867","qtype":12,"rejected":false,"enrichmentsplitterbolt.splitter.begin.ts":"1530978725867","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","trans_id":0,"adapter.geoadapter.begin.ts":"1530978725869","uid":"Cg7uac12cgFflf6Fp7","protocol":"dns","original_string":"DNS | AA:false qclass_name:C_INTERNET id.orig_p:5353 qtype_name:PTR qtype:12 rejected:false id.resp_p:5353 query:_googlecast._tcp.local trans_id:0 TC:false RA:false uid:Cg7uac12cgFflf6Fp7 RD:false proto:udp id.orig_h:192.168.66.1 Z:0 qclass:1 ts:1530978724.85904 id.resp_h:224.0.0.251","ip_dst_addr":"224.0.0.251","threatinteljoinbolt.joiner.ts":"1530978725888","enrichmentjoinbolt.joiner.ts":"1530978725880","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725882","Z":0,"ip_src_addr":"192.168.66.1","qclass":1,"timestamp":1530978724859,"AA":false,"query":"_googlecast._tcp.local","TC":false,"RA":false,"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","RD":false,"threatintelsplitterbolt.splitter.end.ts":"1530978725882","adapter.threatinteladapter.begin.ts":"1530978725884","ip_src_port":5353,"proto":"udp","guid":"01750143-0fe6-4a96-bdd7-55ece2d0fe44"} +{"adapter.threatinteladapter.end.ts":"1530978725884","bro_timestamp":"1530978724.932079","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978725867","enrichmentsplitterbolt.splitter.begin.ts":"1530978725867","adapter.hostfromjsonlistadapter.end.ts":"1530978725876","adapter.geoadapter.begin.ts":"1530978725869","uid":"CUrRne3iLIxXavQtci","trans_depth":225,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/persist\/wizard-data?_=1484169340974 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:225 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978724.932079 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978725888","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978725881","adapter.hostfromjsonlistadapter.begin.ts":"1530978725876","threatintelsplitterbolt.splitter.begin.ts":"1530978725882","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978724932,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/persist\/wizard-data?_=1484169340974","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978725869","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978725882","adapter.threatinteladapter.begin.ts":"1530978725884","ip_src_port":50451,"guid":"71af5025-ebad-4fce-9acb-c50fa3b94af8","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978733714","bro_timestamp":"1530978728.982768","ip_dst_port":8080,"enrichmentsplitterbolt.splitter.end.ts":"1530978733703","enrichmentsplitterbolt.splitter.begin.ts":"1530978733703","adapter.hostfromjsonlistadapter.end.ts":"1530978733706","adapter.geoadapter.begin.ts":"1530978733706","uid":"CUrRne3iLIxXavQtci","trans_depth":240,"protocol":"http","original_string":"HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:\/api\/v1\/clusters\/metron_cluster\/services\/KAFKA\/components\/KAFKA_BROKER?fields=metrics\/kafka\/server\/BrokerTopicMetrics\/AllTopicsBytesInPerSec\/1MinuteRate[1484165785,1484169385,15],metrics\/kafka\/server\/BrokerTopicMetrics\/AllTopicsBytesOutPerSec\/1MinuteRate[1484165785,1484169385,15],metrics\/kafka\/server\/BrokerTopicMetrics\/AllTopicsMessagesInPerSec\/1MinuteRate[1484165785,1484169385,15],metrics\/kafka\/controller\/KafkaController\/ActiveControllerCount[1484165785,1484169385,15],metrics\/kafka\/controller\/ControllerStats\/LeaderElectionRateAndTimeMs\/1MinuteRate[1484165785,1484169385,15],metrics\/kafka\/controller\/ControllerStats\/UncleanLeaderElectionsPerSec\/1MinuteRate[1484165785,1484169385,15],metrics\/kafka\/server\/ReplicaFetcherManager\/Replica-MaxLag[1484165785,1484169385,15],metrics\/kafka\/server\/ReplicaManager\/PartitionCount[1484165785,1484169385,15],metrics\/kafka\/server\/ReplicaManager\/UnderReplicatedPartitions[1484165785,1484169385,15],metrics\/kafka\/server\/ReplicaManager\/LeaderCount[1484165785,1484169385,15]&format=null_padding&_=1484169386025 tags:[] uid:CUrRne3iLIxXavQtci referrer:http:\/\/node1:8080\/ trans_depth:240 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36 ts:1530978728.982768 id.resp_h:192.168.66.121","ip_dst_addr":"192.168.66.121","threatinteljoinbolt.joiner.ts":"1530978733716","host":"node1","enrichmentjoinbolt.joiner.ts":"1530978733709","adapter.hostfromjsonlistadapter.begin.ts":"1530978733706","threatintelsplitterbolt.splitter.begin.ts":"1530978733711","ip_src_addr":"192.168.66.1","user_agent":"Mozilla\/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit\/537.36 (KHTML, like Gecko) Chrome\/55.0.2883.95 Safari\/537.36","timestamp":1530978728982,"method":"GET","request_body_len":0,"uri":"\/api\/v1\/clusters\/metron_cluster\/services\/KAFKA\/components\/KAFKA_BROKER?fields=metrics\/kafka\/server\/BrokerTopicMetrics\/AllTopicsBytesInPerSec\/1MinuteRate[1484165785,1484169385,15],metrics\/kafka\/server\/BrokerTopicMetrics\/AllTopicsBytesOutPerSec\/1MinuteRate[1484165785,1484169385,15],metrics\/kafka\/server\/BrokerTopicMetrics\/AllTopicsMessagesInPerSec\/1MinuteRate[1484165785,1484169385,15],metrics\/kafka\/controller\/KafkaController\/ActiveControllerCount[1484165785,1484169385,15],metrics\/kafka\/controller\/ControllerStats\/LeaderElectionRateAndTimeMs\/1MinuteRate[1484165785,1484169385,15],metrics\/kafka\/controller\/ControllerStats\/UncleanLeaderElectionsPerSec\/1MinuteRate[1484165785,1484169385,15],metrics\/kafka\/server\/ReplicaFetcherManager\/Replica-MaxLag[1484165785,1484169385,15],metrics\/kafka\/server\/ReplicaManager\/PartitionCount[1484165785,1484169385,15],metrics\/kafka\/server\/ReplicaManager\/UnderReplicatedPartitions[1484165785,1484169385,15],metrics\/kafka\/server\/ReplicaManager\/LeaderCount[1484165785,1484169385,15]&format=null_padding&_=1484169386025","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978733706","referrer":"http:\/\/node1:8080\/","threatintelsplitterbolt.splitter.end.ts":"1530978733711","adapter.threatinteladapter.begin.ts":"1530978733714","ip_src_port":50451,"guid":"d351c087-d7f7-44df-90e4-17c8d04eddca","response_body_len":0} +{"adapter.threatinteladapter.end.ts":"1530978733715","bro_timestamp":"1530978728.735143","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978733703","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978733703","adapter.hostfromjsonlistadapter.end.ts":"1530978733706","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978733706","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CO3j3c3y1c3rKDnyWg","resp_mime_types":["image\/x-icon"],"trans_depth":2,"protocol":"http","original_string":"HTTP | id.orig_p:49207 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/favicon.ico tags:[] uid:CO3j3c3y1c3rKDnyWg resp_mime_types:[\"image\\\/x-icon\"] trans_depth:2 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:318 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978728.735143 id.resp_h:95.163.121.204 resp_fuids:[\"FNP9PL3iYwRt9GHHC5\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978733718","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978733709","adapter.hostfromjsonlistadapter.begin.ts":"1530978733706","threatintelsplitterbolt.splitter.begin.ts":"1530978733712","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FNP9PL3iYwRt9GHHC5"],"timestamp":1530978728735,"method":"GET","request_body_len":0,"uri":"\/favicon.ico","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978733706","threatintelsplitterbolt.splitter.end.ts":"1530978733712","adapter.threatinteladapter.begin.ts":"1530978733715","ip_src_port":49207,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"28fa0321-6cf6-4756-83d0-1637c313a2ca","response_body_len":318} +{"adapter.threatinteladapter.end.ts":"1530978733715","bro_timestamp":"1530978728.008015","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978733704","enrichments.geo.ip_dst_addr.city":"Strasbourg","enrichments.geo.ip_dst_addr.latitude":"48.5839","enrichmentsplitterbolt.splitter.begin.ts":"1530978733704","adapter.hostfromjsonlistadapter.end.ts":"1530978733706","enrichments.geo.ip_dst_addr.country":"FR","enrichments.geo.ip_dst_addr.locID":"2973783","adapter.geoadapter.begin.ts":"1530978733706","enrichments.geo.ip_dst_addr.postalCode":"67100","uid":"CfoW9t3ShlDqC1h9Di","resp_mime_types":["text\/html"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49186 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/ tags:[] uid:CfoW9t3ShlDqC1h9Di referrer:http:\/\/va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in\/?285a4d4e4e5a4d4d4649584c5d43064b4745 resp_mime_types:[\"text\\\/html\"] trans_depth:1 host:r03afd2.c3008e.xc07r.b0f.a39.h7f0fa5eu.vb8fbl.e8mfzdgrf7g0.groupprograms.in status_msg:OK id.orig_h:192.168.138.158 response_body_len:121635 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978728.008015 id.resp_h:62.75.195.236 resp_fuids:[\"F88opA2s5OI0StZ2v8\"]","ip_dst_addr":"62.75.195.236","threatinteljoinbolt.joiner.ts":"1530978733718","host":"r03afd2.c3008e.xc07r.b0f.a39.h7f0fa5eu.vb8fbl.e8mfzdgrf7g0.groupprograms.in","enrichmentjoinbolt.joiner.ts":"1530978733709","adapter.hostfromjsonlistadapter.begin.ts":"1530978733706","threatintelsplitterbolt.splitter.begin.ts":"1530978733712","enrichments.geo.ip_dst_addr.longitude":"7.7455","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["F88opA2s5OI0StZ2v8"],"timestamp":1530978728008,"method":"GET","request_body_len":0,"uri":"\/","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978733706","referrer":"http:\/\/va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in\/?285a4d4e4e5a4d4d4649584c5d43064b4745","threatintelsplitterbolt.splitter.end.ts":"1530978733712","adapter.threatinteladapter.begin.ts":"1530978733715","ip_src_port":49186,"enrichments.geo.ip_dst_addr.location_point":"48.5839,7.7455","status_msg":"OK","guid":"47ae2bf0-d99c-44b8-a178-b0682cc13114","response_body_len":121635} +{"adapter.threatinteladapter.end.ts":"1530978733715","bro_timestamp":"1530978728.390198","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978733704","enrichments.geo.ip_dst_addr.city":"Los Angeles","enrichments.geo.ip_dst_addr.latitude":"34.0494","enrichmentsplitterbolt.splitter.begin.ts":"1530978733704","adapter.hostfromjsonlistadapter.end.ts":"1530978733706","enrichments.geo.ip_dst_addr.country":"US","enrichments.geo.ip_dst_addr.locID":"5368361","adapter.geoadapter.begin.ts":"1530978733706","enrichments.geo.ip_dst_addr.postalCode":"90014","uid":"CNR0er3VctfgZs8gPk","resp_mime_types":["text\/plain"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49198 status_code:200 method:POST request_body_len:134 id.resp_p:80 orig_mime_types:[\"text\\\/plain\"] uri:\/wp-content\/themes\/grizzly\/img5.php?c=cdcnw7cfz43rmtg tags:[] uid:CNR0er3VctfgZs8gPk resp_mime_types:[\"text\\\/plain\"] trans_depth:1 orig_fuids:[\"Fye3Fl1hLzBGT9AaSe\"] host:comarksecurity.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:14 user_agent:Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978728.390198 id.resp_h:72.34.49.86 resp_fuids:[\"FcOXvA3E4EdG9Vaaxd\"]","ip_dst_addr":"72.34.49.86","threatinteljoinbolt.joiner.ts":"1530978733718","enrichments.geo.ip_dst_addr.dmaCode":"803","host":"comarksecurity.com","enrichmentjoinbolt.joiner.ts":"1530978733709","adapter.hostfromjsonlistadapter.begin.ts":"1530978733706","threatintelsplitterbolt.splitter.begin.ts":"1530978733712","enrichments.geo.ip_dst_addr.longitude":"-118.2641","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 7.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FcOXvA3E4EdG9Vaaxd"],"timestamp":1530978728390,"method":"POST","request_body_len":134,"orig_mime_types":["text\/plain"],"uri":"\/wp-content\/themes\/grizzly\/img5.php?c=cdcnw7cfz43rmtg","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978733706","threatintelsplitterbolt.splitter.end.ts":"1530978733712","adapter.threatinteladapter.begin.ts":"1530978733715","orig_fuids":["Fye3Fl1hLzBGT9AaSe"],"ip_src_port":49198,"enrichments.geo.ip_dst_addr.location_point":"34.0494,-118.2641","status_msg":"OK","guid":"4f5ba7da-0820-4f3f-8e3f-e28949e025e7","response_body_len":14} +{"adapter.threatinteladapter.end.ts":"1530978733715","bro_timestamp":"1530978728.613828","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978733704","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978733704","adapter.hostfromjsonlistadapter.end.ts":"1530978733706","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978733706","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CcaM7Z1MyBBX9E8EC","resp_mime_types":["image\/png"],"trans_depth":3,"protocol":"http","original_string":"HTTP | id.orig_p:49206 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/flags\/fr.png tags:[] uid:CcaM7Z1MyBBX9E8EC referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:3 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:694 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978728.613828 id.resp_h:95.163.121.204 resp_fuids:[\"FpjJ2mpIuKnU39Gve\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978733718","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978733710","adapter.hostfromjsonlistadapter.begin.ts":"1530978733706","threatintelsplitterbolt.splitter.begin.ts":"1530978733712","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FpjJ2mpIuKnU39Gve"],"timestamp":1530978728613,"method":"GET","request_body_len":0,"uri":"\/img\/flags\/fr.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978733706","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978733712","adapter.threatinteladapter.begin.ts":"1530978733715","ip_src_port":49206,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"6a9e2340-8861-4f67-b81b-521462edacd1","response_body_len":694} +{"adapter.threatinteladapter.end.ts":"1530978733715","qclass_name":"C_INTERNET","bro_timestamp":"1530978728.872321","qtype_name":"PTR","ip_dst_port":5353,"enrichmentsplitterbolt.splitter.end.ts":"1530978733704","qtype":12,"rejected":false,"enrichmentsplitterbolt.splitter.begin.ts":"1530978733704","adapter.hostfromjsonlistadapter.end.ts":"1530978733706","trans_id":0,"adapter.geoadapter.begin.ts":"1530978733706","uid":"CGs8rS1rqhyXRRgA64","protocol":"dns","original_string":"DNS | AA:false qclass_name:C_INTERNET id.orig_p:5353 qtype_name:PTR qtype:12 rejected:false id.resp_p:5353 query:_googlecast._tcp.local trans_id:0 TC:false RA:false uid:CGs8rS1rqhyXRRgA64 RD:false proto:udp id.orig_h:192.168.66.1 Z:0 qclass:1 ts:1530978728.872321 id.resp_h:224.0.0.251","ip_dst_addr":"224.0.0.251","threatinteljoinbolt.joiner.ts":"1530978733718","enrichmentjoinbolt.joiner.ts":"1530978733710","adapter.hostfromjsonlistadapter.begin.ts":"1530978733706","threatintelsplitterbolt.splitter.begin.ts":"1530978733712","Z":0,"ip_src_addr":"192.168.66.1","qclass":1,"timestamp":1530978728872,"AA":false,"query":"_googlecast._tcp.local","TC":false,"RA":false,"source.type":"bro","adapter.geoadapter.end.ts":"1530978733706","RD":false,"threatintelsplitterbolt.splitter.end.ts":"1530978733712","adapter.threatinteladapter.begin.ts":"1530978733715","ip_src_port":5353,"proto":"udp","guid":"c9f200a6-aac8-4f4d-a463-0808d97bdc2a"} +{"TTLs":[29.0],"adapter.threatinteladapter.end.ts":"1530978733715","qclass_name":"C_INTERNET","bro_timestamp":"1530978728.926757","qtype_name":"A","ip_dst_port":53,"enrichmentsplitterbolt.splitter.end.ts":"1530978733704","qtype":1,"rejected":false,"answers":["62.75.195.236"],"enrichmentsplitterbolt.splitter.begin.ts":"1530978733704","adapter.hostfromjsonlistadapter.end.ts":"1530978733706","trans_id":18350,"adapter.geoadapter.begin.ts":"1530978733706","uid":"CVf8zv3sBOdNwWTrbl","protocol":"dns","original_string":"DNS | AA:false TTLs:[29.0] qclass_name:C_INTERNET id.orig_p:60078 qtype_name:A qtype:1 rejected:false id.resp_p:53 query:va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in answers:[\"62.75.195.236\"] trans_id:18350 rcode:0 rcode_name:NOERROR TC:false RA:true uid:CVf8zv3sBOdNwWTrbl RD:true proto:udp id.orig_h:192.168.138.158 Z:0 qclass:1 ts:1530978728.926757 id.resp_h:192.168.138.2","ip_dst_addr":"192.168.138.2","threatinteljoinbolt.joiner.ts":"1530978733718","enrichmentjoinbolt.joiner.ts":"1530978733710","adapter.hostfromjsonlistadapter.begin.ts":"1530978733706","threatintelsplitterbolt.splitter.begin.ts":"1530978733712","Z":0,"ip_src_addr":"192.168.138.158","qclass":1,"timestamp":1530978728926,"AA":false,"query":"va872g.g90e1h.b8.642b63u.j985a2.v33e.37.pa269cc.e8mfzdgrf7g0.groupprograms.in","rcode":0,"rcode_name":"NOERROR","TC":false,"RA":true,"source.type":"bro","adapter.geoadapter.end.ts":"1530978733706","RD":true,"threatintelsplitterbolt.splitter.end.ts":"1530978733712","adapter.threatinteladapter.begin.ts":"1530978733715","ip_src_port":60078,"proto":"udp","guid":"94f632a3-cc25-43d6-8962-a25cd2a051ce"} +{"adapter.threatinteladapter.end.ts":"1530978733715","bro_timestamp":"1530978728.787972","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978733704","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978733704","adapter.hostfromjsonlistadapter.end.ts":"1530978733706","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978733706","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"Cm8nbh1mEqDSWqLB61","resp_mime_types":["image\/png"],"trans_depth":1,"protocol":"http","original_string":"HTTP | id.orig_p:49210 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/lt.png tags:[] uid:Cm8nbh1mEqDSWqLB61 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:1 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:240 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978728.787972 id.resp_h:95.163.121.204 resp_fuids:[\"FUTQYW12ZHQdPoUnCk\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978733718","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978733710","adapter.hostfromjsonlistadapter.begin.ts":"1530978733706","threatintelsplitterbolt.splitter.begin.ts":"1530978733712","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["FUTQYW12ZHQdPoUnCk"],"timestamp":1530978728787,"method":"GET","request_body_len":0,"uri":"\/img\/lt.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978733706","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978733713","adapter.threatinteladapter.begin.ts":"1530978733715","ip_src_port":49210,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"4dfb979e-77b6-4d15-9bf9-79826cc25de5","response_body_len":240} +{"adapter.threatinteladapter.end.ts":"1530978733715","bro_timestamp":"1530978728.911648","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978733704","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978733704","adapter.hostfromjsonlistadapter.end.ts":"1530978733706","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978733706","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"C7jsKl2DTsgO8OIddk","resp_mime_types":["image\/png"],"trans_depth":3,"protocol":"http","original_string":"HTTP | id.orig_p:49210 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/button_pay.png tags:[] uid:C7jsKl2DTsgO8OIddk referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:3 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:727 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978728.911648 id.resp_h:95.163.121.204 resp_fuids:[\"Fc2oDG41kRtOgFcqc1\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978733718","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978733710","adapter.hostfromjsonlistadapter.begin.ts":"1530978733706","threatintelsplitterbolt.splitter.begin.ts":"1530978733713","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["Fc2oDG41kRtOgFcqc1"],"timestamp":1530978728911,"method":"GET","request_body_len":0,"uri":"\/img\/button_pay.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978733706","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978733713","adapter.threatinteladapter.begin.ts":"1530978733715","ip_src_port":49210,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"b21bd1cd-fa36-4d66-9d72-d998adc580d3","response_body_len":727} +{"adapter.threatinteladapter.end.ts":"1530978733715","bro_timestamp":"1530978728.955487","status_code":200,"ip_dst_port":80,"enrichmentsplitterbolt.splitter.end.ts":"1530978733704","enrichments.geo.ip_dst_addr.city":"Elektrostal","enrichments.geo.ip_dst_addr.latitude":"55.7896","enrichmentsplitterbolt.splitter.begin.ts":"1530978733704","adapter.hostfromjsonlistadapter.end.ts":"1530978733706","enrichments.geo.ip_dst_addr.country":"RU","enrichments.geo.ip_dst_addr.locID":"563523","adapter.geoadapter.begin.ts":"1530978733706","enrichments.geo.ip_dst_addr.postalCode":"144004","uid":"CA0G2ASkF1efFirs7","resp_mime_types":["image\/png"],"trans_depth":3,"protocol":"http","original_string":"HTTP | id.orig_p:49210 status_code:200 method:GET request_body_len:0 id.resp_p:80 uri:\/img\/button_pay.png tags:[] uid:CA0G2ASkF1efFirs7 referrer:http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg resp_mime_types:[\"image\\\/png\"] trans_depth:3 host:7oqnsnzwwnm6zb7y.gigapaysun.com status_msg:OK id.orig_h:192.168.138.158 response_body_len:727 user_agent:Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0) ts:1530978728.955487 id.resp_h:95.163.121.204 resp_fuids:[\"F7c5Lp3iMksOUQHIbl\"]","ip_dst_addr":"95.163.121.204","threatinteljoinbolt.joiner.ts":"1530978733718","host":"7oqnsnzwwnm6zb7y.gigapaysun.com","enrichmentjoinbolt.joiner.ts":"1530978733710","adapter.hostfromjsonlistadapter.begin.ts":"1530978733706","threatintelsplitterbolt.splitter.begin.ts":"1530978733713","enrichments.geo.ip_dst_addr.longitude":"38.4467","ip_src_addr":"192.168.138.158","user_agent":"Mozilla\/4.0 (compatible; MSIE 8.0; Windows NT 6.1; WOW64; Trident\/4.0; SLCC2; .NET CLR 2.0.50727; .NET CLR 3.5.30729; .NET CLR 3.0.30729; Media Center PC 6.0)","resp_fuids":["F7c5Lp3iMksOUQHIbl"],"timestamp":1530978728955,"method":"GET","request_body_len":0,"uri":"\/img\/button_pay.png","tags":[],"source.type":"bro","adapter.geoadapter.end.ts":"1530978733706","referrer":"http:\/\/7oqnsnzwwnm6zb7y.gigapaysun.com\/11iQmfg","threatintelsplitterbolt.splitter.end.ts":"1530978733713","adapter.threatinteladapter.begin.ts":"1530978733715","ip_src_port":49210,"enrichments.geo.ip_dst_addr.location_point":"55.7896,38.4467","status_msg":"OK","guid":"243fcd99-2905-403d-9e9e-b0c1f30ca53f","response_body_len":727} diff --git a/metron-analytics/pom.xml b/metron-analytics/pom.xml index b6460eaef1..dc72f3b0ea 100644 --- a/metron-analytics/pom.xml +++ b/metron-analytics/pom.xml @@ -46,6 +46,7 @@ metron-profiler metron-profiler-client metron-profiler-common + metron-profiler-spark diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileResult.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileResult.java index e2aa54d909..82fbe2e717 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileResult.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileResult.java @@ -65,6 +65,10 @@ public ProfileResult(String expression) { this.triageExpressions = new ProfileTriageExpressions(); } + public ProfileResult() { + // needed for serialization and Spark's code generator + } + public ProfileResultExpressions getProfileExpressions() { return profileExpressions; } diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileResultExpressions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileResultExpressions.java index 2cada019f2..464b2463d6 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileResultExpressions.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileResultExpressions.java @@ -35,6 +35,10 @@ public ProfileResultExpressions(String expression) { this.expression = expression; } + public ProfileResultExpressions() { + // needed for serialization and Spark's code generator + } + @JsonValue public String getExpression() { return expression; diff --git a/pom.xml b/pom.xml index 9a8c442bc0..6696ec78e7 100644 --- a/pom.xml +++ b/pom.xml @@ -137,6 +137,7 @@ 0.9.10 8.0 2.1 + 2.3.1 From c6d0721b8f02752cea9f91acb29042ca9ddf0bc9 Mon Sep 17 00:00:00 2001 From: nickwallen Date: Mon, 27 Aug 2018 13:47:28 -0400 Subject: [PATCH 06/25] METRON-1708 Run the Batch Profiler in Spark (nickwallen) closes apache/metron#1161 --- .../metron-profiler-spark/pom.xml | 15 ++ .../src/main/assembly/assembly.xml | 60 ++++++ .../src/main/config/batch-profiler.properties | 20 ++ .../profiler/spark/cli/BatchProfilerCLI.java | 174 ++++++++++++++++++ .../spark/cli/BatchProfilerCLIOptions.java | 144 +++++++++++++++ .../src/main/scripts/start_batch_profiler.sh | 32 ++++ .../spark/cli/BatchProfilerCLITest.java | 61 ++++++ .../src/test/resources/globals.json | 3 + .../src/test/resources/profiles-empty.json | 5 + .../profiles-no-timestamp-field.json | 12 ++ .../src/test/resources/profiles.json | 13 ++ 11 files changed, 539 insertions(+) create mode 100644 metron-analytics/metron-profiler-spark/src/main/assembly/assembly.xml create mode 100644 metron-analytics/metron-profiler-spark/src/main/config/batch-profiler.properties create mode 100644 metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/cli/BatchProfilerCLI.java create mode 100644 metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/cli/BatchProfilerCLIOptions.java create mode 100644 metron-analytics/metron-profiler-spark/src/main/scripts/start_batch_profiler.sh create mode 100644 metron-analytics/metron-profiler-spark/src/test/java/org/apache/metron/profiler/spark/cli/BatchProfilerCLITest.java create mode 100644 metron-analytics/metron-profiler-spark/src/test/resources/globals.json create mode 100644 metron-analytics/metron-profiler-spark/src/test/resources/profiles-empty.json create mode 100644 metron-analytics/metron-profiler-spark/src/test/resources/profiles-no-timestamp-field.json create mode 100644 metron-analytics/metron-profiler-spark/src/test/resources/profiles.json diff --git a/metron-analytics/metron-profiler-spark/pom.xml b/metron-analytics/metron-profiler-spark/pom.xml index 93ce08af30..2d5ec9849b 100644 --- a/metron-analytics/metron-profiler-spark/pom.xml +++ b/metron-analytics/metron-profiler-spark/pom.xml @@ -190,6 +190,21 @@ + + maven-assembly-plugin + + src/main/assembly/assembly.xml + + + + make-assembly + package + + single + + + + diff --git a/metron-analytics/metron-profiler-spark/src/main/assembly/assembly.xml b/metron-analytics/metron-profiler-spark/src/main/assembly/assembly.xml new file mode 100644 index 0000000000..02f97ebf44 --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/main/assembly/assembly.xml @@ -0,0 +1,60 @@ + + + + archive + + tar.gz + + false + + + ${project.basedir}/src/main/config + config + true + + **/*.formatted + **/*.filtered + + 0644 + unix + true + + + ${project.basedir}/src/main/scripts + bin + true + + **/*.formatted + **/*.filtered + + 0755 + unix + true + + + ${project.basedir}/target + + ${project.artifactId}-${project.version}.jar + + lib + true + + + diff --git a/metron-analytics/metron-profiler-spark/src/main/config/batch-profiler.properties b/metron-analytics/metron-profiler-spark/src/main/config/batch-profiler.properties new file mode 100644 index 0000000000..c651791dd6 --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/main/config/batch-profiler.properties @@ -0,0 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# +spark.master=local +spark.app.name=Batch Profiler diff --git a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/cli/BatchProfilerCLI.java b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/cli/BatchProfilerCLI.java new file mode 100644 index 0000000000..bdcf231094 --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/cli/BatchProfilerCLI.java @@ -0,0 +1,174 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.metron.profiler.spark.cli; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.ParseException; +import org.apache.commons.cli.PosixParser; +import org.apache.commons.io.IOUtils; +import org.apache.metron.common.configuration.profiler.ProfilerConfig; +import org.apache.metron.profiler.spark.BatchProfiler; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileInputStream; +import java.io.IOException; +import java.io.Serializable; +import java.lang.invoke.MethodHandles; +import java.util.Properties; + +import static org.apache.metron.profiler.spark.cli.BatchProfilerCLIOptions.CONFIGURATION_FILE; +import static org.apache.metron.profiler.spark.cli.BatchProfilerCLIOptions.GLOBALS_FILE; +import static org.apache.metron.profiler.spark.cli.BatchProfilerCLIOptions.PROFILE_DEFN_FILE; +import static org.apache.metron.profiler.spark.cli.BatchProfilerCLIOptions.parse; + +/** + * The main entry point which launches the Batch Profiler in Spark. + * + * With this class the Batch Profiler can be submitted using the following command. + * + *

{@code
+ *  $SPARK_HOME/bin/spark-submit \
+ *    --class org.apache.metron.profiler.spark.cli.BatchProfilerCLI \
+ *     --properties-file spark.properties \
+ *     metron-profiler-spark-.jar \
+ *     --config profiler.properties \
+ *     --globals global.properties \
+ *     --profiles profiles.json
+ * }
+ */ +public class BatchProfilerCLI implements Serializable { + + protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + public static Properties globals; + public static Properties profilerProps; + public static ProfilerConfig profiles; + + public static void main(String[] args) throws IOException, org.apache.commons.cli.ParseException { + // parse the command line + CommandLine commandLine = parseCommandLine(args); + profilerProps = handleProfilerProperties(commandLine); + globals = handleGlobals(commandLine); + profiles = handleProfileDefinitions(commandLine); + + // the batch profiler must use 'event time' + if(!profiles.getTimestampField().isPresent()) { + throw new IllegalArgumentException("The Batch Profiler must use event time. The 'timestampField' must be defined."); + } + + // one or more profiles must be defined + if(profiles.getProfiles().size() == 0) { + throw new IllegalArgumentException("No profile definitions found."); + } + + SparkSession spark = SparkSession + .builder() + .config(new SparkConf()) + .getOrCreate(); + + BatchProfiler profiler = new BatchProfiler(); + long count = profiler.run(spark, profilerProps, globals, profiles); + LOG.info("Profiler produced {} profile measurement(s)", count); + } + + /** + * Load the Stellar globals from a file. + * + * @param commandLine The command line. + */ + private static Properties handleGlobals(CommandLine commandLine) throws IOException { + Properties globals = new Properties(); + if(GLOBALS_FILE.has(commandLine)) { + String globalsPath = GLOBALS_FILE.get(commandLine); + + LOG.info("Loading global properties from '{}'", globalsPath); + globals.load(new FileInputStream(globalsPath)); + + LOG.info("Globals = {}", globals); + } + return globals; + } + + /** + * Load the Profiler configuration from a file. + * + * @param commandLine The command line. + */ + private static Properties handleProfilerProperties(CommandLine commandLine) throws IOException { + Properties config = new Properties(); + if(CONFIGURATION_FILE.has(commandLine)) { + String propertiesPath = CONFIGURATION_FILE.get(commandLine); + + LOG.info("Loading profiler properties from '{}'", propertiesPath); + config.load(new FileInputStream(propertiesPath)); + + LOG.info("Properties = {}", config.toString()); + } + return config; + } + + /** + * Load the profile definitions from a file. + * + * @param commandLine The command line. + */ + private static ProfilerConfig handleProfileDefinitions(CommandLine commandLine) throws IOException { + ProfilerConfig profiles; + if(PROFILE_DEFN_FILE.has(commandLine)) { + String profilePath = PROFILE_DEFN_FILE.get(commandLine); + + LOG.info("Loading profiles from '{}'", profilePath); + String contents = IOUtils.toString(new FileInputStream(profilePath)); + + profiles = ProfilerConfig.fromJSON(contents); + LOG.info("Loaded {} profile(s)", profiles.getProfiles().size()); + + } else { + throw new IllegalArgumentException("No profile(s) defined"); + } + return profiles; + } + + /** + * Parse the command line arguments submitted by the user. + * @param args The command line arguments to parse. + * @throws org.apache.commons.cli.ParseException + */ + private static CommandLine parseCommandLine(String[] args) throws ParseException { + CommandLineParser parser = new PosixParser(); + return parse(parser, args); + } + + public static Properties getGlobals() { + return globals; + } + + public static Properties getProfilerProps() { + return profilerProps; + } + + public static ProfilerConfig getProfiles() { + return profiles; + } +} \ No newline at end of file diff --git a/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/cli/BatchProfilerCLIOptions.java b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/cli/BatchProfilerCLIOptions.java new file mode 100644 index 0000000000..f5dfe12cbd --- /dev/null +++ b/metron-analytics/metron-profiler-spark/src/main/java/org/apache/metron/profiler/spark/cli/BatchProfilerCLIOptions.java @@ -0,0 +1,144 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.metron.profiler.spark.cli; + +import com.google.common.base.Joiner; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; + +import java.util.function.Supplier; + +/** + * Defines the command line interface (CLI) options accepted by the Batch + * Profiler. + */ +public enum BatchProfilerCLIOptions { + + PROFILE_DEFN_FILE(() -> { + Option o = new Option("p", "profiles", true, "Path to a file containing profile definitions."); + o.setRequired(true); + return o; + }), + + CONFIGURATION_FILE(() -> { + Option o = new Option("c", "config", true, "Path to the profiler properties file."); + o.setRequired(false); + return o; + }), + + GLOBALS_FILE(() -> { + Option o = new Option("g", "globals", true, "Path to the Stellar global config file."); + o.setRequired(false); + return o; + }), + + HELP(() -> { + Option o = new Option("h", "help", false, "Usage instructions."); + o.setRequired(false); + return o; + }); + + private Option option; + + BatchProfilerCLIOptions(Supplier