From 64343bc0d99880ac8bb17137a9226c3f44417da7 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Tue, 13 Feb 2018 09:52:54 -0500 Subject: [PATCH 01/20] METRON-590 Enable Use of Event Time in Profiler --- .../client/stellar/ProfilerFunctions.java | 14 +- .../profiler/DefaultMessageDistributor.java | 209 ++++++- .../profiler/DefaultProfileBuilder.java | 110 ++-- .../metron/profiler/MessageDistributor.java | 48 +- .../apache/metron/profiler/MessageRoute.java | 19 +- .../apache/metron/profiler/MessageRouter.java | 11 +- .../metron/profiler/ProfileBuilder.java | 34 +- .../metron/profiler/ProfileMeasurement.java | 6 +- .../metron/profiler/StandAloneProfiler.java | 100 +++- .../apache/metron/profiler/clock/Clock.java | 18 +- .../metron/profiler/clock/ClockFactory.java | 38 ++ .../profiler/clock/DefaultClockFactory.java | 57 ++ .../metron/profiler/clock/EventTimeClock.java | 72 +++ .../metron/profiler/clock/FixedClock.java | 39 +- .../profiler/clock/FixedClockFactory.java | 44 ++ .../metron/profiler/clock/WallClock.java | 17 +- .../DefaultMessageDistributorTest.java | 171 +++++- .../profiler/DefaultProfileBuilderTest.java | 119 ++-- .../metron/profiler/ProfilePeriodTest.java | 1 - .../profiler/StandAloneProfilerTest.java | 255 +++++++++ .../clock/DefaultClockFactoryTest.java | 75 +++ .../profiler/clock/EventTimeClockTest.java | 115 ++++ .../metron/profiler/clock/WallClockTest.java | 54 ++ metron-analytics/metron-profiler/README.md | 83 ++- .../src/main/config/profiler.properties | 14 +- .../src/main/flux/profiler/remote.yaml | 42 +- .../bolt/FixedFrequencyFlushSignal.java | 126 +++++ .../metron/profiler/bolt/FlushSignal.java | 51 ++ ...tinationHandler.java => HBaseEmitter.java} | 13 +- ...tinationHandler.java => KafkaEmitter.java} | 14 +- .../profiler/bolt/ManualFlushSignal.java | 54 ++ .../profiler/bolt/ProfileBuilderBolt.java | 374 ++++++++++--- ...er.java => ProfileMeasurementEmitter.java} | 15 +- .../profiler/bolt/ProfileSplitterBolt.java | 132 ++++- .../zookeeper/event-time-test/profiler.json | 12 + .../bolt/FixedFrequencyFlushSignalTest.java | 71 +++ ...HandlerTest.java => KafkaEmitterTest.java} | 15 +- .../profiler/bolt/ProfileBuilderBoltTest.java | 516 ++++++++++-------- .../profiler/bolt/ProfileHBaseMapperTest.java | 6 +- .../bolt/ProfileSplitterBoltTest.java | 288 ++++++++-- .../profiler/integration/MessageBuilder.java | 75 +++ .../integration/ProfilerIntegrationTest.java | 235 +++++--- .../configuration/metron-profiler-env.xml | 77 ++- .../package/scripts/params/params_linux.py | 7 + .../package/templates/profiler.properties.j2 | 15 +- .../METRON/CURRENT/themes/metron_theme.json | 118 +++- .../configuration/profiler/ProfileConfig.java | 53 ++ .../profiler/ProfilerConfig.java | 48 +- .../apache/metron/common/utils/JSONUtils.java | 11 +- .../configurations/ProfilerUpdater.java | 1 + .../profiler/ProfileConfigTest.java | 5 +- .../profiler/ProfilerConfigTest.java | 120 ++++ .../components/KafkaComponent.java | 39 +- 53 files changed, 3567 insertions(+), 689 deletions(-) create mode 100644 metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/ClockFactory.java create mode 100644 metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/DefaultClockFactory.java create mode 100644 metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/EventTimeClock.java create mode 100644 metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/FixedClockFactory.java create mode 100644 metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/StandAloneProfilerTest.java create mode 100644 metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/clock/DefaultClockFactoryTest.java create mode 100644 metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/clock/EventTimeClockTest.java create mode 100644 metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/clock/WallClockTest.java create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/FixedFrequencyFlushSignal.java create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/FlushSignal.java rename metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/{HBaseDestinationHandler.java => HBaseEmitter.java} (79%) rename metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/{KafkaDestinationHandler.java => KafkaEmitter.java} (86%) create mode 100644 metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ManualFlushSignal.java rename metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/{DestinationHandler.java => ProfileMeasurementEmitter.java} (76%) create mode 100644 metron-analytics/metron-profiler/src/test/config/zookeeper/event-time-test/profiler.json create mode 100644 metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/FixedFrequencyFlushSignalTest.java rename metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/{KafkaDestinationHandlerTest.java => KafkaEmitterTest.java} (95%) create mode 100644 metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/MessageBuilder.java create mode 100644 metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/profiler/ProfilerConfigTest.java diff --git a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/ProfilerFunctions.java b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/ProfilerFunctions.java index 64c1e2eb25..d6afe1dfd1 100644 --- a/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/ProfilerFunctions.java +++ b/metron-analytics/metron-profiler-client/src/main/java/org/apache/metron/profiler/client/stellar/ProfilerFunctions.java @@ -101,7 +101,10 @@ public Object apply(List args, Context context) { throw new IllegalArgumentException("Invalid profiler configuration", e); } - return new StandAloneProfiler(profilerConfig, periodDurationMillis, context); + // the TTL and max routes do not matter here + long profileTimeToLiveMillis = Long.MAX_VALUE; + long maxNumberOfRoutes = Long.MAX_VALUE; + return new StandAloneProfiler(profilerConfig, periodDurationMillis, profileTimeToLiveMillis, maxNumberOfRoutes, context); } } @@ -138,13 +141,8 @@ public Object apply(List args, Context context) throws ParseException { // user must provide the stand alone profiler StandAloneProfiler profiler = Util.getArg(1, StandAloneProfiler.class, args); - try { - for (JSONObject message : messages) { - profiler.apply(message); - } - - } catch (ExecutionException e) { - throw new IllegalArgumentException(format("Failed to apply message; error=%s", e.getMessage()), e); + for (JSONObject message : messages) { + profiler.apply(message); } return profiler; 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 53377a0f75..10fbbef1b8 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,14 +20,20 @@ package org.apache.metron.profiler; +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 org.apache.metron.common.configuration.profiler.ProfileConfig; -import org.apache.metron.profiler.clock.WallClock; import org.apache.metron.stellar.dsl.Context; import org.json.simple.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.lang.invoke.MethodHandles; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Optional; import java.util.concurrent.ExecutionException; @@ -36,32 +42,83 @@ import static java.lang.String.format; /** - * Distributes a message along a MessageRoute. A MessageRoute will lead to one or - * more ProfileBuilders. + * The default implementation of a {@link MessageDistributor}. + * + *

Two caches are maintained; one for active profiles and another for expired + * profiles. A profile will remain on the active cache as long as it continues + * to receive messages. + * + *

If a profile has not received messages for an extended period of time, it + * is expired and moved to the expired cache. A profile that is expired can no + * longer receive new messages. + * + *

