From bd484b5a0bede9214ec39a3c6f4192d9e948ed35 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Fri, 23 Jun 2017 09:15:02 -0400 Subject: [PATCH 01/43] first pass. Still need to fix storm kafka interaction + possibly cleanup --- .../maas/service/MaasIntegrationTest.java | 27 +++---- .../integration/ConfigUploadComponent.java | 17 ++++- .../integration/ProfilerIntegrationTest.java | 63 ++++++++++------ .../components/ElasticSearchComponent.java | 18 +++++ .../metron/integration/ComponentRunner.java | 28 +++++++- .../metron/integration/InMemoryComponent.java | 5 +- .../components/FluxTopologyComponent.java | 30 ++++++++ .../components/KafkaComponent.java | 71 ++++++++++++++++++- .../components/ZKServerComponent.java | 18 ++++- .../KafkaFunctionsIntegrationTest.java | 8 ++- .../components/ParserTopologyComponent.java | 7 ++ .../org/apache/metron/pcap/query/PcapCli.java | 2 +- .../integration/components/SolrComponent.java | 9 +++ .../storm/kafka/flux/StormKafkaSpout.java | 8 +++ 14 files changed, 268 insertions(+), 43 deletions(-) diff --git a/metron-analytics/metron-maas-service/src/test/java/org/apache/metron/maas/service/MaasIntegrationTest.java b/metron-analytics/metron-maas-service/src/test/java/org/apache/metron/maas/service/MaasIntegrationTest.java index 221a840087..2c224edb08 100644 --- a/metron-analytics/metron-maas-service/src/test/java/org/apache/metron/maas/service/MaasIntegrationTest.java +++ b/metron-analytics/metron-maas-service/src/test/java/org/apache/metron/maas/service/MaasIntegrationTest.java @@ -33,17 +33,14 @@ import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; -import org.apache.curator.test.TestingServer; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.util.JarFinder; import org.apache.hadoop.util.Shell; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.MiniYARNCluster; import org.apache.metron.integration.ComponentRunner; import org.apache.metron.integration.components.YarnComponent; import org.apache.metron.integration.components.ZKServerComponent; @@ -57,19 +54,20 @@ import org.apache.metron.test.utils.UnitTestHelper; import org.apache.zookeeper.KeeperException; import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; -import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; public class MaasIntegrationTest { private static final Log LOG = LogFactory.getLog(MaasIntegrationTest.class); - private CuratorFramework client; - private ComponentRunner runner; - private YarnComponent yarnComponent; - private ZKServerComponent zkServerComponent; - @Before - public void setup() throws Exception { + private static CuratorFramework client; + private static ComponentRunner runner; + private static YarnComponent yarnComponent; + private static ZKServerComponent zkServerComponent; + @BeforeClass + public static void setupBeforeClass() throws Exception { UnitTestHelper.setJavaLoggingLevel(Level.SEVERE); LOG.info("Starting up YARN cluster"); @@ -92,14 +90,19 @@ public void setup() throws Exception { client.start(); } - @After - public void tearDown(){ + @AfterClass + public static void tearDownAfterClass(){ if(client != null){ client.close(); } runner.stop(); } + @After + public void tearDown() { + runner.reset(); + } + @Test(timeout=900000) public void testMaaSWithDomain() throws Exception { testDSShell(true); diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ConfigUploadComponent.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ConfigUploadComponent.java index b3fc6c773d..8cb20fd423 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ConfigUploadComponent.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ConfigUploadComponent.java @@ -20,6 +20,7 @@ package org.apache.metron.profiler.integration; import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.imps.CuratorFrameworkState; import org.apache.metron.integration.InMemoryComponent; import org.apache.metron.integration.UnableToStartException; import org.apache.metron.integration.components.ZKServerComponent; @@ -56,6 +57,18 @@ public void stop() { // nothing to do } + public void update() + throws UnableToStartException { +// withTopologyProperties(topologyProperties); +// withGlobalConfiguration(globalConfiguration); +// withProfilerConfiguration(profilerConfiguration); + try { + upload(); + } catch (Exception e) { + throw new UnableToStartException(e.getMessage(), e); + } + } + /** * Uploads configuration to Zookeeper. * @throws Exception @@ -63,7 +76,9 @@ public void stop() { private void upload() throws Exception { final String zookeeperUrl = topologyProperties.getProperty(ZKServerComponent.ZOOKEEPER_PROPERTY); try(CuratorFramework client = getClient(zookeeperUrl)) { - client.start(); + if(client.getState() != CuratorFrameworkState.STARTED) { + client.start(); + } uploadGlobalConfig(client); uploadProfilerConfig(client); } 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 7591300da2..f158d7a366 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 @@ -33,6 +33,7 @@ import org.apache.metron.hbase.TableProvider; 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; @@ -41,7 +42,9 @@ import org.apache.metron.statistics.OnlineStatisticsProvider; import org.apache.metron.test.mock.MockHTable; import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; +import org.junit.BeforeClass; import org.junit.Test; import java.io.File; @@ -76,7 +79,7 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { * } */ @Multiline - private String message1; + private static String message1; /** * { @@ -87,7 +90,7 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { * } */ @Multiline - private String message2; + private static String message2; /** * { @@ -98,15 +101,16 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { * } */ @Multiline - private String message3; + private static String message3; - private ColumnBuilder columnBuilder; - private ZKServerComponent zkComponent; - private FluxTopologyComponent fluxComponent; - private KafkaComponent kafkaComponent; - private List input; - private ComponentRunner runner; - private MockHTable profilerTable; + private static ColumnBuilder columnBuilder; + private static ZKServerComponent zkComponent; + private static FluxTopologyComponent fluxComponent; + private static KafkaComponent kafkaComponent; + private static ConfigUploadComponent configUploadComponent; + private static List input; + private static ComponentRunner runner; + private static MockHTable profilerTable; private static final String tableName = "profiler"; private static final String columnFamily = "P"; @@ -133,7 +137,7 @@ public HTableInterface getTable(Configuration config, String tableName) throws I @Test public void testExample1() throws Exception { - setup(TEST_RESOURCES + "/config/zookeeper/readme-example-1"); + update(TEST_RESOURCES + "/config/zookeeper/readme-example-1"); // start the topology and write test messages to kafka fluxComponent.submitTopology(); @@ -145,6 +149,7 @@ public void testExample1() throws Exception { // 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); + System.out.println("Actuals are: " + actuals); // verify - there are 5 'HTTP' each with 390 bytes Assert.assertTrue(actuals.stream().anyMatch(val -> @@ -158,7 +163,7 @@ public void testExample1() throws Exception { @Test public void testExample2() throws Exception { - setup(TEST_RESOURCES + "/config/zookeeper/readme-example-2"); + update(TEST_RESOURCES + "/config/zookeeper/readme-example-2"); // start the topology and write test messages to kafka fluxComponent.submitTopology(); @@ -191,7 +196,7 @@ public void testExample2() throws Exception { @Test public void testExample3() throws Exception { - setup(TEST_RESOURCES + "/config/zookeeper/readme-example-3"); + update(TEST_RESOURCES + "/config/zookeeper/readme-example-3"); // start the topology and write test messages to kafka fluxComponent.submitTopology(); @@ -216,7 +221,7 @@ public void testExample3() throws Exception { @Test public void testExample4() throws Exception { - setup(TEST_RESOURCES + "/config/zookeeper/readme-example-4"); + update(TEST_RESOURCES + "/config/zookeeper/readme-example-4"); // start the topology and write test messages to kafka fluxComponent.submitTopology(); @@ -239,7 +244,8 @@ public void testExample4() throws Exception { @Test public void testPercentiles() throws Exception { - setup(TEST_RESOURCES + "/config/zookeeper/percentiles"); + update(TEST_RESOURCES + "/config/zookeeper/percentiles"); + // start the topology and write test messages to kafka fluxComponent.submitTopology(); @@ -277,7 +283,8 @@ private List read(List puts, String family, byte[] qualifier, Class< return results; } - public void setup(String pathToConfig) throws Exception { + @BeforeClass + public static void setup() throws UnableToStartException { columnBuilder = new ValueOnlyColumnBuilder(columnFamily); // create input messages for the profiler to consume @@ -320,10 +327,10 @@ public void setup(String pathToConfig) throws Exception { new KafkaComponent.Topic(outputTopic, 1))); // upload profiler configuration to zookeeper - ConfigUploadComponent configUploadComponent = new ConfigUploadComponent() - .withTopologyProperties(topologyProperties) - .withGlobalConfiguration(pathToConfig) - .withProfilerConfiguration(pathToConfig); + configUploadComponent = new ConfigUploadComponent() + .withTopologyProperties(topologyProperties); +// .withGlobalConfiguration(pathToConfig) +// .withProfilerConfiguration(pathToConfig); // load flux definition for the profiler topology fluxComponent = new FluxTopologyComponent.Builder() @@ -345,11 +352,25 @@ public void setup(String pathToConfig) throws Exception { runner.start(); } + public void update(String path) throws Exception { + configUploadComponent.withGlobalConfiguration(path) + .withProfilerConfiguration(path); + configUploadComponent.update(); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + MockHTable.Provider.clear(); + if (runner != null) { + runner.stop(); + } + } + @After public void tearDown() throws Exception { MockHTable.Provider.clear(); if (runner != null) { - runner.stop(); + runner.reset(); } } } \ No newline at end of file diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java index 7766fe30eb..2787c1c916 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java @@ -24,6 +24,8 @@ import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction; import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; +import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; +import org.elasticsearch.action.admin.indices.get.GetIndexRequest; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; @@ -182,4 +184,20 @@ public void stop() { node = null; client = null; } + + @Override + public void reset() { + String[] indices = client.admin() + .indices() + .getIndex(new GetIndexRequest()) + .actionGet() + .getIndices(); + for(String index : indices) { + client + .admin() + .indices() + .delete(new DeleteIndexRequest(index)) + .actionGet(); + } + } } diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/ComponentRunner.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/ComponentRunner.java index ce7cab8fc3..293e6a5e76 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/ComponentRunner.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/ComponentRunner.java @@ -18,6 +18,8 @@ package org.apache.metron.integration; +import java.util.Arrays; +import java.util.Date; import java.util.LinkedHashMap; import java.util.Map; @@ -26,6 +28,7 @@ public static class Builder { LinkedHashMap components; String[] startupOrder; String[] shutdownOrder; + String[] resetOrder; long timeBetweenAttempts = 1000; int numRetries = 5; long maxTimeMS = 120000; @@ -56,6 +59,10 @@ public Builder withCustomShutdownOrder(String[] shutdownOrder) { this.shutdownOrder = shutdownOrder; return this; } + public Builder withCustomResetOrder(String[] resetOrder) { + this.resetOrder = resetOrder; + return this; + } public Builder withMillisecondsBetweenAttempts(long timeBetweenAttempts) { this.timeBetweenAttempts = timeBetweenAttempts; return this; @@ -75,7 +82,15 @@ public ComponentRunner build() { if(startupOrder == null) { startupOrder = toOrderedList(components); } - return new ComponentRunner(components, startupOrder, shutdownOrder, timeBetweenAttempts, numRetries, maxTimeMS); + if(resetOrder == null) { + // Reset in the order of shutdown, if no reset is defined. Otherwise, just order them. + if (shutdownOrder != null) { + resetOrder = shutdownOrder; + } else { + resetOrder = toOrderedList(components); + } + } + return new ComponentRunner(components, startupOrder, shutdownOrder, resetOrder, timeBetweenAttempts, numRetries, maxTimeMS); } } @@ -83,12 +98,14 @@ public ComponentRunner build() { LinkedHashMap components; String[] startupOrder; String[] shutdownOrder; + String[] resetOrder; long timeBetweenAttempts; int numRetries; long maxTimeMS; public ComponentRunner( LinkedHashMap components , String[] startupOrder , String[] shutdownOrder + , String[] resetOrder , long timeBetweenAttempts , int numRetries , long maxTimeMS @@ -97,6 +114,7 @@ public ComponentRunner( LinkedHashMap components this.components = components; this.startupOrder = startupOrder; this.shutdownOrder = shutdownOrder; + this.resetOrder = resetOrder; this.timeBetweenAttempts = timeBetweenAttempts; this.numRetries = numRetries; this.maxTimeMS = maxTimeMS; @@ -120,6 +138,14 @@ public void stop() { components.get(componentName).stop(); } } + public void reset() { + System.err.println("***** RESET ORDER: " + Arrays.toString(resetOrder)); + for(String componentName : resetOrder) { + System.err.println("***** RESETTING: " + componentName + ", " + new Date()); + components.get(componentName).reset(); + System.err.println("***** RESET: " + componentName + ", " + new Date()); + } + } public ProcessorResult process(Processor successState) { int retryCount = 0; diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/InMemoryComponent.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/InMemoryComponent.java index 8a9ee96df7..90a8615b5e 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/InMemoryComponent.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/InMemoryComponent.java @@ -18,6 +18,7 @@ package org.apache.metron.integration; public interface InMemoryComponent { - public void start() throws UnableToStartException; - public void stop(); + void start() throws UnableToStartException; + void stop(); + default void reset() {} } diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java index d34ff0815e..30c3976779 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java @@ -17,6 +17,8 @@ */ package org.apache.metron.integration.components; +import com.google.common.collect.Lists; +import java.util.Date; import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; @@ -24,6 +26,7 @@ import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.storm.Config; import org.apache.storm.LocalCluster; +import org.apache.storm.generated.KillOptions; import org.apache.storm.generated.StormTopology; import org.apache.storm.generated.TopologyInfo; import org.apache.metron.integration.InMemoryComponent; @@ -178,6 +181,30 @@ public void stop() { } } + @Override + public void reset() { + if (stormCluster != null) { + // lower the kill wait + KillOptions ko = new KillOptions(); + ko.set_wait_secs(10); + System.err.println("******* KILLING TOPOLOGY NAME: <" + topologyName +">"); + System.err.println("****** TOPOLOGIES: " + Lists.newArrayList(stormCluster.getClusterInfo().get_topologies())); + stormCluster.killTopologyWithOpts(topologyName, ko); + try { + Thread.sleep(20000); + } catch (InterruptedException e) { + // Do nothing + } + try { + System.err.println("******* SLEEP DONE: " + new Date()); + System.err.println("****** TOPOLOGIES: " + Lists.newArrayList(stormCluster.getClusterInfo().get_topologies())); + System.err.println("******* Topo status: " + stormCluster.getTopologyInfo(topologyName)); + } catch(Exception e) { + // Do nothing + } + } + } + public static void assassinateSlots() { /* You might be wondering why I'm not just casting to slot here, but that's because the Slot class moved locations @@ -194,6 +221,7 @@ public static void assassinateSlots() { } public void submitTopology() throws NoSuchMethodException, IOException, InstantiationException, TException, IllegalAccessException, InvocationTargetException, ClassNotFoundException, NoSuchFieldException { + System.err.println("****** STARTING TOPOLOGY: <" + getTopologyName() + ">"); startTopology(getTopologyName(), getTopologyLocation(), getTopologyProperties()); } @@ -208,6 +236,8 @@ private void startTopology(String topologyName, File topologyLoc, Properties pro stormCluster.submitTopology(topologyName, conf, topology); } catch(Exception nne) { + System.err.println("Caught exception"); + nne.printStackTrace(); try { Thread.sleep(2000); } catch (InterruptedException e) { 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 e55b3176b9..67e9b1430e 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 @@ -19,6 +19,9 @@ import com.google.common.base.Function; +import java.io.File; +import java.io.FileFilter; +import java.io.IOException; import kafka.api.FetchRequest; import kafka.api.FetchRequestBuilder; import kafka.common.TopicExistsException; @@ -31,6 +34,9 @@ import kafka.message.MessageAndOffset; import kafka.server.*; import kafka.utils.TestUtils; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.filefilter.IOFileFilter; +import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import kafka.utils.*; @@ -50,6 +56,7 @@ public class KafkaComponent implements InMemoryComponent { protected static final Logger LOG = LoggerFactory.getLogger(KafkaComponent.class); + private static final String TMP_KAFKA_LOGS = "/tmp/kafka-logs"; public static class Topic { public int numPartitions; @@ -61,6 +68,8 @@ public Topic(String name, int numPartitions) { } } + private static final String LOG_DIR = "/tmp/kafka-logs"; + private List producersCreated = new ArrayList<>(); private transient KafkaServer kafkaServer; private transient ZkClient zkClient; @@ -149,6 +158,9 @@ public void start() { // setup Broker Properties props = TestUtilsWrapper.createBrokerConfig(0, zookeeperConnectString, brokerPort); props.setProperty("zookeeper.connection.timeout.ms","1000000"); + props.setProperty("log.dir", TMP_KAFKA_LOGS); + + props.setProperty("delete.topic.enable", "true"); KafkaConfig config = new KafkaConfig(props); Time mock = new MockTime(); kafkaServer = TestUtils.createServer(config, mock); @@ -177,17 +189,72 @@ public String getZookeeperConnect() { @Override public void stop() { - shutdownConsumer(); - shutdownProducers(); + // Catch any exception in case things have already been cleared out from another component or reset. + try { + shutdownConsumer(); + shutdownProducers(); + } catch (Exception e) { + // Do nothing + } + if(kafkaServer != null) { kafkaServer.shutdown(); kafkaServer.awaitShutdown(); + try { + FileUtils.deleteDirectory(new File(TMP_KAFKA_LOGS)); + } catch (IOException e) { + // Do nothing + } } if(zkClient != null) { zkClient.close(); } } + @Override + public void reset() { + // Catch any exception in case things have already been cleared out from another component or reset. + try { + shutdownConsumer(); + shutdownProducers(); + } catch (Exception e) { + // Do nothing + } + + + // Delete the actual data. + if(kafkaServer != null) { +// throw new IllegalStateException(FileUtils.listFilesAndDirs(FileUtils.getFile(LOG_DIR), TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).toString()); + for(File f: FileUtils.getFile(LOG_DIR).listFiles()) { + // Delete any data files + if (f.getName().endsWith(".log") || f.getName().endsWith(".index")) { + boolean delete = f.delete(); + if (!delete) { + throw new IllegalStateException("Unable to delete Kafka data at: " + f.getAbsolutePath()); + } + } + + // Delete any directories (consumer offsets and data) + if (f.isDirectory()) { + try { + FileUtils.deleteDirectory(f); + } catch (IOException e) { + throw new IllegalStateException("Unable to delete Kafka data at: " + f.getAbsolutePath()); + } + } + } + } + + // Delete data in ZK + if(zkClient != null) { + for(Topic topic : topics) { + System.err.println("******* Topic path is: " + ZkUtils.getTopicPath(topic.name)); + zkClient.deleteRecursive(ZkUtils.getTopicPath(topic.name)); + } + zkClient.deleteRecursive(ZkUtils.ConsumersPath()); + } + } + public List readMessages(String topic) { SimpleConsumer consumer = new SimpleConsumer("localhost", 6667, 100000, 64 * 1024, "consumer"); FetchRequest req = new FetchRequestBuilder() diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/ZKServerComponent.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/ZKServerComponent.java index 57d814bfd5..cc85d5f624 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/ZKServerComponent.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/ZKServerComponent.java @@ -18,7 +18,8 @@ package org.apache.metron.integration.components; -import com.google.common.base.Function; +import java.io.IOException; +import org.apache.commons.io.FileUtils; import org.apache.metron.integration.InMemoryComponent; import org.apache.metron.integration.UnableToStartException; import org.apache.curator.test.TestingServer; @@ -60,6 +61,19 @@ public void stop() { if (testZkServer != null) { testZkServer.close(); } - }catch(Exception e){} + }catch(Exception e){ + // Do nothing + } + } + + @Override + public void reset() { + if (testZkServer != null) { + try { + FileUtils.deleteDirectory(testZkServer.getTempDirectory()); + } catch (IOException e) { + // Do nothing + } + } } } diff --git a/metron-platform/metron-management/src/test/java/org/apache/metron/management/KafkaFunctionsIntegrationTest.java b/metron-platform/metron-management/src/test/java/org/apache/metron/management/KafkaFunctionsIntegrationTest.java index 28d9489ccd..97cfb65474 100644 --- a/metron-platform/metron-management/src/test/java/org/apache/metron/management/KafkaFunctionsIntegrationTest.java +++ b/metron-platform/metron-management/src/test/java/org/apache/metron/management/KafkaFunctionsIntegrationTest.java @@ -26,6 +26,7 @@ import org.apache.metron.integration.ComponentRunner; import org.apache.metron.integration.components.KafkaComponent; import org.apache.metron.integration.components.ZKServerComponent; +import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -93,10 +94,15 @@ public void setup() { } @AfterClass - public static void tearDown() throws Exception { + public static void tearDownAfterClass() throws Exception { runner.stop(); } + @After + public void tearDown() { + runner.reset(); + } + /** * Write one message, read one message. */ diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java index 6ad7427c89..aaa37d90e9 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java @@ -135,4 +135,11 @@ public void stop() { } } + + @Override + public void reset() { + if (stormCluster != null) { + stormCluster.killTopology(sensorType); + } + } } diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index b02e2e2b98..92b889085d 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -177,7 +177,7 @@ public int run(String[] args) { try { results.cleanup(); } catch(IOException e) { - LOGGER.warn("Unable to cleanup files in HDFS", e); + LOGGER.warn("Unable to reset files in HDFS", e); } } return 0; diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java index 02fbc4dd74..58976a35ca 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java @@ -109,6 +109,15 @@ public void stop() { } } + @Override + public void reset() { + try { + miniSolrCloudCluster.deleteCollection("metron"); + } catch (SolrServerException | IOException e) { + // Do nothing + } + } + public MetronSolrClient getSolrClient() { return new MetronSolrClient(getZookeeperUrl()); } diff --git a/metron-platform/metron-storm-kafka/src/main/java/org/apache/metron/storm/kafka/flux/StormKafkaSpout.java b/metron-platform/metron-storm-kafka/src/main/java/org/apache/metron/storm/kafka/flux/StormKafkaSpout.java index 030348fc41..649d862ae3 100644 --- a/metron-platform/metron-storm-kafka/src/main/java/org/apache/metron/storm/kafka/flux/StormKafkaSpout.java +++ b/metron-platform/metron-storm-kafka/src/main/java/org/apache/metron/storm/kafka/flux/StormKafkaSpout.java @@ -18,6 +18,8 @@ package org.apache.metron.storm.kafka.flux; +import java.util.Arrays; +import java.util.Date; import org.apache.kafka.common.errors.WakeupException; import org.apache.log4j.Logger; import org.apache.storm.kafka.spout.KafkaSpout; @@ -42,7 +44,10 @@ public StormKafkaSpout(SimpleStormKafkaBuilder builder) { @Override public void deactivate() { try { + System.err.println("***** DEACTIVATING SPOUT: " + new Date()); + System.err.println("***** DEACT SPOUT: " + Arrays.toString(new Throwable().getStackTrace())); super.deactivate(); + System.err.println("***** DEACTIVATED SPOUT: " + new Date()); } catch(WakeupException we) { //see https://issues.apache.org/jira/browse/STORM-2184 @@ -53,7 +58,10 @@ public void deactivate() { @Override public void close() { try { + System.err.println("***** CLOSING SPOUT: " + new Date()); + System.err.println("***** CLOSING SPOUT: " + Arrays.toString(new Throwable().getStackTrace())); super.close(); + System.err.println("***** CLOSED SPOUT: " + new Date()); } catch(WakeupException we) { //see https://issues.apache.org/jira/browse/STORM-2184 From a470063422314711ec99a298d646202deda32fba Mon Sep 17 00:00:00 2001 From: justinjleet Date: Fri, 23 Jun 2017 12:04:53 -0400 Subject: [PATCH 02/43] fixes plus logging --- .../integration/ProfilerIntegrationTest.java | 9 ++++++++- .../org/apache/metron/hbase/client/HBaseClient.java | 4 ++++ .../components/FluxTopologyComponent.java | 9 ++++++--- .../metron/storm/kafka/flux/StormKafkaSpout.java | 12 +++++++++++- 4 files changed, 29 insertions(+), 5 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 f158d7a366..f38a9003c2 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 @@ -44,6 +44,7 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; @@ -284,7 +285,7 @@ private List read(List puts, String family, byte[] qualifier, Class< } @BeforeClass - public static void setup() throws UnableToStartException { + public static void setupBeforeClass() throws UnableToStartException { columnBuilder = new ValueOnlyColumnBuilder(columnFamily); // create input messages for the profiler to consume @@ -366,6 +367,12 @@ public static void tearDownAfterClass() throws Exception { } } + @Before + public void setup() { + // create the mock table + profilerTable = (MockHTable) MockHTable.Provider.addToCache(tableName, columnFamily); + } + @After public void tearDown() throws Exception { MockHTable.Provider.clear(); diff --git a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java index 3af64077e7..d141d8725e 100644 --- a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java +++ b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java @@ -20,6 +20,7 @@ package org.apache.metron.hbase.client; +import java.util.Arrays; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; @@ -137,6 +138,9 @@ public void clearMutations() { public void mutate() { Object[] result = new Object[mutations.size()]; try { + System.err.println("*** MUTATIONS: " + mutations); + System.err.println("*** RESULT: " + Arrays.toString(result)); + System.err.println("*** TABLE: " + table); table.batch(mutations, result); mutations.clear(); diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java index 30c3976779..314915a33d 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java @@ -185,15 +185,18 @@ public void stop() { public void reset() { if (stormCluster != null) { // lower the kill wait + // Give it a couple seconds to die. KillOptions ko = new KillOptions(); - ko.set_wait_secs(10); + ko.set_wait_secs(5); System.err.println("******* KILLING TOPOLOGY NAME: <" + topologyName +">"); System.err.println("****** TOPOLOGIES: " + Lists.newArrayList(stormCluster.getClusterInfo().get_topologies())); stormCluster.killTopologyWithOpts(topologyName, ko); +// stormCluster.killTopology(topologyName); try { - Thread.sleep(20000); + // Actually wait for it to die. + Thread.sleep(7500); } catch (InterruptedException e) { - // Do nothing +// Do nothing } try { System.err.println("******* SLEEP DONE: " + new Date()); diff --git a/metron-platform/metron-storm-kafka/src/main/java/org/apache/metron/storm/kafka/flux/StormKafkaSpout.java b/metron-platform/metron-storm-kafka/src/main/java/org/apache/metron/storm/kafka/flux/StormKafkaSpout.java index 649d862ae3..28f7fe0eec 100644 --- a/metron-platform/metron-storm-kafka/src/main/java/org/apache/metron/storm/kafka/flux/StormKafkaSpout.java +++ b/metron-platform/metron-storm-kafka/src/main/java/org/apache/metron/storm/kafka/flux/StormKafkaSpout.java @@ -20,6 +20,7 @@ import java.util.Arrays; import java.util.Date; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.kafka.common.errors.WakeupException; import org.apache.log4j.Logger; import org.apache.storm.kafka.spout.KafkaSpout; @@ -35,6 +36,9 @@ public class StormKafkaSpout extends KafkaSpout { private static final Logger LOG = Logger.getLogger(StormKafkaSpout.class); protected KafkaSpoutConfig _spoutConfig; protected String _topic; + + protected AtomicBoolean isShutdown = new AtomicBoolean(false); + public StormKafkaSpout(SimpleStormKafkaBuilder builder) { super(builder.build()); this._topic = builder.getTopic(); @@ -53,6 +57,9 @@ public void deactivate() { //see https://issues.apache.org/jira/browse/STORM-2184 LOG.warn("You can generally ignore these, as per https://issues.apache.org/jira/browse/STORM-2184 -- " + we.getMessage(), we); } + finally { + isShutdown.set(true); + } } @Override @@ -60,7 +67,10 @@ public void close() { try { System.err.println("***** CLOSING SPOUT: " + new Date()); System.err.println("***** CLOSING SPOUT: " + Arrays.toString(new Throwable().getStackTrace())); - super.close(); + if(!isShutdown.get()) { + super.close(); + isShutdown.set(true); + } System.err.println("***** CLOSED SPOUT: " + new Date()); } catch(WakeupException we) { From 05e0a6468d283a2ee6a0ce4be8c551373be7f75b Mon Sep 17 00:00:00 2001 From: justinjleet Date: Fri, 23 Jun 2017 15:01:27 -0400 Subject: [PATCH 03/43] more stuffs --- .../maas/service/MaasIntegrationTest.java | 3 +- .../integration/ConfigUploadComponent.java | 5 +--- .../integration/ProfilerIntegrationTest.java | 8 +++++- .../metron/hbase/client/HBaseClient.java | 4 --- .../metron/integration/ComponentRunner.java | 5 ---- .../components/FluxTopologyComponent.java | 28 ++++++++----------- .../components/KafkaComponent.java | 5 +--- .../components/ParserTopologyComponent.java | 7 ----- .../org/apache/metron/pcap/query/PcapCli.java | 2 +- .../apache/metron/test/mock/MockHTable.java | 8 +++++- 10 files changed, 30 insertions(+), 45 deletions(-) diff --git a/metron-analytics/metron-maas-service/src/test/java/org/apache/metron/maas/service/MaasIntegrationTest.java b/metron-analytics/metron-maas-service/src/test/java/org/apache/metron/maas/service/MaasIntegrationTest.java index 2c224edb08..a75f2a3ad2 100644 --- a/metron-analytics/metron-maas-service/src/test/java/org/apache/metron/maas/service/MaasIntegrationTest.java +++ b/metron-analytics/metron-maas-service/src/test/java/org/apache/metron/maas/service/MaasIntegrationTest.java @@ -66,14 +66,13 @@ public class MaasIntegrationTest { private static ComponentRunner runner; private static YarnComponent yarnComponent; private static ZKServerComponent zkServerComponent; + @BeforeClass public static void setupBeforeClass() throws Exception { UnitTestHelper.setJavaLoggingLevel(Level.SEVERE); LOG.info("Starting up YARN cluster"); - Map properties = new HashMap<>(); zkServerComponent = new ZKServerComponent(); - yarnComponent = new YarnComponent().withApplicationMasterClass(ApplicationMaster.class).withTestName(MaasIntegrationTest.class.getSimpleName()); runner = new ComponentRunner.Builder() diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ConfigUploadComponent.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ConfigUploadComponent.java index 8cb20fd423..b59d0b5e30 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ConfigUploadComponent.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ConfigUploadComponent.java @@ -59,9 +59,6 @@ public void stop() { public void update() throws UnableToStartException { -// withTopologyProperties(topologyProperties); -// withGlobalConfiguration(globalConfiguration); -// withProfilerConfiguration(profilerConfiguration); try { upload(); } catch (Exception e) { @@ -102,7 +99,7 @@ private void uploadProfilerConfig(CuratorFramework client) throws Exception { * @param client The zookeeper client. */ private void uploadGlobalConfig(CuratorFramework client) throws Exception { - if (globalConfiguration == null) { + if (globalConfiguration != null) { byte[] globalConfig = readGlobalConfigFromFile(globalConfiguration); if (globalConfig.length > 0) { writeGlobalConfigToZookeeper(readGlobalConfigFromFile(globalConfiguration), client); 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 f38a9003c2..c64bd675b3 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 @@ -150,7 +150,6 @@ public void testExample1() throws Exception { // 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); - System.out.println("Actuals are: " + actuals); // verify - there are 5 'HTTP' each with 390 bytes Assert.assertTrue(actuals.stream().anyMatch(val -> @@ -288,6 +287,12 @@ 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()); + System.err.println("*** INPUT NEW: " + inputNew); + // create input messages for the profiler to consume input = Stream.of(message1, message2, message3) .map(Bytes::toBytes) @@ -376,6 +381,7 @@ public void setup() { @After public void tearDown() throws Exception { MockHTable.Provider.clear(); + profilerTable.clear(); if (runner != null) { runner.reset(); } diff --git a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java index d141d8725e..3af64077e7 100644 --- a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java +++ b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/client/HBaseClient.java @@ -20,7 +20,6 @@ package org.apache.metron.hbase.client; -import java.util.Arrays; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; @@ -138,9 +137,6 @@ public void clearMutations() { public void mutate() { Object[] result = new Object[mutations.size()]; try { - System.err.println("*** MUTATIONS: " + mutations); - System.err.println("*** RESULT: " + Arrays.toString(result)); - System.err.println("*** TABLE: " + table); table.batch(mutations, result); mutations.clear(); diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/ComponentRunner.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/ComponentRunner.java index 293e6a5e76..4641e48bd1 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/ComponentRunner.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/ComponentRunner.java @@ -18,8 +18,6 @@ package org.apache.metron.integration; -import java.util.Arrays; -import java.util.Date; import java.util.LinkedHashMap; import java.util.Map; @@ -139,11 +137,8 @@ public void stop() { } } public void reset() { - System.err.println("***** RESET ORDER: " + Arrays.toString(resetOrder)); for(String componentName : resetOrder) { - System.err.println("***** RESETTING: " + componentName + ", " + new Date()); components.get(componentName).reset(); - System.err.println("***** RESET: " + componentName + ", " + new Date()); } } diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java index 314915a33d..7459b2f134 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java @@ -18,17 +18,14 @@ package org.apache.metron.integration.components; import com.google.common.collect.Lists; -import java.util.Date; import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.framework.imps.CuratorFrameworkImpl; import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.storm.Config; import org.apache.storm.LocalCluster; import org.apache.storm.generated.KillOptions; import org.apache.storm.generated.StormTopology; -import org.apache.storm.generated.TopologyInfo; import org.apache.metron.integration.InMemoryComponent; import org.apache.metron.integration.UnableToStartException; import org.apache.storm.flux.FluxBuilder; @@ -50,7 +47,6 @@ import java.nio.file.Paths; import java.util.Comparator; import java.util.Properties; -import java.util.concurrent.TimeUnit; public class FluxTopologyComponent implements InMemoryComponent { @@ -184,27 +180,25 @@ public void stop() { @Override public void reset() { if (stormCluster != null) { - // lower the kill wait - // Give it a couple seconds to die. KillOptions ko = new KillOptions(); - ko.set_wait_secs(5); + ko.set_wait_secs(0); System.err.println("******* KILLING TOPOLOGY NAME: <" + topologyName +">"); System.err.println("****** TOPOLOGIES: " + Lists.newArrayList(stormCluster.getClusterInfo().get_topologies())); stormCluster.killTopologyWithOpts(topologyName, ko); // stormCluster.killTopology(topologyName); - try { +// try { // Actually wait for it to die. - Thread.sleep(7500); - } catch (InterruptedException e) { +// Thread.sleep(7500); +// } catch (InterruptedException e) { // Do nothing - } - try { - System.err.println("******* SLEEP DONE: " + new Date()); +// } +// try { +// System.err.println("******* SLEEP DONE: " + new Date()); System.err.println("****** TOPOLOGIES: " + Lists.newArrayList(stormCluster.getClusterInfo().get_topologies())); - System.err.println("******* Topo status: " + stormCluster.getTopologyInfo(topologyName)); - } catch(Exception e) { +// System.err.println("******* Topo status: " + stormCluster.getTopologyInfo(topologyName)); +// } catch(Exception e) { // Do nothing - } +// } } } @@ -236,7 +230,9 @@ private void startTopology(String topologyName, File topologyLoc, Properties pro Assert.assertNotNull(topology); topology.validate(); try { + System.err.println("****** TOPOLOGIES (STARTING): " + Lists.newArrayList(stormCluster.getClusterInfo().get_topologies())); stormCluster.submitTopology(topologyName, conf, topology); + System.err.println("****** TOPOLOGIES (STARTED): " + Lists.newArrayList(stormCluster.getClusterInfo().get_topologies())); } catch(Exception nne) { System.err.println("Caught exception"); 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 67e9b1430e..53ae17808e 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 @@ -68,8 +68,6 @@ public Topic(String name, int numPartitions) { } } - private static final String LOG_DIR = "/tmp/kafka-logs"; - private List producersCreated = new ArrayList<>(); private transient KafkaServer kafkaServer; private transient ZkClient zkClient; @@ -224,8 +222,7 @@ public void reset() { // Delete the actual data. if(kafkaServer != null) { -// throw new IllegalStateException(FileUtils.listFilesAndDirs(FileUtils.getFile(LOG_DIR), TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).toString()); - for(File f: FileUtils.getFile(LOG_DIR).listFiles()) { + for(File f: FileUtils.getFile(TMP_KAFKA_LOGS).listFiles()) { // Delete any data files if (f.getName().endsWith(".log") || f.getName().endsWith(".index")) { boolean delete = f.delete(); diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java index aaa37d90e9..14a561859c 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java @@ -17,7 +17,6 @@ */ package org.apache.metron.parsers.integration.components; -import com.google.common.collect.ImmutableMap; import org.apache.storm.Config; import org.apache.storm.LocalCluster; import org.apache.storm.topology.TopologyBuilder; @@ -27,12 +26,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.IOException; -import java.nio.file.FileVisitOption; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; import java.util.*; import static org.apache.metron.integration.components.FluxTopologyComponent.assassinateSlots; diff --git a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java index 92b889085d..b02e2e2b98 100644 --- a/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java +++ b/metron-platform/metron-pcap-backend/src/main/java/org/apache/metron/pcap/query/PcapCli.java @@ -177,7 +177,7 @@ public int run(String[] args) { try { results.cleanup(); } catch(IOException e) { - LOGGER.warn("Unable to reset files in HDFS", e); + LOGGER.warn("Unable to cleanup files in HDFS", e); } } return 0; diff --git a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/mock/MockHTable.java b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/mock/MockHTable.java index 631b24bf1a..0403d1b30a 100644 --- a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/mock/MockHTable.java +++ b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/mock/MockHTable.java @@ -50,7 +50,6 @@ import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -479,6 +478,13 @@ public void addToPutLog(Put put) { } } + public void clear() { + synchronized (putLog) { + putLog.clear(); + } + data.clear(); + } + @Override public void put(Put put) throws IOException { addToPutLog(put); From 05b9c1f0d04a76afb97c90c33d82268618129806 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Sat, 24 Jun 2017 13:49:52 -0400 Subject: [PATCH 04/43] more fixes and update --- .../components/ElasticSearchComponent.java | 16 ----- .../integration/IndexingIntegrationTest.java | 2 + .../components/FluxTopologyComponent.java | 45 ++++++-------- .../components/KafkaComponent.java | 62 +++---------------- .../components/ParserTopologyComponent.java | 17 ++++- .../storm/kafka/flux/StormKafkaSpout.java | 6 -- 6 files changed, 43 insertions(+), 105 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java index 2787c1c916..db96ad0b78 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java @@ -184,20 +184,4 @@ public void stop() { node = null; client = null; } - - @Override - public void reset() { - String[] indices = client.admin() - .indices() - .getIndex(new GetIndexRequest()) - .actionGet() - .getIndices(); - for(String index : indices) { - client - .admin() - .indices() - .delete(new DeleteIndexRequest(index)) - .actionGet(); - } - } } diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/integration/IndexingIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/integration/IndexingIntegrationTest.java index da46d9309e..9e20b39595 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/integration/IndexingIntegrationTest.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/integration/IndexingIntegrationTest.java @@ -196,6 +196,8 @@ public void test() throws Exception { // on the field name converter assertInputDocsMatchOutputs(inputDocs, docs, getFieldNameConverter()); assertInputDocsMatchOutputs(inputDocs, readDocsFromDisk(hdfsDir), x -> x); + } catch(Throwable e) { + e.printStackTrace(); } finally { if(runner != null) { diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java index 7459b2f134..2a44f5cf51 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java @@ -6,9 +6,9 @@ * 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 - *

+ * + * 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. @@ -155,6 +155,8 @@ public void stop() { if (stormCluster != null) { try { try { + // Kill the topology directly instead of sitting through the wait period + killTopology(); stormCluster.shutdown(); } catch (IllegalStateException ise) { if (!(ise.getMessage().contains("It took over") && ise.getMessage().contains("to shut down slot"))) { @@ -180,25 +182,19 @@ public void stop() { @Override public void reset() { if (stormCluster != null) { - KillOptions ko = new KillOptions(); - ko.set_wait_secs(0); - System.err.println("******* KILLING TOPOLOGY NAME: <" + topologyName +">"); - System.err.println("****** TOPOLOGIES: " + Lists.newArrayList(stormCluster.getClusterInfo().get_topologies())); - stormCluster.killTopologyWithOpts(topologyName, ko); -// stormCluster.killTopology(topologyName); -// try { - // Actually wait for it to die. -// Thread.sleep(7500); -// } catch (InterruptedException e) { -// Do nothing -// } -// try { -// System.err.println("******* SLEEP DONE: " + new Date()); - System.err.println("****** TOPOLOGIES: " + Lists.newArrayList(stormCluster.getClusterInfo().get_topologies())); -// System.err.println("******* Topo status: " + stormCluster.getTopologyInfo(topologyName)); -// } catch(Exception e) { - // Do nothing -// } + killTopology(); + } + } + + protected void killTopology() { + KillOptions ko = new KillOptions(); + ko.set_wait_secs(0); + stormCluster.killTopologyWithOpts(topologyName, ko); + try { + // Actually wait for it to die. + Thread.sleep(2000); + } catch (InterruptedException e) { + // Do nothing } } @@ -218,7 +214,6 @@ public static void assassinateSlots() { } public void submitTopology() throws NoSuchMethodException, IOException, InstantiationException, TException, IllegalAccessException, InvocationTargetException, ClassNotFoundException, NoSuchFieldException { - System.err.println("****** STARTING TOPOLOGY: <" + getTopologyName() + ">"); startTopology(getTopologyName(), getTopologyLocation(), getTopologyProperties()); } @@ -230,13 +225,9 @@ private void startTopology(String topologyName, File topologyLoc, Properties pro Assert.assertNotNull(topology); topology.validate(); try { - System.err.println("****** TOPOLOGIES (STARTING): " + Lists.newArrayList(stormCluster.getClusterInfo().get_topologies())); stormCluster.submitTopology(topologyName, conf, topology); - System.err.println("****** TOPOLOGIES (STARTED): " + Lists.newArrayList(stormCluster.getClusterInfo().get_topologies())); } catch(Exception nne) { - System.err.println("Caught exception"); - nne.printStackTrace(); try { Thread.sleep(2000); } catch (InterruptedException e) { 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 53ae17808e..dd8b469d1a 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 @@ -22,6 +22,7 @@ import java.io.File; import java.io.FileFilter; import java.io.IOException; +import kafka.admin.AdminUtils; import kafka.api.FetchRequest; import kafka.api.FetchRequestBuilder; import kafka.common.TopicExistsException; @@ -56,7 +57,6 @@ public class KafkaComponent implements InMemoryComponent { protected static final Logger LOG = LoggerFactory.getLogger(KafkaComponent.class); - private static final String TMP_KAFKA_LOGS = "/tmp/kafka-logs"; public static class Topic { public int numPartitions; @@ -156,9 +156,6 @@ public void start() { // setup Broker Properties props = TestUtilsWrapper.createBrokerConfig(0, zookeeperConnectString, brokerPort); props.setProperty("zookeeper.connection.timeout.ms","1000000"); - props.setProperty("log.dir", TMP_KAFKA_LOGS); - - props.setProperty("delete.topic.enable", "true"); KafkaConfig config = new KafkaConfig(props); Time mock = new MockTime(); kafkaServer = TestUtils.createServer(config, mock); @@ -187,22 +184,12 @@ public String getZookeeperConnect() { @Override public void stop() { - // Catch any exception in case things have already been cleared out from another component or reset. - try { - shutdownConsumer(); - shutdownProducers(); - } catch (Exception e) { - // Do nothing - } + shutdownConsumer(); + shutdownProducers(); if(kafkaServer != null) { kafkaServer.shutdown(); kafkaServer.awaitShutdown(); - try { - FileUtils.deleteDirectory(new File(TMP_KAFKA_LOGS)); - } catch (IOException e) { - // Do nothing - } } if(zkClient != null) { zkClient.close(); @@ -211,45 +198,10 @@ public void stop() { @Override public void reset() { - // Catch any exception in case things have already been cleared out from another component or reset. - try { - shutdownConsumer(); - shutdownProducers(); - } catch (Exception e) { - // Do nothing - } - - - // Delete the actual data. - if(kafkaServer != null) { - for(File f: FileUtils.getFile(TMP_KAFKA_LOGS).listFiles()) { - // Delete any data files - if (f.getName().endsWith(".log") || f.getName().endsWith(".index")) { - boolean delete = f.delete(); - if (!delete) { - throw new IllegalStateException("Unable to delete Kafka data at: " + f.getAbsolutePath()); - } - } - - // Delete any directories (consumer offsets and data) - if (f.isDirectory()) { - try { - FileUtils.deleteDirectory(f); - } catch (IOException e) { - throw new IllegalStateException("Unable to delete Kafka data at: " + f.getAbsolutePath()); - } - } - } - } - - // Delete data in ZK - if(zkClient != null) { - for(Topic topic : topics) { - System.err.println("******* Topic path is: " + ZkUtils.getTopicPath(topic.name)); - zkClient.deleteRecursive(ZkUtils.getTopicPath(topic.name)); - } - zkClient.deleteRecursive(ZkUtils.ConsumersPath()); - } + // Unfortunately, there's no clean way to (quickly) purge or delete a topic. + // At least without killing and restarting broker anyway. + stop(); + start(); } public List readMessages(String topic) { diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java index 14a561859c..99924ada8f 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java @@ -19,6 +19,7 @@ import org.apache.storm.Config; import org.apache.storm.LocalCluster; +import org.apache.storm.generated.KillOptions; import org.apache.storm.topology.TopologyBuilder; import org.apache.metron.integration.InMemoryComponent; import org.apache.metron.integration.UnableToStartException; @@ -106,6 +107,8 @@ public void stop() { if (stormCluster != null) { try { try { + // Kill the topology directly instead of sitting through the wait period + killTopology(); stormCluster.shutdown(); } catch (IllegalStateException ise) { if (!(ise.getMessage().contains("It took over") && ise.getMessage().contains("to shut down slot"))) { @@ -132,7 +135,19 @@ public void stop() { @Override public void reset() { if (stormCluster != null) { - stormCluster.killTopology(sensorType); + killTopology(); + } + } + + protected void killTopology() { + KillOptions ko = new KillOptions(); + ko.set_wait_secs(0); + stormCluster.killTopologyWithOpts(sensorType, ko); + try { + // Actually wait for it to die. + Thread.sleep(2000); + } catch (InterruptedException e) { + // Do nothing } } } diff --git a/metron-platform/metron-storm-kafka/src/main/java/org/apache/metron/storm/kafka/flux/StormKafkaSpout.java b/metron-platform/metron-storm-kafka/src/main/java/org/apache/metron/storm/kafka/flux/StormKafkaSpout.java index 28f7fe0eec..514a21dddf 100644 --- a/metron-platform/metron-storm-kafka/src/main/java/org/apache/metron/storm/kafka/flux/StormKafkaSpout.java +++ b/metron-platform/metron-storm-kafka/src/main/java/org/apache/metron/storm/kafka/flux/StormKafkaSpout.java @@ -48,10 +48,7 @@ public StormKafkaSpout(SimpleStormKafkaBuilder builder) { @Override public void deactivate() { try { - System.err.println("***** DEACTIVATING SPOUT: " + new Date()); - System.err.println("***** DEACT SPOUT: " + Arrays.toString(new Throwable().getStackTrace())); super.deactivate(); - System.err.println("***** DEACTIVATED SPOUT: " + new Date()); } catch(WakeupException we) { //see https://issues.apache.org/jira/browse/STORM-2184 @@ -65,13 +62,10 @@ public void deactivate() { @Override public void close() { try { - System.err.println("***** CLOSING SPOUT: " + new Date()); - System.err.println("***** CLOSING SPOUT: " + Arrays.toString(new Throwable().getStackTrace())); if(!isShutdown.get()) { super.close(); isShutdown.set(true); } - System.err.println("***** CLOSED SPOUT: " + new Date()); } catch(WakeupException we) { //see https://issues.apache.org/jira/browse/STORM-2184 From 912575eb92b2baf6f1334991372a883ad252d866 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Sat, 24 Jun 2017 14:37:15 -0400 Subject: [PATCH 05/43] removing error code --- .../metron/profiler/integration/ProfilerIntegrationTest.java | 3 --- 1 file changed, 3 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 c64bd675b3..b863ebcbc1 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 @@ -291,7 +291,6 @@ public static void setupBeforeClass() throws UnableToStartException { .map(m -> Collections.nCopies(5, m)) .flatMap(l -> l.stream()) .collect(Collectors.toList()); - System.err.println("*** INPUT NEW: " + inputNew); // create input messages for the profiler to consume input = Stream.of(message1, message2, message3) @@ -335,8 +334,6 @@ public static void setupBeforeClass() throws UnableToStartException { // upload profiler configuration to zookeeper configUploadComponent = new ConfigUploadComponent() .withTopologyProperties(topologyProperties); -// .withGlobalConfiguration(pathToConfig) -// .withProfilerConfiguration(pathToConfig); // load flux definition for the profiler topology fluxComponent = new FluxTopologyComponent.Builder() From 11473f63841e18cef2fbb046c70c2e052b498b4f Mon Sep 17 00:00:00 2001 From: justinjleet Date: Sat, 24 Jun 2017 14:47:58 -0400 Subject: [PATCH 06/43] maybe making it work for superclasses like the parser tests. Unsure if it'll maven --- .../components/ConfigUploadComponent.java | 5 + .../components/FluxTopologyComponent.java | 1 - .../integration/ParserIntegrationTest.java | 121 +++++++++++++----- .../components/ParserTopologyComponent.java | 3 + 4 files changed, 100 insertions(+), 30 deletions(-) diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/components/ConfigUploadComponent.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/components/ConfigUploadComponent.java index 28965121fb..5901d9f53d 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/components/ConfigUploadComponent.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/components/ConfigUploadComponent.java @@ -124,6 +124,10 @@ public Map getParserSensorConfigs() { @Override public void start() throws UnableToStartException { + update(); + } + + public void update() throws UnableToStartException { try { final String zookeeperUrl = topologyProperties.getProperty(ZKServerComponent.ZOOKEEPER_PROPERTY); @@ -152,6 +156,7 @@ public void start() throws UnableToStartException { } } + public SensorParserConfig getSensorParserConfig(String sensorType) { SensorParserConfig sensorParserConfig = new SensorParserConfig(); CuratorFramework client = getClient(topologyProperties.getProperty(ZKServerComponent.ZOOKEEPER_PROPERTY)); diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java index 2a44f5cf51..c3673fbe22 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java @@ -17,7 +17,6 @@ */ package org.apache.metron.integration.components; -import com.google.common.collect.Lists; import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java index b20445e7ce..951e666a37 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java @@ -30,7 +30,11 @@ import org.apache.metron.parsers.integration.components.ParserTopologyComponent; import org.apache.metron.test.TestDataType; import org.apache.metron.test.utils.SampleDataUtils; +import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import javax.annotation.Nullable; @@ -39,14 +43,15 @@ public abstract class ParserIntegrationTest extends BaseIntegrationTest { protected static final String ERROR_TOPIC = "parser_error"; protected List inputMessages; - @Test - public void test() throws Exception { - final String sensorType = getSensorType(); - inputMessages = TestUtils.readSampleData(SampleDataUtils.getSampleDataPath(sensorType, TestDataType.RAW)); + protected static ParserTopologyComponent parserTopologyComponent; + protected static KafkaComponent kafkaComponent; + protected static ConfigUploadComponent configUploadComponent; + protected static ComponentRunner runner; + @BeforeClass + public static void setupBeforeClass() throws UnableToStartException { final Properties topologyProperties = new Properties(); - final KafkaComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList() {{ - add(new KafkaComponent.Topic(sensorType, 1)); + kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList() {{ add(new KafkaComponent.Topic(Constants.ENRICHMENT_TOPIC, 1)); add(new KafkaComponent.Topic(ERROR_TOPIC,1)); }}); @@ -54,29 +59,87 @@ public void test() throws Exception { ZKServerComponent zkServerComponent = getZKServerComponent(topologyProperties); - ConfigUploadComponent configUploadComponent = new ConfigUploadComponent() - .withTopologyProperties(topologyProperties) - .withGlobalConfigsPath(TestConstants.SAMPLE_CONFIG_PATH) - .withParserConfigsPath(TestConstants.PARSER_CONFIGS_PATH); + configUploadComponent = new ConfigUploadComponent() + .withTopologyProperties(topologyProperties) + .withGlobalConfigsPath(TestConstants.SAMPLE_CONFIG_PATH) + .withParserConfigsPath(TestConstants.PARSER_CONFIGS_PATH); - ParserTopologyComponent parserTopologyComponent = new ParserTopologyComponent.Builder() - .withSensorType(sensorType) - .withTopologyProperties(topologyProperties) - .withOutputTopic(Constants.ENRICHMENT_TOPIC) - .withBrokerUrl(kafkaComponent.getBrokerList()).build(); + parserTopologyComponent = new ParserTopologyComponent.Builder() + .withTopologyProperties(topologyProperties) + .withOutputTopic(Constants.ENRICHMENT_TOPIC) + .withBrokerUrl(kafkaComponent.getBrokerList()).build(); //UnitTestHelper.verboseLogging(); - ComponentRunner runner = new ComponentRunner.Builder() - .withComponent("zk", zkServerComponent) - .withComponent("kafka", kafkaComponent) - .withComponent("config", configUploadComponent) - .withComponent("org/apache/storm", parserTopologyComponent) - .withMillisecondsBetweenAttempts(5000) - .withNumRetries(10) - .withCustomShutdownOrder(new String[] {"org/apache/storm","config","kafka","zk"}) - .build(); - try { - runner.start(); + runner = new ComponentRunner.Builder() + .withComponent("zk", zkServerComponent) + .withComponent("kafka", kafkaComponent) + .withComponent("config", configUploadComponent) + .withComponent("org/apache/storm", parserTopologyComponent) + .withMillisecondsBetweenAttempts(5000) + .withNumRetries(10) + .withCustomStartupOrder(new String[] {"zk", "kafka", "config"}) // Skip starting Parser Topology and config + .withCustomShutdownOrder(new String[] {"org/apache/storm","config","kafka","zk"}) + .build(); + + runner.start(); + } + + @AfterClass + public static void teardownAfterClass() { + runner.stop(); + } + + @Before + public void setup() { + } + + @After + public void teardown() { + runner.reset(); + } + + @Test + public void test() throws Exception { + String sensorType = getSensorType(); + inputMessages = TestUtils.readSampleData(SampleDataUtils.getSampleDataPath(sensorType, TestDataType.RAW)); + parserTopologyComponent.updateSensorType(sensorType); + parserTopologyComponent.start(); + + kafkaComponent.createTopic(sensorType, 1, true); + +// final Properties topologyProperties = new Properties(); +// final KafkaComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList() {{ +// add(new KafkaComponent.Topic(sensorType, 1)); +// add(new KafkaComponent.Topic(Constants.ENRICHMENT_TOPIC, 1)); +// add(new KafkaComponent.Topic(ERROR_TOPIC,1)); +// }}); +// topologyProperties.setProperty("kafka.broker", kafkaComponent.getBrokerList()); + +// ZKServerComponent zkServerComponent = getZKServerComponent(topologyProperties); +// +// ConfigUploadComponent configUploadComponent = new ConfigUploadComponent() +// .withTopologyProperties(topologyProperties) +// .withGlobalConfigsPath(TestConstants.SAMPLE_CONFIG_PATH) +// .withParserConfigsPath(TestConstants.PARSER_CONFIGS_PATH); +// +// ParserTopologyComponent parserTopologyComponent = new ParserTopologyComponent.Builder() +// .withSensorType(sensorType) +// .withTopologyProperties(topologyProperties) +// .withOutputTopic(Constants.ENRICHMENT_TOPIC) +// .withBrokerUrl(kafkaComponent.getBrokerList()).build(); + + //UnitTestHelper.verboseLogging(); +// ComponentRunner runner = new ComponentRunner.Builder() +// .withComponent("zk", zkServerComponent) +// .withComponent("kafka", kafkaComponent) +// .withComponent("config", configUploadComponent) +// .withComponent("org/apache/storm", parserTopologyComponent) +// .withMillisecondsBetweenAttempts(5000) +// .withNumRetries(10) +// .withCustomShutdownOrder(new String[] {"org/apache/storm","config","kafka","zk"}) +// .build(); +// try { +// runner.start(); kafkaComponent.writeMessages(sensorType, inputMessages); ProcessorResult> result = runner.process(getProcessor()); List outputMessages = result.getResult(); @@ -99,9 +162,9 @@ public void test() throws Exception { } } } - } finally { - runner.stop(); - } +// } finally { +// runner.stop(); +// } } public void dumpParsedMessages(List outputMessages, StringBuffer buffer) { diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java index 99924ada8f..b556411892 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/components/ParserTopologyComponent.java @@ -76,6 +76,9 @@ public ParserTopologyComponent(Properties topologyProperties, String brokerUrl, this.outputTopic = outputTopic; } + public void updateSensorType(String sensorType) { + this.sensorType = sensorType; + } @Override public void start() throws UnableToStartException { From fa5c05205e4d5bf17d5ea665fd98711085391403 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Sat, 24 Jun 2017 20:18:00 -0400 Subject: [PATCH 07/43] Removing approximately 1M log errors by actually cleaning up ZK --- .../components/KafkaComponent.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) 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 dd8b469d1a..2ba79e4ab4 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 @@ -57,6 +57,7 @@ public class KafkaComponent implements InMemoryComponent { protected static final Logger LOG = LoggerFactory.getLogger(KafkaComponent.class); + protected static String logDir = ""; public static class Topic { public int numPartitions; @@ -157,6 +158,7 @@ public void start() { Properties props = TestUtilsWrapper.createBrokerConfig(0, zookeeperConnectString, brokerPort); props.setProperty("zookeeper.connection.timeout.ms","1000000"); KafkaConfig config = new KafkaConfig(props); + logDir = config.getString("log.dir"); Time mock = new MockTime(); kafkaServer = TestUtils.createServer(config, mock); @@ -192,6 +194,23 @@ public void stop() { kafkaServer.awaitShutdown(); } if(zkClient != null) { + // Delete data in ZK to avoid startup interference. + for(Topic topic : topics) { + zkClient.deleteRecursive(ZkUtils.getTopicPath(topic.name)); + } + + zkClient.deleteRecursive(ZkUtils.BrokerIdsPath()); + zkClient.deleteRecursive(ZkUtils.BrokerTopicsPath()); + zkClient.deleteRecursive(ZkUtils.ConsumersPath()); + zkClient.deleteRecursive(ZkUtils.ControllerPath()); + zkClient.deleteRecursive(ZkUtils.ControllerEpochPath()); + zkClient.deleteRecursive(ZkUtils.ReassignPartitionsPath()); + zkClient.deleteRecursive(ZkUtils.DeleteTopicsPath()); + zkClient.deleteRecursive(ZkUtils.PreferredReplicaLeaderElectionPath()); + zkClient.deleteRecursive(ZkUtils.BrokerSequenceIdPath()); + zkClient.deleteRecursive(ZkUtils.IsrChangeNotificationPath()); + zkClient.deleteRecursive(ZkUtils.EntityConfigPath()); + zkClient.deleteRecursive(ZkUtils.EntityConfigChangesPath()); zkClient.close(); } } From a5b152e0e300640364fc28a1ae1bc80d7be4f628 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Sat, 24 Jun 2017 22:04:44 -0400 Subject: [PATCH 08/43] empty From c9f072b5c8230e2b829dd15e167fa1297d8042bd Mon Sep 17 00:00:00 2001 From: justinjleet Date: Mon, 26 Jun 2017 09:22:15 -0400 Subject: [PATCH 09/43] Removing our artifacts before caching --- .travis.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.travis.yml b/.travis.yml index 8fb218a646..507e5ee86d 100644 --- a/.travis.yml +++ b/.travis.yml @@ -18,6 +18,8 @@ before_install: script: - | time mvn -q -T 2C -DskipTests install && time mvn -q -T 2C jacoco:prepare-agent surefire:test@unit-tests && mvn -q jacoco:prepare-agent surefire:test@integration-tests && time mvn -q jacoco:prepare-agent test --projects metron-interface/metron-config && time build_utils/verify_licenses.sh +before_cache: + - rm -rf $HOME/.m2/org/apache/metron cache: directories: - $HOME/.m2 From d4ab212cd0867a3d35aa73381b8389063b6b22c1 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Mon, 26 Jun 2017 09:24:46 -0400 Subject: [PATCH 10/43] Trying to flush cache --- .travis.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 507e5ee86d..ab3f1a5162 100644 --- a/.travis.yml +++ b/.travis.yml @@ -16,8 +16,7 @@ before_install: - export M2_HOME=$PWD/apache-maven-3.3.9 - export PATH=$M2_HOME/bin:$PATH script: - - | - time mvn -q -T 2C -DskipTests install && time mvn -q -T 2C jacoco:prepare-agent surefire:test@unit-tests && mvn -q jacoco:prepare-agent surefire:test@integration-tests && time mvn -q jacoco:prepare-agent test --projects metron-interface/metron-config && time build_utils/verify_licenses.sh + - mvn clean before_cache: - rm -rf $HOME/.m2/org/apache/metron cache: From f2979c2b1ca7a47a9a7fca161737cfc1fb14fbe6 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Mon, 26 Jun 2017 09:42:33 -0400 Subject: [PATCH 11/43] Undo flush --- .travis.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index ab3f1a5162..507e5ee86d 100644 --- a/.travis.yml +++ b/.travis.yml @@ -16,7 +16,8 @@ before_install: - export M2_HOME=$PWD/apache-maven-3.3.9 - export PATH=$M2_HOME/bin:$PATH script: - - mvn clean + - | + time mvn -q -T 2C -DskipTests install && time mvn -q -T 2C jacoco:prepare-agent surefire:test@unit-tests && mvn -q jacoco:prepare-agent surefire:test@integration-tests && time mvn -q jacoco:prepare-agent test --projects metron-interface/metron-config && time build_utils/verify_licenses.sh before_cache: - rm -rf $HOME/.m2/org/apache/metron cache: From 97a679e9b53bc4351dc217d6138327996d0bf850 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Fri, 23 Jun 2017 09:21:27 -0400 Subject: [PATCH 12/43] Adding logging to try to figure out what's going on --- .../components/FluxTopologyComponent.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java index c3673fbe22..779db37507 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/components/FluxTopologyComponent.java @@ -162,6 +162,7 @@ public void stop() { throw ise; } else { + LOG.error("Attempting to assassinate slots"); assassinateSlots(); LOG.error("Storm slots didn't shut down entirely cleanly *sigh*. " + "I gave them the old one-two-skadoo and killed the slots with prejudice. " + @@ -202,12 +203,15 @@ public static void assassinateSlots() { You might be wondering why I'm not just casting to slot here, but that's because the Slot class moved locations and we're supporting multiple versions of storm. */ + LOG.error("During slot assassination, all candidate threads: " + Thread.getAllStackTraces().keySet()); Thread.getAllStackTraces().keySet().stream().filter(t -> t instanceof AutoCloseable && t.getName().toLowerCase().contains("slot")).forEach(t -> { - AutoCloseable slot = (AutoCloseable) t; + LOG.error("Attempting to close thread: " + t + " with state: " + t.getState()); + // With extreme prejudice. Safety doesn't matter try { - slot.close(); - } catch (Exception e) { - LOG.error("Tried to kill " + t.getName() + " but.." + e.getMessage(), e); + t.stop(); + LOG.error("Called thread.stop() on " + t.getName() + ". State is: " + t.getState()); + } catch(Exception e) { + // Just swallow anything arising from the threads being killed. } }); } From aa837e07ee1660b53dbafed5e8bc1da1802c1dc7 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Mon, 26 Jun 2017 11:01:51 -0400 Subject: [PATCH 13/43] Adding time to integration test command --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 507e5ee86d..ce2e193c3e 100644 --- a/.travis.yml +++ b/.travis.yml @@ -17,7 +17,7 @@ before_install: - export PATH=$M2_HOME/bin:$PATH script: - | - time mvn -q -T 2C -DskipTests install && time mvn -q -T 2C jacoco:prepare-agent surefire:test@unit-tests && mvn -q jacoco:prepare-agent surefire:test@integration-tests && time mvn -q jacoco:prepare-agent test --projects metron-interface/metron-config && time build_utils/verify_licenses.sh + time mvn -q -T 2C -DskipTests install && time mvn -q -T 2C jacoco:prepare-agent surefire:test@unit-tests && time mvn -q jacoco:prepare-agent surefire:test@integration-tests && time mvn -q jacoco:prepare-agent test --projects metron-interface/metron-config && time build_utils/verify_licenses.sh before_cache: - rm -rf $HOME/.m2/org/apache/metron cache: From b1ede14c67178bed546e379baee9981bb165053f Mon Sep 17 00:00:00 2001 From: cstella Date: Tue, 27 Jun 2017 14:02:18 +0100 Subject: [PATCH 14/43] trimming tests down. --- .../resources/taxii-messages/messages.poll | 2489 ----------------- .../PcapTopologyIntegrationTest.java | 17 - 2 files changed, 2506 deletions(-) diff --git a/metron-platform/metron-data-management/src/test/resources/taxii-messages/messages.poll b/metron-platform/metron-data-management/src/test/resources/taxii-messages/messages.poll index 1c9d529d3a..3408dc52e4 100644 --- a/metron-platform/metron-data-management/src/test/resources/taxii-messages/messages.poll +++ b/metron-platform/metron-data-management/src/test/resources/taxii-messages/messages.poll @@ -417,2495 +417,6 @@ - - - 2016-02-22T15:24:02.972941Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS - - - - Remote Access Trojan - ZeuS - Zbot - Zeus - Zeus, ZeuS, or Zbot is Trojan horse computer malware that runs on computers running under versions of the Microsoft Windows operating system. While it is capable of being used to carry out many malicious and criminal tasks, it is often used to steal banking information by man-in-the-browser keystroke logging and form grabbing. It is also used to install the CryptoLocker ransomware.[1] Zeus is spread mainly through drive-by downloads and phishing schemes. (2014(http://en.wikipedia.org/wiki/Zeus_%28Trojan_horse%29)) - Zeus, ZeuS, or Zbot is Trojan horse computer malware effects Microsoft Windows operating system - - - - - - - - 2016-02-22T15:24:02.974363Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS Tracker (online)| krlsma.com/wp-includes/Text/dom/php/file.php (2014-10-15) | This domain has been identified as malicious by zeustracker.abuse.ch - Domain Watchlist - URL Watchlist - File Hash Watchlist - This domain krlsma.com has been identified as malicious by zeustracker.abuse.ch. For more detailed infomation about this indicator go to [CAUTION!!Read-URL-Before-Click] [https://zeustracker.abuse.ch/monitor.php?host=krlsma.com]. - - - - - - - - zeustracker.abuse.ch - - - 2014-10-15T00:00:00+00:00 - 2014-10-20T19:29:30+00:00 - - - - - - - 2016-02-22T15:24:02.975911Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS - - - - Remote Access Trojan - ZeuS - Zbot - Zeus - Zeus, ZeuS, or Zbot is Trojan horse computer malware that runs on computers running under versions of the Microsoft Windows operating system. While it is capable of being used to carry out many malicious and criminal tasks, it is often used to steal banking information by man-in-the-browser keystroke logging and form grabbing. It is also used to install the CryptoLocker ransomware.[1] Zeus is spread mainly through drive-by downloads and phishing schemes. (2014(http://en.wikipedia.org/wiki/Zeus_%28Trojan_horse%29)) - Zeus, ZeuS, or Zbot is Trojan horse computer malware effects Microsoft Windows operating system - - - - - - - - 2016-02-22T15:24:02.977154Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - - - - - - - - - - - - - 2016-02-22T15:24:02.978111Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - Domain: krlsma.com - Domain: krlsma.com | isFQDN: True | - - - krlsma.com - - - - - - - 2016-02-22T15:24:02.979274Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - URI: http://krlsma.com/wp-includes/Text/dom/php/file.php - URI: http://krlsma.com/wp-includes/Text/dom/php/file.php | Type: URL | - - - http://krlsma.com/wp-includes/Text/dom/php/file.php - - - - - - - 2016-02-22T15:24:02.980513Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - File: file.php - FileName: file.php | FileHash: cccc3d971cc7f2814229e836076664a1 | - - - file.php - php - - - MD5 - cccc3d971cc7f2814229e836076664a1 - - - - - - - - - 2016-02-22T15:24:02.982272Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS Tracker (offline)| goomjav1kaformjavkd.com/neverwind/tmp/pixel.jpg (2014-10-31) | This domain has been identified as malicious by zeustracker.abuse.ch - Domain Watchlist - URL Watchlist - File Hash Watchlist - This domain goomjav1kaformjavkd.com has been identified as malicious by zeustracker.abuse.ch. For more detailed infomation about this indicator go to [CAUTION!!Read-URL-Before-Click] [https://zeustracker.abuse.ch/monitor.php?host=goomjav1kaformjavkd.com]. - - - - - - - - zeustracker.abuse.ch - - - 2014-10-31T00:00:00+00:00 - 2014-10-31T16:44:24+00:00 - - - - - - - 2016-02-22T15:24:02.983991Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS - - - - Remote Access Trojan - ZeuS - Zbot - Zeus - Zeus, ZeuS, or Zbot is Trojan horse computer malware that runs on computers running under versions of the Microsoft Windows operating system. While it is capable of being used to carry out many malicious and criminal tasks, it is often used to steal banking information by man-in-the-browser keystroke logging and form grabbing. It is also used to install the CryptoLocker ransomware.[1] Zeus is spread mainly through drive-by downloads and phishing schemes. (2014(http://en.wikipedia.org/wiki/Zeus_%28Trojan_horse%29)) - Zeus, ZeuS, or Zbot is Trojan horse computer malware effects Microsoft Windows operating system - - - - - - - - 2016-02-22T15:24:02.985825Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - - - - - - - - - - - - - 2016-02-22T15:24:02.987068Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - Domain: goomjav1kaformjavkd.com - Domain: goomjav1kaformjavkd.com | isFQDN: True | - - - goomjav1kaformjavkd.com - - - - - - - 2016-02-22T15:24:02.988668Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - File: pixel.jpg - FileName: pixel.jpg | FileHash: 3903f963c6ff179fe4cff8d54a26326f | - - - pixel.jpg - jpg - - - MD5 - 3903f963c6ff179fe4cff8d54a26326f - - - - - - - - - 2016-02-22T15:24:02.991399Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - URI: http://goomjav1kaformjavkd.com/neverwind/tmp/pixel.jpg - URI: http://goomjav1kaformjavkd.com/neverwind/tmp/pixel.jpg | Type: URL | - - - http://goomjav1kaformjavkd.com/neverwind/tmp/pixel.jpg - - - - - - - 2016-02-22T15:24:02.993399Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS Tracker (online)| 72.55.133.246/~security/zuse/config.bin (2014-10-21) | This IP address has been identified as malicious by zeustracker.abuse.ch - IP Watchlist - URL Watchlist - File Hash Watchlist - This IP address 72.55.133.246 has been identified as malicious by zeustracker.abuse.ch. For more detailed infomation about this indicator go to [CAUTION!!Read-URL-Before-Click] [https://zeustracker.abuse.ch/monitor.php?host=72.55.133.246]. - - - - - - - - zeustracker.abuse.ch - - - 2014-10-21T00:00:00+00:00 - 2014-10-22T15:01:33+00:00 - - - - - - - 2016-02-22T15:24:02.995633Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS - - - - Remote Access Trojan - ZeuS - Zbot - Zeus - Zeus, ZeuS, or Zbot is Trojan horse computer malware that runs on computers running under versions of the Microsoft Windows operating system. While it is capable of being used to carry out many malicious and criminal tasks, it is often used to steal banking information by man-in-the-browser keystroke logging and form grabbing. It is also used to install the CryptoLocker ransomware.[1] Zeus is spread mainly through drive-by downloads and phishing schemes. (2014(http://en.wikipedia.org/wiki/Zeus_%28Trojan_horse%29)) - Zeus, ZeuS, or Zbot is Trojan horse computer malware effects Microsoft Windows operating system - - - - - - - - 2016-02-22T15:24:02.996944Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - - - - - - - - - - - - - 2016-02-22T15:24:02.998013Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - File: config.bin - FileName: config.bin | FileHash: aea75d1e44bc66ba13877654faa003ef | - - - config.bin - bin - - - MD5 - aea75d1e44bc66ba13877654faa003ef - - - - - - - - - 2016-02-22T15:24:02.999665Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - IP: 72.55.133.246 - IPv4: 72.55.133.246 | isSource: True | - - - 72.55.133.246 - - - - - - - 2016-02-22T15:24:03.000922Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - URI: http://72.55.133.246/~security/zuse/config.bin - URI: http://72.55.133.246/~security/zuse/config.bin | Type: URL | - - - http://72.55.133.246/~security/zuse/config.bin - - - - - - - 2016-02-22T15:24:03.002094Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS Tracker (offline)| richcomms.co.za/robot/config.bin (2014-10-21) | This domain has been identified as malicious by zeustracker.abuse.ch - Domain Watchlist - URL Watchlist - File Hash Watchlist - This domain richcomms.co.za has been identified as malicious by zeustracker.abuse.ch. For more detailed infomation about this indicator go to [CAUTION!!Read-URL-Before-Click] [https://zeustracker.abuse.ch/monitor.php?host=richcomms.co.za]. - - - - - - - - zeustracker.abuse.ch - - - 2014-10-21T00:00:00+00:00 - 2014-10-21T15:07:42+00:00 - - - - - - - 2016-02-22T15:24:03.003689Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - - - - - - - - - - - - - 2016-02-22T15:24:03.004759Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - Domain: richcomms.co.za - Domain: richcomms.co.za | isFQDN: True | - - - richcomms.co.za - - - - - - - 2016-02-22T15:24:03.147864Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - File: config.bin - FileName: config.bin | FileHash: 4bdae34ab94fc33bb53df030c0dc32b8 | - - - config.bin - bin - - - MD5 - 4bdae34ab94fc33bb53df030c0dc32b8 - - - - - - - - - 2016-02-22T15:24:03.154798Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - URI: http://richcomms.co.za/robot/config.bin - URI: http://richcomms.co.za/robot/config.bin | Type: URL | - - - http://richcomms.co.za/robot/config.bin - - - - - - - 2016-02-22T15:24:03.159781Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS - - - - Remote Access Trojan - ZeuS - Zbot - Zeus - Zeus, ZeuS, or Zbot is Trojan horse computer malware that runs on computers running under versions of the Microsoft Windows operating system. While it is capable of being used to carry out many malicious and criminal tasks, it is often used to steal banking information by man-in-the-browser keystroke logging and form grabbing. It is also used to install the CryptoLocker ransomware.[1] Zeus is spread mainly through drive-by downloads and phishing schemes. (2014(http://en.wikipedia.org/wiki/Zeus_%28Trojan_horse%29)) - Zeus, ZeuS, or Zbot is Trojan horse computer malware effects Microsoft Windows operating system - - - - - - - - 2016-02-22T15:24:03.164148Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS Tracker (offline)| requipsol.es/user/config.bin (2014-10-25) | This domain has been identified as malicious by zeustracker.abuse.ch - Domain Watchlist - URL Watchlist - File Hash Watchlist - This domain requipsol.es has been identified as malicious by zeustracker.abuse.ch. For more detailed infomation about this indicator go to [CAUTION!!Read-URL-Before-Click] [https://zeustracker.abuse.ch/monitor.php?host=requipsol.es]. - - - - - - - - zeustracker.abuse.ch - - - 2014-10-25T00:00:00+00:00 - 2014-10-31T14:50:30+00:00 - - - - - - - 2016-02-22T15:24:03.165664Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS - - - - Remote Access Trojan - ZeuS - Zbot - Zeus - Zeus, ZeuS, or Zbot is Trojan horse computer malware that runs on computers running under versions of the Microsoft Windows operating system. While it is capable of being used to carry out many malicious and criminal tasks, it is often used to steal banking information by man-in-the-browser keystroke logging and form grabbing. It is also used to install the CryptoLocker ransomware.[1] Zeus is spread mainly through drive-by downloads and phishing schemes. (2014(http://en.wikipedia.org/wiki/Zeus_%28Trojan_horse%29)) - Zeus, ZeuS, or Zbot is Trojan horse computer malware effects Microsoft Windows operating system - - - - - - - - 2016-02-22T15:24:03.166828Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - - - - - - - - - - - - - 2016-02-22T15:24:03.167831Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - URI: http://requipsol.es/user/config.bin - URI: http://requipsol.es/user/config.bin | Type: URL | - - - http://requipsol.es/user/config.bin - - - - - - - 2016-02-22T15:24:03.168967Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - File: config.bin - FileName: config.bin | FileHash: 041e16f955dced02ac65b86f42ecf2a1 | - - - config.bin - bin - - - MD5 - 041e16f955dced02ac65b86f42ecf2a1 - - - - - - - - - 2016-02-22T15:24:03.170646Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - Domain: requipsol.es - Domain: requipsol.es | isFQDN: True | - - - requipsol.es - - - - - - - 2016-02-22T15:24:03.171793Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS Tracker (online)| nozs.nl/nice/config.bin (2014-10-21) | This domain has been identified as malicious by zeustracker.abuse.ch - Domain Watchlist - URL Watchlist - File Hash Watchlist - This domain nozs.nl has been identified as malicious by zeustracker.abuse.ch. For more detailed infomation about this indicator go to [CAUTION!!Read-URL-Before-Click] [https://zeustracker.abuse.ch/monitor.php?host=nozs.nl]. - - - - - - - - zeustracker.abuse.ch - - - 2014-10-21T00:00:00+00:00 - 2014-10-21T15:07:42+00:00 - - - - - - - 2016-02-22T15:24:03.173276Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - - - - - - - - - - - - - 2016-02-22T15:24:03.174316Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - Domain: nozs.nl - Domain: nozs.nl | isFQDN: True | - - - nozs.nl - - - - - - - 2016-02-22T15:24:03.175384Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - File: config.bin - FileName: config.bin | FileHash: 6662f5c8b06318290689647040feba75 | - - - config.bin - bin - - - MD5 - 6662f5c8b06318290689647040feba75 - - - - - - - - - 2016-02-22T15:24:03.176883Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - URI: http://nozs.nl/nice/config.bin - URI: http://nozs.nl/nice/config.bin | Type: URL | - - - http://nozs.nl/nice/config.bin - - - - - - - 2016-02-22T15:24:03.178002Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS - - - - Remote Access Trojan - ZeuS - Zbot - Zeus - Zeus, ZeuS, or Zbot is Trojan horse computer malware that runs on computers running under versions of the Microsoft Windows operating system. While it is capable of being used to carry out many malicious and criminal tasks, it is often used to steal banking information by man-in-the-browser keystroke logging and form grabbing. It is also used to install the CryptoLocker ransomware.[1] Zeus is spread mainly through drive-by downloads and phishing schemes. (2014(http://en.wikipedia.org/wiki/Zeus_%28Trojan_horse%29)) - Zeus, ZeuS, or Zbot is Trojan horse computer malware effects Microsoft Windows operating system - - - - - - - - 2016-02-22T15:24:03.207978Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS Tracker (online)| affiliates.vcom.com/export/fal/cfg.bin (2014-09-10) | This domain has been identified as malicious by zeustracker.abuse.ch - Domain Watchlist - URL Watchlist - File Hash Watchlist - This domain affiliates.vcom.com has been identified as malicious by zeustracker.abuse.ch. For more detailed infomation about this indicator go to [CAUTION!!Read-URL-Before-Click] [https://zeustracker.abuse.ch/monitor.php?host=affiliates.vcom.com]. - - - - - - - - zeustracker.abuse.ch - - - 2014-09-10T00:00:00+00:00 - 2014-09-17T17:01:41+00:00 - - - - - - - 2016-02-22T15:24:03.209593Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - - - - - - - - - - - - - 2016-02-22T15:24:03.210688Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - File: cfg.bin - FileName: cfg.bin | FileHash: 6a8d7dbfa12d55a89c815f46dfe85db6 | - - - cfg.bin - bin - - - MD5 - 6a8d7dbfa12d55a89c815f46dfe85db6 - - - - - - - - - 2016-02-22T15:24:03.212290Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - URI: http://affiliates.vcom.com/export/fal/cfg.bin - URI: http://affiliates.vcom.com/export/fal/cfg.bin | Type: URL | - - - http://affiliates.vcom.com/export/fal/cfg.bin - - - - - - - 2016-02-22T15:24:03.213497Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - Domain: affiliates.vcom.com - Domain: affiliates.vcom.com | isFQDN: True | - - - affiliates.vcom.com - - - - - - - 2016-02-22T15:24:03.214626Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS - - - - Remote Access Trojan - ZeuS - Zbot - Zeus - Zeus, ZeuS, or Zbot is Trojan horse computer malware that runs on computers running under versions of the Microsoft Windows operating system. While it is capable of being used to carry out many malicious and criminal tasks, it is often used to steal banking information by man-in-the-browser keystroke logging and form grabbing. It is also used to install the CryptoLocker ransomware.[1] Zeus is spread mainly through drive-by downloads and phishing schemes. (2014(http://en.wikipedia.org/wiki/Zeus_%28Trojan_horse%29)) - Zeus, ZeuS, or Zbot is Trojan horse computer malware effects Microsoft Windows operating system - - - - - - - - 2016-02-22T15:24:03.215818Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS Tracker (online)| seguralawfirm.com/css/file.php (2014-10-29) | This domain has been identified as malicious by zeustracker.abuse.ch - Domain Watchlist - URL Watchlist - File Hash Watchlist - This domain seguralawfirm.com has been identified as malicious by zeustracker.abuse.ch. For more detailed infomation about this indicator go to [CAUTION!!Read-URL-Before-Click] [https://zeustracker.abuse.ch/monitor.php?host=seguralawfirm.com]. - - - - - - - - zeustracker.abuse.ch - - - 2014-10-29T00:00:00+00:00 - 2014-10-31T16:44:24+00:00 - - - - - - - 2016-02-22T15:24:03.217255Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS - - - - Remote Access Trojan - ZeuS - Zbot - Zeus - Zeus, ZeuS, or Zbot is Trojan horse computer malware that runs on computers running under versions of the Microsoft Windows operating system. While it is capable of being used to carry out many malicious and criminal tasks, it is often used to steal banking information by man-in-the-browser keystroke logging and form grabbing. It is also used to install the CryptoLocker ransomware.[1] Zeus is spread mainly through drive-by downloads and phishing schemes. (2014(http://en.wikipedia.org/wiki/Zeus_%28Trojan_horse%29)) - Zeus, ZeuS, or Zbot is Trojan horse computer malware effects Microsoft Windows operating system - - - - - - - - 2016-02-22T15:24:03.218365Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - - - - - - - - - - - - - 2016-02-22T15:24:03.219316Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - Domain: seguralawfirm.com - Domain: seguralawfirm.com | isFQDN: True | - - - seguralawfirm.com - - - - - - - 2016-02-22T15:24:03.220529Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - File: file.php - FileName: file.php | FileHash: f910639061bb879e85ec24045b8a9111 | - - - file.php - php - - - MD5 - f910639061bb879e85ec24045b8a9111 - - - - - - - - - 2016-02-22T15:24:03.221998Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - URI: http://seguralawfirm.com/css/file.php - URI: http://seguralawfirm.com/css/file.php | Type: URL | - - - http://seguralawfirm.com/css/file.php - - - - - - - 2016-02-22T15:24:03.223060Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS Tracker (offline)| 108.61.205.68/sab/sab1/file.php (2014-10-20) | This IP address has been identified as malicious by zeustracker.abuse.ch - IP Watchlist - URL Watchlist - File Hash Watchlist - This IP address 108.61.205.68 has been identified as malicious by zeustracker.abuse.ch. For more detailed infomation about this indicator go to [CAUTION!!Read-URL-Before-Click] [https://zeustracker.abuse.ch/monitor.php?host=108.61.205.68]. - - - - - - - - zeustracker.abuse.ch - - - 2014-10-20T00:00:00+00:00 - 2014-10-21T15:07:42+00:00 - - - - - - - 2016-02-22T15:24:03.224439Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS - - - - Remote Access Trojan - ZeuS - Zbot - Zeus - Zeus, ZeuS, or Zbot is Trojan horse computer malware that runs on computers running under versions of the Microsoft Windows operating system. While it is capable of being used to carry out many malicious and criminal tasks, it is often used to steal banking information by man-in-the-browser keystroke logging and form grabbing. It is also used to install the CryptoLocker ransomware.[1] Zeus is spread mainly through drive-by downloads and phishing schemes. (2014(http://en.wikipedia.org/wiki/Zeus_%28Trojan_horse%29)) - Zeus, ZeuS, or Zbot is Trojan horse computer malware effects Microsoft Windows operating system - - - - - - - - 2016-02-22T15:24:03.225548Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - - - - - - - - - - - - - 2016-02-22T15:24:03.226473Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - File: file.php - FileName: file.php | FileHash: 143b47feaab8e0ad41a41e278ed58822 | - - - file.php - php - - - MD5 - 143b47feaab8e0ad41a41e278ed58822 - - - - - - - - - 2016-02-22T15:24:03.227946Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - URI: http://108.61.205.68/sab/sab1/file.php - URI: http://108.61.205.68/sab/sab1/file.php | Type: URL | - - - http://108.61.205.68/sab/sab1/file.php - - - - - - - 2016-02-22T15:24:03.228974Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - IP: 108.61.205.68 - IPv4: 108.61.205.68 | isSource: True | - - - 108.61.205.68 - - - - - - - 2016-02-22T15:24:03.266483Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS Tracker (online)| 104.192.103.21/oct/helps/file.php (2014-10-27) | This IP address has been identified as malicious by zeustracker.abuse.ch - IP Watchlist - URL Watchlist - File Hash Watchlist - This IP address 104.192.103.21 has been identified as malicious by zeustracker.abuse.ch. For more detailed infomation about this indicator go to [CAUTION!!Read-URL-Before-Click] [https://zeustracker.abuse.ch/monitor.php?host=104.192.103.21]. - - - - - - - - zeustracker.abuse.ch - - - 2014-10-27T00:00:00+00:00 - 2014-10-31T14:50:30+00:00 - - - - - - - 2016-02-22T15:24:03.267908Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS - - - - Remote Access Trojan - ZeuS - Zbot - Zeus - Zeus, ZeuS, or Zbot is Trojan horse computer malware that runs on computers running under versions of the Microsoft Windows operating system. While it is capable of being used to carry out many malicious and criminal tasks, it is often used to steal banking information by man-in-the-browser keystroke logging and form grabbing. It is also used to install the CryptoLocker ransomware.[1] Zeus is spread mainly through drive-by downloads and phishing schemes. (2014(http://en.wikipedia.org/wiki/Zeus_%28Trojan_horse%29)) - Zeus, ZeuS, or Zbot is Trojan horse computer malware effects Microsoft Windows operating system - - - - - - - - 2016-02-22T15:24:03.269063Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - - - - - - - - - - - - - 2016-02-22T15:24:03.270086Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - File: file.php - FileName: file.php | FileHash: 2b3c17747c654f00155de87758a4ed8c | - - - file.php - php - - - MD5 - 2b3c17747c654f00155de87758a4ed8c - - - - - - - - - 2016-02-22T15:24:03.271600Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - URI: http://104.192.103.21/oct/helps/file.php - URI: http://104.192.103.21/oct/helps/file.php | Type: URL | - - - http://104.192.103.21/oct/helps/file.php - - - - - - - 2016-02-22T15:24:03.272680Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - IP: 104.192.103.21 - IPv4: 104.192.103.21 | isSource: True | - - - 104.192.103.21 - - - - - - - 2016-02-22T15:24:03.273797Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - ZeuS Tracker (offline)| forumforn22everw.com/neverwind/tmp/pixel.jpg (2014-10-31) | This domain has been identified as malicious by zeustracker.abuse.ch - Domain Watchlist - URL Watchlist - File Hash Watchlist - This domain forumforn22everw.com has been identified as malicious by zeustracker.abuse.ch. For more detailed infomation about this indicator go to [CAUTION!!Read-URL-Before-Click] [https://zeustracker.abuse.ch/monitor.php?host=forumforn22everw.com]. - - - - - - - - zeustracker.abuse.ch - - - 2014-10-31T00:00:00+00:00 - 2014-10-31T16:44:24+00:00 - - - - - - - 2016-02-22T15:24:03.275292Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - - - - - - - - - - - - - 2016-02-22T15:24:03.276424Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - URI: http://forumforn22everw.com/neverwind/tmp/pixel.jpg - URI: http://forumforn22everw.com/neverwind/tmp/pixel.jpg | Type: URL | - - - http://forumforn22everw.com/neverwind/tmp/pixel.jpg - - - - - - - 2016-02-22T15:24:03.277577Z - - - - - - - - - ../../../../descendant-or-self::node() - - - zeustracker.abuse.ch | Abuse source[https://sslbl.abuse.ch/blacklist/] - As for all abuse.ch projects, the use of the SSL Blacklist is free for both commercial and non-commercial usage without any limitation. However, if you are a commercial vendor of security software/services and you want to integrate data from the SSL Blacklist into your products / services, you will have to ask for permission first by contacting me using the contact form [http://www.abuse.ch/?page_id=4727].' - - - - Unclassified (Public) - - - - - - - Domain: forumforn22everw.com - Domain: forumforn22everw.com | isFQDN: True | - - - forumforn22everw.com - - - - 2016-02-22T15:24:03.278645Z diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java index 7d1dba8c19..3b0f20d810 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java @@ -89,23 +89,6 @@ public boolean accept(File dir, String name) { }).length; } - @Test - public void testTimestampInPacket() throws Exception { - testTopology(new Function() { - @Nullable - @Override - public Void apply(@Nullable Properties input) { - input.setProperty("kafka.pcap.ts_scheme", Deserializers.FROM_PACKET.toString()); - return null; - } - }, (kafkaComponent, pcapEntries) -> kafkaComponent.writeMessages( KAFKA_TOPIC - , Collections2.transform(pcapEntries - , input -> input.getValue() - ) - ) - , true - ); - } @Test public void testTimestampInKey() throws Exception { testTopology(new Function() { From 6562246129ca3873a297a0746bcdafb613faf9ef Mon Sep 17 00:00:00 2001 From: justinjleet Date: Tue, 27 Jun 2017 11:27:51 -0400 Subject: [PATCH 15/43] Removing jacoco from Travis build --- .travis.yml | 2 +- pom.xml | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index ce2e193c3e..472346f88a 100644 --- a/.travis.yml +++ b/.travis.yml @@ -17,7 +17,7 @@ before_install: - export PATH=$M2_HOME/bin:$PATH script: - | - time mvn -q -T 2C -DskipTests install && time mvn -q -T 2C jacoco:prepare-agent surefire:test@unit-tests && time mvn -q jacoco:prepare-agent surefire:test@integration-tests && time mvn -q jacoco:prepare-agent test --projects metron-interface/metron-config && time build_utils/verify_licenses.sh + time mvn -q -T 2C -DskipTests install && time mvn -q -T 2C surefire:test@unit-tests && time mvn -q surefire:test@integration-tests && time mvn -q test --projects metron-interface/metron-config && time build_utils/verify_licenses.sh before_cache: - rm -rf $HOME/.m2/org/apache/metron cache: diff --git a/pom.xml b/pom.xml index 16b2499986..af97e83b8a 100644 --- a/pom.xml +++ b/pom.xml @@ -108,6 +108,7 @@ 3.0.2 2.18 [3.3.1,) + From ad27176f957cdd87f0559c95d6a0eda42ba9f6d7 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Tue, 27 Jun 2017 15:33:42 -0400 Subject: [PATCH 16/43] Properly handle clearing out Metron artifacts from Maven so we don't cache them --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 472346f88a..991a612901 100644 --- a/.travis.yml +++ b/.travis.yml @@ -19,7 +19,7 @@ script: - | time mvn -q -T 2C -DskipTests install && time mvn -q -T 2C surefire:test@unit-tests && time mvn -q surefire:test@integration-tests && time mvn -q test --projects metron-interface/metron-config && time build_utils/verify_licenses.sh before_cache: - - rm -rf $HOME/.m2/org/apache/metron + - rm -rf $HOME/.m2/repository/org/apache/metron cache: directories: - $HOME/.m2 From 6824ca9937a9e047d68f6eba6dc4e8ec42a0f870 Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 10:06:14 +0100 Subject: [PATCH 17/43] Move unit test to use mock htable rather than real hbase. --- .../client/HBaseProfilerClientTest.java | 28 +++---------------- 1 file changed, 4 insertions(+), 24 deletions(-) diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java index ed75a65537..d3a0fe5e62 100644 --- a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java +++ b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java @@ -20,11 +20,6 @@ package org.apache.metron.profiler.client; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.client.HTableInterface; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.metron.profiler.ProfileMeasurement; import org.apache.metron.profiler.hbase.ColumnBuilder; import org.apache.metron.profiler.hbase.RowKeyBuilder; @@ -32,6 +27,7 @@ import org.apache.metron.profiler.hbase.ValueOnlyColumnBuilder; import org.apache.metron.profiler.stellar.DefaultStellarExecutor; import org.apache.metron.profiler.stellar.StellarExecutor; +import org.apache.metron.test.mock.MockHTable; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -62,30 +58,14 @@ public class HBaseProfilerClientTest { private static final int periodsPerHour = 4; private HBaseProfilerClient client; - private HTableInterface table; + private MockHTable table; private StellarExecutor executor; - private static HBaseTestingUtility util; private ProfileWriter profileWriter; - @BeforeClass - public static void startHBase() throws Exception { - Configuration config = HBaseConfiguration.create(); - config.set("hbase.master.hostname", "localhost"); - config.set("hbase.regionserver.hostname", "localhost"); - util = new HBaseTestingUtility(config); - util.startMiniCluster(); - } - - @AfterClass - public static void stopHBase() throws Exception { - util.shutdownMiniCluster(); - util.cleanupTestDir(); - } - @Before public void setup() throws Exception { - table = util.createTable(Bytes.toBytes(tableName), Bytes.toBytes(columnFamily)); + table = new MockHTable(tableName, columnFamily); executor = new DefaultStellarExecutor(); // used to write values to be read during testing @@ -99,7 +79,7 @@ public void setup() throws Exception { @After public void tearDown() throws Exception { - util.deleteTable(tableName); + table.clear(); } /** From 8eeec0669b41925cf64085312e24027a8ab2dcdf Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 10:22:40 +0100 Subject: [PATCH 18/43] Parallelizing the conditions to the STIX extractor test should speed things up. --- .../extractor/stix/StixExtractorTest.java | 70 ++++++++++++------- 1 file changed, 46 insertions(+), 24 deletions(-) diff --git a/metron-platform/metron-data-management/src/test/java/org/apache/metron/dataloads/extractor/stix/StixExtractorTest.java b/metron-platform/metron-data-management/src/test/java/org/apache/metron/dataloads/extractor/stix/StixExtractorTest.java index 597a3a54ad..dc078ba08e 100644 --- a/metron-platform/metron-data-management/src/test/java/org/apache/metron/dataloads/extractor/stix/StixExtractorTest.java +++ b/metron-platform/metron-data-management/src/test/java/org/apache/metron/dataloads/extractor/stix/StixExtractorTest.java @@ -83,33 +83,55 @@ public void testStixAddressesWithoutCondition() throws Exception { testStixAddresses(stixDocWithoutCondition); } - public void testStixAddresses(String stixDoc) throws Exception { + public void testStixAddresses(final String stixDoc) throws Exception { + Thread t1 = new Thread( () -> { - ExtractorHandler handler = ExtractorHandler.load(stixConfigOnlyIPV4); - Extractor extractor = handler.getExtractor(); - Iterable results = extractor.extract(stixDoc); + try { + ExtractorHandler handler = ExtractorHandler.load(stixConfigOnlyIPV4); + Extractor extractor = handler.getExtractor(); + Iterable results = extractor.extract(stixDoc); - Assert.assertEquals(3, Iterables.size(results)); - Assert.assertEquals("10.0.0.0", ((EnrichmentKey)(Iterables.get(results, 0).getKey())).indicator); - Assert.assertEquals("10.0.0.1", ((EnrichmentKey)(Iterables.get(results, 1).getKey())).indicator); - Assert.assertEquals("10.0.0.2", ((EnrichmentKey)(Iterables.get(results, 2).getKey())).indicator); - } + Assert.assertEquals(3, Iterables.size(results)); + Assert.assertEquals("10.0.0.0", ((EnrichmentKey) (Iterables.get(results, 0).getKey())).indicator); + Assert.assertEquals("10.0.0.1", ((EnrichmentKey) (Iterables.get(results, 1).getKey())).indicator); + Assert.assertEquals("10.0.0.2", ((EnrichmentKey) (Iterables.get(results, 2).getKey())).indicator); + } + catch(Exception ex) { + throw new RuntimeException(ex.getMessage(), ex); + } + }); + Thread t2 = new Thread( () -> { - - ExtractorHandler handler = ExtractorHandler.load(stixConfig); - Extractor extractor = handler.getExtractor(); - Iterable results = extractor.extract(stixDoc); - Assert.assertEquals(3, Iterables.size(results)); - Assert.assertEquals("10.0.0.0", ((EnrichmentKey)(Iterables.get(results, 0).getKey())).indicator); - Assert.assertEquals("10.0.0.1", ((EnrichmentKey)(Iterables.get(results, 1).getKey())).indicator); - Assert.assertEquals("10.0.0.2", ((EnrichmentKey)(Iterables.get(results, 2).getKey())).indicator); - } + try { + ExtractorHandler handler = ExtractorHandler.load(stixConfig); + Extractor extractor = handler.getExtractor(); + Iterable results = extractor.extract(stixDoc); + Assert.assertEquals(3, Iterables.size(results)); + Assert.assertEquals("10.0.0.0", ((EnrichmentKey) (Iterables.get(results, 0).getKey())).indicator); + Assert.assertEquals("10.0.0.1", ((EnrichmentKey) (Iterables.get(results, 1).getKey())).indicator); + Assert.assertEquals("10.0.0.2", ((EnrichmentKey) (Iterables.get(results, 2).getKey())).indicator); + } + catch(Exception ex) { + throw new RuntimeException(ex.getMessage(), ex); + } + }); + Thread t3 = new Thread( () -> { - - ExtractorHandler handler = ExtractorHandler.load(stixConfigOnlyIPV6); - Extractor extractor = handler.getExtractor(); - Iterable results = extractor.extract(stixDoc); - Assert.assertEquals(0, Iterables.size(results)); - } + try { + ExtractorHandler handler = ExtractorHandler.load(stixConfigOnlyIPV6); + Extractor extractor = handler.getExtractor(); + Iterable results = extractor.extract(stixDoc); + Assert.assertEquals(0, Iterables.size(results)); + } + catch(Exception ex) { + throw new RuntimeException(ex.getMessage(), ex); + } + }); + t1.run(); + t2.run(); + t3.run(); + t1.join(); + t2.join(); + t3.join(); } } From feab3ced086cc7bfb2b1ec99323d68e36001329d Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 10:33:20 +0100 Subject: [PATCH 19/43] Config functions test should reuse the zookeeper instance. --- metron-platform/metron-management/pom.xml | 4 ++++ .../management/ConfigurationFunctionsTest.java | 13 +++++++------ metron-platform/metron-parsers/pom.xml | 4 ++++ 3 files changed, 15 insertions(+), 6 deletions(-) diff --git a/metron-platform/metron-management/pom.xml b/metron-platform/metron-management/pom.xml index 638d65f9dd..418466857e 100644 --- a/metron-platform/metron-management/pom.xml +++ b/metron-platform/metron-management/pom.xml @@ -61,6 +61,10 @@ 0.1.0 provided + + commons-lang3 + org.apache.commons + org.slf4j slf4j-log4j12 diff --git a/metron-platform/metron-management/src/test/java/org/apache/metron/management/ConfigurationFunctionsTest.java b/metron-platform/metron-management/src/test/java/org/apache/metron/management/ConfigurationFunctionsTest.java index ee6a362827..972eed7e1b 100644 --- a/metron-platform/metron-management/src/test/java/org/apache/metron/management/ConfigurationFunctionsTest.java +++ b/metron-platform/metron-management/src/test/java/org/apache/metron/management/ConfigurationFunctionsTest.java @@ -32,6 +32,7 @@ import org.json.simple.JSONObject; import org.junit.Assert; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import java.util.HashMap; @@ -42,14 +43,14 @@ import static org.apache.metron.common.utils.StellarProcessorUtils.run; public class ConfigurationFunctionsTest { - private TestingServer testZkServer; - private CuratorFramework client; - private String zookeeperUrl; + private static TestingServer testZkServer; + private static CuratorFramework client; + private static String zookeeperUrl; private Context context = new Context.Builder() .with(Context.Capabilities.ZOOKEEPER_CLIENT, () -> client) .build(); - @Before - public void setup() throws Exception { + @BeforeClass + public static void setup() throws Exception { testZkServer = new TestingServer(true); zookeeperUrl = testZkServer.getConnectString(); client = ConfigurationsUtils.getClient(zookeeperUrl); @@ -61,7 +62,7 @@ public void setup() throws Exception { } - private void pushConfigs(String inputPath) throws Exception { + private static void pushConfigs(String inputPath) throws Exception { String[] args = new String[]{ "-z", zookeeperUrl , "--mode", "PUSH" diff --git a/metron-platform/metron-parsers/pom.xml b/metron-platform/metron-parsers/pom.xml index cce975abf4..b0d77cd9ca 100644 --- a/metron-platform/metron-parsers/pom.xml +++ b/metron-platform/metron-parsers/pom.xml @@ -166,6 +166,10 @@ slf4j-log4j12 org.slf4j + + commons-lang3 + org.apache.commons + From fafa57df28572a9103fa8010ce3507cf917a6273 Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 10:40:59 +0100 Subject: [PATCH 20/43] FSFunctionsTest should reuse infrastructure rather than spinning up hdfs and local clusters 17+ times --- .../management/FileSystemFunctionsTest.java | 55 +++++++++++-------- 1 file changed, 32 insertions(+), 23 deletions(-) diff --git a/metron-platform/metron-management/src/test/java/org/apache/metron/management/FileSystemFunctionsTest.java b/metron-platform/metron-management/src/test/java/org/apache/metron/management/FileSystemFunctionsTest.java index 88eabe00a7..e0bad7990b 100644 --- a/metron-platform/metron-management/src/test/java/org/apache/metron/management/FileSystemFunctionsTest.java +++ b/metron-platform/metron-management/src/test/java/org/apache/metron/management/FileSystemFunctionsTest.java @@ -21,10 +21,7 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.metron.common.dsl.Context; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.*; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -37,8 +34,11 @@ public class FileSystemFunctionsTest { private FileSystemFunctions.FS_TYPE type; private FileSystemFunctions.FileSystemGetter fsGetter = null; - private File baseDir; - private MiniDFSCluster hdfsCluster; + private static File hdfsBaseDir; + private static File localBaseDir; + private static MiniDFSCluster hdfsCluster; + private static String hdfsPrefix; + private static String localPrefix; private String prefix; private Context context = null; private FileSystemFunctions.FileSystemGet get; @@ -59,25 +59,34 @@ public static Collection types() { }); } - @Before - public void setup() throws IOException { - if(type == FileSystemFunctions.FS_TYPE.HDFS) { - baseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); + @BeforeClass + public static void setupFS() throws IOException { + { + hdfsBaseDir = Files.createTempDirectory("test_hdfs").toFile().getAbsoluteFile(); Configuration conf = new Configuration(); - conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, hdfsBaseDir.getAbsolutePath()); MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); hdfsCluster = builder.build(); + hdfsPrefix = "/"; + } + { + localPrefix = "target/fsTest/"; + if (new File(localPrefix).exists()) { + new File(localPrefix).delete(); + } + new File(localPrefix).mkdirs(); + } + } + @Before + public void setup() throws IOException { + if(type == FileSystemFunctions.FS_TYPE.HDFS) { + prefix=hdfsPrefix; fsGetter = () -> hdfsCluster.getFileSystem(); - prefix = "/"; } else { + prefix=localPrefix; fsGetter = FileSystemFunctions.FS_TYPE.LOCAL; - prefix = "target/fsTest/"; - if(new File(prefix).exists()) { - new File(prefix).delete(); - } - new File(prefix).mkdirs(); } get = new FileSystemFunctions.FileSystemGet(fsGetter); @@ -92,14 +101,14 @@ public void setup() throws IOException { rm.initialize(null); } - @After - public void teardown() { - if(type == FileSystemFunctions.FS_TYPE.HDFS) { + @AfterClass + public static void teardown() { + { hdfsCluster.shutdown(); - FileUtil.fullyDelete(baseDir); + FileUtil.fullyDelete(hdfsBaseDir); } - else { - new File(prefix).delete(); + { + new File(localPrefix).delete(); } } From 1c98ae4d5d78d1da2d627e2a20d56e20fd1cbdf6 Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 11:01:14 +0100 Subject: [PATCH 21/43] Refactored HBaseClientTest to not delete tables, but rather just issue deletes. --- .../metron/hbase/client/HBaseClientTest.java | 49 ++++++++++--------- 1 file changed, 25 insertions(+), 24 deletions(-) diff --git a/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java index 27544c009d..1849745f7d 100644 --- a/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java +++ b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/client/HBaseClientTest.java @@ -20,13 +20,12 @@ package org.apache.metron.hbase.client; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.*; import org.apache.storm.tuple.Tuple; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.HTableInterface; -import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.util.Bytes; import org.apache.metron.hbase.Widget; import org.apache.metron.hbase.WidgetMapper; @@ -40,6 +39,7 @@ import org.junit.BeforeClass; import org.junit.Test; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.concurrent.TimeUnit; @@ -61,8 +61,9 @@ public class HBaseClientTest { private static final String tableName = "widgets"; private static HBaseTestingUtility util; - private HBaseClient client; - private HTableInterface table; + private static HBaseClient client; + private static HTableInterface table; + private static Admin admin; private Tuple tuple1; private Tuple tuple2; byte[] rowKey1; @@ -80,17 +81,36 @@ public static void startHBase() throws Exception { config.set("hbase.regionserver.hostname", "localhost"); util = new HBaseTestingUtility(config); util.startMiniCluster(); + admin = util.getHBaseAdmin(); + // create the table + table = util.createTable(Bytes.toBytes(tableName), WidgetMapper.CF); + util.waitTableEnabled(table.getName()); + // setup the client + client = new HBaseClient((c,t) -> table, table.getConfiguration(), tableName); } @AfterClass public static void stopHBase() throws Exception { + util.deleteTable(tableName); util.shutdownMiniCluster(); util.cleanupTestDir(); } + @After + public void clearTable() throws Exception { + List deletions = new ArrayList<>(); + for(Result r : table.getScanner(new Scan())) { + deletions.add(new Delete(r.getRow())); + } + table.delete(deletions); + } + @Before public void setupTuples() throws Exception { + // create a mapper + mapper = new WidgetMapper(); + // setup the first tuple widget1 = new Widget("widget1", 100); tuple1 = mock(Tuple.class); @@ -108,25 +128,6 @@ public void setupTuples() throws Exception { cols2 = mapper.columns(tuple2); } - @Before - public void setup() throws Exception { - - // create a mapper - mapper = new WidgetMapper(); - - // create the table - table = util.createTable(Bytes.toBytes(tableName), WidgetMapper.CF); - util.waitTableEnabled(table.getName()); - - // setup the client - client = new HBaseClient((c,t) -> table, table.getConfiguration(), tableName); - } - - @After - public void tearDown() throws Exception { - util.deleteTable(tableName); - } - /** * Should be able to read/write a single Widget. */ From 9033f49f19e19079746993c9f0e7fe43886df1d2 Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 09:53:38 +0100 Subject: [PATCH 22/43] Setting global cache for npm and removing int tests for initial run. --- .travis.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 991a612901..d2484223f8 100644 --- a/.travis.yml +++ b/.travis.yml @@ -15,11 +15,13 @@ before_install: - unzip -qq apache-maven-3.3.9-bin.zip - export M2_HOME=$PWD/apache-maven-3.3.9 - export PATH=$M2_HOME/bin:$PATH + - npm config set cache $HOME\.npm-cache --global script: - | - time mvn -q -T 2C -DskipTests install && time mvn -q -T 2C surefire:test@unit-tests && time mvn -q surefire:test@integration-tests && time mvn -q test --projects metron-interface/metron-config && time build_utils/verify_licenses.sh + time mvn -q -T 2C -DskipTests install && time mvn -q -T 2C surefire:test@unit-tests && time mvn -q test --projects metron-interface/metron-config && time build_utils/verify_licenses.sh before_cache: - rm -rf $HOME/.m2/repository/org/apache/metron cache: directories: + - $HOME/.npm-cache - $HOME/.m2 From a6f2e9f384558799d094c427a774af42241209c3 Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 10:58:08 +0100 Subject: [PATCH 23/43] Updating travis. --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index d2484223f8..0e33cf65ad 100644 --- a/.travis.yml +++ b/.travis.yml @@ -15,7 +15,7 @@ before_install: - unzip -qq apache-maven-3.3.9-bin.zip - export M2_HOME=$PWD/apache-maven-3.3.9 - export PATH=$M2_HOME/bin:$PATH - - npm config set cache $HOME\.npm-cache --global + - npm config set cache $HOME/.npm-cache --global script: - | time mvn -q -T 2C -DskipTests install && time mvn -q -T 2C surefire:test@unit-tests && time mvn -q test --projects metron-interface/metron-config && time build_utils/verify_licenses.sh From 46aacd7d4145c7cdbefb6949398722c79a1b82f3 Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 14:43:36 +0100 Subject: [PATCH 24/43] Updating travis to cache all forms of npm cache. --- .travis.yml | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 0e33cf65ad..6fb31f0d7f 100644 --- a/.travis.yml +++ b/.travis.yml @@ -16,12 +16,17 @@ before_install: - export M2_HOME=$PWD/apache-maven-3.3.9 - export PATH=$M2_HOME/bin:$PATH - npm config set cache $HOME/.npm-cache --global + - npm config set prefix $HOME/.npm-prefix --global + - [ -d $HOME/.npm-local/node_modules ] && mv $HOME/.npm-local/node_modules metron/metron-interface/metron-config/node_modules || mkdir -p $HOME/.npm-local/node_modules script: - | - time mvn -q -T 2C -DskipTests install && time mvn -q -T 2C surefire:test@unit-tests && time mvn -q test --projects metron-interface/metron-config && time build_utils/verify_licenses.sh + time mvn -q -T 2C -DskipTests install before_cache: - rm -rf $HOME/.m2/repository/org/apache/metron + - [ -d metron/metron-interface/metron-config/node_modules ] && mv metron/metron-interface/metron-config/node_modules $HOME/.npm-local/node_modules cache: directories: - $HOME/.npm-cache + - $HOME/.npm-prefix + - $HOME/.npm-local/node_modules - $HOME/.m2 From a8368fefe06c1c261a8c71f204efe82bcf836c72 Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 14:50:04 +0100 Subject: [PATCH 25/43] Updating. --- .travis.yml | 5 +++-- pop_npm_cache.sh | 20 ++++++++++++++++++++ push_npm_cache.sh | 20 ++++++++++++++++++++ 3 files changed, 43 insertions(+), 2 deletions(-) create mode 100755 pop_npm_cache.sh create mode 100755 push_npm_cache.sh diff --git a/.travis.yml b/.travis.yml index 6fb31f0d7f..11af8fc3d2 100644 --- a/.travis.yml +++ b/.travis.yml @@ -17,13 +17,14 @@ before_install: - export PATH=$M2_HOME/bin:$PATH - npm config set cache $HOME/.npm-cache --global - npm config set prefix $HOME/.npm-prefix --global - - [ -d $HOME/.npm-local/node_modules ] && mv $HOME/.npm-local/node_modules metron/metron-interface/metron-config/node_modules || mkdir -p $HOME/.npm-local/node_modules + - bash metron/push_npm_cache.sh script: - | time mvn -q -T 2C -DskipTests install before_cache: - rm -rf $HOME/.m2/repository/org/apache/metron - - [ -d metron/metron-interface/metron-config/node_modules ] && mv metron/metron-interface/metron-config/node_modules $HOME/.npm-local/node_modules + - bash metron/pop_npm_cache.sh + cache: directories: - $HOME/.npm-cache diff --git a/pop_npm_cache.sh b/pop_npm_cache.sh new file mode 100755 index 0000000000..1d6536a154 --- /dev/null +++ b/pop_npm_cache.sh @@ -0,0 +1,20 @@ +#!/bin/bash +#. +# 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. +# + +[ -d metron/metron-interface/metron-config/node_modules ] && mv metron/metron-interface/metron-config/node_modules $HOME/.npm-local/node_modules diff --git a/push_npm_cache.sh b/push_npm_cache.sh new file mode 100755 index 0000000000..045b81655c --- /dev/null +++ b/push_npm_cache.sh @@ -0,0 +1,20 @@ +#!/bin/bash +#. +# 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. +#. + +[ -d $HOME/.npm-local/node_modules ] && mv $HOME/.npm-local/node_modules metron/metron-interface/metron-config/node_modules || mkdir -p $HOME/.npm-local/node_modules From e208bcad07c2b28c75fbfe5ccb81fec70fe81bc7 Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 14:55:24 +0100 Subject: [PATCH 26/43] updating travis again. --- .travis.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 11af8fc3d2..353afaddbf 100644 --- a/.travis.yml +++ b/.travis.yml @@ -17,13 +17,13 @@ before_install: - export PATH=$M2_HOME/bin:$PATH - npm config set cache $HOME/.npm-cache --global - npm config set prefix $HOME/.npm-prefix --global - - bash metron/push_npm_cache.sh + - bash ./push_npm_cache.sh script: - | time mvn -q -T 2C -DskipTests install before_cache: - rm -rf $HOME/.m2/repository/org/apache/metron - - bash metron/pop_npm_cache.sh + - bash ./pop_npm_cache.sh cache: directories: From fdddd6d8740543695383705cb9dd6e05c1e7da10 Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 15:10:23 +0100 Subject: [PATCH 27/43] Updating travis. --- .travis.yml | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index 353afaddbf..a380295de2 100644 --- a/.travis.yml +++ b/.travis.yml @@ -17,17 +17,15 @@ before_install: - export PATH=$M2_HOME/bin:$PATH - npm config set cache $HOME/.npm-cache --global - npm config set prefix $HOME/.npm-prefix --global - - bash ./push_npm_cache.sh script: - | time mvn -q -T 2C -DskipTests install before_cache: - rm -rf $HOME/.m2/repository/org/apache/metron - - bash ./pop_npm_cache.sh cache: directories: - $HOME/.npm-cache - $HOME/.npm-prefix - - $HOME/.npm-local/node_modules + - metron/metron-interface/metron-config/node_modules - $HOME/.m2 From de042bfeaa902b2d8ac92824db7a538689fb4d1f Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 16:04:35 +0100 Subject: [PATCH 28/43] travis update --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index a380295de2..71da2b89ec 100644 --- a/.travis.yml +++ b/.travis.yml @@ -27,5 +27,5 @@ cache: directories: - $HOME/.npm-cache - $HOME/.npm-prefix - - metron/metron-interface/metron-config/node_modules + - metron-interface/metron-config/node_modules - $HOME/.m2 From cb930fcfddab3e60d25422d967607482160adc65 Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 16:29:57 +0100 Subject: [PATCH 29/43] adding longer timeout for great success. --- .travis.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.travis.yml b/.travis.yml index 71da2b89ec..44b11c02d2 100644 --- a/.travis.yml +++ b/.travis.yml @@ -24,6 +24,7 @@ before_cache: - rm -rf $HOME/.m2/repository/org/apache/metron cache: + timeout: 1000 directories: - $HOME/.npm-cache - $HOME/.npm-prefix From af9f186b06e27dd6f1e6641c2040416972b9a677 Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 16:43:19 +0100 Subject: [PATCH 30/43] removing quiet mode --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 44b11c02d2..067ce7fa10 100644 --- a/.travis.yml +++ b/.travis.yml @@ -19,7 +19,7 @@ before_install: - npm config set prefix $HOME/.npm-prefix --global script: - | - time mvn -q -T 2C -DskipTests install + time mvn -T 2C -DskipTests install before_cache: - rm -rf $HOME/.m2/repository/org/apache/metron From 0e3cf34aaf97cfb17becc4f4b6396f92a8de8e6e Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 17:04:19 +0100 Subject: [PATCH 31/43] Resetting cache. --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 067ce7fa10..96e5779735 100644 --- a/.travis.yml +++ b/.travis.yml @@ -19,7 +19,7 @@ before_install: - npm config set prefix $HOME/.npm-prefix --global script: - | - time mvn -T 2C -DskipTests install + time mvn -q -T 2C -DskipTests install && time mvn -q -T 2C surefire:test@unit-tests && time mvn -q surefire:test@integration-tests && time mvn -q test --projects metron-interface/metron-config && time build_utils/verify_licenses.sh before_cache: - rm -rf $HOME/.m2/repository/org/apache/metron From 9e0911e12d5129774086a1064e679c74b7045fb2 Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 28 Jun 2017 17:04:46 +0100 Subject: [PATCH 32/43] Removed extraneous shell scripts. --- pop_npm_cache.sh | 20 -------------------- push_npm_cache.sh | 20 -------------------- 2 files changed, 40 deletions(-) delete mode 100755 pop_npm_cache.sh delete mode 100755 push_npm_cache.sh diff --git a/pop_npm_cache.sh b/pop_npm_cache.sh deleted file mode 100755 index 1d6536a154..0000000000 --- a/pop_npm_cache.sh +++ /dev/null @@ -1,20 +0,0 @@ -#!/bin/bash -#. -# 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. -# - -[ -d metron/metron-interface/metron-config/node_modules ] && mv metron/metron-interface/metron-config/node_modules $HOME/.npm-local/node_modules diff --git a/push_npm_cache.sh b/push_npm_cache.sh deleted file mode 100755 index 045b81655c..0000000000 --- a/push_npm_cache.sh +++ /dev/null @@ -1,20 +0,0 @@ -#!/bin/bash -#. -# 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. -#. - -[ -d $HOME/.npm-local/node_modules ] && mv $HOME/.npm-local/node_modules metron/metron-interface/metron-config/node_modules || mkdir -p $HOME/.npm-local/node_modules From e8e13a88d662c64c25c8197c2b425800b3071858 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 29 Jun 2017 11:00:34 +0100 Subject: [PATCH 33/43] Migrating the parser integration tests to quasi-unit-tests. --- .../metron/common/bolt/ConfiguredBolt.java | 4 + .../metron/parsers/bolt/ParserBolt.java | 18 +- .../parsers/integration/ParserDriver.java | 168 ++++++++++++++++++ .../integration/ParserIntegrationTest.java | 157 +++------------- 4 files changed, 210 insertions(+), 137 deletions(-) create mode 100644 metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserDriver.java diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java index 45463bf1aa..8163981d19 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/bolt/ConfiguredBolt.java @@ -69,6 +69,10 @@ public CONFIG_T getConfigurations() { @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + prepCache(); + } + + protected void prepCache() { try { if (client == null) { RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3); diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java index 2c43c2331c..d84df51ec9 100644 --- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java +++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java @@ -170,16 +170,20 @@ public void execute(Tuple tuple) { collector.ack(tuple); } } catch (Throwable ex) { - MetronError error = new MetronError() - .withErrorType(Constants.ErrorType.PARSER_ERROR) - .withThrowable(ex) - .withSensorType(getSensorType()) - .addRawMessage(originalMessage); - ErrorUtils.handleError(collector, error); - collector.ack(tuple); + handleError(originalMessage, tuple, ex, collector); } } + protected void handleError(byte[] originalMessage, Tuple tuple, Throwable ex, OutputCollector collector) { + MetronError error = new MetronError() + .withErrorType(Constants.ErrorType.PARSER_ERROR) + .withThrowable(ex) + .withSensorType(getSensorType()) + .addRawMessage(originalMessage); + ErrorUtils.handleError(collector, error); + collector.ack(tuple); + } + private List getFailedValidators(JSONObject input, List validators) { List failedValidators = new ArrayList<>(); for(FieldValidator validator : validators) { diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserDriver.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserDriver.java new file mode 100644 index 0000000000..fb1e0d0154 --- /dev/null +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserDriver.java @@ -0,0 +1,168 @@ +/** + * 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.parsers.integration; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.collect.ImmutableList; +import org.apache.metron.common.configuration.ConfigurationsUtils; +import org.apache.metron.common.configuration.FieldValidator; +import org.apache.metron.common.configuration.ParserConfigurations; +import org.apache.metron.common.configuration.SensorParserConfig; +import org.apache.metron.common.configuration.writer.WriterConfiguration; +import org.apache.metron.common.utils.JSONUtils; +import org.apache.metron.common.utils.ReflectionUtils; +import org.apache.metron.common.writer.MessageWriter; +import org.apache.metron.integration.ProcessorResult; +import org.apache.metron.parsers.bolt.ParserBolt; +import org.apache.metron.parsers.bolt.WriterHandler; +import org.apache.metron.parsers.interfaces.MessageParser; +import org.apache.storm.generated.GlobalStreamId; +import org.apache.storm.task.GeneralTopologyContext; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.MessageId; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.TupleImpl; +import org.json.simple.JSONObject; +import org.mockito.Matchers; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ParserDriver { + private static final Logger LOG = LoggerFactory.getLogger(ParserBolt.class); + public static class CollectingWriter implements MessageWriter{ + List output; + public CollectingWriter(List output) { + this.output = output; + } + + @Override + public void init() { + + } + + @Override + public void write(String sensorType, WriterConfiguration configurations, Tuple tuple, JSONObject message) throws Exception { + output.add(message.toJSONString().getBytes()); + } + + @Override + public String getName() { + return "collecting"; + } + + @Override + public void close() throws Exception { + } + + public List getOutput() { + return output; + } + } + + private class ShimParserBolt extends ParserBolt { + List output; + List errors = new ArrayList<>(); + public ShimParserBolt(List output) { + super(null + , sensorType == null?config.getSensorTopic():sensorType + , ReflectionUtils.createInstance(config.getParserClassName()) + , new WriterHandler( new CollectingWriter(output)) + ); + this.output = output; + } + + @Override + public ParserConfigurations getConfigurations() { + return new ParserConfigurations() { + @Override + public SensorParserConfig getSensorParserConfig(String sensorType) { + return config; + } + + @Override + public Map getGlobalConfig() { + return globalConfig; + } + + @Override + public List getFieldValidations() { + return new ArrayList<>(); + } + }; + } + + @Override + protected void prepCache() { + } + + @Override + protected void handleError(byte[] originalMessage, Tuple tuple, Throwable ex, OutputCollector collector) { + errors.add(originalMessage); + LOG.error("Error parsing message: " + ex.getMessage(), ex); + } + + public ProcessorResult> getResults() { + return new ProcessorResult.Builder>().withProcessErrors(errors) + .withResult(output) + .build(); + + } + } + + + private SensorParserConfig config; + private Map globalConfig; + private String sensorType; + + public ParserDriver(String sensorType, String parserConfig, String globalConfig) throws IOException { + config = SensorParserConfig.fromBytes(parserConfig.getBytes()); + this.sensorType = sensorType; + this.globalConfig = JSONUtils.INSTANCE.load(globalConfig, new TypeReference>() { + }); + } + + public ProcessorResult> run(List in) { + ShimParserBolt bolt = new ShimParserBolt(new ArrayList<>()); + OutputCollector collector = mock(OutputCollector.class); + bolt.prepare(null, null, collector); + for(byte[] record : in) { + bolt.execute(toTuple(record)); + } + return bolt.getResults(); + } + + public Tuple toTuple(byte[] record) { + Tuple ret = mock(Tuple.class); + when(ret.getBinary(eq(0))).thenReturn(record); + return ret; + } + +} diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java index 951e666a37..ef722eb364 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java @@ -38,133 +38,52 @@ import org.junit.Test; import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; import java.util.*; public abstract class ParserIntegrationTest extends BaseIntegrationTest { - protected static final String ERROR_TOPIC = "parser_error"; protected List inputMessages; - protected static ParserTopologyComponent parserTopologyComponent; - protected static KafkaComponent kafkaComponent; - protected static ConfigUploadComponent configUploadComponent; - protected static ComponentRunner runner; - @BeforeClass - public static void setupBeforeClass() throws UnableToStartException { - final Properties topologyProperties = new Properties(); - kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList() {{ - add(new KafkaComponent.Topic(Constants.ENRICHMENT_TOPIC, 1)); - add(new KafkaComponent.Topic(ERROR_TOPIC,1)); - }}); - topologyProperties.setProperty("kafka.broker", kafkaComponent.getBrokerList()); - - ZKServerComponent zkServerComponent = getZKServerComponent(topologyProperties); - - configUploadComponent = new ConfigUploadComponent() - .withTopologyProperties(topologyProperties) - .withGlobalConfigsPath(TestConstants.SAMPLE_CONFIG_PATH) - .withParserConfigsPath(TestConstants.PARSER_CONFIGS_PATH); - - parserTopologyComponent = new ParserTopologyComponent.Builder() - .withTopologyProperties(topologyProperties) - .withOutputTopic(Constants.ENRICHMENT_TOPIC) - .withBrokerUrl(kafkaComponent.getBrokerList()).build(); - - //UnitTestHelper.verboseLogging(); - runner = new ComponentRunner.Builder() - .withComponent("zk", zkServerComponent) - .withComponent("kafka", kafkaComponent) - .withComponent("config", configUploadComponent) - .withComponent("org/apache/storm", parserTopologyComponent) - .withMillisecondsBetweenAttempts(5000) - .withNumRetries(10) - .withCustomStartupOrder(new String[] {"zk", "kafka", "config"}) // Skip starting Parser Topology and config - .withCustomShutdownOrder(new String[] {"org/apache/storm","config","kafka","zk"}) - .build(); - - runner.start(); - } - - @AfterClass - public static void teardownAfterClass() { - runner.stop(); + private String readGlobalConfig() throws IOException { + File configsRoot = new File(TestConstants.SAMPLE_CONFIG_PATH); + return new String(Files.readAllBytes(new File(configsRoot, "global.json").toPath())); } - @Before - public void setup() { - } - - @After - public void teardown() { - runner.reset(); + private String readSensorConfig(String sensorType) throws IOException { + File configsRoot = new File(TestConstants.PARSER_CONFIGS_PATH); + File parsersRoot = new File(configsRoot, "parsers"); + return new String(Files.readAllBytes(new File(parsersRoot, sensorType + ".json").toPath())); } @Test public void test() throws Exception { String sensorType = getSensorType(); + ParserDriver driver = new ParserDriver(sensorType, readSensorConfig(sensorType), readGlobalConfig()); inputMessages = TestUtils.readSampleData(SampleDataUtils.getSampleDataPath(sensorType, TestDataType.RAW)); - parserTopologyComponent.updateSensorType(sensorType); - parserTopologyComponent.start(); - - kafkaComponent.createTopic(sensorType, 1, true); - -// final Properties topologyProperties = new Properties(); -// final KafkaComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList() {{ -// add(new KafkaComponent.Topic(sensorType, 1)); -// add(new KafkaComponent.Topic(Constants.ENRICHMENT_TOPIC, 1)); -// add(new KafkaComponent.Topic(ERROR_TOPIC,1)); -// }}); -// topologyProperties.setProperty("kafka.broker", kafkaComponent.getBrokerList()); - -// ZKServerComponent zkServerComponent = getZKServerComponent(topologyProperties); -// -// ConfigUploadComponent configUploadComponent = new ConfigUploadComponent() -// .withTopologyProperties(topologyProperties) -// .withGlobalConfigsPath(TestConstants.SAMPLE_CONFIG_PATH) -// .withParserConfigsPath(TestConstants.PARSER_CONFIGS_PATH); -// -// ParserTopologyComponent parserTopologyComponent = new ParserTopologyComponent.Builder() -// .withSensorType(sensorType) -// .withTopologyProperties(topologyProperties) -// .withOutputTopic(Constants.ENRICHMENT_TOPIC) -// .withBrokerUrl(kafkaComponent.getBrokerList()).build(); - //UnitTestHelper.verboseLogging(); -// ComponentRunner runner = new ComponentRunner.Builder() -// .withComponent("zk", zkServerComponent) -// .withComponent("kafka", kafkaComponent) -// .withComponent("config", configUploadComponent) -// .withComponent("org/apache/storm", parserTopologyComponent) -// .withMillisecondsBetweenAttempts(5000) -// .withNumRetries(10) -// .withCustomShutdownOrder(new String[] {"org/apache/storm","config","kafka","zk"}) -// .build(); -// try { -// runner.start(); - kafkaComponent.writeMessages(sensorType, inputMessages); - ProcessorResult> result = runner.process(getProcessor()); - List outputMessages = result.getResult(); - StringBuffer buffer = new StringBuffer(); - if (result.failed()){ - result.getBadResults(buffer); - buffer.append(String.format("%d Valid Messages Processed", outputMessages.size())).append("\n"); + ProcessorResult> result = driver.run(inputMessages); + List outputMessages = result.getResult(); + StringBuffer buffer = new StringBuffer(); + if (result.failed()){ + result.getBadResults(buffer); + buffer.append(String.format("%d Valid Messages Processed", outputMessages.size())).append("\n"); + dumpParsedMessages(outputMessages,buffer); + Assert.fail(buffer.toString()); + } else { + List validations = getValidations(); + if (validations == null || validations.isEmpty()) { + buffer.append("No validations configured for sensorType " + sensorType + ". Dumping parsed messages").append("\n"); dumpParsedMessages(outputMessages,buffer); Assert.fail(buffer.toString()); } else { - List validations = getValidations(); - if (validations == null || validations.isEmpty()) { - buffer.append("No validations configured for sensorType " + sensorType + ". Dumping parsed messages").append("\n"); - dumpParsedMessages(outputMessages,buffer); - Assert.fail(buffer.toString()); - } else { - for (ParserValidation validation : validations) { - System.out.println("Running " + validation.getName() + " on sensorType " + sensorType); - validation.validate(sensorType, outputMessages); - } + for (ParserValidation validation : validations) { + System.out.println("Running " + validation.getName() + " on sensorType " + sensorType); + validation.validate(sensorType, outputMessages); } } -// } finally { -// runner.stop(); -// } + } } public void dumpParsedMessages(List outputMessages, StringBuffer buffer) { @@ -173,28 +92,6 @@ public void dumpParsedMessages(List outputMessages, StringBuffer buffer) } } - @SuppressWarnings("unchecked") - private KafkaProcessor> getProcessor(){ - - return new KafkaProcessor<>() - .withKafkaComponentName("kafka") - .withReadTopic(Constants.ENRICHMENT_TOPIC) - .withErrorTopic(ERROR_TOPIC) - .withValidateReadMessages(new Function() { - @Nullable - @Override - public Boolean apply(@Nullable KafkaMessageSet messageSet) { - return (messageSet.getMessages().size() + messageSet.getErrors().size() == inputMessages.size()); - } - }) - .withProvideResult(new Function>(){ - @Nullable - @Override - public List apply(@Nullable KafkaMessageSet messageSet) { - return messageSet.getMessages(); - } - }); - } abstract String getSensorType(); abstract List getValidations(); From 195960b61fc688b30cd716763a7e6f65a74c77b4 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 29 Jun 2017 11:41:25 +0100 Subject: [PATCH 34/43] allowing grok parsers to work too. --- .../main/java/org/apache/metron/parsers/bolt/ParserBolt.java | 4 ++++ .../org/apache/metron/parsers/integration/ParserDriver.java | 2 ++ 2 files changed, 6 insertions(+) diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java index d84df51ec9..56506a70c3 100644 --- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java +++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/ParserBolt.java @@ -79,6 +79,10 @@ public ParserBolt withMessageFilter(MessageFilter filter) { return this; } + public MessageParser getParser() { + return parser; + } + @SuppressWarnings("unchecked") @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserDriver.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserDriver.java index fb1e0d0154..b844104026 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserDriver.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserDriver.java @@ -90,6 +90,7 @@ public List getOutput() { private class ShimParserBolt extends ParserBolt { List output; List errors = new ArrayList<>(); + public ShimParserBolt(List output) { super(null , sensorType == null?config.getSensorTopic():sensorType @@ -97,6 +98,7 @@ public ShimParserBolt(List output) { , new WriterHandler( new CollectingWriter(output)) ); this.output = output; + getParser().configure(config.getParserConfig()); } @Override From 87c21c08531d617e0c96acd7d483e58dbe18d794 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 29 Jun 2017 12:15:37 +0100 Subject: [PATCH 35/43] Making the kafka integration test more resilient. --- .../KafkaControllerIntegrationTest.java | 59 ++++++++++++++++--- 1 file changed, 50 insertions(+), 9 deletions(-) diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/KafkaControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/KafkaControllerIntegrationTest.java index 745bc560d7..26fa24e997 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/KafkaControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/KafkaControllerIntegrationTest.java @@ -62,6 +62,38 @@ public class KafkaControllerIntegrationTest { @Autowired private KafkaComponent kafkaWithZKComponent; + + interface Evaluation { + void tryTest() throws Exception; + } + + private void testAndRetry(Evaluation evaluation) throws Exception{ + testAndRetry(KAFKA_RETRY, evaluation); + } + + private void testAndRetry(int numRetries, Evaluation evaluation) throws Exception { + AssertionError lastError = null; + for(int i = 0;i < numRetries;++i) { + try { + evaluation.tryTest(); + return; + } + catch(AssertionError error) { + if(error.getMessage().contains("but was:<404>")) { + lastError = error; + Thread.sleep(1000); + continue; + } + else { + throw error; + } + } + } + if(lastError != null) { + throw lastError; + } + } + class SampleDataRunner implements Runnable { private boolean stop = false; @@ -142,34 +174,40 @@ public void test() throws Exception { this.kafkaService.deleteTopic("bro"); this.kafkaService.deleteTopic("someTopic"); Thread.sleep(1000); + testAndRetry(() -> this.mockMvc.perform(delete(kafkaUrl + "/topic/bro").with(httpBasic(user,password)).with(csrf())) + .andExpect(status().isNotFound()) + ); - this.mockMvc.perform(delete(kafkaUrl + "/topic/bro").with(httpBasic(user,password)).with(csrf())) - .andExpect(status().isNotFound()); - + testAndRetry(() -> this.mockMvc.perform(post(kafkaUrl + "/topic").with(httpBasic(user,password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(broTopic)) .andExpect(status().isCreated()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) .andExpect(jsonPath("$.name").value("bro")) .andExpect(jsonPath("$.numPartitions").value(1)) - .andExpect(jsonPath("$.replicationFactor").value(1)); - + .andExpect(jsonPath("$.replicationFactor").value(1)) + ); sampleDataThread.start(); Thread.sleep(1000); - + testAndRetry(() -> this.mockMvc.perform(get(kafkaUrl + "/topic/bro").with(httpBasic(user,password))) .andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) .andExpect(jsonPath("$.name").value("bro")) .andExpect(jsonPath("$.numPartitions").value(1)) - .andExpect(jsonPath("$.replicationFactor").value(1)); + .andExpect(jsonPath("$.replicationFactor").value(1)) + ); + this.mockMvc.perform(get(kafkaUrl + "/topic/someTopic").with(httpBasic(user,password))) .andExpect(status().isNotFound()); + testAndRetry(() -> this.mockMvc.perform(get(kafkaUrl + "/topic").with(httpBasic(user,password))) .andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) - .andExpect(jsonPath("$", Matchers.hasItem("bro"))); + .andExpect(jsonPath("$", Matchers.hasItem("bro"))) + ); + for(int i = 0;i < KAFKA_RETRY;++i) { MvcResult result = this.mockMvc.perform(get(kafkaUrl + "/topic/bro/sample").with(httpBasic(user, password))) .andReturn(); @@ -178,10 +216,13 @@ public void test() throws Exception { } Thread.sleep(1000); } + + testAndRetry(() -> this.mockMvc.perform(get(kafkaUrl + "/topic/bro/sample").with(httpBasic(user,password))) .andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("text/plain;charset=UTF-8"))) - .andExpect(jsonPath("$").isNotEmpty()); + .andExpect(jsonPath("$").isNotEmpty()) + ); this.mockMvc.perform(get(kafkaUrl + "/topic/someTopic/sample").with(httpBasic(user,password))) .andExpect(status().isNotFound()); From 63cad19dcefeeb509441d970a58e2270144de357 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 29 Jun 2017 16:27:58 +0100 Subject: [PATCH 36/43] Adding integration test document for parsers. --- .../metron-parsers/parser-testing.md | 70 +++++++++++++++++++ .../integration/ParserIntegrationTest.java | 4 +- 2 files changed, 72 insertions(+), 2 deletions(-) create mode 100644 metron-platform/metron-parsers/parser-testing.md diff --git a/metron-platform/metron-parsers/parser-testing.md b/metron-platform/metron-parsers/parser-testing.md new file mode 100644 index 0000000000..e30a7b703e --- /dev/null +++ b/metron-platform/metron-parsers/parser-testing.md @@ -0,0 +1,70 @@ +# Parser Contribution and Testing + +So you want to contribute a parser to Apache Metron. First off, on behalf +of the community, thank you very much! Now that you have implemented a parser +by writing a java class which implements `org.apache.metron.parsers.interfaces.MessageParser` +what are the testing expectations for a new parser? + +It is expected that a new parser have two tests: +* A JUnit test directly testing your parser class. +* An Integration test validating that your parser class can parse messages +inside the `ParserBolt`. + +## The JUnit Test + +The JUnit Test should be focused on testing your Parser directly. You +should feel free to use mocks or stubs or whatever else you need to completely +test that unit of functionality. + +## The Integration Test + +Integration tests are more structured. The intent is that the parser that +you have implemented can be driven successfully from `org.apache.metron.parsers.bolt.ParserBolt`. + +The procedure for creating a new test is as follows: +* Create an integration test that extends `org.apache.metron.parsers.integration.ParserIntegrationTest` + * Override `getSensorType()` to return the sensor type to be used in the test (referred to as `${sensor_type}` at times) + * Override `getValidations()` to indicate how you want the output of the parser to be validated (more on validations later) + * Optionally `readSensorConfig(String sensorType)` to read the sensor config + * By default, we will pull this from `metron-parsers/src/main/config/zookeeper/parsers/${sensor_type}`. Override if you want to provide your own + * Optionally `readGlobalConfig()` to return the global config + * By default, we will pull this from `metron-integration-test/src/main/config/zookeeper/global.json)`. Override if you want to provide your own +* Place sample input data in `metron-integration-test/src/main/sample/data/${sensor_type}/raw` + * It should be one line per input record. +* Place expected output based on sample data in `metron-integration-test/src/main/sample/data/${sensor_type}/parsed` + * Line `k` in the expected data should match with line `k` + +The way these tests function is by creating a `ParserBolt` instance with your specified global configuration and +sensor configuration. It will then send your specified sample input data in line-by-line. It will then +perform some basic sanity validation: +* Ensure no errors were logged +* Execute your specified validation methods + +### Validations + +Validations are functions which indicate how one should validate the parsed messages. The basic one which is sufficient +for most cases is `org.apache.metron.parsers.integration.validation.SampleDataValidation`. This will read the expected results +from `metron-integration-test/src/main/sample/data/${sensor_type}/parsed` and validate that the actual parsed message +conforms (excluding timestamp). + +If you have special validations required, you may implement your own and return an instance of that in the `getValidations()` +method of your Integration Test. + +### Sample Integration Test + +A sample integration test for the `snort` parser is as follows: +``` +public class SnortIntegrationTest extends ParserIntegrationTest { + @Override + String getSensorType() { + return "snort"; + } + + @Override + List getValidations() { + return new ArrayList() {{ + add(new SampleDataValidation()); + }}; + } +} +``` diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java index ef722eb364..cd3d005411 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/integration/ParserIntegrationTest.java @@ -46,12 +46,12 @@ public abstract class ParserIntegrationTest extends BaseIntegrationTest { protected List inputMessages; - private String readGlobalConfig() throws IOException { + protected String readGlobalConfig() throws IOException { File configsRoot = new File(TestConstants.SAMPLE_CONFIG_PATH); return new String(Files.readAllBytes(new File(configsRoot, "global.json").toPath())); } - private String readSensorConfig(String sensorType) throws IOException { + protected String readSensorConfig(String sensorType) throws IOException { File configsRoot = new File(TestConstants.PARSER_CONFIGS_PATH); File parsersRoot = new File(configsRoot, "parsers"); return new String(Files.readAllBytes(new File(parsersRoot, sensorType + ".json").toPath())); From 5c9780e6b59900acb639b69e209ba5244f056e00 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Thu, 29 Jun 2017 09:08:37 -0400 Subject: [PATCH 37/43] Trying out a VM instead of container --- .travis.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.travis.yml b/.travis.yml index 96e5779735..97a816c36f 100644 --- a/.travis.yml +++ b/.travis.yml @@ -6,6 +6,7 @@ addons: - ubuntu-toolchain-r-test packages: - g++-4.8 +sudo: required install: true language: java jdk: From 6c8fe98b65a9ed302f58c943aee429b075889af3 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Thu, 29 Jun 2017 14:18:53 -0400 Subject: [PATCH 38/43] responding to review comment --- .../integration/components/ElasticSearchComponent.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java index db96ad0b78..7766fe30eb 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java @@ -24,8 +24,6 @@ import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction; import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; -import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; -import org.elasticsearch.action.admin.indices.get.GetIndexRequest; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; From 730c1c44fcc7e4d35b32830301b3e4ad1be08757 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Thu, 29 Jun 2017 14:38:01 -0400 Subject: [PATCH 39/43] Adding back in PcapTopologyIntegrationTest.testTimestampInPacket, but disabling --- .../PcapTopologyIntegrationTest.java | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java index 3b0f20d810..e988c3097d 100644 --- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java +++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java @@ -64,6 +64,7 @@ import org.apache.metron.test.utils.UnitTestHelper; import org.json.simple.JSONObject; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Test; public class PcapTopologyIntegrationTest { @@ -89,6 +90,26 @@ public boolean accept(File dir, String name) { }).length; } + // This will eventually be completely deprecated. As it takes a significant amount of testing, the test is being disabled. + @Ignore + @Test + public void testTimestampInPacket() throws Exception { + testTopology(new Function() { + @Nullable + @Override + public Void apply(@Nullable Properties input) { + input.setProperty("kafka.pcap.ts_scheme", Deserializers.FROM_PACKET.toString()); + return null; + } + }, (kafkaComponent, pcapEntries) -> kafkaComponent.writeMessages( KAFKA_TOPIC + , Collections2.transform(pcapEntries + , input -> input.getValue() + ) + ) + , true + ); + } + @Test public void testTimestampInKey() throws Exception { testTopology(new Function() { From bb6007be669bb5ebca68241fd9d9b861d6ff4be9 Mon Sep 17 00:00:00 2001 From: Casey Stella Date: Thu, 29 Jun 2017 21:55:39 +0100 Subject: [PATCH 40/43] kafka weirdness fixed (maybe) (#13) * Catching weird kafak issue. * removing extraneous imports. * reverted pom file change. --- metron-interface/metron-rest/pom.xml | 1 + .../apache/metron/rest/config/TestConfig.java | 3 +-- .../integration/components/KafkaComponent.java | 18 +++++++++--------- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/metron-interface/metron-rest/pom.xml b/metron-interface/metron-rest/pom.xml index 1c3ff928d9..94ed64b925 100644 --- a/metron-interface/metron-rest/pom.xml +++ b/metron-interface/metron-rest/pom.xml @@ -257,6 +257,7 @@ 1.7 test + diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java index adfe056faa..01e4e03e78 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java @@ -65,8 +65,7 @@ public KafkaComponent kafkaWithZKComponent(Properties zkProperties) { return new KafkaComponent().withTopologyProperties(zkProperties); } - //@Bean(destroyMethod = "stop") - @Bean + @Bean(destroyMethod = "stop") public ComponentRunner componentRunner(ZKServerComponent zkServerComponent, KafkaComponent kafkaWithZKComponent) { ComponentRunner runner = new ComponentRunner.Builder() .withComponent("zk", zkServerComponent) 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 2ba79e4ab4..d9c1dcc67b 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 @@ -19,10 +19,6 @@ import com.google.common.base.Function; -import java.io.File; -import java.io.FileFilter; -import java.io.IOException; -import kafka.admin.AdminUtils; import kafka.api.FetchRequest; import kafka.api.FetchRequestBuilder; import kafka.common.TopicExistsException; @@ -35,9 +31,6 @@ import kafka.message.MessageAndOffset; import kafka.server.*; import kafka.utils.TestUtils; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.filefilter.IOFileFilter; -import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import kafka.utils.*; @@ -190,8 +183,15 @@ public void stop() { shutdownProducers(); if(kafkaServer != null) { - kafkaServer.shutdown(); - kafkaServer.awaitShutdown(); + try { + kafkaServer.shutdown(); + kafkaServer.awaitShutdown(); + } + catch(Throwable fnf) { + if(!fnf.getMessage().contains("Error writing to highwatermark file")) { + throw fnf; + } + } } if(zkClient != null) { // Delete data in ZK to avoid startup interference. From ca1a9e6708e57d253893550101314fc6d54f4ced Mon Sep 17 00:00:00 2001 From: justinjleet Date: Thu, 29 Jun 2017 16:56:45 -0400 Subject: [PATCH 41/43] Removing extraneous field from earlier testing --- .../apache/metron/integration/components/KafkaComponent.java | 2 -- 1 file changed, 2 deletions(-) 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 d9c1dcc67b..6ec1314da6 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 @@ -50,7 +50,6 @@ public class KafkaComponent implements InMemoryComponent { protected static final Logger LOG = LoggerFactory.getLogger(KafkaComponent.class); - protected static String logDir = ""; public static class Topic { public int numPartitions; @@ -151,7 +150,6 @@ public void start() { Properties props = TestUtilsWrapper.createBrokerConfig(0, zookeeperConnectString, brokerPort); props.setProperty("zookeeper.connection.timeout.ms","1000000"); KafkaConfig config = new KafkaConfig(props); - logDir = config.getString("log.dir"); Time mock = new MockTime(); kafkaServer = TestUtils.createServer(config, mock); From 1621a82fab2b482f5c5535b7a80d4d95a62b53e0 Mon Sep 17 00:00:00 2001 From: merrimanr Date: Thu, 29 Jun 2017 16:28:03 -0500 Subject: [PATCH 42/43] Kafka embedded server only started/stopped in KafkaControllerIntegrationTest --- .../org/apache/metron/rest/config/TestConfig.java | 3 +-- .../controller/KafkaControllerIntegrationTest.java | 14 +++++++++++++- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java index 01e4e03e78..5c2acf7155 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java @@ -69,8 +69,7 @@ public KafkaComponent kafkaWithZKComponent(Properties zkProperties) { public ComponentRunner componentRunner(ZKServerComponent zkServerComponent, KafkaComponent kafkaWithZKComponent) { ComponentRunner runner = new ComponentRunner.Builder() .withComponent("zk", zkServerComponent) - .withComponent("kafka", kafkaWithZKComponent) - .withCustomShutdownOrder(new String[]{"kafka", "zk"}) + .withCustomShutdownOrder(new String[]{"zk"}) .build(); try { runner.start(); diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/KafkaControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/KafkaControllerIntegrationTest.java index 26fa24e997..4db3a3323f 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/KafkaControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/KafkaControllerIntegrationTest.java @@ -19,6 +19,8 @@ import kafka.common.TopicAlreadyMarkedForDeletionException; import org.adrianwalker.multilinestring.Multiline; +import org.apache.metron.integration.ComponentRunner; +import org.apache.metron.integration.UnableToStartException; import org.apache.metron.integration.components.KafkaComponent; import org.apache.metron.rest.generator.SampleDataGenerator; import org.apache.metron.rest.service.KafkaService; @@ -35,7 +37,6 @@ import org.springframework.test.context.junit4.SpringRunner; import org.springframework.test.web.servlet.MockMvc; import org.springframework.test.web.servlet.MvcResult; -import org.springframework.test.web.servlet.ResultActions; import org.springframework.test.web.servlet.setup.MockMvcBuilders; import org.springframework.web.context.WebApplicationContext; import org.springframework.web.util.NestedServletException; @@ -61,6 +62,7 @@ public class KafkaControllerIntegrationTest { private static final int KAFKA_RETRY = 10; @Autowired private KafkaComponent kafkaWithZKComponent; + private ComponentRunner runner; interface Evaluation { @@ -148,6 +150,15 @@ public void stop() { @Before public void setup() throws Exception { + runner = new ComponentRunner.Builder() + .withComponent("kafka", kafkaWithZKComponent) + .withCustomShutdownOrder(new String[]{"kafka"}) + .build(); + try { + runner.start(); + } catch (UnableToStartException e) { + e.printStackTrace(); + } this.mockMvc = MockMvcBuilders.webAppContextSetup(this.wac).apply(springSecurity()).build(); } @@ -257,5 +268,6 @@ public void test() throws Exception { @After public void tearDown() { sampleDataRunner.stop(); + runner.stop(); } } From 232703f88c5524aedda00cd5d9f56925c18a9f6a Mon Sep 17 00:00:00 2001 From: justinjleet Date: Fri, 30 Jun 2017 06:20:28 -0400 Subject: [PATCH 43/43] Couple review comments --- .../rest/controller/KafkaControllerIntegrationTest.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/KafkaControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/KafkaControllerIntegrationTest.java index 4db3a3323f..9e6d4088db 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/KafkaControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/KafkaControllerIntegrationTest.java @@ -60,6 +60,9 @@ public class KafkaControllerIntegrationTest { private static final int KAFKA_RETRY = 10; + // A bug in Spring and/or Kafka forced us to move into a component that is spun up and down per test-case + // Given the large spinup time of components, please avoid this pattern until we upgrade Spring. + // See: https://issues.apache.org/jira/browse/METRON-1009 @Autowired private KafkaComponent kafkaWithZKComponent; private ComponentRunner runner; @@ -113,7 +116,7 @@ public void run() { broSampleDataGenerator.generateSampleData(path); } } catch (ParseException|IOException e) { - e.printStackTrace(); + throw new IllegalStateException("Caught an error generating sample data", e); } }