A profile is stored in the expired cache for a fixed period of time so that + * a client can flush the state of expired profiles. If the client does not flush + * the expired profiles using `flushExpired`, the state of these profiles will be + * lost. * - * A ProfileBuilder is responsible for maintaining the state of a single profile, - * for a single entity. There will be one ProfileBuilder for each (profile, entity) pair. - * This class ensures that each ProfileBuilder receives the telemetry messages that - * it needs. */ public class DefaultMessageDistributor implements MessageDistributor { + protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + /** * The duration of each profile period in milliseconds. */ private long periodDurationMillis; + // TODO why are these marked transient? + + /** + * A cache of active profiles. + * + * A profile will remain on the active cache as long as it continues to receive + * messages. Once it has not received messages for a period of time, it is + * moved to the expired cache. + */ + private transient Cache activeCache; + /** - * Maintains the state of a profile which is unique to a profile/entity pair. + * A cache of expired profiles. + * + * When a profile expires from the active cache, it is moved here for a + * period of time. In the expired cache a profile can no longer receive + * new messages. A profile waits on the expired cache so that the client + * 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 profileCache; + private transient Cache expiredCache; /** * Create a new message distributor. + * + * @param periodDurationMillis The period duration in milliseconds. + * @param profileTimeToLiveMillis The time-to-live of a profile in milliseconds. + * @param maxNumberOfRoutes The max number of unique routes to maintain. After this is exceeded, lesser + * used routes will be evicted from the internal cache. + */ + public DefaultMessageDistributor( + long periodDurationMillis, + long profileTimeToLiveMillis, + long maxNumberOfRoutes) { + this(periodDurationMillis, profileTimeToLiveMillis, maxNumberOfRoutes, Ticker.systemTicker()); + } + + /** + * Create a new message distributor. + * * @param periodDurationMillis The period duration in milliseconds. - * @param profileTimeToLiveMillis The TTL of a profile in milliseconds. + * @param profileTimeToLiveMillis The time-to-live of a profile in milliseconds. + * @param maxNumberOfRoutes The max number of unique routes to maintain. After this is exceeded, lesser + * used routes will be evicted from the internal cache. + * @param ticker The ticker used to drive time for the caches. Only needs set for testing. */ - public DefaultMessageDistributor(long periodDurationMillis, long profileTimeToLiveMillis) { + public DefaultMessageDistributor( + long periodDurationMillis, + long profileTimeToLiveMillis, + long maxNumberOfRoutes, + Ticker ticker) { + if(profileTimeToLiveMillis < periodDurationMillis) { throw new IllegalStateException(format( "invalid configuration: expect profile TTL (%d) to be greater than period duration (%d)", @@ -69,9 +126,23 @@ public DefaultMessageDistributor(long periodDurationMillis, long profileTimeToLi periodDurationMillis)); } this.periodDurationMillis = periodDurationMillis; - this.profileCache = CacheBuilder + + // build the cache of active profiles + this.activeCache = CacheBuilder .newBuilder() + .maximumSize(maxNumberOfRoutes) .expireAfterAccess(profileTimeToLiveMillis, TimeUnit.MILLISECONDS) + .removalListener(new ActiveCacheRemovalListener()) + .ticker(ticker) + .build(); + + // build the cache of expired profiles + this.expiredCache = CacheBuilder + .newBuilder() + .maximumSize(maxNumberOfRoutes) + .expireAfterWrite(profileTimeToLiveMillis, TimeUnit.MILLISECONDS) + .removalListener(new ExpiredCacheRemovalListener()) + .ticker(ticker) .build(); } @@ -79,57 +150,120 @@ public DefaultMessageDistributor(long periodDurationMillis, long profileTimeToLi * 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. * @throws ExecutionException */ @Override - public void distribute(JSONObject message, MessageRoute route, Context context) throws ExecutionException { - getBuilder(route, context).apply(message); + public void distribute(JSONObject message, long timestamp, MessageRoute route, Context context) { + try { + ProfileBuilder builder = getBuilder(route, context); + builder.apply(message, timestamp); + + } catch(ExecutionException e) { + LOG.error("Unexpected error", e); + throw new RuntimeException(e); + } } /** - * Flushes all profiles. Flushes all ProfileBuilders that this distributor is responsible for. + * Flush all active profiles. + * + *

A profile will remain active as long as it continues to receive messages. If a profile + * does not receive a message for an extended duration, it may be marked as expired. + * + *

Flushes all active {@link ProfileBuilder} objects that this distributor is responsible for. * - * @return The profile measurements; one for each (profile, entity) pair. + * @return The {@link ProfileMeasurement} values; one for each (profile, entity) pair. */ @Override public List flush() { + + // cache maintenance needed here to ensure active profiles will expire + activeCache.cleanUp(); + expiredCache.cleanUp(); + + List measurements = flushCache(activeCache); + return measurements; + } + + /** + * Flush all expired profiles. + * + *

Flushes all expired {@link ProfileBuilder}s that this distributor is responsible for. + * + *

If a profile has not received messages for an extended period of time, it will be marked as + * expired. When a profile is expired, it can no longer receive new messages. Expired profiles + * remain only to give the client a chance to flush them. + * + *

If the client does not flush the expired profiles periodically, any state maintained in the + * profile since the last flush may be lost. + * + * @return The {@link ProfileMeasurement} values; one for each (profile, entity) pair. + */ + @Override + public List flushExpired() { + + // cache maintenance needed here to ensure active profiles will expire + activeCache.cleanUp(); + expiredCache.cleanUp(); + + // flush all expired profiles + List measurements = flushCache(expiredCache); + + // once the expired profiles have been flushed, they are no longer needed + expiredCache.invalidateAll(); + + return measurements; + } + + /** + * Flush all of the profiles maintained in a cache. + * + * @param cache The cache to flush. + * @return The measurements captured when flushing the profiles. + */ + private List flushCache(Cache cache) { + List measurements = new ArrayList<>(); + for(ProfileBuilder profileBuilder: cache.asMap().values()) { - profileCache.asMap().forEach((key, profileBuilder) -> { + // only need to flush, if the profile has been initialized if(profileBuilder.isInitialized()) { + + // flush the profiler and save the measurement, if one exists Optional measurement = profileBuilder.flush(); - measurement.ifPresent(measurements::add); + measurement.ifPresent(m -> measurements.add(m)); } - }); + } - profileCache.cleanUp(); return measurements; } /** * Retrieves the cached ProfileBuilder that is used to build and maintain the Profile. If none exists, * one will be created and returned. + * * @param route The message route. * @param context The Stellar execution context. */ public ProfileBuilder getBuilder(MessageRoute route, Context context) throws ExecutionException { ProfileConfig profile = route.getProfileDefinition(); String entity = route.getEntity(); - return profileCache.get( + return activeCache.get( cacheKey(profile, entity), () -> new DefaultProfileBuilder.Builder() .withDefinition(profile) .withEntity(entity) .withPeriodDurationMillis(periodDurationMillis) .withContext(context) - .withClock(new WallClock()) .build()); } /** - * Builds the key that is used to lookup the ProfileState within the cache. + * Builds the key that is used to lookup the {@link ProfileBuilder} within the cache. + * * @param profile The profile definition. * @param entity The entity. */ @@ -145,4 +279,33 @@ public DefaultMessageDistributor withPeriodDurationMillis(long periodDurationMil public DefaultMessageDistributor withPeriodDuration(int duration, TimeUnit units) { return withPeriodDurationMillis(units.toMillis(duration)); } + + /** + * A listener that is notified when profiles expire from the active cache. + */ + private class ActiveCacheRemovalListener implements RemovalListener { + + @Override + public void onRemoval(RemovalNotification notification) { + + String key = notification.getKey(); + ProfileBuilder expired = notification.getValue(); + + LOG.warn("Profile expired from active cache; key={}", key); + expiredCache.put(key, expired); + } + } + + /** + * A listener that is notified when profiles expire from the active cache. + */ + private class ExpiredCacheRemovalListener implements RemovalListener { + + @Override + public void onRemoval(RemovalNotification notification) { + + String key = notification.getKey(); + LOG.debug("Profile removed from expired cache; key={}", key); + } + } } 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 2e3416007b..4b564c9191 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 @@ -20,7 +20,18 @@ package org.apache.metron.profiler; -import static java.lang.String.format; +import org.apache.commons.collections4.ListUtils; +import org.apache.commons.collections4.MapUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.metron.common.configuration.profiler.ProfileConfig; +import org.apache.metron.stellar.common.DefaultStellarStatefulExecutor; +import org.apache.metron.stellar.common.StellarStatefulExecutor; +import org.apache.metron.stellar.dsl.Context; +import org.apache.metron.stellar.dsl.ParseException; +import org.apache.metron.stellar.dsl.StellarFunctions; +import org.json.simple.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.Serializable; import java.lang.invoke.MethodHandles; @@ -34,20 +45,8 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import org.apache.commons.collections4.ListUtils; -import org.apache.commons.collections4.MapUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.metron.common.configuration.profiler.ProfileConfig; -import org.apache.metron.profiler.clock.Clock; -import org.apache.metron.profiler.clock.WallClock; -import org.apache.metron.stellar.common.DefaultStellarStatefulExecutor; -import org.apache.metron.stellar.common.StellarStatefulExecutor; -import org.apache.metron.stellar.dsl.Context; -import org.apache.metron.stellar.dsl.ParseException; -import org.apache.metron.stellar.dsl.StellarFunctions; -import org.json.simple.JSONObject; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + +import static java.lang.String.format; /** * Responsible for building and maintaining a Profile. @@ -94,16 +93,15 @@ public class DefaultProfileBuilder implements ProfileBuilder, Serializable { private long periodDurationMillis; /** - * A clock is used to tell time; imagine that. + * Tracks the latest timestamp for use when flushing the profile. */ - private Clock clock; + private long maxTimestamp; /** - * Use the ProfileBuilder.Builder to create a new ProfileBuilder. + * Private constructor. Use the {@link Builder} to create a new {@link ProfileBuilder). */ private DefaultProfileBuilder(ProfileConfig definition, String entity, - Clock clock, long periodDurationMillis, Context stellarContext) { @@ -111,27 +109,37 @@ private DefaultProfileBuilder(ProfileConfig definition, this.definition = definition; this.profileName = definition.getProfile(); this.entity = entity; - this.clock = clock; this.periodDurationMillis = periodDurationMillis; this.executor = new DefaultStellarStatefulExecutor(); StellarFunctions.initialize(stellarContext); this.executor.setContext(stellarContext); + this.maxTimestamp = 0; } /** * Apply a message to the profile. + * * @param message The message to apply. + * @param timestamp The timestamp of the message. */ @Override - public void apply(JSONObject message) { + public void apply(JSONObject message, long timestamp) { try { if (!isInitialized()) { + + // execute each 'init' expression assign(definition.getInit(), message, "init"); isInitialized = true; } + // execute each 'update' expression assign(definition.getUpdate(), message, "update"); + // keep track of the 'latest' timestamp seen for use when flushing the profile + if(timestamp > maxTimestamp) { + maxTimestamp = timestamp; + } + } catch(Throwable e) { LOG.error(format("Unable to apply message to profile: %s", e.getMessage()), e); } @@ -140,23 +148,30 @@ public void apply(JSONObject message) { /** * Flush the Profile. * - * Completes and emits the ProfileMeasurement. Clears all state in preparation for + *

Completes and emits the {@link ProfileMeasurement}. Clears all state in preparation for * the next window period. * - * @return Returns the completed profile measurement. + * @return Returns the completed {@link ProfileMeasurement}. */ @Override public Optional flush() { - LOG.debug("Flushing profile: profile={}, entity={}", profileName, entity); - Optional result = Optional.empty(); - ProfilePeriod period = new ProfilePeriod(clock.currentTimeMillis(), periodDurationMillis, TimeUnit.MILLISECONDS); + + Optional result; + ProfilePeriod period = new ProfilePeriod(maxTimestamp, periodDurationMillis, TimeUnit.MILLISECONDS); try { - // execute the 'profile' expression(s) - Object profileValue = execute(definition.getResult().getProfileExpressions().getExpression(), "result/profile"); + // execute the 'profile' expression + String profileExpression = definition + .getResult() + .getProfileExpressions() + .getExpression(); + Object profileValue = execute(profileExpression, "result/profile"); // execute the 'triage' expression(s) - Map triageValues = definition.getResult().getTriageExpressions().getExpressions() + Map triageValues = definition + .getResult() + .getTriageExpressions() + .getExpressions() .entrySet() .stream() .collect(Collectors.toMap( @@ -185,10 +200,21 @@ public Optional flush() { .withDefinition(definition)); } catch(Throwable e) { + // if any of the Stellar expressions fail, a measurement should NOT be returned LOG.error(format("Unable to flush profile: error=%s", e.getMessage()), e); + result = Optional.empty(); } + LOG.debug("Flushed profile: profile={}, entity={}, maxTime={}, period={}, start={}, end={}, duration={}", + profileName, + entity, + maxTimestamp, + period.getPeriod(), + period.getStartTimeMillis(), + period.getEndTimeMillis(), + period.getDurationMillis()); + isInitialized = false; return result; } @@ -214,6 +240,7 @@ public ProfileConfig getDefinition() { /** * Executes an expression contained within the profile definition. + * * @param expression The expression to execute. * @param transientState Additional transient state provided to the expression. * @param expressionType The type of expression; init, update, result. Provides additional context if expression execution fails. @@ -232,6 +259,7 @@ private Object execute(String expression, Map transientState, St /** * Executes an expression contained within the profile definition. + * * @param expression The expression to execute. * @param expressionType The type of expression; init, update, result. Provides additional context if expression execution fails. * @return The result of executing the expression. @@ -242,6 +270,7 @@ private Object execute(String expression, String expressionType) { /** * Executes a set of expressions whose results need to be assigned to a variable. + * * @param expressions Maps the name of a variable to the expression whose result should be assigned to it. * @param transientState Additional transient state provided to the expression. * @param expressionType The type of expression; init, update, result. Provides additional context if expression execution fails. @@ -254,6 +283,7 @@ private void assign(Map expressions, Map transie String expr = entry.getValue(); try { + // assign the result of the expression to the variable executor.assign(var, expr, transientState); @@ -274,6 +304,7 @@ private void assign(Map expressions, Map transie /** * Executes the expressions contained within the profile definition. + * * @param expressions A list of expressions to execute. * @param transientState Additional transient state provided to the expressions. * @param expressionType The type of expression; init, update, result. Provides additional context if expression execution fails. @@ -284,6 +315,7 @@ private List execute(List expressions, Map trans for(String expr: ListUtils.emptyIfNull(expressions)) { try { + // execute an expression Object result = executor.execute(expr, transientState, Object.class); results.add(result); @@ -305,15 +337,19 @@ private List execute(List expressions, Map trans return results; } + @Override + public String getEntity() { + return entity; + } + /** - * A builder used to construct a new ProfileBuilder. + * A builder should be used to construct a new {@link ProfileBuilder} object. */ public static class Builder { private ProfileConfig definition; private String entity; - private long periodDurationMillis; - private Clock clock = new WallClock(); + private Long periodDurationMillis; private Context context; public Builder withContext(Context context) { @@ -321,11 +357,6 @@ public Builder withContext(Context context) { return this; } - public Builder withClock(Clock clock) { - this.clock = clock; - return this; - } - /** * @param definition The profiler definition. */ @@ -370,8 +401,11 @@ public ProfileBuilder build() { if(StringUtils.isEmpty(entity)) { throw new IllegalArgumentException(format("missing entity name; got '%s'", entity)); } + if(periodDurationMillis == null) { + throw new IllegalArgumentException("missing period duration"); + } - return new DefaultProfileBuilder(definition, entity, clock, periodDurationMillis, context); + return new DefaultProfileBuilder(definition, entity, periodDurationMillis, context); } } } 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 a60446f4a6..ea5be0f9c6 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 @@ -24,33 +24,57 @@ import org.json.simple.JSONObject; import java.util.List; -import java.util.concurrent.ExecutionException; /** - * Distributes a message along a MessageRoute. A MessageRoute will lead to one or - * more ProfileBuilders. + * Distributes a telemetry message along a {@link MessageRoute}. A {@link MessageRoute} will lead to a + * {@link ProfileBuilder} that is responsible for building and maintaining a profile. * - * A ProfileBuilder is responsible for maintaining the state of a single profile, - * for a single entity. There will be one ProfileBuilder for each (profile, entity) pair. - * This class ensures that each ProfileBuilder receives the telemetry messages that - * it needs. + *

A {@link ProfileBuilder} is responsible for maintaining the state of a single (profile, entity) + * pairing. There will be one {@link ProfileBuilder} for each (profile, entity) pair. + * + *

A {@link MessageDistributor} ensures that each {@link ProfileBuilder} receives the telemetry + * messages that it needs. + * + * @see MessageRoute + * @see ProfileMeasurement */ public interface MessageDistributor { /** - * Distribute a message along a MessageRoute. + * 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. - * @throws ExecutionException */ - void distribute(JSONObject message, MessageRoute route, Context context) throws ExecutionException; + void distribute(JSONObject message, long timestamp, MessageRoute route, Context context); /** - * Flushes all profiles. Flushes all ProfileBuilders that this distributor is responsible for. + * Flush all active profiles. + * + *

A profile will remain active as long as it continues to receive messages. If a profile + * does not receive a message for an extended duration, it may be marked as expired. * - * @return The profile measurements; one for each (profile, entity) pair. + *

Flushes all active {@link ProfileBuilder} objects that this distributor is responsible for. + * + * @return The {@link ProfileMeasurement} values; one for each (profile, entity) pair. */ List flush(); + + /** + * Flush all expired profiles. + * + *

If a profile has not received messages for an extended period of time, it will be marked as + * expired. When a profile is expired, it can no longer receive new messages. Expired profiles + * remain only to give the client a chance to flush them. + * + *

If the client does not flush the expired profiles periodically, any state maintained in the + * profile since the last flush may be lost. + * + *

Flushes all expired {@link ProfileBuilder} objects that this distributor is responsible for. + * + * @return The {@link ProfileMeasurement} values; one for each (profile, entity) pair. + */ + List flushExpired(); } 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 19456710a8..7288f03468 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 @@ -23,12 +23,15 @@ import org.apache.metron.common.configuration.profiler.ProfileConfig; /** - * A MessageRoute defines the profile and entity that a telemetry message needs applied to. This - * allows a message to be routed to the profile and entity that needs it. + * Defines the 'route' a message must take through the Profiler. * - * One telemetry message may need multiple routes. This is the case when a message is needed by - * more than one profile. In this case, there will be multiple MessageRoute objects for a single - * message. + *

A {@link MessageRoute} defines the profile and entity that a telemetry message needs applied to. + * + *

If a message is needed by multiple profiles, then multiple {@link MessageRoute} values + * will exist. If a message is not needed by any profiles, then no {@link MessageRoute} values + * will exist. + * + * @see MessageRouter */ public class MessageRoute { @@ -42,6 +45,12 @@ public class MessageRoute { */ private String entity; + /** + * Create a {@link MessageRoute}. + * + * @param profileDefinition The profile definition. + * @param entity The entity. + */ public MessageRoute(ProfileConfig profileDefinition, String entity) { this.entity = entity; this.profileDefinition = profileDefinition; diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRouter.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRouter.java index 99c98a3388..4c18062896 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRouter.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/MessageRouter.java @@ -27,15 +27,18 @@ import java.util.List; /** - * Routes incoming telemetry messages. + * Routes incoming telemetry messages through the Profiler. * - * A single telemetry message may need to take multiple routes. This is the case - * when a message is needed by more than one profile. + *

If a message is needed by multiple profiles, then multiple {@link MessageRoute} values + * will be returned. If a message is not needed by any profiles, then no {@link MessageRoute} values + * will be returned. + * + * @see MessageRoute */ public interface MessageRouter { /** - * Route a telemetry message. Finds all routes for a given telemetry message. + * Finds all routes for a telemetry message. * * @param message The telemetry message that needs routed. * @param config The configuration for the Profiler. diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileBuilder.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileBuilder.java index c09b0b6ba9..07372d790e 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileBuilder.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/ProfileBuilder.java @@ -28,47 +28,61 @@ /** * Responsible for building and maintaining a Profile. * - * One or more messages are applied to the Profile with `apply` and a profile measurement is - * produced by calling `flush`. + *

Telemetry messages are applied to a profile using {@link ProfileBuilder#apply(JSONObject, long)}. A + * {@link ProfileMeasurement} is generated by calling {@link ProfileBuilder#flush()}. * - * Any one instance is responsible only for building the profile for a specific [profile, entity] - * pairing. There will exist many instances, one for each [profile, entity] pair that exists + * A {@link ProfileBuilder} is responsible only for building the profile for a specific [profile, entity] + * pair. There will exist many instances, one for each [profile, entity] pair that exists * within the incoming telemetry data applied to the profile. */ public interface ProfileBuilder { /** * Apply a message to the profile. + * * @param message The message to apply. + * @param timestamp The timestamp of the message. */ - void apply(JSONObject message); + void apply(JSONObject message, long timestamp); /** * Flush the Profile. * - * Completes and emits the ProfileMeasurement. Clears all state in preparation for + *

Completes the period and returns the {@link ProfileMeasurement}. Clears all state in preparation for * the next window period. * - * @return Returns the completed profile measurement. + * @return Returns the {@link ProfileMeasurement}. */ Optional flush(); /** - * Has the ProfileBuilder been initialized? + * Has the {@link ProfileBuilder} been initialized? + * * @return True, if initialization has occurred. False, otherwise. */ boolean isInitialized(); /** * Returns the definition of the profile being built. - * @return ProfileConfig definition of the profile + * + * @return The profile definition. */ ProfileConfig getDefinition(); /** - * Returns the value of a variable being maintained by the builder. + * Returns the value of a variable within the current profile state. + * * @param variable The variable name. * @return The value of the variable. */ Object valueOf(String variable); + + /** + * Returns the name of the entity. + * + *

Each {@code ProfileBuilder} instance is responsible for one (profile, entity) pair. + * + * @return The entity. + */ + String getEntity(); } 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 0e773e9168..f6cc286e13 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 @@ -28,10 +28,10 @@ import java.util.concurrent.TimeUnit; /** - * Represents a single data point within a Profile. + * Represents a single data point within a profile. * - * A Profile is effectively a time series. To this end a Profile is composed - * of many ProfileMeasurement values which in aggregate form a time series. + *

A profile contains many individual {@link ProfileMeasurement} values captured over a + * period of time. These values in aggregate form a time series. */ public class ProfileMeasurement { 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 6db70790fb..f79efe6568 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 @@ -21,18 +21,29 @@ package org.apache.metron.profiler; 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.dsl.Context; import org.json.simple.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.lang.invoke.MethodHandles; import java.util.List; +import java.util.Optional; import java.util.concurrent.ExecutionException; /** - * A stand alone version of the Profiler that does not require a - * distributed execution environment like Apache Storm. + * A stand alone version of the Profiler that does not require a distributed + * execution environment like Apache Storm. + * + *

This class is used to create and manage profiles within the REPL environment. */ public class StandAloneProfiler { + protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + /** * The Stellar execution context. */ @@ -53,6 +64,11 @@ public class StandAloneProfiler { */ private MessageDistributor distributor; + /** + * The factory that creates Clock objects. + */ + private ClockFactory clockFactory; + /** * Counts the number of messages that have been applied. */ @@ -67,12 +83,26 @@ public class StandAloneProfiler { */ private int routeCount; - public StandAloneProfiler(ProfilerConfig config, long periodDurationMillis, Context context) { + /** + * Create a new Profiler. + * + * @param config The Profiler configuration. + * @param periodDurationMillis The period duration in milliseconds. + * @param profileTimeToLiveMillis The time-to-live of a profile in milliseconds. + * @param maxNumberOfRoutes The max number of unique routes to maintain. After this is exceeded, lesser + * used routes will be evicted from the internal cache. + * @param context The Stellar execution context. + */ + public StandAloneProfiler(ProfilerConfig config, + long periodDurationMillis, + long profileTimeToLiveMillis, + long maxNumberOfRoutes, + Context context) { this.context = context; this.config = config; this.router = new DefaultMessageRouter(context); - // the period TTL does not matter in this context - this.distributor = new DefaultMessageDistributor(periodDurationMillis, Long.MAX_VALUE); + this.distributor = new DefaultMessageDistributor(periodDurationMillis, profileTimeToLiveMillis, maxNumberOfRoutes); + this.clockFactory = new DefaultClockFactory(); this.messageCount = 0; this.routeCount = 0; } @@ -80,26 +110,28 @@ public StandAloneProfiler(ProfilerConfig config, long periodDurationMillis, Cont /** * Apply a message to a set of profiles. * @param message The message to apply. - * @throws ExecutionException */ - public void apply(JSONObject message) throws ExecutionException { + public void apply(JSONObject message) { - List routes = router.route(message, config, context); - for(MessageRoute route : routes) { - distributor.distribute(message, route, context); - } + // what time is it? + Clock clock = clockFactory.createClock(config); + Optional timestamp = clock.currentTimeMillis(message); - routeCount += routes.size(); - messageCount += 1; - } + // can only route the message, if we have a timestamp + if(timestamp.isPresent()) { - @Override - public String toString() { - return "Profiler{" + - getProfileCount() + " profile(s), " + - getMessageCount() + " messages(s), " + - getRouteCount() + " route(s)" + - '}'; + // 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."); + } } /** @@ -110,19 +142,45 @@ public List flush() { return distributor.flush(); } + /** + * Returns the Profiler configuration. + * @return The Profiler configuration. + */ public ProfilerConfig getConfig() { return config; } + /** + * Returns the number of defined profiles. + * @return The number of defined profiles. + */ public int getProfileCount() { return (config == null) ? 0: config.getProfiles().size(); } + /** + * Returns the number of messages that have been applied. + * @return The number of messages that have been applied. + */ public int getMessageCount() { return messageCount; } + /** + * Returns the number of routes. + * @return The number of routes. + * @see MessageRoute + */ public int getRouteCount() { return routeCount; } + + @Override + public String toString() { + return "Profiler{" + + getProfileCount() + " profile(s), " + + getMessageCount() + " messages(s), " + + getRouteCount() + " route(s)" + + '}'; + } } diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/Clock.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/Clock.java index 6730e49079..b07c0ed759 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/Clock.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/Clock.java @@ -20,16 +20,24 @@ package org.apache.metron.profiler.clock; +import org.json.simple.JSONObject; + +import java.util.Optional; + /** - * A clock can tell time; imagine that. + * A {@link Clock} manages the progression of time in the Profiler. * - * This allows the Profiler to support different treatments of time like wall clock versus event time. + *

The Profiler can operate on either processing time or event time. This + * abstraction deals with the differences between the two. */ public interface Clock { /** - * The current time in epoch milliseconds. + * Returns the current time in epoch milliseconds. + * + * @param message The telemetry message. + * @return An optional value containing the current time in epoch milliseconds, if + * the current time is known. Otherwise, empty. */ - long currentTimeMillis(); - + Optional currentTimeMillis(JSONObject message); } diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/ClockFactory.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/ClockFactory.java new file mode 100644 index 0000000000..5435c48452 --- /dev/null +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/ClockFactory.java @@ -0,0 +1,38 @@ +/* + * + * 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.clock; + +import org.apache.metron.common.configuration.profiler.ProfilerConfig; + +/** + * A factory for creating {@link Clock} objects. + * + * The type of {@link Clock} needed will depend on the Profiler configuration. + */ +public interface ClockFactory { + + /** + * Creates and returns a {@link Clock}. + * + * @param config The profiler configuration. + * @return A {@link Clock}. + */ + Clock createClock(ProfilerConfig config); +} 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 new file mode 100644 index 0000000000..d62e62bd49 --- /dev/null +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/DefaultClockFactory.java @@ -0,0 +1,57 @@ +/* + * + * 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.clock; + +import org.apache.metron.common.configuration.profiler.ProfilerConfig; + +/** + * Creates a {@link Clock} based on the profiler configuration. + * + *

If the Profiler is configured to use event time, a {@link EventTimeClock} will + * be created. Otherwise, a {@link WallClock} will be created. + * + *

The default implementation of a {@link ClockFactory}. + */ +public class DefaultClockFactory implements ClockFactory { + + /** + * @param config The profiler configuration. + * @return The appropriate Clock based on the profiler configuration. + */ + @Override + public Clock createClock(ProfilerConfig config) { + Clock clock; + + boolean isEventTime = config.getTimestampField().isPresent(); + if(isEventTime) { + + // using event time + String timestampField = config.getTimestampField().get(); + clock = new EventTimeClock(timestampField); + + } else { + + // using processing time + clock = new WallClock(); + } + + return clock; + } +} 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 new file mode 100644 index 0000000000..5cd574eff4 --- /dev/null +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/EventTimeClock.java @@ -0,0 +1,72 @@ +/* + * + * 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.clock; + +import org.apache.metron.stellar.common.utils.ConversionUtils; +import org.json.simple.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.invoke.MethodHandles; +import java.util.Optional; + +/** + * A {@link Clock} that advances based on event time. + * + * Event time is advanced by the timestamps contained within telemetry messages, rather + * than the system clock. + */ +public class EventTimeClock implements Clock { + + protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + /** + * The name of the field from which the timestamp will + */ + private String timestampField; + + /** + * @param timestampField The name of the field containing a timestamp. + */ + public EventTimeClock(String timestampField) { + this.timestampField = timestampField; + } + + @Override + public Optional currentTimeMillis(JSONObject message) { + + Long result; + if(message != null && message.containsKey(timestampField)) { + + // extract the timestamp and convert to a long + Object timestamp = message.get(timestampField); + result = ConversionUtils.convert(timestamp, Long.class); + + } else { + + // the message does not contain the specified timestamp field + LOG.debug("message does not contain timestamp field '{}': message will be ignored: message='{}'", + timestampField, JSONObject.toJSONString(message)); + result = null; + } + + return Optional.ofNullable(result); + } +} diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/FixedClock.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/FixedClock.java index c6e93cde16..8259ed0336 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/FixedClock.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/FixedClock.java @@ -20,21 +20,50 @@ package org.apache.metron.profiler.clock; -import java.io.Serializable; +import org.json.simple.JSONObject; + +import java.util.Optional; /** - * A clock that reports whatever time you tell it to. Most useful for testing. + * A {@link Clock} that always reports the same time. + * + *

This is only useful for testing. */ -public class FixedClock implements Clock, Serializable { +public class FixedClock implements Clock { + /** + * The time in milliseconds since the epoch. + */ private long epochMillis; + /** + * Create a {@link Clock}. The time defaults to the epoch. + */ + public FixedClock() { + this(0); + } + + /** + * Create a {@link Clock}. + * @param epochMillis The time in milliseconds since the epoch. + */ + public FixedClock(long epochMillis) { + this.setTime(epochMillis); + } + + /** + * Set the current time. + * @param epochMillis The time in milliseconds since the epoch. + */ public void setTime(long epochMillis) { this.epochMillis = epochMillis; } + /** + * @return The time in milliseconds since the epoch. + */ @Override - public long currentTimeMillis() { - return this.epochMillis; + public Optional currentTimeMillis(JSONObject message) { + return Optional.of(this.epochMillis); } } diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/FixedClockFactory.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/FixedClockFactory.java new file mode 100644 index 0000000000..b0248cdbb9 --- /dev/null +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/FixedClockFactory.java @@ -0,0 +1,44 @@ +/* + * + * 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.clock; + +import org.apache.metron.common.configuration.profiler.ProfilerConfig; + +/** + * A {@link ClockFactory} that always returns a {@link FixedClock}. + * + *

A {@link FixedClock} always returns the same time and is only useful for testing. + */ +public class FixedClockFactory implements ClockFactory { + + private long timestamp; + + /** + * @param timestamp The timestamp that all {@link Clock} objects created by this factory will report. + */ + public FixedClockFactory(long timestamp) { + this.timestamp = timestamp; + } + + @Override + public Clock createClock(ProfilerConfig config) { + return new FixedClock(timestamp); + } +} diff --git a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/WallClock.java b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/WallClock.java index 1a20c94112..20f62e3847 100644 --- a/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/WallClock.java +++ b/metron-analytics/metron-profiler-common/src/main/java/org/apache/metron/profiler/clock/WallClock.java @@ -20,15 +20,22 @@ package org.apache.metron.profiler.clock; -import java.io.Serializable; +import org.json.simple.JSONObject; + +import java.util.Optional; /** - * A clock that uses the system clock to provide wall clock time. + * A {@link Clock} that advances based on system time. + * + *

This {@link Clock} is used to advance time when the Profiler is running + * on processing time, rather than event time. */ -public class WallClock implements Clock, Serializable { +public class WallClock implements Clock { @Override - public long currentTimeMillis() { - return System.currentTimeMillis(); + public Optional currentTimeMillis(JSONObject message) { + + // the message does not matter; use system time + return Optional.of(System.currentTimeMillis()); } } 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 ff4c289520..ea9c5c66f9 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 @@ -20,6 +20,7 @@ package org.apache.metron.profiler; +import com.google.common.base.Ticker; import org.adrianwalker.multilinestring.Multiline; import org.apache.metron.common.configuration.profiler.ProfileConfig; import org.apache.metron.common.utils.JSONUtils; @@ -33,6 +34,9 @@ import java.util.List; import java.util.concurrent.TimeUnit; +import static java.util.concurrent.TimeUnit.HOURS; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.MINUTES; import static org.junit.Assert.assertEquals; public class DefaultMessageDistributorTest { @@ -83,16 +87,22 @@ public class DefaultMessageDistributorTest { private DefaultMessageDistributor distributor; private Context context; + private long periodDurationMillis = MINUTES.toMillis(15); + private long profileTimeToLiveMillis = MINUTES.toMillis(30); + private long maxNumberOfRoutes = Long.MAX_VALUE; @Before public void setup() throws Exception { + context = Context.EMPTY_CONTEXT(); JSONParser parser = new JSONParser(); messageOne = (JSONObject) parser.parse(inputOne); messageTwo = (JSONObject) parser.parse(inputTwo); + distributor = new DefaultMessageDistributor( - TimeUnit.MINUTES.toMillis(15), - TimeUnit.MINUTES.toMillis(30)); + periodDurationMillis, + profileTimeToLiveMillis, + maxNumberOfRoutes); } /** @@ -108,15 +118,18 @@ private ProfileConfig createDefinition(String json) throws IOException { */ @Test public void testDistribute() throws Exception { + + // setup + long timestamp = 100; ProfileConfig definition = createDefinition(profileOne); String entity = (String) messageOne.get("ip_src_addr"); MessageRoute route = new MessageRoute(definition, entity); - // distribute one message - distributor.distribute(messageOne, route, context); + // distribute one message and flush + distributor.distribute(messageOne, timestamp, route, context); + List measurements = distributor.flush(); // expect one measurement coming from one profile - List measurements = distributor.flush(); assertEquals(1, measurements.size()); ProfileMeasurement m = measurements.get(0); assertEquals(definition.getProfile(), m.getProfileName()); @@ -126,12 +139,17 @@ public void testDistribute() throws Exception { @Test public void testDistributeWithTwoProfiles() throws Exception { - // distribute one message to the first profile + // setup + long timestamp = 100; String entity = (String) messageOne.get("ip_src_addr"); - distributor.distribute(messageOne, new MessageRoute(createDefinition(profileOne), entity), context); + + // distribute one message to the first profile + MessageRoute routeOne = new MessageRoute(createDefinition(profileOne), entity); + distributor.distribute(messageOne, timestamp, routeOne, context); // distribute another message to the second profile, but same entity - distributor.distribute(messageOne, new MessageRoute(createDefinition(profileTwo), entity), context); + MessageRoute routeTwo = new MessageRoute(createDefinition(profileTwo), entity); + distributor.distribute(messageOne, timestamp, routeTwo, context); // expect 2 measurements; 1 for each profile List measurements = distributor.flush(); @@ -141,17 +159,150 @@ public void testDistributeWithTwoProfiles() throws Exception { @Test public void testDistributeWithTwoEntities() throws Exception { + // setup + long timestamp = 100; + // distribute one message String entityOne = (String) messageOne.get("ip_src_addr"); - distributor.distribute(messageOne, new MessageRoute(createDefinition(profileOne), entityOne), context); + MessageRoute routeOne = new MessageRoute(createDefinition(profileOne), entityOne); + distributor.distribute(messageOne, timestamp, routeOne, context); // distribute another message with a different entity String entityTwo = (String) messageTwo.get("ip_src_addr"); - distributor.distribute(messageTwo, new MessageRoute(createDefinition(profileTwo), entityTwo), context); + MessageRoute routeTwo = new MessageRoute(createDefinition(profileTwo), entityTwo); + distributor.distribute(messageTwo, timestamp, routeTwo, context); // expect 2 measurements; 1 for each entity List measurements = distributor.flush(); assertEquals(2, measurements.size()); } + /** + * A profile should expire after a fixed period of time. This test ensures that + * profiles are not expired before they are supposed to be. + */ + @Test + public void testNotYetTimeToExpireProfiles() throws Exception { + + // the ticker drives time to allow us to test cache expiration + FixedTicker ticker = new FixedTicker(); + + // setup + ProfileConfig definition = createDefinition(profileOne); + String entity = (String) messageOne.get("ip_src_addr"); + MessageRoute route = new MessageRoute(definition, entity); + distributor = new DefaultMessageDistributor( + periodDurationMillis, + profileTimeToLiveMillis, + maxNumberOfRoutes, + ticker); + + // distribute one message + distributor.distribute(messageOne, 1000000, route, context); + + // advance time to just shy of the profile TTL + ticker.advanceTime(profileTimeToLiveMillis - 1000, MILLISECONDS); + + // the profile should NOT have expired yet + assertEquals(0, distributor.flushExpired().size()); + assertEquals(1, distributor.flush().size()); + } + + /** + * A profile should expire after a fixed period of time. + */ + @Test + public void testProfilesShouldExpire() throws Exception { + + // the ticker drives time to allow us to test cache expiration + FixedTicker ticker = new FixedTicker(); + + // setup + ProfileConfig definition = createDefinition(profileOne); + String entity = (String) messageOne.get("ip_src_addr"); + MessageRoute route = new MessageRoute(definition, entity); + distributor = new DefaultMessageDistributor( + periodDurationMillis, + profileTimeToLiveMillis, + maxNumberOfRoutes, + ticker); + + // distribute one message + distributor.distribute(messageOne, 100000, route, context); + + // advance time to just beyond the period duration + ticker.advanceTime(profileTimeToLiveMillis + 1000, MILLISECONDS); + + // the profile should have expired by now + assertEquals(1, distributor.flushExpired().size()); + assertEquals(0, distributor.flush().size()); + } + + /** + * An expired profile is only kept around for a fixed period of time. It should be removed, if it + * has been on the expired cache for too long. + */ + @Test + public void testExpiredProfilesShouldBeRemoved() throws Exception { + + // the ticker drives time to allow us to test cache expiration + FixedTicker ticker = new FixedTicker(); + + // setup + ProfileConfig definition = createDefinition(profileOne); + String entity = (String) messageOne.get("ip_src_addr"); + MessageRoute route = new MessageRoute(definition, entity); + distributor = new DefaultMessageDistributor( + periodDurationMillis, + profileTimeToLiveMillis, + maxNumberOfRoutes, + ticker); + + // distribute one message + distributor.distribute(messageOne, 1000000, route, context); + + // advance time a couple of hours + ticker.advanceTime(2, HOURS); + + // the profile should have been expired + assertEquals(0, distributor.flush().size()); + + // advance time a couple of hours + ticker.advanceTime(2, HOURS); + + // the profile should have been removed from the expired cache + assertEquals(0, distributor.flushExpired().size()); + } + + /** + * An implementation of Ticker that can be used to drive time + * when testing the Guava caches. + */ + private class FixedTicker extends Ticker { + + /** + * The time that will be reported. + */ + private long timestampNanos; + + public FixedTicker() { + this.timestampNanos = Ticker.systemTicker().read(); + } + + public FixedTicker startAt(long timestampNanos) { + this.timestampNanos = timestampNanos; + return this; + } + + public FixedTicker advanceTime(long time, TimeUnit units) { + this.timestampNanos += units.toNanos(time); + return this; + } + + @Override + public long read() { + return this.timestampNanos; + } + } + } 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 d25b7ff13a..24eb5f8fb7 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 @@ -23,8 +23,6 @@ import org.adrianwalker.multilinestring.Multiline; import org.apache.metron.common.configuration.profiler.ProfileConfig; import org.apache.metron.common.utils.JSONUtils; -import org.apache.metron.profiler.clock.Clock; -import org.apache.metron.profiler.clock.FixedClock; import org.apache.metron.stellar.dsl.Context; import org.json.simple.JSONObject; import org.json.simple.parser.JSONParser; @@ -82,7 +80,9 @@ public void setup() throws Exception { */ @Test public void testInit() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(testInitProfile, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -92,7 +92,7 @@ public void testInit() throws Exception { .build(); // execute - builder.apply(message); + builder.apply(message, timestamp); Optional m = builder.flush(); assertTrue(m.isPresent()); @@ -106,7 +106,9 @@ public void testInit() throws Exception { */ @Test public void testInitWithNoMessage() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(testInitProfile, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -146,7 +148,9 @@ public void testInitWithNoMessage() throws Exception { */ @Test public void testUpdate() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(testUpdateProfile, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -158,7 +162,12 @@ public void testUpdate() throws Exception { // execute int count = 10; for(int i=0; i m = builder.flush(); assertTrue(m.isPresent()); @@ -183,7 +192,9 @@ public void testUpdate() throws Exception { */ @Test public void testResult() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(testResultProfile, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -193,7 +204,7 @@ public void testResult() throws Exception { .build(); // execute - builder.apply(message); + builder.apply(message, timestamp); Optional m = builder.flush(); assertTrue(m.isPresent()); @@ -206,40 +217,38 @@ public void testResult() throws Exception { */ @Test public void testProfilePeriodOnFlush() throws Exception { - // setup - FixedClock clock = new FixedClock(); - clock.setTime(100); + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(testResultProfile, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) .withEntity("10.0.0.1") .withPeriodDuration(10, TimeUnit.MINUTES) .withContext(Context.EMPTY_CONTEXT()) - .withClock(clock) .build(); { // apply a message and flush - builder.apply(message); + builder.apply(message, timestamp); Optional m = builder.flush(); assertTrue(m.isPresent()); // validate the profile period - ProfilePeriod expected = new ProfilePeriod(clock.currentTimeMillis(), 10, TimeUnit.MINUTES); + ProfilePeriod expected = new ProfilePeriod(timestamp, 10, TimeUnit.MINUTES); assertEquals(expected, m.get().getPeriod()); } { - // advance time by at least one period - 10 minutes - clock.setTime(clock.currentTimeMillis() + TimeUnit.MINUTES.toMillis(10)); + // advance time by at least one period... about 10 minutes + timestamp += TimeUnit.MINUTES.toMillis(10); // apply a message and flush again - builder.apply(message); + builder.apply(message, timestamp); Optional m = builder.flush(); assertTrue(m.isPresent()); // validate the profile period - ProfilePeriod expected = new ProfilePeriod(clock.currentTimeMillis(), 10, TimeUnit.MINUTES); + ProfilePeriod expected = new ProfilePeriod(timestamp, 10, TimeUnit.MINUTES); assertEquals(expected, m.get().getPeriod()); } } @@ -262,7 +271,9 @@ public void testProfilePeriodOnFlush() throws Exception { */ @Test public void testGroupBy() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(testGroupByProfile, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -272,7 +283,7 @@ public void testGroupBy() throws Exception { .build(); // execute - builder.apply(message); + builder.apply(message, timestamp); Optional m = builder.flush(); assertTrue(m.isPresent()); @@ -300,23 +311,20 @@ public void testGroupBy() throws Exception { */ @Test public void testStateAvailableToGroupBy() throws Exception { - FixedClock clock = new FixedClock(); - clock.setTime(1503081070340L); - long periodDurationMillis = TimeUnit.MINUTES.toMillis(10); - ProfilePeriod period = new ProfilePeriod(clock.currentTimeMillis(), 10, TimeUnit.MINUTES); // setup + long timestamp = 1503081070340L; + ProfilePeriod period = new ProfilePeriod(timestamp, 10, TimeUnit.MINUTES); definition = JSONUtils.INSTANCE.load(testStateAvailableToGroupBy, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) .withEntity("10.0.0.1") .withPeriodDuration(10, TimeUnit.MINUTES) .withContext(Context.EMPTY_CONTEXT()) - .withClock(clock) .build(); // execute - builder.apply(message); + builder.apply(message, timestamp); Optional m = builder.flush(); assertTrue(m.isPresent()); @@ -350,7 +358,9 @@ public void testStateAvailableToGroupBy() throws Exception { @Test public void testFlushDoesNotClearsState() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(testFlushProfile, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -362,16 +372,24 @@ public void testFlushDoesNotClearsState() throws Exception { // execute - accumulate some state then flush it int count = 10; for(int i=0; i m = builder.flush(); - assertTrue(m.isPresent()); // validate + assertTrue(m.isPresent()); assertEquals(33, m.get().getProfileValue()); } @@ -395,7 +413,9 @@ public void testFlushDoesNotClearsState() throws Exception { @Test public void testFlushDoesNotClearsStateButInitDoes() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(testFlushProfileWithNaiveInit, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -407,18 +427,27 @@ public void testFlushDoesNotClearsStateButInitDoes() throws Exception { // execute - accumulate some state then flush it int count = 10; for(int i=0; i m = builder.flush(); assertTrue(m.isPresent()); // validate assertEquals(3, m.get().getProfileValue()); } + /** * { * "profile": "test", @@ -434,7 +463,9 @@ public void testFlushDoesNotClearsStateButInitDoes() throws Exception { */ @Test public void testEntity() throws Exception { + // setup + long timestamp = 100; final String entity = "10.0.0.1"; definition = JSONUtils.INSTANCE.load(testFlushProfile, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() @@ -445,7 +476,7 @@ public void testEntity() throws Exception { .build(); // execute - builder.apply(message); + builder.apply(message, timestamp); Optional m = builder.flush(); assertTrue(m.isPresent()); @@ -473,7 +504,9 @@ public void testEntity() throws Exception { */ @Test public void testResultWithProfileExpression() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(testResultWithProfileExpression, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -483,7 +516,7 @@ public void testResultWithProfileExpression() throws Exception { .build(); // execute - builder.apply(message); + builder.apply(message, timestamp); Optional m = builder.flush(); assertTrue(m.isPresent()); @@ -515,7 +548,9 @@ public void testResultWithProfileExpression() throws Exception { */ @Test public void testResultWithTriageExpression() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(testResultWithTriageExpression, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -525,7 +560,7 @@ public void testResultWithTriageExpression() throws Exception { .build(); // execute - builder.apply(message); + builder.apply(message, timestamp); Optional m = builder.flush(); assertTrue(m.isPresent()); @@ -550,7 +585,9 @@ public void testResultWithTriageExpression() throws Exception { @Test public void testBadInitExpression() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(badInitProfile, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -560,7 +597,7 @@ public void testBadInitExpression() throws Exception { .build(); // due to the bad expression, there should be no result - builder.apply(message); + builder.apply(message, timestamp); assertFalse(builder.flush().isPresent()); } @@ -579,7 +616,9 @@ public void testBadInitExpression() throws Exception { @Test public void testBadResultExpression() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(badSimpleResultProfile, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -589,7 +628,7 @@ public void testBadResultExpression() throws Exception { .build(); // due to the bad expression, there should be no result - builder.apply(message); + builder.apply(message, timestamp); assertFalse(builder.flush().isPresent()); } @@ -608,7 +647,9 @@ public void testBadResultExpression() throws Exception { @Test public void testBadGroupByExpression() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(badGroupByProfile, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -618,7 +659,7 @@ public void testBadGroupByExpression() throws Exception { .build(); // due to the bad expression, there should be no result - builder.apply(message); + builder.apply(message, timestamp); assertFalse(builder.flush().isPresent()); } @@ -641,7 +682,9 @@ public void testBadGroupByExpression() throws Exception { @Test public void testBadResultProfileExpression() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(badResultProfile, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -651,7 +694,7 @@ public void testBadResultProfileExpression() throws Exception { .build(); // due to the bad expression, there should be no result - builder.apply(message); + builder.apply(message, timestamp); assertFalse(builder.flush().isPresent()); } @@ -674,7 +717,9 @@ public void testBadResultProfileExpression() throws Exception { @Test public void testBadResultTriageExpression() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(badResultTriage, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -684,7 +729,7 @@ public void testBadResultTriageExpression() throws Exception { .build(); // due to the bad expression, there should be no result - builder.apply(message); + builder.apply(message, timestamp); assertFalse(builder.flush().isPresent()); } @@ -707,7 +752,9 @@ public void testBadResultTriageExpression() throws Exception { */ @Test public void testBadUpdateExpression() throws Exception { + // setup + long timestamp = 100; definition = JSONUtils.INSTANCE.load(badUpdateProfile, ProfileConfig.class); builder = new DefaultProfileBuilder.Builder() .withDefinition(definition) @@ -717,7 +764,7 @@ public void testBadUpdateExpression() throws Exception { .build(); // execute - builder.apply(message); + builder.apply(message, timestamp); // if the update expression fails, the profile should still flush. Optional m = builder.flush(); 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 3a51ea4924..1a72111368 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 @@ -20,7 +20,6 @@ package org.apache.metron.profiler; -import org.apache.metron.profiler.ProfilePeriod; import org.junit.Test; import java.util.concurrent.TimeUnit; diff --git a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/StandAloneProfilerTest.java b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/StandAloneProfilerTest.java new file mode 100644 index 0000000000..2269c86084 --- /dev/null +++ b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/StandAloneProfilerTest.java @@ -0,0 +1,255 @@ +/* + * + * 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; + +import org.adrianwalker.multilinestring.Multiline; +import org.apache.metron.common.configuration.profiler.ProfilerConfig; +import org.apache.metron.common.utils.JSONUtils; +import org.apache.metron.stellar.dsl.Context; +import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Tests the StandAloneProfiler class. + */ +public class StandAloneProfilerTest { + + /** + * { + * "profiles": [ + * ] + * } + */ + @Multiline + private String noProfiles; + + /** + * { + * "profiles": [ + * { + * "profile": "profile1", + * "foreach": "'global'", + * "init": { "count": 0 }, + * "update": { "count": "count + 1" }, + * "result": "count" + * } + * ] + * } + */ + @Multiline + private String oneProfile; + + /** + * { + * "profiles": [ + * { + * "profile": "profile1", + * "foreach": "'global1'", + * "result": "'result'" + * }, + * { + * "profile": "profile2", + * "foreach": "'global2'", + * "result": "'result'" + * } + * ] + * } + */ + @Multiline + private String twoProfiles; + + /** + * { + * "ip_src_addr": "10.0.0.1", + * "ip_dst_addr": "10.0.0.20", + * "protocol": "HTTP", + * "timestamp": 2222222222222, + * } + */ + @Multiline + private String messageJson; + + private JSONObject message; + + private long periodDurationMillis = TimeUnit.MINUTES.toMillis(15); + + private Context context = Context.EMPTY_CONTEXT(); + + @Before + public void setup() throws Exception { + + // parse the input message + JSONParser parser = new JSONParser(); + message = (JSONObject) parser.parse(messageJson); + } + + @Test + public void testWithOneProfile() throws Exception { + + StandAloneProfiler profiler = createProfiler(oneProfile); + profiler.apply(message); + profiler.apply(message); + profiler.apply(message); + + List measurements = profiler.flush(); + assertEquals(1, measurements.size()); + + // expect 1 measurement for the 1 profile that has been defined + ProfileMeasurement m = measurements.get(0); + assertEquals("profile1", m.getProfileName()); + assertEquals(3, m.getProfileValue()); + } + + + @Test + public void testWithTwoProfiles() throws Exception { + + StandAloneProfiler profiler = createProfiler(twoProfiles); + profiler.apply(message); + profiler.apply(message); + profiler.apply(message); + + List measurements = profiler.flush(); + assertEquals(2, measurements.size()); + + // expect 2 measurements, 1 for each profile + List expected = Arrays.asList(new String[] { "profile1", "profile2" }); + { + ProfileMeasurement m = measurements.get(0); + assertTrue(expected.contains(m.getProfileName())); + assertEquals("result", m.getProfileValue()); + } + { + ProfileMeasurement m = measurements.get(1); + assertTrue(expected.contains(m.getProfileName())); + assertEquals("result", m.getProfileValue()); + } + } + + /** + * The message count and route count will always be equal, if there is only one + * profile defined. The message count and route count can be different when there + * are multiple profiles defined that each use the same message. + */ + @Test + public void testRouteAndMessageCounters() throws Exception { + { + StandAloneProfiler profiler = createProfiler(noProfiles); + + profiler.apply(message); + assertEquals(1, profiler.getMessageCount()); + assertEquals(0, profiler.getRouteCount()); + + profiler.apply(message); + assertEquals(2, profiler.getMessageCount()); + assertEquals(0, profiler.getRouteCount()); + + profiler.apply(message); + assertEquals(3, profiler.getMessageCount()); + assertEquals(0, profiler.getRouteCount()); + } + { + StandAloneProfiler profiler = createProfiler(oneProfile); + + profiler.apply(message); + assertEquals(1, profiler.getMessageCount()); + assertEquals(1, profiler.getRouteCount()); + + profiler.apply(message); + assertEquals(2, profiler.getMessageCount()); + assertEquals(2, profiler.getRouteCount()); + + profiler.apply(message); + assertEquals(3, profiler.getMessageCount()); + assertEquals(3, profiler.getRouteCount()); + } + { + StandAloneProfiler profiler = createProfiler(twoProfiles); + + profiler.apply(message); + assertEquals(1, profiler.getMessageCount()); + assertEquals(2, profiler.getRouteCount()); + + profiler.apply(message); + assertEquals(2, profiler.getMessageCount()); + assertEquals(4, profiler.getRouteCount()); + + profiler.apply(message); + assertEquals(3, profiler.getMessageCount()); + assertEquals(6, profiler.getRouteCount()); + } + } + + @Test + public void testProfileCount() throws Exception { + { + StandAloneProfiler profiler = createProfiler(noProfiles); + assertEquals(0, profiler.getProfileCount()); + } + { + StandAloneProfiler profiler = createProfiler(oneProfile); + assertEquals(1, profiler.getProfileCount()); + } + { + StandAloneProfiler profiler = createProfiler(twoProfiles); + assertEquals(2, profiler.getProfileCount()); + } + } + + /** + * Creates a ProfilerConfig based on a string containing JSON. + * + * @param configAsJSON The config as JSON. + * @return The ProfilerConfig. + * @throws Exception + */ + private ProfilerConfig toProfilerConfig(String configAsJSON) throws Exception { + + InputStream in = new ByteArrayInputStream(configAsJSON.getBytes("UTF-8")); + return JSONUtils.INSTANCE.load(in, ProfilerConfig.class); + } + + /** + * Creates the StandAloneProfiler + * + * @param profileJson The Profiler configuration to use as a String containing JSON. + * @throws Exception + */ + private StandAloneProfiler createProfiler(String profileJson) throws Exception { + + // the TTL and max routes need not be bounded + long profileTimeToLiveMillis = Long.MAX_VALUE; + long maxNumberOfRoutes = Long.MAX_VALUE; + + ProfilerConfig config = toProfilerConfig(profileJson); + return new StandAloneProfiler(config, periodDurationMillis, profileTimeToLiveMillis, maxNumberOfRoutes, context); + } +} diff --git a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/clock/DefaultClockFactoryTest.java b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/clock/DefaultClockFactoryTest.java new file mode 100644 index 0000000000..c99b40149b --- /dev/null +++ b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/clock/DefaultClockFactoryTest.java @@ -0,0 +1,75 @@ +/* + * + * 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.clock; + +import static org.junit.Assert.assertTrue; + +import org.apache.metron.common.configuration.profiler.ProfilerConfig; +import org.junit.Before; +import org.junit.Test; + +import java.util.Optional; + +/** + * Tests the DefaultClockFactory. + */ +public class DefaultClockFactoryTest { + + /** + * The object under test. + */ + private DefaultClockFactory clockFactory; + + @Before + public void setup() { + clockFactory = new DefaultClockFactory(); + } + + /** + * When a 'timestampField' is defined the factory should return a clock + * that deals with event time. + */ + @Test + public void testCreateEventTimeClock() { + + // configure the profiler to use event time + ProfilerConfig config = new ProfilerConfig(); + config.setTimestampField(Optional.of("timestamp")); + + // the factory should return a clock that handles 'event time' + Clock clock = clockFactory.createClock(config); + assertTrue(clock instanceof EventTimeClock); + } + + /** + * When a 'timestampField' is defined the factory should return a clock + * that deals with processing time. + */ + @Test + public void testCreateProcessingTimeClock() { + + // the profiler uses processing time by default + ProfilerConfig config = new ProfilerConfig(); + + // the factory should return a clock that handles 'processing time' + Clock clock = clockFactory.createClock(config); + assertTrue(clock instanceof WallClock); + } +} diff --git a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/clock/EventTimeClockTest.java b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/clock/EventTimeClockTest.java new file mode 100644 index 0000000000..03972508b5 --- /dev/null +++ b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/clock/EventTimeClockTest.java @@ -0,0 +1,115 @@ +/* + * + * 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.clock; + +import org.json.simple.JSONObject; +import org.junit.Test; + +import java.util.Optional; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class EventTimeClockTest { + + private final String timestampField = "timestamp"; + + public JSONObject createMessage() { + return new JSONObject(); + } + + /** + * The event time should be extracted from a field contained within a message. + */ + @Test + public void testEventTime() { + + JSONObject message = createMessage(); + + // add a field containing a timestamp to the message + final Long timestamp = System.currentTimeMillis(); + message.put(timestampField, timestamp); + + // what time is it? + EventTimeClock clock = new EventTimeClock(timestampField); + Optional result = clock.currentTimeMillis(message); + + // validate + assertTrue(result.isPresent()); + assertEquals(timestamp, result.get()); + } + + /** + * If the timestamp field is a String, it should be converted to Long and used as-is. + */ + @Test + public void testEventTimeWithString() { + JSONObject message = createMessage(); + + // the timestamp field is a string + final Long timestamp = System.currentTimeMillis(); + message.put(timestampField, timestamp.toString()); + + // what time is it? + EventTimeClock clock = new EventTimeClock(timestampField); + Optional result = clock.currentTimeMillis(message); + + // validate + assertTrue(result.isPresent()); + assertEquals(timestamp, result.get()); + } + + /** + * If the message does not contain the timestamp field, then nothing should be returned. + */ + @Test + public void testMissingTimestampField() { + + // no timestamp added to the message + JSONObject message = createMessage(); + + // what time is it? + EventTimeClock clock = new EventTimeClock(timestampField); + Optional result = clock.currentTimeMillis(message); + + // validate + assertFalse(result.isPresent()); + } + + /** + * No timestamp should be returned if the value stored in the timestamp field + * cannot be coerced into a valid timestamp. + */ + @Test + public void testInvalidValue() { + + // create a message with an invalid value stored in the timestamp field + JSONObject message = createMessage(); + message.put(timestampField, "invalid-timestamp-value"); + + // what time is it? + EventTimeClock clock = new EventTimeClock(timestampField); + Optional result = clock.currentTimeMillis(message); + + // no value should be returned + assertFalse(result.isPresent()); + } +} diff --git a/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/clock/WallClockTest.java b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/clock/WallClockTest.java new file mode 100644 index 0000000000..76b2d7baa0 --- /dev/null +++ b/metron-analytics/metron-profiler-common/src/test/java/org/apache/metron/profiler/clock/WallClockTest.java @@ -0,0 +1,54 @@ +/* + * + * 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.clock; + +import org.json.simple.JSONObject; +import org.junit.Test; + +import java.util.Optional; + +import static org.junit.Assert.assertTrue; + +public class WallClockTest { + + public JSONObject createMessage() { + return new JSONObject(); + } + + /** + * The wall clock time ALWAYS comes from the system clock. + */ + @Test + public void testCurrentTimeMillis() { + + JSONObject message = createMessage(); + long before = System.currentTimeMillis(); + + // what time is it? + WallClock clock = new WallClock(); + Optional result = clock.currentTimeMillis(message); + + // validate + long after = System.currentTimeMillis(); + assertTrue(result.isPresent()); + assertTrue(result.get() >= before); + assertTrue(result.get() <= after); + } +} diff --git a/metron-analytics/metron-profiler/README.md b/metron-analytics/metron-profiler/README.md index dc5ec07ce8..576b15a145 100644 --- a/metron-analytics/metron-profiler/README.md +++ b/metron-analytics/metron-profiler/README.md @@ -328,6 +328,47 @@ Continuing the previous running example, at this point, you have seen how your p ## Anatomy of a Profile +### Profiler + +The Profiler configuration contains only two fields; only one of which is required. + + ``` + { + "profiles": [ + { "profile": "one", ... }, + { "profile": "two", ... } + ], + "timestampField": "timestamp" + } + ``` + +#### `profiles` + +*Required* + +A list of zero or more Profile definitions. + +#### `timestampField` + +*Optional* + +##### Processing Time + +By default, no `timestampField` is defined. In this case, the Profiler uses system time when generating profiles. This means that the profiles are generated based on when the data has been processed by the Profiler. This is also known as 'processing time'. + +This is the simplest mode of operation, but has some draw backs. If the Profiler is consuming live data and all is well, the processing and event times will likely remain similar and consistent. When processing time diverges from event time, the profiles produced can be skewed. + +There are a few scenarios that could cause skewed profiles when using processing time. For example when a system has undergone a scheduled maintenance window and is restarted, a high volume of messages will need to be processed by the Profiler. The output of the Profiler will indicate an increase in activity during this time, although no change in activity actually occurred on the target network. The same situation could occur if an upstream system which provides telemetry undergoes an outage. + +##### Event Time + +Alternatively, you can define a `timestampField`. This must be the name of a field contained within the telemetry processed by the Profiler. The Profiler will extract and use the timestamp contained within this field. + +* The field must contain a timestamp in epoch milliseconds. +* If a message does not contain this field, it will be dropped by the Profiler. + +### Profiles + A profile definition requires a JSON-formatted set of elements, many of which can contain Stellar code. The specification contains the following elements. (For the impatient, skip ahead to the [Examples](#examples).) | Name | | Description @@ -466,15 +507,19 @@ The values can be changed on disk and then the Profiler topology must be restart | Setting | Description |--- |--- -| [`profiler.input.topic`](#profilerinputtopic) | The name of the Kafka topic from which to consume data. -| [`profiler.output.topic`](#profileroutputtopic) | The name of the Kafka topic to which profile data is written. Only used with profiles that define the [`triage` result field](#result). +| [`profiler.input.topic`](#profilerinputtopic) | The name of the input Kafka topic. +| [`profiler.output.topic`](#profileroutputtopic) | The name of the output Kafka topic. | [`profiler.period.duration`](#profilerperiodduration) | The duration of each profile period. -| [`profiler.period.duration.units`](#profilerperioddurationunits) | The units used to specify the [`profiler.period.duration`](#profilerperiodduration). +| [`profiler.period.duration.units`](#profilerperioddurationunits) | The units used to specify the [`profiler.period.duration`](#profilerperiodduration). +| [`profiler.window.duration`](#profilerwindowduration) | The duration of each profile window. +| [`profiler.window.duration.units`](#profilerpwindowdurationunits) | The units used to specify the [`profiler.window.duration`](#profilerwindowduration). +| [`profiler.window.lag`](#profilerwindowlag) | The maximum time lag for timestamps. +| [`profiler.window.lag.units`](#profilerpwindowlagunits) | The units used to specify the [`profiler.window.lag`](#profilerwindowlag). | [`profiler.workers`](#profilerworkers) | The number of worker processes for the topology. | [`profiler.executors`](#profilerexecutors) | The number of executors to spawn per component. | [`profiler.ttl`](#profilerttl) | If a message has not been applied to a Profile in this period of time, the Profile will be forgotten and its resources will be cleaned up. | [`profiler.ttl.units`](#profilerttlunits) | The units used to specify the `profiler.ttl`. -| [`profiler.hbase.salt.divisor`](#profilerhbasesaltdivisor) | A salt is prepended to the row key to help prevent hotspotting. +| [`profiler.hbase.salt.divisor`](#profilerhbasesaltdivisor) | A salt is prepended to the row key to help prevent hot-spotting. | [`profiler.hbase.table`](#profilerhbasetable) | The name of the HBase table that profiles are written to. | [`profiler.hbase.column.family`](#profilerhbasecolumnfamily) | The column family used to store profiles. | [`profiler.hbase.batch`](#profilerhbasebatch) | The number of puts that are written to HBase in a single batch. @@ -508,6 +553,36 @@ The units used to specify the `profiler.period.duration`. This value should be *Important*: To read a profile using the Profiler Client, the Profiler Client's `profiler.client.period.duration.units` property must match this value. Otherwise, the [Profiler Client](metron-analytics/metron-profiler-client) will be unable to read the profile data. +### `profiler.window.duration` + +*Default*: 30 + +The duration of each profile window. Telemetry that arrives within a slice of time is processed within a single window. + +Many windows of telemetry will be processed during a single profile period. This does not change the output of the Profiler, it only changes how the Profiler processes data. The window defines how much data the Profiler processes in a single pass. + +This value should be defined along with [`profiler.window.duration.units`](#profilerwindowdurationunits). + +This value must be less than the period duration as defined by [`profiler.period.duration`](#profilerperiodduration) and [`profiler.period.duration.units`](#profilerperioddurationunits). + +### `profiler.window.duration.units` + +*Default*: SECONDS + +The units used to specify the `profiler.window.duration`. This value should be defined along with [`profiler.window.duration`](#profilerwindowduration). + +### `profiler.window.lag` + +*Default*: 1 + +The maximum time lag for timestamps. Timestamps cannot arrive out-of-order by more than this amount. This value should be defined along with [`profiler.window.lag.units`](#profilerwindowlagunits). + +### `profiler.window.lag.units` + +*Default*: SECONDS + +The units used to specify the `profiler.window.lag`. This value should be defined along with [`profiler.window.lag`](#profilerwindowlag). + ### `profiler.workers` *Default*: 1 diff --git a/metron-analytics/metron-profiler/src/main/config/profiler.properties b/metron-analytics/metron-profiler/src/main/config/profiler.properties index 896f8d57bb..7d5b8e3552 100644 --- a/metron-analytics/metron-profiler/src/main/config/profiler.properties +++ b/metron-analytics/metron-profiler/src/main/config/profiler.properties @@ -22,6 +22,10 @@ topology.worker.childopts= topology.auto-credentials= +profiler.workers=1 +profiler.executors=0 +topology.message.timeout.secs=30 +topology.max.spout.pending=100000 ##### Profiler ##### @@ -29,10 +33,16 @@ profiler.input.topic=indexing profiler.output.topic=enrichments profiler.period.duration=15 profiler.period.duration.units=MINUTES -profiler.workers=1 -profiler.executors=0 +profiler.window.duration=30 +profiler.window.duration.units=SECONDS profiler.ttl=30 profiler.ttl.units=MINUTES +profiler.event.time.lag=1 +profiler.event.time.lag.units=MINUTES +profiler.max.routes.per.bolt=10000 + +##### HBase ##### + profiler.hbase.salt.divisor=1000 profiler.hbase.table=profiler profiler.hbase.column.family=P diff --git a/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml b/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml index 9ec5ba40cb..c1c2f1acf6 100644 --- a/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml +++ b/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml @@ -17,10 +17,12 @@ name: "profiler" config: - topology.worker.childopts: ${topology.worker.childopts} topology.workers: ${profiler.workers} topology.acker.executors: ${profiler.executors} + topology.worker.childopts: ${topology.worker.childopts} topology.auto-credentials: ${topology.auto-credentials} + topology.message.timeout.secs: ${topology.message.timeout.secs} + topology.max.spout.pending: ${topology.max.spout.pending} components: @@ -107,11 +109,23 @@ components: - name: "withProducerConfigs" args: [ref: "kafkaWriterProps"] - - id: "kafkaDestinationHandler" - className: "org.apache.metron.profiler.bolt.KafkaDestinationHandler" + - id: "kafkaEmitter" + className: "org.apache.metron.profiler.bolt.KafkaEmitter" + + - id: "hbaseEmitter" + className: "org.apache.metron.profiler.bolt.HBaseEmitter" + + - id: "windowDuration" + className: "org.apache.storm.topology.base.BaseWindowedBolt$Duration" + constructorArgs: + - ${profiler.window.duration} + - "${profiler.window.duration.units}" - - id: "hbaseDestinationHandler" - className: "org.apache.metron.profiler.bolt.HBaseDestinationHandler" + - id: "eventTimeLag" + className: "org.apache.storm.topology.base.BaseWindowedBolt$Duration" + constructorArgs: + - ${profiler.window.lag} + - "${profiler.window.lag.units}" spouts: @@ -129,17 +143,23 @@ bolts: - id: "builderBolt" className: "org.apache.metron.profiler.bolt.ProfileBuilderBolt" - constructorArgs: - - "${kafka.zk}" configMethods: + - name: "withZookeeperUrl" + args: ["${kafka.zk}"] - name: "withPeriodDuration" args: [${profiler.period.duration}, "${profiler.period.duration.units}"] - name: "withProfileTimeToLive" args: [${profiler.ttl}, "${profiler.ttl.units}"] - - name: "withDestinationHandler" - args: [ref: "kafkaDestinationHandler"] - - name: "withDestinationHandler" - args: [ref: "hbaseDestinationHandler"] + - name: "withEmitter" + args: [ref: "kafkaEmitter"] + - name: "withEmitter" + args: [ref: "hbaseEmitter"] + - name: "withTumblingWindow" + args: [ref: "windowDuration"] + - name: "withLag" + args: [ref: "eventTimeLag"] + - name: "withMaxNumberOfRoutes" + args: [${profiler.max.routes.per.bolt}] - id: "hbaseBolt" className: "org.apache.metron.hbase.bolt.HBaseBolt" diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/FixedFrequencyFlushSignal.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/FixedFrequencyFlushSignal.java new file mode 100644 index 0000000000..b9f57dd54f --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/FixedFrequencyFlushSignal.java @@ -0,0 +1,126 @@ +/* + * 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.bolt; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.invoke.MethodHandles; + +/** + * Signals a flush on a fixed frequency; every X milliseconds. + */ +public class FixedFrequencyFlushSignal implements FlushSignal { + + protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + /** + * The latest known timestamp. + */ + private long currentTime; + + /** + * The time when the next flush should occur. + */ + private long flushTime; + + /** + * The amount of time between flushes in milliseconds. + */ + private long flushFrequency; + + public FixedFrequencyFlushSignal(long flushFrequencyMillis) { + + if(flushFrequencyMillis < 0) { + throw new IllegalArgumentException("flush frequency must be >= 0"); + } + + this.flushFrequency = flushFrequencyMillis; + reset(); + } + + /** + * Resets the state used to keep track of time. + */ + @Override + public void reset() { + flushTime = 0; + currentTime = 0; + + LOG.debug("Flush counters reset"); + } + + /** + * Update the internal state which tracks time. + * + * @param timestamp The timestamp received within a tuple. + */ + @Override + public void update(long timestamp) { + + if(timestamp > currentTime) { + + // need to update current time + LOG.debug("Updating current time; last={}, new={}", currentTime, timestamp); + currentTime = timestamp; + + } else if ((currentTime - timestamp) > flushFrequency) { + + // significantly out-of-order timestamps + LOG.warn("Timestamps out-of-order by '{}' ms. This may indicate a problem in the data. last={}, current={}", + (currentTime - timestamp), + timestamp, + currentTime); + } + + if(flushTime == 0) { + + // set the next time to flush + flushTime = currentTime + flushFrequency; + LOG.debug("Setting flush time; flushTime={}, currentTime={}, flushFreq={}", + flushTime, + currentTime, + flushFrequency); + } + } + + /** + * Returns true, if it is time to flush. + * + * @return True if time to flush. Otherwise, false. + */ + @Override + public boolean isTimeToFlush() { + + boolean flush = currentTime > flushTime; + LOG.debug("Flush={}, '{}' ms until flush; currentTime={}, flushTime={}", + flush, + flush ? 0 : (flushTime-currentTime), + currentTime, + flushTime); + + return flush; + } + + @Override + public long currentTimeMillis() { + return currentTime; + } +} diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/FlushSignal.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/FlushSignal.java new file mode 100644 index 0000000000..0a9fc76c87 --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/FlushSignal.java @@ -0,0 +1,51 @@ +/* + * 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.bolt; + +/** + * Signals when it is time to flush a profile. + */ +public interface FlushSignal { + + /** + * Returns true, if it is time to flush. + * + * @return True if time to flush. Otherwise, false. + */ + boolean isTimeToFlush(); + + /** + * Update the signaller with a known timestamp. + * + * @param timestamp A timestamp expected to be epoch milliseconds + */ + void update(long timestamp); + + /** + * Reset the signaller. + */ + void reset(); + + /** + * Returns the current time in epoch milliseconds. + * @return The current time in epoch milliseconds. + */ + long currentTimeMillis(); +} diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/HBaseDestinationHandler.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/HBaseEmitter.java similarity index 79% rename from metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/HBaseDestinationHandler.java rename to metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/HBaseEmitter.java index 4fa5dc1ac3..8e1229a4d5 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/HBaseDestinationHandler.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/HBaseEmitter.java @@ -23,19 +23,24 @@ import org.apache.storm.topology.OutputFieldsDeclarer; import org.apache.storm.tuple.Fields; import org.apache.storm.tuple.Values; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.Serializable; +import java.lang.invoke.MethodHandles; /** - * Handles emitting a ProfileMeasurement to the stream which writes - * profile measurements to HBase. + * Responsible for emitting a {@link ProfileMeasurement} to an output stream that will + * persist data in HBase. */ -public class HBaseDestinationHandler implements DestinationHandler, Serializable { +public class HBaseEmitter implements ProfileMeasurementEmitter, Serializable { + + protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); /** * The stream identifier used for this destination; */ - private String streamId = "hbase"; + private String streamId = "hbase"; @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaDestinationHandler.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java similarity index 86% rename from metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaDestinationHandler.java rename to metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java index be8246869e..29d1a49943 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaDestinationHandler.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java @@ -32,10 +32,10 @@ import org.slf4j.LoggerFactory; /** - * Handles emitting a ProfileMeasurement to the stream which writes - * profile measurements to Kafka. + * Responsible for emitting a {@link ProfileMeasurement} to an output stream that will + * persist data in HBase. */ -public class KafkaDestinationHandler implements DestinationHandler, Serializable { +public class KafkaEmitter implements ProfileMeasurementEmitter, Serializable { protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); @@ -75,8 +75,12 @@ public void emit(ProfileMeasurement measurement, OutputCollector collector) { message.put(key, value); } else { - LOG.error(String.format("triage expression has invalid type. expect primitive types only. skipping: profile=%s, entity=%s, expression=%s, type=%s", - measurement.getDefinition().getProfile(), measurement.getEntity(), key, ClassUtils.getShortClassName(value, "null"))); + LOG.error(String.format( + "triage expression must result in primitive type, skipping; type=%s, profile=%s, entity=%s, expr=%s", + ClassUtils.getShortClassName(value, "null"), + measurement.getDefinition().getProfile(), + measurement.getEntity(), + key)); } }); diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ManualFlushSignal.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ManualFlushSignal.java new file mode 100644 index 0000000000..d8e9539a52 --- /dev/null +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ManualFlushSignal.java @@ -0,0 +1,54 @@ +/* + * 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.bolt; + +/** + * Signals that a flush should occur. + * + *

The flush signal can be turned on or off like a switch as needed. Most useful for testing. + */ +public class ManualFlushSignal implements FlushSignal { + + private boolean flushNow = false; + + public void setFlushNow(boolean flushNow) { + this.flushNow = flushNow; + } + + @Override + public boolean isTimeToFlush() { + return flushNow; + } + + @Override + public void update(long timestamp) { + // nothing to do + } + + @Override + public void reset() { + // nothing to do. + } + + @Override + public long currentTimeMillis() { + // not needed + return 0; + } +} 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 3c8d875680..ffe823ff2e 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 @@ -20,19 +20,36 @@ package org.apache.metron.profiler.bolt; -import org.apache.metron.common.bolt.ConfiguredProfilerBolt; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.curator.RetryPolicy; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.recipes.cache.TreeCacheEvent; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.metron.common.Constants; +import org.apache.metron.common.configuration.ConfigurationType; +import org.apache.metron.common.configuration.ConfigurationsUtils; import org.apache.metron.common.configuration.profiler.ProfileConfig; +import org.apache.metron.common.configuration.profiler.ProfilerConfigurations; +import org.apache.metron.common.zookeeper.configurations.ConfigurationsUpdater; +import org.apache.metron.common.zookeeper.configurations.ProfilerUpdater; +import org.apache.metron.common.zookeeper.configurations.Reloadable; 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.stellar.common.utils.ConversionUtils; import org.apache.metron.stellar.dsl.Context; +import org.apache.metron.zookeeper.SimpleEventListener; +import org.apache.metron.zookeeper.ZKCache; import org.apache.storm.Config; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseWindowedBolt; import org.apache.storm.tuple.Tuple; import org.apache.storm.utils.TupleUtils; +import org.apache.storm.windowing.TupleWindow; import org.json.simple.JSONObject; import org.json.simple.parser.JSONParser; import org.slf4j.Logger; @@ -42,42 +59,76 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import static java.lang.String.format; +import static org.apache.metron.profiler.bolt.ProfileSplitterBolt.ENTITY_TUPLE_FIELD; +import static org.apache.metron.profiler.bolt.ProfileSplitterBolt.MESSAGE_TUPLE_FIELD; +import static org.apache.metron.profiler.bolt.ProfileSplitterBolt.PROFILE_TUPLE_FIELD; +import static org.apache.metron.profiler.bolt.ProfileSplitterBolt.TIMESTAMP_TUPLE_FIELD; /** - * A bolt that is responsible for building a Profile. - * - * This bolt maintains the state required to build a Profile. When the window - * period expires, the data is summarized as a ProfileMeasurement, all state is - * flushed, and the ProfileMeasurement is emitted. + * A Storm bolt that is responsible for building a profile. * + *

This bolt maintains the state required to build a Profile. When the window + * period expires, the data is summarized as a {@link ProfileMeasurement}, all state is + * flushed, and the {@link ProfileMeasurement} is emitted. */ -public class ProfileBuilderBolt extends ConfiguredProfilerBolt { +public class ProfileBuilderBolt extends BaseWindowedBolt implements Reloadable { protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private OutputCollector collector; + /** + * The URL to connect to Zookeeper. + */ + private String zookeeperUrl; + + /** + * The Zookeeper client connection. + */ + protected CuratorFramework zookeeperClient; + + /** + * The Zookeeper cache. + */ + protected ZKCache zookeeperCache; + + /** + * Manages configuration for the Profiler. + */ + private ProfilerConfigurations configurations; + /** * The duration of each profile period in milliseconds. */ private long periodDurationMillis; + /** + * The duration of Storm's event window. + */ + private long windowDurationMillis; + /** * If a message has not been applied to a Profile in this number of milliseconds, * the Profile will be forgotten and its resources will be cleaned up. * - * WARNING: The TTL must be at least greater than the period duration. + *

WARNING: The TTL must be at least greater than the period duration. */ private long profileTimeToLiveMillis; + /** + * The maximum number of {@link MessageRoute} routes that will be maintained by + * this bolt. After this value is exceeded, lesser used routes will be evicted + * from the internal cache. + */ + private long maxNumberOfRoutes; + /** * Distributes messages to the profile builders. */ - private DefaultMessageDistributor messageDistributor; + private MessageDistributor messageDistributor; /** * Parses JSON messages. @@ -85,112 +136,245 @@ public class ProfileBuilderBolt extends ConfiguredProfilerBolt { private transient JSONParser parser; /** - * The measurements produced by a profile can be written to multiple destinations. Each - * destination is handled by a separate `DestinationHandler`. + * Responsible for emitting {@link ProfileMeasurement} values. + * + *

The {@link ProfileMeasurement} values generated by a profile can be written to + * multiple endpoints like HBase or Kafka. Each endpoint is handled by a separate + * {@link ProfileMeasurementEmitter}. */ - private List destinationHandlers; + private List emitters; /** - * @param zookeeperUrl The Zookeeper URL that contains the configuration data. + * Signals when it is time to flush. */ - public ProfileBuilderBolt(String zookeeperUrl) { - super(zookeeperUrl); - this.destinationHandlers = new ArrayList<>(); - } + private FlushSignal flushSignal; - /** - * Defines the frequency at which the bolt will receive tick tuples. Tick tuples are - * used to control how often a profile is flushed. - */ - @Override - public Map getComponentConfiguration() { - // how frequently should the bolt receive tick tuples? - Config conf = new Config(); - conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, TimeUnit.MILLISECONDS.toSeconds(periodDurationMillis)); - return conf; + public ProfileBuilderBolt() { + this.emitters = new ArrayList<>(); } @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { super.prepare(stormConf, context, collector); + if(periodDurationMillis <= 0) { + throw new IllegalArgumentException("expect 'profiler.period.duration' >= 0"); + } + if(profileTimeToLiveMillis <= 0) { + throw new IllegalArgumentException("expect 'profiler.ttl' >= 0"); + } if(profileTimeToLiveMillis < periodDurationMillis) { - throw new IllegalStateException(format( - "invalid configuration: expect profile TTL (%d) to be greater than period duration (%d)", - profileTimeToLiveMillis, - periodDurationMillis)); + throw new IllegalArgumentException("expect 'profiler.ttl' >= 'profiler.period.duration'"); } + if(maxNumberOfRoutes <= 0) { + throw new IllegalArgumentException("expect 'profiler.max.routes.per.bolt' > 0"); + } + if(windowDurationMillis <= 0) { + throw new IllegalArgumentException("expect 'profiler.window.duration' > 0"); + } + if(windowDurationMillis > periodDurationMillis) { + throw new IllegalArgumentException("expect 'profiler.period.duration' >= 'profiler.window.duration'"); + } + if(periodDurationMillis % windowDurationMillis != 0) { + throw new IllegalArgumentException("expect 'profiler.period.duration' % 'profiler.window.duration' == 0"); + } + this.collector = collector; this.parser = new JSONParser(); - this.messageDistributor = new DefaultMessageDistributor(periodDurationMillis, profileTimeToLiveMillis); + this.messageDistributor = new DefaultMessageDistributor(periodDurationMillis, profileTimeToLiveMillis, maxNumberOfRoutes); + this.configurations = new ProfilerConfigurations(); + this.flushSignal = new FixedFrequencyFlushSignal(periodDurationMillis); + setupZookeeper(); + } + + @Override + public void cleanup() { + zookeeperCache.close(); + zookeeperClient.close(); + } + + private void setupZookeeper() { + try { + if (zookeeperClient == null) { + RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3); + zookeeperClient = CuratorFrameworkFactory.newClient(zookeeperUrl, retryPolicy); + } + zookeeperClient.start(); + + // this is temporary to ensure that any validation passes. the individual bolt + // will reinitialize stellar to dynamically pull from zookeeper. + ConfigurationsUtils.setupStellarStatically(zookeeperClient); + if (zookeeperCache == null) { + ConfigurationsUpdater updater = createUpdater(); + SimpleEventListener listener = new SimpleEventListener.Builder() + .with( updater::update, TreeCacheEvent.Type.NODE_ADDED, TreeCacheEvent.Type.NODE_UPDATED) + .with( updater::delete, TreeCacheEvent.Type.NODE_REMOVED) + .build(); + zookeeperCache = new ZKCache.Builder() + .withClient(zookeeperClient) + .withListener(listener) + .withRoot(Constants.ZOOKEEPER_TOPOLOGY_ROOT) + .build(); + updater.forceUpdate(zookeeperClient); + zookeeperCache.start(); + } + } catch (Exception e) { + LOG.error(e.getMessage(), e); + throw new RuntimeException(e); + } + } + + protected ConfigurationsUpdater createUpdater() { + return new ProfilerUpdater(this, this::getConfigurations); + } + + public ProfilerConfigurations getConfigurations() { + return configurations; + } + + @Override + public void reloadCallback(String name, ConfigurationType type) { + // nothing to do } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { - if(destinationHandlers.size() == 0) { + + if(emitters.size() == 0) { throw new IllegalStateException("At least one destination handler must be defined."); } - // each destination will define its own stream - destinationHandlers.forEach(dest -> dest.declareOutputFields(declarer)); + // allow each emitter to define its own stream + emitters.forEach(emitter -> emitter.declareOutputFields(declarer)); + } + + /** + * Defines the frequency at which the bolt will receive tick tuples. Tick tuples are + * used to control how often a profile is flushed. + */ + @Override + public Map getComponentConfiguration() { + + Map conf = super.getComponentConfiguration(); + conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, TimeUnit.MILLISECONDS.toSeconds(profileTimeToLiveMillis)); + return conf; } private Context getStellarContext() { + Map global = getConfigurations().getGlobalConfig(); return new Context.Builder() - .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client) + .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> zookeeperClient) .with(Context.Capabilities.GLOBAL_CONFIG, () -> global) .with(Context.Capabilities.STELLAR_CONFIG, () -> global) .build(); } - /** - * Expect to receive either a tick tuple or a telemetry message that needs applied - * to a profile. - * @param input The tuple. - */ @Override - public void execute(Tuple input) { + public void execute(TupleWindow window) { + + LOG.debug("Tuple window contains {} tuple(s), {} expired, {} new", + CollectionUtils.size(window.get()), + CollectionUtils.size(window.getExpired()), + CollectionUtils.size(window.getNew())); + try { - if(TupleUtils.isTick(input)) { - handleTick(); - } else { - handleMessage(input); + // handle each tuple in the window + for(Tuple tuple : window.get()) { + + if(TupleUtils.isTick(tuple)) { + handleTick(); + + } else { + handleMessage(tuple); + } + } + + // time to flush? + if(flushSignal.isTimeToFlush()) { + flushSignal.reset(); + + // flush the active profiles + List measurements = messageDistributor.flush(); + emitMeasurements(measurements); + + LOG.debug("Flushed active profiles and found {} measurement(s).", measurements.size()); } } catch (Throwable e) { - LOG.error(format("Unexpected failure: message='%s', tuple='%s'", e.getMessage(), input), e); - collector.reportError(e); - } finally { - collector.ack(input); + LOG.error("Unexpected error", e); + collector.reportError(e); } } /** - * Handles a telemetry message - * @param input The tuple. + * Flush all expired profiles when a 'tick' is received. + * + * If a profile has not received a message for an extended period of time then it is + * marked as expired. Periodically we need to flush these expired profiles to ensure + * that their state is not lost. */ - private void handleMessage(Tuple input) throws ExecutionException { - JSONObject message = getField("message", input, JSONObject.class); - ProfileConfig definition = getField("profile", input, ProfileConfig.class); - String entity = getField("entity", input, String.class); - MessageRoute route = new MessageRoute(definition, entity); + private void handleTick() { + + // flush the expired profiles + List measurements = messageDistributor.flushExpired(); + emitMeasurements(measurements); - messageDistributor.distribute(message, route, getStellarContext()); + LOG.debug("Flushed expired profiles and found {} measurement(s).", measurements.size()); } /** - * Handles a tick tuple. + * Handles the processing of a single tuple. + * + * @param input The tuple containing a telemetry message. */ - private void handleTick() { - List measurements = messageDistributor.flush(); + private void handleMessage(Tuple input) { + + // crack open the tuple + JSONObject message = getField(MESSAGE_TUPLE_FIELD, input, JSONObject.class); + ProfileConfig definition = getField(PROFILE_TUPLE_FIELD, input, ProfileConfig.class); + String entity = getField(ENTITY_TUPLE_FIELD, input, String.class); + Long timestamp = getField(TIMESTAMP_TUPLE_FIELD, input, Long.class); + + // keep track of time + flushSignal.update(timestamp); + + // distribute the message + MessageRoute route = new MessageRoute(definition, entity); + messageDistributor.distribute(message, timestamp, route, getStellarContext()); - // forward the measurements to each destination handler - for(ProfileMeasurement m : measurements ) { - destinationHandlers.forEach(handler -> handler.emit(m, collector)); + LOG.debug("Message distributed: profile={}, entity={}, timestamp={}", definition.getProfile(), entity, timestamp); + } + + /** + * Handles the {@code ProfileMeasurement}s that are created when a profile is flushed. + * + * @param measurements The measurements to handle. + */ + private void emitMeasurements(List measurements) { + + // flush each profile + for(ProfileMeasurement measurement: measurements) { + + // allow each 'emitter' to emit the measurement + for (ProfileMeasurementEmitter emitter : emitters) { + emitter.emit(measurement, collector); + + LOG.debug("Measurement emitted; stream={}, profile={}, entity={}, value={}, start={}, end={}, duration={}, period={}", + emitter.getStreamId(), + measurement.getProfileName(), + measurement.getEntity(), + measurement.getProfileValue(), + measurement.getPeriod().getStartTimeMillis(), + measurement.getPeriod().getEndTimeMillis(), + measurement.getPeriod().getDurationMillis(), + measurement.getPeriod().getPeriod()); + } } + + LOG.debug("Emitted {} measurement(s).", measurements.size()); } /** @@ -202,14 +386,27 @@ private void handleTick() { * @param The type of the field value. */ private T getField(String fieldName, Tuple tuple, Class clazz) { + T value = ConversionUtils.convert(tuple.getValueByField(fieldName), clazz); if(value == null) { - throw new IllegalStateException(format("invalid tuple received: missing or invalid field '%s'", fieldName)); + throw new IllegalStateException(format("Invalid tuple: missing or invalid field '%s'", fieldName)); } return value; } + @Override + public BaseWindowedBolt withTumblingWindow(BaseWindowedBolt.Duration duration) { + + // need to capture the window duration for setting the flush count down + this.windowDurationMillis = duration.value; + return super.withTumblingWindow(duration); + } + + public long getPeriodDurationMillis() { + return periodDurationMillis; + } + public ProfileBuilderBolt withPeriodDurationMillis(long periodDurationMillis) { this.periodDurationMillis = periodDurationMillis; return this; @@ -224,16 +421,55 @@ public ProfileBuilderBolt withProfileTimeToLiveMillis(long timeToLiveMillis) { return this; } + public long getWindowDurationMillis() { + return windowDurationMillis; + } + public ProfileBuilderBolt withProfileTimeToLive(int duration, TimeUnit units) { return withProfileTimeToLiveMillis(units.toMillis(duration)); } - public ProfileBuilderBolt withDestinationHandler(DestinationHandler handler) { - this.destinationHandlers.add(handler); + public ProfileBuilderBolt withEmitter(ProfileMeasurementEmitter emitter) { + this.emitters.add(emitter); return this; } - public DefaultMessageDistributor getMessageDistributor() { + public MessageDistributor getMessageDistributor() { return messageDistributor; } + + public ProfileBuilderBolt withZookeeperUrl(String zookeeperUrl) { + this.zookeeperUrl = zookeeperUrl; + return this; + } + + public ProfileBuilderBolt withZookeeperClient(CuratorFramework zookeeperClient) { + this.zookeeperClient = zookeeperClient; + return this; + } + + public ProfileBuilderBolt withZookeeperCache(ZKCache zookeeperCache) { + this.zookeeperCache = zookeeperCache; + return this; + } + + public ProfileBuilderBolt withProfilerConfigurations(ProfilerConfigurations configurations) { + this.configurations = configurations; + return this; + } + + public ProfileBuilderBolt withMaxNumberOfRoutes(long maxNumberOfRoutes) { + this.maxNumberOfRoutes = maxNumberOfRoutes; + return this; + } + + public ProfileBuilderBolt withFlushSignal(FlushSignal flushSignal) { + this.flushSignal = flushSignal; + return this; + } + + public ProfileBuilderBolt withMessageDistributor(MessageDistributor messageDistributor) { + this.messageDistributor = messageDistributor; + return this; + } } diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/DestinationHandler.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileMeasurementEmitter.java similarity index 76% rename from metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/DestinationHandler.java rename to metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileMeasurementEmitter.java index 225778431b..e1fe4e1501 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/DestinationHandler.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/ProfileMeasurementEmitter.java @@ -23,14 +23,17 @@ import org.apache.storm.topology.OutputFieldsDeclarer; /** - * This class handles the mechanics of emitting a profile measurement to a - * stream responsible for writing to a specific destination. + * Handles the mechanics of emitting a {@link ProfileMeasurement} to an output + * stream. * - * The measurements produced by a profile can be written to one or more - * destinations; HBase, Kafka, etc. Each of the destinations leverage a - * separate stream within the topology definition. + *

The Profiler allows the measurements produced by a profile to be written to + * multiple endpoints such as HBase and Kafka. Each of these endpoints will have + * a unique stream that the measurements are written to. + * + *

Implementors of this interface are responsible for defining and managing the + * output stream for a specific endpoint. */ -public interface DestinationHandler { +public interface ProfileMeasurementEmitter { /** * Each destination leverages a unique stream. This method defines 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 a453c6663b..4e62eee135 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 @@ -21,10 +21,14 @@ package org.apache.metron.profiler.bolt; import org.apache.metron.common.bolt.ConfiguredProfilerBolt; +import org.apache.metron.common.configuration.profiler.ProfileConfig; import org.apache.metron.common.configuration.profiler.ProfilerConfig; -import org.apache.metron.profiler.MessageRouter; -import org.apache.metron.profiler.MessageRoute; import org.apache.metron.profiler.DefaultMessageRouter; +import org.apache.metron.profiler.MessageRoute; +import org.apache.metron.profiler.MessageRouter; +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.dsl.Context; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; @@ -42,16 +46,45 @@ import java.lang.invoke.MethodHandles; import java.util.List; import java.util.Map; +import java.util.Optional; /** - * The bolt responsible for filtering incoming messages and directing - * each to the one or more bolts responsible for building a Profile. Each - * message may be needed by 0, 1 or even many Profiles. + * The Storm bolt responsible for filtering incoming messages and directing + * each to the downstream bolts responsible for building a Profile. */ public class ProfileSplitterBolt extends ConfiguredProfilerBolt { protected static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + /** + * The name of the tuple field containing the entity. + * + * This is the result of executing a profile's 'entity' Stellar expression within + * the context of the telemetry message. + */ + protected static final String ENTITY_TUPLE_FIELD = "entity"; + + /** + * The name of the tuple field containing the profile definition. + */ + protected static final String PROFILE_TUPLE_FIELD = "profile"; + + /** + * The name of the tuple field containing the telemetry message. + */ + protected static final String MESSAGE_TUPLE_FIELD = "message"; + + /** + * The name of the tuple field containing the timestamp of the telemetry message. + * + *

If a 'timestampField' has been configured, the timestamp was extracted + * from a field within the telemetry message. This enables event time processing. + * + *

If a 'timestampField' has not been configured, then the Profiler uses + * processing time and the timestamp originated from the system clock. + */ + protected static final String TIMESTAMP_TUPLE_FIELD = "timestamp"; + private OutputCollector collector; /** @@ -62,7 +95,12 @@ public class ProfileSplitterBolt extends ConfiguredProfilerBolt { /** * The router responsible for routing incoming messages. */ - private MessageRouter router; + 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. @@ -77,6 +115,7 @@ 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() { @@ -88,13 +127,26 @@ private Context getStellarContext() { .build(); } + /** + * This bolt consumes telemetry messages and determines if the message is needed + * by any of the profiles. The message is then routed to one or more downstream + * bolts that are responsible for building each profile + * + *

The outgoing tuples are timestamped so that Storm's window and event-time + * processing functionality can recognize the time of each message. + * + *

The timestamp that is attached to each outgoing tuple is what decides if + * the Profiler is operating on processing time or event time. + * + * @param input The tuple. + */ @Override public void execute(Tuple input) { try { doExecute(input); } catch (IllegalArgumentException | ParseException | UnsupportedEncodingException e) { - LOG.error("Unexpected failure: message='{}', tuple='{}'", e.getMessage(), input, e); + LOG.error("Unexpected error", e); collector.reportError(e); } finally { @@ -103,41 +155,85 @@ public void execute(Tuple input) { } private void doExecute(Tuple input) throws ParseException, UnsupportedEncodingException { + // retrieve the input message byte[] data = input.getBinary(0); JSONObject message = (JSONObject) parser.parse(new String(data, "UTF8")); // ensure there is a valid profiler configuration ProfilerConfig config = getProfilerConfig(); - if(config != null) { + if(config != null && config.getProfiles().size() > 0) { + + // what time is it? + Clock clock = clockFactory.createClock(config); + Optional timestamp = clock.currentTimeMillis(message); - // emit a message for each 'route' - List routes = router.route(message, config, getStellarContext()); - for(MessageRoute route : routes) { - collector.emit(input, new Values(route.getEntity(), route.getProfileDefinition(), 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 { - LOG.warn("No Profiler configuration found. Nothing to do."); + LOG.debug("No Profiler configuration found. Nothing to do."); } } + /** + * Route a message based on the Profiler configuration. + * @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) { + + // emit a tuple for each 'route' + List routes = router.route(message, config, getStellarContext()); + for (MessageRoute route : routes) { + + Values values = createValues(message, timestamp, route); + collector.emit(input, values); + } + + LOG.debug("Found {} route(s) for message with timestamp={}", routes.size(), timestamp); + } + /** * Each emitted tuple contains the following fields. *

*

    - *
  1. entity - The name of the entity. The actual result of executing the Stellar expression. - *
  2. profile - The profile definition that the message needs applied to. - *
  3. message - The message containing JSON-formatted data that needs applied to a profile. + *
  4. message - The message containing JSON-formatted data that needs applied to a profile. + *
  5. timestamp - The timestamp of the message. + *
  6. entity - The name of the entity. The actual result of executing the Stellar expression. + *
  7. profile - The profile definition that the message needs applied to. *
*

*/ @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("entity", "profile", "message")); + + // the order here must match 'createValues' + Fields fields = new Fields(MESSAGE_TUPLE_FIELD, TIMESTAMP_TUPLE_FIELD, ENTITY_TUPLE_FIELD, PROFILE_TUPLE_FIELD); + declarer.declare(fields); + } + + /** + * 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) { + + // the order here must match `declareOutputFields` + return new Values(message, timestamp, route.getEntity(), route.getProfileDefinition()); } protected MessageRouter getMessageRouter() { return router; } + + public void setClockFactory(ClockFactory clockFactory) { + this.clockFactory = clockFactory; + } } diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/event-time-test/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/event-time-test/profiler.json new file mode 100644 index 0000000000..9d727a314b --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/config/zookeeper/event-time-test/profiler.json @@ -0,0 +1,12 @@ +{ + "profiles": [ + { + "profile": "event-time-test", + "foreach": "ip_src_addr", + "init": { "counter": "0" }, + "update": { "counter": "counter + 1" }, + "result": "counter" + } + ], + "timestampField": "timestamp" +} \ No newline at end of file diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/FixedFrequencyFlushSignalTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/FixedFrequencyFlushSignalTest.java new file mode 100644 index 0000000000..b8949c5c6f --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/FixedFrequencyFlushSignalTest.java @@ -0,0 +1,71 @@ +/* + * 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.bolt; + +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests the {@code FixedFrequencyFlushSignal} class. + */ +public class FixedFrequencyFlushSignalTest { + + @Test + public void testSignalFlush() { + + FixedFrequencyFlushSignal signal = new FixedFrequencyFlushSignal(1000); + + // not time to flush yet + assertFalse(signal.isTimeToFlush()); + + // advance time + signal.update(5000); + + // not time to flush yet + assertFalse(signal.isTimeToFlush()); + + // advance time + signal.update(7000); + + // time to flush + assertTrue(signal.isTimeToFlush()); + } + + @Test + public void testOutOfOrderTimestamps() { + FixedFrequencyFlushSignal signal = new FixedFrequencyFlushSignal(1000); + + // advance time, out-of-order + signal.update(5000); + signal.update(1000); + signal.update(7000); + signal.update(3000); + + // need to flush @ 5000 + 1000 = 6000. if anything > 6000 (even out-of-order), then it should signal a flush + assertTrue(signal.isTimeToFlush()); + } + + @Test(expected = IllegalArgumentException.class) + public void testNegativeFrequency() { + new FixedFrequencyFlushSignal(-1000); + } +} diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaDestinationHandlerTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java similarity index 95% rename from metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaDestinationHandlerTest.java rename to metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java index c3f2584ec4..b02e377c78 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaDestinationHandlerTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java @@ -39,13 +39,18 @@ import java.util.Map; import java.util.concurrent.TimeUnit; -import static org.mockito.Mockito.*; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; /** * Tests the KafkaDestinationHandler. */ -public class KafkaDestinationHandlerTest { +public class KafkaEmitterTest { /** * { @@ -59,13 +64,13 @@ public class KafkaDestinationHandlerTest { @Multiline private String profileDefinition; - private KafkaDestinationHandler handler; + private KafkaEmitter handler; private ProfileConfig profile; private OutputCollector collector; @Before public void setup() throws Exception { - handler = new KafkaDestinationHandler(); + handler = new KafkaEmitter(); profile = createDefinition(profileDefinition); collector = Mockito.mock(OutputCollector.class); } 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 21d61ab369..78e20e0221 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 @@ -20,35 +20,37 @@ package org.apache.metron.profiler.bolt; -import org.adrianwalker.multilinestring.Multiline; -import org.apache.commons.lang3.StringUtils; import org.apache.metron.common.configuration.profiler.ProfileConfig; -import org.apache.metron.common.utils.JSONUtils; +import org.apache.metron.common.configuration.profiler.ProfilerConfigurations; +import org.apache.metron.profiler.MessageDistributor; import org.apache.metron.profiler.MessageRoute; -import org.apache.metron.profiler.ProfileBuilder; import org.apache.metron.profiler.ProfileMeasurement; -import org.apache.metron.stellar.dsl.Context; +import org.apache.metron.profiler.integration.MessageBuilder; import org.apache.metron.test.bolt.BaseBoltTest; -import org.apache.storm.Constants; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseWindowedBolt; +import org.apache.storm.tuple.Fields; import org.apache.storm.tuple.Tuple; import org.apache.storm.tuple.Values; +import org.apache.storm.windowing.TupleWindow; import org.json.simple.JSONObject; -import org.json.simple.parser.JSONParser; import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; -import static org.apache.metron.stellar.common.utils.ConversionUtils.convert; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -59,284 +61,348 @@ */ public class ProfileBuilderBoltTest extends BaseBoltTest { - /** - * { - * "ip_src_addr": "10.0.0.1", - * "value": "22" - * } - */ - @Multiline - private String inputOne; - private JSONObject messageOne; + private JSONObject message1; + private JSONObject message2; + private ProfileConfig profile1; + private ProfileConfig profile2; + private ProfileMeasurementEmitter emitter; + private ManualFlushSignal flushSignal; - /** - * { - * "ip_src_addr": "10.0.0.2", - * "value": "22" - * } - */ - @Multiline - private String inputTwo; - private JSONObject messageTwo; + @Before + public void setup() throws Exception { + + message1 = new MessageBuilder() + .withField("ip_src_addr", "10.0.0.1") + .withField("value", "22") + .build(); + + message2 = new MessageBuilder() + .withField("ip_src_addr", "10.0.0.2") + .withField("value", "22") + .build(); + + profile1 = new ProfileConfig() + .withProfile("profile1") + .withForeach("ip_src_addr") + .withInit("x", "0") + .withUpdate("x", "x + 1") + .withResult("x"); + + profile2 = new ProfileConfig() + .withProfile("profile2") + .withForeach("ip_src_addr") + .withInit(Collections.singletonMap("x", "0")) + .withUpdate(Collections.singletonMap("x", "x + 1")) + .withResult("x"); + + flushSignal = new ManualFlushSignal(); + flushSignal.setFlushNow(false); + } /** - * { - * "profile": "profileOne", - * "foreach": "ip_src_addr", - * "init": { "x": "0" }, - * "update": { "x": "x + 1" }, - * "result": "x" - * } + * The bolt should extract a message and timestamp from a tuple and + * pass that to a {@code MessageDistributor}. */ - @Multiline - private String profileOne; + @Test + public void testExtractMessage() throws Exception { + ProfileBuilderBolt bolt = createBolt(); - /** - * { - * "profile": "profileTwo", - * "foreach": "ip_src_addr", - * "init": { "x": "0" }, - * "update": { "x": "x + 1" }, - * "result": "x" - * } - */ - @Multiline - private String profileTwo; + // create a mock + MessageDistributor distributor = mock(MessageDistributor.class); + bolt.withMessageDistributor(distributor); - public static Tuple mockTickTuple() { - Tuple tuple = mock(Tuple.class); - when(tuple.getSourceComponent()).thenReturn(Constants.SYSTEM_COMPONENT_ID); - when(tuple.getSourceStreamId()).thenReturn(Constants.SYSTEM_TICK_STREAM_ID); - return tuple; - } + // create a tuple + final long timestamp1 = 100000000L; + Tuple tuple1 = createTuple("entity1", message1, profile1, timestamp1); - @Before - public void setup() throws Exception { - JSONParser parser = new JSONParser(); - messageOne = (JSONObject) parser.parse(inputOne); - messageTwo = (JSONObject) parser.parse(inputTwo); + // execute the bolt + TupleWindow tupleWindow = createWindow(tuple1); + 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()); } + /** - * Creates a profile definition based on a string of JSON. - * @param json The string of JSON. + * If the {@code FlushSignal} tells the bolt to flush, it should flush the {@code MessageDistributor} + * and emit the {@code ProfileMeasurement} values. */ - private ProfileConfig createDefinition(String json) throws IOException { - return JSONUtils.INSTANCE.load(json, ProfileConfig.class); + @Test + public void testEmitWhenFlush() throws Exception { + + ProfileBuilderBolt bolt = createBolt(); + + // create a profile measurement + ProfileMeasurement m = new ProfileMeasurement() + .withEntity("entity1") + .withProfileName("profile1") + .withPeriod(1000, 500, TimeUnit.MILLISECONDS) + .withProfileValue(22); + + // create a mock that returns the profile measurement above + MessageDistributor distributor = mock(MessageDistributor.class); + when(distributor.flush()).thenReturn(Collections.singletonList(m)); + bolt.withMessageDistributor(distributor); + + // signal the bolt to flush + flushSignal.setFlushNow(true); + + // execute the bolt + Tuple tuple1 = createTuple("entity1", message1, profile1, 1000L); + TupleWindow tupleWindow = createWindow(tuple1); + bolt.execute(tupleWindow); + + // a profile measurement should be emitted by the bolt + List measurements = getProfileMeasurements(outputCollector, 1); + assertEquals(1, measurements.size()); + assertEquals(m, measurements.get(0)); } /** - * Create a tuple that will contain the message, the entity name, and profile definition. - * @param entity The entity name - * @param message The telemetry message. - * @param profile The profile definition. + * If the {@code FlushSignal} tells the bolt NOT to flush, nothing should be emitted. */ - private Tuple createTuple(String entity, JSONObject message, ProfileConfig profile) { - Tuple tuple = mock(Tuple.class); - when(tuple.getValueByField(eq("message"))).thenReturn(message); - when(tuple.getValueByField(eq("entity"))).thenReturn(entity); - when(tuple.getValueByField(eq("profile"))).thenReturn(profile); - return tuple; + @Test + public void testDoNotEmitWhenNoFlush() throws Exception { + + ProfileBuilderBolt bolt = createBolt(); + + // create a profile measurement + ProfileMeasurement m = new ProfileMeasurement() + .withEntity("entity1") + .withProfileName("profile1") + .withPeriod(1000, 500, TimeUnit.MILLISECONDS) + .withProfileValue(22); + + // create a mock that returns the profile measurement above + MessageDistributor distributor = mock(MessageDistributor.class); + when(distributor.flush()).thenReturn(Collections.singletonList(m)); + bolt.withMessageDistributor(distributor); + + // no flush signal + flushSignal.setFlushNow(false); + + // execute the bolt + Tuple tuple1 = createTuple("entity1", message1, profile1, 1000L); + TupleWindow tupleWindow = createWindow(tuple1); + bolt.execute(tupleWindow); + + // nothing should have been emitted + getProfileMeasurements(outputCollector, 0); } /** - * Create a ProfileBuilderBolt to test + * A {@link ProfileMeasurement} is built for each profile/entity pair. The measurement should be emitted to each + * destination defined by the profile. By default, a profile uses both Kafka and HBase as destinations. */ - private ProfileBuilderBolt createBolt() throws IOException { + @Test + public void testEmitters() throws Exception { + + // defines the zk configurations accessible from the bolt + ProfilerConfigurations configurations = new ProfilerConfigurations(); + configurations.updateGlobalConfig(Collections.emptyMap()); + + // create the bolt with 3 destinations + ProfileBuilderBolt bolt = (ProfileBuilderBolt) new ProfileBuilderBolt() + .withProfileTimeToLive(30, TimeUnit.MINUTES) + .withPeriodDuration(10, TimeUnit.MINUTES) + .withMaxNumberOfRoutes(Long.MAX_VALUE) + .withZookeeperClient(client) + .withZookeeperCache(cache) + .withEmitter(new TestEmitter("destination1")) + .withEmitter(new TestEmitter("destination2")) + .withEmitter(new TestEmitter("destination3")) + .withProfilerConfigurations(configurations) + .withTumblingWindow(new BaseWindowedBolt.Duration(10, TimeUnit.MINUTES)); + bolt.prepare(new HashMap<>(), topologyContext, outputCollector); - ProfileBuilderBolt bolt = new ProfileBuilderBolt("zookeeperURL"); - bolt.setCuratorFramework(client); - bolt.setZKCache(cache); - bolt.withPeriodDuration(10, TimeUnit.MINUTES); - bolt.withProfileTimeToLive(30, TimeUnit.MINUTES); + // signal the bolt to flush + bolt.withFlushSignal(flushSignal); + flushSignal.setFlushNow(true); - // define the valid destinations for the profiler - bolt.withDestinationHandler(new HBaseDestinationHandler()); - bolt.withDestinationHandler(new KafkaDestinationHandler()); + // execute the bolt + Tuple tuple1 = createTuple("entity", message1, profile1, System.currentTimeMillis()); + TupleWindow window = createWindow(tuple1); + bolt.execute(window); - bolt.prepare(new HashMap<>(), topologyContext, outputCollector); - return bolt; + // validate measurements emitted to each + verify(outputCollector, times(1)).emit(eq("destination1"), any()); + verify(outputCollector, times(1)).emit(eq("destination2"), any()); + verify(outputCollector, times(1)).emit(eq("destination3"), any()); } - /** - * The bolt should create a ProfileBuilder to manage a profile. - */ @Test - public void testCreateProfileBuilder() throws Exception { + public void testFlushExpiredWithTick() throws Exception { ProfileBuilderBolt bolt = createBolt(); - ProfileConfig definition = createDefinition(profileOne); - String entity = (String) messageOne.get("ip_src_addr"); - Tuple tupleOne = createTuple(entity, messageOne, definition); - // execute - send two tuples with different entities - bolt.execute(tupleOne); + // create a mock + MessageDistributor distributor = mock(MessageDistributor.class); + bolt.withMessageDistributor(distributor); + + // tell the bolt to flush on the first window + flushSignal.setFlushNow(true); - // validate - 1 messages applied - MessageRoute route = new MessageRoute(definition, entity); - ProfileBuilder builderOne = bolt.getMessageDistributor().getBuilder(route, Context.EMPTY_CONTEXT()); - assertEquals(1, (int) convert(builderOne.valueOf("x"), Integer.class)); + // execute the bolt; include a tick tuple in the window + Tuple tuple1 = createTuple("entity", message1, profile1, 100000000L); + TupleWindow tupleWindow = createWindow(tuple1, mockTickTuple()); + bolt.execute(tupleWindow); + + // ensure the expired profiles were flushed when the tick tuple was received + verify(distributor).flushExpired(); } - /** - * This test creates two different messages, with different entities that are applied to - * the same profile. The bolt should create separate ProfileBuilder objects to handle each - * profile/entity pair. - */ @Test - public void testCreateProfileBuilderForEachEntity() throws Exception { + public void testFlushExpiredWithNoTick() throws Exception { - // setup ProfileBuilderBolt bolt = createBolt(); - ProfileConfig definition = createDefinition(profileOne); - - // apply a message to the profile - String entityOne = (String) messageOne.get("ip_src_addr"); - Tuple tupleOne = createTuple(entityOne, messageOne, definition); - bolt.execute(tupleOne); - bolt.execute(tupleOne); - - // apply a different message (with different entity) to the same profile - String entityTwo = (String) messageTwo.get("ip_src_addr"); - Tuple tupleTwo = createTuple(entityTwo, messageTwo, definition); - bolt.execute(tupleTwo); - - // validate - 2 messages applied - MessageRoute routeOne = new MessageRoute(definition, entityOne); - ProfileBuilder builderOne = bolt.getMessageDistributor().getBuilder(routeOne, Context.EMPTY_CONTEXT()); - assertTrue(builderOne.isInitialized()); - assertEquals(2, (int) convert(builderOne.valueOf("x"), Integer.class)); - - // validate - 1 message applied - MessageRoute routeTwo = new MessageRoute(definition, entityTwo); - ProfileBuilder builderTwo = bolt.getMessageDistributor().getBuilder(routeTwo, Context.EMPTY_CONTEXT()); - assertTrue(builderTwo.isInitialized()); - assertEquals(1, (int) convert(builderTwo.valueOf("x"), Integer.class)); - - assertNotSame(builderOne, builderTwo); + + // create a mock + MessageDistributor distributor = mock(MessageDistributor.class); + bolt.withMessageDistributor(distributor); + + // tell the bolt to flush on the first window + flushSignal.setFlushNow(true); + + // execute the bolt; NO tick tuple + Tuple tuple1 = createTuple("entity", message1, profile1, 100000000L); + TupleWindow tupleWindow = createWindow(tuple1); + bolt.execute(tupleWindow); + + // there was no tick tuple; the expired profiles should NOT have been flushed + verify(distributor, times(0)).flushExpired(); } /** - * The bolt should create separate ProfileBuilder objects to handle each - * profile/entity pair. + * Creates a mock tick tuple to use for testing. + * @return A mock tick tuple. */ - @Test - public void testCreateProfileBuilderForEachProfile() throws Exception { + private Tuple mockTickTuple() { - // setup - apply one message to different profile definitions - ProfileBuilderBolt bolt = createBolt(); - String entity = (String) messageOne.get("ip_src_addr"); - - // apply a message to the first profile - ProfileConfig definitionOne = createDefinition(profileOne); - Tuple tupleOne = createTuple(entity, messageOne, definitionOne); - bolt.execute(tupleOne); - - // apply the same message to the second profile - ProfileConfig definitionTwo = createDefinition(profileTwo); - Tuple tupleTwo = createTuple(entity, messageOne, definitionTwo); - bolt.execute(tupleTwo); - - // validate - 1 message applied - MessageRoute routeOne = new MessageRoute(definitionOne, entity); - ProfileBuilder builderOne = bolt.getMessageDistributor().getBuilder(routeOne, Context.EMPTY_CONTEXT()); - assertTrue(builderOne.isInitialized()); - assertEquals(1, (int) convert(builderOne.valueOf("x"), Integer.class)); - - // validate - 1 message applied - MessageRoute routeTwo = new MessageRoute(definitionTwo, entity); - ProfileBuilder builderTwo = bolt.getMessageDistributor().getBuilder(routeTwo, Context.EMPTY_CONTEXT()); - assertTrue(builderTwo.isInitialized()); - assertEquals(1, (int) convert(builderTwo.valueOf("x"), Integer.class)); - - assertNotSame(builderOne, builderTwo); + Tuple tuple = mock(Tuple.class); + when(tuple.getSourceComponent()).thenReturn("__system"); + when(tuple.getSourceStreamId()).thenReturn("__tick"); + + return tuple; } /** - * A ProfileMeasurement is build for each profile/entity pair. A measurement for each profile/entity - * pair should be emitted. + * Retrieves the ProfileMeasurement(s) (if any) that have been emitted. + * + * @param collector The Storm output collector. + * @param expected The number of measurements expected. + * @return A list of ProfileMeasurement(s). */ - @Test - public void testEmitMeasurements() throws Exception { - - // setup - ProfileBuilderBolt bolt = createBolt(); - final String entity = (String) messageOne.get("ip_src_addr"); + private List getProfileMeasurements(OutputCollector collector, int expected) { - // apply the message to the first profile - ProfileConfig definitionOne = createDefinition(profileOne); - Tuple tupleOne = createTuple(entity, messageOne, definitionOne); - bolt.execute(tupleOne); + // the 'streamId' is defined by the DestinationHandler being used by the bolt + final String streamId = emitter.getStreamId(); - // apply the same message to the second profile - ProfileConfig definitionTwo = createDefinition(profileTwo); - Tuple tupleTwo = createTuple(entity, messageOne, definitionTwo); - bolt.execute(tupleTwo); + // capture the emitted tuple(s) + ArgumentCaptor argCaptor = ArgumentCaptor.forClass(Values.class); + verify(collector, times(expected)) + .emit(eq(streamId), argCaptor.capture()); - // execute - the tick tuple triggers a flush of the profile - bolt.execute(mockTickTuple()); + // return the profile measurements that were emitted + return argCaptor.getAllValues() + .stream() + .map(val -> (ProfileMeasurement) val.get(0)) + .collect(Collectors.toList()); + } - // capture the ProfileMeasurement that should be emitted - ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); + /** + * Create a tuple that will contain the message, the entity name, and profile definition. + * @param entity The entity name + * @param message The telemetry message. + * @param profile The profile definition. + */ + private Tuple createTuple(String entity, JSONObject message, ProfileConfig profile, long timestamp) { - // validate emitted measurements for hbase - verify(outputCollector, atLeastOnce()).emit(eq("hbase"), arg.capture()); - for (Values value : arg.getAllValues()) { + Tuple tuple = mock(Tuple.class); + when(tuple.getValueByField(eq(ProfileSplitterBolt.MESSAGE_TUPLE_FIELD))).thenReturn(message); + when(tuple.getValueByField(eq(ProfileSplitterBolt.TIMESTAMP_TUPLE_FIELD))).thenReturn(timestamp); + when(tuple.getValueByField(eq(ProfileSplitterBolt.ENTITY_TUPLE_FIELD))).thenReturn(entity); + when(tuple.getValueByField(eq(ProfileSplitterBolt.PROFILE_TUPLE_FIELD))).thenReturn(profile); - ProfileMeasurement measurement = (ProfileMeasurement) value.get(0); - ProfileConfig definition = measurement.getDefinition(); + return tuple; + } - if (StringUtils.equals(definitionTwo.getProfile(), definition.getProfile())) { + /** + * Create a ProfileBuilderBolt to test. + * @return A {@link ProfileBuilderBolt} to test. + */ + private ProfileBuilderBolt createBolt() throws IOException { - // validate measurement emitted for profile two - assertEquals(definitionTwo, definition); - assertEquals(entity, measurement.getEntity()); - assertEquals(definitionTwo.getProfile(), measurement.getProfileName()); - assertEquals(1, (int) convert(measurement.getProfileValue(), Integer.class)); + return createBolt(30, TimeUnit.SECONDS); + } - } else if (StringUtils.equals(definitionOne.getProfile(), definition.getProfile())) { + /** + * Create a ProfileBuilderBolt to test. + * + * @param windowDuration The event window duration. + * @param windowDurationUnits The units of the event window duration. + * @return A {@link ProfileBuilderBolt} to test. + */ + private ProfileBuilderBolt createBolt(int windowDuration, TimeUnit windowDurationUnits) throws IOException { + + // defines the zk configurations accessible from the bolt + ProfilerConfigurations configurations = new ProfilerConfigurations(); + configurations.updateGlobalConfig(Collections.emptyMap()); + + emitter = new HBaseEmitter(); + ProfileBuilderBolt bolt = (ProfileBuilderBolt) new ProfileBuilderBolt() + .withProfileTimeToLive(30, TimeUnit.MINUTES) + .withMaxNumberOfRoutes(Long.MAX_VALUE) + .withZookeeperClient(client) + .withZookeeperCache(cache) + .withEmitter(emitter) + .withProfilerConfigurations(configurations) + .withPeriodDuration(1, TimeUnit.MINUTES) + .withTumblingWindow(new BaseWindowedBolt.Duration(windowDuration, windowDurationUnits)); + bolt.prepare(new HashMap<>(), topologyContext, outputCollector); - // validate measurement emitted for profile one - assertEquals(definitionOne, definition); - assertEquals(entity, measurement.getEntity()); - assertEquals(definitionOne.getProfile(), measurement.getProfileName()); - assertEquals(1, (int) convert(measurement.getProfileValue(), Integer.class)); + // set the flush signal AFTER calling 'prepare' + bolt.withFlushSignal(flushSignal); - } else { - fail(); - } - } + return bolt; } /** - * A ProfileMeasurement is build for each profile/entity pair. The measurement should be emitted to each - * destination defined by the profile. By default, a profile uses both Kafka and HBase as destinations. + * Creates a mock TupleWindow containing multiple tuples. + * @param tuples The tuples to add to the window. */ - @Test - public void testDestinationHandlers() throws Exception { + private TupleWindow createWindow(Tuple... tuples) { - // setup - ProfileBuilderBolt bolt = createBolt(); - ProfileConfig definitionOne = createDefinition(profileOne); + TupleWindow window = mock(TupleWindow.class); + when(window.get()).thenReturn(Arrays.asList(tuples)); + return window; + } - // apply the message to the first profile - final String entity = (String) messageOne.get("ip_src_addr"); - Tuple tupleOne = createTuple(entity, messageOne, definitionOne); - bolt.execute(tupleOne); + /** + * An implementation for testing purposes only. + */ + private class TestEmitter implements ProfileMeasurementEmitter { - // trigger a flush of the profile - bolt.execute(mockTickTuple()); + private String streamId; - // capture the values that should be emitted - ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); + public TestEmitter(String streamId) { + this.streamId = streamId; + } - // validate measurements emitted to HBase - verify(outputCollector, times(1)).emit(eq("hbase"), arg.capture()); - assertTrue(arg.getValue().get(0) instanceof ProfileMeasurement); + @Override + public String getStreamId() { + return streamId; + } - // validate measurements emitted to Kafka - verify(outputCollector, times(1)).emit(eq("kafka"), arg.capture()); - assertTrue(arg.getValue().get(0) instanceof JSONObject); + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declareStream(getStreamId(), new Fields("measurement")); + } + + @Override + public void emit(ProfileMeasurement measurement, OutputCollector collector) { + collector.emit(getStreamId(), new Values(measurement)); + } } } diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileHBaseMapperTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileHBaseMapperTest.java index 17d6827766..04c774c34d 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileHBaseMapperTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/ProfileHBaseMapperTest.java @@ -20,11 +20,11 @@ package org.apache.metron.profiler.bolt; -import org.apache.metron.common.configuration.profiler.ProfileResult; -import org.apache.storm.tuple.Tuple; import org.apache.metron.common.configuration.profiler.ProfileConfig; +import org.apache.metron.common.configuration.profiler.ProfileResult; import org.apache.metron.profiler.ProfileMeasurement; import org.apache.metron.profiler.hbase.RowKeyBuilder; +import org.apache.storm.tuple.Tuple; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -32,10 +32,8 @@ import java.util.Optional; import java.util.concurrent.TimeUnit; -import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; /** 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 beab8d50a9..bf819239ef 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 @@ -21,7 +21,10 @@ package org.apache.metron.profiler.bolt; import org.adrianwalker.multilinestring.Multiline; -import org.apache.metron.stellar.common.DefaultStellarStatefulExecutor; +import org.apache.metron.common.configuration.profiler.ProfileConfig; +import org.apache.metron.common.configuration.profiler.ProfilerConfig; +import org.apache.metron.profiler.clock.FixedClockFactory; +import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.test.bolt.BaseBoltTest; import org.apache.storm.tuple.Tuple; import org.apache.storm.tuple.Values; @@ -31,12 +34,15 @@ import org.junit.Before; import org.junit.Test; -import java.io.IOException; +import java.io.ByteArrayInputStream; +import java.io.InputStream; import java.util.HashMap; import static org.mockito.Matchers.any; -import static org.mockito.Matchers.refEq; -import static org.mockito.Mockito.*; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** * Tests the ProfileSplitterBolt. @@ -47,7 +53,9 @@ public class ProfileSplitterBoltTest extends BaseBoltTest { * { * "ip_src_addr": "10.0.0.1", * "ip_dst_addr": "10.0.0.20", - * "protocol": "HTTP" + * "protocol": "HTTP", + * "timestamp.custom": 2222222222222, + * "timestamp.string": "3333333333333" * } */ @Multiline @@ -68,7 +76,7 @@ public class ProfileSplitterBoltTest extends BaseBoltTest { * } */ @Multiline - private String onlyIfTrue; + private String profileWithOnlyIfTrue; /** * { @@ -85,7 +93,7 @@ public class ProfileSplitterBoltTest extends BaseBoltTest { * } */ @Multiline - private String onlyIfFalse; + private String profileWithOnlyIfFalse; /** * { @@ -101,7 +109,7 @@ public class ProfileSplitterBoltTest extends BaseBoltTest { * } */ @Multiline - private String onlyIfMissing; + private String profileWithOnlyIfMissing; /** * { @@ -118,9 +126,89 @@ public class ProfileSplitterBoltTest extends BaseBoltTest { * } */ @Multiline - private String onlyIfInvalid; + private String profileWithOnlyIfInvalid; + + /** + * { + * "profiles": [ + * { + * "profile": "test", + * "foreach": "ip_src_addr", + * "init": {}, + * "update": {}, + * "result": "2" + * } + * ], + * "timestampField": "timestamp.custom" + * } + */ + @Multiline + private String profileUsingCustomTimestampField; + + /** + * { + * "profiles": [ + * { + * "profile": "test", + * "foreach": "ip_src_addr", + * "init": {}, + * "update": {}, + * "result": "2" + * } + * ], + * "timestampField": "timestamp.missing" + * } + */ + @Multiline + private String profileUsingMissingTimestampField; + + /** + * { + * "profiles": [ + * { + * "profile": "test", + * "foreach": "ip_src_addr", + * "init": {}, + * "update": {}, + * "result": "2" + * } + * ], + * "timestampField": "timestamp.string" + * } + */ + @Multiline + private String profileUsingStringTimestampField; + + /** + * { + * "profiles": [ + * ] + * } + */ + @Multiline + private String noProfilesDefined; + + /** + * { + * "profiles": [ + * { + * "profile": "profile1", + * "foreach": "'global'", + * "result": "1" + * }, + * { + * "profile": "profile2", + * "foreach": "'global'", + * "result": "2" + * } + * ] + * } + */ + @Multiline + private String twoProfilesDefined; private JSONObject message; + private long timestamp = 3333333; @Before public void setup() throws ParseException { @@ -134,17 +222,83 @@ public void setup() throws ParseException { } /** - * Create a ProfileSplitterBolt to test + * Ensure that a tuple with the correct fields is emitted to downstream bolts + * when a profile is defined. */ - private ProfileSplitterBolt createBolt(String profilerConfig) throws IOException { + @Test + public void testEmitTupleWithOneProfile() throws Exception { - ProfileSplitterBolt bolt = new ProfileSplitterBolt("zookeeperURL"); - bolt.setCuratorFramework(client); - bolt.setZKCache(cache); - bolt.getConfigurations().updateProfilerConfig(profilerConfig.getBytes("UTF-8")); - bolt.prepare(new HashMap<>(), topologyContext, outputCollector); + // setup the bolt and execute a tuple + ProfilerConfig config = toProfilerConfig(profileWithOnlyIfTrue); + ProfileSplitterBolt bolt = createBolt(config); + bolt.execute(tuple); - return bolt; + // the expected tuple fields + String expectedEntity = "10.0.0.1"; + ProfileConfig expectedConfig = config.getProfiles().get(0); + Values expected = new Values(message, timestamp, expectedEntity, expectedConfig); + + // a tuple should be emitted for the downstream profile builder + verify(outputCollector, times(1)) + .emit(eq(tuple), eq(expected)); + + // the original tuple should be ack'd + verify(outputCollector, times(1)) + .ack(eq(tuple)); + } + + /** + * If there are two profiles that need the same message, then two tuples should + * be emitted. One tuple for each profile. + */ + @Test + public void testEmitTupleWithTwoProfiles() throws Exception { + + // setup the bolt and execute a tuple + ProfilerConfig config = toProfilerConfig(twoProfilesDefined); + ProfileSplitterBolt bolt = createBolt(config); + bolt.execute(tuple); + + // the expected tuple fields + final String expectedEntity = "global"; + { + // a tuple should be emitted for the first profile + ProfileConfig profile1 = config.getProfiles().get(0); + Values expected = new Values(message, timestamp, expectedEntity, profile1); + verify(outputCollector, times(1)) + .emit(eq(tuple), eq(expected)); + } + { + // a tuple should be emitted for the second profile + ProfileConfig profile2 = config.getProfiles().get(1); + Values expected = new Values(message, timestamp, expectedEntity, profile2); + verify(outputCollector, times(1)) + .emit(eq(tuple), eq(expected)); + } + + // the original tuple should be ack'd + verify(outputCollector, times(1)) + .ack(eq(tuple)); + } + + /** + * No tuples should be emitted, if no profiles are defined. + */ + @Test + public void testNoProfilesDefined() throws Exception { + + // setup the bolt and execute a tuple + ProfilerConfig config = toProfilerConfig(noProfilesDefined); + ProfileSplitterBolt bolt = createBolt(config); + bolt.execute(tuple); + + // no tuple should be emitted + verify(outputCollector, times(0)) + .emit(any(Tuple.class), any()); + + // the original tuple should be ack'd + verify(outputCollector, times(1)) + .ack(eq(tuple)); } /** @@ -154,17 +308,17 @@ private ProfileSplitterBolt createBolt(String profilerConfig) throws IOException @Test public void testOnlyIfTrue() throws Exception { - // setup - ProfileSplitterBolt bolt = createBolt(onlyIfTrue); - - // execute + ProfilerConfig config = toProfilerConfig(profileWithOnlyIfTrue); + ProfileSplitterBolt bolt = createBolt(config); bolt.execute(tuple); // a tuple should be emitted for the downstream profile builder - verify(outputCollector, times(1)).emit(refEq(tuple), any(Values.class)); + verify(outputCollector, times(1)) + .emit(eq(tuple), any(Values.class)); // the original tuple should be ack'd - verify(outputCollector, times(1)).ack(tuple); + verify(outputCollector, times(1)) + .ack(eq(tuple)); } /** @@ -174,17 +328,17 @@ public void testOnlyIfTrue() throws Exception { @Test public void testOnlyIfMissing() throws Exception { - // setup - ProfileSplitterBolt bolt = createBolt(onlyIfMissing); - - // execute + ProfilerConfig config = toProfilerConfig(profileWithOnlyIfMissing); + ProfileSplitterBolt bolt = createBolt(config); bolt.execute(tuple); // a tuple should be emitted for the downstream profile builder - verify(outputCollector, times(1)).emit(refEq(tuple), any(Values.class)); + verify(outputCollector, times(1)) + .emit(eq(tuple), any(Values.class)); // the original tuple should be ack'd - verify(outputCollector, times(1)).ack(tuple); + verify(outputCollector, times(1)) + .ack(eq(tuple)); } /** @@ -194,36 +348,45 @@ public void testOnlyIfMissing() throws Exception { @Test public void testOnlyIfFalse() throws Exception { - // setup - ProfileSplitterBolt bolt = createBolt(onlyIfFalse); - - // execute + ProfilerConfig config = toProfilerConfig(profileWithOnlyIfFalse); + ProfileSplitterBolt bolt = createBolt(config); bolt.execute(tuple); // a tuple should NOT be emitted for the downstream profile builder - verify(outputCollector, times(0)).emit(any(Values.class)); + verify(outputCollector, times(0)) + .emit(any()); // the original tuple should be ack'd - verify(outputCollector, times(1)).ack(tuple); + verify(outputCollector, times(1)) + .ack(eq(tuple)); } /** - * The entity associated with a ProfileMeasurement can be defined using a variable that is resolved - * via Stella. In this case the entity is defined as 'ip_src_addr' which is resolved to - * '10.0.0.1' based on the data contained within the message. + * The entity associated with a profile is defined with a Stellar expression. That expression + * can refer to any field within the message. + * + * In this case the entity is defined as 'ip_src_addr' which is resolved to '10.0.0.1' based on + * the data contained within the message. */ @Test public void testResolveEntityName() throws Exception { - // setup - ProfileSplitterBolt bolt = createBolt(onlyIfTrue); - - // execute + ProfilerConfig config = toProfilerConfig(profileWithOnlyIfTrue); + ProfileSplitterBolt bolt = createBolt(config); bolt.execute(tuple); - // verify - the entity name comes from variable resolution in stella + // expected values String expectedEntity = "10.0.0.1"; - verify(outputCollector, times(1)).emit(any(Tuple.class), refEq(new Values(expectedEntity, onlyIfTrue, message))); + ProfileConfig expectedConfig = config.getProfiles().get(0); + Values expected = new Values(message, timestamp, expectedEntity, expectedConfig); + + // a tuple should be emitted for the downstream profile builder + verify(outputCollector, times(1)) + .emit(eq(tuple), eq(expected)); + + // the original tuple should be ack'd + verify(outputCollector, times(1)) + .ack(eq(tuple)); } /** @@ -232,11 +395,42 @@ public void testResolveEntityName() throws Exception { @Test public void testOnlyIfInvalid() throws Exception { - // setup - ProfileSplitterBolt bolt = createBolt(onlyIfInvalid); + ProfilerConfig config = toProfilerConfig(profileWithOnlyIfInvalid); + ProfileSplitterBolt bolt = createBolt(config); bolt.execute(tuple); // a tuple should NOT be emitted for the downstream profile builder - verify(outputCollector, times(0)).emit(any(Values.class)); + verify(outputCollector, times(0)) + .emit(any(Values.class)); + } + + /** + * Creates a ProfilerConfig based on a string containing JSON. + * + * @param configAsJSON The config as JSON. + * @return The ProfilerConfig. + * @throws Exception + */ + private ProfilerConfig toProfilerConfig(String configAsJSON) throws Exception { + InputStream in = new ByteArrayInputStream(configAsJSON.getBytes("UTF-8")); + return JSONUtils.INSTANCE.load(in, ProfilerConfig.class); } + + /** + * Create a ProfileSplitterBolt to test + */ + private ProfileSplitterBolt createBolt(ProfilerConfig config) throws Exception { + + ProfileSplitterBolt bolt = new ProfileSplitterBolt("zookeeperURL"); + bolt.setCuratorFramework(client); + bolt.setZKCache(cache); + bolt.getConfigurations().updateProfilerConfig(config); + bolt.prepare(new HashMap<>(), topologyContext, outputCollector); + + // set the clock factory AFTER calling prepare to use the fixed clock factory + bolt.setClockFactory(new FixedClockFactory(timestamp)); + + return bolt; + } + } diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/MessageBuilder.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/MessageBuilder.java new file mode 100644 index 0000000000..7e1628ba85 --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/MessageBuilder.java @@ -0,0 +1,75 @@ +/* + * + * 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.integration; + +import org.json.simple.JSONObject; + +import java.util.HashMap; +import java.util.Map; + +/** + * Enables simple creation of telemetry messages for testing. + */ +public class MessageBuilder { + + private Map fields; + + /** + * Create a new {@link MessageBuilder}. + */ + public MessageBuilder() { + this.fields = new HashMap<>(); + } + + /** + * Adds all of the fields from a message to this message. + * + * @param prototype The other message that is treated as a prototype. + * @return A {@link MessageBuilder} + */ + public MessageBuilder withFields(JSONObject prototype) { + prototype.forEach((key, val) -> this.fields.put(key, val)); + return this; + } + + /** + * Adds a field to the message. + * + * @param key The field name. + * @param value The field value. + * @return A {@link MessageBuilder} + */ + public MessageBuilder withField(String key, Object value) { + this.fields.put(key, value); + return this; + } + + /** + * Build the message. + * + *

This should be called after defining all of the message fields. + * + * @return A {@link MessageBuilder}. + */ + public JSONObject build() { + return new JSONObject(fields); + } +} diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java index 0d1b4654be..9d48406ae8 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java @@ -28,15 +28,18 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.metron.common.Constants; import org.apache.metron.common.utils.SerDeUtils; -import org.apache.metron.hbase.mock.MockHTable; import org.apache.metron.hbase.mock.MockHBaseTableProvider; +import org.apache.metron.hbase.mock.MockHTable; import org.apache.metron.integration.BaseIntegrationTest; import org.apache.metron.integration.ComponentRunner; import org.apache.metron.integration.UnableToStartException; import org.apache.metron.integration.components.FluxTopologyComponent; import org.apache.metron.integration.components.KafkaComponent; import org.apache.metron.integration.components.ZKServerComponent; +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.statistics.OnlineStatisticsProvider; import org.junit.After; @@ -49,15 +52,15 @@ import java.io.File; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Properties; -import java.util.stream.Collectors; -import java.util.stream.Stream; +import java.util.concurrent.TimeUnit; import static com.google.code.tempusfugit.temporal.Duration.seconds; import static com.google.code.tempusfugit.temporal.Timeout.timeout; import static com.google.code.tempusfugit.temporal.WaitFor.waitOrTimeout; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; /** * An integration test of the Profiler topology. @@ -105,7 +108,6 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { private static FluxTopologyComponent fluxComponent; private static KafkaComponent kafkaComponent; private static ConfigUploadComponent configUploadComponent; - private static List input; private static ComponentRunner runner; private static MockHTable profilerTable; @@ -114,7 +116,13 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { private static final double epsilon = 0.001; private static final String inputTopic = Constants.INDEXING_TOPIC; private static final String outputTopic = "profiles"; + private static final int saltDivisor = 10; + private static final long eventTimeLagMillis = TimeUnit.SECONDS.toMillis(5); + private static final long windowDurationMillis = TimeUnit.SECONDS.toMillis(5); + private static final long periodDurationMillis = TimeUnit.SECONDS.toMillis(15); + private static final long profileTimeToLiveMillis = TimeUnit.SECONDS.toMillis(20); + private static final long maxRoutesPerBolt = 100000; /** * Tests the first example contained within the README. @@ -122,22 +130,25 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { @Test public void testExample1() throws Exception { - update(TEST_RESOURCES + "/config/zookeeper/readme-example-1"); + uploadConfig(TEST_RESOURCES + "/config/zookeeper/readme-example-1"); // start the topology and write test messages to kafka fluxComponent.submitTopology(); - kafkaComponent.writeMessages(inputTopic, input); + kafkaComponent.writeMessages(inputTopic, message1, message1, message1); + kafkaComponent.writeMessages(inputTopic, message2, message2, message2); + kafkaComponent.writeMessages(inputTopic, message3, message3, message3); // verify - ensure the profile is being persisted waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(90))); + timeout(seconds(180))); // verify - only 10.0.0.2 sends 'HTTP', thus there should be only 1 value - List actuals = read(profilerTable.getPutLog(), columnFamily, columnBuilder.getColumnQualifier("value"), Double.class); + List actuals = read(profilerTable.getPutLog(), columnFamily, + columnBuilder.getColumnQualifier("value"), Double.class); - // verify - there are 5 'HTTP' each with 390 bytes + // verify - there are 3 'HTTP' each with 390 bytes Assert.assertTrue(actuals.stream().anyMatch(val -> - MathUtils.equals(390.0 * 5, val, epsilon) + MathUtils.equals(390.0 * 3, val, epsilon) )); } @@ -147,11 +158,13 @@ public void testExample1() throws Exception { @Test public void testExample2() throws Exception { - update(TEST_RESOURCES + "/config/zookeeper/readme-example-2"); + uploadConfig(TEST_RESOURCES + "/config/zookeeper/readme-example-2"); // start the topology and write test messages to kafka fluxComponent.submitTopology(); - kafkaComponent.writeMessages(inputTopic, input); + kafkaComponent.writeMessages(inputTopic, message1, message1, message1); + kafkaComponent.writeMessages(inputTopic, message2, message2, message2); + kafkaComponent.writeMessages(inputTopic, message3, message3, message3); // expect 2 values written by the profile; one for 10.0.0.2 and another for 10.0.0.3 final int expected = 2; @@ -161,16 +174,17 @@ public void testExample2() throws Exception { timeout(seconds(90))); // verify - expect 2 results as 2 hosts involved; 10.0.0.2 sends 'HTTP' and 10.0.0.3 send 'DNS' - List actuals = read(profilerTable.getPutLog(), columnFamily, columnBuilder.getColumnQualifier("value"), Double.class); + List actuals = read(profilerTable.getPutLog(), columnFamily, + columnBuilder.getColumnQualifier("value"), Double.class); - // verify - 10.0.0.3 -> 1/6 - Assert.assertTrue( "Could not find a value near 1/6. Actual values read are are: " + Joiner.on(",").join(actuals) - , actuals.stream().anyMatch(val -> MathUtils.equals(val, 1.0/6.0, epsilon) + // verify - 10.0.0.3 -> 1/4 + Assert.assertTrue( "Could not find a value near 1/4. Actual values read are are: " + Joiner.on(",").join(actuals), + actuals.stream().anyMatch(val -> MathUtils.equals(val, 1.0/4.0, epsilon) )); - // verify - 10.0.0.2 -> 6/1 - Assert.assertTrue("Could not find a value near 6. Actual values read are are: " + Joiner.on(",").join(actuals) - ,actuals.stream().anyMatch(val -> MathUtils.equals(val, 6.0/1.0, epsilon) + // verify - 10.0.0.2 -> 4/1 + Assert.assertTrue("Could not find a value near 4. Actual values read are are: " + Joiner.on(",").join(actuals), + actuals.stream().anyMatch(val -> MathUtils.equals(val, 4.0/1.0, epsilon) )); } @@ -180,22 +194,25 @@ public void testExample2() throws Exception { @Test public void testExample3() throws Exception { - update(TEST_RESOURCES + "/config/zookeeper/readme-example-3"); + uploadConfig(TEST_RESOURCES + "/config/zookeeper/readme-example-3"); // start the topology and write test messages to kafka fluxComponent.submitTopology(); - kafkaComponent.writeMessages(inputTopic, input); + kafkaComponent.writeMessages(inputTopic, message1, message1, message1); + kafkaComponent.writeMessages(inputTopic, message2, message2, message2); + kafkaComponent.writeMessages(inputTopic, message3, message3, message3); // verify - ensure the profile is being persisted waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, timeout(seconds(90))); // verify - only 10.0.0.2 sends 'HTTP', thus there should be only 1 value - List actuals = read(profilerTable.getPutLog(), columnFamily, columnBuilder.getColumnQualifier("value"), Double.class); + List actuals = read(profilerTable.getPutLog(), columnFamily, + columnBuilder.getColumnQualifier("value"), Double.class); // verify - there are 5 'HTTP' messages each with a length of 20, thus the average should be 20 - Assert.assertTrue("Could not find a value near 20. Actual values read are are: " + Joiner.on(",").join(actuals) - , actuals.stream().anyMatch(val -> MathUtils.equals(val, 20.0, epsilon) + Assert.assertTrue("Could not find a value near 20. Actual values read are are: " + Joiner.on(",").join(actuals), + actuals.stream().anyMatch(val -> MathUtils.equals(val, 20.0, epsilon) )); } @@ -205,11 +222,13 @@ public void testExample3() throws Exception { @Test public void testExample4() throws Exception { - update(TEST_RESOURCES + "/config/zookeeper/readme-example-4"); + uploadConfig(TEST_RESOURCES + "/config/zookeeper/readme-example-4"); // start the topology and write test messages to kafka fluxComponent.submitTopology(); - kafkaComponent.writeMessages(inputTopic, input); + kafkaComponent.writeMessages(inputTopic, message1, message1, message1); + kafkaComponent.writeMessages(inputTopic, message2, message2, message2); + kafkaComponent.writeMessages(inputTopic, message3, message3, message3); // verify - ensure the profile is being persisted waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, @@ -220,34 +239,109 @@ public void testExample4() throws Exception { List actuals = read(profilerTable.getPutLog(), columnFamily, column, OnlineStatisticsProvider.class); // verify - there are 5 'HTTP' messages each with a length of 20, thus the average should be 20 - Assert.assertTrue("Could not find a value near 20. Actual values read are are: " + Joiner.on(",").join(actuals) - , actuals.stream().anyMatch(val -> MathUtils.equals(val.getMean(), 20.0, epsilon) + Assert.assertTrue("Could not find a value near 20. Actual values read are are: " + Joiner.on(",").join(actuals), + actuals.stream().anyMatch(val -> MathUtils.equals(val.getMean(), 20.0, epsilon) )); } @Test public void testPercentiles() throws Exception { - update(TEST_RESOURCES + "/config/zookeeper/percentiles"); + uploadConfig(TEST_RESOURCES + "/config/zookeeper/percentiles"); + + // start the topology and write test messages to kafka + fluxComponent.submitTopology(); + kafkaComponent.writeMessages(inputTopic, message1, message1, message1); + kafkaComponent.writeMessages(inputTopic, message2, message2, message2); + kafkaComponent.writeMessages(inputTopic, message3, message3, message3); + + // verify - ensure the profile is being persisted + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, + timeout(seconds(90))); + + List actuals = read(profilerTable.getPutLog(), columnFamily, + columnBuilder.getColumnQualifier("value"), Double.class); + // verify - the 70th percentile of x3, 20s = 20.0 + Assert.assertTrue("Could not find a value near 20. Actual values read are are: " + Joiner.on(",").join(actuals), + actuals.stream().anyMatch(val -> MathUtils.equals(val, 20.0, epsilon))); + } + + /** + * The Profiler can optionally perform event time processing. With event time processing, + * the Profiler uses timestamps contained in the source telemetry. + * + *

Defining a 'timestampField' within the Profiler configuration tells the Profiler + * from which field the timestamp should be extracted. + */ + @Test + public void testEventTimeProcessing() throws Exception { + + // constants used for the test + final long startAt = 10; + final String entity = "10.0.0.1"; + final String profileName = "event-time-test"; + + // create some messages that contain a timestamp - a really old timestamp; close to 1970 + String message1 = new MessageBuilder() + .withField("ip_src_addr", entity) + .withField("timestamp", startAt) + .build() + .toJSONString(); + + String message2 = new MessageBuilder() + .withField("ip_src_addr", entity) + .withField("timestamp", startAt + 100) + .build() + .toJSONString(); + + uploadConfig(TEST_RESOURCES + "/config/zookeeper/event-time-test"); // start the topology and write test messages to kafka fluxComponent.submitTopology(); - kafkaComponent.writeMessages(inputTopic, input); + kafkaComponent.writeMessages(inputTopic, message1, message2); // verify - ensure the profile is being persisted waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, timeout(seconds(90))); - List actuals = read(profilerTable.getPutLog(), columnFamily, columnBuilder.getColumnQualifier("value"), Double.class); + List puts = profilerTable.getPutLog(); + assertEquals(1, puts.size()); + + // inspect the row key to ensure the profiler used event time correctly. the timestamp + // embedded in the row key should match those in the source telemetry + byte[] expectedRowKey = generateExpectedRowKey(profileName, entity, startAt); + byte[] actualRowKey = puts.get(0).getRow(); + String msg = String.format("expected '%s', got '%s'", + new String(expectedRowKey, "UTF-8"), + new String(actualRowKey, "UTF-8")); + assertArrayEquals(msg, expectedRowKey, actualRowKey); + } - // verify - the 70th percentile of 5 x 20s = 20.0 - Assert.assertTrue("Could not find a value near 20. Actual values read are are: " + Joiner.on(",").join(actuals) - , actuals.stream().anyMatch(val -> MathUtils.equals(val, 20.0, epsilon))); + /** + * Generates the expected row key. + * + * @param profileName The name of the profile. + * @param entity The entity. + * @param whenMillis A timestamp in epoch milliseconds. + * @return A row key. + */ + private byte[] generateExpectedRowKey(String profileName, String entity, long whenMillis) { + + // only the profile name, entity, and period are used to generate the row key + ProfileMeasurement measurement = new ProfileMeasurement() + .withProfileName(profileName) + .withEntity(entity) + .withPeriod(whenMillis, periodDurationMillis, TimeUnit.MILLISECONDS); + + // build the row key + RowKeyBuilder rowKeyBuilder = new SaltyRowKeyBuilder(saltDivisor, periodDurationMillis, TimeUnit.MILLISECONDS); + return rowKeyBuilder.rowKey(measurement); } /** * Reads a value written by the Profiler. + * * @param family The column family. * @param qualifier The column qualifier. * @param clazz The expected type of the value. @@ -258,7 +352,8 @@ private List read(List puts, String family, byte[] qualifier, Class< List results = new ArrayList<>(); for(Put put: puts) { - for(Cell cell: put.get(Bytes.toBytes(family), qualifier)) { + List cells = put.get(Bytes.toBytes(family), qualifier); + for(Cell cell : cells) { T value = SerDeUtils.fromBytes(cell.getValue(), clazz); results.add(value); } @@ -271,39 +366,41 @@ private List read(List puts, String family, byte[] qualifier, Class< public static void setupBeforeClass() throws UnableToStartException { columnBuilder = new ValueOnlyColumnBuilder(columnFamily); - List inputNew = Stream.of(message1, message2, message3) - .map(m -> Collections.nCopies(5, m)) - .flatMap(l -> l.stream()) - .collect(Collectors.toList()); - - // create input messages for the profiler to consume - input = Stream.of(message1, message2, message3) - .map(Bytes::toBytes) - .map(m -> Collections.nCopies(5, m)) - .flatMap(l -> l.stream()) - .collect(Collectors.toList()); - // storm topology properties final Properties topologyProperties = new Properties() {{ - setProperty("kafka.start", "UNCOMMITTED_EARLIEST"); + + // storm settings setProperty("profiler.workers", "1"); setProperty("profiler.executors", "0"); + setProperty("storm.auto.credentials", "[]"); + setProperty("topology.auto-credentials", "[]"); + setProperty("topology.message.timeout.secs", "60"); + setProperty("topology.max.spout.pending", "100000"); + + // kafka settings setProperty("profiler.input.topic", inputTopic); setProperty("profiler.output.topic", outputTopic); - setProperty("profiler.period.duration", "20"); - setProperty("profiler.period.duration.units", "SECONDS"); - setProperty("profiler.ttl", "30"); - setProperty("profiler.ttl.units", "MINUTES"); - setProperty("profiler.hbase.salt.divisor", "10"); + setProperty("kafka.start", "UNCOMMITTED_EARLIEST"); + setProperty("kafka.security.protocol", "PLAINTEXT"); + + // hbase settings + setProperty("profiler.hbase.salt.divisor", Integer.toString(saltDivisor)); setProperty("profiler.hbase.table", tableName); setProperty("profiler.hbase.column.family", columnFamily); setProperty("profiler.hbase.batch", "10"); setProperty("profiler.hbase.flush.interval.seconds", "1"); - setProperty("profiler.profile.ttl", "20"); setProperty("hbase.provider.impl", "" + MockHBaseTableProvider.class.getName()); - setProperty("storm.auto.credentials", "[]"); - setProperty("kafka.security.protocol", "PLAINTEXT"); - setProperty("topology.auto-credentials", "[]"); + + // profile settings + setProperty("profiler.period.duration", Long.toString(periodDurationMillis)); + setProperty("profiler.period.duration.units", "MILLISECONDS"); + setProperty("profiler.ttl", Long.toString(profileTimeToLiveMillis)); + setProperty("profiler.ttl.units", "MILLISECONDS"); + setProperty("profiler.window.duration", Long.toString(windowDurationMillis)); + setProperty("profiler.window.duration.units", "MILLISECONDS"); + setProperty("profiler.event.time.lag", Long.toString(eventTimeLagMillis)); + setProperty("profiler.event.time.lag.units", "MILLISECONDS"); + setProperty("profiler.max.routes.per.bolt", Long.toString(maxRoutesPerBolt)); }}; // create the mock table @@ -311,7 +408,7 @@ public static void setupBeforeClass() throws UnableToStartException { zkComponent = getZKServerComponent(topologyProperties); - // create the input topic + // create the input and output topics kafkaComponent = getKafkaComponent(topologyProperties, Arrays.asList( new KafkaComponent.Topic(inputTopic, 1), new KafkaComponent.Topic(outputTopic, 1))); @@ -340,12 +437,6 @@ public static void setupBeforeClass() throws UnableToStartException { runner.start(); } - public void update(String path) throws Exception { - configUploadComponent.withGlobalConfiguration(path) - .withProfilerConfiguration(path); - configUploadComponent.update(); - } - @AfterClass public static void tearDownAfterClass() throws Exception { MockHBaseTableProvider.clear(); @@ -368,4 +459,16 @@ public void tearDown() throws Exception { runner.reset(); } } -} \ No newline at end of file + + /** + * Uploads config values to Zookeeper. + * @param path The path on the local filesystem to the config values. + * @throws Exception + */ + public void uploadConfig(String path) throws Exception { + configUploadComponent + .withGlobalConfiguration(path) + .withProfilerConfiguration(path) + .update(); + } +} diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-profiler-env.xml b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-profiler-env.xml index c7f6ce25bf..8546b565ad 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-profiler-env.xml +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-profiler-env.xml @@ -57,8 +57,32 @@ value-list - DAYS + HOURS + + MINUTES + + + SECONDS + + + 1 + + + + profiler_window_duration + 30 + The duration of each profile window. This value should be defined along with profiler.window.duration.units + Window Duration + + + profiler_window_units + SECONDS + The units used to specify the profiler.window.duration. This value should be defined along with profiler.window.duration. + Window Units + + value-list + HOURS @@ -71,7 +95,6 @@ 1 - profiler_ttl @@ -104,8 +127,54 @@ 1 - - + + + profiler_window_lag + 1 + The maximum time lag for timestamps. Timestamps cannot arrive out-of-order by more than this amount. + Window Time Lag + + + profiler_window_lag_units + MINUTES + The units used to specify the Event Time Lag. + Window Lag Units + + value-list + + + HOURS + + + MINUTES + + + SECONDS + + + 1 + + + + profiler_topology_message_timeout_secs + The maximum amount of time a message has to complete before it is considered failed. + Profiler Topology Message Timeout + 900 + + + profiler_topology_max_spout_pending + Profiler Topology Spout Max Pending Tuples + Spout Max Pending Tuples + + + true + + + + profiler_max_routes_per_bolt + 100000 + The max number of routes allowed per bolt. The number of routes increases as the number of profiles and entities increases. + Max Routes Per Bolt profiler_hbase_table diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py index ceb9e4e370..ccce022842 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py @@ -287,6 +287,8 @@ profiler_kafka_start = config['configurations']['metron-profiler-env']['profiler_kafka_start'] profiler_period_duration = config['configurations']['metron-profiler-env']['profiler_period_duration'] profiler_period_units = config['configurations']['metron-profiler-env']['profiler_period_units'] +profiler_window_duration = config['configurations']['metron-profiler-env']['profiler_window_duration'] +profiler_window_units = config['configurations']['metron-profiler-env']['profiler_window_units'] profiler_ttl = config['configurations']['metron-profiler-env']['profiler_ttl'] profiler_ttl_units = config['configurations']['metron-profiler-env']['profiler_ttl_units'] profiler_hbase_batch = config['configurations']['metron-profiler-env']['profiler_hbase_batch'] @@ -302,6 +304,11 @@ if not len(profiler_topology_worker_childopts) == 0: profiler_topology_worker_childopts += ' ' profiler_topology_worker_childopts += config['configurations']['metron-profiler-env']['profiler_topology_worker_childopts'] +profiler_max_routes_per_bolt=config['configurations']['metron-profiler-env']['profiler_max_routes_per_bolt'] +profiler_window_lag=config['configurations']['metron-profiler-env']['profiler_window_lag'] +profiler_window_lag_units=config['configurations']['metron-profiler-env']['profiler_window_lag_units'] +profiler_topology_message_timeout_secs=config['configurations']['metron-profiler-env']['profiler_topology_message_timeout_secs'] +profiler_topology_max_spout_pending=config['configurations']['metron-profiler-env']['profiler_topology_max_spout_pending'] # Indexing ra_indexing_kafka_start = config['configurations']['metron-indexing-env']['ra_indexing_kafka_start'] diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/profiler.properties.j2 b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/profiler.properties.j2 index 06fd2097cc..1fd3019b0c 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/profiler.properties.j2 +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/profiler.properties.j2 @@ -22,6 +22,10 @@ topology.worker.childopts={{profiler_topology_worker_childopts}} topology.auto-credentials={{topology_auto_credentials}} +profiler.workers={{profiler_topology_workers}} +profiler.executors={{profiler_acker_executors}} +topology.message.timeout.secs={{profiler_topology_message_timeout_secs}} +topology.max.spout.pending={{profiler_topology_max_spout_pending}} ##### Profiler ##### @@ -29,10 +33,16 @@ profiler.input.topic={{enrichment_output_topic}} profiler.output.topic={{enrichment_input_topic}} profiler.period.duration={{profiler_period_duration}} profiler.period.duration.units={{profiler_period_units}} -profiler.workers={{profiler_topology_workers}} -profiler.executors={{profiler_acker_executors}} +profiler.window.duration={{profiler_window_duration}} +profiler.window.duration.units={{profiler_window_units}} profiler.ttl={{profiler_ttl}} profiler.ttl.units={{profiler_ttl_units}} +profiler.event.time.lag={{profiler_event_time_lag}} +profiler.event.time.lag.units={{profiler_event_time_lag_units}} +profiler.max.routes.per.bolt={{profiler_max_routes_per_bolt}} + +##### HBase ##### + profiler.hbase.salt.divisor=1000 profiler.hbase.table={{profiler_hbase_table}} profiler.hbase.column.family={{profiler_hbase_cf}} @@ -43,6 +53,5 @@ profiler.hbase.flush.interval.seconds={{profiler_hbase_flush_interval}} kafka.zk={{zookeeper_quorum}} kafka.broker={{kafka_brokers}} -# One of EARLIEST, LATEST, UNCOMMITTED_EARLIEST, UNCOMMITTED_LATEST kafka.start={{profiler_kafka_start}} kafka.security.protocol={{kafka_security_protocol}} diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/themes/metron_theme.json b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/themes/metron_theme.json index cef9a3bfae..234b551c2e 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/themes/metron_theme.json +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/themes/metron_theme.json @@ -220,9 +220,28 @@ "tab-columns": "1", "tab-rows": "3", "sections": [ + { + "name": "section-profiler-setup", + "row-index": "0", + "column-index": "0", + "row-span": "1", + "column-span": "1", + "section-columns": "1", + "section-rows": "1", + "subsections": [ + { + "name": "subsection-profiler-setup", + "display-name": "Profiler Setup", + "row-index": "0", + "column-index": "0", + "row-span": "1", + "column-span": "1" + } + ] + }, { "name": "section-profiler-kafka", - "row-index": "0", + "row-index": "1", "column-index": "0", "row-span": "1", "column-span": "1", @@ -240,8 +259,8 @@ ] }, { - "name": "section-profiler-setup", - "row-index": "1", + "name": "section-profiler-storm", + "row-index": "2", "column-index": "0", "row-span": "1", "column-span": "1", @@ -249,8 +268,8 @@ "section-rows": "1", "subsections": [ { - "name": "subsection-profiler-setup", - "display-name": "Profiler Setup", + "name": "subsection-profiler-storm", + "display-name": "Storm", "row-index": "0", "column-index": "0", "row-span": "1", @@ -259,8 +278,8 @@ ] }, { - "name": "section-profiler-storm", - "row-index": "2", + "name": "section-profiler-hbase", + "row-index": "3", "column-index": "0", "row-span": "1", "column-span": "1", @@ -268,8 +287,8 @@ "section-rows": "1", "subsections": [ { - "name": "subsection-profiler-storm", - "display-name": "Storm", + "name": "subsection-profiler-hbase", + "display-name": "HBase", "row-index": "0", "column-index": "0", "row-span": "1", @@ -568,7 +587,6 @@ "config": "metron-indexing-env/bolt_hdfs_rotation_policy_count", "subsection-name": "subsection-indexing-hdfs" }, - { "config": "metron-profiler-env/profiler_kafka_start", "subsection-name": "subsection-profiler-kafka" @@ -581,6 +599,14 @@ "config": "metron-profiler-env/profiler_period_units", "subsection-name": "subsection-profiler-setup" }, + { + "config": "metron-profiler-env/profiler_window_duration", + "subsection-name": "subsection-profiler-setup" + }, + { + "config": "metron-profiler-env/profiler_window_units", + "subsection-name": "subsection-profiler-setup" + }, { "config": "metron-profiler-env/profiler_ttl", "subsection-name": "subsection-profiler-setup" @@ -590,20 +616,32 @@ "subsection-name": "subsection-profiler-setup" }, { - "config": "metron-profiler-env/profiler_hbase_table", + "config": "metron-profiler-env/profiler_window_lag", "subsection-name": "subsection-profiler-setup" }, { - "config": "metron-profiler-env/profiler_hbase_cf", + "config": "metron-profiler-env/profiler_window_lag_units", "subsection-name": "subsection-profiler-setup" }, { - "config": "metron-profiler-env/profiler_hbase_batch", + "config": "metron-profiler-env/profiler_max_routes_per_bolt", "subsection-name": "subsection-profiler-setup" }, + { + "config": "metron-profiler-env/profiler_hbase_table", + "subsection-name": "subsection-profiler-hbase" + }, + { + "config": "metron-profiler-env/profiler_hbase_cf", + "subsection-name": "subsection-profiler-hbase" + }, + { + "config": "metron-profiler-env/profiler_hbase_batch", + "subsection-name": "subsection-profiler-hbase" + }, { "config": "metron-profiler-env/profiler_hbase_flush_interval", - "subsection-name": "subsection-profiler-setup" + "subsection-name": "subsection-profiler-hbase" }, { "config": "metron-profiler-env/profiler_topology_worker_childopts", @@ -617,6 +655,14 @@ "config": "metron-profiler-env/profiler_acker_executors", "subsection-name": "subsection-profiler-storm" }, + { + "config": "metron-profiler-env/profiler_topology_message_timeout_secs", + "subsection-name": "subsection-profiler-storm" + }, + { + "config": "metron-profiler-env/profiler_topology_max_spout_pending", + "subsection-name": "subsection-profiler-storm" + }, { "config": "metron-rest-env/metron_rest_port", "subsection-name": "subsection-rest" @@ -905,7 +951,6 @@ "type": "text-field" } }, - { "config": "metron-indexing-env/batch_indexing_acker_executors", "widget": { @@ -1003,6 +1048,18 @@ "type": "combo" } }, + { + "config": "metron-profiler-env/profiler_window_duration", + "widget": { + "type": "text-field" + } + }, + { + "config": "metron-profiler-env/profiler_window_units", + "widget": { + "type": "combo" + } + }, { "config": "metron-profiler-env/profiler_ttl", "widget": { @@ -1015,6 +1072,24 @@ "type": "combo" } }, + { + "config": "metron-profiler-env/profiler_max_routes_per_bolt", + "widget": { + "type": "text-field" + } + }, + { + "config": "metron-profiler-env/profiler_window_lag", + "widget": { + "type": "text-field" + } + }, + { + "config": "metron-profiler-env/profiler_window_lag_units", + "widget": { + "type": "combo" + } + }, { "config": "metron-profiler-env/profiler_hbase_table", "widget": { @@ -1057,7 +1132,18 @@ "type": "text-field" } }, - + { + "config": "metron-profiler-env/profiler_topology_max_spout_pending", + "widget": { + "type": "text-field" + } + }, + { + "config": "metron-profiler-env/profiler_topology_message_timeout_secs", + "widget": { + "type": "text-field" + } + }, { "config": "metron-rest-env/metron_rest_port", "widget": { diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java index 06c82d26c7..6205fbf377 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java @@ -89,6 +89,9 @@ public class ProfileConfig implements Serializable { */ private Long expires; + public ProfileConfig() { + } + /** * A profile definition requires at the very least the profile name, the foreach, and result * expressions. @@ -114,6 +117,11 @@ public void setProfile(String profile) { this.profile = profile; } + public ProfileConfig withProfile(String profile) { + this.profile = profile; + return this; + } + public String getForeach() { return foreach; } @@ -122,6 +130,11 @@ public void setForeach(String foreach) { this.foreach = foreach; } + public ProfileConfig withForeach(String foreach) { + this.foreach = foreach; + return this; + } + public String getOnlyif() { return onlyif; } @@ -130,6 +143,11 @@ public void setOnlyif(String onlyif) { this.onlyif = onlyif; } + public ProfileConfig withOnlyif(String onlyif) { + this.onlyif = onlyif; + return this; + } + public Map getInit() { return init; } @@ -138,6 +156,16 @@ public void setInit(Map init) { this.init = init; } + public ProfileConfig withInit(Map init) { + this.init.putAll(init); + return this; + } + + public ProfileConfig withInit(String var, String expression) { + this.init.put(var, expression); + return this; + } + public Map getUpdate() { return update; } @@ -146,6 +174,16 @@ public void setUpdate(Map update) { this.update = update; } + public ProfileConfig withUpdate(Map update) { + this.update.putAll(update); + return this; + } + + public ProfileConfig withUpdate(String var, String expression) { + this.update.put(var, expression); + return this; + } + public List getGroupBy() { return groupBy; } @@ -154,6 +192,11 @@ public void setGroupBy(List groupBy) { this.groupBy = groupBy; } + public ProfileConfig withGroupBy(List groupBy) { + this.groupBy = groupBy; + return this; + } + public ProfileResult getResult() { return result; } @@ -162,6 +205,11 @@ public void setResult(ProfileResult result) { this.result = result; } + public ProfileConfig withResult(String profileExpression) { + this.result = new ProfileResult(profileExpression); + return this; + } + public Long getExpires() { return expires; } @@ -170,6 +218,11 @@ public void setExpires(Long expiresDays) { this.expires = expiresDays; } + public ProfileConfig withExpires(Long expiresDays) { + this.expires = TimeUnit.DAYS.toMillis(expiresDays); + return this; + } + @Override public boolean equals(Object o) { if (this == o) return true; diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfilerConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfilerConfig.java index e7c081a0a2..0bdb7e2ffc 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfilerConfig.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfilerConfig.java @@ -20,9 +20,10 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.List; +import java.util.Optional; /** - * The definition for entire Profiler, which may contain many Profile definitions. + * The configuration object for the Profiler, which may contain many Profile definitions. */ public class ProfilerConfig implements Serializable { @@ -31,6 +32,20 @@ public class ProfilerConfig implements Serializable { */ private List profiles = new ArrayList<>(); + /** + * The name of a field containing the timestamp that is used to + * generate profiles. + * + *

By default, the processing time of the Profiler is used rather + * than event time; a value contained within the message itself. + * + *

The field must contain a timestamp in epoch milliseconds. + * + *

If a message does NOT contain this field, it will be dropped + * and not included in any profiles. + */ + private Optional timestampField = Optional.empty(); + public List getProfiles() { return profiles; } @@ -39,10 +54,33 @@ public void setProfiles(List profiles) { this.profiles = profiles; } + public ProfilerConfig withProfile(ProfileConfig profileConfig) { + this.profiles.add(profileConfig); + return this; + } + + public Optional getTimestampField() { + return timestampField; + } + + public void setTimestampField(String timestampField) { + this.timestampField = Optional.of(timestampField); + } + + public void setTimestampField(Optional timestampField) { + this.timestampField = timestampField; + } + + public ProfilerConfig withTimestampField(Optional timestampField) { + this.timestampField = timestampField; + return this; + } + @Override public String toString() { return "ProfilerConfig{" + "profiles=" + profiles + + ", timestampField='" + timestampField + '\'' + '}'; } @@ -50,13 +88,15 @@ public String toString() { public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - ProfilerConfig that = (ProfilerConfig) o; - return profiles != null ? profiles.equals(that.profiles) : that.profiles == null; + if (profiles != null ? !profiles.equals(that.profiles) : that.profiles != null) return false; + return timestampField != null ? timestampField.equals(that.timestampField) : that.timestampField == null; } @Override public int hashCode() { - return profiles != null ? profiles.hashCode() : 0; + int result = profiles != null ? profiles.hashCode() : 0; + result = 31 * result + (timestampField != null ? timestampField.hashCode() : 0); + return result; } } diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/JSONUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/JSONUtils.java index c02f19de2e..02e6015f53 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/JSONUtils.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/JSONUtils.java @@ -24,6 +24,10 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.flipkart.zjsonpatch.JsonPatch; +import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; +import org.json.simple.parser.ParseException; + import java.io.BufferedInputStream; import java.io.File; import java.io.FileInputStream; @@ -31,17 +35,10 @@ import java.io.InputStream; import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; -import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.function.Supplier; -import com.google.common.reflect.TypeToken; -import org.json.simple.JSONObject; -import org.json.simple.parser.JSONParser; -import org.json.simple.parser.ParseException; - public enum JSONUtils { INSTANCE; diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/zookeeper/configurations/ProfilerUpdater.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/zookeeper/configurations/ProfilerUpdater.java index 68c5203ecd..4976d30078 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/zookeeper/configurations/ProfilerUpdater.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/zookeeper/configurations/ProfilerUpdater.java @@ -33,6 +33,7 @@ import java.util.function.Supplier; public class ProfilerUpdater extends ConfigurationsUpdater { + public ProfilerUpdater(Reloadable reloadable, Supplier configSupplier) { super(reloadable, configSupplier); } diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/profiler/ProfileConfigTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/profiler/ProfileConfigTest.java index a0e115de23..e178ee02e1 100644 --- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/profiler/ProfileConfigTest.java +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/profiler/ProfileConfigTest.java @@ -27,10 +27,11 @@ import java.io.IOException; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; /** - * Ensures that Profile definitions have the expected defaults + * Tests the {@link ProfileConfig} class. + * + * Ensures that profile definitions have the expected defaults * and can be (de)serialized to and from JSON. */ public class ProfileConfigTest { diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/profiler/ProfilerConfigTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/profiler/ProfilerConfigTest.java new file mode 100644 index 0000000000..2e73cde724 --- /dev/null +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/configuration/profiler/ProfilerConfigTest.java @@ -0,0 +1,120 @@ +/* + * + * 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.common.configuration.profiler; + +import org.adrianwalker.multilinestring.Multiline; +import org.apache.metron.common.utils.JSONUtils; +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests the {@link ProfilerConfig} class. + */ +public class ProfilerConfigTest { + + /** + * { + * "profiles": [ + * { + * "profile": "profile1", + * "foreach": "ip_src_addr", + * "init": { "count": "0" }, + * "update": { "count": "count + 1" }, + * "result": "count" + * } + * ] + * } + */ + @Multiline + private String noTimestampField; + + /** + * If no 'timestampField' is defined, it should not be present by default. + */ + @Test + public void testNoTimestampField() throws IOException { + ProfilerConfig conf = JSONUtils.INSTANCE.load(noTimestampField, ProfilerConfig.class); + assertFalse(conf.getTimestampField().isPresent()); + } + + /** + * { + * "profiles": [ + * { + * "profile": "profile1", + * "foreach": "ip_src_addr", + * "init": { "count": "0" }, + * "update": { "count": "count + 1" }, + * "result": "count" + * } + * ], + * "timestampField": "timestamp" + * } + */ + @Multiline + private String timestampField; + + /** + * If no 'timestampField' is defined, it should not be present by default. + */ + @Test + public void testTimestampField() throws IOException { + ProfilerConfig conf = JSONUtils.INSTANCE.load(timestampField, ProfilerConfig.class); + assertTrue(conf.getTimestampField().isPresent()); + } + + /** + * { + * "profiles": [ + * { + * "profile": "profile1", + * "foreach": "ip_src_addr", + * "init": { "count": "0" }, + * "update": { "count": "count + 1" }, + * "result": "count" + * }, + * { + * "profile": "profile2", + * "foreach": "ip_dst_addr", + * "init": { "count": "0" }, + * "update": { "count": "count + 1" }, + * "result": "count" + * } + * ] + * } + */ + @Multiline + private String twoProfiles; + + /** + * The 'onlyif' field should default to 'true' when it is not specified. + */ + @Test + public void testTwoProfiles() throws IOException { + ProfilerConfig conf = JSONUtils.INSTANCE.load(twoProfiles, ProfilerConfig.class); + assertEquals(2, conf.getProfiles().size()); + } + +} diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/KafkaComponent.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/KafkaComponent.java index 9d8c57e7c0..08910be73d 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/KafkaComponent.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/KafkaComponent.java @@ -30,6 +30,9 @@ import java.util.Map; import java.util.Properties; import java.util.logging.Level; +import java.util.stream.Collectors; +import java.util.stream.Stream; + import kafka.api.FetchRequest; import kafka.api.FetchRequestBuilder; import kafka.common.TopicExistsException; @@ -48,6 +51,7 @@ import kafka.utils.ZKStringSerializer$; import kafka.utils.ZkUtils; import org.I0Itec.zkclient.ZkClient; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.metron.integration.InMemoryComponent; @@ -314,11 +318,44 @@ public void createTopic(String name, int numPartitions, boolean waitUntilMetadat } } + /** + * Write a collection of messages to a Kafka topic. + * + * @param topic The name of the Kafka topic. + * @param messages The collection of messages to write. + */ public void writeMessages(String topic, Collection messages) { try(KafkaProducer kafkaProducer = createProducer()) { for (byte[] message : messages) { - kafkaProducer.send(new ProducerRecord(topic, message)); + kafkaProducer.send(new ProducerRecord<>(topic, message)); } } } + + /** + * Write messages to a Kafka topic. + * + * @param topic The name of the Kafka topic. + * @param messages The messages to write. + */ + public void writeMessages(String topic, String ...messages) { + + // convert each message to raw bytes + List messagesAsBytes = Stream.of(messages) + .map(Bytes::toBytes) + .collect(Collectors.toList()); + + writeMessages(topic, messagesAsBytes); + } + + /** + * Write messages to a Kafka topic. + * + * @param topic The name of the Kafka topic. + * @param messages The messages to write. + */ + public void writeMessages(String topic, List messages) { + + writeMessages(topic, messages.toArray(new String[] {})); + } } From 3f615715d759ee8af7ec873f6bef2f20c07bc35f Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 15 Mar 2018 17:03:00 -0400 Subject: [PATCH 02/20] Updated README with more details on event time. --- .../profiler/DefaultMessageDistributor.java | 2 - metron-analytics/metron-profiler/README.md | 43 +++++++++++++------ 2 files changed, 29 insertions(+), 16 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 10fbbef1b8..ea5126f5c6 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 @@ -67,8 +67,6 @@ public class DefaultMessageDistributor implements MessageDistributor { */ private long periodDurationMillis; - // TODO why are these marked transient? - /** * A cache of active profiles. * diff --git a/metron-analytics/metron-profiler/README.md b/metron-analytics/metron-profiler/README.md index 576b15a145..8e8006dd3e 100644 --- a/metron-analytics/metron-profiler/README.md +++ b/metron-analytics/metron-profiler/README.md @@ -332,15 +332,21 @@ Continuing the previous running example, at this point, you have seen how your p The Profiler configuration contains only two fields; only one of which is required. - ``` - { - "profiles": [ - { "profile": "one", ... }, - { "profile": "two", ... } - ], - "timestampField": "timestamp" - } - ``` +``` +{ + "profiles": [ + { "profile": "one", ... }, + { "profile": "two", ... } + ], + "timestampField": "timestamp" +} +``` + +| Name | | Description +|--- |--- |--- +| [profiles](#profiles) | Required | A list of zero or more Profile definitions. +| [timestampField](#timestampfield) | Optional | Indicates whether processing time or event time should be used. + #### `profiles` @@ -352,20 +358,29 @@ A list of zero or more Profile definitions. *Optional* +Indicates whether processing time or event time should be used. + ##### Processing Time By default, no `timestampField` is defined. In this case, the Profiler uses system time when generating profiles. This means that the profiles are generated based on when the data has been processed by the Profiler. This is also known as 'processing time'. -This is the simplest mode of operation, but has some draw backs. If the Profiler is consuming live data and all is well, the processing and event times will likely remain similar and consistent. When processing time diverges from event time, the profiles produced can be skewed. +This is the simplest mode of operation, but has some draw backs. If the Profiler is consuming live data and all is well, the processing and event times will likely remain similar and consistent. If processing time diverges from event time, then the Profiler will generate skewed profiles. + +There are a few scenarios that might cause skewed profiles when using processing time. For example when a system has undergone a scheduled maintenance window and is restarted, a high volume of messages will need to be processed by the Profiler. The output of the Profiler might indicate an increase in activity during this time, although no change in activity actually occurred on the target network. The same situation could occur if an upstream system which provides telemetry undergoes an outage. -There are a few scenarios that could cause skewed profiles when using processing time. For example when a system has undergone a scheduled maintenance window and is restarted, a high volume of messages will need to be processed by the Profiler. The output of the Profiler will indicate an increase in activity during this time, although no change in activity actually occurred on the target network. The same situation could occur if an upstream system which provides telemetry undergoes an outage. +[Event Time](#event-time) can be used to mitigate these problems. ##### Event Time -Alternatively, you can define a `timestampField`. This must be the name of a field contained within the telemetry processed by the Profiler. The Profiler will extract and use the timestamp contained within this field. +Alternatively, a `timestampField` can be defined. This must be the name of a field contained within the telemetry processed by the Profiler. The Profiler will extract and use the timestamp contained within this field. + +* If a message does not contain this field, it will be dropped. + +* The field must contain a timestamp in epoch milliseconds expressed as either a numeric or string. Otherwise, the message will be dropped. + +* The Profiler will use the same field across all telemetry sources and for all profiles. -* The field must contain a timestamp in epoch milliseconds. -* If a message does not contain this field, it will be dropped by the Profiler. +* Be aware of clock skew across telemetry sources. If your profile is processing telemetry from multiple sources where the clock differs significantly, the Profiler may assume that some of those messages are late and will be ignored. Adjusting the [`profiler.window.duration`](#profilerwindowduration) and [`profiler.window.lag`](#profilerwindowlag) can help accommodate skewed clocks. ### Profiles From de754649e1f3177c4e3b209fe9495826e4b76e86 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 15 Mar 2018 17:12:03 -0400 Subject: [PATCH 03/20] Fixed the property name in the integration tests --- .../metron/profiler/integration/ProfilerIntegrationTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java index 9d48406ae8..016d61a823 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java @@ -398,8 +398,8 @@ public static void setupBeforeClass() throws UnableToStartException { setProperty("profiler.ttl.units", "MILLISECONDS"); setProperty("profiler.window.duration", Long.toString(windowDurationMillis)); setProperty("profiler.window.duration.units", "MILLISECONDS"); - setProperty("profiler.event.time.lag", Long.toString(eventTimeLagMillis)); - setProperty("profiler.event.time.lag.units", "MILLISECONDS"); + setProperty("profiler.window.lag", Long.toString(eventTimeLagMillis)); + setProperty("profiler.window.lag.units", "MILLISECONDS"); setProperty("profiler.max.routes.per.bolt", Long.toString(maxRoutesPerBolt)); }}; From 36923c0f841f6cc24f17794b696c1e22881806c7 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 15 Mar 2018 17:53:01 -0400 Subject: [PATCH 04/20] Fixed a few items I missed in rename to 'window lag' --- .../metron-profiler/src/main/config/profiler.properties | 4 ++-- .../metron-profiler/src/main/flux/profiler/remote.yaml | 4 ++-- .../metron/profiler/integration/ProfilerIntegrationTest.java | 4 ++-- .../METRON/CURRENT/package/templates/profiler.properties.j2 | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/metron-analytics/metron-profiler/src/main/config/profiler.properties b/metron-analytics/metron-profiler/src/main/config/profiler.properties index 7d5b8e3552..fe3c475162 100644 --- a/metron-analytics/metron-profiler/src/main/config/profiler.properties +++ b/metron-analytics/metron-profiler/src/main/config/profiler.properties @@ -37,8 +37,8 @@ profiler.window.duration=30 profiler.window.duration.units=SECONDS profiler.ttl=30 profiler.ttl.units=MINUTES -profiler.event.time.lag=1 -profiler.event.time.lag.units=MINUTES +profiler.window.lag=1 +profiler.window.lag.units=MINUTES profiler.max.routes.per.bolt=10000 ##### HBase ##### diff --git a/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml b/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml index c1c2f1acf6..83c9fde872 100644 --- a/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml +++ b/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml @@ -121,7 +121,7 @@ components: - ${profiler.window.duration} - "${profiler.window.duration.units}" - - id: "eventTimeLag" + - id: "windowLag" className: "org.apache.storm.topology.base.BaseWindowedBolt$Duration" constructorArgs: - ${profiler.window.lag} @@ -157,7 +157,7 @@ bolts: - name: "withTumblingWindow" args: [ref: "windowDuration"] - name: "withLag" - args: [ref: "eventTimeLag"] + args: [ref: "windowLag"] - name: "withMaxNumberOfRoutes" args: [${profiler.max.routes.per.bolt}] diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java index 016d61a823..c48a3e911f 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java @@ -118,7 +118,7 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { private static final String outputTopic = "profiles"; private static final int saltDivisor = 10; - private static final long eventTimeLagMillis = TimeUnit.SECONDS.toMillis(5); + private static final long windowLagMillis = TimeUnit.SECONDS.toMillis(5); private static final long windowDurationMillis = TimeUnit.SECONDS.toMillis(5); private static final long periodDurationMillis = TimeUnit.SECONDS.toMillis(15); private static final long profileTimeToLiveMillis = TimeUnit.SECONDS.toMillis(20); @@ -398,7 +398,7 @@ public static void setupBeforeClass() throws UnableToStartException { setProperty("profiler.ttl.units", "MILLISECONDS"); setProperty("profiler.window.duration", Long.toString(windowDurationMillis)); setProperty("profiler.window.duration.units", "MILLISECONDS"); - setProperty("profiler.window.lag", Long.toString(eventTimeLagMillis)); + setProperty("profiler.window.lag", Long.toString(windowLagMillis)); setProperty("profiler.window.lag.units", "MILLISECONDS"); setProperty("profiler.max.routes.per.bolt", Long.toString(maxRoutesPerBolt)); }}; diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/profiler.properties.j2 b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/profiler.properties.j2 index 1fd3019b0c..fabdaa7778 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/profiler.properties.j2 +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/profiler.properties.j2 @@ -37,8 +37,8 @@ profiler.window.duration={{profiler_window_duration}} profiler.window.duration.units={{profiler_window_units}} profiler.ttl={{profiler_ttl}} profiler.ttl.units={{profiler_ttl_units}} -profiler.event.time.lag={{profiler_event_time_lag}} -profiler.event.time.lag.units={{profiler_event_time_lag_units}} +profiler.window.lag={{profiler_window_lag}} +profiler.window.lag.units={{profiler_window_lag_units}} profiler.max.routes.per.bolt={{profiler_max_routes_per_bolt}} ##### HBase ##### From 1e8b76b9819e983f6a649183fd3a89e7c5ab1817 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 16 Mar 2018 12:01:46 -0400 Subject: [PATCH 05/20] METRON-1494 Profiler Emits Messages to Kafka When Not Needed --- .../metron/profiler/bolt/KafkaEmitter.java | 65 ++++-- .../profiler/bolt/KafkaEmitterTest.java | 201 +++++++++++++----- 2 files changed, 195 insertions(+), 71 deletions(-) diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java index 29d1a49943..65c7b1eae1 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java @@ -21,6 +21,10 @@ import java.io.Serializable; import java.lang.invoke.MethodHandles; +import java.util.Map; + +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.ClassUtils; import org.apache.metron.profiler.ProfileMeasurement; import org.apache.storm.task.OutputCollector; @@ -58,19 +62,38 @@ public void declareOutputFields(OutputFieldsDeclarer declarer) { @Override public void emit(ProfileMeasurement measurement, OutputCollector collector) { - JSONObject message = new JSONObject(); - message.put("profile", measurement.getDefinition().getProfile()); - message.put("entity", measurement.getEntity()); - message.put("period", measurement.getPeriod().getPeriod()); - message.put("period.start", measurement.getPeriod().getStartTimeMillis()); - message.put("period.end", measurement.getPeriod().getEndTimeMillis()); - message.put("timestamp", System.currentTimeMillis()); - message.put("source.type", sourceType); - message.put("is_alert", "true"); + // only need to emit, if there are triage values + Map triageValues = measurement.getTriageValues(); + if(MapUtils.isNotEmpty(triageValues)) { + + JSONObject message = createMessage(measurement); + appendTriageValues(measurement, message); + collector.emit(getStreamId(), new Values(message)); + + } else { + LOG.debug("No triage values, nothing to emit; profile={}, entity={}, period={}, start={}, end={}", + measurement.getProfileName(), + measurement.getEntity(), + measurement.getPeriod().getPeriod(), + measurement.getPeriod().getStartTimeMillis(), + measurement.getPeriod().getEndTimeMillis()); + } + } + + /** + * Appends triage values obtained from a {@code ProfileMeasurement} to the + * outgoing message. + * + * @param measurement The measurement that may contain triage values. + * @param message The message that the triage values are appended to. + */ + private void appendTriageValues(ProfileMeasurement measurement, JSONObject message) { - // append each of the triage values to the message - measurement.getTriageValues().forEach((key, value) -> { + // for each triage value... + Map triageValues = MapUtils.emptyIfNull(measurement.getTriageValues()); + triageValues.forEach((key, value) -> { + // append the triage value to the message if(isValidType(value)) { message.put(key, value); @@ -83,8 +106,26 @@ public void emit(ProfileMeasurement measurement, OutputCollector collector) { key)); } }); + } - collector.emit(getStreamId(), new Values(message)); + /** + * Creates a message that will be emitted to Kafka. + * + * @param measurement The profile measurement used as a basis for the message. + * @return A message that can be emitted to Kafka. + */ + private JSONObject createMessage(ProfileMeasurement measurement) { + + JSONObject message = new JSONObject(); + message.put("profile", measurement.getDefinition().getProfile()); + message.put("entity", measurement.getEntity()); + message.put("period", measurement.getPeriod().getPeriod()); + message.put("period.start", measurement.getPeriod().getStartTimeMillis()); + message.put("period.end", measurement.getPeriod().getEndTimeMillis()); + message.put("timestamp", System.currentTimeMillis()); + message.put("source.type", sourceType); + message.put("is_alert", "true"); + return message; } /** diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java index b02e377c78..0cb5653afd 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java @@ -43,6 +43,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -58,54 +59,128 @@ public class KafkaEmitterTest { * "foreach": "ip_src_addr", * "init": { "x": "0" }, * "update": { "x": "x + 1" }, - * "result": "x" + * "result": { + * "profile": "x", + * "triage": { + * "value": "x" + * } + * } * } */ @Multiline - private String profileDefinition; + private String profileDefinitionWithTriage; - private KafkaEmitter handler; + private KafkaEmitter kafkaEmitter; private ProfileConfig profile; private OutputCollector collector; @Before public void setup() throws Exception { - handler = new KafkaEmitter(); - profile = createDefinition(profileDefinition); + kafkaEmitter = new KafkaEmitter(); + profile = createDefinition(profileDefinitionWithTriage); collector = Mockito.mock(OutputCollector.class); } /** - * The handler must serialize the ProfileMeasurement into a JSONObject. + * The handler should emit a message when a result/triage expression(s) has been defined. */ @Test - public void testSerialization() throws Exception { + public void testEmit() throws Exception { + // create a measurement that has triage values + ProfileMeasurement measurement = new ProfileMeasurement() + .withProfileName("profile") + .withEntity("entity") + .withPeriod(20000, 15, TimeUnit.MINUTES) + .withDefinition(profile) + .withTriageValues(Collections.singletonMap("triage-key", "triage-value")); + + // execute the test + kafkaEmitter.emit(measurement, collector); + + // a message should be emitted + verify(collector, times(1)).emit(eq(kafkaEmitter.getStreamId()), any()); + } + + /** + * The handler should NOT emit a message when there is NO result/triage value(s). + */ + @Test + public void testDoNotEmit() throws Exception { + + // create a measurement with NO triage values ProfileMeasurement measurement = new ProfileMeasurement() .withProfileName("profile") .withEntity("entity") .withPeriod(20000, 15, TimeUnit.MINUTES) - .withTriageValues(Collections.singletonMap("triage-key", "triage-value")) .withDefinition(profile); - handler.emit(measurement, collector); - ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); - verify(collector, times(1)).emit(eq(handler.getStreamId()), arg.capture()); + // execute the test + kafkaEmitter.emit(measurement, collector); - // expect a JSONObject - Values values = arg.getValue(); - assertTrue(values.get(0) instanceof JSONObject); + // a message should NOT be emitted + verify(collector, times(0)).emit(eq(kafkaEmitter.getStreamId()), any()); + } - // validate the json - JSONObject actual = (JSONObject) values.get(0); - assertEquals(measurement.getDefinition().getProfile(), actual.get("profile")); - assertEquals(measurement.getEntity(), actual.get("entity")); - assertEquals(measurement.getPeriod().getPeriod(), actual.get("period")); - assertEquals(measurement.getPeriod().getStartTimeMillis(), actual.get("period.start")); - assertEquals(measurement.getPeriod().getEndTimeMillis(), actual.get("period.end")); - assertEquals(measurement.getTriageValues().get("triage-key"), actual.get("triage-key")); + /** + * Validate that the message generated for Kafka should include the triage value. + */ + @Test + public void testTriageValueInMessage() throws Exception { + + // create a measurement that has triage values + ProfileMeasurement measurement = new ProfileMeasurement() + .withDefinition(profile) + .withProfileName(profile.getProfile()) + .withEntity("entity") + .withPeriod(20000, 15, TimeUnit.MINUTES) + .withTriageValues(Collections.singletonMap("triage-key", "triage-value")); + + // execute the test + kafkaEmitter.emit(measurement, collector); + JSONObject actual = expectJsonObject(kafkaEmitter, collector); + + // validate the core parts of the message + assertEquals(measurement.getProfileName(), actual.get("profile")); + assertEquals(measurement.getEntity(), actual.get("entity")); + assertEquals(measurement.getPeriod().getPeriod(), actual.get("period")); + assertEquals(measurement.getPeriod().getStartTimeMillis(), actual.get("period.start")); + assertEquals(measurement.getPeriod().getEndTimeMillis(), actual.get("period.end")); + assertEquals("profiler", actual.get("source.type")); assertNotNull(actual.get("timestamp")); - assertEquals("profiler", actual.get("source.type")); + + // validate that the triage value has been added + assertEquals(measurement.getTriageValues().get("triage-key"), actual.get("triage-key")); + } + + /** + * Validate that the message generated for Kafka can include multiple triage values. + */ + @Test + public void testMultipleTriageValueInMessage() throws Exception { + + // multiple triage values have been defined + Map triageValues = ImmutableMap.of( + "x", 2, + "y", "4", + "z", 6.0); + + // create a measurement that has multiple triage values + ProfileMeasurement measurement = new ProfileMeasurement() + .withDefinition(profile) + .withProfileName(profile.getProfile()) + .withEntity("entity") + .withPeriod(20000, 15, TimeUnit.MINUTES) + .withTriageValues(triageValues); + + // execute the test + kafkaEmitter.emit(measurement, collector); + JSONObject actual = expectJsonObject(kafkaEmitter, collector); + + // validate that ALL of the triage values have been added + assertEquals(measurement.getTriageValues().get("x"), actual.get("x")); + assertEquals(measurement.getTriageValues().get("y"), actual.get("y")); + assertEquals(measurement.getTriageValues().get("z"), actual.get("z")); } /** @@ -120,30 +195,27 @@ public void testInvalidType() throws Exception { "invalid", new OnlineStatisticsProvider(), "valid", 4); + // create the measurement with a Map as a triage value; this is not allowed ProfileMeasurement measurement = new ProfileMeasurement() - .withProfileName("profile") + .withDefinition(profile) + .withProfileName(profile.getProfile()) .withEntity("entity") .withPeriod(20000, 15, TimeUnit.MINUTES) - .withTriageValues(triageValues) - .withDefinition(profile); - handler.emit(measurement, collector); + .withTriageValues(triageValues); - ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); - verify(collector, times(1)).emit(eq(handler.getStreamId()), arg.capture()); - Values values = arg.getValue(); - assertTrue(values.get(0) instanceof JSONObject); + // execute the test + kafkaEmitter.emit(measurement, collector); + JSONObject actual = expectJsonObject(kafkaEmitter, collector); - // only the triage expression value itself should have been skipped, all others should be there - JSONObject actual = (JSONObject) values.get(0); - assertEquals(measurement.getDefinition().getProfile(), actual.get("profile")); - assertEquals(measurement.getEntity(), actual.get("entity")); - assertEquals(measurement.getPeriod().getPeriod(), actual.get("period")); - assertEquals(measurement.getPeriod().getStartTimeMillis(), actual.get("period.start")); - assertEquals(measurement.getPeriod().getEndTimeMillis(), actual.get("period.end")); - assertNotNull(actual.get("timestamp")); - assertEquals("profiler", actual.get("source.type")); + // validate the core parts of the message still exist + assertEquals(measurement.getProfileName(), actual.get("profile")); + assertEquals(measurement.getEntity(), actual.get("entity")); + assertEquals(measurement.getPeriod().getPeriod(), actual.get("period")); + assertEquals(measurement.getPeriod().getStartTimeMillis(), actual.get("period.start")); + assertEquals(measurement.getPeriod().getEndTimeMillis(), actual.get("period.end")); + assertEquals("profiler", actual.get("source.type")); - // the invalid expression should be skipped due to invalid type + // the invalid expression should be skipped and not included in the message assertFalse(actual.containsKey("invalid")); // but the valid expression should still be there @@ -156,19 +228,18 @@ public void testInvalidType() throws Exception { */ @Test public void testIntegerIsValidType() throws Exception { + + // create a measurement with a triage value that is an integer ProfileMeasurement measurement = new ProfileMeasurement() - .withProfileName("profile") + .withDefinition(profile) + .withProfileName(profile.getProfile()) .withEntity("entity") .withPeriod(20000, 15, TimeUnit.MINUTES) - .withTriageValues(Collections.singletonMap("triage-key", 123)) - .withDefinition(profile); - handler.emit(measurement, collector); + .withTriageValues(Collections.singletonMap("triage-key", 123)); - ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); - verify(collector, times(1)).emit(eq(handler.getStreamId()), arg.capture()); - Values values = arg.getValue(); - assertTrue(values.get(0) instanceof JSONObject); - JSONObject actual = (JSONObject) values.get(0); + // execute the test + kafkaEmitter.emit(measurement, collector); + JSONObject actual = expectJsonObject(kafkaEmitter, collector); // the triage expression is valid assertEquals(measurement.getTriageValues().get("triage-key"), actual.get("triage-key")); @@ -180,24 +251,36 @@ public void testIntegerIsValidType() throws Exception { */ @Test public void testStringIsValidType() throws Exception { + + // create a measurement with a triage value that is a string ProfileMeasurement measurement = new ProfileMeasurement() - .withProfileName("profile") + .withDefinition(profile) + .withProfileName(profile.getProfile()) .withEntity("entity") .withPeriod(20000, 15, TimeUnit.MINUTES) - .withTriageValues(Collections.singletonMap("triage-key", "value")) - .withDefinition(profile); - handler.emit(measurement, collector); + .withTriageValues(Collections.singletonMap("triage-key", "value")); - ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); - verify(collector, times(1)).emit(eq(handler.getStreamId()), arg.capture()); - Values values = arg.getValue(); - assertTrue(values.get(0) instanceof JSONObject); - JSONObject actual = (JSONObject) values.get(0); + // execute the test + kafkaEmitter.emit(measurement, collector); + JSONObject actual = expectJsonObject(kafkaEmitter, collector); // the triage expression is valid assertEquals(measurement.getTriageValues().get("triage-key"), actual.get("triage-key")); } + /** + * Verifies that the KafkaEmitter does emit a JSONObject. + * @return The JSONObject that was emitted + */ + private JSONObject expectJsonObject(KafkaEmitter kafkaEmitter, OutputCollector collector) { + + ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); + verify(collector, times(1)).emit(eq(kafkaEmitter.getStreamId()), arg.capture()); + Values values = arg.getValue(); + assertTrue(values.get(0) instanceof JSONObject); + return (JSONObject) values.get(0); + } + /** * Creates a profile definition based on a string of JSON. * @param json The string of JSON. From d78ba493620a4e902c7d6b0dc6719b1c9d3c0bca Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 16 Mar 2018 13:26:05 -0400 Subject: [PATCH 06/20] Cleaning up imports --- .../org/apache/metron/profiler/bolt/KafkaEmitter.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java index 65c7b1eae1..f1ca555f4d 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java @@ -19,11 +19,6 @@ package org.apache.metron.profiler.bolt; -import java.io.Serializable; -import java.lang.invoke.MethodHandles; -import java.util.Map; - -import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.ClassUtils; import org.apache.metron.profiler.ProfileMeasurement; @@ -35,6 +30,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Serializable; +import java.lang.invoke.MethodHandles; +import java.util.Map; + /** * Responsible for emitting a {@link ProfileMeasurement} to an output stream that will * persist data in HBase. From ab75a1d7d887351bfcbadb4221ad10938e3998ee Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 16 Mar 2018 15:59:23 -0400 Subject: [PATCH 07/20] Fixed comment --- .../org/apache/metron/profiler/bolt/ProfileBuilderBolt.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 ffe823ff2e..27fb860864 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 @@ -340,7 +340,7 @@ private void handleMessage(Tuple input) { // keep track of time flushSignal.update(timestamp); - + // distribute the message MessageRoute route = new MessageRoute(definition, entity); messageDistributor.distribute(message, timestamp, route, getStellarContext()); @@ -355,7 +355,6 @@ private void handleMessage(Tuple input) { */ private void emitMeasurements(List measurements) { - // flush each profile for(ProfileMeasurement measurement: measurements) { // allow each 'emitter' to emit the measurement From 9aa09f4cdb9a6e1947606f0b7823d85823121cf7 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Wed, 21 Mar 2018 14:41:55 -0400 Subject: [PATCH 08/20] Added tests for this fix --- .../profiler/bolt/KafkaEmitterTest.java | 201 +++++++++++++----- 1 file changed, 142 insertions(+), 59 deletions(-) diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java index b02e377c78..0cb5653afd 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java @@ -43,6 +43,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -58,54 +59,128 @@ public class KafkaEmitterTest { * "foreach": "ip_src_addr", * "init": { "x": "0" }, * "update": { "x": "x + 1" }, - * "result": "x" + * "result": { + * "profile": "x", + * "triage": { + * "value": "x" + * } + * } * } */ @Multiline - private String profileDefinition; + private String profileDefinitionWithTriage; - private KafkaEmitter handler; + private KafkaEmitter kafkaEmitter; private ProfileConfig profile; private OutputCollector collector; @Before public void setup() throws Exception { - handler = new KafkaEmitter(); - profile = createDefinition(profileDefinition); + kafkaEmitter = new KafkaEmitter(); + profile = createDefinition(profileDefinitionWithTriage); collector = Mockito.mock(OutputCollector.class); } /** - * The handler must serialize the ProfileMeasurement into a JSONObject. + * The handler should emit a message when a result/triage expression(s) has been defined. */ @Test - public void testSerialization() throws Exception { + public void testEmit() throws Exception { + // create a measurement that has triage values + ProfileMeasurement measurement = new ProfileMeasurement() + .withProfileName("profile") + .withEntity("entity") + .withPeriod(20000, 15, TimeUnit.MINUTES) + .withDefinition(profile) + .withTriageValues(Collections.singletonMap("triage-key", "triage-value")); + + // execute the test + kafkaEmitter.emit(measurement, collector); + + // a message should be emitted + verify(collector, times(1)).emit(eq(kafkaEmitter.getStreamId()), any()); + } + + /** + * The handler should NOT emit a message when there is NO result/triage value(s). + */ + @Test + public void testDoNotEmit() throws Exception { + + // create a measurement with NO triage values ProfileMeasurement measurement = new ProfileMeasurement() .withProfileName("profile") .withEntity("entity") .withPeriod(20000, 15, TimeUnit.MINUTES) - .withTriageValues(Collections.singletonMap("triage-key", "triage-value")) .withDefinition(profile); - handler.emit(measurement, collector); - ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); - verify(collector, times(1)).emit(eq(handler.getStreamId()), arg.capture()); + // execute the test + kafkaEmitter.emit(measurement, collector); - // expect a JSONObject - Values values = arg.getValue(); - assertTrue(values.get(0) instanceof JSONObject); + // a message should NOT be emitted + verify(collector, times(0)).emit(eq(kafkaEmitter.getStreamId()), any()); + } - // validate the json - JSONObject actual = (JSONObject) values.get(0); - assertEquals(measurement.getDefinition().getProfile(), actual.get("profile")); - assertEquals(measurement.getEntity(), actual.get("entity")); - assertEquals(measurement.getPeriod().getPeriod(), actual.get("period")); - assertEquals(measurement.getPeriod().getStartTimeMillis(), actual.get("period.start")); - assertEquals(measurement.getPeriod().getEndTimeMillis(), actual.get("period.end")); - assertEquals(measurement.getTriageValues().get("triage-key"), actual.get("triage-key")); + /** + * Validate that the message generated for Kafka should include the triage value. + */ + @Test + public void testTriageValueInMessage() throws Exception { + + // create a measurement that has triage values + ProfileMeasurement measurement = new ProfileMeasurement() + .withDefinition(profile) + .withProfileName(profile.getProfile()) + .withEntity("entity") + .withPeriod(20000, 15, TimeUnit.MINUTES) + .withTriageValues(Collections.singletonMap("triage-key", "triage-value")); + + // execute the test + kafkaEmitter.emit(measurement, collector); + JSONObject actual = expectJsonObject(kafkaEmitter, collector); + + // validate the core parts of the message + assertEquals(measurement.getProfileName(), actual.get("profile")); + assertEquals(measurement.getEntity(), actual.get("entity")); + assertEquals(measurement.getPeriod().getPeriod(), actual.get("period")); + assertEquals(measurement.getPeriod().getStartTimeMillis(), actual.get("period.start")); + assertEquals(measurement.getPeriod().getEndTimeMillis(), actual.get("period.end")); + assertEquals("profiler", actual.get("source.type")); assertNotNull(actual.get("timestamp")); - assertEquals("profiler", actual.get("source.type")); + + // validate that the triage value has been added + assertEquals(measurement.getTriageValues().get("triage-key"), actual.get("triage-key")); + } + + /** + * Validate that the message generated for Kafka can include multiple triage values. + */ + @Test + public void testMultipleTriageValueInMessage() throws Exception { + + // multiple triage values have been defined + Map triageValues = ImmutableMap.of( + "x", 2, + "y", "4", + "z", 6.0); + + // create a measurement that has multiple triage values + ProfileMeasurement measurement = new ProfileMeasurement() + .withDefinition(profile) + .withProfileName(profile.getProfile()) + .withEntity("entity") + .withPeriod(20000, 15, TimeUnit.MINUTES) + .withTriageValues(triageValues); + + // execute the test + kafkaEmitter.emit(measurement, collector); + JSONObject actual = expectJsonObject(kafkaEmitter, collector); + + // validate that ALL of the triage values have been added + assertEquals(measurement.getTriageValues().get("x"), actual.get("x")); + assertEquals(measurement.getTriageValues().get("y"), actual.get("y")); + assertEquals(measurement.getTriageValues().get("z"), actual.get("z")); } /** @@ -120,30 +195,27 @@ public void testInvalidType() throws Exception { "invalid", new OnlineStatisticsProvider(), "valid", 4); + // create the measurement with a Map as a triage value; this is not allowed ProfileMeasurement measurement = new ProfileMeasurement() - .withProfileName("profile") + .withDefinition(profile) + .withProfileName(profile.getProfile()) .withEntity("entity") .withPeriod(20000, 15, TimeUnit.MINUTES) - .withTriageValues(triageValues) - .withDefinition(profile); - handler.emit(measurement, collector); + .withTriageValues(triageValues); - ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); - verify(collector, times(1)).emit(eq(handler.getStreamId()), arg.capture()); - Values values = arg.getValue(); - assertTrue(values.get(0) instanceof JSONObject); + // execute the test + kafkaEmitter.emit(measurement, collector); + JSONObject actual = expectJsonObject(kafkaEmitter, collector); - // only the triage expression value itself should have been skipped, all others should be there - JSONObject actual = (JSONObject) values.get(0); - assertEquals(measurement.getDefinition().getProfile(), actual.get("profile")); - assertEquals(measurement.getEntity(), actual.get("entity")); - assertEquals(measurement.getPeriod().getPeriod(), actual.get("period")); - assertEquals(measurement.getPeriod().getStartTimeMillis(), actual.get("period.start")); - assertEquals(measurement.getPeriod().getEndTimeMillis(), actual.get("period.end")); - assertNotNull(actual.get("timestamp")); - assertEquals("profiler", actual.get("source.type")); + // validate the core parts of the message still exist + assertEquals(measurement.getProfileName(), actual.get("profile")); + assertEquals(measurement.getEntity(), actual.get("entity")); + assertEquals(measurement.getPeriod().getPeriod(), actual.get("period")); + assertEquals(measurement.getPeriod().getStartTimeMillis(), actual.get("period.start")); + assertEquals(measurement.getPeriod().getEndTimeMillis(), actual.get("period.end")); + assertEquals("profiler", actual.get("source.type")); - // the invalid expression should be skipped due to invalid type + // the invalid expression should be skipped and not included in the message assertFalse(actual.containsKey("invalid")); // but the valid expression should still be there @@ -156,19 +228,18 @@ public void testInvalidType() throws Exception { */ @Test public void testIntegerIsValidType() throws Exception { + + // create a measurement with a triage value that is an integer ProfileMeasurement measurement = new ProfileMeasurement() - .withProfileName("profile") + .withDefinition(profile) + .withProfileName(profile.getProfile()) .withEntity("entity") .withPeriod(20000, 15, TimeUnit.MINUTES) - .withTriageValues(Collections.singletonMap("triage-key", 123)) - .withDefinition(profile); - handler.emit(measurement, collector); + .withTriageValues(Collections.singletonMap("triage-key", 123)); - ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); - verify(collector, times(1)).emit(eq(handler.getStreamId()), arg.capture()); - Values values = arg.getValue(); - assertTrue(values.get(0) instanceof JSONObject); - JSONObject actual = (JSONObject) values.get(0); + // execute the test + kafkaEmitter.emit(measurement, collector); + JSONObject actual = expectJsonObject(kafkaEmitter, collector); // the triage expression is valid assertEquals(measurement.getTriageValues().get("triage-key"), actual.get("triage-key")); @@ -180,24 +251,36 @@ public void testIntegerIsValidType() throws Exception { */ @Test public void testStringIsValidType() throws Exception { + + // create a measurement with a triage value that is a string ProfileMeasurement measurement = new ProfileMeasurement() - .withProfileName("profile") + .withDefinition(profile) + .withProfileName(profile.getProfile()) .withEntity("entity") .withPeriod(20000, 15, TimeUnit.MINUTES) - .withTriageValues(Collections.singletonMap("triage-key", "value")) - .withDefinition(profile); - handler.emit(measurement, collector); + .withTriageValues(Collections.singletonMap("triage-key", "value")); - ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); - verify(collector, times(1)).emit(eq(handler.getStreamId()), arg.capture()); - Values values = arg.getValue(); - assertTrue(values.get(0) instanceof JSONObject); - JSONObject actual = (JSONObject) values.get(0); + // execute the test + kafkaEmitter.emit(measurement, collector); + JSONObject actual = expectJsonObject(kafkaEmitter, collector); // the triage expression is valid assertEquals(measurement.getTriageValues().get("triage-key"), actual.get("triage-key")); } + /** + * Verifies that the KafkaEmitter does emit a JSONObject. + * @return The JSONObject that was emitted + */ + private JSONObject expectJsonObject(KafkaEmitter kafkaEmitter, OutputCollector collector) { + + ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); + verify(collector, times(1)).emit(eq(kafkaEmitter.getStreamId()), arg.capture()); + Values values = arg.getValue(); + assertTrue(values.get(0) instanceof JSONObject); + return (JSONObject) values.get(0); + } + /** * Creates a profile definition based on a string of JSON. * @param json The string of JSON. From 6be13340255d1f9f606765d2f87ad0744cea07ec Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Wed, 21 Mar 2018 16:28:08 -0400 Subject: [PATCH 09/20] Making sure the timeout for all profiler integration tests is the same --- .../integration/ProfilerIntegrationTest.java | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java index c48a3e911f..f70400caa7 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java @@ -20,6 +20,7 @@ package org.apache.metron.profiler.integration; +import com.google.code.tempusfugit.temporal.Duration; import com.google.common.base.Joiner; import org.adrianwalker.multilinestring.Multiline; import org.apache.commons.math.util.MathUtils; @@ -123,6 +124,7 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { private static final long periodDurationMillis = TimeUnit.SECONDS.toMillis(15); private static final long profileTimeToLiveMillis = TimeUnit.SECONDS.toMillis(20); private static final long maxRoutesPerBolt = 100000; + private static final Duration testTimeout = seconds(180); /** * Tests the first example contained within the README. @@ -139,8 +141,7 @@ public void testExample1() throws Exception { kafkaComponent.writeMessages(inputTopic, message3, message3, message3); // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(180))); + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, timeout(testTimeout)); // verify - only 10.0.0.2 sends 'HTTP', thus there should be only 1 value List actuals = read(profilerTable.getPutLog(), columnFamily, @@ -170,8 +171,7 @@ public void testExample2() throws Exception { final int expected = 2; // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() >= expected, - timeout(seconds(90))); + waitOrTimeout(() -> profilerTable.getPutLog().size() >= expected, timeout(testTimeout)); // verify - expect 2 results as 2 hosts involved; 10.0.0.2 sends 'HTTP' and 10.0.0.3 send 'DNS' List actuals = read(profilerTable.getPutLog(), columnFamily, @@ -203,8 +203,7 @@ public void testExample3() throws Exception { kafkaComponent.writeMessages(inputTopic, message3, message3, message3); // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(90))); + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, timeout(testTimeout)); // verify - only 10.0.0.2 sends 'HTTP', thus there should be only 1 value List actuals = read(profilerTable.getPutLog(), columnFamily, @@ -231,8 +230,7 @@ public void testExample4() throws Exception { kafkaComponent.writeMessages(inputTopic, message3, message3, message3); // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(90))); + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, timeout(testTimeout)); // verify - only 10.0.0.2 sends 'HTTP', thus there should be only 1 value byte[] column = columnBuilder.getColumnQualifier("value"); @@ -256,8 +254,7 @@ public void testPercentiles() throws Exception { kafkaComponent.writeMessages(inputTopic, message3, message3, message3); // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(90))); + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, timeout(testTimeout)); List actuals = read(profilerTable.getPutLog(), columnFamily, columnBuilder.getColumnQualifier("value"), Double.class); @@ -302,8 +299,7 @@ public void testEventTimeProcessing() throws Exception { kafkaComponent.writeMessages(inputTopic, message1, message2); // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(90))); + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, timeout(testTimeout)); List puts = profilerTable.getPutLog(); assertEquals(1, puts.size()); From 044c0a72db9dffda67243678f515a0a195fcf3a4 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Wed, 21 Mar 2018 16:30:29 -0400 Subject: [PATCH 10/20] Fix alignment --- .../org/apache/metron/profiler/bolt/KafkaEmitterTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java index 0cb5653afd..95a2d29475 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/KafkaEmitterTest.java @@ -146,7 +146,7 @@ public void testTriageValueInMessage() throws Exception { assertEquals(measurement.getPeriod().getPeriod(), actual.get("period")); assertEquals(measurement.getPeriod().getStartTimeMillis(), actual.get("period.start")); assertEquals(measurement.getPeriod().getEndTimeMillis(), actual.get("period.end")); - assertEquals("profiler", actual.get("source.type")); + assertEquals("profiler", actual.get("source.type")); assertNotNull(actual.get("timestamp")); // validate that the triage value has been added @@ -213,7 +213,7 @@ public void testInvalidType() throws Exception { assertEquals(measurement.getPeriod().getPeriod(), actual.get("period")); assertEquals(measurement.getPeriod().getStartTimeMillis(), actual.get("period.start")); assertEquals(measurement.getPeriod().getEndTimeMillis(), actual.get("period.end")); - assertEquals("profiler", actual.get("source.type")); + assertEquals("profiler", actual.get("source.type")); // the invalid expression should be skipped and not included in the message assertFalse(actual.containsKey("invalid")); From cf9ce2604ad4de3a3b5957606253b5d43921094c Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Wed, 21 Mar 2018 18:05:09 -0400 Subject: [PATCH 11/20] Added unit tests for HBaseEmitter and additional logging --- .../metron/profiler/bolt/HBaseEmitter.java | 12 +- .../metron/profiler/bolt/KafkaEmitter.java | 12 +- .../profiler/bolt/ProfileBuilderBolt.java | 13 +- .../profiler/bolt/HBaseEmitterTest.java | 119 ++++++++++++++++++ 4 files changed, 142 insertions(+), 14 deletions(-) create mode 100644 metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/HBaseEmitterTest.java diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/HBaseEmitter.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/HBaseEmitter.java index 8e1229a4d5..e4e35528a4 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/HBaseEmitter.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/HBaseEmitter.java @@ -40,7 +40,7 @@ public class HBaseEmitter implements ProfileMeasurementEmitter, Serializable { /** * The stream identifier used for this destination; */ - private String streamId = "hbase"; + private String streamId = "hbase"; @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { @@ -49,7 +49,17 @@ public void declareOutputFields(OutputFieldsDeclarer declarer) { @Override public void emit(ProfileMeasurement measurement, OutputCollector collector) { + + // measurements are always emitted to hbase collector.emit(getStreamId(), new Values(measurement)); + + LOG.debug("Emitted measurement; stream={}, profile={}, entity={}, period={}, start={}, end={}", + getStreamId(), + measurement.getProfileName(), + measurement.getEntity(), + measurement.getPeriod().getPeriod(), + measurement.getPeriod().getStartTimeMillis(), + measurement.getPeriod().getEndTimeMillis()); } @Override diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java index f1ca555f4d..87920da400 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/KafkaEmitter.java @@ -69,8 +69,18 @@ public void emit(ProfileMeasurement measurement, OutputCollector collector) { appendTriageValues(measurement, message); collector.emit(getStreamId(), new Values(message)); + LOG.debug("Emitted measurement; stream={}, profile={}, entity={}, period={}, start={}, end={}", + getStreamId(), + measurement.getProfileName(), + measurement.getEntity(), + measurement.getPeriod().getPeriod(), + measurement.getPeriod().getStartTimeMillis(), + measurement.getPeriod().getEndTimeMillis()); + } else { - LOG.debug("No triage values, nothing to emit; profile={}, entity={}, period={}, start={}, end={}", + + LOG.debug("No triage values, nothing to emit; stream={}, profile={}, entity={}, period={}, start={}, end={}", + getStreamId(), measurement.getProfileName(), measurement.getEntity(), measurement.getPeriod().getPeriod(), 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 27fb860864..48ab9187aa 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 @@ -355,21 +355,10 @@ private void handleMessage(Tuple input) { */ private void emitMeasurements(List measurements) { + // allow each 'emitter' to emit each measurement for(ProfileMeasurement measurement: measurements) { - - // allow each 'emitter' to emit the measurement for (ProfileMeasurementEmitter emitter : emitters) { emitter.emit(measurement, collector); - - LOG.debug("Measurement emitted; stream={}, profile={}, entity={}, value={}, start={}, end={}, duration={}, period={}", - emitter.getStreamId(), - measurement.getProfileName(), - measurement.getEntity(), - measurement.getProfileValue(), - measurement.getPeriod().getStartTimeMillis(), - measurement.getPeriod().getEndTimeMillis(), - measurement.getPeriod().getDurationMillis(), - measurement.getPeriod().getPeriod()); } } diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/HBaseEmitterTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/HBaseEmitterTest.java new file mode 100644 index 0000000000..dd658f650e --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/HBaseEmitterTest.java @@ -0,0 +1,119 @@ +/* + * + * 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.bolt; + +import org.adrianwalker.multilinestring.Multiline; +import org.apache.metron.common.configuration.profiler.ProfileConfig; +import org.apache.metron.common.utils.JSONUtils; +import org.apache.metron.profiler.ProfileMeasurement; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.tuple.Values; +import org.json.simple.JSONObject; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.Collections; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +/** + * Tests the HBaseEmitter class. + */ +public class HBaseEmitterTest { + + /** + * { + * "profile": "profile-one", + * "foreach": "ip_src_addr", + * "init": { "x": "0" }, + * "update": { "x": "x + 1" }, + * "result": "x" + * } + */ + @Multiline + private String profileDefinition; + + private HBaseEmitter emitter; + private ProfileConfig profile; + private OutputCollector collector; + + @Before + public void setup() throws Exception { + emitter = new HBaseEmitter(); + profile = createDefinition(profileDefinition); + collector = Mockito.mock(OutputCollector.class); + } + + /** + * The handler should emit a message when a result/triage expression(s) has been defined. + */ + @Test + public void testEmit() throws Exception { + + // create a measurement that has triage values + ProfileMeasurement measurement = new ProfileMeasurement() + .withProfileName("profile") + .withEntity("entity") + .withPeriod(20000, 15, TimeUnit.MINUTES) + .withDefinition(profile) + .withProfileValue(22); + + // execute the test + emitter.emit(measurement, collector); + + // the measurement should be emitted as-is + ProfileMeasurement actual = expectMeasurement(emitter, collector); + assertEquals(measurement, actual); + } + + /** + * Verifies that the emitter does emit a {@code ProfileMeasurement}. + * + * @return The {@code ProfileMeasurement} that was emitted + */ + private ProfileMeasurement expectMeasurement(HBaseEmitter hbaseEmitter, OutputCollector collector) { + + ArgumentCaptor arg = ArgumentCaptor.forClass(Values.class); + verify(collector, times(1)).emit(eq(hbaseEmitter.getStreamId()), arg.capture()); + Values values = arg.getValue(); + assertTrue(values.get(0) instanceof ProfileMeasurement); + return (ProfileMeasurement) values.get(0); + } + + /** + * Creates a profile definition based on a string of JSON. + * @param json The string of JSON. + */ + private ProfileConfig createDefinition(String json) throws IOException { + return JSONUtils.INSTANCE.load(json, ProfileConfig.class); + } +} From 19f4998d978b32ee5f522c223dadd959f6ffc3d0 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Wed, 21 Mar 2018 18:05:41 -0400 Subject: [PATCH 12/20] Setting Profiler logs to DEBUG --- .../metron-profiler/src/test/resources/log4j.properties | 2 ++ 1 file changed, 2 insertions(+) diff --git a/metron-analytics/metron-profiler/src/test/resources/log4j.properties b/metron-analytics/metron-profiler/src/test/resources/log4j.properties index 541f368c4d..1938c78c87 100644 --- a/metron-analytics/metron-profiler/src/test/resources/log4j.properties +++ b/metron-analytics/metron-profiler/src/test/resources/log4j.properties @@ -21,6 +21,8 @@ # Root logger option log4j.rootLogger=ERROR, stdout +log4j.logger.org.apache.metron.profiler=DEBUG + # Direct log messages to stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.Target=System.out From e8c203a297939d4c0b32aa55e5efb42f81b00e14 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 22 Mar 2018 12:13:58 -0400 Subject: [PATCH 13/20] Added additional logging to track down intermittent integration test failure --- .../metron/profiler/DefaultProfileBuilder.java | 5 +++++ .../profiler/bolt/FixedFrequencyFlushSignal.java | 13 +++++++++++-- .../metron/profiler/bolt/ProfileSplitterBolt.java | 7 +++++-- .../integration/ProfilerIntegrationTest.java | 2 +- 4 files changed, 22 insertions(+), 5 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 4b564c9191..66034ac7ac 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 @@ -124,8 +124,13 @@ private DefaultProfileBuilder(ProfileConfig definition, */ @Override public void apply(JSONObject message, long timestamp) { + LOG.debug("Applying message to profile; profile={}, entity={}, timestamp={}", + profileName, entity, timestamp); + try { if (!isInitialized()) { + LOG.debug("Initializing profile; profile={}, entity={}, timestamp={}", + profileName, entity, timestamp); // execute each 'init' expression assign(definition.getInit(), message, "init"); diff --git a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/FixedFrequencyFlushSignal.java b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/FixedFrequencyFlushSignal.java index b9f57dd54f..8c0a0b1482 100644 --- a/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/FixedFrequencyFlushSignal.java +++ b/metron-analytics/metron-profiler/src/main/java/org/apache/metron/profiler/bolt/FixedFrequencyFlushSignal.java @@ -94,7 +94,8 @@ public void update(long timestamp) { // set the next time to flush flushTime = currentTime + flushFrequency; - LOG.debug("Setting flush time; flushTime={}, currentTime={}, flushFreq={}", + LOG.debug("Setting flush time; '{}' ms until flush; flushTime={}, currentTime={}, flushFreq={}", + timeToNextFlush(), flushTime, currentTime, flushFrequency); @@ -112,7 +113,7 @@ public boolean isTimeToFlush() { boolean flush = currentTime > flushTime; LOG.debug("Flush={}, '{}' ms until flush; currentTime={}, flushTime={}", flush, - flush ? 0 : (flushTime-currentTime), + timeToNextFlush(), currentTime, flushTime); @@ -123,4 +124,12 @@ public boolean isTimeToFlush() { public long currentTimeMillis() { return currentTime; } + + /** + * Returns the number of milliseconds to the next flush. + * @return The time left until the next flush. + */ + private long timeToNextFlush() { + return Math.max(0, flushTime - currentTime); + } } 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 4e62eee135..be4d2952fe 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 @@ -191,9 +191,12 @@ private void routeMessage(Tuple input, JSONObject message, ProfilerConfig config Values values = createValues(message, timestamp, route); collector.emit(input, values); - } - LOG.debug("Found {} route(s) for message with timestamp={}", routes.size(), timestamp); + LOG.debug("Found route for message; profile={}, entity={}, timestamp={}", + route.getProfileDefinition().getProfile(), + route.getEntity(), + timestamp); + } } /** diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java index f70400caa7..19ab8f9c86 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java @@ -124,7 +124,7 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { private static final long periodDurationMillis = TimeUnit.SECONDS.toMillis(15); private static final long profileTimeToLiveMillis = TimeUnit.SECONDS.toMillis(20); private static final long maxRoutesPerBolt = 100000; - private static final Duration testTimeout = seconds(180); + private static final Duration testTimeout = seconds(120); /** * Tests the first example contained within the README. From 6faee6a4d58e9d2b65083422ce67a7e6f4e684c2 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 22 Mar 2018 13:13:48 -0400 Subject: [PATCH 14/20] This log statement is helpful --- .../org/apache/metron/profiler/bolt/ProfileSplitterBolt.java | 2 ++ 1 file changed, 2 insertions(+) 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 be4d2952fe..b93c98eb7a 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 @@ -197,6 +197,8 @@ private void routeMessage(Tuple input, JSONObject message, ProfilerConfig config route.getEntity(), timestamp); } + + LOG.debug("Found {} route(s) for message with timestamp={}", routes.size(), timestamp); } /** From e613836ffc585d38cc59ef6c4859d843df413918 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 22 Mar 2018 13:14:06 -0400 Subject: [PATCH 15/20] Fixed comment --- .../java/org/apache/metron/profiler/bolt/HBaseEmitterTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/HBaseEmitterTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/HBaseEmitterTest.java index dd658f650e..35ca4d9788 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/HBaseEmitterTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/bolt/HBaseEmitterTest.java @@ -74,7 +74,8 @@ public void setup() throws Exception { } /** - * The handler should emit a message when a result/triage expression(s) has been defined. + * The handler should emit a message containing the result of executing + * the 'result/profile' expression. */ @Test public void testEmit() throws Exception { From d3465b9a36a971a1a347a445a67f96a0e1affed1 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 22 Mar 2018 13:14:22 -0400 Subject: [PATCH 16/20] Increase logging for Storm's windowing functionality --- .../metron-profiler/src/test/resources/log4j.properties | 1 + 1 file changed, 1 insertion(+) diff --git a/metron-analytics/metron-profiler/src/test/resources/log4j.properties b/metron-analytics/metron-profiler/src/test/resources/log4j.properties index 1938c78c87..4fadbb3865 100644 --- a/metron-analytics/metron-profiler/src/test/resources/log4j.properties +++ b/metron-analytics/metron-profiler/src/test/resources/log4j.properties @@ -22,6 +22,7 @@ log4j.rootLogger=ERROR, stdout log4j.logger.org.apache.metron.profiler=DEBUG +log4j.logger.org.apache.storm.windowing=ALL # Direct log messages to stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender From e9185e2249bfcca1c47d3809e05bf742a73ff80e Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Mon, 26 Mar 2018 08:13:26 -0400 Subject: [PATCH 17/20] Added debug logging to HbaseBolt --- .../apache/metron/hbase/bolt/HBaseBolt.java | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java index d16e2f6ddc..479cdddfac 100644 --- a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java +++ b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java @@ -77,6 +77,8 @@ public class HBaseBolt extends BaseRichBolt { /** * The name of the class that should be used as a table provider. + * + *

Defaults to 'org.apache.metron.hbase.HTableProvider'. */ protected String tableProviderClazzName = "org.apache.metron.hbase.HTableProvider"; @@ -126,6 +128,8 @@ public void setClient(HBaseClient hbaseClient) { @Override public Map getComponentConfiguration() { + LOG.debug("Tick tuples expected every {} second(s)", flushIntervalSecs); + Config conf = new Config(); conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, flushIntervalSecs); return conf; @@ -136,7 +140,13 @@ public void prepare(Map map, TopologyContext topologyContext, OutputCollector co this.collector = collector; this.batchHelper = new BatchHelper(batchSize, collector); - TableProvider provider = this.tableProvider == null ?getTableProvider(tableProviderClazzName):this.tableProvider; + TableProvider provider; + if(this.tableProvider == null) { + provider = createTableProvider(tableProviderClazzName); + } else { + provider = this.tableProvider; + } + hbaseClient = new HBaseClient(provider, HBaseConfiguration.create(), tableName); } @@ -147,6 +157,8 @@ public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) { @Override public void execute(Tuple tuple) { + LOG.trace("Received a tuple."); + try { if (batchHelper.shouldHandle(tuple)) { save(tuple); @@ -179,12 +191,15 @@ private void save(Tuple tuple) { } batchHelper.addBatch(tuple); + LOG.debug("Added mutation to the batch; size={}", batchHelper.getBatchSize()); } /** * Flush all saved operations. */ private void flush() { + LOG.debug("About to flush a batch of {} mutation(s)", batchHelper.getBatchSize()); + this.hbaseClient.mutate(); batchHelper.ack(); } @@ -193,7 +208,8 @@ private void flush() { * Creates a TableProvider based on a class name. * @param connectorImpl The class name of a TableProvider */ - private static TableProvider getTableProvider(String connectorImpl) { + private static TableProvider createTableProvider(String connectorImpl) { + LOG.trace("Creating table provider; className={}", connectorImpl); // if class name not defined, use a reasonable default if(StringUtils.isEmpty(connectorImpl) || connectorImpl.charAt(0) == '$') { From ab3e6ad2cefb236a3de8a7c3dec3c100761ffc57 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Mon, 26 Mar 2018 11:34:13 -0400 Subject: [PATCH 18/20] Corrected cache key usage and logging --- .../profiler/DefaultMessageDistributor.java | 53 ++++++++++++++----- .../integration/ProfilerIntegrationTest.java | 4 +- .../configuration/profiler/ProfileConfig.java | 49 ++++++++++------- 3 files changed, 70 insertions(+), 36 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 ea5126f5c6..b4c18ec766 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 @@ -25,6 +25,7 @@ import com.google.common.cache.CacheBuilder; import com.google.common.cache.RemovalListener; import com.google.common.cache.RemovalNotification; +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; @@ -74,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 transient Cache activeCache; /** * A cache of expired profiles. @@ -85,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 transient Cache expiredCache; /** * Create a new message distributor. @@ -222,7 +223,7 @@ public List flushExpired() { * @param cache The cache to flush. * @return The measurements captured when flushing the profiles. */ - private List flushCache(Cache cache) { + private List flushCache(Cache cache) { List measurements = new ArrayList<>(); for(ProfileBuilder profileBuilder: cache.asMap().values()) { @@ -262,11 +263,19 @@ public ProfileBuilder getBuilder(MessageRoute route, Context context) throws Exe /** * Builds the key that is used to lookup the {@link ProfileBuilder} within the cache. * + *

The cache key is built using the hash codes of the profile and entity name. If the profile + * definition is ever changed, the same cache entry will not be reused. This ensures that no + * state can be carried over from the old definition into the new, which might result in an + * invalid profile measurement. + * * @param profile The profile definition. * @param entity The entity. */ - private String cacheKey(ProfileConfig profile, String entity) { - return format("%s:%s", profile, entity); + private int cacheKey(ProfileConfig profile, String entity) { + return new HashCodeBuilder(17, 37) + .append(profile) + .append(entity) + .hashCode(); } public DefaultMessageDistributor withPeriodDurationMillis(long periodDurationMillis) { @@ -281,29 +290,45 @@ 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 { @Override - public void onRemoval(RemovalNotification notification) { + public void onRemoval(RemovalNotification notification) { - String key = notification.getKey(); ProfileBuilder expired = notification.getValue(); + LOG.warn("Profile expired from active cache; profile={}, entity={}", + expired.getDefinition().getProfile(), + expired.getEntity()); - LOG.warn("Profile expired from active cache; key={}", key); - expiredCache.put(key, expired); + // add the profile to the expired cache + expiredCache.put(notification.getKey(), expired); } } /** * A listener that is notified when profiles expire from the active cache. */ - private class ExpiredCacheRemovalListener implements RemovalListener { + private class ExpiredCacheRemovalListener implements RemovalListener { @Override - public void onRemoval(RemovalNotification notification) { + public void onRemoval(RemovalNotification notification) { + + if(notification.wasEvicted()) { + + // the expired profile was NOT flushed in time + ProfileBuilder expired = notification.getValue(); + LOG.warn("Expired profile NOT flushed before removal, some state lost; profile={}, entity={}", + expired.getDefinition().getProfile(), + expired.getEntity()); - String key = notification.getKey(); - LOG.debug("Profile removed from expired cache; key={}", key); + } else { + + // the expired profile was flushed successfully + ProfileBuilder expired = notification.getValue(); + LOG.debug("Expired profile successfully flushed; profile={}, entity={}", + expired.getDefinition().getProfile(), + expired.getEntity()); + } } } } diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java index c48a3e911f..636af28fee 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java @@ -120,8 +120,8 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { private static final long windowLagMillis = TimeUnit.SECONDS.toMillis(5); private static final long windowDurationMillis = TimeUnit.SECONDS.toMillis(5); - private static final long periodDurationMillis = TimeUnit.SECONDS.toMillis(15); - private static final long profileTimeToLiveMillis = TimeUnit.SECONDS.toMillis(20); + private static final long periodDurationMillis = TimeUnit.SECONDS.toMillis(10); + private static final long profileTimeToLiveMillis = TimeUnit.SECONDS.toMillis(15); private static final long maxRoutesPerBolt = 100000; /** diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java index 6205fbf377..f5b46e6256 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/profiler/ProfileConfig.java @@ -18,6 +18,8 @@ package org.apache.metron.common.configuration.profiler; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.apache.commons.lang.builder.HashCodeBuilder; import java.io.Serializable; import java.util.ArrayList; @@ -225,32 +227,39 @@ public ProfileConfig withExpires(Long expiresDays) { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } - ProfileConfig that = (ProfileConfig) o; + if (o == null || getClass() != o.getClass()) { + return false; + } - if (profile != null ? !profile.equals(that.profile) : that.profile != null) return false; - if (foreach != null ? !foreach.equals(that.foreach) : that.foreach != null) return false; - if (onlyif != null ? !onlyif.equals(that.onlyif) : that.onlyif != null) return false; - if (init != null ? !init.equals(that.init) : that.init != null) return false; - if (update != null ? !update.equals(that.update) : that.update != null) return false; - if (groupBy != null ? !groupBy.equals(that.groupBy) : that.groupBy != null) return false; - if (result != null ? !result.equals(that.result) : that.result != null) return false; - return expires != null ? expires.equals(that.expires) : that.expires == null; + ProfileConfig that = (ProfileConfig) o; + return new EqualsBuilder() + .append(profile, that.profile) + .append(foreach, that.foreach) + .append(onlyif, that.onlyif) + .append(init, that.init) + .append(update, that.update) + .append(groupBy, that.groupBy) + .append(result, that.result) + .append(expires, that.expires) + .isEquals(); } @Override public int hashCode() { - int result1 = profile != null ? profile.hashCode() : 0; - result1 = 31 * result1 + (foreach != null ? foreach.hashCode() : 0); - result1 = 31 * result1 + (onlyif != null ? onlyif.hashCode() : 0); - result1 = 31 * result1 + (init != null ? init.hashCode() : 0); - result1 = 31 * result1 + (update != null ? update.hashCode() : 0); - result1 = 31 * result1 + (groupBy != null ? groupBy.hashCode() : 0); - result1 = 31 * result1 + (result != null ? result.hashCode() : 0); - result1 = 31 * result1 + (expires != null ? expires.hashCode() : 0); - return result1; + return new HashCodeBuilder(17, 37) + .append(profile) + .append(foreach) + .append(onlyif) + .append(init) + .append(update) + .append(groupBy) + .append(result) + .append(expires) + .toHashCode(); } @Override From 1cda9523fe65ce8ef9ec40305b873e6ed620cc78 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 23 Mar 2018 18:48:38 -0400 Subject: [PATCH 19/20] A separate timer thread is used to flush expired profiles. --- .../profiler/DefaultMessageDistributor.java | 1 - .../src/main/flux/profiler/remote.yaml | 2 + .../profiler/bolt/ProfileBuilderBolt.java | 149 +++++++--- .../profiler/bolt/ProfileSplitterBolt.java | 1 - .../zookeeper/percentiles/profiler.json | 12 - .../processing-time-test/profiler.json | 11 + .../zookeeper/readme-example-1/profiler.json | 17 -- .../zookeeper/readme-example-2/profiler.json | 18 -- .../zookeeper/readme-example-3/profiler.json | 11 - .../zookeeper/readme-example-4/profiler.json | 11 - .../profiler/bolt/ProfileBuilderBoltTest.java | 130 +++------ .../integration/ProfilerIntegrationTest.java | 270 +++++------------- .../ZKConfigurationsCacheIntegrationTest.java | 4 +- .../apache/metron/hbase/bolt/HBaseBolt.java | 2 +- 14 files changed, 231 insertions(+), 408 deletions(-) delete mode 100644 metron-analytics/metron-profiler/src/test/config/zookeeper/percentiles/profiler.json create mode 100644 metron-analytics/metron-profiler/src/test/config/zookeeper/processing-time-test/profiler.json delete mode 100644 metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-1/profiler.json delete mode 100644 metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-2/profiler.json delete mode 100644 metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-3/profiler.json delete mode 100644 metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-4/profiler.json 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 b4c18ec766..70f4228653 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 @@ -34,7 +34,6 @@ import java.lang.invoke.MethodHandles; import java.util.ArrayList; -import java.util.Collection; import java.util.List; import java.util.Optional; import java.util.concurrent.ExecutionException; diff --git a/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml b/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml index 83c9fde872..6ad007bf8b 100644 --- a/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml +++ b/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml @@ -160,6 +160,8 @@ bolts: args: [ref: "windowLag"] - name: "withMaxNumberOfRoutes" args: [${profiler.max.routes.per.bolt}] + - name: "withTimestampField" + args: ["timestamp"] - id: "hbaseBolt" className: "org.apache.metron.hbase.bolt.HBaseBolt" 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 ffe823ff2e..fb3d2d00d3 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 @@ -42,13 +42,13 @@ import org.apache.metron.stellar.dsl.Context; import org.apache.metron.zookeeper.SimpleEventListener; import org.apache.metron.zookeeper.ZKCache; -import org.apache.storm.Config; +import org.apache.storm.StormTimer; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; import org.apache.storm.topology.OutputFieldsDeclarer; import org.apache.storm.topology.base.BaseWindowedBolt; import org.apache.storm.tuple.Tuple; -import org.apache.storm.utils.TupleUtils; +import org.apache.storm.utils.Utils; import org.apache.storm.windowing.TupleWindow; import org.json.simple.JSONObject; import org.json.simple.parser.JSONParser; @@ -60,6 +60,8 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.function.Supplier; import static java.lang.String.format; import static org.apache.metron.profiler.bolt.ProfileSplitterBolt.ENTITY_TUPLE_FIELD; @@ -127,6 +129,9 @@ public class ProfileBuilderBolt extends BaseWindowedBolt implements Reloadable { /** * Distributes messages to the profile builders. + * + *

Since expired profiles are flushed on a separate thread, all access to this + * {@code MessageDistributor} needs to be protected. */ private MessageDistributor messageDistributor; @@ -145,9 +150,21 @@ public class ProfileBuilderBolt extends BaseWindowedBolt implements Reloadable { private List emitters; /** - * Signals when it is time to flush. + * Signals when it is time to flush the active profiles. + */ + private FlushSignal activeFlushSignal; + + /** + * A timer that flushes expired profiles on a regular interval. The expired profiles + * are flushed on a separate thread. + * + *

Flushing expired profiles ensures that any profiles that stop receiving messages + * for an extended period of time will continue to be flushed. + * + *

This introduces concurrency issues as the bolt is no longer single threaded. Due + * to this, all access to the {@code MessageDistributor} needs to be protected. */ - private FlushSignal flushSignal; + private StormTimer expiredFlushTimer; public ProfileBuilderBolt() { this.emitters = new ArrayList<>(); @@ -183,16 +200,26 @@ public void prepare(Map stormConf, TopologyContext context, OutputCollector coll this.parser = new JSONParser(); this.messageDistributor = new DefaultMessageDistributor(periodDurationMillis, profileTimeToLiveMillis, maxNumberOfRoutes); this.configurations = new ProfilerConfigurations(); - this.flushSignal = new FixedFrequencyFlushSignal(periodDurationMillis); + this.activeFlushSignal = new FixedFrequencyFlushSignal(periodDurationMillis); setupZookeeper(); + startExpiredFlushTimer(); } @Override public void cleanup() { - zookeeperCache.close(); - zookeeperClient.close(); + try { + zookeeperCache.close(); + zookeeperClient.close(); + expiredFlushTimer.close(); + + } catch(Throwable e) { + LOG.error("Exception when cleaning up", e); + } } + /** + * Setup connectivity to Zookeeper which provides the necessary configuration for the bolt. + */ private void setupZookeeper() { try { if (zookeeperClient == null) { @@ -248,18 +275,6 @@ public void declareOutputFields(OutputFieldsDeclarer declarer) { emitters.forEach(emitter -> emitter.declareOutputFields(declarer)); } - /** - * Defines the frequency at which the bolt will receive tick tuples. Tick tuples are - * used to control how often a profile is flushed. - */ - @Override - public Map getComponentConfiguration() { - - Map conf = super.getComponentConfiguration(); - conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, TimeUnit.MILLISECONDS.toSeconds(profileTimeToLiveMillis)); - return conf; - } - private Context getStellarContext() { Map global = getConfigurations().getGlobalConfig(); @@ -282,24 +297,12 @@ public void execute(TupleWindow window) { // handle each tuple in the window for(Tuple tuple : window.get()) { - - if(TupleUtils.isTick(tuple)) { - handleTick(); - - } else { - handleMessage(tuple); - } + handleMessage(tuple); } - // time to flush? - if(flushSignal.isTimeToFlush()) { - flushSignal.reset(); - - // flush the active profiles - List measurements = messageDistributor.flush(); - emitMeasurements(measurements); - - LOG.debug("Flushed active profiles and found {} measurement(s).", measurements.size()); + // time to flush active profiles? + if(activeFlushSignal.isTimeToFlush()) { + flushActive(); } } catch (Throwable e) { @@ -310,17 +313,37 @@ public void execute(TupleWindow window) { } /** - * Flush all expired profiles when a 'tick' is received. + * Flush all active profiles. + */ + protected void flushActive() { + activeFlushSignal.reset(); + + // flush the active profiles + List measurements; + synchronized(messageDistributor) { + measurements = messageDistributor.flush(); + emitMeasurements(measurements); + } + + LOG.debug("Flushed active profiles and found {} measurement(s).", measurements.size()); + + } + + /** + * Flushes all expired profiles. * - * If a profile has not received a message for an extended period of time then it is + *

If a profile has not received a message for an extended period of time then it is * marked as expired. Periodically we need to flush these expired profiles to ensure * that their state is not lost. */ - private void handleTick() { + protected void flushExpired() { // flush the expired profiles - List measurements = messageDistributor.flushExpired(); - emitMeasurements(measurements); + List measurements; + synchronized (messageDistributor) { + measurements = messageDistributor.flushExpired(); + emitMeasurements(measurements); + } LOG.debug("Flushed expired profiles and found {} measurement(s).", measurements.size()); } @@ -339,11 +362,13 @@ private void handleMessage(Tuple input) { Long timestamp = getField(TIMESTAMP_TUPLE_FIELD, input, Long.class); // keep track of time - flushSignal.update(timestamp); + activeFlushSignal.update(timestamp); // distribute the message MessageRoute route = new MessageRoute(definition, entity); - messageDistributor.distribute(message, timestamp, route, getStellarContext()); + synchronized (messageDistributor) { + messageDistributor.distribute(message, timestamp, route, getStellarContext()); + } LOG.debug("Message distributed: profile={}, entity={}, timestamp={}", definition.getProfile(), entity, timestamp); } @@ -395,10 +420,46 @@ private T getField(String fieldName, Tuple tuple, Class clazz) { return value; } + /** + * Converts milliseconds to seconds and handles an ugly cast. + * + * @param millis Duration in milliseconds. + * @return Duration in seconds. + */ + private int toSeconds(long millis) { + return (int) TimeUnit.MILLISECONDS.toSeconds(millis); + } + + /** + * Creates a timer that regularly flushes expired profiles on a separate thread. + */ + private void startExpiredFlushTimer() { + + expiredFlushTimer = createTimer("flush-expired-profiles-timer"); + expiredFlushTimer.scheduleRecurring(0, toSeconds(profileTimeToLiveMillis), () -> flushExpired()); + } + + /** + * Creates a timer that can execute a task on a fixed interval. + * + *

If the timer encounters an exception, the entire process will be killed. + * + * @param name The name of the timer. + * @return The timer. + */ + private StormTimer createTimer(String name) { + + return new StormTimer(name, (thread, exception) -> { + String msg = String.format("Unexpected exception in timer task; timer=%s", name); + LOG.error(msg, exception); + Utils.exitProcess(1, msg); + }); + } + @Override public BaseWindowedBolt withTumblingWindow(BaseWindowedBolt.Duration duration) { - // need to capture the window duration for setting the flush count down + // need to capture the window duration to validate it along with other profiler settings this.windowDurationMillis = duration.value; return super.withTumblingWindow(duration); } @@ -464,7 +525,7 @@ public ProfileBuilderBolt withMaxNumberOfRoutes(long maxNumberOfRoutes) { } public ProfileBuilderBolt withFlushSignal(FlushSignal flushSignal) { - this.flushSignal = flushSignal; + this.activeFlushSignal = flushSignal; return this; } 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 4e62eee135..a92a432508 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 @@ -21,7 +21,6 @@ package org.apache.metron.profiler.bolt; import org.apache.metron.common.bolt.ConfiguredProfilerBolt; -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.MessageRoute; diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/percentiles/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/percentiles/profiler.json deleted file mode 100644 index 8a54834c89..0000000000 --- a/metron-analytics/metron-profiler/src/test/config/zookeeper/percentiles/profiler.json +++ /dev/null @@ -1,12 +0,0 @@ -{ - "profiles": [ - { - "profile": "percentiles", - "foreach": "ip_src_addr", - "onlyif": "protocol == 'HTTP'", - "init": { "s": "STATS_INIT(100)" }, - "update": { "s": "STATS_ADD(s, length)" }, - "result": "STATS_PERCENTILE(s, 0.7)" - } - ] -} \ No newline at end of file diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/processing-time-test/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/processing-time-test/profiler.json new file mode 100644 index 0000000000..e75ec0fede --- /dev/null +++ b/metron-analytics/metron-profiler/src/test/config/zookeeper/processing-time-test/profiler.json @@ -0,0 +1,11 @@ +{ + "profiles": [ + { + "profile": "processing-time-test", + "foreach": "ip_src_addr", + "init": { "counter": "0" }, + "update": { "counter": "counter + 1" }, + "result": "counter" + } + ] +} \ No newline at end of file diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-1/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-1/profiler.json deleted file mode 100644 index 96c60a186b..0000000000 --- a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-1/profiler.json +++ /dev/null @@ -1,17 +0,0 @@ -{ - "profiles": [ - { - "profile": "example1", - "foreach": "ip_src_addr", - "onlyif": "protocol == 'HTTP'", - "init": { - "total_bytes": 0.0 - }, - "update": { - "total_bytes": "total_bytes + bytes_in" - }, - "result": "total_bytes", - "expires": 30 - } - ] -} \ No newline at end of file diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-2/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-2/profiler.json deleted file mode 100644 index e5d8f3978b..0000000000 --- a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-2/profiler.json +++ /dev/null @@ -1,18 +0,0 @@ -{ - "profiles": [ - { - "profile": "example2", - "foreach": "ip_src_addr", - "onlyif": "protocol == 'DNS' or protocol == 'HTTP'", - "init": { - "num_dns": 1.0, - "num_http": 1.0 - }, - "update": { - "num_dns": "num_dns + (if protocol == 'DNS' then 1 else 0)", - "num_http": "num_http + (if protocol == 'HTTP' then 1 else 0)" - }, - "result": "num_dns / num_http" - } - ] -} \ No newline at end of file diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-3/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-3/profiler.json deleted file mode 100644 index 67cdefdea2..0000000000 --- a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-3/profiler.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "profiles": [ - { - "profile": "example3", - "foreach": "ip_src_addr", - "onlyif": "protocol == 'HTTP'", - "update": { "s": "STATS_ADD(s, length)" }, - "result": "STATS_MEAN(s)" - } - ] -} \ No newline at end of file diff --git a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-4/profiler.json b/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-4/profiler.json deleted file mode 100644 index b003ce02be..0000000000 --- a/metron-analytics/metron-profiler/src/test/config/zookeeper/readme-example-4/profiler.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "profiles": [ - { - "profile": "example4", - "foreach": "ip_src_addr", - "onlyif": "protocol == 'HTTP'", - "update": { "s": "STATS_ADD(s, length)" }, - "result": "s" - } - ] -} \ No newline at end of file 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 78e20e0221..3d009fb46b 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 @@ -48,7 +48,6 @@ import java.util.stream.Collectors; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; @@ -67,6 +66,7 @@ public class ProfileBuilderBoltTest extends BaseBoltTest { private ProfileConfig profile2; private ProfileMeasurementEmitter emitter; private ManualFlushSignal flushSignal; + private ProfileMeasurement measurement; @Before public void setup() throws Exception { @@ -95,6 +95,12 @@ public void setup() throws Exception { .withUpdate(Collections.singletonMap("x", "x + 1")) .withResult("x"); + measurement = new ProfileMeasurement() + .withEntity("entity1") + .withProfileName("profile1") + .withPeriod(1000, 500, TimeUnit.MILLISECONDS) + .withProfileValue(22); + flushSignal = new ManualFlushSignal(); flushSignal.setFlushNow(false); } @@ -127,23 +133,16 @@ public void testExtractMessage() throws Exception { /** * If the {@code FlushSignal} tells the bolt to flush, it should flush the {@code MessageDistributor} - * and emit the {@code ProfileMeasurement} values. + * and emit the {@code ProfileMeasurement} values from all active profiles. */ @Test - public void testEmitWhenFlush() throws Exception { + public void testFlushActiveProfiles() throws Exception { ProfileBuilderBolt bolt = createBolt(); - // create a profile measurement - ProfileMeasurement m = new ProfileMeasurement() - .withEntity("entity1") - .withProfileName("profile1") - .withPeriod(1000, 500, TimeUnit.MILLISECONDS) - .withProfileValue(22); - // create a mock that returns the profile measurement above MessageDistributor distributor = mock(MessageDistributor.class); - when(distributor.flush()).thenReturn(Collections.singletonList(m)); + when(distributor.flush()).thenReturn(Collections.singletonList(measurement)); bolt.withMessageDistributor(distributor); // signal the bolt to flush @@ -157,30 +156,23 @@ public void testEmitWhenFlush() throws Exception { // a profile measurement should be emitted by the bolt List measurements = getProfileMeasurements(outputCollector, 1); assertEquals(1, measurements.size()); - assertEquals(m, measurements.get(0)); + assertEquals(measurement, measurements.get(0)); } /** * If the {@code FlushSignal} tells the bolt NOT to flush, nothing should be emitted. */ @Test - public void testDoNotEmitWhenNoFlush() throws Exception { + public void testDoNotFlushActiveProfiles() throws Exception { ProfileBuilderBolt bolt = createBolt(); - // create a profile measurement - ProfileMeasurement m = new ProfileMeasurement() - .withEntity("entity1") - .withProfileName("profile1") - .withPeriod(1000, 500, TimeUnit.MILLISECONDS) - .withProfileValue(22); - - // create a mock that returns the profile measurement above + // create a mock where flush() returns the profile measurement above MessageDistributor distributor = mock(MessageDistributor.class); - when(distributor.flush()).thenReturn(Collections.singletonList(m)); + when(distributor.flush()).thenReturn(Collections.singletonList(measurement)); bolt.withMessageDistributor(distributor); - // no flush signal + // there is no flush signal flushSignal.setFlushNow(false); // execute the bolt @@ -192,6 +184,29 @@ public void testDoNotEmitWhenNoFlush() throws Exception { getProfileMeasurements(outputCollector, 0); } + /** + * Expired profiles should be flushed regularly, even if no input telemetry + * has been received. + */ + @Test + public void testFlushExpiredProfiles() throws Exception { + + ProfileBuilderBolt bolt = createBolt(); + + // create a mock where flushExpired() returns the profile measurement above + MessageDistributor distributor = mock(MessageDistributor.class); + when(distributor.flushExpired()).thenReturn(Collections.singletonList(measurement)); + bolt.withMessageDistributor(distributor); + + // execute test by flushing expired profiles. this is normally triggered by a timer task. + bolt.flushExpired(); + + // a profile measurement should be emitted by the bolt + List measurements = getProfileMeasurements(outputCollector, 1); + assertEquals(1, measurements.size()); + assertEquals(measurement, measurements.get(0)); + } + /** * A {@link ProfileMeasurement} is built for each profile/entity pair. The measurement should be emitted to each * destination defined by the profile. By default, a profile uses both Kafka and HBase as destinations. @@ -232,61 +247,6 @@ public void testEmitters() throws Exception { verify(outputCollector, times(1)).emit(eq("destination3"), any()); } - @Test - public void testFlushExpiredWithTick() throws Exception { - - ProfileBuilderBolt bolt = createBolt(); - - // create a mock - MessageDistributor distributor = mock(MessageDistributor.class); - bolt.withMessageDistributor(distributor); - - // tell the bolt to flush on the first window - flushSignal.setFlushNow(true); - - // execute the bolt; include a tick tuple in the window - Tuple tuple1 = createTuple("entity", message1, profile1, 100000000L); - TupleWindow tupleWindow = createWindow(tuple1, mockTickTuple()); - bolt.execute(tupleWindow); - - // ensure the expired profiles were flushed when the tick tuple was received - verify(distributor).flushExpired(); - } - - @Test - public void testFlushExpiredWithNoTick() throws Exception { - - ProfileBuilderBolt bolt = createBolt(); - - // create a mock - MessageDistributor distributor = mock(MessageDistributor.class); - bolt.withMessageDistributor(distributor); - - // tell the bolt to flush on the first window - flushSignal.setFlushNow(true); - - // execute the bolt; NO tick tuple - Tuple tuple1 = createTuple("entity", message1, profile1, 100000000L); - TupleWindow tupleWindow = createWindow(tuple1); - bolt.execute(tupleWindow); - - // there was no tick tuple; the expired profiles should NOT have been flushed - verify(distributor, times(0)).flushExpired(); - } - - /** - * Creates a mock tick tuple to use for testing. - * @return A mock tick tuple. - */ - private Tuple mockTickTuple() { - - Tuple tuple = mock(Tuple.class); - when(tuple.getSourceComponent()).thenReturn("__system"); - when(tuple.getSourceStreamId()).thenReturn("__tick"); - - return tuple; - } - /** * Retrieves the ProfileMeasurement(s) (if any) that have been emitted. * @@ -334,18 +294,6 @@ private Tuple createTuple(String entity, JSONObject message, ProfileConfig profi */ private ProfileBuilderBolt createBolt() throws IOException { - return createBolt(30, TimeUnit.SECONDS); - } - - /** - * Create a ProfileBuilderBolt to test. - * - * @param windowDuration The event window duration. - * @param windowDurationUnits The units of the event window duration. - * @return A {@link ProfileBuilderBolt} to test. - */ - private ProfileBuilderBolt createBolt(int windowDuration, TimeUnit windowDurationUnits) throws IOException { - // defines the zk configurations accessible from the bolt ProfilerConfigurations configurations = new ProfilerConfigurations(); configurations.updateGlobalConfig(Collections.emptyMap()); @@ -359,7 +307,7 @@ private ProfileBuilderBolt createBolt(int windowDuration, TimeUnit windowDuratio .withEmitter(emitter) .withProfilerConfigurations(configurations) .withPeriodDuration(1, TimeUnit.MINUTES) - .withTumblingWindow(new BaseWindowedBolt.Duration(windowDuration, windowDurationUnits)); + .withTumblingWindow(new BaseWindowedBolt.Duration(30, TimeUnit.SECONDS)); bolt.prepare(new HashMap<>(), topologyContext, outputCollector); // set the flush signal AFTER calling 'prepare' diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java index 636af28fee..8f5ced3a88 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java @@ -20,9 +20,6 @@ package org.apache.metron.profiler.integration; -import com.google.common.base.Joiner; -import org.adrianwalker.multilinestring.Multiline; -import org.apache.commons.math.util.MathUtils; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.util.Bytes; @@ -41,10 +38,8 @@ 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.statistics.OnlineStatisticsProvider; import org.junit.After; import org.junit.AfterClass; -import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -61,6 +56,7 @@ import static com.google.code.tempusfugit.temporal.WaitFor.waitOrTimeout; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** * An integration test of the Profiler topology. @@ -70,247 +66,103 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { private static final String TEST_RESOURCES = "../../metron-analytics/metron-profiler/src/test"; private static final String FLUX_PATH = "../metron-profiler/src/main/flux/profiler/remote.yaml"; - /** - * { - * "ip_src_addr": "10.0.0.1", - * "protocol": "HTTPS", - * "length": 10, - * "bytes_in": 234 - * } - */ - @Multiline - private static String message1; - - /** - * { - * "ip_src_addr": "10.0.0.2", - * "protocol": "HTTP", - * "length": 20, - * "bytes_in": 390 - * } - */ - @Multiline - private static String message2; - - /** - * { - * "ip_src_addr": "10.0.0.3", - * "protocol": "DNS", - * "length": 30, - * "bytes_in": 560 - * } - */ - @Multiline - private static String message3; - - private static ColumnBuilder columnBuilder; - private static ZKServerComponent zkComponent; - private static FluxTopologyComponent fluxComponent; - private static KafkaComponent kafkaComponent; - private static ConfigUploadComponent configUploadComponent; - private static ComponentRunner runner; - private static MockHTable profilerTable; + public static final long startAt = 10; + public static final String entity = "10.0.0.1"; private static final String tableName = "profiler"; private static final String columnFamily = "P"; - private static final double epsilon = 0.001; private static final String inputTopic = Constants.INDEXING_TOPIC; private static final String outputTopic = "profiles"; private static final int saltDivisor = 10; - private static final long windowLagMillis = TimeUnit.SECONDS.toMillis(5); + private static final long windowLagMillis = TimeUnit.SECONDS.toMillis(1); private static final long windowDurationMillis = TimeUnit.SECONDS.toMillis(5); private static final long periodDurationMillis = TimeUnit.SECONDS.toMillis(10); private static final long profileTimeToLiveMillis = TimeUnit.SECONDS.toMillis(15); private static final long maxRoutesPerBolt = 100000; - /** - * Tests the first example contained within the README. - */ - @Test - public void testExample1() throws Exception { - - uploadConfig(TEST_RESOURCES + "/config/zookeeper/readme-example-1"); - - // start the topology and write test messages to kafka - fluxComponent.submitTopology(); - kafkaComponent.writeMessages(inputTopic, message1, message1, message1); - kafkaComponent.writeMessages(inputTopic, message2, message2, message2); - kafkaComponent.writeMessages(inputTopic, message3, message3, message3); - - // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(180))); - - // verify - only 10.0.0.2 sends 'HTTP', thus there should be only 1 value - List actuals = read(profilerTable.getPutLog(), columnFamily, - columnBuilder.getColumnQualifier("value"), Double.class); - - // verify - there are 3 'HTTP' each with 390 bytes - Assert.assertTrue(actuals.stream().anyMatch(val -> - MathUtils.equals(390.0 * 3, val, epsilon) - )); - } - - /** - * Tests the second example contained within the README. - */ - @Test - public void testExample2() throws Exception { - - uploadConfig(TEST_RESOURCES + "/config/zookeeper/readme-example-2"); - - // start the topology and write test messages to kafka - fluxComponent.submitTopology(); - kafkaComponent.writeMessages(inputTopic, message1, message1, message1); - kafkaComponent.writeMessages(inputTopic, message2, message2, message2); - kafkaComponent.writeMessages(inputTopic, message3, message3, message3); - - // expect 2 values written by the profile; one for 10.0.0.2 and another for 10.0.0.3 - final int expected = 2; - - // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() >= expected, - timeout(seconds(90))); - - // verify - expect 2 results as 2 hosts involved; 10.0.0.2 sends 'HTTP' and 10.0.0.3 send 'DNS' - List actuals = read(profilerTable.getPutLog(), columnFamily, - columnBuilder.getColumnQualifier("value"), Double.class); - - // verify - 10.0.0.3 -> 1/4 - Assert.assertTrue( "Could not find a value near 1/4. Actual values read are are: " + Joiner.on(",").join(actuals), - actuals.stream().anyMatch(val -> MathUtils.equals(val, 1.0/4.0, epsilon) - )); - - // verify - 10.0.0.2 -> 4/1 - Assert.assertTrue("Could not find a value near 4. Actual values read are are: " + Joiner.on(",").join(actuals), - actuals.stream().anyMatch(val -> MathUtils.equals(val, 4.0/1.0, epsilon) - )); - } - - /** - * Tests the third example contained within the README. - */ - @Test - public void testExample3() throws Exception { - - uploadConfig(TEST_RESOURCES + "/config/zookeeper/readme-example-3"); + private static ColumnBuilder columnBuilder; + private static ZKServerComponent zkComponent; + private static FluxTopologyComponent fluxComponent; + private static KafkaComponent kafkaComponent; + private static ConfigUploadComponent configUploadComponent; + private static ComponentRunner runner; + private static MockHTable profilerTable; - // start the topology and write test messages to kafka - fluxComponent.submitTopology(); - kafkaComponent.writeMessages(inputTopic, message1, message1, message1); - kafkaComponent.writeMessages(inputTopic, message2, message2, message2); - kafkaComponent.writeMessages(inputTopic, message3, message3, message3); - - // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(90))); - - // verify - only 10.0.0.2 sends 'HTTP', thus there should be only 1 value - List actuals = read(profilerTable.getPutLog(), columnFamily, - columnBuilder.getColumnQualifier("value"), Double.class); - - // verify - there are 5 'HTTP' messages each with a length of 20, thus the average should be 20 - Assert.assertTrue("Could not find a value near 20. Actual values read are are: " + Joiner.on(",").join(actuals), - actuals.stream().anyMatch(val -> MathUtils.equals(val, 20.0, epsilon) - )); - } + private static String message1; + private static String message2; + private static String message3; /** - * Tests the fourth example contained within the README. + * The Profiler can generate profiles based on processing time. With processing time, + * the Profiler builds profiles based on when the telemetry was processed. + * + *

Not defining a 'timestampField' within the Profiler configuration tells the Profiler + * to use processing time. */ @Test - public void testExample4() throws Exception { + public void testProcessingTime() throws Exception { - uploadConfig(TEST_RESOURCES + "/config/zookeeper/readme-example-4"); + // upload the config to zookeeper + uploadConfig(TEST_RESOURCES + "/config/zookeeper/processing-time-test"); // start the topology and write test messages to kafka fluxComponent.submitTopology(); - kafkaComponent.writeMessages(inputTopic, message1, message1, message1); - kafkaComponent.writeMessages(inputTopic, message2, message2, message2); - kafkaComponent.writeMessages(inputTopic, message3, message3, message3); - - // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(90))); - - // verify - only 10.0.0.2 sends 'HTTP', thus there should be only 1 value - byte[] column = columnBuilder.getColumnQualifier("value"); - List actuals = read(profilerTable.getPutLog(), columnFamily, column, OnlineStatisticsProvider.class); - - // verify - there are 5 'HTTP' messages each with a length of 20, thus the average should be 20 - Assert.assertTrue("Could not find a value near 20. Actual values read are are: " + Joiner.on(",").join(actuals), - actuals.stream().anyMatch(val -> MathUtils.equals(val.getMean(), 20.0, epsilon) - )); - } - @Test - public void testPercentiles() throws Exception { - - uploadConfig(TEST_RESOURCES + "/config/zookeeper/percentiles"); + // the messages that will be applied to the profile + kafkaComponent.writeMessages(inputTopic, message1); + kafkaComponent.writeMessages(inputTopic, message2); + kafkaComponent.writeMessages(inputTopic, message3); - // start the topology and write test messages to kafka - fluxComponent.submitTopology(); - kafkaComponent.writeMessages(inputTopic, message1, message1, message1); - kafkaComponent.writeMessages(inputTopic, message2, message2, message2); - kafkaComponent.writeMessages(inputTopic, message3, message3, message3); + // storm needs at least one message to close its event window + int attempt = 0; + while(profilerTable.getPutLog().size() == 0 && attempt++ < 10) { - // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(90))); + // sleep, at least beyond the current window + Thread.sleep(windowDurationMillis + windowLagMillis); - List actuals = read(profilerTable.getPutLog(), columnFamily, - columnBuilder.getColumnQualifier("value"), Double.class); + // send another message to help close the current event window + kafkaComponent.writeMessages(inputTopic, message2); + } - // verify - the 70th percentile of x3, 20s = 20.0 - Assert.assertTrue("Could not find a value near 20. Actual values read are are: " + Joiner.on(",").join(actuals), - actuals.stream().anyMatch(val -> MathUtils.equals(val, 20.0, epsilon))); + // validate what was flushed + List actuals = read( + profilerTable.getPutLog(), + columnFamily, + columnBuilder.getColumnQualifier("value"), + Integer.class); + assertEquals(1, actuals.size()); + assertTrue(actuals.get(0) >= 3); } /** - * The Profiler can optionally perform event time processing. With event time processing, + * The Profiler can generate profiles using event time. With event time processing, * the Profiler uses timestamps contained in the source telemetry. * *

Defining a 'timestampField' within the Profiler configuration tells the Profiler * from which field the timestamp should be extracted. */ @Test - public void testEventTimeProcessing() throws Exception { - - // constants used for the test - final long startAt = 10; - final String entity = "10.0.0.1"; - final String profileName = "event-time-test"; - - // create some messages that contain a timestamp - a really old timestamp; close to 1970 - String message1 = new MessageBuilder() - .withField("ip_src_addr", entity) - .withField("timestamp", startAt) - .build() - .toJSONString(); - - String message2 = new MessageBuilder() - .withField("ip_src_addr", entity) - .withField("timestamp", startAt + 100) - .build() - .toJSONString(); + public void testEventTime() throws Exception { + // upload the profiler config to zookeeper uploadConfig(TEST_RESOURCES + "/config/zookeeper/event-time-test"); // start the topology and write test messages to kafka fluxComponent.submitTopology(); - kafkaComponent.writeMessages(inputTopic, message1, message2); + kafkaComponent.writeMessages(inputTopic, message1); + kafkaComponent.writeMessages(inputTopic, message2); + kafkaComponent.writeMessages(inputTopic, message3); - // verify - ensure the profile is being persisted - waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, - timeout(seconds(90))); + // wait until the profile is flushed + waitOrTimeout(() -> profilerTable.getPutLog().size() > 0, timeout(seconds(90))); List puts = profilerTable.getPutLog(); assertEquals(1, puts.size()); // inspect the row key to ensure the profiler used event time correctly. the timestamp // embedded in the row key should match those in the source telemetry - byte[] expectedRowKey = generateExpectedRowKey(profileName, entity, startAt); + byte[] expectedRowKey = generateExpectedRowKey("event-time-test", entity, startAt); byte[] actualRowKey = puts.get(0).getRow(); String msg = String.format("expected '%s', got '%s'", new String(expectedRowKey, "UTF-8"), @@ -364,6 +216,26 @@ private List read(List puts, String family, byte[] qualifier, Class< @BeforeClass public static void setupBeforeClass() throws UnableToStartException { + + // create some messages that contain a timestamp - a really old timestamp; close to 1970 + message1 = new MessageBuilder() + .withField("ip_src_addr", entity) + .withField("timestamp", startAt) + .build() + .toJSONString(); + + message2 = new MessageBuilder() + .withField("ip_src_addr", entity) + .withField("timestamp", startAt + 100) + .build() + .toJSONString(); + + message3 = new MessageBuilder() + .withField("ip_src_addr", entity) + .withField("timestamp", startAt + (windowDurationMillis * 2)) + .build() + .toJSONString(); + columnBuilder = new ValueOnlyColumnBuilder(columnFamily); // storm topology properties diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/zookeeper/ZKConfigurationsCacheIntegrationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/zookeeper/ZKConfigurationsCacheIntegrationTest.java index ec4a98a2c0..5240d7ac18 100644 --- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/zookeeper/ZKConfigurationsCacheIntegrationTest.java +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/zookeeper/ZKConfigurationsCacheIntegrationTest.java @@ -154,7 +154,7 @@ public void setup() throws Exception { } { //profiler - byte[] config = IOUtils.toByteArray(new FileInputStream(new File(profilerDir, "/readme-example-1/profiler.json"))); + byte[] config = IOUtils.toByteArray(new FileInputStream(new File(profilerDir, "/event-time-test/profiler.json"))); ConfigurationsUtils.writeProfilerConfigToZookeeper( config, client); } { @@ -284,7 +284,7 @@ public void validateBaseWrite() throws Exception { } //profiler { - File inFile = new File(profilerDir, "/readme-example-1/profiler.json"); + File inFile = new File(profilerDir, "/event-time-test/profiler.json"); ProfilerConfig expectedConfig = JSONUtils.INSTANCE.load(inFile, ProfilerConfig.class); ProfilerConfigurations config = cache.get( ProfilerConfigurations.class); assertEventually(() -> Assert.assertEquals(expectedConfig, config.getProfilerConfig())); diff --git a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java index 479cdddfac..6953b188a6 100644 --- a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java +++ b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/bolt/HBaseBolt.java @@ -24,7 +24,7 @@ import java.lang.reflect.InvocationTargetException; import java.util.Map; import java.util.Optional; -import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.client.Durability; import org.apache.metron.hbase.HTableProvider; From cf7ee194c92a49c7fe7e33a81ea4813f98210d1a Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Mon, 2 Apr 2018 14:46:02 -0400 Subject: [PATCH 20/20] Do not want to commit debug logging --- .../metron-profiler/src/test/resources/log4j.properties | 3 --- 1 file changed, 3 deletions(-) diff --git a/metron-analytics/metron-profiler/src/test/resources/log4j.properties b/metron-analytics/metron-profiler/src/test/resources/log4j.properties index 4fadbb3865..541f368c4d 100644 --- a/metron-analytics/metron-profiler/src/test/resources/log4j.properties +++ b/metron-analytics/metron-profiler/src/test/resources/log4j.properties @@ -21,9 +21,6 @@ # Root logger option log4j.rootLogger=ERROR, stdout -log4j.logger.org.apache.metron.profiler=DEBUG -log4j.logger.org.apache.storm.windowing=ALL - # Direct log messages to stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.Target=System.out