From 4a329ed241f60057136de00e1d716fb81d32d337 Mon Sep 17 00:00:00 2001 From: rmerriman Date: Mon, 6 Feb 2017 15:19:45 -0600 Subject: [PATCH 01/16] Initial commit --- .../org/apache/metron/common/Constants.java | 53 ++++- .../common/configuration/FieldValidator.java | 2 + .../metron/common/error/MetronError.java | 200 ++++++++++++++++++ .../common/message/BytesFromPosition.java | 36 ++++ .../metron/common/message/JSONFromField.java | 37 ++++ .../common/message/JSONFromPosition.java | 49 +++++ .../common/message/MessageGetStrategy.java | 25 +++ .../metron/common/message/MessageGetters.java | 35 +++ .../metron/common/message/ValueFromField.java | 36 ++++ .../metron/common/utils/ErrorUtils.java | 87 +------- .../metron/common/error/MetronErrorTest.java | 110 ++++++++++ .../src/main/config/elasticsearch.properties | 2 +- .../config/elasticsearch_error.properties | 69 ++++++ .../start_elasticsearch_error_topology.sh | 22 ++ .../ElasticsearchIndexingIntegrationTest.java | 2 +- .../src/main/config/enrichment.properties | 12 +- .../bolt/GenericEnrichmentBolt.java | 16 +- .../metron/enrichment/bolt/JoinBolt.java | 43 ++-- .../bolt/BulkMessageWriterBoltTest.java | 4 + .../metron/enrichment/bolt/JoinBoltTest.java | 2 + .../EnrichmentIntegrationTest.java | 111 +++++++--- .../main/config/zookeeper/indexing/error.json | 17 ++ .../src/main/flux/error/remote.yaml | 122 +++++++++++ .../src/main/flux/indexing/remote.yaml | 5 +- .../integration/IndexingIntegrationTest.java | 5 +- .../metron/integration/ProcessorResult.java | 23 +- .../processors/KafkaMessageSet.java | 7 +- .../processors/KafkaProcessor.java | 16 +- .../metron/parsers/bolt/ParserBolt.java | 49 +++-- .../metron/parsers/bolt/WriterBolt.java | 28 ++- .../metron/parsers/bolt/WriterHandler.java | 8 +- .../topology/ParserTopologyBuilder.java | 39 +--- .../parsers/topology/ParserTopologyCLI.java | 2 - .../integration/ParserIntegrationTest.java | 6 +- .../components/ParserTopologyComponent.java | 9 +- .../WriterBoltIntegrationTest.java | 30 ++- .../SolrIndexingIntegrationTest.java | 2 +- .../metron/writer/BulkWriterComponent.java | 39 ++-- .../writer/bolt/BulkMessageWriterBolt.java | 17 +- .../writer/BulkWriterComponentTest.java | 197 +++++++++++++++++ 40 files changed, 1275 insertions(+), 299 deletions(-) create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/error/MetronError.java create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/message/BytesFromPosition.java create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/message/JSONFromField.java create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/message/JSONFromPosition.java create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetStrategy.java create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetters.java create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/message/ValueFromField.java create mode 100644 metron-platform/metron-common/src/test/java/org/apache/metron/common/error/MetronErrorTest.java create mode 100644 metron-platform/metron-elasticsearch/src/main/config/elasticsearch_error.properties create mode 100755 metron-platform/metron-elasticsearch/src/main/scripts/start_elasticsearch_error_topology.sh create mode 100644 metron-platform/metron-indexing/src/main/config/zookeeper/indexing/error.json create mode 100644 metron-platform/metron-indexing/src/main/flux/error/remote.yaml create mode 100644 metron-platform/metron-writer/src/test/java/org/apache/metron/writer/BulkWriterComponentTest.java diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java index 4230678f36..4ad09c211c 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java @@ -26,15 +26,10 @@ public class Constants { public static final String ZOOKEEPER_TOPOLOGY_ROOT = ZOOKEEPER_ROOT + "/topology"; public static final long DEFAULT_CONFIGURED_BOLT_TIMEOUT = 5000; public static final String SENSOR_TYPE = "source.type"; + public static final String ERROR_TOPIC = "error"; public static final String ENRICHMENT_TOPIC = "enrichments"; - public static final String ENRICHMENT_ERROR_TOPIC = "enrichments_error"; - public static final String THREAT_INTEL_ERROR_TOPIC = "threatintel_error"; public static final String INDEXING_TOPIC = "indexing"; - public static final String INDEXING_ERROR_TOPIC = "indexing_error"; - public static final String DEFAULT_PARSER_ERROR_TOPIC = "parser_error"; - public static final String DEFAULT_PARSER_INVALID_TOPIC = "parser_invalid"; public static final String ERROR_STREAM = "error"; - public static final String INVALID_STREAM = "invalid"; public static final String SIMPLE_HBASE_ENRICHMENT = "hbaseEnrichment"; public static final String SIMPLE_HBASE_THREAT_INTEL = "hbaseThreatIntel"; @@ -72,5 +67,51 @@ public static Fields fromString(String fieldName) { } } + public enum ErrorFields { + MESSAGE("message") + ,FAILED_SENSOR_TYPE("failed_sensor_type") + ,ERROR_TYPE("error_type") + ,EXCEPTION("exception") + ,STACK("stack") + ,TIMESTAMP("timestamp") + ,HOSTNAME("hostname") + ,RAW_MESSAGE("raw_message") + ,RAW_MESSAGE_BYTES("raw_message_bytes") + ,ERROR_FIELDS("error_fields") + ,ERROR_HASH("error_hash") + ; + + private String name; + + ErrorFields(String name) { + this.name = name; + } + + public String getName() { + return name; + } + } + + public enum ErrorType { + + PARSER_ERROR("parser_error") + ,PARSER_INVALID("parser_invalid") + ,ENRICHMENT_ERROR("enrichments_error") + ,THREAT_INTEL_ERROR("threatintel_error") + ,INDEXING_ERROR("indexing_error") + ,DEFAULT_ERROR("error") + ; + + private String type; + + ErrorType(String type) { + this.type = type; + } + + public String getType() { + return type; + } + } + } diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/FieldValidator.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/FieldValidator.java index 3302426ad0..970566c4fa 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/FieldValidator.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/FieldValidator.java @@ -66,6 +66,8 @@ else if(inputObj instanceof List) { for(Object inputO : (List)inputObj) { input.add(inputO.toString()); } + } else { + input = new ArrayList<>(); } config = Config.CONFIG.get(validatorConfig, Map.class); if(config == null) { diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/error/MetronError.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/error/MetronError.java new file mode 100644 index 0000000000..65026781a2 --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/error/MetronError.java @@ -0,0 +1,200 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.metron.common.error; + +import org.apache.commons.codec.digest.DigestUtils; +import org.apache.commons.lang.exception.ExceptionUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.metron.common.Constants; +import org.apache.metron.common.Constants.ErrorType; +import org.json.simple.JSONObject; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.metron.common.Constants.ErrorFields; + +public class MetronError { + + private String message; + private Throwable throwable; + private String sensorType = "error"; + private ErrorType errorType = ErrorType.DEFAULT_ERROR; + private Set errorFields; + private List rawMessages; + + public MetronError withMessage(String message) { + this.message = message; + return this; + } + + public MetronError withThrowable(Throwable throwable) { + this.throwable = throwable; + return this; + } + + public MetronError withSensorType(String sensorType) { + this.sensorType = sensorType; + return this; + } + + public MetronError withErrorType(ErrorType errorType) { + this.errorType = errorType; + return this; + } + + public MetronError withErrorFields(Set errorFields) { + this.errorFields = errorFields; + return this; + } + + + public MetronError addRawMessage(Object rawMessage) { + if (rawMessage != null) { + if (this.rawMessages == null) { + this.rawMessages = new ArrayList<>(); + } + this.rawMessages.add(rawMessage); + } + return this; + } + + public MetronError withRawMessages(List rawMessages) { + this.rawMessages = rawMessages; + return this; + } + + public Optional getThrowable() { + return throwable != null ? Optional.of(throwable) : Optional.empty(); + } + + @SuppressWarnings({"unchecked"}) + public JSONObject getJSONObject() { + JSONObject errorMessage = new JSONObject(); + errorMessage.put(Constants.SENSOR_TYPE, "error"); + + /* + * Save full stack trace in object. + */ + if (throwable != null) { + String stackTrace = ExceptionUtils.getStackTrace(throwable); + String exception = throwable.toString(); + errorMessage.put(ErrorFields.EXCEPTION.getName(), exception); + errorMessage.put(ErrorFields.STACK.getName(), stackTrace); + } + + errorMessage.put(ErrorFields.TIMESTAMP.getName(), System.currentTimeMillis()); + try { + errorMessage.put(ErrorFields.HOSTNAME.getName(), InetAddress.getLocalHost().getHostName()); + } catch (UnknownHostException ex) { + + } + if(rawMessages != null) { + for(int i = 0; i < rawMessages.size(); i++) { + Object rawMessage = rawMessages.get(i); + // If multiple messages are included add an index to the field name, otherwise leave it off + String rawMessageField = rawMessages.size() == 1 ? ErrorFields.RAW_MESSAGE.getName() : ErrorFields.RAW_MESSAGE.getName() + "_" + i; + String rawMessageBytesField = rawMessages.size() == 1 ? ErrorFields.RAW_MESSAGE_BYTES.getName() : ErrorFields.RAW_MESSAGE_BYTES.getName() + "_" + i; + if(rawMessage instanceof byte[]) { + errorMessage.put(rawMessageField, Bytes.toString((byte[])rawMessage)); + errorMessage.put(rawMessageBytesField, toByteArrayList((byte[])rawMessage)); + } else { + errorMessage.put(rawMessageField, rawMessage); + } + } + } + + if (rawMessages != null && rawMessages.size() == 1) { + Object rawMessage = rawMessages.get(0); + if (rawMessage instanceof JSONObject) { + JSONObject rawJSON = (JSONObject) rawMessage; + if (errorFields != null) { + String errorFieldString = String.join(",", errorFields); + errorMessage.put(ErrorFields.ERROR_FIELDS.getName(), errorFieldString); + List hashElements = errorFields.stream().map(errorField -> + String.format("%s-%s", errorField, rawJSON.get(errorField))).collect(Collectors.toList()); + errorMessage.put(ErrorFields.ERROR_HASH.getName(), DigestUtils.sha256Hex(String.join("|", hashElements).getBytes(UTF_8))); + } else { + errorMessage.put(ErrorFields.ERROR_HASH.getName(), DigestUtils.sha256Hex(rawJSON.toJSONString().getBytes(UTF_8))); + } + } else if (rawMessage instanceof byte[]) { + errorMessage.put(ErrorFields.ERROR_HASH.getName(), DigestUtils.sha256Hex((byte[])rawMessage)); + } else { + errorMessage.put(ErrorFields.ERROR_HASH.getName(), DigestUtils.sha256Hex(rawMessage.toString().getBytes(UTF_8))); + } + } + + if (message != null) { + errorMessage.put(ErrorFields.MESSAGE.getName(), message); + } else if (throwable != null) { + errorMessage.put(ErrorFields.MESSAGE.getName(), throwable.getMessage()); + } + + errorMessage.put(ErrorFields.FAILED_SENSOR_TYPE.getName(), sensorType); + errorMessage.put(ErrorFields.ERROR_TYPE.getName(), errorType.getType()); + + return errorMessage; + } + + protected List toByteArrayList(byte[] list) { + List ret = new ArrayList<>(); + for(byte b : list) { + ret.add(b); + } + return ret; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + MetronError that = (MetronError) o; + + if (message != null ? !message.equals(that.message) : that.message != null) + return false; + if (throwable != null ? !throwable.equals(that.throwable) : that.throwable != null) + return false; + if (sensorType != null ? !sensorType.equals(that.sensorType) : that.sensorType != null) + return false; + if (errorType != null ? !errorType.equals(that.errorType) : that.errorType != null) + return false; + if (errorFields != null ? !errorFields.equals(that.errorFields) : that.errorFields != null) + return false; + return rawMessages != null ? rawMessages.equals(that.rawMessages) : that.rawMessages == null; + + } + + @Override + public int hashCode() { + int result = message != null ? message.hashCode() : 0; + result = 31 * result + (throwable != null ? throwable.hashCode() : 0); + result = 31 * result + (sensorType != null ? sensorType.hashCode() : 0); + result = 31 * result + (errorType != null ? errorType.hashCode() : 0); + result = 31 * result + (errorFields != null ? errorFields.hashCode() : 0); + result = 31 * result + (rawMessages != null ? rawMessages.hashCode() : 0); + return result; + } + +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/BytesFromPosition.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/BytesFromPosition.java new file mode 100644 index 0000000000..b73228f247 --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/BytesFromPosition.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.metron.common.message; + +import org.apache.storm.tuple.Tuple; + +public class BytesFromPosition implements MessageGetStrategy { + + private int position = 0; + + public BytesFromPosition() {}; + + public BytesFromPosition(int position) { + this.position = position; + } + + @Override + public byte[] get(Tuple tuple) { + return tuple.getBinary(position); + } +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/JSONFromField.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/JSONFromField.java new file mode 100644 index 0000000000..39fe9dd6af --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/JSONFromField.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.metron.common.message; + +import org.apache.storm.tuple.Tuple; +import org.json.simple.JSONObject; + +public class JSONFromField implements MessageGetStrategy { + + private String fieldValue = "message"; + + public JSONFromField() {}; + + public JSONFromField(String fieldValue) { + this.fieldValue = fieldValue; + } + + @Override + public JSONObject get(Tuple tuple) { + return (JSONObject) ((JSONObject) tuple.getValueByField(fieldValue)).clone(); + } +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/JSONFromPosition.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/JSONFromPosition.java new file mode 100644 index 0000000000..4407d4f709 --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/JSONFromPosition.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.metron.common.message; + +import org.apache.storm.tuple.Tuple; +import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; + +public class JSONFromPosition implements MessageGetStrategy { + + private int position = 0; + + private ThreadLocal parser = new ThreadLocal() { + @Override + protected JSONParser initialValue() { + return new JSONParser(); + } + }; + + public JSONFromPosition() {}; + + public JSONFromPosition(int position) { + this.position = position; + } + + @Override + public JSONObject get(Tuple tuple) { + try { + return (JSONObject) parser.get().parse(new String(tuple.getBinary(position), "UTF8")); + } catch (Exception e) { + throw new IllegalStateException(e.getMessage(), e); + } + } +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetStrategy.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetStrategy.java new file mode 100644 index 0000000000..0595ce1bfa --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetStrategy.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.metron.common.message; + +import org.apache.storm.tuple.Tuple; + +public interface MessageGetStrategy { + + Object get(Tuple tuple); +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetters.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetters.java new file mode 100644 index 0000000000..8e57952ebb --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetters.java @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.metron.common.message; + +public enum MessageGetters { + + BYTES_FROM_POSITION(new BytesFromPosition()), + JSON_FROM_POSITION(new JSONFromPosition()), + JSON_FROM_FIELD(new JSONFromField()); + + MessageGetStrategy messageGetStrategy; + + MessageGetters(MessageGetStrategy messageGetStrategy) { + this.messageGetStrategy = messageGetStrategy; + } + + public MessageGetStrategy get() { + return messageGetStrategy; + } +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/ValueFromField.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/ValueFromField.java new file mode 100644 index 0000000000..d5a5799b61 --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/ValueFromField.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.metron.common.message; + +import org.apache.storm.tuple.Tuple; + +public class ValueFromField implements MessageGetStrategy { + + private String fieldValue = "message"; + + public ValueFromField() {}; + + public ValueFromField(String fieldValue) { + this.fieldValue = fieldValue; + } + + @Override + public Object get(Tuple tuple) { + return tuple.getValueByField(fieldValue); + } +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/ErrorUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/ErrorUtils.java index 3b3f42683f..f4e3a8d01e 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/ErrorUtils.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/ErrorUtils.java @@ -17,30 +17,17 @@ */ package org.apache.metron.common.utils; -import java.net.InetAddress; -import java.net.UnknownHostException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Optional; - -import org.apache.storm.task.OutputCollector; -import org.apache.storm.tuple.Values; -import org.apache.commons.beanutils.Converter; -import org.apache.commons.lang.exception.ExceptionUtils; -import org.apache.commons.lang3.ArrayUtils; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.commons.lang3.tuple.Pair; import org.apache.metron.common.Constants; -import org.json.simple.JSONObject; +import org.apache.metron.common.error.MetronError; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.tuple.Values; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.lang.management.ManagementFactory; import java.lang.management.ThreadInfo; import java.lang.management.ThreadMXBean; -import java.net.InetAddress; -import java.net.UnknownHostException; import java.util.Optional; import java.util.function.Function; @@ -102,74 +89,16 @@ private static String formatReason(String reason, Optional t) { } } - @SuppressWarnings("unchecked") - public static JSONObject generateErrorMessage(String message, Throwable t) + public static void handleError(OutputCollector collector, MetronError error) { - return generateErrorMessage(message, t, Optional.empty(), Optional.empty()); - } - public static JSONObject generateErrorMessage(String message - , Throwable t - , Optional sensorType - , Optional rawMessage - ) - { - JSONObject error_message = new JSONObject(); - - /* - * Save full stack trace in object. - */ - String stackTrace = ExceptionUtils.getStackTrace(t); - - String exception = t.toString(); - - - error_message.put("time", System.currentTimeMillis()); - try { - error_message.put("hostname", InetAddress.getLocalHost().getHostName()); - } catch (UnknownHostException ex) { - + collector.emit(Constants.ERROR_STREAM, new Values(error.getJSONObject())); + Optional throwable = error.getThrowable(); + if (throwable.isPresent()) { + collector.reportError(throwable.get()); } - if(rawMessage.isPresent()) { - if(rawMessage.get() instanceof byte[]) { - error_message.put("rawMessage", Bytes.toString((byte[])rawMessage.get())); - error_message.put("rawMessage_bytes", toByteArrayList((byte[])rawMessage.get())); - } - else { - error_message.put("rawMessage", rawMessage.get()); - } - } - error_message.put("message", message); - error_message.put(Constants.SENSOR_TYPE, StringUtils.join("_", sensorType, Optional.of("error"))); - error_message.put("exception", exception); - error_message.put("stack", stackTrace); - return error_message; - } - - private static List toByteArrayList(byte[] list) { - List ret = new ArrayList<>(); - for(byte b : list) { - ret.add(b); - } - return ret; } - public static void handleError(OutputCollector collector, Throwable t, String errorStream) { - handleError(collector, t, errorStream, Optional.empty(), Optional.empty()); - } - public static void handleError(OutputCollector collector - , Throwable t - , String errorStream - , Optional sensorType - , Optional rawMessage - ) - { - JSONObject error = ErrorUtils.generateErrorMessage(t.getMessage(), t, sensorType, rawMessage); - collector.emit(errorStream, new Values(error)); - collector.reportError(t); - } - - public static String generateThreadDump() { final StringBuilder dump = new StringBuilder(); final ThreadMXBean threadMXBean = ManagementFactory.getThreadMXBean(); diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/error/MetronErrorTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/error/MetronErrorTest.java new file mode 100644 index 0000000000..226c6289b5 --- /dev/null +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/error/MetronErrorTest.java @@ -0,0 +1,110 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.metron.common.error; + +import com.google.common.collect.Sets; +import org.apache.metron.common.Constants; +import org.json.simple.JSONObject; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; + +import static org.apache.metron.common.Constants.ErrorFields.RAW_MESSAGE; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class MetronErrorTest { + + private JSONObject message1 = new JSONObject(); + private JSONObject message2 = new JSONObject(); + + @Before + public void setup() { + message1.put("value", "message1"); + message2.put("value", "message2"); + } + + @Test + public void getJSONObjectShouldReturnBasicInformation() { + MetronError error = new MetronError() + .withMessage("test message") + .withErrorType(Constants.ErrorType.PARSER_ERROR) + .withSensorType("sensorType"); + + JSONObject errorJSON = error.getJSONObject(); + assertEquals("test message", errorJSON.get(Constants.ErrorFields.MESSAGE.getName())); + assertEquals(Constants.ErrorType.PARSER_ERROR.getType(), errorJSON.get(Constants.ErrorFields.ERROR_TYPE.getName())); + assertEquals("error", errorJSON.get(Constants.SENSOR_TYPE)); + assertEquals("sensorType", errorJSON.get(Constants.ErrorFields.FAILED_SENSOR_TYPE.getName())); + assertTrue(((String) errorJSON.get(Constants.ErrorFields.HOSTNAME.getName())).length() > 0); + assertTrue(((long) errorJSON.get(Constants.ErrorFields.TIMESTAMP.getName())) > 0); + } + + @Test + public void getJSONObjectShouldHandleThrowable() { + Throwable e = new Exception("test exception"); + MetronError error = new MetronError().withThrowable(e); + + JSONObject errorJSON = error.getJSONObject(); + assertEquals("java.lang.Exception: test exception", errorJSON.get(Constants.ErrorFields.EXCEPTION.getName())); + assertTrue(((String) errorJSON.get(Constants.ErrorFields.STACK.getName())).startsWith("java.lang.Exception: test exception")); + assertEquals(e.getMessage(), errorJSON.get(Constants.ErrorFields.MESSAGE.getName())); + } + + @Test + public void getJSONObjectShouldIncludeRawMessages() { + JSONObject message1 = new JSONObject(); + JSONObject message2 = new JSONObject(); + message1.put("value", "message1"); + message2.put("value", "message2"); + MetronError error = new MetronError().withRawMessages(Arrays.asList(message1, message2)); + + JSONObject errorJSON = error.getJSONObject(); + JSONObject expected1 = new JSONObject(); + JSONObject expected2 = new JSONObject(); + expected1.put("value", "message1"); + expected2.put("value", "message2"); + assertEquals(expected1, errorJSON.get(Constants.ErrorFields.RAW_MESSAGE.getName() + "_0")); + assertEquals(expected2, errorJSON.get(Constants.ErrorFields.RAW_MESSAGE.getName() + "_1")); + + error = new MetronError().addRawMessage("raw message".getBytes()); + errorJSON = error.getJSONObject(); + assertEquals("raw message", errorJSON.get(Constants.ErrorFields.RAW_MESSAGE.getName())); + assertEquals(error.toByteArrayList("raw message".getBytes()), errorJSON.get(Constants.ErrorFields.RAW_MESSAGE_BYTES.getName())); + assertEquals("3b02cb29676bc448c69da1ec5eef7c89f4d6dc6a5a7ce0296ea25b207eea36be", errorJSON.get(Constants.ErrorFields.ERROR_HASH.getName())); + + error = new MetronError().addRawMessage(message1); + errorJSON = error.getJSONObject(); + assertEquals(expected1, errorJSON.get(Constants.ErrorFields.RAW_MESSAGE.getName())); + assertEquals("e8aaf87c8494d345aac2d612ffd94fcf0b98c975fe6c4b991e2f8280a3a0bd10", errorJSON.get(Constants.ErrorFields.ERROR_HASH.getName())); + } + + @Test + public void getJSONObjectShouldIncludeErrorFields() { + JSONObject message = new JSONObject(); + message.put("field1", "value1"); + message.put("field2", "value2"); + + MetronError error = new MetronError().addRawMessage(message).withErrorFields(Sets.newHashSet("field1", "field2")); + + JSONObject errorJSON = error.getJSONObject(); + assertEquals(Sets.newHashSet("field1", "field2"), Sets.newHashSet(((String) errorJSON.get(Constants.ErrorFields.ERROR_FIELDS.getName())).split(","))); + assertEquals("04a2629c39e098c3944be85f35c75876598f2b44b8e5e3f52c59fa1ac182817c", errorJSON.get(Constants.ErrorFields.ERROR_HASH.getName())); + } +} diff --git a/metron-platform/metron-elasticsearch/src/main/config/elasticsearch.properties b/metron-platform/metron-elasticsearch/src/main/config/elasticsearch.properties index c2c10af059..7cfc1d85e4 100644 --- a/metron-platform/metron-elasticsearch/src/main/config/elasticsearch.properties +++ b/metron-platform/metron-elasticsearch/src/main/config/elasticsearch.properties @@ -26,7 +26,7 @@ kafka.start=WHERE_I_LEFT_OFF ##### Indexing ##### index.input.topic=indexing -index.error.topic=indexing_error +index.error.topic=error writer.class.name=org.apache.metron.elasticsearch.writer.ElasticsearchWriter ##### ElasticSearch ##### diff --git a/metron-platform/metron-elasticsearch/src/main/config/elasticsearch_error.properties b/metron-platform/metron-elasticsearch/src/main/config/elasticsearch_error.properties new file mode 100644 index 0000000000..e96b2a7fd6 --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/config/elasticsearch_error.properties @@ -0,0 +1,69 @@ +# 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. + +##### Storm ##### +indexing.workers=1 +indexing.executors=0 + +##### Kafka ##### + +kafka.zk=node1:2181 +kafka.broker=node1:6667 +kafka.start=WHERE_I_LEFT_OFF + +##### Indexing ##### +index.input.topic=error +index.error.topic=error +writer.class.name=org.apache.metron.elasticsearch.writer.ElasticsearchWriter + +##### ElasticSearch ##### + +es.ip=10.22.0.214 +es.port=9300 +es.clustername=elasticsearch + +##### Metrics ##### + +#reporters +org.apache.metron.metrics.reporter.graphite=true +org.apache.metron.metrics.reporter.console=false +org.apache.metron.metrics.reporter.jmx=false + +#Graphite Addresses + +org.apache.metron.metrics.graphite.address=localhost +org.apache.metron.metrics.graphite.port=2023 + +#TelemetryParserBolt +org.apache.metron.metrics.TelemetryParserBolt.acks=true +org.apache.metron.metrics.TelemetryParserBolt.emits=true +org.apache.metron.metrics.TelemetryParserBolt.fails=true + + +##### HDFS ##### + +bolt.hdfs.batch.size=5000 +bolt.hdfs.field.delimiter=| +bolt.hdfs.rotation.policy=org.apache.storm.hdfs.bolt.rotation.TimedRotationPolicy +bolt.hdfs.rotation.policy.units=DAYS +bolt.hdfs.rotation.policy.count=1 + +bolt.hdfs.file.rotation.size.in.mb=5 +bolt.hdfs.file.system.url=hdfs://iot01.cloud.hortonworks.com:8020 +bolt.hdfs.wip.file.path=/paloalto/wip +bolt.hdfs.finished.file.path=/paloalto/rotated +bolt.hdfs.compression.codec.class=org.apache.hadoop.io.compress.SnappyCodec +index.hdfs.output=/tmp/metron/error diff --git a/metron-platform/metron-elasticsearch/src/main/scripts/start_elasticsearch_error_topology.sh b/metron-platform/metron-elasticsearch/src/main/scripts/start_elasticsearch_error_topology.sh new file mode 100755 index 0000000000..5cb63912b8 --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/scripts/start_elasticsearch_error_topology.sh @@ -0,0 +1,22 @@ +#!/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. +# +METRON_VERSION=${project.version} +METRON_HOME=/usr/metron/$METRON_VERSION +TOPOLOGY_JAR=${project.artifactId}-$METRON_VERSION-uber.jar +storm jar $METRON_HOME/lib/$TOPOLOGY_JAR org.apache.storm.flux.Flux --remote $METRON_HOME/flux/error/remote.yaml --filter $METRON_HOME/config/elasticsearch_error.properties diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchIndexingIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchIndexingIntegrationTest.java index acc1565e5c..9894d7a08e 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchIndexingIntegrationTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchIndexingIntegrationTest.java @@ -76,7 +76,7 @@ public ReadinessState process(ComponentRunner runner) { throw new IllegalStateException("Unable to retrieve indexed documents.", e); } if (docs.size() < inputMessages.size() || docs.size() != docsFromDisk.size()) { - errors = kafkaComponent.readMessages(Constants.INDEXING_ERROR_TOPIC); + errors = kafkaComponent.readMessages(Constants.ERROR_TOPIC); if(errors.size() > 0){ return ReadinessState.READY; } diff --git a/metron-platform/metron-enrichment/src/main/config/enrichment.properties b/metron-platform/metron-enrichment/src/main/config/enrichment.properties index 84d8461ee4..7091a008da 100644 --- a/metron-platform/metron-enrichment/src/main/config/enrichment.properties +++ b/metron-platform/metron-enrichment/src/main/config/enrichment.properties @@ -20,8 +20,8 @@ kafka.zk=node1:2181 kafka.broker=node1:6667 enrichment.output.topic=indexing -enrichment.error.topic=enrichments_error -threat.intel.error.topic=threatintel_error +enrichment.error.topic=error +threat.intel.error.topic=error ##### Metrics ##### @@ -71,7 +71,11 @@ bolt.hbase.partitioner.region.info.refresh.interval.mins=60 ##### Threat Intel ##### -threat.intel.tracker.table= -threat.intel.tracker.cf= +threat.intel.tracker.table=access_tracker +threat.intel.tracker.cf=t threat.intel.ip.table= threat.intel.ip.cf= +threat.intel.simple.hbase.table=threatintel +threat.intel.simple.hbase.cf=t +enrichment.simple.hbase.table=enrichment +enrichment.simple.hbase.cf=t diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java index f3ed5b4aa7..a6a9acd9c2 100644 --- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java +++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBolt.java @@ -18,6 +18,7 @@ package org.apache.metron.enrichment.bolt; +import org.apache.metron.common.error.MetronError; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; import org.apache.storm.topology.OutputFieldsDeclarer; @@ -41,7 +42,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashSet; import java.util.Map; +import java.util.Optional; import java.util.concurrent.TimeUnit; /** @@ -224,6 +227,12 @@ public void execute(Tuple tuple) { catch(Exception e) { LOG.error(e.getMessage(), e); error = true; + MetronError metronError = new MetronError() + .withErrorType(Constants.ErrorType.ENRICHMENT_ERROR) + .withThrowable(e) + .withErrorFields(new HashSet() {{ add(field); }}) + .addRawMessage(rawMessage); + ErrorUtils.handleError(collector, metronError); continue; } } @@ -256,11 +265,14 @@ public void execute(Tuple tuple) { // errors, so this is made available in order to ensure ERROR_STREAM is output properly. protected void handleError(String key, JSONObject rawMessage, String subGroup, JSONObject enrichedMessage, Exception e) { LOG.error("[Metron] Unable to enrich message: " + rawMessage, e); - JSONObject error = ErrorUtils.generateErrorMessage("Enrichment problem: " + rawMessage, e); if (key != null) { collector.emit(enrichmentType, new Values(key, enrichedMessage, subGroup)); } - collector.emit(ERROR_STREAM, new Values(error)); + MetronError error = new MetronError() + .withErrorType(Constants.ErrorType.ENRICHMENT_ERROR) + .withThrowable(e) + .addRawMessage(rawMessage); + ErrorUtils.handleError(collector, error); } @Override diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java index 101d056a9b..2e987e376b 100644 --- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java +++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java @@ -17,27 +17,29 @@ */ package org.apache.metron.enrichment.bolt; -import org.apache.storm.task.OutputCollector; -import org.apache.storm.task.TopologyContext; -import org.apache.storm.topology.OutputFieldsDeclarer; -import org.apache.storm.tuple.Fields; -import org.apache.storm.tuple.Tuple; -import org.apache.storm.tuple.Values; import com.google.common.base.Joiner; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.common.collect.Sets; -import org.apache.metron.common.bolt.ConfiguredBolt; -import org.apache.metron.common.utils.ErrorUtils; -import org.json.simple.JSONObject; +import org.apache.metron.common.Constants; import org.apache.metron.common.bolt.ConfiguredEnrichmentBolt; +import org.apache.metron.common.error.MetronError; +import org.apache.metron.common.message.MessageGetStrategy; +import org.apache.metron.common.message.ValueFromField; +import org.apache.metron.common.utils.ErrorUtils; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import java.util.HashMap; import java.util.Map; import java.util.Set; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; public abstract class JoinBolt extends ConfiguredEnrichmentBolt { @@ -48,6 +50,9 @@ public abstract class JoinBolt extends ConfiguredEnrichmentBolt { protected transient CacheLoader> loader; protected transient LoadingCache> cache; + private transient MessageGetStrategy keyGetStrategy; + private transient MessageGetStrategy subgroupGetStrategy; + private transient MessageGetStrategy messageGetStrategy; protected Long maxCacheSize; protected Long maxTimeRetain; @@ -68,6 +73,9 @@ public JoinBolt withMaxTimeRetain(long maxTimeRetain) { @Override public void prepare(Map map, TopologyContext topologyContext, OutputCollector outputCollector) { super.prepare(map, topologyContext, outputCollector); + keyGetStrategy = new ValueFromField("key"); + subgroupGetStrategy = new ValueFromField("subgroup"); + messageGetStrategy = new ValueFromField("message"); this.collector = outputCollector; if (this.maxCacheSize == null) { throw new IllegalStateException("maxCacheSize must be specified"); @@ -91,10 +99,10 @@ public Map load(String key) throws Exception { @Override public void execute(Tuple tuple) { String streamId = tuple.getSourceStreamId(); - String key = (String) tuple.getValueByField("key"); - String subgroup = (String) tuple.getValueByField("subgroup"); + String key = (String) keyGetStrategy.get(tuple); + String subgroup = (String) subgroupGetStrategy.get(tuple); streamId = Joiner.on(":").join("" + streamId, subgroup == null?"":subgroup); - V message = (V) tuple.getValueByField("message"); + V message = (V) messageGetStrategy.get(tuple); try { Map streamMessageMap = cache.get(key); if (streamMessageMap.containsKey(streamId)) { @@ -127,10 +135,13 @@ public void execute(Tuple tuple) { } } catch (Exception e) { LOG.error("[Metron] Unable to join messages: " + message, e); - JSONObject error = ErrorUtils.generateErrorMessage("Joining problem: " + message, e); + MetronError error = new MetronError() + .withErrorType(Constants.ErrorType.ENRICHMENT_ERROR) + .withMessage("Joining problem: " + message) + .withThrowable(e) + .addRawMessage(message); + ErrorUtils.handleError(collector, error); collector.ack(tuple); - collector.emit("error", new Values(error)); - collector.reportError(e); } } diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java index c5c1294f47..2016ef3934 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java @@ -18,6 +18,7 @@ package org.apache.metron.enrichment.bolt; import org.apache.log4j.Level; +import org.apache.metron.common.message.MessageGetStrategy; import org.apache.metron.common.writer.BulkWriterResponse; import org.apache.metron.test.utils.UnitTestHelper; import org.apache.metron.writer.BulkWriterComponent; @@ -112,6 +113,9 @@ public void parseMessages() throws ParseException { @Mock private BulkMessageWriter bulkMessageWriter; + @Mock + private MessageGetStrategy messageGetStrategy; + @Test public void test() throws Exception { BulkMessageWriterBolt bulkMessageWriterBolt = new BulkMessageWriterBolt("zookeeperUrl").withBulkMessageWriter(bulkMessageWriter); diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/JoinBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/JoinBoltTest.java index 5b06d336fa..1e47d542c3 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/JoinBoltTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/JoinBoltTest.java @@ -17,6 +17,7 @@ */ package org.apache.metron.enrichment.bolt; +import org.apache.metron.common.message.MessageGetStrategy; import org.apache.storm.task.TopologyContext; import org.apache.storm.tuple.Values; import org.adrianwalker.multilinestring.Multiline; @@ -36,6 +37,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.argThat; import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java index 27c1d11516..db301e3325 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java @@ -26,7 +26,6 @@ import org.apache.metron.common.Constants; import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase; -import org.apache.metron.enrichment.bolt.ErrorEnrichmentBolt; import org.apache.metron.enrichment.converter.EnrichmentHelper; import org.apache.metron.enrichment.converter.EnrichmentKey; import org.apache.metron.enrichment.converter.EnrichmentValue; @@ -119,13 +118,13 @@ public void test() throws Exception { setProperty("enrichment.simple.hbase.table", enrichmentsTableName); setProperty("enrichment.simple.hbase.cf", cf); setProperty("enrichment.output.topic", Constants.INDEXING_TOPIC); - setProperty("enrichment.error.topic", Constants.ENRICHMENT_ERROR_TOPIC); + setProperty("enrichment.error.topic", Constants.ERROR_TOPIC); }}; final ZKServerComponent zkServerComponent = getZKServerComponent(topologyProperties); final KafkaComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList() {{ add(new KafkaComponent.Topic(Constants.ENRICHMENT_TOPIC, 1)); add(new KafkaComponent.Topic(Constants.INDEXING_TOPIC, 1)); - add(new KafkaComponent.Topic(Constants.ENRICHMENT_ERROR_TOPIC, 1)); + add(new KafkaComponent.Topic(Constants.ERROR_TOPIC, 1)); }}); String globalConfigStr = null; { @@ -181,15 +180,14 @@ public void test() throws Exception { fluxComponent.submitTopology(); kafkaComponent.writeMessages(Constants.ENRICHMENT_TOPIC, inputMessages); - ProcessorResult>> result = runner.process(getProcessor()); - // We expect failures, so we don't care if result returned failure or not - List> docs = result.getResult(); + ProcessorResult>>> result = runner.process(getProcessor()); + Map>> outputMessages = result.getResult(); + List> docs = outputMessages.get(Constants.INDEXING_TOPIC); Assert.assertEquals(inputMessages.size(), docs.size()); validateAll(docs); - - List errors = result.getProcessErrors(); + List> errors = outputMessages.get(Constants.ERROR_TOPIC); Assert.assertEquals(inputMessages.size(), errors.size()); - validateErrors(result.getProcessErrors()); + validateErrors(errors); } finally { runner.stop(); } @@ -214,10 +212,12 @@ public static void validateAll(List> docs) { } } - protected void validateErrors(List errors) { - for(byte[] error : errors) { - // Don't reconstruct the entire message, just ensure it contains the known error message inside. - Assert.assertTrue(new String(error).contains(ErrorEnrichmentBolt.TEST_ERROR_MESSAGE)); + protected void validateErrors(List> errors) { + for(Map error : errors) { + Assert.assertEquals("Test throwing error from ErrorEnrichmentBolt", error.get(Constants.ErrorFields.MESSAGE.getName())); + Assert.assertEquals("java.lang.IllegalStateException: Test throwing error from ErrorEnrichmentBolt", error.get(Constants.ErrorFields.EXCEPTION.getName())); + Assert.assertEquals(Constants.ErrorType.ENRICHMENT_ERROR.getType(), error.get(Constants.ErrorFields.ERROR_TYPE.getName())); + Assert.assertEquals(new HashMap() {{ put("rawMessage", "Error Test Raw Message String" ); }}, error.get(Constants.ErrorFields.RAW_MESSAGE.getName())); } } @@ -466,39 +466,82 @@ private static Set setOf(String... items) { return ret; } +// @SuppressWarnings("unchecked") +// private Processor>> getProcessor() { +// +// KafkaProcessor>> kafkaProcessor = new KafkaProcessor<>().withKafkaComponentName("kafka") +// .withReadTopic(Constants.INDEXING_TOPIC) +// .withErrorTopic(Constants.ERROR_TOPIC) +// .withValidateReadMessages(new Function() { +// @Nullable +// @Override +// public Boolean apply(@Nullable KafkaMessageSet messageSet) { +// // this test is written to return 10 errors and 10 messages +// // we can just check when the messages match here +// // if they do then we are good +// return messageSet.getMessages().size() == inputMessages.size(); +// } +// }) +// .withProvideResult(new Function>>() { +// @Nullable +// @Override +// public List> apply(@Nullable KafkaMessageSet messageSet) { +// List> docs = new ArrayList<>(); +// for (byte[] message : messageSet.getMessages()) { +// try { +// docs.add(JSONUtils.INSTANCE.load(new String(message), new TypeReference>() { +// })); +// } catch (IOException e) { +// throw new IllegalStateException(e.getMessage(), e); +// } +// } +// return docs; +// } +// }); +// return kafkaProcessor; +// } + + private static List> loadMessages(List outputMessages) { + List> tmp = new ArrayList<>(); + Iterables.addAll(tmp + , Iterables.transform(outputMessages + , message -> { + try { + return new HashMap<>(JSONUtils.INSTANCE.load(new String(message) + , new TypeReference>() {} + ) + ); + } catch (Exception ex) { + throw new IllegalStateException(ex); + } + } + ) + ); + return tmp; + } @SuppressWarnings("unchecked") - private Processor>> getProcessor() { + private KafkaProcessor>>> getProcessor(){ - KafkaProcessor>> kafkaProcessor = new KafkaProcessor<>().withKafkaComponentName("kafka") + return new KafkaProcessor<>() + .withKafkaComponentName("kafka") .withReadTopic(Constants.INDEXING_TOPIC) - .withErrorTopic(Constants.ENRICHMENT_ERROR_TOPIC) - .withInvalidTopic(Constants.INVALID_STREAM) + .withErrorTopic(Constants.ERROR_TOPIC) .withValidateReadMessages(new Function() { @Nullable @Override public Boolean apply(@Nullable KafkaMessageSet messageSet) { - // this test is written to return 10 errors and 10 messages - // we can just check when the messages match here - // if they do then we are good - return messageSet.getMessages().size() == inputMessages.size(); + return (messageSet.getMessages().size() == inputMessages.size()) && (messageSet.getErrors().size() == inputMessages.size()); } }) - .withProvideResult(new Function>>() { + .withProvideResult(new Function>>>(){ @Nullable @Override - public List> apply(@Nullable KafkaMessageSet messageSet) { - List> docs = new ArrayList<>(); - for (byte[] message : messageSet.getMessages()) { - try { - docs.add(JSONUtils.INSTANCE.load(new String(message), new TypeReference>() { - })); - } catch (IOException e) { - throw new IllegalStateException(e.getMessage(), e); - } - } - return docs; + public Map>> apply(@Nullable KafkaMessageSet messageSet) { + return new HashMap>>() {{ + put(Constants.INDEXING_TOPIC, loadMessages(messageSet.getMessages())); + put(Constants.ERROR_TOPIC, loadMessages(messageSet.getErrors())); + }}; } }); - return kafkaProcessor; } } diff --git a/metron-platform/metron-indexing/src/main/config/zookeeper/indexing/error.json b/metron-platform/metron-indexing/src/main/config/zookeeper/indexing/error.json new file mode 100644 index 0000000000..4c9d786223 --- /dev/null +++ b/metron-platform/metron-indexing/src/main/config/zookeeper/indexing/error.json @@ -0,0 +1,17 @@ +{ + "hdfs" : { + "index": "error", + "batchSize": 5, + "enabled" : true + }, + "elasticsearch" : { + "index": "error", + "batchSize": 5, + "enabled" : true + }, + "solr" : { + "index": "error", + "batchSize": 5, + "enabled" : true + } +} diff --git a/metron-platform/metron-indexing/src/main/flux/error/remote.yaml b/metron-platform/metron-indexing/src/main/flux/error/remote.yaml new file mode 100644 index 0000000000..bc6a3bbcfc --- /dev/null +++ b/metron-platform/metron-indexing/src/main/flux/error/remote.yaml @@ -0,0 +1,122 @@ +# 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. + +name: "error" +config: + topology.workers: ${indexing.workers} + topology.acker.executors: ${indexing.executors} + +components: + + - id: "fileNameFormat" + className: "org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat" + configMethods: + - name: "withPrefix" + args: + - "error-" + - name: "withExtension" + args: + - ".json" + - name: "withPath" + args: + - "${index.hdfs.output}" + + - id: "hdfsRotationPolicy" + className: "${bolt.hdfs.rotation.policy}" + constructorArgs: + - ${bolt.hdfs.rotation.policy.count} + - "${bolt.hdfs.rotation.policy.units}" +#indexing + - id: "hdfsWriter" + className: "org.apache.metron.writer.hdfs.HdfsWriter" + configMethods: + - name: "withFileNameFormat" + args: + - ref: "fileNameFormat" + - name: "withRotationPolicy" + args: + - ref: "hdfsRotationPolicy" + + - id: "indexWriter" + className: "${writer.class.name}" + +#kafka/zookeeper + - id: "zkHosts" + className: "org.apache.storm.kafka.ZkHosts" + constructorArgs: + - "${kafka.zk}" + - id: "kafkaConfig" + className: "org.apache.metron.common.spout.kafka.SpoutConfig" + constructorArgs: + # zookeeper hosts + - ref: "zkHosts" + # topic name + - "${index.input.topic}" + # zk root + - "" + # id + - "error" + configMethods: + - name: "from" + args: + - "${kafka.start}" + +spouts: + - id: "kafkaSpout" + className: "org.apache.storm.kafka.KafkaSpout" + constructorArgs: + - ref: "kafkaConfig" +bolts: + +# Indexing Bolts + - id: "indexingBolt" + className: "org.apache.metron.writer.bolt.BulkMessageWriterBolt" + constructorArgs: + - "${kafka.zk}" + configMethods: + - name: "withBulkMessageWriter" + args: + - ref: "indexWriter" + - name: "withMessageGetter" + args: + - "JSON_FROM_POSITION" + + - id: "hdfsIndexingBolt" + className: "org.apache.metron.writer.bolt.BulkMessageWriterBolt" + constructorArgs: + - "${kafka.zk}" + configMethods: + - name: "withBulkMessageWriter" + args: + - ref: "hdfsWriter" + - name: "withMessageGetter" + args: + - "JSON_FROM_POSITION" + +streams: + + - name: "spout -> indexing" + from: "kafkaSpout" + to: "indexingBolt" + grouping: + type: SHUFFLE + + - name: "spout -> hdfs" + from: "kafkaSpout" + to: "hdfsIndexingBolt" + grouping: + type: SHUFFLE + diff --git a/metron-platform/metron-indexing/src/main/flux/indexing/remote.yaml b/metron-platform/metron-indexing/src/main/flux/indexing/remote.yaml index 8bf8f48a40..987753e3f9 100644 --- a/metron-platform/metron-indexing/src/main/flux/indexing/remote.yaml +++ b/metron-platform/metron-indexing/src/main/flux/indexing/remote.yaml @@ -101,7 +101,8 @@ bolts: - ref: "indexWriter" - name: "withMessageGetter" args: - - "RAW" + - "JSON_FROM_POSITION" + - id: "hdfsIndexingBolt" className: "org.apache.metron.writer.bolt.BulkMessageWriterBolt" constructorArgs: @@ -112,7 +113,7 @@ bolts: - ref: "hdfsWriter" - name: "withMessageGetter" args: - - "RAW" + - "JSON_FROM_POSITION" - id: "indexingErrorBolt" className: "org.apache.metron.writer.bolt.BulkMessageWriterBolt" 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 a93c442ab0..d790fd9d5d 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 @@ -44,7 +44,6 @@ import org.junit.Test; import javax.annotation.Nullable; -import java.io.ByteArrayInputStream; import java.io.File; import java.io.IOException; import java.util.*; @@ -125,7 +124,7 @@ public void test() throws Exception { setProperty("indexing.workers", "1"); setProperty("indexing.executors", "0"); setProperty("index.input.topic", Constants.INDEXING_TOPIC); - setProperty("index.error.topic", Constants.INDEXING_ERROR_TOPIC); + setProperty("index.error.topic", Constants.ERROR_TOPIC); setProperty("index.date.format", dateFormat); //HDFS settings @@ -138,7 +137,7 @@ public void test() throws Exception { final ZKServerComponent zkServerComponent = getZKServerComponent(topologyProperties); final KafkaComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList() {{ add(new KafkaComponent.Topic(Constants.INDEXING_TOPIC, 1)); - add(new KafkaComponent.Topic(Constants.INDEXING_ERROR_TOPIC, 1)); + add(new KafkaComponent.Topic(Constants.ERROR_TOPIC, 1)); }}); List> inputDocs = new ArrayList<>(); for(byte[] b : inputMessages) { diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/ProcessorResult.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/ProcessorResult.java index 3eb4e8f074..c3096863d4 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/ProcessorResult.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/ProcessorResult.java @@ -24,7 +24,6 @@ public class ProcessorResult { public static class Builder{ T result; List processErrors; - List processInvalids; public Builder(){} @@ -38,25 +37,18 @@ public Builder withProcessErrors(List processErrors){ return this; } - public Builder withProcessInvalids(List processInvalids){ - this.processInvalids = processInvalids; - return this; - } - public ProcessorResult build(){ - return new ProcessorResult(result,processErrors,processInvalids); + return new ProcessorResult(result,processErrors); } } T result; List processErrors; - List processInvalids; @SuppressWarnings("unchecked") - public ProcessorResult(T result,List processErrors, List processInvalids){ + public ProcessorResult(T result,List processErrors){ this.result = result; this.processErrors = processErrors == null ? new ArrayList() : processErrors; - this.processInvalids = processInvalids == null ? new ArrayList() : processInvalids; } public T getResult(){ @@ -67,12 +59,8 @@ public List getProcessErrors(){ return processErrors; } - public List getProcessInvalids(){ - return processInvalids; - } - public boolean failed(){ - return processErrors.size() > 0 || processInvalids.size() > 0; + return processErrors.size() > 0; } public void getBadResults(StringBuffer buffer){ @@ -84,10 +72,5 @@ public void getBadResults(StringBuffer buffer){ buffer.append(new String(outputMessage)); } buffer.append("\n"); - buffer.append(String.format("%d Invalid Messages", processInvalids.size())); - for (byte[] outputMessage : processInvalids) { - buffer.append(new String(outputMessage)); - } - buffer.append("\n"); } } diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/processors/KafkaMessageSet.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/processors/KafkaMessageSet.java index 4227933f8c..683fe6a92f 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/processors/KafkaMessageSet.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/processors/KafkaMessageSet.java @@ -23,12 +23,10 @@ public class KafkaMessageSet{ public List messages; public List errors; - public List invalids; - public KafkaMessageSet(List messages, List errors, List invalids) { + public KafkaMessageSet(List messages, List errors) { this.messages = messages; this.errors = errors; - this.invalids = invalids; } @@ -38,7 +36,4 @@ public List getMessages() { public List getErrors() { return errors; } - public List getInvalids() { - return invalids; - } } diff --git a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/processors/KafkaProcessor.java b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/processors/KafkaProcessor.java index 6fdbbf4a73..63f073ddbe 100644 --- a/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/processors/KafkaProcessor.java +++ b/metron-platform/metron-integration-test/src/main/java/org/apache/metron/integration/processors/KafkaProcessor.java @@ -30,10 +30,8 @@ public class KafkaProcessor implements Processor { private String kafkaComponentName; private String readTopic; private String errorTopic; - private String invalidTopic; private List messages = new LinkedList<>(); private List errors = new LinkedList<>(); - private List invalids = new LinkedList<>(); public KafkaProcessor(){} public KafkaProcessor withKafkaComponentName(String name){ @@ -48,10 +46,6 @@ public KafkaProcessor withErrorTopic(String topicName){ this.errorTopic = topicName; return this; } - public KafkaProcessor withInvalidTopic(String topicName){ - this.invalidTopic = topicName; - return this; - } public KafkaProcessor withValidateReadMessages(Function validate){ this.validateReadMessages = validate; return this; @@ -68,25 +62,19 @@ public ReadinessState process(ComponentRunner runner){ KafkaComponent kafkaComponent = runner.getComponent(kafkaComponentName, KafkaComponent.class); LinkedList outputMessages = new LinkedList<>(kafkaComponent.readMessages(readTopic)); LinkedList outputErrors = null; - LinkedList outputInvalids = null; if (errorTopic != null) { outputErrors = new LinkedList<>(kafkaComponent.readMessages(errorTopic)); } - if (invalidTopic != null) { - outputInvalids = new LinkedList<>(kafkaComponent.readMessages(invalidTopic)); - } - Boolean validated = validateReadMessages.apply(new KafkaMessageSet(outputMessages,outputErrors,outputInvalids)); + Boolean validated = validateReadMessages.apply(new KafkaMessageSet(outputMessages,outputErrors)); if(validated == null){ validated = false; } if(validated){ messages.addAll(outputMessages); errors.addAll(outputErrors); - invalids.addAll(outputInvalids); outputMessages.clear(); outputErrors.clear(); - outputInvalids.clear(); return ReadinessState.READY; } return ReadinessState.NOT_READY; @@ -94,7 +82,7 @@ public ReadinessState process(ComponentRunner runner){ @SuppressWarnings("unchecked") public ProcessorResult getResult(){ ProcessorResult.Builder builder = new ProcessorResult.Builder(); - return builder.withResult(provideResult.apply(new KafkaMessageSet(messages,errors,invalids))).withProcessErrors(errors).withProcessInvalids(invalids).build(); + return builder.withResult(provideResult.apply(new KafkaMessageSet(messages,errors))).withProcessErrors(errors).build(); } } 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 416d3ed829..8226d712ba 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 @@ -17,6 +17,10 @@ */ package org.apache.metron.parsers.bolt; +import org.apache.metron.common.error.MetronError; +import org.apache.metron.common.message.BytesFromPosition; +import org.apache.metron.common.message.MessageGetStrategy; +import org.apache.metron.common.message.MessageGetters; import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase; import org.apache.commons.lang3.StringUtils; import org.apache.storm.task.OutputCollector; @@ -24,7 +28,6 @@ import org.apache.storm.topology.OutputFieldsDeclarer; import org.apache.storm.tuple.Fields; import org.apache.storm.tuple.Tuple; -import org.apache.storm.tuple.Values; import org.apache.metron.common.Constants; import org.apache.metron.common.bolt.ConfiguredParserBolt; import org.apache.metron.common.configuration.FieldValidator; @@ -42,6 +45,7 @@ import java.io.Serializable; import java.util.*; +import java.util.stream.Collectors; public class ParserBolt extends ConfiguredParserBolt implements Serializable { @@ -52,6 +56,7 @@ public class ParserBolt extends ConfiguredParserBolt implements Serializable { private MessageFilter filter; private WriterHandler writer; private org.apache.metron.common.dsl.Context stellarContext; + private transient MessageGetStrategy messageGetStrategy; public ParserBolt( String zookeeperUrl , String sensorType , MessageParser parser @@ -73,6 +78,7 @@ public ParserBolt withMessageFilter(MessageFilter filter) { @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { super.prepare(stormConf, context, collector); + messageGetStrategy = MessageGetters.BYTES_FROM_POSITION.get(); String hdfsFile = (String) getConfigurations().getGlobalConfig().get(GeoLiteDatabase.GEO_HDFS_FILE); GeoLiteDatabase.INSTANCE.update(hdfsFile); this.collector = collector; @@ -111,7 +117,8 @@ protected void initializeStellar() { @SuppressWarnings("unchecked") @Override public void execute(Tuple tuple) { - byte[] originalMessage = tuple.getBinary(0); + byte[] originalMessage = (byte[]) messageGetStrategy.get(tuple); + //byte[] originalMessage = (byte[]) messageGetter.getMessage(tuple); SensorParserConfig sensorParserConfig = getSensorParserConfig(); try { //we want to ack the tuple in the situation where we have are not doing a bulk write @@ -130,12 +137,22 @@ public void execute(Tuple tuple) { } if (parser.validate(message) && (filter == null || filter.emitTuple(message, stellarContext))) { numWritten++; - if(!isGloballyValid(message, fieldValidations)) { - message.put(Constants.SENSOR_TYPE, getSensorType()+ ".invalid"); - collector.emit(Constants.INVALID_STREAM, new Values(message)); + List failedValidators = getFailedValidators(message, fieldValidations); + if(failedValidators.size() > 0) { + MetronError error = new MetronError() + .withErrorType(Constants.ErrorType.PARSER_INVALID) + .withSensorType(getSensorType()) + .addRawMessage(message); + Set errorFields = failedValidators.stream() + .flatMap(fieldValidator -> fieldValidator.getInput().stream()) + .collect(Collectors.toSet()); + if (!errorFields.isEmpty()) { + error.withErrorFields(errorFields); + } + ErrorUtils.handleError(collector, error); } else { - writer.write(getSensorType(), tuple, message, getConfigurations()); + writer.write(getSensorType(), tuple, message, getConfigurations(), messageGetStrategy); } } } @@ -147,28 +164,28 @@ public void execute(Tuple tuple) { collector.ack(tuple); } } catch (Throwable ex) { - ErrorUtils.handleError( collector - , ex - , Constants.ERROR_STREAM - , Optional.of(getSensorType()) - , Optional.ofNullable(originalMessage) - ); + MetronError error = new MetronError() + .withErrorType(Constants.ErrorType.PARSER_ERROR) + .withThrowable(ex) + .withSensorType(getSensorType()) + .addRawMessage(originalMessage); + ErrorUtils.handleError(collector, error); collector.ack(tuple); } } - private boolean isGloballyValid(JSONObject input, List validators) { + private List getFailedValidators(JSONObject input, List validators) { + List failedValidators = new ArrayList<>(); for(FieldValidator validator : validators) { if(!validator.isValid(input, getConfigurations().getGlobalConfig(), stellarContext)) { - return false; + failedValidators.add(validator); } } - return true; + return failedValidators; } @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declareStream(Constants.INVALID_STREAM, new Fields("message")); declarer.declareStream(Constants.ERROR_STREAM, new Fields("message")); } } diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/WriterBolt.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/WriterBolt.java index 8eb065614d..ee8a627275 100644 --- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/WriterBolt.java +++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/WriterBolt.java @@ -18,6 +18,10 @@ package org.apache.metron.parsers.bolt; +import org.apache.metron.common.error.MetronError; +import org.apache.metron.common.message.JSONFromField; +import org.apache.metron.common.message.MessageGetStrategy; +import org.apache.metron.common.message.MessageGetters; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; import org.apache.storm.topology.OutputFieldsDeclarer; @@ -35,6 +39,8 @@ public class WriterBolt extends BaseRichBolt { private WriterHandler handler; private ParserConfigurations configuration; private String sensorType; + private Constants.ErrorType errorType = Constants.ErrorType.DEFAULT_ERROR; + private transient MessageGetStrategy messageGetStrategy; private transient OutputCollector collector; public WriterBolt(WriterHandler handler, ParserConfigurations configuration, String sensorType) { this.handler = handler; @@ -42,9 +48,15 @@ public WriterBolt(WriterHandler handler, ParserConfigurations configuration, Str this.sensorType = sensorType; } + public WriterBolt withErrorType(Constants.ErrorType errorType) { + this.errorType = errorType; + return this; + } + @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { this.collector = collector; + messageGetStrategy = MessageGetters.JSON_FROM_FIELD.get(); handler.init(stormConf, collector, configuration); } @@ -65,18 +77,18 @@ private JSONObject getMessage(Tuple tuple) { public void execute(Tuple tuple) { JSONObject message = null; try { - message = (JSONObject)((JSONObject) tuple.getValueByField("message")).clone(); - handler.write(sensorType, tuple, message, configuration); + message = (JSONObject) messageGetStrategy.get(tuple); + handler.write(sensorType, tuple, message, configuration, messageGetStrategy); if(!handler.handleAck()) { collector.ack(tuple); } } catch (Throwable e) { - ErrorUtils.handleError( collector - , e - , Constants.ERROR_STREAM - , Optional.of(sensorType) - , Optional.ofNullable(message) - ); + MetronError error = new MetronError() + .withErrorType(errorType) + .withThrowable(e) + .withSensorType(sensorType) + .addRawMessage(message); + ErrorUtils.handleError(collector, error); collector.ack(tuple); } } diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/WriterHandler.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/WriterHandler.java index 38425b5891..3273ca7971 100644 --- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/WriterHandler.java +++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/WriterHandler.java @@ -18,6 +18,7 @@ package org.apache.metron.parsers.bolt; +import org.apache.metron.common.message.MessageGetStrategy; import org.apache.storm.task.OutputCollector; import org.apache.storm.tuple.Tuple; import org.apache.metron.common.configuration.ParserConfigurations; @@ -80,11 +81,12 @@ public void write( String sensorType , Tuple tuple , JSONObject message , ParserConfigurations configurations + , MessageGetStrategy messageGetStrategy ) throws Exception { - writerComponent.write(sensorType, tuple, message, messageWriter, writerTransformer.apply(configurations)); + writerComponent.write(sensorType, tuple, message, messageWriter, writerTransformer.apply(configurations), messageGetStrategy); } - public void errorAll(String sensorType, Throwable e) { - writerComponent.errorAll(sensorType, e); + public void errorAll(String sensorType, Throwable e, MessageGetStrategy messageGetStrategy) { + writerComponent.errorAll(sensorType, e, messageGetStrategy); } } diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java index 5f1927e9bc..545f624ffb 100644 --- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java +++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java @@ -57,8 +57,6 @@ public class ParserTopologyBuilder { * @param spoutNumTasks Number of tasks for the spout * @param parserParallelism Parallelism hint for the parser bolt * @param parserNumTasks Number of tasks for the parser bolt - * @param invalidWriterParallelism Parallelism hint for the bolt that handles invalid data - * @param invalidWriterNumTasks Number of tasks for the bolt that handles invalid data * @param errorWriterParallelism Parallelism hint for the bolt that handles errors * @param errorWriterNumTasks Number of tasks for the bolt that handles errors * @param kafkaSpoutConfig Configuration options for the kafka spout @@ -73,8 +71,6 @@ public static TopologyBuilder build(String zookeeperUrl, int spoutNumTasks, int parserParallelism, int parserNumTasks, - int invalidWriterParallelism, - int invalidWriterNumTasks, int errorWriterParallelism, int errorWriterNumTasks, EnumMap kafkaSpoutConfig @@ -104,14 +100,6 @@ public static TopologyBuilder build(String zookeeperUrl, .shuffleGrouping("parserBolt", Constants.ERROR_STREAM); } - // create the invalid bolt, if needed - if (invalidWriterNumTasks > 0) { - WriterBolt invalidBolt = createInvalidBolt(brokerUrl, sensorType, configs, parserConfig); - builder.setBolt("invalidMessageWriter", invalidBolt, invalidWriterParallelism) - .setNumTasks(invalidWriterNumTasks) - .shuffleGrouping("parserBolt", Constants.INVALID_STREAM); - } - return builder; } @@ -161,29 +149,6 @@ private static ParserBolt createParserBolt(String zookeeperUrl, String brokerUrl return new ParserBolt(zookeeperUrl, sensorType, parser, writerHandler); } - /** - * Create a bolt that handles invalid messages. - * - * @param brokerUrl The Kafka Broker URL - * @param sensorType Type of sensor that is being consumed. - * @param configs - * @param parserConfig - * @return A Storm bolt that handles invalid messages. - */ - private static WriterBolt createInvalidBolt(String brokerUrl, String sensorType, ParserConfigurations configs, SensorParserConfig parserConfig) { - - // create writer - if not configured uses a sensible default - AbstractWriter writer = parserConfig.getErrorWriterClassName() == null - ? new KafkaWriter(brokerUrl).withTopic(Constants.DEFAULT_PARSER_INVALID_TOPIC).withConfigPrefix("invalid") - : ReflectionUtils.createInstance(parserConfig.getWriterClassName()); - writer.configure(sensorType, new ParserWriterConfiguration(configs)); - - // create a writer handler - WriterHandler writerHandler = createWriterHandler(writer); - - return new WriterBolt(writerHandler, configs, sensorType); - } - /** * Create a bolt that handles error messages. * @@ -197,14 +162,14 @@ private static WriterBolt createErrorBolt(String brokerUrl, String sensorType, P // create writer - if not configured uses a sensible default AbstractWriter writer = parserConfig.getErrorWriterClassName() == null - ? new KafkaWriter(brokerUrl).withTopic(Constants.DEFAULT_PARSER_ERROR_TOPIC).withConfigPrefix("error") + ? new KafkaWriter(brokerUrl).withTopic(Constants.ERROR_TOPIC).withConfigPrefix("error") : ReflectionUtils.createInstance(parserConfig.getWriterClassName()); writer.configure(sensorType, new ParserWriterConfiguration(configs)); // create a writer handler WriterHandler writerHandler = createWriterHandler(writer); - return new WriterBolt(writerHandler, configs, sensorType); + return new WriterBolt(writerHandler, configs, sensorType).withErrorType(Constants.ErrorType.PARSER_ERROR); } /** diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyCLI.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyCLI.java index 5ea561c709..2bf484ec4e 100644 --- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyCLI.java +++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyCLI.java @@ -302,8 +302,6 @@ public static void main(String[] args) { spoutNumTasks, parserParallelism, parserNumTasks, - invalidParallelism, - invalidNumTasks, errorParallelism, errorNumTasks, spoutConfig 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 4ba1c43a76..cca0852747 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 @@ -47,8 +47,7 @@ public void test() throws Exception { 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(Constants.INVALID_STREAM,1)); - add(new KafkaComponent.Topic(Constants.ERROR_STREAM,1)); + add(new KafkaComponent.Topic(Constants.ERROR_TOPIC,1)); }}); topologyProperties.setProperty("kafka.broker", kafkaComponent.getBrokerList()); @@ -116,12 +115,11 @@ private KafkaProcessor> getProcessor(){ .withKafkaComponentName("kafka") .withReadTopic(Constants.ENRICHMENT_TOPIC) .withErrorTopic(Constants.ERROR_STREAM) - .withInvalidTopic(Constants.INVALID_STREAM) .withValidateReadMessages(new Function() { @Nullable @Override public Boolean apply(@Nullable KafkaMessageSet messageSet) { - return (messageSet.getMessages().size() + messageSet.getErrors().size() + messageSet.getInvalids().size()) == inputMessages.size(); + return (messageSet.getMessages().size() + messageSet.getErrors().size() == inputMessages.size()); } }) .withProvideResult(new Function>(){ 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 80b6ebdb87..73d38274e1 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 @@ -34,6 +34,7 @@ public class ParserTopologyComponent implements InMemoryComponent { private Properties topologyProperties; private String brokerUrl; private String sensorType; + private SpoutConfig.Offset offset = SpoutConfig.Offset.BEGINNING; private LocalCluster stormCluster; public static class Builder { @@ -64,15 +65,17 @@ public ParserTopologyComponent(Properties topologyProperties, String brokerUrl, this.sensorType = sensorType; } + public void setOffset(SpoutConfig.Offset offset) { + this.offset = offset; + } + @Override public void start() throws UnableToStartException { try { TopologyBuilder topologyBuilder = ParserTopologyBuilder.build(topologyProperties.getProperty("kafka.zk") , brokerUrl , sensorType - , SpoutConfig.Offset.BEGINNING - , 1 - , 1 + , offset , 1 , 1 , 1 diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/WriterBoltIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/WriterBoltIntegrationTest.java index eaeaeb5f96..b2284303b5 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/WriterBoltIntegrationTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/WriterBoltIntegrationTest.java @@ -101,8 +101,7 @@ public void test() throws UnableToStartException, IOException { final ZKServerComponent zkServerComponent = getZKServerComponent(topologyProperties); final KafkaComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList() {{ add(new KafkaComponent.Topic(sensorType, 1)); - add(new KafkaComponent.Topic(Constants.DEFAULT_PARSER_ERROR_TOPIC, 1)); - add(new KafkaComponent.Topic(Constants.DEFAULT_PARSER_INVALID_TOPIC, 1)); + add(new KafkaComponent.Topic(Constants.ERROR_TOPIC, 1)); add(new KafkaComponent.Topic(Constants.ENRICHMENT_TOPIC, 1)); }}); topologyProperties.setProperty("kafka.broker", kafkaComponent.getBrokerList()); @@ -132,17 +131,18 @@ public void test() throws UnableToStartException, IOException { kafkaComponent.writeMessages(sensorType, inputMessages); ProcessorResult>> result = runner.process(getProcessor()); Map> outputMessages = result.getResult(); - Assert.assertEquals(3, outputMessages.size()); + Assert.assertEquals(2, outputMessages.size()); Assert.assertEquals(1, outputMessages.get(Constants.ENRICHMENT_TOPIC).size()); Assert.assertEquals("valid", outputMessages.get(Constants.ENRICHMENT_TOPIC).get(0).get("action")); - Assert.assertEquals(1, outputMessages.get(Constants.DEFAULT_PARSER_ERROR_TOPIC).size()); - Assert.assertEquals("error", outputMessages.get(Constants.DEFAULT_PARSER_ERROR_TOPIC).get(0).get("rawMessage")); - Assert.assertTrue(Arrays.equals(listToBytes(outputMessages.get(Constants.DEFAULT_PARSER_ERROR_TOPIC).get(0).get("rawMessage_bytes")) - , "error".getBytes() - ) - ); - Assert.assertEquals(1, outputMessages.get(Constants.DEFAULT_PARSER_INVALID_TOPIC).size()); - Assert.assertEquals("invalid", outputMessages.get(Constants.DEFAULT_PARSER_INVALID_TOPIC).get(0).get("action")); + Assert.assertEquals(2, outputMessages.get(Constants.ERROR_TOPIC).size()); + JSONObject invalidMessage = outputMessages.get(Constants.ERROR_TOPIC).get(0); + Assert.assertEquals(Constants.ErrorType.PARSER_INVALID.getType(), invalidMessage.get(Constants.ErrorFields.ERROR_TYPE.getName())); + Assert.assertEquals("foo", ((Map) invalidMessage.get(Constants.ErrorFields.RAW_MESSAGE.getName())).get("dummy")); + Assert.assertEquals("invalid", ((Map) invalidMessage.get(Constants.ErrorFields.RAW_MESSAGE.getName())).get("action")); + JSONObject errorMessage = outputMessages.get(Constants.ERROR_TOPIC).get(1); + Assert.assertEquals(Constants.ErrorType.PARSER_ERROR.getType(), errorMessage.get(Constants.ErrorFields.ERROR_TYPE.getName())); + Assert.assertEquals("error", errorMessage.get(Constants.ErrorFields.RAW_MESSAGE.getName())); + Assert.assertTrue(Arrays.equals(listToBytes(errorMessage.get(Constants.ErrorFields.RAW_MESSAGE_BYTES.getName())), "error".getBytes())); } finally { if(runner != null) { @@ -183,13 +183,12 @@ private KafkaProcessor>> getProcessor(){ return new KafkaProcessor<>() .withKafkaComponentName("kafka") .withReadTopic(Constants.ENRICHMENT_TOPIC) - .withErrorTopic(Constants.DEFAULT_PARSER_ERROR_TOPIC) - .withInvalidTopic(Constants.DEFAULT_PARSER_INVALID_TOPIC) + .withErrorTopic(Constants.ERROR_TOPIC) .withValidateReadMessages(new Function() { @Nullable @Override public Boolean apply(@Nullable KafkaMessageSet messageSet) { - return (messageSet.getMessages().size() == 1) && (messageSet.getErrors().size() == 1) && (messageSet.getInvalids().size() ==1); + return (messageSet.getMessages().size() == 1) && (messageSet.getErrors().size() == 2); } }) .withProvideResult(new Function>>(){ @@ -198,8 +197,7 @@ public Boolean apply(@Nullable KafkaMessageSet messageSet) { public Map> apply(@Nullable KafkaMessageSet messageSet) { return new HashMap>() {{ put(Constants.ENRICHMENT_TOPIC, loadMessages(messageSet.getMessages())); - put(Constants.DEFAULT_PARSER_ERROR_TOPIC, loadMessages(messageSet.getErrors())); - put(Constants.DEFAULT_PARSER_INVALID_TOPIC, loadMessages(messageSet.getInvalids())); + put(Constants.ERROR_TOPIC, loadMessages(messageSet.getErrors())); }}; } }); diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrIndexingIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrIndexingIntegrationTest.java index abf3a8a279..1c32adb1bc 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrIndexingIntegrationTest.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrIndexingIntegrationTest.java @@ -89,7 +89,7 @@ public ReadinessState process(ComponentRunner runner) { throw new IllegalStateException("Unable to retrieve indexed documents.", e); } if (docs.size() < inputMessages.size() || docs.size() != docsFromDisk.size()) { - errors = kafkaComponent.readMessages(Constants.INDEXING_ERROR_TOPIC); + errors = kafkaComponent.readMessages(Constants.ERROR_TOPIC); if(errors.size() > 0){ return ReadinessState.READY; } diff --git a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/BulkWriterComponent.java b/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/BulkWriterComponent.java index 124ffd3b36..c25300b652 100644 --- a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/BulkWriterComponent.java +++ b/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/BulkWriterComponent.java @@ -18,6 +18,10 @@ package org.apache.metron.writer; +import org.apache.metron.common.error.MetronError; +import org.apache.metron.common.message.MessageGetStrategy; +import org.apache.metron.writer.message.MessageGetter; +import org.apache.metron.writer.message.MessageGetters; import org.apache.storm.task.OutputCollector; import org.apache.storm.tuple.Tuple; import com.google.common.collect.Iterables; @@ -30,6 +34,9 @@ import org.slf4j.LoggerFactory; import java.util.*; +import java.util.stream.Collector; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; public class BulkWriterComponent { public static final Logger LOG = LoggerFactory @@ -60,18 +67,23 @@ public void commit(BulkWriterResponse response) { commit(response.getSuccesses()); } - public void error(Throwable e, Iterable tuples) { + public void error(String sensorType, Throwable e, Iterable tuples, MessageGetStrategy messageGetStrategy) { tuples.forEach(t -> collector.ack(t)); + MetronError error = new MetronError() + .withSensorType(sensorType) + .withErrorType(Constants.ErrorType.INDEXING_ERROR) + .withThrowable(e); if(!Iterables.isEmpty(tuples)) { LOG.error("Failing " + Iterables.size(tuples) + " tuples", e); - ErrorUtils.handleError(collector, e, Constants.ERROR_STREAM); } + tuples.forEach(t -> error.addRawMessage(messageGetStrategy.get(t))); + ErrorUtils.handleError(collector, error); } - public void error(BulkWriterResponse errors) { + public void error(String sensorType, BulkWriterResponse errors, MessageGetStrategy messageGetStrategy) { Map> errorMap = errors.getErrors(); for(Map.Entry> entry : errorMap.entrySet()) { - error(entry.getKey(), entry.getValue()); + error(sensorType, entry.getKey(), entry.getValue(), messageGetStrategy); } } @@ -80,24 +92,25 @@ protected Collection createTupleCollection() { } - public void errorAll(Throwable e) { - for(Map.Entry> kv : sensorTupleMap.entrySet()) { - error(e, kv.getValue()); - sensorTupleMap.remove(kv.getKey()); - sensorMessageMap.remove(kv.getKey()); + public void errorAll(Throwable e, MessageGetStrategy messageGetStrategy) { + for(String key : new HashSet<>(sensorTupleMap.keySet())) { + errorAll(key, e, messageGetStrategy); } } - public void errorAll(String sensorType, Throwable e) { - error(e, Optional.ofNullable(sensorTupleMap.get(sensorType)).orElse(new ArrayList<>())); + public void errorAll(String sensorType, Throwable e, MessageGetStrategy messageGetStrategy) { + Collection tuples = Optional.ofNullable(sensorTupleMap.get(sensorType)).orElse(new ArrayList<>()); + error(sensorType, e, tuples, messageGetStrategy); sensorTupleMap.remove(sensorType); sensorMessageMap.remove(sensorType); } + public void write( String sensorType , Tuple tuple , MESSAGE_T message , BulkMessageWriter bulkMessageWriter , WriterConfiguration configurations + , MessageGetStrategy messageGetStrategy ) throws Exception { if(!configurations.isEnabled(sensorType)) { @@ -129,13 +142,13 @@ public void write( String sensorType } if(handleError) { - error(response); + error(sensorType, response, messageGetStrategy); } else if (response.hasErrors()) { throw new IllegalStateException("Unhandled bulk errors in response: " + response.getErrors()); } } catch (Throwable e) { if(handleError) { - error(e, tupleList); + error(sensorType, e, tupleList, messageGetStrategy); } else { throw e; diff --git a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/bolt/BulkMessageWriterBolt.java b/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/bolt/BulkMessageWriterBolt.java index 66c4c73d43..b0e61c77a3 100644 --- a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/bolt/BulkMessageWriterBolt.java +++ b/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/bolt/BulkMessageWriterBolt.java @@ -18,6 +18,8 @@ package org.apache.metron.writer.bolt; import org.apache.metron.common.bolt.ConfiguredIndexingBolt; +import org.apache.metron.common.message.MessageGetStrategy; +import org.apache.metron.common.message.MessageGetters; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; import org.apache.storm.topology.OutputFieldsDeclarer; @@ -31,8 +33,6 @@ import org.apache.metron.common.writer.BulkMessageWriter; import org.apache.metron.writer.BulkWriterComponent; import org.apache.metron.writer.WriterToBulkWriter; -import org.apache.metron.writer.message.MessageGetter; -import org.apache.metron.writer.message.MessageGetters; import org.json.simple.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,8 +46,8 @@ public class BulkMessageWriterBolt extends ConfiguredIndexingBolt { .getLogger(BulkMessageWriterBolt.class); private BulkMessageWriter bulkMessageWriter; private BulkWriterComponent writerComponent; - private String messageGetterStr = MessageGetters.NAMED.name(); - private transient MessageGetter messageGetter = null; + private String messageGetStrategyType = MessageGetters.JSON_FROM_FIELD.name(); + private transient MessageGetStrategy messageGetStrategy; private transient OutputCollector collector; private transient Function configurationTransformation; public BulkMessageWriterBolt(String zookeeperUrl) { @@ -64,8 +64,8 @@ public BulkMessageWriterBolt withMessageWriter(MessageWriter message return this; } - public BulkMessageWriterBolt withMessageGetter(String messageGetter) { - this.messageGetterStr = messageGetter; + public BulkMessageWriterBolt withMessageGetter(String messageGetStrategyType) { + this.messageGetStrategyType = messageGetStrategyType; return this; } @@ -74,7 +74,7 @@ public void prepare(Map stormConf, TopologyContext context, OutputCollector coll this.writerComponent = new BulkWriterComponent<>(collector); this.collector = collector; super.prepare(stormConf, context, collector); - messageGetter = MessageGetters.valueOf(messageGetterStr); + messageGetStrategy = MessageGetters.valueOf(messageGetStrategyType).get(); if(bulkMessageWriter instanceof WriterToBulkWriter) { configurationTransformation = WriterToBulkWriter.TRANSFORMATION; } @@ -93,7 +93,7 @@ public void prepare(Map stormConf, TopologyContext context, OutputCollector coll @SuppressWarnings("unchecked") @Override public void execute(Tuple tuple) { - JSONObject message = messageGetter.getMessage(tuple); + JSONObject message = (JSONObject) messageGetStrategy.get(tuple); String sensorType = MessageUtils.getSensorType(message); try { @@ -107,6 +107,7 @@ public void execute(Tuple tuple) { , message , bulkMessageWriter , writerConfiguration + , messageGetStrategy ); LOG.trace("Writing enrichment message: {}", message); } diff --git a/metron-platform/metron-writer/src/test/java/org/apache/metron/writer/BulkWriterComponentTest.java b/metron-platform/metron-writer/src/test/java/org/apache/metron/writer/BulkWriterComponentTest.java new file mode 100644 index 0000000000..c560b30686 --- /dev/null +++ b/metron-platform/metron-writer/src/test/java/org/apache/metron/writer/BulkWriterComponentTest.java @@ -0,0 +1,197 @@ +/** + * 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.writer; + +import org.apache.metron.common.Constants; +import org.apache.metron.common.configuration.writer.WriterConfiguration; +import org.apache.metron.common.error.MetronError; +import org.apache.metron.common.message.MessageGetStrategy; +import org.apache.metron.common.message.MessageGetters; +import org.apache.metron.common.utils.ErrorUtils; +import org.apache.metron.common.writer.BulkMessageWriter; +import org.apache.metron.common.writer.BulkWriterResponse; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.tuple.Tuple; +import org.json.simple.JSONObject; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.powermock.api.mockito.PowerMockito.mockStatic; +import static org.powermock.api.mockito.PowerMockito.verifyStatic; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({BulkWriterComponent.class, ErrorUtils.class}) +public class BulkWriterComponentTest { + + @Rule + public final ExpectedException exception = ExpectedException.none(); + + @Mock + private OutputCollector collector; + + @Mock + private BulkMessageWriter bulkMessageWriter; + + @Mock + private WriterConfiguration configurations; + + @Mock + private Tuple tuple1; + + @Mock + private Tuple tuple2; + + @Mock + private MessageGetStrategy messageGetStrategy; + + private String sensorType = "testSensor"; + private List tupleList; + private JSONObject message1 = new JSONObject(); + private JSONObject message2 = new JSONObject(); + + @Before + public void setup() { + MockitoAnnotations.initMocks(this); + mockStatic(ErrorUtils.class); + message1.put("value", "message1"); + message2.put("value", "message2"); + when(tuple1.getValueByField("message")).thenReturn(message1); + when(tuple2.getValueByField("message")).thenReturn(message2); + tupleList = Arrays.asList(tuple1, tuple2); + when(configurations.isEnabled(any())).thenReturn(true); + when(configurations.getBatchSize(any())).thenReturn(2); + when(messageGetStrategy.get(tuple1)).thenReturn(message1); + when(messageGetStrategy.get(tuple2)).thenReturn(message2); + } + + @Test + public void writeShouldProperlyAckTuplesInBatch() throws Exception { + BulkWriterResponse response = new BulkWriterResponse(); + response.addAllSuccesses(tupleList); + + when(bulkMessageWriter.write(sensorType, configurations, Arrays.asList(tuple1, tuple2), Arrays.asList(message1, message2))).thenReturn(response); + + BulkWriterComponent bulkWriterComponent = new BulkWriterComponent<>(collector); + bulkWriterComponent.write(sensorType, tuple1, message1, bulkMessageWriter, configurations, messageGetStrategy); + + verify(bulkMessageWriter, times(0)).write(sensorType, configurations, Collections.singletonList(tuple1), Collections.singletonList(message1)); + verify(collector, times(0)).ack(tuple1); + verify(collector, times(0)).ack(tuple2); + + bulkWriterComponent.write(sensorType, tuple2, message2, bulkMessageWriter, configurations, messageGetStrategy); + + verify(collector, times(1)).ack(tuple1); + verify(collector, times(1)).ack(tuple2); + verifyStatic(times(0)); + ErrorUtils.handleError(eq(collector), any(MetronError.class)); + } + + @Test + public void writeShouldProperlyHandleWriterErrors() throws Exception { + Throwable e = new Exception("test exception"); + MetronError error = new MetronError() + .withSensorType(sensorType) + .withErrorType(Constants.ErrorType.INDEXING_ERROR).withThrowable(e).withRawMessages(Arrays.asList(message1, message2)); + BulkWriterResponse response = new BulkWriterResponse(); + response.addAllErrors(e, tupleList); + + when(bulkMessageWriter.write(sensorType, configurations, Arrays.asList(tuple1, tuple2), Arrays.asList(message1, message2))).thenReturn(response); + + BulkWriterComponent bulkWriterComponent = new BulkWriterComponent<>(collector); + bulkWriterComponent.write(sensorType, tuple1, message1, bulkMessageWriter, configurations, messageGetStrategy); + bulkWriterComponent.write(sensorType, tuple2, message2, bulkMessageWriter, configurations, messageGetStrategy); + + verifyStatic(times(1)); + ErrorUtils.handleError(collector, error); + } + + @Test + public void writeShouldThrowExceptionWhenHandleErrorIsFalse() throws Exception { + exception.expect(IllegalStateException.class); + + Throwable e = new Exception("test exception"); + BulkWriterResponse response = new BulkWriterResponse(); + response.addAllErrors(e, tupleList); + + when(bulkMessageWriter.write(sensorType, configurations, Arrays.asList(tuple1, tuple2), Arrays.asList(message1, message2))).thenReturn(response); + + BulkWriterComponent bulkWriterComponent = new BulkWriterComponent<>(collector, true, false); + bulkWriterComponent.write(sensorType, tuple1, message1, bulkMessageWriter, configurations, messageGetStrategy); + bulkWriterComponent.write(sensorType, tuple2, message2, bulkMessageWriter, configurations, messageGetStrategy); + } + + @Test + public void writeShouldProperlyHandleWriterException() throws Exception { + Throwable e = new Exception("test exception"); + MetronError error = new MetronError() + .withSensorType(sensorType) + .withErrorType(Constants.ErrorType.INDEXING_ERROR).withThrowable(e).withRawMessages(Arrays.asList(message1, message2)); + BulkWriterResponse response = new BulkWriterResponse(); + response.addAllErrors(e, tupleList); + + when(bulkMessageWriter.write(sensorType, configurations, Arrays.asList(tuple1, tuple2), Arrays.asList(message1, message2))).thenThrow(e); + + BulkWriterComponent bulkWriterComponent = new BulkWriterComponent<>(collector); + bulkWriterComponent.write(sensorType, tuple1, message1, bulkMessageWriter, configurations, messageGetStrategy); + bulkWriterComponent.write(sensorType, tuple2, message2, bulkMessageWriter, configurations, messageGetStrategy); + + verifyStatic(times(1)); + ErrorUtils.handleError(collector, error); + } + + @Test + public void errorAllShouldClearMapsAndHandleErrors() throws Exception { + Throwable e = new Exception("test exception"); + MetronError error1 = new MetronError() + .withSensorType("sensor1") + .withErrorType(Constants.ErrorType.INDEXING_ERROR).withThrowable(e).withRawMessages(Collections.singletonList(message1)); + MetronError error2 = new MetronError() + .withSensorType("sensor2") + .withErrorType(Constants.ErrorType.INDEXING_ERROR).withThrowable(e).withRawMessages(Collections.singletonList(message2)); + + BulkWriterComponent bulkWriterComponent = new BulkWriterComponent<>(collector); + bulkWriterComponent.write("sensor1", tuple1, message1, bulkMessageWriter, configurations, messageGetStrategy); + bulkWriterComponent.write("sensor2", tuple2, message2, bulkMessageWriter, configurations, messageGetStrategy); + bulkWriterComponent.errorAll(e, messageGetStrategy); + + verifyStatic(times(1)); + ErrorUtils.handleError(collector, error1); + ErrorUtils.handleError(collector, error2); + + bulkWriterComponent.write("sensor1", tuple1, message1, bulkMessageWriter, configurations, messageGetStrategy); + verify(bulkMessageWriter, times(0)).write(sensorType, configurations, Collections.singletonList(tuple1), Collections.singletonList(message1)); + } + + +} From a24e621d79a105ebe1a1c69d0fb1601d7940f96b Mon Sep 17 00:00:00 2001 From: rmerriman Date: Tue, 7 Feb 2017 18:22:16 -0600 Subject: [PATCH 02/16] Updated tests to include error conditions --- metron-platform/metron-common/pom.xml | 6 ++ .../metron/common/utils/ErrorUtilsTest.java | 21 ++++++ .../bolt/BulkMessageWriterBoltTest.java | 2 +- .../bolt/GenericEnrichmentBoltTest.java | 44 ++++++++++--- .../metron/enrichment/bolt/JoinBoltTest.java | 23 +++++-- .../EnrichmentIntegrationTest.java | 35 ---------- .../metron/parsers/bolt/ParserBolt.java | 31 +++++---- .../metron/parsers/bolt/ParserBoltTest.java | 66 +++++++++++++++++++ .../metron/parsers/bolt/WriterBoltTest.java | 36 ++++++---- .../test/error/MetronErrorJSONMatcher.java} | 29 ++++---- .../metron/writer/BulkWriterComponent.java | 23 +++---- .../writer/bolt/BulkMessageWriterBolt.java | 13 +++- .../metron/writer/message/MessageGetter.java | 26 -------- .../writer/message/NamedMessageGetter.java | 34 ---------- .../writer/message/RawMessageGetter.java | 50 -------------- 15 files changed, 230 insertions(+), 209 deletions(-) rename metron-platform/{metron-writer/src/main/java/org/apache/metron/writer/message/MessageGetters.java => metron-test-utilities/src/main/java/org/apache/metron/test/error/MetronErrorJSONMatcher.java} (59%) delete mode 100644 metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/MessageGetter.java delete mode 100644 metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/NamedMessageGetter.java delete mode 100644 metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/RawMessageGetter.java diff --git a/metron-platform/metron-common/pom.xml b/metron-platform/metron-common/pom.xml index e122dbd373..afa581586b 100644 --- a/metron-platform/metron-common/pom.xml +++ b/metron-platform/metron-common/pom.xml @@ -331,6 +331,12 @@ stream 2.9.5 + + org.apache.metron + metron-test-utilities + ${project.parent.version} + test + diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/ErrorUtilsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/ErrorUtilsTest.java index f11a5c9842..77ea9da197 100644 --- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/ErrorUtilsTest.java +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/ErrorUtilsTest.java @@ -17,6 +17,10 @@ */ package org.apache.metron.common.utils; +import org.apache.metron.common.Constants; +import org.apache.metron.common.error.MetronError; +import org.apache.metron.test.error.MetronErrorJSONMatcher; +import org.apache.storm.task.OutputCollector; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -25,6 +29,12 @@ import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.nullValue; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.argThat; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; public class ErrorUtilsTest { @@ -62,4 +72,15 @@ public void illegal_state_throws_exception_with_reason_and_cause() throws Except exception.expectCause(instanceOf(IOException.class)); ErrorUtils.RuntimeErrors.ILLEGAL_STATE.throwRuntime("illegal state happened", new IOException("bad io")); } + + @Test + public void handleErrorShouldEmitAndReportError() throws Exception { + Throwable e = new Exception("error"); + MetronError error = new MetronError().withMessage("error message").withThrowable(e); + OutputCollector collector = mock(OutputCollector.class); + + ErrorUtils.handleError(collector, error); + verify(collector, times(1)).emit(eq(Constants.ERROR_STREAM), argThat(new MetronErrorJSONMatcher(error.getJSONObject()))); + verify(collector, times(1)).reportError(any()); + } } diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java index 2016ef3934..aca644e68e 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java @@ -118,7 +118,7 @@ public void parseMessages() throws ParseException { @Test public void test() throws Exception { - BulkMessageWriterBolt bulkMessageWriterBolt = new BulkMessageWriterBolt("zookeeperUrl").withBulkMessageWriter(bulkMessageWriter); + BulkMessageWriterBolt bulkMessageWriterBolt = new BulkMessageWriterBolt("zookeeperUrl").withBulkMessageWriter(bulkMessageWriter).withMessageGetterField("message"); bulkMessageWriterBolt.setCuratorFramework(client); bulkMessageWriterBolt.setTreeCache(cache); bulkMessageWriterBolt.getConfigurations().updateSensorIndexingConfig(sensorType, new FileInputStream(sampleSensorIndexingConfigPath)); diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java index b0076a4d19..90322fe172 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/GenericEnrichmentBoltTest.java @@ -17,18 +17,22 @@ */ package org.apache.metron.enrichment.bolt; -import org.apache.log4j.Level; -import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase; -import org.apache.metron.test.utils.UnitTestHelper; -import org.apache.storm.tuple.Values; +import com.google.common.cache.CacheLoader; import com.google.common.collect.ImmutableMap; import org.adrianwalker.multilinestring.Multiline; +import org.apache.log4j.Level; import org.apache.metron.TestConstants; -import org.apache.metron.test.bolt.BaseEnrichmentBoltTest; -import org.apache.metron.enrichment.configuration.Enrichment; +import org.apache.metron.common.Constants; +import org.apache.metron.common.configuration.ConfigurationsUtils; import org.apache.metron.common.configuration.enrichment.SensorEnrichmentConfig; +import org.apache.metron.common.error.MetronError; +import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase; +import org.apache.metron.enrichment.configuration.Enrichment; import org.apache.metron.enrichment.interfaces.EnrichmentAdapter; -import org.apache.metron.common.configuration.ConfigurationsUtils; +import org.apache.metron.test.bolt.BaseEnrichmentBoltTest; +import org.apache.metron.test.error.MetronErrorJSONMatcher; +import org.apache.metron.test.utils.UnitTestHelper; +import org.apache.storm.tuple.Values; import org.hamcrest.Description; import org.json.simple.JSONObject; import org.json.simple.parser.JSONParser; @@ -43,12 +47,16 @@ import java.io.FileInputStream; import java.io.IOException; import java.util.HashMap; +import java.util.HashSet; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.argThat; import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class GenericEnrichmentBoltTest extends BaseEnrichmentBoltTest { @@ -194,7 +202,10 @@ protected void initializeStellar() { UnitTestHelper.setLog4jLevel(GenericEnrichmentBolt.class, Level.FATAL); genericEnrichmentBolt.execute(tuple); UnitTestHelper.setLog4jLevel(GenericEnrichmentBolt.class, Level.ERROR); - verify(outputCollector, times(1)).emit(eq("error"), any(Values.class)); + MetronError error = new MetronError() + .withErrorType(Constants.ErrorType.ENRICHMENT_ERROR) + .withThrowable(new Exception("Could not parse binary stream to JSON")); + verify(outputCollector, times(1)).emit(eq(Constants.ERROR_STREAM), argThat(new MetronErrorJSONMatcher(error.getJSONObject()))); when(tuple.getStringByField("key")).thenReturn(key); when(tuple.getValueByField("message")).thenReturn(originalMessage); when(enrichmentAdapter.enrich(any())).thenReturn(new JSONObject()); @@ -217,6 +228,19 @@ protected void initializeStellar() { verify(enrichmentAdapter, times(1)).logAccess(cacheKey2); verify(outputCollector, times(1)).emit(eq(enrichmentType), argThat(new EnrichedMessageMatcher(key, enrichedMessage))); - + reset(outputCollector); + genericEnrichmentBolt.cache.invalidateAll(); + when(enrichmentAdapter.enrich(cacheKey1)).thenReturn(null); + genericEnrichmentBolt.execute(tuple); + error = new MetronError() + .withErrorType(Constants.ErrorType.ENRICHMENT_ERROR) + .withErrorFields(new HashSet() {{ add("field1"); }}) + .addRawMessage(new JSONObject() {{ + put("field1", "value1"); + put("field2", "value2"); + put("source.type", "test"); + }}) + .withThrowable(new CacheLoader.InvalidCacheLoadException("CacheLoader returned null for key CacheKey{field='field1', value='value1'}.")); + verify(outputCollector, times(1)).emit(eq(Constants.ERROR_STREAM), argThat(new MetronErrorJSONMatcher(error.getJSONObject()))); } } diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/JoinBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/JoinBoltTest.java index 1e47d542c3..9f12fcd858 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/JoinBoltTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/JoinBoltTest.java @@ -17,11 +17,14 @@ */ package org.apache.metron.enrichment.bolt; -import org.apache.metron.common.message.MessageGetStrategy; -import org.apache.storm.task.TopologyContext; -import org.apache.storm.tuple.Values; +import com.google.common.cache.LoadingCache; import org.adrianwalker.multilinestring.Multiline; +import org.apache.metron.common.Constants; +import org.apache.metron.common.error.MetronError; import org.apache.metron.test.bolt.BaseEnrichmentBoltTest; +import org.apache.metron.test.error.MetronErrorJSONMatcher; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.tuple.Values; import org.json.simple.JSONObject; import org.json.simple.parser.JSONParser; import org.json.simple.parser.ParseException; @@ -32,6 +35,7 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutionException; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; @@ -91,7 +95,7 @@ public void parseMessages() { } @Test - public void test() { + public void test() throws Exception { StandAloneJoinBolt joinBolt = new StandAloneJoinBolt("zookeeperUrl"); joinBolt.setCuratorFramework(client); joinBolt.setTreeCache(cache); @@ -128,5 +132,16 @@ public void test() { joinBolt.execute(tuple); verify(outputCollector, times(1)).emit(eq("message"), any(tuple.getClass()), eq(new Values(key, joinedMessage))); verify(outputCollector, times(1)).ack(tuple); + + joinBolt.cache = mock(LoadingCache.class); + when(joinBolt.cache.get(key)).thenThrow(new ExecutionException(new Exception("join exception"))); + joinBolt.execute(tuple); + + MetronError error = new MetronError() + .withErrorType(Constants.ErrorType.ENRICHMENT_ERROR) + .withMessage("Joining problem: {}") + .withThrowable(new ExecutionException(new Exception("join exception"))) + .addRawMessage(new JSONObject()); + verify(outputCollector, times(1)).emit(eq(Constants.ERROR_STREAM), argThat(new MetronErrorJSONMatcher(error.getJSONObject()))); } } diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java index db301e3325..0d32345f7b 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java @@ -466,41 +466,6 @@ private static Set setOf(String... items) { return ret; } -// @SuppressWarnings("unchecked") -// private Processor>> getProcessor() { -// -// KafkaProcessor>> kafkaProcessor = new KafkaProcessor<>().withKafkaComponentName("kafka") -// .withReadTopic(Constants.INDEXING_TOPIC) -// .withErrorTopic(Constants.ERROR_TOPIC) -// .withValidateReadMessages(new Function() { -// @Nullable -// @Override -// public Boolean apply(@Nullable KafkaMessageSet messageSet) { -// // this test is written to return 10 errors and 10 messages -// // we can just check when the messages match here -// // if they do then we are good -// return messageSet.getMessages().size() == inputMessages.size(); -// } -// }) -// .withProvideResult(new Function>>() { -// @Nullable -// @Override -// public List> apply(@Nullable KafkaMessageSet messageSet) { -// List> docs = new ArrayList<>(); -// for (byte[] message : messageSet.getMessages()) { -// try { -// docs.add(JSONUtils.INSTANCE.load(new String(message), new TypeReference>() { -// })); -// } catch (IOException e) { -// throw new IllegalStateException(e.getMessage(), e); -// } -// } -// return docs; -// } -// }); -// return kafkaProcessor; -// } - private static List> loadMessages(List outputMessages) { List> tmp = new ArrayList<>(); Iterables.addAll(tmp 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 8226d712ba..83bdc4ff20 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 @@ -17,34 +17,38 @@ */ package org.apache.metron.parsers.bolt; -import org.apache.metron.common.error.MetronError; -import org.apache.metron.common.message.BytesFromPosition; -import org.apache.metron.common.message.MessageGetStrategy; -import org.apache.metron.common.message.MessageGetters; -import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase; import org.apache.commons.lang3.StringUtils; -import org.apache.storm.task.OutputCollector; -import org.apache.storm.task.TopologyContext; -import org.apache.storm.topology.OutputFieldsDeclarer; -import org.apache.storm.tuple.Fields; -import org.apache.storm.tuple.Tuple; import org.apache.metron.common.Constants; import org.apache.metron.common.bolt.ConfiguredParserBolt; +import org.apache.metron.common.configuration.FieldTransformer; import org.apache.metron.common.configuration.FieldValidator; import org.apache.metron.common.configuration.SensorParserConfig; import org.apache.metron.common.dsl.Context; import org.apache.metron.common.dsl.StellarFunctions; -import org.apache.metron.parsers.filters.Filters; -import org.apache.metron.common.configuration.FieldTransformer; +import org.apache.metron.common.error.MetronError; +import org.apache.metron.common.message.MessageGetStrategy; +import org.apache.metron.common.message.MessageGetters; import org.apache.metron.common.utils.ErrorUtils; +import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase; +import org.apache.metron.parsers.filters.Filters; import org.apache.metron.parsers.interfaces.MessageFilter; import org.apache.metron.parsers.interfaces.MessageParser; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; import org.json.simple.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.Serializable; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; public class ParserBolt extends ConfiguredParserBolt implements Serializable { @@ -118,7 +122,6 @@ protected void initializeStellar() { @Override public void execute(Tuple tuple) { byte[] originalMessage = (byte[]) messageGetStrategy.get(tuple); - //byte[] originalMessage = (byte[]) messageGetter.getMessage(tuple); SensorParserConfig sensorParserConfig = getSensorParserConfig(); try { //we want to ack the tuple in the situation where we have are not doing a bulk write diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bolt/ParserBoltTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bolt/ParserBoltTest.java index de98a64537..f11783813d 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bolt/ParserBoltTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bolt/ParserBoltTest.java @@ -17,9 +17,12 @@ */ package org.apache.metron.parsers.bolt; +import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.*; +import org.apache.metron.common.error.MetronError; import org.apache.metron.enrichment.adapters.geo.GeoLiteDatabase; +import org.apache.metron.test.error.MetronErrorJSONMatcher; import org.apache.metron.test.utils.UnitTestHelper; import org.apache.storm.task.OutputCollector; import org.apache.storm.tuple.Tuple; @@ -154,6 +157,64 @@ public Map getParserConfig() { verify(parser, times(0)).validate(any()); verify(writer, times(0)).write(eq(sensorType), any(ParserWriterConfiguration.class), eq(tuple), any()); verify(outputCollector, times(1)).ack(tuple); + + MetronError error = new MetronError() + .withErrorType(Constants.ErrorType.PARSER_ERROR) + .withThrowable(new NullPointerException()) + .withSensorType(sensorType) + .addRawMessage(sampleBinary); + verify(outputCollector, times(1)).emit(eq(Constants.ERROR_STREAM), argThat(new MetronErrorJSONMatcher(error.getJSONObject()))); + } + + @Test + public void testInvalid() throws Exception { + String sensorType = "yaf"; + ParserBolt parserBolt = new ParserBolt("zookeeperUrl", sensorType, parser, new WriterHandler(writer)) { + @Override + protected ParserConfigurations defaultConfigurations() { + return new ParserConfigurations() { + @Override + public SensorParserConfig getSensorParserConfig(String sensorType) { + return new SensorParserConfig() { + @Override + public Map getParserConfig() { + return new HashMap() {{ + }}; + } + + + }; + } + }; + } + + }; + + buildGlobalConfig(parserBolt); + + parserBolt.setCuratorFramework(client); + parserBolt.setTreeCache(cache); + parserBolt.prepare(new HashMap(), topologyContext, outputCollector); + byte[] sampleBinary = "some binary message".getBytes(); + + when(tuple.getBinary(0)).thenReturn(sampleBinary); + JSONObject parsedMessage = new JSONObject(); + parsedMessage.put("field", "invalidValue"); + List messageList = new ArrayList<>(); + messageList.add(parsedMessage); + when(parser.parseOptional(sampleBinary)).thenReturn(Optional.of(messageList)); + when(parser.validate(parsedMessage)).thenReturn(true); + parserBolt.execute(tuple); + + MetronError error = new MetronError() + .withErrorType(Constants.ErrorType.PARSER_INVALID) + .withSensorType(sensorType) + .withErrorFields(new HashSet() {{ add("field"); }}) + .addRawMessage(new JSONObject(){{ + put("field", "invalidValue"); + put("source.type", "yaf"); + }}); + verify(outputCollector, times(1)).emit(eq(Constants.ERROR_STREAM), argThat(new MetronErrorJSONMatcher(error.getJSONObject()))); } @Test @@ -555,6 +616,11 @@ protected void buildGlobalConfig(ParserBolt parserBolt) { String baseDir = UnitTestHelper.findDir("GeoLite"); File geoHdfsFile = new File(new File(baseDir), "GeoIP2-City-Test.mmdb.gz"); globalConfig.put(GeoLiteDatabase.GEO_HDFS_FILE, geoHdfsFile.getAbsolutePath()); + Map fieldValidation = new HashMap<>(); + fieldValidation.put("input", Arrays.asList("field")); + fieldValidation.put("validation", "STELLAR"); + fieldValidation.put("config", new HashMap(){{ put("condition", "field != 'invalidValue'"); }}); + globalConfig.put("fieldValidations", Arrays.asList(fieldValidation)); parserBolt.getConfigurations().updateGlobalConfig(globalConfig); } diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bolt/WriterBoltTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bolt/WriterBoltTest.java index 4693829b1c..4511b5598e 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bolt/WriterBoltTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bolt/WriterBoltTest.java @@ -17,22 +17,23 @@ */ package org.apache.metron.parsers.bolt; + import org.apache.log4j.Level; +import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.IndexingConfigurations; -import org.apache.metron.test.utils.UnitTestHelper; -import org.apache.metron.writer.BulkWriterComponent; -import org.apache.storm.task.OutputCollector; -import org.apache.storm.task.TopologyContext; -import org.apache.storm.tuple.Tuple; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import org.apache.metron.common.configuration.ParserConfigurations; import org.apache.metron.common.configuration.SensorParserConfig; -import org.apache.metron.common.configuration.writer.ParserWriterConfiguration; +import org.apache.metron.common.error.MetronError; import org.apache.metron.common.writer.BulkMessageWriter; -import org.apache.metron.common.writer.MessageWriter; import org.apache.metron.common.writer.BulkWriterResponse; +import org.apache.metron.common.writer.MessageWriter; import org.apache.metron.test.bolt.BaseBoltTest; +import org.apache.metron.test.error.MetronErrorJSONMatcher; +import org.apache.metron.test.utils.UnitTestHelper; +import org.apache.metron.writer.BulkWriterComponent; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.tuple.Tuple; import org.json.simple.JSONObject; import org.junit.Test; import org.mockito.Mock; @@ -44,7 +45,12 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class WriterBoltTest extends BaseBoltTest{ @Mock @@ -143,16 +149,24 @@ public void testNonBatchErrorPathErrorInWrite() throws Exception { ParserConfigurations configurations = getConfigurations(1); String sensorType = "test"; Tuple t = mock(Tuple.class); + when(t.toString()).thenReturn("tuple"); when(t.getValueByField(eq("message"))).thenReturn(new JSONObject()); WriterBolt bolt = new WriterBolt(new WriterHandler(writer), configurations, sensorType); bolt.prepare(new HashMap(), topologyContext, outputCollector); - doThrow(new Exception()).when(writer).write(any(), any(), any(), any()); + doThrow(new Exception("write error")).when(writer).write(any(), any(), any(), any()); verify(writer, times(1)).init(); bolt.execute(t); verify(outputCollector, times(1)).ack(t); verify(writer, times(1)).write(eq(sensorType), any(), any(), any()); verify(outputCollector, times(1)).reportError(any()); verify(outputCollector, times(0)).fail(any()); + + MetronError error = new MetronError() + .withErrorType(Constants.ErrorType.DEFAULT_ERROR) + .withThrowable(new IllegalStateException("Unhandled bulk errors in response: {java.lang.Exception: write error=[tuple]}")) + .withSensorType(sensorType) + .addRawMessage(new JSONObject()); + verify(outputCollector, times(1)).emit(eq(Constants.ERROR_STREAM), argThat(new MetronErrorJSONMatcher(error.getJSONObject()))); } @Test diff --git a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/MessageGetters.java b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/error/MetronErrorJSONMatcher.java similarity index 59% rename from metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/MessageGetters.java rename to metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/error/MetronErrorJSONMatcher.java index da4a5495e1..ad2428372f 100644 --- a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/MessageGetters.java +++ b/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/error/MetronErrorJSONMatcher.java @@ -15,23 +15,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.metron.test.error; -package org.apache.metron.writer.message; +import org.apache.storm.tuple.Values; +import org.json.simple.JSONObject; +import org.mockito.ArgumentMatcher; +public class MetronErrorJSONMatcher extends ArgumentMatcher { -import org.apache.storm.tuple.Tuple; -import org.json.simple.JSONObject; + private JSONObject expected; -public enum MessageGetters implements MessageGetter{ - RAW(RawMessageGetter.DEFAULT) - ,NAMED(NamedMessageGetter.DEFAULT) - ; - MessageGetter getter; - MessageGetters(MessageGetter getter) { - this.getter = getter; + public MetronErrorJSONMatcher(JSONObject expected) { + this.expected = expected; } + @Override - public JSONObject getMessage(Tuple t) { - return getter.getMessage(t); + public boolean matches(Object o) { + Values values = (Values) o; + JSONObject actual = (JSONObject) values.get(0); + actual.remove("timestamp"); + expected.remove("timestamp"); + actual.remove("stack"); + expected.remove("stack"); + return actual.equals(expected); } } diff --git a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/BulkWriterComponent.java b/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/BulkWriterComponent.java index c25300b652..0a9e5144b7 100644 --- a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/BulkWriterComponent.java +++ b/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/BulkWriterComponent.java @@ -18,25 +18,26 @@ package org.apache.metron.writer; -import org.apache.metron.common.error.MetronError; -import org.apache.metron.common.message.MessageGetStrategy; -import org.apache.metron.writer.message.MessageGetter; -import org.apache.metron.writer.message.MessageGetters; -import org.apache.storm.task.OutputCollector; -import org.apache.storm.tuple.Tuple; import com.google.common.collect.Iterables; import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.writer.WriterConfiguration; +import org.apache.metron.common.error.MetronError; +import org.apache.metron.common.message.MessageGetStrategy; +import org.apache.metron.common.utils.ErrorUtils; import org.apache.metron.common.writer.BulkMessageWriter; import org.apache.metron.common.writer.BulkWriterResponse; -import org.apache.metron.common.utils.ErrorUtils; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.tuple.Tuple; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; -import java.util.stream.Collector; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; public class BulkWriterComponent { public static final Logger LOG = LoggerFactory diff --git a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/bolt/BulkMessageWriterBolt.java b/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/bolt/BulkMessageWriterBolt.java index b0e61c77a3..2070f655b0 100644 --- a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/bolt/BulkMessageWriterBolt.java +++ b/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/bolt/BulkMessageWriterBolt.java @@ -18,6 +18,7 @@ package org.apache.metron.writer.bolt; import org.apache.metron.common.bolt.ConfiguredIndexingBolt; +import org.apache.metron.common.message.JSONFromField; import org.apache.metron.common.message.MessageGetStrategy; import org.apache.metron.common.message.MessageGetters; import org.apache.storm.task.OutputCollector; @@ -47,6 +48,7 @@ public class BulkMessageWriterBolt extends ConfiguredIndexingBolt { private BulkMessageWriter bulkMessageWriter; private BulkWriterComponent writerComponent; private String messageGetStrategyType = MessageGetters.JSON_FROM_FIELD.name(); + private String messageGetField; private transient MessageGetStrategy messageGetStrategy; private transient OutputCollector collector; private transient Function configurationTransformation; @@ -69,12 +71,21 @@ public BulkMessageWriterBolt withMessageGetter(String messageGetStrategyType) { return this; } + public BulkMessageWriterBolt withMessageGetterField(String messageGetField) { + this.messageGetField = messageGetField; + return this; + } + @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { this.writerComponent = new BulkWriterComponent<>(collector); this.collector = collector; super.prepare(stormConf, context, collector); - messageGetStrategy = MessageGetters.valueOf(messageGetStrategyType).get(); + if (messageGetStrategyType.equals(MessageGetters.JSON_FROM_FIELD.name()) && messageGetField != null) { + messageGetStrategy = new JSONFromField(messageGetField); + } else { + messageGetStrategy = MessageGetters.valueOf(messageGetStrategyType).get(); + } if(bulkMessageWriter instanceof WriterToBulkWriter) { configurationTransformation = WriterToBulkWriter.TRANSFORMATION; } diff --git a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/MessageGetter.java b/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/MessageGetter.java deleted file mode 100644 index 99c825fa4f..0000000000 --- a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/MessageGetter.java +++ /dev/null @@ -1,26 +0,0 @@ -/** - * 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.writer.message; - -import org.apache.storm.tuple.Tuple; -import org.json.simple.JSONObject; - -public interface MessageGetter { - JSONObject getMessage(Tuple t); -} diff --git a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/NamedMessageGetter.java b/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/NamedMessageGetter.java deleted file mode 100644 index fdd5fb8081..0000000000 --- a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/NamedMessageGetter.java +++ /dev/null @@ -1,34 +0,0 @@ -/** - * 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.writer.message; - -import org.apache.storm.tuple.Tuple; -import org.json.simple.JSONObject; - -public class NamedMessageGetter implements MessageGetter { - public static NamedMessageGetter DEFAULT = new NamedMessageGetter("message"); - private String messageName; - public NamedMessageGetter(String name) { - this.messageName = name; - } - @Override - public JSONObject getMessage(Tuple tuple) { - return (JSONObject)tuple.getValueByField(messageName); - } -} diff --git a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/RawMessageGetter.java b/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/RawMessageGetter.java deleted file mode 100644 index 99a837869b..0000000000 --- a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/message/RawMessageGetter.java +++ /dev/null @@ -1,50 +0,0 @@ -/** - * 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.writer.message; - -import org.apache.storm.tuple.Tuple; -import org.apache.metron.common.utils.JSONUtils; -import org.json.simple.JSONObject; -import org.json.simple.parser.JSONParser; -import org.json.simple.parser.ParseException; - -import java.io.UnsupportedEncodingException; - -public class RawMessageGetter implements MessageGetter { - public static RawMessageGetter DEFAULT = new RawMessageGetter(0); - private ThreadLocal parser = new ThreadLocal() { - @Override - protected JSONParser initialValue() { - return new JSONParser(); - } - }; - int position = 0; - public RawMessageGetter(int position) { - this.position = position; - } - @Override - public JSONObject getMessage(Tuple t) { - byte[] data = t.getBinary(position); - try { - return (JSONObject) parser.get().parse(new String(data, "UTF8")); - } catch (Exception e) { - throw new IllegalStateException(e.getMessage(), e); - } - } -} From 6b897d5afadde00870e2579af9c3c57d9ef9d076 Mon Sep 17 00:00:00 2001 From: rmerriman Date: Tue, 7 Feb 2017 18:22:47 -0600 Subject: [PATCH 03/16] Added error topology to Ambari MPack --- .../METRON/CURRENT/role_command_order.json | 1 + .../CURRENT/configuration/metron-env.xml | 57 +++++++ .../METRON/CURRENT/metainfo.xml | 55 +++++++ .../package/files/error_index.template | 52 ++++++ .../package/scripts/enrichment_commands.py | 6 +- .../CURRENT/package/scripts/error_commands.py | 154 ++++++++++++++++++ .../CURRENT/package/scripts/error_master.py | 100 ++++++++++++ .../CURRENT/package/scripts/metron_service.py | 4 + .../package/scripts/params/params_linux.py | 7 +- .../package/scripts/params/status_params.py | 6 +- .../package/scripts/parser_commands.py | 12 -- .../CURRENT/package/scripts/service_check.py | 5 +- .../METRON/CURRENT/service_advisor.py | 5 + .../docker/rpm-docker/SPECS/metron.spec | 4 + 14 files changed, 445 insertions(+), 23 deletions(-) create mode 100644 metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template create mode 100755 metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_commands.py create mode 100755 metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_master.py diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/addon-services/METRON/CURRENT/role_command_order.json b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/addon-services/METRON/CURRENT/role_command_order.json index e08f401c86..af1839c6b7 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/addon-services/METRON/CURRENT/role_command_order.json +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/addon-services/METRON/CURRENT/role_command_order.json @@ -8,6 +8,7 @@ "METRON_ENRICHMENT_MASTER-START" : ["NAMENODE-START", "ZOOKEEPER_SERVER-START", "KAFKA_BROKER-START", "STORM_REST_API-START", "HBASE_MASTER-START", "HBASE_REGIONSERVER-START"], "METRON_ENRICHMENT_SERVICE_CHECK-SERVICE_CHECK" : ["METRON_ENRICHMENT_MASTER-START"], "METRON_INDEXING-START" : ["NAMENODE-START", "ZOOKEEPER_SERVER-START", "KAFKA_BROKER-START", "STORM_REST_API-START","METRON_PARSERS-START"], + "METRON_ERROR-START" : ["NAMENODE-START", "ZOOKEEPER_SERVER-START", "KAFKA_BROKER-START", "STORM_REST_API-START","METRON_PARSERS-START"], "METRON_SERVICE_CHECK-SERVICE_CHECK" : ["METRON_PARSERS-START","METRON_INDEXING-START"] } } diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-env.xml b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-env.xml index eeb20372db..1e08350a44 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-env.xml +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-env.xml @@ -36,6 +36,12 @@ Indexing bolts will write to this HDFS directory Metron apps indexed HDFS dir + + metron_apps_error_hdfs_dir + {{metron_apps_hdfs_dir}}/error/indexed + Error indexing bolts will write to this HDFS directory + Metron apps error indexed HDFS dir + metron_zookeeper_config_dir config/zookeeper @@ -74,6 +80,12 @@ The Storm topology name for Indexing Indexing Topology Name + + metron_error_topology + error + The Storm topology name for Error + Error Topology Name + es_cluster_name metron @@ -203,4 +215,49 @@ index.hdfs.output={{ metron_apps_indexed_hdfs_dir }} + + elasticsearch-error-properties + The template for the elasticsearch_error.properties file. + elasticsearch_error.properties template + +##### Storm ##### +indexing.workers=1 +indexing.executors=0 +##### Kafka ##### +kafka.zk={{ zookeeper_quorum }} +kafka.broker={{ kafka_brokers }} +kafka.start=WHERE_I_LEFT_OFF +##### Indexing ##### +index.input.topic=error +writer.class.name=org.apache.metron.elasticsearch.writer.ElasticsearchWriter +##### Metrics ##### +#reporters +org.apache.metron.metrics.reporter.graphite=true +org.apache.metron.metrics.reporter.console=false +org.apache.metron.metrics.reporter.jmx=false +#Graphite Addresses +org.apache.metron.metrics.graphite.address=localhost +org.apache.metron.metrics.graphite.port=2023 +#TelemetryParserBolt +org.apache.metron.metrics.TelemetryParserBolt.acks=true +org.apache.metron.metrics.TelemetryParserBolt.emits=true +org.apache.metron.metrics.TelemetryParserBolt.fails=true +##### HDFS ##### +bolt.hdfs.batch.size=5000 +bolt.hdfs.field.delimiter=| +bolt.hdfs.rotation.policy=org.apache.storm.hdfs.bolt.rotation.TimedRotationPolicy +bolt.hdfs.rotation.policy.units=DAYS +bolt.hdfs.rotation.policy.count=1 +bolt.hdfs.file.rotation.size.in.mb=5 +bolt.hdfs.file.system.url={{ default_fs }} +bolt.hdfs.wip.file.path=/paloalto/wip +bolt.hdfs.finished.file.path=/paloalto/rotated +bolt.hdfs.compression.codec.class=org.apache.hadoop.io.compress.SnappyCodec +index.hdfs.output={{ metron_apps_error_hdfs_dir }} + + + content + + + diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/metainfo.xml b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/metainfo.xml index bbe71870aa..83bcbb9c23 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/metainfo.xml +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/metainfo.xml @@ -174,6 +174,61 @@ + + + METRON_ERROR + Metron Error + MASTER + 1 + false + false + + + + ZOOKEEPER/ZOOKEEPER_SERVER + cluster + + true + + + + STORM/SUPERVISOR + host + + true + + + + KAFKA/KAFKA_BROKER + host + + true + + + + + + PYTHON + + + + ELASTICSEARCH_TEMPLATE_INSTALL + + + PYTHON + 600 + + + + ELASTICSEARCH_TEMPLATE_DELETE + + + PYTHON + 600 + + + + diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template new file mode 100644 index 0000000000..9a6e4b71a2 --- /dev/null +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template @@ -0,0 +1,52 @@ +{ + "template": "error_index*", + "mappings": { + "error_doc": { + "_timestamp": { + "enabled": true + }, + "properties": { + "exception": { + "type": "string", + "index": "not_analyzed" + }, + "hostname": { + "type": "string", + "index": "not_analyzed" + }, + "stack": { + "type": "string", + "index": "not_analyzed" + }, + "time": { + "type": "date", + "format": "epoch_millis" + }, + "message": { + "type": "string", + "index": "not_analyzed" + }, + "raw_message": { + "type": "string", + "index": "not_analyzed" + }, + "raw_message_bytes": { + "type": "binary", + "index": "no" + }, + "raw_message_hash": { + "type": "string", + "index": "not_analyzed" + }, + "source_type": { + "type": "string", + "index": "not_analyzed" + }, + "error_type": { + "type": "string", + "index": "not_analyzed" + } + } + } + } +} \ No newline at end of file diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/enrichment_commands.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/enrichment_commands.py index bc73c87eb3..817f2668f2 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/enrichment_commands.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/enrichment_commands.py @@ -28,8 +28,6 @@ class EnrichmentCommands: __params = None __enrichment_topology = None __enrichment_topic = None - __enrichment_error_topic = None - __threat_intel_error_topic = None __configured = False def __init__(self, params): @@ -38,8 +36,6 @@ def __init__(self, params): self.__params = params self.__enrichment_topology = params.metron_enrichment_topology self.__enrichment_topic = params.metron_enrichment_topic - self.__enrichment_error_topic = params.metron_enrichment_error_topic - self.__threat_intel_error_topic = params.metron_threat_intel_error_topic self.__configured = os.path.isfile(self.__params.enrichment_configured_flag_file) def is_configured(self): @@ -121,7 +117,7 @@ def init_kafka_topics(self): retention_bytes = retention_gigabytes * 1024 * 1024 * 1024 Logger.info("Creating topics for enrichment") - topics = [self.__enrichment_topic, self.__enrichment_error_topic, self.__threat_intel_error_topic] + topics = [self.__enrichment_topic] for topic in topics: Logger.info("Creating topic'{0}'".format(topic)) Execute(command_template.format(self.__params.kafka_bin_dir, diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_commands.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_commands.py new file mode 100755 index 0000000000..f568063a72 --- /dev/null +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_commands.py @@ -0,0 +1,154 @@ +#!/usr/bin/env python +""" +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. +""" + +import os +import time + +from resource_management.core.logger import Logger +from resource_management.core.resources.system import Execute, File + +import metron_service + + +# Wrap major operations and functionality in this class +class ErrorCommands: + __params = None + __error = None + __configured = False + + def __init__(self, params): + if params is None: + raise ValueError("params argument is required for initialization") + self.__params = params + self.__error = params.metron_error_topology + self.__configured = os.path.isfile(self.__params.error_configured_flag_file) + + def is_configured(self): + return self.__configured + + def set_configured(self): + File(self.__params.error_configured_flag_file, + content="", + owner=self.__params.metron_user, + mode=0775) + + def setup_repo(self): + def local_repo(): + Logger.info("Setting up local repo") + Execute("yum -y install createrepo") + Execute("createrepo /localrepo") + Execute("chmod -R o-w+r /localrepo") + Execute("echo \"[METRON-${metron.version}]\n" + "name=Metron ${metron.version} packages\n" + "baseurl=file:///localrepo\n" + "gpgcheck=0\n" + "enabled=1\" > /etc/yum.repos.d/local.repo") + + def remote_repo(): + print('Using remote repo') + + yum_repo_types = { + 'local': local_repo, + 'remote': remote_repo + } + repo_type = self.__params.yum_repo_type + if repo_type in yum_repo_types: + yum_repo_types[repo_type]() + else: + raise ValueError("Unsupported repo type '{0}'".format(repo_type)) + + def init_kafka_topics(self): + Logger.info('Creating Kafka topics') + command_template = """{0}/kafka-topics.sh \ + --zookeeper {1} \ + --create \ + --topic {2} \ + --partitions {3} \ + --replication-factor {4} \ + --config retention.bytes={5}""" + num_partitions = 1 + replication_factor = 1 + retention_gigabytes = int(self.__params.metron_topic_retention) + retention_bytes = retention_gigabytes * 1024 * 1024 * 1024 + Logger.info("Creating topics for error") + + Logger.info("Creating topic'{0}'".format(self.__error)) + Execute(command_template.format(self.__params.kafka_bin_dir, + self.__params.zookeeper_quorum, + self.__error, + num_partitions, + replication_factor, + retention_bytes)) + Logger.info("Done creating Kafka topics") + + def init_hdfs_dir(self): + Logger.info('Creating HDFS error directory') + self.__params.HdfsResource(self.__params.metron_apps_error_hdfs_dir, + type="directory", + action="create_on_execute", + owner=self.__params.metron_user, + group=self.__params.user_group, + mode=0775, + ) + Logger.info('Done creating HDFS error directory') + + + def start_error_topology(self): + Logger.info("Starting Metron error topology: {0}".format(self.__error)) + start_cmd_template = """{0}/bin/start_elasticsearch_error_topology.sh \ + -s {1} \ + -z {2}""" + Logger.info('Starting ' + self.__error) + Execute(start_cmd_template.format(self.__params.metron_home, self.__error, self.__params.zookeeper_quorum)) + + Logger.info('Finished starting error topology') + + def stop_error_topology(self): + Logger.info('Stopping ' + self.__error) + stop_cmd = 'storm kill ' + self.__error + Execute(stop_cmd) + Logger.info('Done stopping error topology') + + def restart_error_topology(self, env): + Logger.info('Restarting the error topology') + self.stop_error_topology() + + # Wait for old topology to be cleaned up by Storm, before starting again. + retries = 0 + topology_active = self.is_topology_active(env) + while self.is_topology_active(env) and retries < 3: + Logger.info('Existing topology still active. Will wait and retry') + time.sleep(10) + retries += 1 + + if not topology_active: + Logger.info('Waiting for storm kill to complete') + time.sleep(30) + self.start_error_topology() + Logger.info('Done restarting the error topology') + else: + Logger.warning('Retries exhausted. Existing topology not cleaned up. Aborting topology start.') + + def is_topology_active(self, env): + env.set_params(self.__params) + active = True + topologies = metron_service.get_running_topologies() + is_running = False + if self.__error in topologies: + is_running = topologies[self.__error] in ['ACTIVE', 'REBALANCING'] + active &= is_running + return active diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_master.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_master.py new file mode 100755 index 0000000000..f49335f34f --- /dev/null +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_master.py @@ -0,0 +1,100 @@ +""" +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. +""" + +import os +from resource_management.core.exceptions import ComponentIsNotRunning +from resource_management.core.logger import Logger +from resource_management.core.resources.system import Execute +from resource_management.core.resources.system import File +from resource_management.core.source import StaticFile +from resource_management.libraries.functions import format as ambari_format +from resource_management.libraries.script import Script + +import metron_service +from error_commands import ErrorCommands + + +class Error(Script): + __configured = False + + def install(self, env): + from params import params + env.set_params(params) + commands = ErrorCommands(params) + commands.setup_repo() + self.install_packages(env) + + def configure(self, env, upgrade_type=None, config_dir=None): + from params import params + env.set_params(params) + + commands = ErrorCommands(params) + metron_service.load_global_config(params) + + if not commands.is_configured(): + commands.init_kafka_topics() + commands.init_hdfs_dir() + commands.set_configured() + + def start(self, env, upgrade_type=None): + from params import params + env.set_params(params) + self.configure(env) + commands = ErrorCommands(params) + commands.start_error_topology() + + def stop(self, env, upgrade_type=None): + from params import params + env.set_params(params) + commands = ErrorCommands(params) + commands.stop_error_topology() + + def status(self, env): + from params import status_params + env.set_params(status_params) + commands = ErrorCommands(status_params) + if not commands.is_topology_active(env): + raise ComponentIsNotRunning() + + def restart(self, env): + from params import params + env.set_params(params) + self.configure(env) + commands = ErrorCommands(params) + commands.restart_error_topology(env) + + def elasticsearch_error_template_install(self, env): + from params import params + env.set_params(params) + + File(params.error_index_path, + mode=0755, + content=StaticFile('error_index.template') + ) + + error_cmd = ambari_format( + 'curl -s -XPOST http://{es_http_url}/_template/error_index -d @{error_index_path}') + Execute(error_cmd, logoutput=True) + + def elasticsearch_template_delete(self, env): + from params import params + env.set_params(params) + + error_cmd = ambari_format('curl -s -XDELETE "http://{es_http_url}/error_index*"') + Execute(error_cmd, logoutput=True) + +if __name__ == "__main__": + Error().execute() diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/metron_service.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/metron_service.py index 57da2c7c64..39b2e2fd36 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/metron_service.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/metron_service.py @@ -73,4 +73,8 @@ def load_global_config(params): owner=params.metron_user, content=InlineTemplate(params.global_properties_template)) + File("{0}/elasticsearch_error.properties".format(params.metron_zookeeper_config_path + '/..'), + owner=params.metron_user, + content=InlineTemplate(params.error_properties_template)) + init_config() diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py index 2427d255f9..9dad13b1a5 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py @@ -45,6 +45,7 @@ geoip_url = config['configurations']['metron-env']['geoip_url'] geoip_hdfs_dir = "/apps/metron/geo/default/" metron_indexing_topology = status_params.metron_indexing_topology +metron_error_topology = status_params.metron_error_topology metron_user = config['configurations']['metron-env']['metron_user'] metron_group = config['configurations']['metron-env']['metron_group'] metron_config_path = metron_home + '/config' @@ -53,8 +54,10 @@ parsers_configured_flag_file = status_params.parsers_configured_flag_file enrichment_configured_flag_file = status_params.enrichment_configured_flag_file indexing_configured_flag_file = status_params.indexing_configured_flag_file +error_configured_flag_file = status_params.error_configured_flag_file global_json_template = config['configurations']['metron-env']['global-json'] global_properties_template = config['configurations']['metron-env']['elasticsearch-properties'] +error_properties_template = config['configurations']['metron-env']['elasticsearch-error-properties'] # Elasticsearch hosts and port management es_cluster_name = config['configurations']['metron-env']['es_cluster_name'] @@ -116,6 +119,7 @@ # the double "format" is not an error - we are pulling in a jinja-templated param. This is a bit of a hack, but works # well enough until we find a better way via Ambari metron_apps_indexed_hdfs_dir = format(format(config['configurations']['metron-env']['metron_apps_indexed_hdfs_dir'])) +metron_apps_error_hdfs_dir = format(format(config['configurations']['metron-env']['metron_apps_error_hdfs_dir'])) metron_topic_retention = config['configurations']['metron-env']['metron_topic_retention'] local_grok_patterns_dir = format("{metron_home}/patterns") @@ -158,13 +162,12 @@ metron_enrichment_topology = status_params.metron_enrichment_topology metron_enrichment_topic = status_params.metron_enrichment_topic -metron_enrichment_error_topic = status_params.metron_enrichment_error_topic -metron_threat_intel_error_topic = status_params.metron_threat_intel_error_topic # ES Templates bro_index_path = tmp_dir + "/bro_index.template" snort_index_path = tmp_dir + "/snort_index.template" yaf_index_path = tmp_dir + "/yaf_index.template" +error_index_path = tmp_dir + "/error_index.template" # Zeppelin Notebooks metron_config_zeppelin_path = format("{metron_config_path}/zeppelin") diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py index e8a8568e4c..5fe7eb9ee1 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py @@ -34,8 +34,6 @@ # Enrichment metron_enrichment_topology = 'enrichment' metron_enrichment_topic = 'enrichments' -metron_enrichment_error_topic = 'enrichments_error' -metron_threat_intel_error_topic = 'threatintel_error' enrichment_table = 'enrichment' enrichment_cf = 't' @@ -46,6 +44,10 @@ metron_indexing_topology = config['configurations']['metron-env']['metron_indexing_topology'] indexing_configured_flag_file = metron_zookeeper_config_path + '/../metron_indexing_configured' +# Error +metron_error_topology = config['configurations']['metron-env']['metron_error_topology'] +error_configured_flag_file = metron_zookeeper_config_path + '/../metron_error_configured' + # Enrichment enrichment_configured_flag_file = metron_zookeeper_config_path + '/../metron_enrichment_configured' diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/parser_commands.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/parser_commands.py index e6f0f3a704..574d7e81a9 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/parser_commands.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/parser_commands.py @@ -118,18 +118,6 @@ def init_kafka_topics(self): num_partitions, replication_factor, retention_bytes)) - Logger.info("Creating topics for error handling") - Execute(command_template.format(self.__params.kafka_bin_dir, - self.__params.zookeeper_quorum, - "parser_invalid", - num_partitions, - replication_factor, - retention_bytes)) - Execute(command_template.format(self.__params.kafka_bin_dir, - self.__params.zookeeper_quorum, - "parser_error", - num_partitions, replication_factor, - retention_bytes)) Logger.info("Done creating Kafka topics") def start_parser_topologies(self): diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/service_check.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/service_check.py index 7dd9dfb8fb..2d25f7b2e9 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/service_check.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/service_check.py @@ -23,14 +23,15 @@ from indexing_commands import IndexingCommands from parser_commands import ParserCommands - +from error_commands import ErrorCommands class ServiceCheck(Script): def service_check(self, env): from params import params parsercommands = ParserCommands(params) indexingcommands = IndexingCommands(params) - all_found = parsercommands.topologies_running(env) and indexingcommands.is_topology_active(env) + errorcommands = ErrorCommands(params) + all_found = parsercommands.topologies_running(env) and indexingcommands.is_topology_active(env) and errorcommands.is_topology_active(env) if all_found: exit(0) else: diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/service_advisor.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/service_advisor.py index 8aff88c6ea..99c0a7979d 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/service_advisor.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/service_advisor.py @@ -44,6 +44,7 @@ def getServiceComponentLayoutValidations(self, services, hosts): metronParsersHost = self.getHosts(componentsList, "METRON_PARSERS")[0] metronEnrichmentMaster = self.getHosts(componentsList, "METRON_ENRICHMENT_MASTER")[0] metronIndexingHost = self.getHosts(componentsList, "METRON_INDEXING")[0] + metronErrorHost = self.getHosts(componentsList, "METRON_ERROR")[0] hbaseClientHosts = self.getHosts(componentsList, "HBASE_CLIENT") hdfsClientHosts = self.getHosts(componentsList, "HDFS_CLIENT") @@ -71,6 +72,10 @@ def getServiceComponentLayoutValidations(self, services, hosts): message = "Metron Indexing must be co-located with Metron Parsers on {0}".format(metronParsersHost) items.append({ "type": 'host-component', "level": 'ERROR', "message": message, "component-name": 'METRON_INDEXING', "host": metronIndexingHost }) + if metronParsersHost != metronErrorHost: + message = "Metron Error must be co-located with Metron Parsers on {0}".format(metronParsersHost) + items.append({ "type": 'host-component', "level": 'ERROR', "message": message, "component-name": 'METRON_ERROR', "host": metronErrorHost }) + # Enrichment Master also needs ZK Client, but this is already guaranteed by being colocated with Parsers Master if metronParsersHost not in zookeeperClientHosts: message = "Metron must be co-located with an instance of Zookeeper Client" diff --git a/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec b/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec index 9466b68446..6c3c099a9a 100644 --- a/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec +++ b/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec @@ -157,7 +157,9 @@ This package installs the Metron Elasticsearch files %dir %{metron_home}/config %dir %{metron_home}/lib %{metron_home}/bin/start_elasticsearch_topology.sh +%{metron_home}/bin/start_elasticsearch_error_topology.sh %{metron_home}/config/elasticsearch.properties +%{metron_home}/config/elasticsearch_error.properties %attr(0644,root,root) %{metron_home}/lib/metron-elasticsearch-%{full_version}-uber.jar # ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ @@ -255,11 +257,13 @@ This package installs the Metron Indexing files %dir %{metron_home}/flux %dir %{metron_home}/flux/indexing %{metron_home}/flux/indexing/remote.yaml +%{metron_home}/flux/error/remote.yaml %{metron_home}/config/zookeeper/indexing/bro.json %{metron_home}/config/zookeeper/indexing/snort.json %{metron_home}/config/zookeeper/indexing/websphere.json %{metron_home}/config/zookeeper/indexing/yaf.json %{metron_home}/config/zookeeper/indexing/asa.json +%{metron_home}/config/zookeeper/indexing/error.json %{metron_home}/config/zeppelin/metron/metron-yaf-telemetry.json # ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ From f0141b2320103c4af24d04a20504041702be8aeb Mon Sep 17 00:00:00 2001 From: rmerriman Date: Tue, 28 Feb 2017 17:37:38 -0600 Subject: [PATCH 04/16] Changes from latests feedback. Made error topic configurable. --- .../METRON/CURRENT/role_command_order.json | 1 - .../CURRENT/configuration/metron-env.xml | 60 +------ .../METRON/CURRENT/metainfo.xml | 55 ------- .../package/files/error_index.template | 52 ------ .../package/scripts/enrichment_commands.py | 6 +- .../CURRENT/package/scripts/error_commands.py | 154 ------------------ .../CURRENT/package/scripts/error_master.py | 100 ------------ .../CURRENT/package/scripts/metron_service.py | 4 - .../package/scripts/params/params_linux.py | 7 +- .../package/scripts/params/status_params.py | 6 +- .../CURRENT/package/scripts/service_check.py | 5 +- .../METRON/CURRENT/service_advisor.py | 5 - .../docker/rpm-docker/SPECS/metron.spec | 3 - .../config/elasticsearch.global.json | 3 +- .../org/apache/metron/common/Constants.java | 1 - .../metron/common/error/MetronError.java | 87 ++++++---- .../metron/common/message/MessageGetters.java | 24 ++- ...lueFromField.java => ObjectFromField.java} | 6 +- .../apache/metron/common/utils/HashUtils.java | 44 +++++ .../metron/common/error/MetronErrorTest.java | 14 +- .../metron/common/utils/HashUtilsTest.java | 55 +++++++ .../src/main/config/elasticsearch.properties | 2 +- .../config/elasticsearch_error.properties | 69 -------- .../ElasticsearchIndexingIntegrationTest.java | 5 +- .../src/main/config/enrichment.properties | 4 +- .../metron/enrichment/bolt/JoinBolt.java | 8 +- .../bolt/BulkMessageWriterBoltTest.java | 4 +- .../EnrichmentIntegrationTest.java | 13 +- .../src/main/flux/error/remote.yaml | 122 -------------- .../src/main/flux/indexing/remote.yaml | 4 +- .../integration/IndexingIntegrationTest.java | 5 +- .../metron/parsers/bolt/ParserBolt.java | 2 +- .../metron/parsers/bolt/WriterBolt.java | 10 +- .../topology/ParserTopologyBuilder.java | 2 +- .../integration/ParserIntegrationTest.java | 5 +- .../WriterBoltIntegrationTest.java | 28 ++-- .../SolrIndexingIntegrationTest.java | 9 +- .../writer/bolt/BulkMessageWriterBolt.java | 25 ++- 38 files changed, 264 insertions(+), 745 deletions(-) delete mode 100644 metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template delete mode 100755 metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_commands.py delete mode 100755 metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_master.py rename metron-platform/metron-common/src/main/java/org/apache/metron/common/message/{ValueFromField.java => ObjectFromField.java} (88%) create mode 100644 metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HashUtils.java create mode 100644 metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/HashUtilsTest.java delete mode 100644 metron-platform/metron-elasticsearch/src/main/config/elasticsearch_error.properties delete mode 100644 metron-platform/metron-indexing/src/main/flux/error/remote.yaml diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/addon-services/METRON/CURRENT/role_command_order.json b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/addon-services/METRON/CURRENT/role_command_order.json index af1839c6b7..e08f401c86 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/addon-services/METRON/CURRENT/role_command_order.json +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/addon-services/METRON/CURRENT/role_command_order.json @@ -8,7 +8,6 @@ "METRON_ENRICHMENT_MASTER-START" : ["NAMENODE-START", "ZOOKEEPER_SERVER-START", "KAFKA_BROKER-START", "STORM_REST_API-START", "HBASE_MASTER-START", "HBASE_REGIONSERVER-START"], "METRON_ENRICHMENT_SERVICE_CHECK-SERVICE_CHECK" : ["METRON_ENRICHMENT_MASTER-START"], "METRON_INDEXING-START" : ["NAMENODE-START", "ZOOKEEPER_SERVER-START", "KAFKA_BROKER-START", "STORM_REST_API-START","METRON_PARSERS-START"], - "METRON_ERROR-START" : ["NAMENODE-START", "ZOOKEEPER_SERVER-START", "KAFKA_BROKER-START", "STORM_REST_API-START","METRON_PARSERS-START"], "METRON_SERVICE_CHECK-SERVICE_CHECK" : ["METRON_PARSERS-START","METRON_INDEXING-START"] } } diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-env.xml b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-env.xml index 1e08350a44..b87bb360e7 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-env.xml +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-env.xml @@ -36,12 +36,6 @@ Indexing bolts will write to this HDFS directory Metron apps indexed HDFS dir - - metron_apps_error_hdfs_dir - {{metron_apps_hdfs_dir}}/error/indexed - Error indexing bolts will write to this HDFS directory - Metron apps error indexed HDFS dir - metron_zookeeper_config_dir config/zookeeper @@ -80,12 +74,6 @@ The Storm topology name for Indexing Indexing Topology Name - - metron_error_topology - error - The Storm topology name for Error - Error Topology Name - es_cluster_name metron @@ -162,7 +150,8 @@ { "es.clustername": "{{ es_cluster_name }}", "es.ip": "{{ es_url }}", -"es.date.format": "yyyy.MM.dd.HH" +"es.date.format": "yyyy.MM.dd.HH", +"parser.error.topic": "indexing" } @@ -215,49 +204,4 @@ index.hdfs.output={{ metron_apps_indexed_hdfs_dir }} - - elasticsearch-error-properties - The template for the elasticsearch_error.properties file. - elasticsearch_error.properties template - -##### Storm ##### -indexing.workers=1 -indexing.executors=0 -##### Kafka ##### -kafka.zk={{ zookeeper_quorum }} -kafka.broker={{ kafka_brokers }} -kafka.start=WHERE_I_LEFT_OFF -##### Indexing ##### -index.input.topic=error -writer.class.name=org.apache.metron.elasticsearch.writer.ElasticsearchWriter -##### Metrics ##### -#reporters -org.apache.metron.metrics.reporter.graphite=true -org.apache.metron.metrics.reporter.console=false -org.apache.metron.metrics.reporter.jmx=false -#Graphite Addresses -org.apache.metron.metrics.graphite.address=localhost -org.apache.metron.metrics.graphite.port=2023 -#TelemetryParserBolt -org.apache.metron.metrics.TelemetryParserBolt.acks=true -org.apache.metron.metrics.TelemetryParserBolt.emits=true -org.apache.metron.metrics.TelemetryParserBolt.fails=true -##### HDFS ##### -bolt.hdfs.batch.size=5000 -bolt.hdfs.field.delimiter=| -bolt.hdfs.rotation.policy=org.apache.storm.hdfs.bolt.rotation.TimedRotationPolicy -bolt.hdfs.rotation.policy.units=DAYS -bolt.hdfs.rotation.policy.count=1 -bolt.hdfs.file.rotation.size.in.mb=5 -bolt.hdfs.file.system.url={{ default_fs }} -bolt.hdfs.wip.file.path=/paloalto/wip -bolt.hdfs.finished.file.path=/paloalto/rotated -bolt.hdfs.compression.codec.class=org.apache.hadoop.io.compress.SnappyCodec -index.hdfs.output={{ metron_apps_error_hdfs_dir }} - - - content - - - diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/metainfo.xml b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/metainfo.xml index 83bcbb9c23..bbe71870aa 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/metainfo.xml +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/metainfo.xml @@ -174,61 +174,6 @@ - - - METRON_ERROR - Metron Error - MASTER - 1 - false - false - - - - ZOOKEEPER/ZOOKEEPER_SERVER - cluster - - true - - - - STORM/SUPERVISOR - host - - true - - - - KAFKA/KAFKA_BROKER - host - - true - - - - - - PYTHON - - - - ELASTICSEARCH_TEMPLATE_INSTALL - - - PYTHON - 600 - - - - ELASTICSEARCH_TEMPLATE_DELETE - - - PYTHON - 600 - - - - diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template deleted file mode 100644 index 9a6e4b71a2..0000000000 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template +++ /dev/null @@ -1,52 +0,0 @@ -{ - "template": "error_index*", - "mappings": { - "error_doc": { - "_timestamp": { - "enabled": true - }, - "properties": { - "exception": { - "type": "string", - "index": "not_analyzed" - }, - "hostname": { - "type": "string", - "index": "not_analyzed" - }, - "stack": { - "type": "string", - "index": "not_analyzed" - }, - "time": { - "type": "date", - "format": "epoch_millis" - }, - "message": { - "type": "string", - "index": "not_analyzed" - }, - "raw_message": { - "type": "string", - "index": "not_analyzed" - }, - "raw_message_bytes": { - "type": "binary", - "index": "no" - }, - "raw_message_hash": { - "type": "string", - "index": "not_analyzed" - }, - "source_type": { - "type": "string", - "index": "not_analyzed" - }, - "error_type": { - "type": "string", - "index": "not_analyzed" - } - } - } - } -} \ No newline at end of file diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/enrichment_commands.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/enrichment_commands.py index 817f2668f2..bc73c87eb3 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/enrichment_commands.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/enrichment_commands.py @@ -28,6 +28,8 @@ class EnrichmentCommands: __params = None __enrichment_topology = None __enrichment_topic = None + __enrichment_error_topic = None + __threat_intel_error_topic = None __configured = False def __init__(self, params): @@ -36,6 +38,8 @@ def __init__(self, params): self.__params = params self.__enrichment_topology = params.metron_enrichment_topology self.__enrichment_topic = params.metron_enrichment_topic + self.__enrichment_error_topic = params.metron_enrichment_error_topic + self.__threat_intel_error_topic = params.metron_threat_intel_error_topic self.__configured = os.path.isfile(self.__params.enrichment_configured_flag_file) def is_configured(self): @@ -117,7 +121,7 @@ def init_kafka_topics(self): retention_bytes = retention_gigabytes * 1024 * 1024 * 1024 Logger.info("Creating topics for enrichment") - topics = [self.__enrichment_topic] + topics = [self.__enrichment_topic, self.__enrichment_error_topic, self.__threat_intel_error_topic] for topic in topics: Logger.info("Creating topic'{0}'".format(topic)) Execute(command_template.format(self.__params.kafka_bin_dir, diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_commands.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_commands.py deleted file mode 100755 index f568063a72..0000000000 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_commands.py +++ /dev/null @@ -1,154 +0,0 @@ -#!/usr/bin/env python -""" -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. -""" - -import os -import time - -from resource_management.core.logger import Logger -from resource_management.core.resources.system import Execute, File - -import metron_service - - -# Wrap major operations and functionality in this class -class ErrorCommands: - __params = None - __error = None - __configured = False - - def __init__(self, params): - if params is None: - raise ValueError("params argument is required for initialization") - self.__params = params - self.__error = params.metron_error_topology - self.__configured = os.path.isfile(self.__params.error_configured_flag_file) - - def is_configured(self): - return self.__configured - - def set_configured(self): - File(self.__params.error_configured_flag_file, - content="", - owner=self.__params.metron_user, - mode=0775) - - def setup_repo(self): - def local_repo(): - Logger.info("Setting up local repo") - Execute("yum -y install createrepo") - Execute("createrepo /localrepo") - Execute("chmod -R o-w+r /localrepo") - Execute("echo \"[METRON-${metron.version}]\n" - "name=Metron ${metron.version} packages\n" - "baseurl=file:///localrepo\n" - "gpgcheck=0\n" - "enabled=1\" > /etc/yum.repos.d/local.repo") - - def remote_repo(): - print('Using remote repo') - - yum_repo_types = { - 'local': local_repo, - 'remote': remote_repo - } - repo_type = self.__params.yum_repo_type - if repo_type in yum_repo_types: - yum_repo_types[repo_type]() - else: - raise ValueError("Unsupported repo type '{0}'".format(repo_type)) - - def init_kafka_topics(self): - Logger.info('Creating Kafka topics') - command_template = """{0}/kafka-topics.sh \ - --zookeeper {1} \ - --create \ - --topic {2} \ - --partitions {3} \ - --replication-factor {4} \ - --config retention.bytes={5}""" - num_partitions = 1 - replication_factor = 1 - retention_gigabytes = int(self.__params.metron_topic_retention) - retention_bytes = retention_gigabytes * 1024 * 1024 * 1024 - Logger.info("Creating topics for error") - - Logger.info("Creating topic'{0}'".format(self.__error)) - Execute(command_template.format(self.__params.kafka_bin_dir, - self.__params.zookeeper_quorum, - self.__error, - num_partitions, - replication_factor, - retention_bytes)) - Logger.info("Done creating Kafka topics") - - def init_hdfs_dir(self): - Logger.info('Creating HDFS error directory') - self.__params.HdfsResource(self.__params.metron_apps_error_hdfs_dir, - type="directory", - action="create_on_execute", - owner=self.__params.metron_user, - group=self.__params.user_group, - mode=0775, - ) - Logger.info('Done creating HDFS error directory') - - - def start_error_topology(self): - Logger.info("Starting Metron error topology: {0}".format(self.__error)) - start_cmd_template = """{0}/bin/start_elasticsearch_error_topology.sh \ - -s {1} \ - -z {2}""" - Logger.info('Starting ' + self.__error) - Execute(start_cmd_template.format(self.__params.metron_home, self.__error, self.__params.zookeeper_quorum)) - - Logger.info('Finished starting error topology') - - def stop_error_topology(self): - Logger.info('Stopping ' + self.__error) - stop_cmd = 'storm kill ' + self.__error - Execute(stop_cmd) - Logger.info('Done stopping error topology') - - def restart_error_topology(self, env): - Logger.info('Restarting the error topology') - self.stop_error_topology() - - # Wait for old topology to be cleaned up by Storm, before starting again. - retries = 0 - topology_active = self.is_topology_active(env) - while self.is_topology_active(env) and retries < 3: - Logger.info('Existing topology still active. Will wait and retry') - time.sleep(10) - retries += 1 - - if not topology_active: - Logger.info('Waiting for storm kill to complete') - time.sleep(30) - self.start_error_topology() - Logger.info('Done restarting the error topology') - else: - Logger.warning('Retries exhausted. Existing topology not cleaned up. Aborting topology start.') - - def is_topology_active(self, env): - env.set_params(self.__params) - active = True - topologies = metron_service.get_running_topologies() - is_running = False - if self.__error in topologies: - is_running = topologies[self.__error] in ['ACTIVE', 'REBALANCING'] - active &= is_running - return active diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_master.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_master.py deleted file mode 100755 index f49335f34f..0000000000 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/error_master.py +++ /dev/null @@ -1,100 +0,0 @@ -""" -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. -""" - -import os -from resource_management.core.exceptions import ComponentIsNotRunning -from resource_management.core.logger import Logger -from resource_management.core.resources.system import Execute -from resource_management.core.resources.system import File -from resource_management.core.source import StaticFile -from resource_management.libraries.functions import format as ambari_format -from resource_management.libraries.script import Script - -import metron_service -from error_commands import ErrorCommands - - -class Error(Script): - __configured = False - - def install(self, env): - from params import params - env.set_params(params) - commands = ErrorCommands(params) - commands.setup_repo() - self.install_packages(env) - - def configure(self, env, upgrade_type=None, config_dir=None): - from params import params - env.set_params(params) - - commands = ErrorCommands(params) - metron_service.load_global_config(params) - - if not commands.is_configured(): - commands.init_kafka_topics() - commands.init_hdfs_dir() - commands.set_configured() - - def start(self, env, upgrade_type=None): - from params import params - env.set_params(params) - self.configure(env) - commands = ErrorCommands(params) - commands.start_error_topology() - - def stop(self, env, upgrade_type=None): - from params import params - env.set_params(params) - commands = ErrorCommands(params) - commands.stop_error_topology() - - def status(self, env): - from params import status_params - env.set_params(status_params) - commands = ErrorCommands(status_params) - if not commands.is_topology_active(env): - raise ComponentIsNotRunning() - - def restart(self, env): - from params import params - env.set_params(params) - self.configure(env) - commands = ErrorCommands(params) - commands.restart_error_topology(env) - - def elasticsearch_error_template_install(self, env): - from params import params - env.set_params(params) - - File(params.error_index_path, - mode=0755, - content=StaticFile('error_index.template') - ) - - error_cmd = ambari_format( - 'curl -s -XPOST http://{es_http_url}/_template/error_index -d @{error_index_path}') - Execute(error_cmd, logoutput=True) - - def elasticsearch_template_delete(self, env): - from params import params - env.set_params(params) - - error_cmd = ambari_format('curl -s -XDELETE "http://{es_http_url}/error_index*"') - Execute(error_cmd, logoutput=True) - -if __name__ == "__main__": - Error().execute() diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/metron_service.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/metron_service.py index 39b2e2fd36..57da2c7c64 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/metron_service.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/metron_service.py @@ -73,8 +73,4 @@ def load_global_config(params): owner=params.metron_user, content=InlineTemplate(params.global_properties_template)) - File("{0}/elasticsearch_error.properties".format(params.metron_zookeeper_config_path + '/..'), - owner=params.metron_user, - content=InlineTemplate(params.error_properties_template)) - init_config() diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py index 9dad13b1a5..2427d255f9 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py @@ -45,7 +45,6 @@ geoip_url = config['configurations']['metron-env']['geoip_url'] geoip_hdfs_dir = "/apps/metron/geo/default/" metron_indexing_topology = status_params.metron_indexing_topology -metron_error_topology = status_params.metron_error_topology metron_user = config['configurations']['metron-env']['metron_user'] metron_group = config['configurations']['metron-env']['metron_group'] metron_config_path = metron_home + '/config' @@ -54,10 +53,8 @@ parsers_configured_flag_file = status_params.parsers_configured_flag_file enrichment_configured_flag_file = status_params.enrichment_configured_flag_file indexing_configured_flag_file = status_params.indexing_configured_flag_file -error_configured_flag_file = status_params.error_configured_flag_file global_json_template = config['configurations']['metron-env']['global-json'] global_properties_template = config['configurations']['metron-env']['elasticsearch-properties'] -error_properties_template = config['configurations']['metron-env']['elasticsearch-error-properties'] # Elasticsearch hosts and port management es_cluster_name = config['configurations']['metron-env']['es_cluster_name'] @@ -119,7 +116,6 @@ # the double "format" is not an error - we are pulling in a jinja-templated param. This is a bit of a hack, but works # well enough until we find a better way via Ambari metron_apps_indexed_hdfs_dir = format(format(config['configurations']['metron-env']['metron_apps_indexed_hdfs_dir'])) -metron_apps_error_hdfs_dir = format(format(config['configurations']['metron-env']['metron_apps_error_hdfs_dir'])) metron_topic_retention = config['configurations']['metron-env']['metron_topic_retention'] local_grok_patterns_dir = format("{metron_home}/patterns") @@ -162,12 +158,13 @@ metron_enrichment_topology = status_params.metron_enrichment_topology metron_enrichment_topic = status_params.metron_enrichment_topic +metron_enrichment_error_topic = status_params.metron_enrichment_error_topic +metron_threat_intel_error_topic = status_params.metron_threat_intel_error_topic # ES Templates bro_index_path = tmp_dir + "/bro_index.template" snort_index_path = tmp_dir + "/snort_index.template" yaf_index_path = tmp_dir + "/yaf_index.template" -error_index_path = tmp_dir + "/error_index.template" # Zeppelin Notebooks metron_config_zeppelin_path = format("{metron_config_path}/zeppelin") diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py index 5fe7eb9ee1..2151610072 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py @@ -34,6 +34,8 @@ # Enrichment metron_enrichment_topology = 'enrichment' metron_enrichment_topic = 'enrichments' +metron_enrichment_error_topic = 'indexing' +metron_threat_intel_error_topic = 'indexing' enrichment_table = 'enrichment' enrichment_cf = 't' @@ -44,10 +46,6 @@ metron_indexing_topology = config['configurations']['metron-env']['metron_indexing_topology'] indexing_configured_flag_file = metron_zookeeper_config_path + '/../metron_indexing_configured' -# Error -metron_error_topology = config['configurations']['metron-env']['metron_error_topology'] -error_configured_flag_file = metron_zookeeper_config_path + '/../metron_error_configured' - # Enrichment enrichment_configured_flag_file = metron_zookeeper_config_path + '/../metron_enrichment_configured' diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/service_check.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/service_check.py index 2d25f7b2e9..7dd9dfb8fb 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/service_check.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/service_check.py @@ -23,15 +23,14 @@ from indexing_commands import IndexingCommands from parser_commands import ParserCommands -from error_commands import ErrorCommands + class ServiceCheck(Script): def service_check(self, env): from params import params parsercommands = ParserCommands(params) indexingcommands = IndexingCommands(params) - errorcommands = ErrorCommands(params) - all_found = parsercommands.topologies_running(env) and indexingcommands.is_topology_active(env) and errorcommands.is_topology_active(env) + all_found = parsercommands.topologies_running(env) and indexingcommands.is_topology_active(env) if all_found: exit(0) else: diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/service_advisor.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/service_advisor.py index 99c0a7979d..8aff88c6ea 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/service_advisor.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/service_advisor.py @@ -44,7 +44,6 @@ def getServiceComponentLayoutValidations(self, services, hosts): metronParsersHost = self.getHosts(componentsList, "METRON_PARSERS")[0] metronEnrichmentMaster = self.getHosts(componentsList, "METRON_ENRICHMENT_MASTER")[0] metronIndexingHost = self.getHosts(componentsList, "METRON_INDEXING")[0] - metronErrorHost = self.getHosts(componentsList, "METRON_ERROR")[0] hbaseClientHosts = self.getHosts(componentsList, "HBASE_CLIENT") hdfsClientHosts = self.getHosts(componentsList, "HDFS_CLIENT") @@ -72,10 +71,6 @@ def getServiceComponentLayoutValidations(self, services, hosts): message = "Metron Indexing must be co-located with Metron Parsers on {0}".format(metronParsersHost) items.append({ "type": 'host-component', "level": 'ERROR', "message": message, "component-name": 'METRON_INDEXING', "host": metronIndexingHost }) - if metronParsersHost != metronErrorHost: - message = "Metron Error must be co-located with Metron Parsers on {0}".format(metronParsersHost) - items.append({ "type": 'host-component', "level": 'ERROR', "message": message, "component-name": 'METRON_ERROR', "host": metronErrorHost }) - # Enrichment Master also needs ZK Client, but this is already guaranteed by being colocated with Parsers Master if metronParsersHost not in zookeeperClientHosts: message = "Metron must be co-located with an instance of Zookeeper Client" diff --git a/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec b/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec index f66c827ddb..2c619e11a9 100644 --- a/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec +++ b/metron-deployment/packaging/docker/rpm-docker/SPECS/metron.spec @@ -157,9 +157,7 @@ This package installs the Metron Elasticsearch files %dir %{metron_home}/config %dir %{metron_home}/lib %{metron_home}/bin/start_elasticsearch_topology.sh -%{metron_home}/bin/start_elasticsearch_error_topology.sh %{metron_home}/config/elasticsearch.properties -%{metron_home}/config/elasticsearch_error.properties %attr(0644,root,root) %{metron_home}/lib/metron-elasticsearch-%{full_version}-uber.jar # ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ @@ -257,7 +255,6 @@ This package installs the Metron Indexing files %dir %{metron_home}/flux %dir %{metron_home}/flux/indexing %{metron_home}/flux/indexing/remote.yaml -%{metron_home}/flux/error/remote.yaml %{metron_home}/config/zookeeper/indexing/bro.json %{metron_home}/config/zookeeper/indexing/snort.json %{metron_home}/config/zookeeper/indexing/websphere.json diff --git a/metron-deployment/roles/metron_streaming/templates/config/elasticsearch.global.json b/metron-deployment/roles/metron_streaming/templates/config/elasticsearch.global.json index 817710247f..87af1c0de8 100644 --- a/metron-deployment/roles/metron_streaming/templates/config/elasticsearch.global.json +++ b/metron-deployment/roles/metron_streaming/templates/config/elasticsearch.global.json @@ -2,5 +2,6 @@ "es.clustername": "{{ elasticsearch_cluster_name }}", "es.ip": "{{ groups.search[0] }}", "es.port": "{{ elasticsearch_transport_port }}", - "es.date.format": "yyyy.MM.dd.HH" + "es.date.format": "yyyy.MM.dd.HH", + "parser.error.topic": "indexing" } diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java index 4ad09c211c..29be31e8e3 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java @@ -26,7 +26,6 @@ public class Constants { public static final String ZOOKEEPER_TOPOLOGY_ROOT = ZOOKEEPER_ROOT + "/topology"; public static final long DEFAULT_CONFIGURED_BOLT_TIMEOUT = 5000; public static final String SENSOR_TYPE = "source.type"; - public static final String ERROR_TOPIC = "error"; public static final String ENRICHMENT_TOPIC = "enrichments"; public static final String INDEXING_TOPIC = "indexing"; public static final String ERROR_STREAM = "error"; diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/error/MetronError.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/error/MetronError.java index 65026781a2..1121c31591 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/error/MetronError.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/error/MetronError.java @@ -17,11 +17,11 @@ */ package org.apache.metron.common.error; -import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.metron.common.Constants; import org.apache.metron.common.Constants.ErrorType; +import org.apache.metron.common.utils.HashUtils; import org.json.simple.JSONObject; import java.net.InetAddress; @@ -30,7 +30,6 @@ import java.util.List; import java.util.Optional; import java.util.Set; -import java.util.stream.Collectors; import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.metron.common.Constants.ErrorFields; @@ -93,23 +92,54 @@ public Optional getThrowable() { public JSONObject getJSONObject() { JSONObject errorMessage = new JSONObject(); errorMessage.put(Constants.SENSOR_TYPE, "error"); + errorMessage.put(ErrorFields.FAILED_SENSOR_TYPE.getName(), sensorType); + errorMessage.put(ErrorFields.ERROR_TYPE.getName(), errorType.getType()); + + addMessageString(errorMessage); + addStacktrace(errorMessage); + addTimestamp(errorMessage); + addHostname(errorMessage); + addRawMessages(errorMessage); + addErrorHash(errorMessage); + + return errorMessage; + } + + @SuppressWarnings({"unchecked"}) + private void addMessageString(JSONObject errorMessage) { + if (message != null) { + errorMessage.put(ErrorFields.MESSAGE.getName(), message); + } else if (throwable != null) { + errorMessage.put(ErrorFields.MESSAGE.getName(), throwable.getMessage()); + } + } - /* - * Save full stack trace in object. - */ + @SuppressWarnings({"unchecked"}) + private void addStacktrace(JSONObject errorMessage) { if (throwable != null) { String stackTrace = ExceptionUtils.getStackTrace(throwable); String exception = throwable.toString(); errorMessage.put(ErrorFields.EXCEPTION.getName(), exception); errorMessage.put(ErrorFields.STACK.getName(), stackTrace); } + } + @SuppressWarnings({"unchecked"}) + private void addTimestamp(JSONObject errorMessage) { errorMessage.put(ErrorFields.TIMESTAMP.getName(), System.currentTimeMillis()); + } + + @SuppressWarnings({"unchecked"}) + private void addHostname(JSONObject errorMessage) { try { errorMessage.put(ErrorFields.HOSTNAME.getName(), InetAddress.getLocalHost().getHostName()); } catch (UnknownHostException ex) { - + // Leave the hostname field off if it cannot be found } + } + + @SuppressWarnings({"unchecked"}) + private void addRawMessages(JSONObject errorMessage) { if(rawMessages != null) { for(int i = 0; i < rawMessages.size(); i++) { Object rawMessage = rawMessages.get(i); @@ -118,51 +148,38 @@ public JSONObject getJSONObject() { String rawMessageBytesField = rawMessages.size() == 1 ? ErrorFields.RAW_MESSAGE_BYTES.getName() : ErrorFields.RAW_MESSAGE_BYTES.getName() + "_" + i; if(rawMessage instanceof byte[]) { errorMessage.put(rawMessageField, Bytes.toString((byte[])rawMessage)); - errorMessage.put(rawMessageBytesField, toByteArrayList((byte[])rawMessage)); + errorMessage.put(rawMessageBytesField, com.google.common.primitives.Bytes.asList((byte[])rawMessage)); + } else if (rawMessage instanceof JSONObject) { + JSONObject rawMessageJSON = (JSONObject) rawMessage; + String rawMessageJSONString = rawMessageJSON.toJSONString(); + errorMessage.put(rawMessageField, rawMessageJSONString); + errorMessage.put(rawMessageBytesField, com.google.common.primitives.Bytes.asList(rawMessageJSONString.getBytes(UTF_8))); } else { - errorMessage.put(rawMessageField, rawMessage); + errorMessage.put(rawMessageField, rawMessage.toString()); + errorMessage.put(rawMessageBytesField, com.google.common.primitives.Bytes.asList(rawMessage.toString().getBytes(UTF_8))); } } } + } + @SuppressWarnings({"unchecked"}) + private void addErrorHash(JSONObject errorMessage) { if (rawMessages != null && rawMessages.size() == 1) { Object rawMessage = rawMessages.get(0); if (rawMessage instanceof JSONObject) { JSONObject rawJSON = (JSONObject) rawMessage; if (errorFields != null) { - String errorFieldString = String.join(",", errorFields); - errorMessage.put(ErrorFields.ERROR_FIELDS.getName(), errorFieldString); - List hashElements = errorFields.stream().map(errorField -> - String.format("%s-%s", errorField, rawJSON.get(errorField))).collect(Collectors.toList()); - errorMessage.put(ErrorFields.ERROR_HASH.getName(), DigestUtils.sha256Hex(String.join("|", hashElements).getBytes(UTF_8))); + errorMessage.put(ErrorFields.ERROR_FIELDS.getName(), String.join(",", errorFields)); + errorMessage.put(ErrorFields.ERROR_HASH.getName(), HashUtils.getMessageHash(rawJSON, errorFields)); } else { - errorMessage.put(ErrorFields.ERROR_HASH.getName(), DigestUtils.sha256Hex(rawJSON.toJSONString().getBytes(UTF_8))); + errorMessage.put(ErrorFields.ERROR_HASH.getName(), HashUtils.getMessageHash(rawJSON)); } } else if (rawMessage instanceof byte[]) { - errorMessage.put(ErrorFields.ERROR_HASH.getName(), DigestUtils.sha256Hex((byte[])rawMessage)); + errorMessage.put(ErrorFields.ERROR_HASH.getName(), HashUtils.getMessageHash((byte[])rawMessage)); } else { - errorMessage.put(ErrorFields.ERROR_HASH.getName(), DigestUtils.sha256Hex(rawMessage.toString().getBytes(UTF_8))); + errorMessage.put(ErrorFields.ERROR_HASH.getName(), HashUtils.getMessageHash(rawMessage.toString().getBytes(UTF_8))); } } - - if (message != null) { - errorMessage.put(ErrorFields.MESSAGE.getName(), message); - } else if (throwable != null) { - errorMessage.put(ErrorFields.MESSAGE.getName(), throwable.getMessage()); - } - - errorMessage.put(ErrorFields.FAILED_SENSOR_TYPE.getName(), sensorType); - errorMessage.put(ErrorFields.ERROR_TYPE.getName(), errorType.getType()); - - return errorMessage; - } - - protected List toByteArrayList(byte[] list) { - List ret = new ArrayList<>(); - for(byte b : list) { - ret.add(b); - } - return ret; } @Override diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetters.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetters.java index 8e57952ebb..68ffa8cba6 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetters.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetters.java @@ -17,18 +17,36 @@ */ package org.apache.metron.common.message; +import org.apache.metron.common.utils.ConversionUtils; + +import java.util.function.Function; + public enum MessageGetters { - BYTES_FROM_POSITION(new BytesFromPosition()), - JSON_FROM_POSITION(new JSONFromPosition()), - JSON_FROM_FIELD(new JSONFromField()); + BYTES_FROM_POSITION((String arg) -> new BytesFromPosition(ConversionUtils.convert(arg, Integer.class))), + JSON_FROM_POSITION((String arg) -> new JSONFromPosition(ConversionUtils.convert(arg, Integer.class))), + JSON_FROM_FIELD((String arg) -> new JSONFromField(arg)), + OBJECT_FROM_FIELD((String arg) -> new ObjectFromField(arg)), + DEFAULT_BYTES_FROM_POSITION(new BytesFromPosition()), + DEFAULT_JSON_FROM_POSITION(new JSONFromPosition()), + DEFAULT_JSON_FROM_FIELD(new JSONFromField()), + DEFAULT_OBJECT_FROM_FIELD(new ObjectFromField()); + Function messageGetStrategyFunction; MessageGetStrategy messageGetStrategy; MessageGetters(MessageGetStrategy messageGetStrategy) { this.messageGetStrategy = messageGetStrategy; } + MessageGetters(Function messageGetStrategy) { + this.messageGetStrategyFunction = messageGetStrategy; + } + + public MessageGetStrategy get(String arg) { + return messageGetStrategyFunction.apply(arg); + } + public MessageGetStrategy get() { return messageGetStrategy; } diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/ValueFromField.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/ObjectFromField.java similarity index 88% rename from metron-platform/metron-common/src/main/java/org/apache/metron/common/message/ValueFromField.java rename to metron-platform/metron-common/src/main/java/org/apache/metron/common/message/ObjectFromField.java index d5a5799b61..120c09c921 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/ValueFromField.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/ObjectFromField.java @@ -19,13 +19,13 @@ import org.apache.storm.tuple.Tuple; -public class ValueFromField implements MessageGetStrategy { +public class ObjectFromField implements MessageGetStrategy { private String fieldValue = "message"; - public ValueFromField() {}; + public ObjectFromField() {}; - public ValueFromField(String fieldValue) { + public ObjectFromField(String fieldValue) { this.fieldValue = fieldValue; } diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HashUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HashUtils.java new file mode 100644 index 0000000000..b5170cef57 --- /dev/null +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/HashUtils.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.metron.common.utils; + +import org.apache.commons.codec.digest.DigestUtils; +import org.json.simple.JSONObject; + +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; + +import static java.nio.charset.StandardCharsets.UTF_8; + +public class HashUtils { + + public static String getMessageHash(JSONObject message, Collection hashFields) { + List hashElements = hashFields.stream().map(errorField -> + String.format("%s-%s", errorField, message.get(errorField))).collect(Collectors.toList()); + return DigestUtils.sha256Hex(String.join("|", hashElements).getBytes(UTF_8)); + } + + public static String getMessageHash(JSONObject message) { + return DigestUtils.sha256Hex(message.toJSONString().getBytes(UTF_8)); + } + + public static String getMessageHash(byte[] bytes) { + return DigestUtils.sha256Hex(bytes); + } +} diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/error/MetronErrorTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/error/MetronErrorTest.java index 226c6289b5..dcbf6067bf 100644 --- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/error/MetronErrorTest.java +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/error/MetronErrorTest.java @@ -18,6 +18,7 @@ package org.apache.metron.common.error; import com.google.common.collect.Sets; +import com.google.common.primitives.Bytes; import org.apache.metron.common.Constants; import org.json.simple.JSONObject; import org.junit.Before; @@ -76,22 +77,19 @@ public void getJSONObjectShouldIncludeRawMessages() { MetronError error = new MetronError().withRawMessages(Arrays.asList(message1, message2)); JSONObject errorJSON = error.getJSONObject(); - JSONObject expected1 = new JSONObject(); - JSONObject expected2 = new JSONObject(); - expected1.put("value", "message1"); - expected2.put("value", "message2"); - assertEquals(expected1, errorJSON.get(Constants.ErrorFields.RAW_MESSAGE.getName() + "_0")); - assertEquals(expected2, errorJSON.get(Constants.ErrorFields.RAW_MESSAGE.getName() + "_1")); + + assertEquals("{\"value\":\"message1\"}", errorJSON.get(Constants.ErrorFields.RAW_MESSAGE.getName() + "_0")); + assertEquals("{\"value\":\"message2\"}", errorJSON.get(Constants.ErrorFields.RAW_MESSAGE.getName() + "_1")); error = new MetronError().addRawMessage("raw message".getBytes()); errorJSON = error.getJSONObject(); assertEquals("raw message", errorJSON.get(Constants.ErrorFields.RAW_MESSAGE.getName())); - assertEquals(error.toByteArrayList("raw message".getBytes()), errorJSON.get(Constants.ErrorFields.RAW_MESSAGE_BYTES.getName())); + assertEquals(Bytes.asList("raw message".getBytes()), errorJSON.get(Constants.ErrorFields.RAW_MESSAGE_BYTES.getName())); assertEquals("3b02cb29676bc448c69da1ec5eef7c89f4d6dc6a5a7ce0296ea25b207eea36be", errorJSON.get(Constants.ErrorFields.ERROR_HASH.getName())); error = new MetronError().addRawMessage(message1); errorJSON = error.getJSONObject(); - assertEquals(expected1, errorJSON.get(Constants.ErrorFields.RAW_MESSAGE.getName())); + assertEquals("{\"value\":\"message1\"}", errorJSON.get(Constants.ErrorFields.RAW_MESSAGE.getName())); assertEquals("e8aaf87c8494d345aac2d612ffd94fcf0b98c975fe6c4b991e2f8280a3a0bd10", errorJSON.get(Constants.ErrorFields.ERROR_HASH.getName())); } diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/HashUtilsTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/HashUtilsTest.java new file mode 100644 index 0000000000..303734101f --- /dev/null +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/utils/HashUtilsTest.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.metron.common.utils; + +import org.json.simple.JSONObject; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collection; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.Assert.assertEquals; + +@SuppressWarnings({"unchecked"}) +public class HashUtilsTest { + + @Test + public void getMessageHashShouldReturnHashForHashFields() { + JSONObject message = new JSONObject(); + message.put("field1", "value1"); + message.put("field2", "value2"); + message.put("field3", "value3"); + Collection fields = Arrays.asList("field2", "field3"); + assertEquals("6eab1c2c827387803ce457c76552f0511858fc1f9505c7dc620e198c0d1f4d02", HashUtils.getMessageHash(message, fields)); + } + + @Test + public void getMessageHashShouldReturnHashForMessage() { + JSONObject message = new JSONObject(); + message.put("field1", "value1"); + message.put("field2", "value2"); + message.put("field3", "value3"); + assertEquals("a76cdafc5aa49180c0b22c78d4415c505f9997c54847cec6c623f4cacf6a2811", HashUtils.getMessageHash(message)); + } + + @Test + public void getMessageHashShouldReturnHashForBytes() { + assertEquals("ab530a13e45914982b79f9b7e3fba994cfd1f3fb22f71cea1afbf02b460c6d1d", HashUtils.getMessageHash("message".getBytes(UTF_8))); + } +} diff --git a/metron-platform/metron-elasticsearch/src/main/config/elasticsearch.properties b/metron-platform/metron-elasticsearch/src/main/config/elasticsearch.properties index 7cfc1d85e4..c2c10af059 100644 --- a/metron-platform/metron-elasticsearch/src/main/config/elasticsearch.properties +++ b/metron-platform/metron-elasticsearch/src/main/config/elasticsearch.properties @@ -26,7 +26,7 @@ kafka.start=WHERE_I_LEFT_OFF ##### Indexing ##### index.input.topic=indexing -index.error.topic=error +index.error.topic=indexing_error writer.class.name=org.apache.metron.elasticsearch.writer.ElasticsearchWriter ##### ElasticSearch ##### diff --git a/metron-platform/metron-elasticsearch/src/main/config/elasticsearch_error.properties b/metron-platform/metron-elasticsearch/src/main/config/elasticsearch_error.properties deleted file mode 100644 index e96b2a7fd6..0000000000 --- a/metron-platform/metron-elasticsearch/src/main/config/elasticsearch_error.properties +++ /dev/null @@ -1,69 +0,0 @@ -# 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. - -##### Storm ##### -indexing.workers=1 -indexing.executors=0 - -##### Kafka ##### - -kafka.zk=node1:2181 -kafka.broker=node1:6667 -kafka.start=WHERE_I_LEFT_OFF - -##### Indexing ##### -index.input.topic=error -index.error.topic=error -writer.class.name=org.apache.metron.elasticsearch.writer.ElasticsearchWriter - -##### ElasticSearch ##### - -es.ip=10.22.0.214 -es.port=9300 -es.clustername=elasticsearch - -##### Metrics ##### - -#reporters -org.apache.metron.metrics.reporter.graphite=true -org.apache.metron.metrics.reporter.console=false -org.apache.metron.metrics.reporter.jmx=false - -#Graphite Addresses - -org.apache.metron.metrics.graphite.address=localhost -org.apache.metron.metrics.graphite.port=2023 - -#TelemetryParserBolt -org.apache.metron.metrics.TelemetryParserBolt.acks=true -org.apache.metron.metrics.TelemetryParserBolt.emits=true -org.apache.metron.metrics.TelemetryParserBolt.fails=true - - -##### HDFS ##### - -bolt.hdfs.batch.size=5000 -bolt.hdfs.field.delimiter=| -bolt.hdfs.rotation.policy=org.apache.storm.hdfs.bolt.rotation.TimedRotationPolicy -bolt.hdfs.rotation.policy.units=DAYS -bolt.hdfs.rotation.policy.count=1 - -bolt.hdfs.file.rotation.size.in.mb=5 -bolt.hdfs.file.system.url=hdfs://iot01.cloud.hortonworks.com:8020 -bolt.hdfs.wip.file.path=/paloalto/wip -bolt.hdfs.finished.file.path=/paloalto/rotated -bolt.hdfs.compression.codec.class=org.apache.hadoop.io.compress.SnappyCodec -index.hdfs.output=/tmp/metron/error diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchIndexingIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchIndexingIntegrationTest.java index 9894d7a08e..87c0081bfd 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchIndexingIntegrationTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchIndexingIntegrationTest.java @@ -17,8 +17,8 @@ */ package org.apache.metron.elasticsearch.integration; -import org.apache.metron.common.Constants; import org.apache.metron.common.interfaces.FieldNameConverter; +import org.apache.metron.elasticsearch.integration.components.ElasticSearchComponent; import org.apache.metron.elasticsearch.writer.ElasticsearchFieldNameConverter; import org.apache.metron.indexing.integration.IndexingIntegrationTest; import org.apache.metron.integration.ComponentRunner; @@ -26,7 +26,6 @@ import org.apache.metron.integration.Processor; import org.apache.metron.integration.ProcessorResult; import org.apache.metron.integration.ReadinessState; -import org.apache.metron.elasticsearch.integration.components.ElasticSearchComponent; import org.apache.metron.integration.components.KafkaComponent; import java.io.File; @@ -76,7 +75,7 @@ public ReadinessState process(ComponentRunner runner) { throw new IllegalStateException("Unable to retrieve indexed documents.", e); } if (docs.size() < inputMessages.size() || docs.size() != docsFromDisk.size()) { - errors = kafkaComponent.readMessages(Constants.ERROR_TOPIC); + errors = kafkaComponent.readMessages(ERROR_TOPIC); if(errors.size() > 0){ return ReadinessState.READY; } diff --git a/metron-platform/metron-enrichment/src/main/config/enrichment.properties b/metron-platform/metron-enrichment/src/main/config/enrichment.properties index 7091a008da..c905d30ed0 100644 --- a/metron-platform/metron-enrichment/src/main/config/enrichment.properties +++ b/metron-platform/metron-enrichment/src/main/config/enrichment.properties @@ -20,8 +20,8 @@ kafka.zk=node1:2181 kafka.broker=node1:6667 enrichment.output.topic=indexing -enrichment.error.topic=error -threat.intel.error.topic=error +enrichment.error.topic=indexing +threat.intel.error.topic=indexing ##### Metrics ##### diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java index 2e987e376b..3bbb3f50fd 100644 --- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java +++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/bolt/JoinBolt.java @@ -26,7 +26,7 @@ import org.apache.metron.common.bolt.ConfiguredEnrichmentBolt; import org.apache.metron.common.error.MetronError; import org.apache.metron.common.message.MessageGetStrategy; -import org.apache.metron.common.message.ValueFromField; +import org.apache.metron.common.message.MessageGetters; import org.apache.metron.common.utils.ErrorUtils; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; @@ -73,9 +73,9 @@ public JoinBolt withMaxTimeRetain(long maxTimeRetain) { @Override public void prepare(Map map, TopologyContext topologyContext, OutputCollector outputCollector) { super.prepare(map, topologyContext, outputCollector); - keyGetStrategy = new ValueFromField("key"); - subgroupGetStrategy = new ValueFromField("subgroup"); - messageGetStrategy = new ValueFromField("message"); + keyGetStrategy = MessageGetters.OBJECT_FROM_FIELD.get("key"); + subgroupGetStrategy = MessageGetters.OBJECT_FROM_FIELD.get("subgroup"); + messageGetStrategy = MessageGetters.OBJECT_FROM_FIELD.get("message"); this.collector = outputCollector; if (this.maxCacheSize == null) { throw new IllegalStateException("maxCacheSize must be specified"); diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java index aca644e68e..1f33060ebf 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/bolt/BulkMessageWriterBoltTest.java @@ -19,6 +19,7 @@ import org.apache.log4j.Level; import org.apache.metron.common.message.MessageGetStrategy; +import org.apache.metron.common.message.MessageGetters; import org.apache.metron.common.writer.BulkWriterResponse; import org.apache.metron.test.utils.UnitTestHelper; import org.apache.metron.writer.BulkWriterComponent; @@ -118,7 +119,8 @@ public void parseMessages() throws ParseException { @Test public void test() throws Exception { - BulkMessageWriterBolt bulkMessageWriterBolt = new BulkMessageWriterBolt("zookeeperUrl").withBulkMessageWriter(bulkMessageWriter).withMessageGetterField("message"); + BulkMessageWriterBolt bulkMessageWriterBolt = new BulkMessageWriterBolt("zookeeperUrl") + .withBulkMessageWriter(bulkMessageWriter).withMessageGetter(MessageGetters.JSON_FROM_FIELD.name()).withMessageGetterField("message"); bulkMessageWriterBolt.setCuratorFramework(client); bulkMessageWriterBolt.setTreeCache(cache); bulkMessageWriterBolt.getConfigurations().updateSensorIndexingConfig(sensorType, new FileInputStream(sampleSensorIndexingConfigPath)); diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java index 0d32345f7b..c77e7e3c0f 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java @@ -55,6 +55,7 @@ import java.util.*; public class EnrichmentIntegrationTest extends BaseIntegrationTest { + private static final String ERROR_TOPIC = "enrichment_error"; private static final String SRC_IP = "ip_src_addr"; private static final String DST_IP = "ip_dst_addr"; private static final String MALICIOUS_IP_TYPE = "malicious_ip"; @@ -118,13 +119,13 @@ public void test() throws Exception { setProperty("enrichment.simple.hbase.table", enrichmentsTableName); setProperty("enrichment.simple.hbase.cf", cf); setProperty("enrichment.output.topic", Constants.INDEXING_TOPIC); - setProperty("enrichment.error.topic", Constants.ERROR_TOPIC); + setProperty("enrichment.error.topic", ERROR_TOPIC); }}; final ZKServerComponent zkServerComponent = getZKServerComponent(topologyProperties); final KafkaComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList() {{ add(new KafkaComponent.Topic(Constants.ENRICHMENT_TOPIC, 1)); add(new KafkaComponent.Topic(Constants.INDEXING_TOPIC, 1)); - add(new KafkaComponent.Topic(Constants.ERROR_TOPIC, 1)); + add(new KafkaComponent.Topic(ERROR_TOPIC, 1)); }}); String globalConfigStr = null; { @@ -185,7 +186,7 @@ public void test() throws Exception { List> docs = outputMessages.get(Constants.INDEXING_TOPIC); Assert.assertEquals(inputMessages.size(), docs.size()); validateAll(docs); - List> errors = outputMessages.get(Constants.ERROR_TOPIC); + List> errors = outputMessages.get(ERROR_TOPIC); Assert.assertEquals(inputMessages.size(), errors.size()); validateErrors(errors); } finally { @@ -217,7 +218,7 @@ protected void validateErrors(List> errors) { Assert.assertEquals("Test throwing error from ErrorEnrichmentBolt", error.get(Constants.ErrorFields.MESSAGE.getName())); Assert.assertEquals("java.lang.IllegalStateException: Test throwing error from ErrorEnrichmentBolt", error.get(Constants.ErrorFields.EXCEPTION.getName())); Assert.assertEquals(Constants.ErrorType.ENRICHMENT_ERROR.getType(), error.get(Constants.ErrorFields.ERROR_TYPE.getName())); - Assert.assertEquals(new HashMap() {{ put("rawMessage", "Error Test Raw Message String" ); }}, error.get(Constants.ErrorFields.RAW_MESSAGE.getName())); + Assert.assertEquals("{\"rawMessage\":\"Error Test Raw Message String\"}", error.get(Constants.ErrorFields.RAW_MESSAGE.getName())); } } @@ -490,7 +491,7 @@ private KafkaProcessor>>> getProcessor(){ return new KafkaProcessor<>() .withKafkaComponentName("kafka") .withReadTopic(Constants.INDEXING_TOPIC) - .withErrorTopic(Constants.ERROR_TOPIC) + .withErrorTopic(ERROR_TOPIC) .withValidateReadMessages(new Function() { @Nullable @Override @@ -504,7 +505,7 @@ public Boolean apply(@Nullable KafkaMessageSet messageSet) { public Map>> apply(@Nullable KafkaMessageSet messageSet) { return new HashMap>>() {{ put(Constants.INDEXING_TOPIC, loadMessages(messageSet.getMessages())); - put(Constants.ERROR_TOPIC, loadMessages(messageSet.getErrors())); + put(ERROR_TOPIC, loadMessages(messageSet.getErrors())); }}; } }); diff --git a/metron-platform/metron-indexing/src/main/flux/error/remote.yaml b/metron-platform/metron-indexing/src/main/flux/error/remote.yaml deleted file mode 100644 index bc6a3bbcfc..0000000000 --- a/metron-platform/metron-indexing/src/main/flux/error/remote.yaml +++ /dev/null @@ -1,122 +0,0 @@ -# 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. - -name: "error" -config: - topology.workers: ${indexing.workers} - topology.acker.executors: ${indexing.executors} - -components: - - - id: "fileNameFormat" - className: "org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat" - configMethods: - - name: "withPrefix" - args: - - "error-" - - name: "withExtension" - args: - - ".json" - - name: "withPath" - args: - - "${index.hdfs.output}" - - - id: "hdfsRotationPolicy" - className: "${bolt.hdfs.rotation.policy}" - constructorArgs: - - ${bolt.hdfs.rotation.policy.count} - - "${bolt.hdfs.rotation.policy.units}" -#indexing - - id: "hdfsWriter" - className: "org.apache.metron.writer.hdfs.HdfsWriter" - configMethods: - - name: "withFileNameFormat" - args: - - ref: "fileNameFormat" - - name: "withRotationPolicy" - args: - - ref: "hdfsRotationPolicy" - - - id: "indexWriter" - className: "${writer.class.name}" - -#kafka/zookeeper - - id: "zkHosts" - className: "org.apache.storm.kafka.ZkHosts" - constructorArgs: - - "${kafka.zk}" - - id: "kafkaConfig" - className: "org.apache.metron.common.spout.kafka.SpoutConfig" - constructorArgs: - # zookeeper hosts - - ref: "zkHosts" - # topic name - - "${index.input.topic}" - # zk root - - "" - # id - - "error" - configMethods: - - name: "from" - args: - - "${kafka.start}" - -spouts: - - id: "kafkaSpout" - className: "org.apache.storm.kafka.KafkaSpout" - constructorArgs: - - ref: "kafkaConfig" -bolts: - -# Indexing Bolts - - id: "indexingBolt" - className: "org.apache.metron.writer.bolt.BulkMessageWriterBolt" - constructorArgs: - - "${kafka.zk}" - configMethods: - - name: "withBulkMessageWriter" - args: - - ref: "indexWriter" - - name: "withMessageGetter" - args: - - "JSON_FROM_POSITION" - - - id: "hdfsIndexingBolt" - className: "org.apache.metron.writer.bolt.BulkMessageWriterBolt" - constructorArgs: - - "${kafka.zk}" - configMethods: - - name: "withBulkMessageWriter" - args: - - ref: "hdfsWriter" - - name: "withMessageGetter" - args: - - "JSON_FROM_POSITION" - -streams: - - - name: "spout -> indexing" - from: "kafkaSpout" - to: "indexingBolt" - grouping: - type: SHUFFLE - - - name: "spout -> hdfs" - from: "kafkaSpout" - to: "hdfsIndexingBolt" - grouping: - type: SHUFFLE - diff --git a/metron-platform/metron-indexing/src/main/flux/indexing/remote.yaml b/metron-platform/metron-indexing/src/main/flux/indexing/remote.yaml index 987753e3f9..3e329f4008 100644 --- a/metron-platform/metron-indexing/src/main/flux/indexing/remote.yaml +++ b/metron-platform/metron-indexing/src/main/flux/indexing/remote.yaml @@ -101,7 +101,7 @@ bolts: - ref: "indexWriter" - name: "withMessageGetter" args: - - "JSON_FROM_POSITION" + - "DEFAULT_JSON_FROM_POSITION" - id: "hdfsIndexingBolt" className: "org.apache.metron.writer.bolt.BulkMessageWriterBolt" @@ -113,7 +113,7 @@ bolts: - ref: "hdfsWriter" - name: "withMessageGetter" args: - - "JSON_FROM_POSITION" + - "DEFAULT_JSON_FROM_POSITION" - id: "indexingErrorBolt" className: "org.apache.metron.writer.bolt.BulkMessageWriterBolt" 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 5752d921d7..60cd1d1c6a 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 @@ -52,6 +52,7 @@ import static org.apache.metron.common.configuration.ConfigurationsUtils.getClient; public abstract class IndexingIntegrationTest extends BaseIntegrationTest { + protected static final String ERROR_TOPIC = "indexing_error"; protected String hdfsDir = "target/indexingIntegrationTest/hdfs"; protected String sampleParsedPath = TestConstants.SAMPLE_DATA_PARSED_PATH + "TestExampleParsed"; protected String fluxPath = "../metron-indexing/src/main/flux/indexing/remote.yaml"; @@ -124,7 +125,7 @@ public void test() throws Exception { setProperty("indexing.workers", "1"); setProperty("indexing.executors", "0"); setProperty("index.input.topic", Constants.INDEXING_TOPIC); - setProperty("index.error.topic", Constants.ERROR_TOPIC); + setProperty("index.error.topic", ERROR_TOPIC); setProperty("index.date.format", dateFormat); //HDFS settings @@ -137,7 +138,7 @@ public void test() throws Exception { final ZKServerComponent zkServerComponent = getZKServerComponent(topologyProperties); final KafkaComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList() {{ add(new KafkaComponent.Topic(Constants.INDEXING_TOPIC, 1)); - add(new KafkaComponent.Topic(Constants.ERROR_TOPIC, 1)); + add(new KafkaComponent.Topic(ERROR_TOPIC, 1)); }}); List> inputDocs = new ArrayList<>(); for(byte[] b : inputMessages) { 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 83bdc4ff20..b649332bf9 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 @@ -82,7 +82,7 @@ public ParserBolt withMessageFilter(MessageFilter filter) { @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { super.prepare(stormConf, context, collector); - messageGetStrategy = MessageGetters.BYTES_FROM_POSITION.get(); + messageGetStrategy = MessageGetters.DEFAULT_BYTES_FROM_POSITION.get(); String hdfsFile = (String) getConfigurations().getGlobalConfig().get(GeoLiteDatabase.GEO_HDFS_FILE); GeoLiteDatabase.INSTANCE.update(hdfsFile); this.collector = collector; diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/WriterBolt.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/WriterBolt.java index ee8a627275..ef7288b80e 100644 --- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/WriterBolt.java +++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/bolt/WriterBolt.java @@ -18,22 +18,20 @@ package org.apache.metron.parsers.bolt; +import org.apache.metron.common.Constants; +import org.apache.metron.common.configuration.ParserConfigurations; import org.apache.metron.common.error.MetronError; -import org.apache.metron.common.message.JSONFromField; import org.apache.metron.common.message.MessageGetStrategy; import org.apache.metron.common.message.MessageGetters; +import org.apache.metron.common.utils.ErrorUtils; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; import org.apache.storm.topology.OutputFieldsDeclarer; import org.apache.storm.topology.base.BaseRichBolt; import org.apache.storm.tuple.Tuple; -import org.apache.metron.common.Constants; -import org.apache.metron.common.configuration.ParserConfigurations; -import org.apache.metron.common.utils.ErrorUtils; import org.json.simple.JSONObject; import java.util.Map; -import java.util.Optional; public class WriterBolt extends BaseRichBolt { private WriterHandler handler; @@ -56,7 +54,7 @@ public WriterBolt withErrorType(Constants.ErrorType errorType) { @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { this.collector = collector; - messageGetStrategy = MessageGetters.JSON_FROM_FIELD.get(); + messageGetStrategy = MessageGetters.DEFAULT_JSON_FROM_FIELD.get(); handler.init(stormConf, collector, configuration); } diff --git a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java index 545f624ffb..aeac33caa2 100644 --- a/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java +++ b/metron-platform/metron-parsers/src/main/java/org/apache/metron/parsers/topology/ParserTopologyBuilder.java @@ -162,7 +162,7 @@ private static WriterBolt createErrorBolt(String brokerUrl, String sensorType, P // create writer - if not configured uses a sensible default AbstractWriter writer = parserConfig.getErrorWriterClassName() == null - ? new KafkaWriter(brokerUrl).withTopic(Constants.ERROR_TOPIC).withConfigPrefix("error") + ? new KafkaWriter(brokerUrl).withTopic((String) configs.getGlobalConfig().get("parser.error.topic")).withConfigPrefix("error") : ReflectionUtils.createInstance(parserConfig.getWriterClassName()); writer.configure(sensorType, new ParserWriterConfiguration(configs)); 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 cca0852747..a170a2c06f 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 @@ -37,6 +37,7 @@ import java.util.*; public abstract class ParserIntegrationTest extends BaseIntegrationTest { + protected static final String ERROR_TOPIC = "parser_error"; protected List inputMessages; @Test public void test() throws Exception { @@ -47,7 +48,7 @@ public void test() throws Exception { 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(Constants.ERROR_TOPIC,1)); + add(new KafkaComponent.Topic(ERROR_TOPIC,1)); }}); topologyProperties.setProperty("kafka.broker", kafkaComponent.getBrokerList()); @@ -114,7 +115,7 @@ private KafkaProcessor> getProcessor(){ return new KafkaProcessor<>() .withKafkaComponentName("kafka") .withReadTopic(Constants.ENRICHMENT_TOPIC) - .withErrorTopic(Constants.ERROR_STREAM) + .withErrorTopic(ERROR_TOPIC) .withValidateReadMessages(new Function() { @Nullable @Override diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/WriterBoltIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/WriterBoltIntegrationTest.java index b2284303b5..d2c56b2d2e 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/WriterBoltIntegrationTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/WriterBoltIntegrationTest.java @@ -15,8 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.metron.writers.integration; + import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.collect.Iterables; @@ -37,6 +37,8 @@ import org.apache.metron.parsers.integration.components.ParserTopologyComponent; import org.apache.metron.test.utils.UnitTestHelper; import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; +import org.json.simple.parser.ParseException; import org.junit.Assert; import org.junit.Test; @@ -45,6 +47,8 @@ import java.util.*; public class WriterBoltIntegrationTest extends BaseIntegrationTest { + private static final String ERROR_TOPIC = "parser_error"; + public static class MockValidator implements FieldValidation{ @Override @@ -66,7 +70,8 @@ public void initialize(Map validationConfig, Map { "validation" : "org.apache.metron.writers.integration.WriterBoltIntegrationTest$MockValidator" } - ] + ], + "parser.error.topic":"parser_error" } */ @Multiline @@ -89,7 +94,7 @@ public void initialize(Map validationConfig, Map public static String parserConfig; @Test - public void test() throws UnableToStartException, IOException { + public void test() throws UnableToStartException, IOException, ParseException { UnitTestHelper.setLog4jLevel(CSVParser.class, org.apache.log4j.Level.FATAL); final String sensorType = "dummy"; final List inputMessages = new ArrayList() {{ @@ -101,7 +106,7 @@ public void test() throws UnableToStartException, IOException { final ZKServerComponent zkServerComponent = getZKServerComponent(topologyProperties); final KafkaComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList() {{ add(new KafkaComponent.Topic(sensorType, 1)); - add(new KafkaComponent.Topic(Constants.ERROR_TOPIC, 1)); + add(new KafkaComponent.Topic(ERROR_TOPIC, 1)); add(new KafkaComponent.Topic(Constants.ENRICHMENT_TOPIC, 1)); }}); topologyProperties.setProperty("kafka.broker", kafkaComponent.getBrokerList()); @@ -134,12 +139,13 @@ public void test() throws UnableToStartException, IOException { Assert.assertEquals(2, outputMessages.size()); Assert.assertEquals(1, outputMessages.get(Constants.ENRICHMENT_TOPIC).size()); Assert.assertEquals("valid", outputMessages.get(Constants.ENRICHMENT_TOPIC).get(0).get("action")); - Assert.assertEquals(2, outputMessages.get(Constants.ERROR_TOPIC).size()); - JSONObject invalidMessage = outputMessages.get(Constants.ERROR_TOPIC).get(0); + Assert.assertEquals(2, outputMessages.get(ERROR_TOPIC).size()); + JSONObject invalidMessage = outputMessages.get(ERROR_TOPIC).get(0); Assert.assertEquals(Constants.ErrorType.PARSER_INVALID.getType(), invalidMessage.get(Constants.ErrorFields.ERROR_TYPE.getName())); - Assert.assertEquals("foo", ((Map) invalidMessage.get(Constants.ErrorFields.RAW_MESSAGE.getName())).get("dummy")); - Assert.assertEquals("invalid", ((Map) invalidMessage.get(Constants.ErrorFields.RAW_MESSAGE.getName())).get("action")); - JSONObject errorMessage = outputMessages.get(Constants.ERROR_TOPIC).get(1); + JSONObject rawMessage = JSONUtils.INSTANCE.load((String) invalidMessage.get(Constants.ErrorFields.RAW_MESSAGE.getName()), JSONObject.class); + Assert.assertEquals("foo", rawMessage.get("dummy")); + Assert.assertEquals("invalid", rawMessage.get("action")); + JSONObject errorMessage = outputMessages.get(ERROR_TOPIC).get(1); Assert.assertEquals(Constants.ErrorType.PARSER_ERROR.getType(), errorMessage.get(Constants.ErrorFields.ERROR_TYPE.getName())); Assert.assertEquals("error", errorMessage.get(Constants.ErrorFields.RAW_MESSAGE.getName())); Assert.assertTrue(Arrays.equals(listToBytes(errorMessage.get(Constants.ErrorFields.RAW_MESSAGE_BYTES.getName())), "error".getBytes())); @@ -183,7 +189,7 @@ private KafkaProcessor>> getProcessor(){ return new KafkaProcessor<>() .withKafkaComponentName("kafka") .withReadTopic(Constants.ENRICHMENT_TOPIC) - .withErrorTopic(Constants.ERROR_TOPIC) + .withErrorTopic(ERROR_TOPIC) .withValidateReadMessages(new Function() { @Nullable @Override @@ -197,7 +203,7 @@ public Boolean apply(@Nullable KafkaMessageSet messageSet) { public Map> apply(@Nullable KafkaMessageSet messageSet) { return new HashMap>() {{ put(Constants.ENRICHMENT_TOPIC, loadMessages(messageSet.getMessages())); - put(Constants.ERROR_TOPIC, loadMessages(messageSet.getErrors())); + put(ERROR_TOPIC, loadMessages(messageSet.getErrors())); }}; } }); diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrIndexingIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrIndexingIntegrationTest.java index 1c32adb1bc..c209ef346a 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrIndexingIntegrationTest.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrIndexingIntegrationTest.java @@ -18,14 +18,17 @@ package org.apache.metron.solr.integration; import com.google.common.base.Function; -import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.Configurations; import org.apache.metron.common.configuration.ConfigurationsUtils; import org.apache.metron.common.interfaces.FieldNameConverter; import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.enrichment.integration.utils.SampleUtil; import org.apache.metron.indexing.integration.IndexingIntegrationTest; -import org.apache.metron.integration.*; +import org.apache.metron.integration.ComponentRunner; +import org.apache.metron.integration.InMemoryComponent; +import org.apache.metron.integration.Processor; +import org.apache.metron.integration.ProcessorResult; +import org.apache.metron.integration.ReadinessState; import org.apache.metron.integration.components.KafkaComponent; import org.apache.metron.integration.components.ZKServerComponent; import org.apache.metron.solr.integration.components.SolrComponent; @@ -89,7 +92,7 @@ public ReadinessState process(ComponentRunner runner) { throw new IllegalStateException("Unable to retrieve indexed documents.", e); } if (docs.size() < inputMessages.size() || docs.size() != docsFromDisk.size()) { - errors = kafkaComponent.readMessages(Constants.ERROR_TOPIC); + errors = kafkaComponent.readMessages(ERROR_TOPIC); if(errors.size() > 0){ return ReadinessState.READY; } diff --git a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/bolt/BulkMessageWriterBolt.java b/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/bolt/BulkMessageWriterBolt.java index 2070f655b0..085ca5c182 100644 --- a/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/bolt/BulkMessageWriterBolt.java +++ b/metron-platform/metron-writer/src/main/java/org/apache/metron/writer/bolt/BulkMessageWriterBolt.java @@ -17,28 +17,27 @@ */ package org.apache.metron.writer.bolt; +import org.apache.metron.common.Constants; import org.apache.metron.common.bolt.ConfiguredIndexingBolt; -import org.apache.metron.common.message.JSONFromField; +import org.apache.metron.common.configuration.writer.IndexingWriterConfiguration; +import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.common.message.MessageGetStrategy; import org.apache.metron.common.message.MessageGetters; +import org.apache.metron.common.utils.MessageUtils; +import org.apache.metron.common.writer.BulkMessageWriter; +import org.apache.metron.common.writer.MessageWriter; +import org.apache.metron.writer.BulkWriterComponent; +import org.apache.metron.writer.WriterToBulkWriter; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; import org.apache.storm.topology.OutputFieldsDeclarer; import org.apache.storm.tuple.Fields; import org.apache.storm.tuple.Tuple; -import org.apache.metron.common.Constants; -import org.apache.metron.common.configuration.writer.IndexingWriterConfiguration; -import org.apache.metron.common.configuration.writer.WriterConfiguration; -import org.apache.metron.common.writer.MessageWriter; -import org.apache.metron.common.utils.MessageUtils; -import org.apache.metron.common.writer.BulkMessageWriter; -import org.apache.metron.writer.BulkWriterComponent; -import org.apache.metron.writer.WriterToBulkWriter; import org.json.simple.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; +import java.util.Map; import java.util.function.Function; public class BulkMessageWriterBolt extends ConfiguredIndexingBolt { @@ -47,7 +46,7 @@ public class BulkMessageWriterBolt extends ConfiguredIndexingBolt { .getLogger(BulkMessageWriterBolt.class); private BulkMessageWriter bulkMessageWriter; private BulkWriterComponent writerComponent; - private String messageGetStrategyType = MessageGetters.JSON_FROM_FIELD.name(); + private String messageGetStrategyType = MessageGetters.DEFAULT_JSON_FROM_FIELD.name(); private String messageGetField; private transient MessageGetStrategy messageGetStrategy; private transient OutputCollector collector; @@ -81,8 +80,8 @@ public void prepare(Map stormConf, TopologyContext context, OutputCollector coll this.writerComponent = new BulkWriterComponent<>(collector); this.collector = collector; super.prepare(stormConf, context, collector); - if (messageGetStrategyType.equals(MessageGetters.JSON_FROM_FIELD.name()) && messageGetField != null) { - messageGetStrategy = new JSONFromField(messageGetField); + if (messageGetField != null) { + messageGetStrategy = MessageGetters.valueOf(messageGetStrategyType).get(messageGetField); } else { messageGetStrategy = MessageGetters.valueOf(messageGetStrategyType).get(); } From 0b8512d4a346ca640d7cc68eaf03b17665a25222 Mon Sep 17 00:00:00 2001 From: rmerriman Date: Tue, 28 Feb 2017 20:11:52 -0600 Subject: [PATCH 05/16] Added buildGlobalConfig method to testInvalid --- .../java/org/apache/metron/parsers/bolt/ParserBoltTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bolt/ParserBoltTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bolt/ParserBoltTest.java index 6f29a92b7d..b3e15b2c9d 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bolt/ParserBoltTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/parsers/bolt/ParserBoltTest.java @@ -187,6 +187,8 @@ public Map getParserConfig() { }; + buildGlobalConfig(parserBolt); + parserBolt.setCuratorFramework(client); parserBolt.setTreeCache(cache); parserBolt.prepare(new HashMap(), topologyContext, outputCollector); From 4e7d7462000449ce3b3151b20c9518d9e5df7e0d Mon Sep 17 00:00:00 2001 From: rmerriman Date: Wed, 1 Mar 2017 08:15:37 -0600 Subject: [PATCH 06/16] Removed old error topology start script --- .../start_elasticsearch_error_topology.sh | 22 ------------------- 1 file changed, 22 deletions(-) delete mode 100755 metron-platform/metron-elasticsearch/src/main/scripts/start_elasticsearch_error_topology.sh diff --git a/metron-platform/metron-elasticsearch/src/main/scripts/start_elasticsearch_error_topology.sh b/metron-platform/metron-elasticsearch/src/main/scripts/start_elasticsearch_error_topology.sh deleted file mode 100755 index 5cb63912b8..0000000000 --- a/metron-platform/metron-elasticsearch/src/main/scripts/start_elasticsearch_error_topology.sh +++ /dev/null @@ -1,22 +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. -# -METRON_VERSION=${project.version} -METRON_HOME=/usr/metron/$METRON_VERSION -TOPOLOGY_JAR=${project.artifactId}-$METRON_VERSION-uber.jar -storm jar $METRON_HOME/lib/$TOPOLOGY_JAR org.apache.storm.flux.Flux --remote $METRON_HOME/flux/error/remote.yaml --filter $METRON_HOME/config/elasticsearch_error.properties From 770e1cf11e16c087f504180eba7c4c6323e73859 Mon Sep 17 00:00:00 2001 From: rmerriman Date: Wed, 1 Mar 2017 13:58:35 -0600 Subject: [PATCH 07/16] MPack changes --- .../METRON/CURRENT/role_command_order.json | 3 +- .../package/files/error_index.template | 52 +++++++++++++++++++ .../package/scripts/enrichment_commands.py | 6 +-- .../package/scripts/indexing_master.py | 10 ++++ .../package/scripts/params/params_linux.py | 3 +- .../package/scripts/params/status_params.py | 2 - 6 files changed, 66 insertions(+), 10 deletions(-) create mode 100644 metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/addon-services/METRON/CURRENT/role_command_order.json b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/addon-services/METRON/CURRENT/role_command_order.json index e08f401c86..0b04f128fa 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/addon-services/METRON/CURRENT/role_command_order.json +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/addon-services/METRON/CURRENT/role_command_order.json @@ -4,8 +4,9 @@ "general_deps" : { "_comment" : "dependencies for all cases", "METRON_INDEXING-INSTALL" : ["METRON_PARSERS-INSTALL"], + "METRON_ENRICHMENT-INSTALL" : ["METRON_INDEXING-INSTALL"], "METRON_PARSERS-START" : ["NAMENODE-START", "ZOOKEEPER_SERVER-START", "KAFKA_BROKER-START", "STORM_REST_API-START"], - "METRON_ENRICHMENT_MASTER-START" : ["NAMENODE-START", "ZOOKEEPER_SERVER-START", "KAFKA_BROKER-START", "STORM_REST_API-START", "HBASE_MASTER-START", "HBASE_REGIONSERVER-START"], + "METRON_ENRICHMENT_MASTER-START" : ["NAMENODE-START", "ZOOKEEPER_SERVER-START", "KAFKA_BROKER-START", "STORM_REST_API-START", "HBASE_MASTER-START", "HBASE_REGIONSERVER-START", "METRON_INDEXING-START"], "METRON_ENRICHMENT_SERVICE_CHECK-SERVICE_CHECK" : ["METRON_ENRICHMENT_MASTER-START"], "METRON_INDEXING-START" : ["NAMENODE-START", "ZOOKEEPER_SERVER-START", "KAFKA_BROKER-START", "STORM_REST_API-START","METRON_PARSERS-START"], "METRON_SERVICE_CHECK-SERVICE_CHECK" : ["METRON_PARSERS-START","METRON_INDEXING-START"] diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template new file mode 100644 index 0000000000..9a6e4b71a2 --- /dev/null +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template @@ -0,0 +1,52 @@ +{ + "template": "error_index*", + "mappings": { + "error_doc": { + "_timestamp": { + "enabled": true + }, + "properties": { + "exception": { + "type": "string", + "index": "not_analyzed" + }, + "hostname": { + "type": "string", + "index": "not_analyzed" + }, + "stack": { + "type": "string", + "index": "not_analyzed" + }, + "time": { + "type": "date", + "format": "epoch_millis" + }, + "message": { + "type": "string", + "index": "not_analyzed" + }, + "raw_message": { + "type": "string", + "index": "not_analyzed" + }, + "raw_message_bytes": { + "type": "binary", + "index": "no" + }, + "raw_message_hash": { + "type": "string", + "index": "not_analyzed" + }, + "source_type": { + "type": "string", + "index": "not_analyzed" + }, + "error_type": { + "type": "string", + "index": "not_analyzed" + } + } + } + } +} \ No newline at end of file diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/enrichment_commands.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/enrichment_commands.py index bc73c87eb3..817f2668f2 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/enrichment_commands.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/enrichment_commands.py @@ -28,8 +28,6 @@ class EnrichmentCommands: __params = None __enrichment_topology = None __enrichment_topic = None - __enrichment_error_topic = None - __threat_intel_error_topic = None __configured = False def __init__(self, params): @@ -38,8 +36,6 @@ def __init__(self, params): self.__params = params self.__enrichment_topology = params.metron_enrichment_topology self.__enrichment_topic = params.metron_enrichment_topic - self.__enrichment_error_topic = params.metron_enrichment_error_topic - self.__threat_intel_error_topic = params.metron_threat_intel_error_topic self.__configured = os.path.isfile(self.__params.enrichment_configured_flag_file) def is_configured(self): @@ -121,7 +117,7 @@ def init_kafka_topics(self): retention_bytes = retention_gigabytes * 1024 * 1024 * 1024 Logger.info("Creating topics for enrichment") - topics = [self.__enrichment_topic, self.__enrichment_error_topic, self.__threat_intel_error_topic] + topics = [self.__enrichment_topic] for topic in topics: Logger.info("Creating topic'{0}'".format(topic)) Execute(command_template.format(self.__params.kafka_bin_dir, diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/indexing_master.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/indexing_master.py index efc048d93a..53fb17ba99 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/indexing_master.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/indexing_master.py @@ -95,6 +95,11 @@ def elasticsearch_template_install(self, env): content=StaticFile('yaf_index.template') ) + File(params.error_index_path, + mode=0755, + content=StaticFile('error_index.template') + ) + bro_cmd = ambari_format( 'curl -s -XPOST http://{es_http_url}/_template/bro_index -d @{bro_index_path}') Execute(bro_cmd, logoutput=True) @@ -104,6 +109,9 @@ def elasticsearch_template_install(self, env): yaf_cmd = ambari_format( 'curl -s -XPOST http://{es_http_url}/_template/yaf_index -d @{yaf_index_path}') Execute(yaf_cmd, logoutput=True) + error_cmd = ambari_format( + 'curl -s -XPOST http://{es_http_url}/_template/error_index -d @{error_index_path}') + Execute(error_cmd, logoutput=True) def elasticsearch_template_delete(self, env): from params import params @@ -115,6 +123,8 @@ def elasticsearch_template_delete(self, env): Execute(snort_cmd, logoutput=True) yaf_cmd = ambari_format('curl -s -XDELETE "http://{es_http_url}/yaf_index*"') Execute(yaf_cmd, logoutput=True) + error_cmd = ambari_format('curl -s -XDELETE "http://{es_http_url}/error_index*"') + Execute(error_cmd, logoutput=True) def zeppelin_notebook_import(self, env): from params import params diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py index 2427d255f9..2b8276b59f 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py @@ -158,13 +158,12 @@ metron_enrichment_topology = status_params.metron_enrichment_topology metron_enrichment_topic = status_params.metron_enrichment_topic -metron_enrichment_error_topic = status_params.metron_enrichment_error_topic -metron_threat_intel_error_topic = status_params.metron_threat_intel_error_topic # ES Templates bro_index_path = tmp_dir + "/bro_index.template" snort_index_path = tmp_dir + "/snort_index.template" yaf_index_path = tmp_dir + "/yaf_index.template" +error_index_path = tmp_dir + "/error_index.template" # Zeppelin Notebooks metron_config_zeppelin_path = format("{metron_config_path}/zeppelin") diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py index 2151610072..961102fdcb 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py @@ -34,8 +34,6 @@ # Enrichment metron_enrichment_topology = 'enrichment' metron_enrichment_topic = 'enrichments' -metron_enrichment_error_topic = 'indexing' -metron_threat_intel_error_topic = 'indexing' enrichment_table = 'enrichment' enrichment_cf = 't' From c804ecfef60eb5c52b35c693a7d776e8198c4d5c Mon Sep 17 00:00:00 2001 From: merrimanr Date: Thu, 2 Mar 2017 17:05:51 -0600 Subject: [PATCH 08/16] Changed indexing error topic to indexing, added error index template, added tests and javadocs for MessageGetters and commented out rawMessageBytes field --- .../CURRENT/configuration/metron-env.xml | 2 +- .../package/files/error_index.template | 5 +- .../files/es_templates/error_index.template | 53 +++++++ .../metron/common/error/MetronError.java | 9 +- .../metron/common/message/MessageGetters.java | 15 ++ .../metron/common/error/MetronErrorTest.java | 3 +- .../common/message/MessageGettersTest.java | 134 ++++++++++++++++++ .../src/main/config/elasticsearch.properties | 2 +- .../WriterBoltIntegrationTest.java | 3 +- 9 files changed, 216 insertions(+), 10 deletions(-) create mode 100644 metron-deployment/roles/metron_elasticsearch_templates/files/es_templates/error_index.template create mode 100644 metron-platform/metron-common/src/test/java/org/apache/metron/common/message/MessageGettersTest.java diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-env.xml b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-env.xml index b87bb360e7..cbff4a90e4 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-env.xml +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-env.xml @@ -172,7 +172,7 @@ kafka.broker={{ kafka_brokers }} kafka.start=WHERE_I_LEFT_OFF ##### Indexing ##### index.input.topic=indexing -index.error.topic=indexing_error +index.error.topic=indexing writer.class.name=org.apache.metron.elasticsearch.writer.ElasticsearchWriter ##### Metrics ##### #reporters diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template index 9a6e4b71a2..070c90f369 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template @@ -28,7 +28,8 @@ }, "raw_message": { "type": "string", - "index": "not_analyzed" + "index": "not_analyzed", + "ignore_above": 8191 }, "raw_message_bytes": { "type": "binary", @@ -49,4 +50,4 @@ } } } -} \ No newline at end of file +} diff --git a/metron-deployment/roles/metron_elasticsearch_templates/files/es_templates/error_index.template b/metron-deployment/roles/metron_elasticsearch_templates/files/es_templates/error_index.template new file mode 100644 index 0000000000..070c90f369 --- /dev/null +++ b/metron-deployment/roles/metron_elasticsearch_templates/files/es_templates/error_index.template @@ -0,0 +1,53 @@ +{ + "template": "error_index*", + "mappings": { + "error_doc": { + "_timestamp": { + "enabled": true + }, + "properties": { + "exception": { + "type": "string", + "index": "not_analyzed" + }, + "hostname": { + "type": "string", + "index": "not_analyzed" + }, + "stack": { + "type": "string", + "index": "not_analyzed" + }, + "time": { + "type": "date", + "format": "epoch_millis" + }, + "message": { + "type": "string", + "index": "not_analyzed" + }, + "raw_message": { + "type": "string", + "index": "not_analyzed", + "ignore_above": 8191 + }, + "raw_message_bytes": { + "type": "binary", + "index": "no" + }, + "raw_message_hash": { + "type": "string", + "index": "not_analyzed" + }, + "source_type": { + "type": "string", + "index": "not_analyzed" + }, + "error_type": { + "type": "string", + "index": "not_analyzed" + } + } + } + } +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/error/MetronError.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/error/MetronError.java index 1121c31591..2837d34382 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/error/MetronError.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/error/MetronError.java @@ -145,18 +145,19 @@ private void addRawMessages(JSONObject errorMessage) { Object rawMessage = rawMessages.get(i); // If multiple messages are included add an index to the field name, otherwise leave it off String rawMessageField = rawMessages.size() == 1 ? ErrorFields.RAW_MESSAGE.getName() : ErrorFields.RAW_MESSAGE.getName() + "_" + i; - String rawMessageBytesField = rawMessages.size() == 1 ? ErrorFields.RAW_MESSAGE_BYTES.getName() : ErrorFields.RAW_MESSAGE_BYTES.getName() + "_" + i; + // It's unclear if we need a rawMessageBytes field so commenting out for now + //String rawMessageBytesField = rawMessages.size() == 1 ? ErrorFields.RAW_MESSAGE_BYTES.getName() : ErrorFields.RAW_MESSAGE_BYTES.getName() + "_" + i; if(rawMessage instanceof byte[]) { errorMessage.put(rawMessageField, Bytes.toString((byte[])rawMessage)); - errorMessage.put(rawMessageBytesField, com.google.common.primitives.Bytes.asList((byte[])rawMessage)); + //errorMessage.put(rawMessageBytesField, com.google.common.primitives.Bytes.asList((byte[])rawMessage)); } else if (rawMessage instanceof JSONObject) { JSONObject rawMessageJSON = (JSONObject) rawMessage; String rawMessageJSONString = rawMessageJSON.toJSONString(); errorMessage.put(rawMessageField, rawMessageJSONString); - errorMessage.put(rawMessageBytesField, com.google.common.primitives.Bytes.asList(rawMessageJSONString.getBytes(UTF_8))); + //errorMessage.put(rawMessageBytesField, com.google.common.primitives.Bytes.asList(rawMessageJSONString.getBytes(UTF_8))); } else { errorMessage.put(rawMessageField, rawMessage.toString()); - errorMessage.put(rawMessageBytesField, com.google.common.primitives.Bytes.asList(rawMessage.toString().getBytes(UTF_8))); + //errorMessage.put(rawMessageBytesField, com.google.common.primitives.Bytes.asList(rawMessage.toString().getBytes(UTF_8))); } } } diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetters.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetters.java index 68ffa8cba6..a496e08768 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetters.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/message/MessageGetters.java @@ -21,6 +21,21 @@ import java.util.function.Function; +/** + * MessageGetters is a convenience enum for looking up the various implementations of MessageGetStrategy. The MessageGetStrategy + * abstraction returns a value from a tuple. The implementations include: + *

    + *
  • BYTES_FROM_POSITION - gets a byte array from the provided position
  • + *
  • JSON_FROM_POSITION - gets a byte array from the provided position then converts to a string and parses the string to JSON
  • + *
  • JSON_FROM_FIELD - gets a JSONObject from the provided field
  • + *
  • OBJECT_FROM_FIELD - gets an Object from the provided field
  • + *
  • DEFAULT_BYTES_FROM_POSITION - gets a byte array from position 0
  • + *
  • DEFAULT_JSON_FROM_POSITION - gets a byte array from position 0 then converts to a string and parses the string to JSON
  • + *
  • DEFAULT_JSON_FROM_FIELD - gets a JSONObject from the "message" field
  • + *
  • DEFAULT_OBJECT_FROM_FIELD - gets an Object from the "message" field
  • + *
+ * + */ public enum MessageGetters { BYTES_FROM_POSITION((String arg) -> new BytesFromPosition(ConversionUtils.convert(arg, Integer.class))), diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/error/MetronErrorTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/error/MetronErrorTest.java index dcbf6067bf..5e505a8579 100644 --- a/metron-platform/metron-common/src/test/java/org/apache/metron/common/error/MetronErrorTest.java +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/error/MetronErrorTest.java @@ -84,7 +84,8 @@ public void getJSONObjectShouldIncludeRawMessages() { error = new MetronError().addRawMessage("raw message".getBytes()); errorJSON = error.getJSONObject(); assertEquals("raw message", errorJSON.get(Constants.ErrorFields.RAW_MESSAGE.getName())); - assertEquals(Bytes.asList("raw message".getBytes()), errorJSON.get(Constants.ErrorFields.RAW_MESSAGE_BYTES.getName())); + // It's unclear if we need a rawMessageBytes field so commenting out for now + //assertEquals(Bytes.asList("raw message".getBytes()), errorJSON.get(Constants.ErrorFields.RAW_MESSAGE_BYTES.getName())); assertEquals("3b02cb29676bc448c69da1ec5eef7c89f4d6dc6a5a7ce0296ea25b207eea36be", errorJSON.get(Constants.ErrorFields.ERROR_HASH.getName())); error = new MetronError().addRawMessage(message1); diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/message/MessageGettersTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/message/MessageGettersTest.java new file mode 100644 index 0000000000..ea7583ae07 --- /dev/null +++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/message/MessageGettersTest.java @@ -0,0 +1,134 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.metron.common.message; + +import org.apache.storm.tuple.Tuple; +import org.json.simple.JSONObject; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class MessageGettersTest { + + @Rule + public final ExpectedException exception = ExpectedException.none(); + + @Test + public void bytesFromPositionShouldReturnBytes() { + Tuple tuple = mock(Tuple.class); + when(tuple.getBinary(1)).thenReturn("bytes".getBytes(UTF_8)); + + MessageGetStrategy messageGetStrategy = MessageGetters.BYTES_FROM_POSITION.get("1"); + assertEquals("bytes", new String((byte[]) messageGetStrategy.get(tuple), UTF_8)); + } + + @Test + public void jsonFromPositionShouldReturnJSON() { + Tuple tuple = mock(Tuple.class); + when(tuple.getBinary(1)).thenReturn("{\"field\":\"value\"}".getBytes(UTF_8)); + + JSONObject expected = new JSONObject(); + expected.put("field", "value"); + MessageGetStrategy messageGetStrategy = MessageGetters.JSON_FROM_POSITION.get("1"); + assertEquals(expected, messageGetStrategy.get(tuple)); + } + + @Test + public void jsonFromPositionShouldThrowException() { + exception.expect(IllegalStateException.class); + + Tuple tuple = mock(Tuple.class); + when(tuple.getBinary(1)).thenReturn("{\"field\":".getBytes(UTF_8)); + + MessageGetStrategy messageGetStrategy = MessageGetters.JSON_FROM_POSITION.get("1"); + messageGetStrategy.get(tuple); + } + + @Test + public void jsonFromFieldShouldReturnJSON() { + JSONObject actual = new JSONObject(); + actual.put("field", "value"); + Tuple tuple = mock(Tuple.class); + when(tuple.getValueByField("tuple_field")).thenReturn(actual); + + JSONObject expected = new JSONObject(); + expected.put("field", "value"); + MessageGetStrategy messageGetStrategy = MessageGetters.JSON_FROM_FIELD.get("tuple_field"); + assertEquals(expected, messageGetStrategy.get(tuple)); + } + + @Test + public void objectFromFieldShouldReturnObject() { + Object actual = "object"; + Tuple tuple = mock(Tuple.class); + when(tuple.getValueByField("tuple_field")).thenReturn(actual); + + Object expected = "object"; + MessageGetStrategy messageGetStrategy = MessageGetters.OBJECT_FROM_FIELD.get("tuple_field"); + assertEquals(expected, messageGetStrategy.get(tuple)); + } + + @Test + public void defaultBytesFromPositionShouldReturnBytes() { + Tuple tuple = mock(Tuple.class); + when(tuple.getBinary(0)).thenReturn("bytes".getBytes(UTF_8)); + + MessageGetStrategy messageGetStrategy = MessageGetters.DEFAULT_BYTES_FROM_POSITION.get(); + assertEquals("bytes", new String((byte[]) messageGetStrategy.get(tuple), UTF_8)); + } + + @Test + public void defaultJSONFromPositionShouldReturnJSON() { + Tuple tuple = mock(Tuple.class); + when(tuple.getBinary(0)).thenReturn("{\"field\":\"value\"}".getBytes(UTF_8)); + + JSONObject expected = new JSONObject(); + expected.put("field", "value"); + MessageGetStrategy messageGetStrategy = MessageGetters.DEFAULT_JSON_FROM_POSITION.get(); + assertEquals(expected, messageGetStrategy.get(tuple)); + } + + @Test + public void defaultJSONFromFieldShouldReturnJSON() { + JSONObject actual = new JSONObject(); + actual.put("field", "value"); + Tuple tuple = mock(Tuple.class); + when(tuple.getValueByField("message")).thenReturn(actual); + + JSONObject expected = new JSONObject(); + expected.put("field", "value"); + MessageGetStrategy messageGetStrategy = MessageGetters.DEFAULT_JSON_FROM_FIELD.get(); + assertEquals(expected, messageGetStrategy.get(tuple)); + } + + @Test + public void defaultObjectFromFieldShouldReturnObject() { + Object actual = "object"; + Tuple tuple = mock(Tuple.class); + when(tuple.getValueByField("message")).thenReturn(actual); + + Object expected = "object"; + MessageGetStrategy messageGetStrategy = MessageGetters.DEFAULT_OBJECT_FROM_FIELD.get(); + assertEquals(expected, messageGetStrategy.get(tuple)); + } +} diff --git a/metron-platform/metron-elasticsearch/src/main/config/elasticsearch.properties b/metron-platform/metron-elasticsearch/src/main/config/elasticsearch.properties index c2c10af059..27e9173a33 100644 --- a/metron-platform/metron-elasticsearch/src/main/config/elasticsearch.properties +++ b/metron-platform/metron-elasticsearch/src/main/config/elasticsearch.properties @@ -26,7 +26,7 @@ kafka.start=WHERE_I_LEFT_OFF ##### Indexing ##### index.input.topic=indexing -index.error.topic=indexing_error +index.error.topic=indexing writer.class.name=org.apache.metron.elasticsearch.writer.ElasticsearchWriter ##### ElasticSearch ##### diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/WriterBoltIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/WriterBoltIntegrationTest.java index 60142735da..7476bcf250 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/WriterBoltIntegrationTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/WriterBoltIntegrationTest.java @@ -147,7 +147,8 @@ public void test() throws UnableToStartException, IOException, ParseException { JSONObject errorMessage = outputMessages.get(ERROR_TOPIC).get(1); Assert.assertEquals(Constants.ErrorType.PARSER_ERROR.getType(), errorMessage.get(Constants.ErrorFields.ERROR_TYPE.getName())); Assert.assertEquals("error", errorMessage.get(Constants.ErrorFields.RAW_MESSAGE.getName())); - Assert.assertTrue(Arrays.equals(listToBytes(errorMessage.get(Constants.ErrorFields.RAW_MESSAGE_BYTES.getName())), "error".getBytes())); + // It's unclear if we need a rawMessageBytes field so commenting out for now + //Assert.assertTrue(Arrays.equals(listToBytes(errorMessage.get(Constants.ErrorFields.RAW_MESSAGE_BYTES.getName())), "error".getBytes())); } finally { if(runner != null) { From 58fb335055783c4a66af9ab678c3c76c531f86a3 Mon Sep 17 00:00:00 2001 From: merrimanr Date: Mon, 6 Mar 2017 08:49:02 -0600 Subject: [PATCH 09/16] Updated documentation --- metron-platform/metron-common/README.md | 37 +++++++++++++++++++ metron-platform/metron-indexing/README.md | 2 +- metron-platform/metron-parsers/README.md | 8 +--- .../src/main/config/solr.properties | 2 +- 4 files changed, 40 insertions(+), 9 deletions(-) diff --git a/metron-platform/metron-common/README.md b/metron-platform/metron-common/README.md index 4ffb8f1cb7..ea02a0c2dd 100644 --- a/metron-platform/metron-common/README.md +++ b/metron-platform/metron-common/README.md @@ -781,6 +781,7 @@ This configuration is stored in zookeeper, but looks something like "es.ip": "node1", "es.port": "9300", "es.date.format": "yyyy.MM.dd.HH", + "parser.error.topic": "indexing" "fieldValidations" : [ { "input" : [ "ip_src_addr", "ip_dst_addr" ], @@ -850,3 +851,39 @@ Usage examples: * To dump the existing configs from zookeeper on the singlenode vagrant machine: `$METRON_HOME/bin/zk_load_configs.sh -z node1:2181 -m DUMP` * To push the configs into zookeeper on the singlenode vagrant machine: `$METRON_HOME/bin/zk_load_configs.sh -z node1:2181 -m PUSH -i $METRON_HOME/config/zookeeper` * To pull the configs from zookeeper to the singlenode vagrant machine disk: `$METRON_HOME/bin/zk_load_configs.sh -z node1:2181 -m PULL -o $METRON_HOME/config/zookeeper -f` + +# Topology Errors + +Errors generated in Metron topologies are transformed into JSON format and follow this structure: + +``` +{ + "exception": "java.lang.IllegalStateException: Unable to parse Message: ...", + "failed_sensor_type": "bro", + "stack": "java.lang.IllegalStateException: Unable to parse Message: ...", + "hostname": "node1", + "source:type": "error", + "raw_message": "{\"http\": {\"ts\":1488809627.000000.31915,\"uid\":\"C9JpSd2vFAWo3mXKz1\", ...", + "error_hash": "f7baf053f2d3c801a01d196f40f3468e87eea81788b2567423030100865c5061", + "error_type": "parser_error", + "message": "Unable to parse Message: {\"http\": {\"ts\":1488809627.000000.31915,\"uid\":\"C9JpSd2vFAWo3mXKz1\", ...", + "timestamp": 1488809630698 +} +``` + +Each topology can be configured to send error messages to a specific Kafka topic. The parser topologies retrieve this setting from the the `parser.error.topic` setting in the global config: +``` +{ + "es.clustername": "metron", + "es.ip": "node1", + "es.port": "9300", + "es.date.format": "yyyy.MM.dd.HH", + "parser.error.topic": "indexing" +} +``` + +Error topics for enrichment and threat intel errors are passed into the enrichment topology as flux properties named `enrichment.error.topic` and `threat.intel.error.topic`. These properties can be found in `$METRON_HOME/config/enrichment.properties`. + +The error topic for indexing errors is passed into the indexing topology as a flux property named `index.error.topic`. This property can be found in either `$METRON_HOME/config/elasticsearch.properties` or `$METRON_HOME/config/solr.properties` depending on the search engine selected. + +By default all error messages are sent to the `indexing` topic so that they are indexed and archived, just like other messages. The indexing config for error messages can be found at `$METRON_HOME/config/zookeeper/indexing/error.json`. diff --git a/metron-platform/metron-indexing/README.md b/metron-platform/metron-indexing/README.md index 5296ea0a05..f243eefd3d 100644 --- a/metron-platform/metron-indexing/README.md +++ b/metron-platform/metron-indexing/README.md @@ -24,7 +24,7 @@ and sent to * An indexing bolt configured to write to either elasticsearch or Solr * An indexing bolt configured to write to HDFS under `/apps/metron/enrichment/indexed` -Errors during indexing are sent to a kafka queue called `index_errors` +By default, errors during indexing are sent back into the `indexing` kafka queue so that they can be indexed and archived. ##Sensor Indexing Configuration The sensor specific configuration is intended to configure the diff --git a/metron-platform/metron-parsers/README.md b/metron-platform/metron-parsers/README.md index 3c4310dd41..2cf9bbfc16 100644 --- a/metron-platform/metron-parsers/README.md +++ b/metron-platform/metron-parsers/README.md @@ -30,7 +30,7 @@ Data flows through the parser bolt via kafka and into the `enrichments` topology in kafka. Errors are collected with the context of the error (e.g. stacktrace) and original message causing the error and sent to an `error` queue. Invalid messages as determined by global validation -functions are sent to an `invalid` queue. +functions are also treated as errors and sent to an `error` queue. ##Message Format @@ -277,9 +277,6 @@ usage: start_parser_topology.sh -ewp,--error_writer_p Error Writer Parallelism Hint -h,--help This screen - -iwnt,--invalid_writer_num_tasks Invalid Writer Num Tasks - -iwp,--invalid_writer_p Invalid Message Writer - Parallelism Hint -k,--kafka Kafka Broker URL -mt,--message_timeout Message Timeout in Seconds -mtp,--max_task_parallelism Max task parallelism @@ -365,9 +362,6 @@ be customized by modifying the arguments sent to this utility. * The Error Message Writer Bolt * `--error_writer_num_tasks` : The number of tasks for the error writer bolt * `--error_writer_p` : The parallelism hint for the error writer bolt -* The Invalid Message Writer Bolt - * `--invalid_writer_num_tasks` : The number of tasks for the error writer bolt - * `--invalid_writer_p` : The parallelism hint for the error writer bolt Finally, if workers and executors are new to you, the following might be of use to you: * [Understanding the Parallelism of a Storm Topology](http://www.michael-noll.com/blog/2012/10/16/understanding-the-parallelism-of-a-storm-topology/) diff --git a/metron-platform/metron-solr/src/main/config/solr.properties b/metron-platform/metron-solr/src/main/config/solr.properties index 8489d8e2b9..d8dd25f1bc 100644 --- a/metron-platform/metron-solr/src/main/config/solr.properties +++ b/metron-platform/metron-solr/src/main/config/solr.properties @@ -26,7 +26,7 @@ kafka.start=WHERE_I_LEFT_OFF ##### Indexing ##### index.input.topic=indexing -index.error.topic=indexing_error +index.error.topic=indexing writer.class.name=org.apache.metron.solr.writer.SolrWriter ##### Metrics ##### From 53bf2f851cf5b2d803350d273a0827edd9866d06 Mon Sep 17 00:00:00 2001 From: merrimanr Date: Mon, 6 Mar 2017 10:27:15 -0600 Subject: [PATCH 10/16] Resolved merge conflict --- .../metron-profiler/src/main/flux/profiler/remote.yaml | 2 -- 1 file changed, 2 deletions(-) diff --git a/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml b/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml index 0a26b73f42..f97b97ab98 100644 --- a/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml +++ b/metron-analytics/metron-profiler/src/main/flux/profiler/remote.yaml @@ -127,8 +127,6 @@ bolts: configMethods: - name: "withMessageWriter" args: [ref: "kafkaWriter"] - - name: "withMessageGetter" - args: ["NAMED"] streams: From 7412fc678d7e9ddf856b9d852291623c79388864 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Thu, 9 Feb 2017 11:40:37 -0500 Subject: [PATCH 11/16] Error dashboards + minor fixes --- .../package/scripts/dashboard/dashboard.p | 2186 +++++++++++------ .../package/files/error_index.template | 10 +- 2 files changed, 1385 insertions(+), 811 deletions(-) mode change 100755 => 100644 metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p old mode 100755 new mode 100644 index 8327eb8ad9..6e9b9aa5cd --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p @@ -5,310 +5,310 @@ p3 F1 sV_type p4 -Vvisualization +Vindex-pattern p5 sV_id p6 -VWeb-Request-Type +Vbro* p7 sV_source p8 (dp9 -VvisState +Vfields p10 -V{"title":"Web Request Type","type":"pie","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"isDonut":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"method","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} +V[{"name":"TTLs","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"qclass_name","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"bro_timestamp","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"answers","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentjoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:begin:ts","type":"date","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"resp_mime_types","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"protocol","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"original_string","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:threatinteladapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"host","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:end:ts","type":"date","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"AA","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"method","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"query","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rcode","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"orig_mime_types","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"RA","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"RD","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"orig_fuids","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"proto","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:threatinteladapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"response_body_len","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"qtype_name","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"status_code","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"ip_dst_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatinteljoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rejected","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"qtype","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"trans_id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"uid","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"trans_depth","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_dst_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"Z","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"user_agent","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"qclass","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"resp_fuids","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"request_body_len","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"uri","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rcode_name","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"TC","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"referrer","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"status_msg","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":1,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":2,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] p11 -sVdescription +sVtimeFieldName p12 -V -sVtitle +Vtimestamp p13 -VWeb Request Type +sVtitle p14 -sVuiStateJSON +Vbro* p15 -V{} +ssV_index p16 -sVversion +V.kibana p17 -I1 -sVsavedSearchId -p18 -Vweb-search +sa(dp18 +V_score p19 -sVkibanaSavedObjectMeta +F1 +sV_type p20 -(dp21 -VsearchSourceJSON +Vsearch +p21 +sV_id p22 -V{"filter":[]} +Vsnort-search p23 -sssV_index +sV_source p24 -V.kibana -p25 -sa(dp26 -V_score -p27 -F1 -sV_type +(dp25 +Vsort +p26 +(lp27 +Vtimestamp p28 -Vvisualization +aVdesc p29 -sV_id +asVhits p30 -VTop-Snort-Alerts-by-Source +I0 +sVdescription p31 -sV_source +V +sVtitle p32 -(dp33 -VvisState +VSnort Alerts +p33 +sVversion p34 -V{"title":"Top Snort Alerts by Source","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"ip_src_addr","size":10,"order":"desc","orderBy":"1","customLabel":"Source IP"}}],"listeners":{}} +I1 +sVkibanaSavedObjectMeta p35 -sVdescription -p36 -V -sVtitle +(dp36 +VsearchSourceJSON p37 -VTop Snort Alerts by Source +V{"index":"snort*","query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} p38 -sVuiStateJSON +ssVcolumns p39 -V{} -p40 -sVversion +(lp40 +Vmsg p41 -I1 -sVkibanaSavedObjectMeta +aVsig_id p42 -(dp43 -VsearchSourceJSON +aVip_src_addr +p43 +aVip_src_port p44 -V{"index":"snort*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +aVip_dst_addr p45 -sssV_index +aVip_dst_port p46 -V.kibana +assV_index p47 -sa(dp48 +V.kibana +p48 +sa(dp49 V_score -p49 +p50 F1 sV_type -p50 -Vvisualization p51 -sV_id +Vsearch p52 -VWelcome +sV_id p53 -sV_source +Vyaf-search p54 -(dp55 -VvisState -p56 -V{"title":"Welcome to Apache Metron","type":"markdown","params":{"markdown":"This dashboard enables the validation of Apache Metron and the end-to-end functioning of its default sensor suite. The default sensor suite includes [Snort](https://www.snort.org/), [Bro](https://www.bro.org/), and [YAF](https://tools.netsa.cert.org/yaf/). One of Apache Metron's primary goals is to simplify the onboarding of additional sources of telemetry. In a production deployment these default sensors should be replaced with ones applicable to the target environment.\u005cn\u005cnApache Metron enables disparate sources of telemetry to all be viewed under a 'single pane of glass.' Telemetry from each of the default sensors can be searched, aggregated, summarized, and viewed within this dashboard. This dashboard should be used as a springboard upon which to create your own customized dashboards.\u005cn\u005cnThe panels below highlight the volume and variety of events that are currently being consumed by Apache Metron."},"aggs":[],"listeners":{}} +sV_source +p55 +(dp56 +Vsort p57 -sVdescription -p58 -V -sVtitle +(lp58 +Vtimestamp p59 -VWelcome to Apache Metron +aVdesc p60 -sVuiStateJSON +asVhits p61 -V{} +I0 +sVdescription p62 -sVversion +V +sVtitle p63 +VYAF +p64 +sVversion +p65 I1 sVkibanaSavedObjectMeta -p64 -(dp65 -VsearchSourceJSON p66 -V{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} -p67 -sssV_index +(dp67 +VsearchSourceJSON p68 -V.kibana +V{"index":"yaf*","filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647},"query":{"query_string":{"query":"*","analyze_wildcard":true}}} p69 -sa(dp70 -V_score -p71 -F1 -sV_type +ssVcolumns +p70 +(lp71 +Vip_src_addr p72 -Vsearch +aVip_src_port p73 -sV_id +aVip_dst_addr p74 -Vsnort-search +aVip_dst_port p75 -sV_source +aVprotocol p76 -(dp77 -Vsort +aVduration +p77 +aVpkt p78 -(lp79 -Vtimestamp +assV_index +p79 +V.kibana p80 -aVdesc -p81 -asVhits +sa(dp81 +V_score p82 -I0 -sVdescription +F1 +sV_type p83 -V -sVtitle +Vvisualization p84 -VSnort Alerts +sV_id p85 -sVversion +VWelcome p86 -I1 -sVkibanaSavedObjectMeta +sV_source p87 (dp88 -VsearchSourceJSON +VvisState p89 -V{"index":"snort*","query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} +V{"title":"Welcome to Apache Metron","type":"markdown","params":{"markdown":"This dashboard enables the validation of Apache Metron and the end-to-end functioning of its default sensor suite. The default sensor suite includes [Snort](https://www.snort.org/), [Bro](https://www.bro.org/), and [YAF](https://tools.netsa.cert.org/yaf/). One of Apache Metron's primary goals is to simplify the onboarding of additional sources of telemetry. In a production deployment these default sensors should be replaced with ones applicable to the target environment.\u005cn\u005cnApache Metron enables disparate sources of telemetry to all be viewed under a 'single pane of glass.' Telemetry from each of the default sensors can be searched, aggregated, summarized, and viewed within this dashboard. This dashboard should be used as a springboard upon which to create your own customized dashboards.\u005cn\u005cnThe panels below highlight the volume and variety of events that are currently being consumed by Apache Metron."},"aggs":[],"listeners":{}} p90 -ssVcolumns +sVdescription p91 -(lp92 -Vmsg +V +sVtitle +p92 +VWelcome to Apache Metron p93 -aVsig_id +sVuiStateJSON p94 -aVip_src_addr +V{} p95 -aVip_src_port +sVversion p96 -aVip_dst_addr +I1 +sVkibanaSavedObjectMeta p97 -aVip_dst_port -p98 -assV_index +(dp98 +VsearchSourceJSON p99 -V.kibana +V{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} p100 -sa(dp101 -V_score +sssV_index +p101 +V.kibana p102 +sa(dp103 +V_score +p104 F1 sV_type -p103 -Vsearch -p104 -sV_id p105 -Vyaf-search +Vvisualization p106 -sV_source +sV_id p107 -(dp108 -Vsort +VTop-Snort-Alerts-by-Source +p108 +sV_source p109 -(lp110 -Vtimestamp +(dp110 +VvisState p111 -aVdesc +V{"title":"Top Snort Alerts by Source","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"ip_src_addr","size":10,"order":"desc","orderBy":"1","customLabel":"Source IP"}}],"listeners":{}} p112 -asVhits -p113 -I0 sVdescription -p114 +p113 V sVtitle +p114 +VTop Snort Alerts by Source p115 -VYAF +sVuiStateJSON p116 -sVversion +V{} p117 +sVversion +p118 I1 sVkibanaSavedObjectMeta -p118 -(dp119 +p119 +(dp120 VsearchSourceJSON -p120 -V{"index":"yaf*","filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647},"query":{"query_string":{"query":"*","analyze_wildcard":true}}} p121 -ssVcolumns +V{"index":"snort*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p122 -(lp123 -Vip_src_addr +sssV_index +p123 +V.kibana p124 -aVip_src_port -p125 -aVip_dst_addr +sa(dp125 +V_score p126 -aVip_dst_port +F1 +sV_type p127 -aVprotocol +Vvisualization p128 -aVduration +sV_id p129 -aVpkt +VWeb-Request-Type p130 -assV_index +sV_source p131 -V.kibana -p132 -sa(dp133 -V_score +(dp132 +VvisState +p133 +V{"title":"Web Request Type","type":"pie","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"isDonut":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"method","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} p134 -F1 -sV_type +sVdescription p135 -Vconfig +V +sVtitle p136 -sV_id +VWeb Request Type p137 -V4.5.1 +sVuiStateJSON p138 -sV_source +V{} p139 -(dp140 -VbuildNum +sVversion +p140 +I1 +sVsavedSearchId p141 -I9892 -sVdefaultIndex +Vweb-search p142 -Vbro* +sVkibanaSavedObjectMeta p143 -ssV_index -p144 -V.kibana +(dp144 +VsearchSourceJSON p145 -sa(dp146 -V_score +V{"filter":[]} +p146 +sssV_index p147 -F1 -sV_type +V.kibana p148 -Vindex-pattern -p149 -sV_id +sa(dp149 +V_score p150 -Vbro* +F1 +sV_type p151 -sV_source +Vconfig p152 -(dp153 -Vfields +sV_id +p153 +V4.5.1 p154 -V[{"name":"TTLs","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"qclass_name","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"bro_timestamp","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"answers","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentjoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:begin:ts","type":"date","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"resp_mime_types","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"protocol","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"original_string","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:threatinteladapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"host","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:end:ts","type":"date","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"AA","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"method","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"query","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rcode","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"orig_mime_types","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"RA","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"RD","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"orig_fuids","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"proto","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:threatinteladapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"response_body_len","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"qtype_name","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"status_code","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"ip_dst_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatinteljoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rejected","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"qtype","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"trans_id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"uid","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"trans_depth","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_dst_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"Z","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"user_agent","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"qclass","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"resp_fuids","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"request_body_len","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"uri","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rcode_name","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"TC","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"referrer","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"status_msg","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":1,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":2,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] +sV_source p155 -sVtimeFieldName -p156 -Vtimestamp +(dp156 +VbuildNum p157 -sVtitle +I9892 +sVdefaultIndex p158 Vbro* p159 @@ -326,25 +326,25 @@ Vvisualization p165 sV_id p166 -VFlow-Duration +VWeb-Request-Header p167 sV_source p168 (dp169 VvisState p170 -V{"title":"Flow Duration","type":"area","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"smoothLines":false,"scale":"linear","interpolate":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"histogram","schema":"segment","params":{"field":"duration","interval":10,"extended_bounds":{},"customLabel":"Flow Duration (seconds)"}}],"listeners":{}} +V{"title":"Web Request Header","type":"markdown","params":{"markdown":"The [Bro Network Security Monitor](https://www.bro.org/) is extracting application-level information from raw network packets. In this example, Bro is extracting HTTP(S) requests being made over the network. "},"aggs":[],"listeners":{}} p171 sVdescription p172 V sVtitle p173 -VFlow Duration +VWeb Request Header p174 sVuiStateJSON p175 -V{"vis":{"legendOpen":false}} +V{} p176 sVversion p177 @@ -354,7 +354,7 @@ p178 (dp179 VsearchSourceJSON p180 -V{"index":"yaf*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p181 sssV_index p182 @@ -370,25 +370,25 @@ Vvisualization p187 sV_id p188 -VEvents +VError-Type-Proportion p189 sV_source p190 (dp191 VvisState p192 -V{"title":"Events","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"date_histogram","schema":"segment","params":{"field":"timestamp","interval":"auto","customInterval":"2h","min_doc_count":1,"extended_bounds":{}}},{"id":"3","type":"terms","schema":"group","params":{"field":"source:type","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} +V{"title":"Error Type Proportion","type":"pie","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"isDonut":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"error_type","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} p193 sVdescription p194 V sVtitle p195 -VEvents +VError Type Proportion p196 sVuiStateJSON p197 -V{"vis":{"legendOpen":false}} +V{} p198 sVversion p199 @@ -398,7 +398,7 @@ p200 (dp201 VsearchSourceJSON p202 -V{"index":["yaf*", "bro*", "snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p203 sssV_index p204 @@ -414,25 +414,25 @@ Vvisualization p209 sV_id p210 -VWeb-Request-Header +VFlow-Duration p211 sV_source p212 (dp213 VvisState p214 -V{"title":"Web Request Header","type":"markdown","params":{"markdown":"The [Bro Network Security Monitor](https://www.bro.org/) is extracting application-level information from raw network packets. In this example, Bro is extracting HTTP(S) requests being made over the network. "},"aggs":[],"listeners":{}} +V{"title":"Flow Duration","type":"area","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"smoothLines":false,"scale":"linear","interpolate":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"histogram","schema":"segment","params":{"field":"duration","interval":10,"extended_bounds":{},"customLabel":"Flow Duration (seconds)"}}],"listeners":{}} p215 sVdescription p216 V sVtitle p217 -VWeb Request Header +VFlow Duration p218 sVuiStateJSON p219 -V{} +V{"vis":{"legendOpen":false}} p220 sVversion p221 @@ -442,7 +442,7 @@ p222 (dp223 VsearchSourceJSON p224 -V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"index":"yaf*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p225 sssV_index p226 @@ -458,21 +458,21 @@ Vvisualization p231 sV_id p232 -VTop-Alerts-By-Host +VErrors-By-Source p233 sV_source p234 (dp235 VvisState p236 -V{"title":"New Visualization","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"ip_src_addr","size":5,"order":"desc","orderBy":"1","customLabel":"Source"}},{"id":"3","type":"terms","schema":"bucket","params":{"field":"ip_dst_addr","size":5,"order":"desc","orderBy":"1","customLabel":"Destination"}}],"listeners":{}} +V{"title":"Errors By Source","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"source_type","size":5,"order":"desc","orderBy":"1","customLabel":"Source"}}],"listeners":{}} p237 sVdescription p238 V sVtitle p239 -VTop Alerts By Host +VErrors By Source p240 sVuiStateJSON p241 @@ -481,1059 +481,1629 @@ p242 sVversion p243 I1 -sVsavedSearchId -p244 -Vsnort-search -p245 sVkibanaSavedObjectMeta -p246 -(dp247 +p244 +(dp245 VsearchSourceJSON -p248 -V{"filter":[]} -p249 +p246 +V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p247 sssV_index -p250 +p248 V.kibana -p251 -sa(dp252 +p249 +sa(dp250 V_score -p253 +p251 F1 sV_type -p254 +p252 Vvisualization -p255 +p253 sV_id -p256 -VYAF-Flow(s) -p257 +p254 +VEvents +p255 sV_source -p258 -(dp259 +p256 +(dp257 VvisState -p260 -V{"title":"YAF Flows","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} -p261 +p258 +V{"title":"Events","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"date_histogram","schema":"segment","params":{"field":"timestamp","interval":"auto","customInterval":"2h","min_doc_count":1,"extended_bounds":{}}},{"id":"3","type":"terms","schema":"group","params":{"field":"source:type","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} +p259 sVdescription -p262 +p260 V sVtitle +p261 +VEvents +p262 +sVuiStateJSON p263 -VYAF Flows +V{"vis":{"legendOpen":false}} p264 -sVuiStateJSON -p265 -V{} -p266 sVversion -p267 +p265 I1 sVkibanaSavedObjectMeta -p268 -(dp269 +p266 +(dp267 VsearchSourceJSON -p270 -V{"index":"yaf*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p271 +p268 +V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p269 sssV_index -p272 +p270 V.kibana -p273 -sa(dp274 +p271 +sa(dp272 V_score -p275 +p273 F1 sV_type -p276 +p274 Vvisualization -p277 +p275 sV_id -p278 -VTop-DNS-Query -p279 +p276 +VUnique-Error-Messages +p277 sV_source -p280 -(dp281 +p278 +(dp279 VvisState -p282 -V{"title":"Top DNS Query","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":10,"order":"desc","orderBy":"1"}}],"listeners":{}} -p283 +p280 +V{\u000a "title": "Total Unique Error Messages",\u000a "type": "metric",\u000a "params": {\u000a "handleNoResults": true,\u000a "fontSize": 60\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "error_hash",\u000a "customLabel": "Unique Error Messages"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +p281 sVdescription -p284 +p282 V sVtitle -p285 -VTop DNS Query -p286 +p283 +VUnique Error Messages +p284 sVuiStateJSON -p287 +p285 V{} -p288 +p286 sVversion -p289 +p287 I1 sVkibanaSavedObjectMeta -p290 -(dp291 +p288 +(dp289 VsearchSourceJSON -p292 -V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p293 +p290 +V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "query": "*",\u000a "analyze_wildcard": true\u000a }\u000a },\u000a "filter": []\u000a} +p291 sssV_index -p294 +p292 V.kibana -p295 -sa(dp296 +p293 +sa(dp294 V_score -p297 +p295 F1 sV_type -p298 +p296 Vvisualization -p299 +p297 sV_id -p300 -VTotal-Events -p301 +p298 +VErrors-By-Error-Type +p299 sV_source -p302 -(dp303 +p300 +(dp301 VvisState -p304 -V{"title":"Event Count","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{"customLabel":"Events"}}],"listeners":{}} -p305 +p302 +V{\u000a "title": "Errors By Error Type",\u000a "type": "histogram",\u000a "params": {\u000a "addLegend": true,\u000a "addTimeMarker": false,\u000a "addTooltip": true,\u000a "defaultYExtents": false,\u000a "mode": "grouped",\u000a "scale": "linear",\u000a "setYExtents": false,\u000a "shareYAxis": true,\u000a "times": [],\u000a "yAxis": {}\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {\u000a "customLabel": "Count"\u000a }\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "error_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1"\u000a }\u000a },\u000a {\u000a "id": "4",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "error_hash",\u000a "customLabel": "Unique Datapoint Count"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +p303 sVdescription -p306 +p304 V sVtitle +p305 +VErrors By Error Type +p306 +sVuiStateJSON p307 -VEvent Count +V{\u000a "vis": {\u000a "colors": {\u000a "Unique Datapoint Count": "#806EB7",\u000a "Count": "#614D93"\u000a }\u000a }\u000a} p308 -sVuiStateJSON -p309 -V{} -p310 sVversion -p311 +p309 I1 sVkibanaSavedObjectMeta -p312 -(dp313 +p310 +(dp311 VsearchSourceJSON -p314 -V{"index":["yaf*", "bro*", "snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p315 +p312 +V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "analyze_wildcard": true,\u000a "query": "*"\u000a }\u000a },\u000a "filter": []\u000a} +p313 sssV_index -p316 +p314 V.kibana -p317 -sa(dp318 +p315 +sa(dp316 V_score -p319 +p317 F1 sV_type +p318 +Vsearch +p319 +sV_id p320 -Vvisualization +VErrors p321 -sV_id -p322 -VEvent-Types -p323 sV_source +p322 +(dp323 +Vsort p324 -(dp325 -VvisState +(lp325 +Vtimestamp p326 -V{"title":"Event Sources","type":"pie","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"isDonut":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"source:type","size":10,"order":"desc","orderBy":"1"}}],"listeners":{}} +aVdesc p327 -sVdescription +asVhits p328 +I0 +sVdescription +p329 V sVtitle -p329 -VEvent Sources p330 -sVuiStateJSON +VErrors p331 -V{} -p332 sVversion -p333 +p332 I1 sVkibanaSavedObjectMeta -p334 -(dp335 +p333 +(dp334 VsearchSourceJSON +p335 +V{"index":"error*","query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} p336 -V{"index":["yaf*", "bro*", "snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +ssVcolumns p337 -sssV_index -p338 -V.kibana +(lp338 +Vfailed_sensor_type p339 -sa(dp340 -V_score +aVerror_type +p340 +aVexception p341 -F1 -sV_type +aVhostname p342 -Vvisualization +aVmessage p343 -sV_id +aVraw_message p344 -VUnique-Location(s) +aVerror_hash p345 -sV_source +assV_index p346 -(dp347 -VvisState -p348 -V{"title":"Geo-IP Locations","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"cardinality","schema":"metric","params":{"field":"enrichments:geo:ip_src_addr:locID","customLabel":"Unique Location(s)"}}],"listeners":{}} +V.kibana +p347 +sa(dp348 +V_score p349 -sVdescription +F1 +sV_type p350 -V -sVtitle +Vdashboard p351 -VGeo-IP Locations +sV_id p352 -sVuiStateJSON +VMetron-Dashboard p353 -V{} +sV_source p354 -sVversion -p355 -I1 -sVkibanaSavedObjectMeta +(dp355 +Vhits p356 -(dp357 -VsearchSourceJSON +I0 +sVtimeRestore +p357 +I00 +sVdescription p358 -V{"index":["yaf*", "bro*", "snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V +sVtitle p359 -sssV_index +VMetron Dashboard p360 -V.kibana +sVuiStateJSON p361 -sa(dp362 -V_score +V{"P-23":{"spy":{"mode":{"name":null,"fill":false}}},"P-34":{"vis":{"legendOpen":false}}} +p362 +sVpanelsJSON p363 -F1 -sV_type +V[{"col":1,"id":"Welcome","panelIndex":30,"row":1,"size_x":11,"size_y":2,"type":"visualization"},{"col":1,"id":"Total-Events","panelIndex":6,"row":3,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"id":"Events","panelIndex":16,"row":3,"size_x":8,"size_y":4,"type":"visualization"},{"col":1,"id":"Event-Types","panelIndex":15,"row":5,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Location-Header","panelIndex":24,"row":7,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Unique-Location(s)","panelIndex":23,"row":9,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"id":"Flow-Locations","panelIndex":32,"row":7,"size_x":8,"size_y":6,"type":"visualization"},{"col":1,"id":"Country","panelIndex":8,"row":11,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"YAF-Flows-Header","panelIndex":27,"row":13,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"YAF-Flow(s)","panelIndex":21,"row":15,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["ip_src_addr","ip_src_port","ip_dst_addr","ip_dst_port","protocol","duration","pkt"],"id":"yaf-search","panelIndex":20,"row":13,"size_x":8,"size_y":6,"sort":["duration","desc"],"type":"search"},{"col":1,"id":"Flow-Duration","panelIndex":31,"row":17,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Snort-Header","panelIndex":25,"row":19,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["msg","sig_id","ip_src_addr","ip_src_port","ip_dst_addr","ip_dst_port"],"id":"snort-search","panelIndex":3,"row":19,"size_x":8,"size_y":6,"sort":["timestamp","desc"],"type":"search"},{"col":1,"id":"Snort-Alert-Types","panelIndex":10,"row":21,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Top-Alerts-By-Host","panelIndex":19,"row":23,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Web-Request-Header","panelIndex":26,"row":25,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["method","host","uri","referrer","user_agent","ip_src_addr","ip_dst_addr"],"id":"web-search","panelIndex":4,"row":25,"size_x":8,"size_y":6,"sort":["timestamp","desc"],"type":"search"},{"col":1,"id":"HTTP(S)-Requests","panelIndex":17,"row":27,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"DNS-Requests-Header","panelIndex":29,"row":31,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["query","qtype_name","answers","ip_src_addr","ip_dst_addr"],"id":"dns-search","panelIndex":5,"row":31,"size_x":8,"size_y":6,"sort":["timestamp","desc"],"type":"search"},{"col":1,"id":"DNS-Request(s)","panelIndex":14,"row":33,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Web-Request-Type","panelIndex":33,"row":29,"size_x":3,"size_y":2,"type":"visualization"}] p364 -Vvisualization +sVoptionsJSON p365 -sV_id +V{"darkTheme":false} p366 -VSnort-Header +sVversion p367 -sV_source +I1 +sVkibanaSavedObjectMeta p368 (dp369 -VvisState +VsearchSourceJSON p370 -V{"title":"Snort","type":"markdown","params":{"markdown":"[Snort](https://www.snort.org/) is a Network Intrusion Detection System (NIDS) that is being used to generate alerts identifying known bad events. Snort relies on a fixed set of rules that act as signatures for identifying abnormal events."},"aggs":[],"listeners":{}} +V{"filter":[{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}}}]} p371 -sVdescription +sssV_index p372 -V -sVtitle +V.kibana p373 -VSnort -p374 -sVuiStateJSON +sa(dp374 +V_score p375 -V{} +F1 +sV_type p376 -sVversion +Vvisualization p377 -I1 -sVkibanaSavedObjectMeta +sV_id p378 -(dp379 -VsearchSourceJSON +VSnort-Header +p379 +sV_source p380 -V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p381 -sssV_index +(dp381 +VvisState p382 -V.kibana +V{"title":"Snort","type":"markdown","params":{"markdown":"[Snort](https://www.snort.org/) is a Network Intrusion Detection System (NIDS) that is being used to generate alerts identifying known bad events. Snort relies on a fixed set of rules that act as signatures for identifying abnormal events."},"aggs":[],"listeners":{}} p383 -sa(dp384 -V_score +sVdescription +p384 +V +sVtitle p385 -F1 -sV_type +VSnort p386 -Vdashboard +sVuiStateJSON p387 -sV_id +V{} p388 -VMetron-Dashboard +sVversion p389 -sV_source +I1 +sVkibanaSavedObjectMeta p390 (dp391 -Vhits +VsearchSourceJSON p392 -I0 -sVtimeRestore +V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p393 -I00 -sVdescription +sssV_index p394 -V -sVtitle +V.kibana p395 -VMetron Dashboard -p396 -sVuiStateJSON +sa(dp396 +V_score p397 -V{"P-23":{"spy":{"mode":{"name":null,"fill":false}}},"P-34":{"vis":{"legendOpen":false}}} +F1 +sV_type p398 -sVpanelsJSON +Vvisualization p399 -V[{"col":1,"id":"Welcome","panelIndex":30,"row":1,"size_x":11,"size_y":2,"type":"visualization"},{"col":1,"id":"Total-Events","panelIndex":6,"row":3,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"id":"Events","panelIndex":16,"row":3,"size_x":8,"size_y":4,"type":"visualization"},{"col":1,"id":"Event-Types","panelIndex":15,"row":5,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Location-Header","panelIndex":24,"row":7,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Unique-Location(s)","panelIndex":23,"row":9,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"id":"Flow-Locations","panelIndex":32,"row":7,"size_x":8,"size_y":6,"type":"visualization"},{"col":1,"id":"Country","panelIndex":8,"row":11,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"YAF-Flows-Header","panelIndex":27,"row":13,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"YAF-Flow(s)","panelIndex":21,"row":15,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["ip_src_addr","ip_src_port","ip_dst_addr","ip_dst_port","protocol","duration","pkt"],"id":"yaf-search","panelIndex":20,"row":13,"size_x":8,"size_y":6,"sort":["duration","desc"],"type":"search"},{"col":1,"id":"Flow-Duration","panelIndex":31,"row":17,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Snort-Header","panelIndex":25,"row":19,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["msg","sig_id","ip_src_addr","ip_src_port","ip_dst_addr","ip_dst_port"],"id":"snort-search","panelIndex":3,"row":19,"size_x":8,"size_y":6,"sort":["timestamp","desc"],"type":"search"},{"col":1,"id":"Snort-Alert-Types","panelIndex":10,"row":21,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Top-Alerts-By-Host","panelIndex":19,"row":23,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Web-Request-Header","panelIndex":26,"row":25,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["method","host","uri","referrer","user_agent","ip_src_addr","ip_dst_addr"],"id":"web-search","panelIndex":4,"row":25,"size_x":8,"size_y":6,"sort":["timestamp","desc"],"type":"search"},{"col":1,"id":"HTTP(S)-Requests","panelIndex":17,"row":27,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"DNS-Requests-Header","panelIndex":29,"row":31,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["query","qtype_name","answers","ip_src_addr","ip_dst_addr"],"id":"dns-search","panelIndex":5,"row":31,"size_x":8,"size_y":6,"sort":["timestamp","desc"],"type":"search"},{"col":1,"id":"DNS-Request(s)","panelIndex":14,"row":33,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Web-Request-Type","panelIndex":33,"row":29,"size_x":3,"size_y":2,"type":"visualization"}] +sV_id p400 -sVoptionsJSON +VYAF-Flow(s) p401 -V{"darkTheme":false} +sV_source p402 +(dp403 +VvisState +p404 +V{"title":"YAF Flows","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} +p405 +sVdescription +p406 +V +sVtitle +p407 +VYAF Flows +p408 +sVuiStateJSON +p409 +V{} +p410 sVversion -p403 +p411 I1 sVkibanaSavedObjectMeta -p404 -(dp405 +p412 +(dp413 VsearchSourceJSON -p406 -V{"filter":[{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}}}]} -p407 +p414 +V{"index":"yaf*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p415 sssV_index -p408 +p416 V.kibana -p409 -sa(dp410 +p417 +sa(dp418 V_score -p411 +p419 F1 sV_type -p412 +p420 Vvisualization -p413 +p421 sV_id -p414 -VSnort-Alert-Types -p415 +p422 +VTop-DNS-Query +p423 sV_source -p416 -(dp417 +p424 +(dp425 VvisState -p418 -V{"title":"Snort Alert Types","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"cardinality","schema":"metric","params":{"field":"sig_id","customLabel":"Alert Type(s)"}}],"listeners":{}} -p419 +p426 +V{"title":"Top DNS Query","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":10,"order":"desc","orderBy":"1"}}],"listeners":{}} +p427 sVdescription -p420 +p428 V sVtitle -p421 -VSnort Alert Types -p422 +p429 +VTop DNS Query +p430 sVuiStateJSON -p423 +p431 V{} -p424 +p432 sVversion -p425 +p433 I1 sVkibanaSavedObjectMeta -p426 -(dp427 +p434 +(dp435 VsearchSourceJSON -p428 -V{"index":"snort*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p429 +p436 +V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p437 sssV_index -p430 +p438 V.kibana -p431 -sa(dp432 +p439 +sa(dp440 V_score -p433 +p441 F1 sV_type -p434 +p442 Vvisualization -p435 +p443 sV_id -p436 -VFrequent-DNS-Queries -p437 +p444 +VEvent-Types +p445 sV_source -p438 -(dp439 +p446 +(dp447 VvisState -p440 -V{"title":"Frequent DNS Requests","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} -p441 +p448 +V{"title":"Event Sources","type":"pie","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"isDonut":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"source:type","size":10,"order":"desc","orderBy":"1"}}],"listeners":{}} +p449 sVdescription -p442 +p450 V sVtitle -p443 -VFrequent DNS Requests -p444 +p451 +VEvent Sources +p452 sVuiStateJSON -p445 +p453 V{} -p446 +p454 sVversion -p447 +p455 I1 sVkibanaSavedObjectMeta -p448 -(dp449 +p456 +(dp457 VsearchSourceJSON -p450 -V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p451 +p458 +V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p459 sssV_index -p452 +p460 V.kibana -p453 -sa(dp454 +p461 +sa(dp462 V_score -p455 +p463 F1 sV_type -p456 +p464 Vvisualization -p457 +p465 sV_id -p458 -VLocation-Header -p459 +p466 +VTotal-Events +p467 sV_source -p460 -(dp461 +p468 +(dp469 VvisState -p462 -V{"title":"Enrichment","type":"markdown","params":{"markdown":"Apache Metron can perform real-time enrichment of telemetry data as it is consumed. To highlight this feature, all of the IP address fields collected from the default sensor suite were used to perform geo-ip lookups. This data was then used to pinpoint each location on the map."},"aggs":[],"listeners":{}} -p463 +p470 +V{"title":"Event Count","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{"customLabel":"Events"}}],"listeners":{}} +p471 sVdescription -p464 +p472 V sVtitle -p465 -VEnrichment -p466 +p473 +VEvent Count +p474 sVuiStateJSON -p467 +p475 V{} -p468 +p476 sVversion -p469 +p477 I1 sVkibanaSavedObjectMeta -p470 -(dp471 +p478 +(dp479 VsearchSourceJSON -p472 -V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p473 +p480 +V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p481 sssV_index -p474 +p482 V.kibana -p475 -sa(dp476 +p483 +sa(dp484 V_score -p477 +p485 F1 sV_type -p478 -Vsearch -p479 -sV_id -p480 -Vweb-search -p481 -sV_source -p482 -(dp483 -Vsort -p484 -(lp485 -Vtimestamp p486 -aVdesc +Vvisualization p487 -asVhits +sV_id p488 -I0 -sVdescription +VUnique-Location(s) p489 -V -sVtitle +sV_source p490 -VWeb Requests -p491 -sVversion +(dp491 +VvisState p492 -I1 -sVkibanaSavedObjectMeta +V{"title":"Geo-IP Locations","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"cardinality","schema":"metric","params":{"field":"enrichments:geo:ip_src_addr:locID","customLabel":"Unique Location(s)"}}],"listeners":{}} p493 -(dp494 -VsearchSourceJSON +sVdescription +p494 +V +sVtitle p495 -V{"index":"bro*","query":{"query_string":{"query":"protocol: http OR protocol: https","analyze_wildcard":true}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} +VGeo-IP Locations p496 -ssVcolumns +sVuiStateJSON p497 -(lp498 -Vmethod +V{} +p498 +sVversion p499 -aVhost +I1 +sVkibanaSavedObjectMeta p500 -aVuri -p501 -aVreferrer +(dp501 +VsearchSourceJSON p502 -aVip_src_addr +V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p503 -aVip_dst_addr +sssV_index p504 -assV_index -p505 V.kibana -p506 -sa(dp507 +p505 +sa(dp506 V_score -p508 +p507 F1 sV_type +p508 +Vvisualization p509 -Vindex-pattern -p510 sV_id +p510 +VTop-Alerts-By-Host p511 -Vsnort* +sV_source p512 +(dp513 +VvisState +p514 +V{"title":"Top Alerts By Host","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"ip_src_addr","size":5,"order":"desc","orderBy":"1","customLabel":"Source"}},{"id":"3","type":"terms","schema":"bucket","params":{"field":"ip_dst_addr","size":5,"order":"desc","orderBy":"1","customLabel":"Destination"}}],"listeners":{}} +p515 +sVdescription +p516 +V +sVtitle +p517 +VTop Alerts By Host +p518 +sVuiStateJSON +p519 +V{} +p520 +sVversion +p521 +I1 +sVsavedSearchId +p522 +Vsnort-search +p523 +sVkibanaSavedObjectMeta +p524 +(dp525 +VsearchSourceJSON +p526 +V{"filter":[]} +p527 +sssV_index +p528 +V.kibana +p529 +sa(dp530 +V_score +p531 +F1 +sV_type +p532 +Vvisualization +p533 +sV_id +p534 +VTotal-Error-Messages +p535 +sV_source +p536 +(dp537 +VvisState +p538 +V{"title":"Total Errored Messages","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{"customLabel":"Total Error Messages"}}],"listeners":{}} +p539 +sVdescription +p540 +V +sVtitle +p541 +VTotal Error Messages +p542 +sVuiStateJSON +p543 +V{} +p544 +sVversion +p545 +I1 +sVkibanaSavedObjectMeta +p546 +(dp547 +VsearchSourceJSON +p548 +V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p549 +sssV_index +p550 +V.kibana +p551 +sa(dp552 +V_score +p553 +F1 +sV_type +p554 +Vvisualization +p555 +sV_id +p556 +VErrors-By-Source-Type +p557 +sV_source +p558 +(dp559 +VvisState +p560 +V{\u000a "title": "Errors By Source Type",\u000a "type": "histogram",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "scale": "linear",\u000a "mode": "grouped",\u000a "times": [],\u000a "addTimeMarker": false,\u000a "defaultYExtents": false,\u000a "setYExtents": false,\u000a "yAxis": {}\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {\u000a "customLabel": "Count"\u000a }\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "failed_sensor_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1"\u000a }\u000a },\u000a {\u000a "id": "4",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "error_hash",\u000a "customLabel": "Unique Datapoint Count"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +p561 +sVdescription +p562 +V +sVtitle +p563 +VErrors By Source Type +p564 +sVuiStateJSON +p565 +V{\u000a "vis": {\u000a "colors": {\u000a "Unique Datapoint Count": "#0A50A1",\u000a "Count": "#5195CE"\u000a }\u000a }\u000a} +p566 +sVversion +p567 +I1 +sVkibanaSavedObjectMeta +p568 +(dp569 +VsearchSourceJSON +p570 +V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "analyze_wildcard": true,\u000a "query": "*"\u000a }\u000a },\u000a "filter": []\u000a} +p571 +sssV_index +p572 +V.kibana +p573 +sa(dp574 +V_score +p575 +F1 +sV_type +p576 +Vdashboard +p577 +sV_id +p578 +VError-Dashboard +p579 +sV_source +p580 +(dp581 +Vhits +p582 +I0 +sVtimeRestore +p583 +I00 +sVdescription +p584 +V +sVtitle +p585 +VError Dashboard +p586 +sVuiStateJSON +p587 +V{"P-2":{"vis":{"legendOpen":true}}} +p588 +sVpanelsJSON +p589 +V[{"col":5,"id":"Errors-By-Error-Type","panelIndex":2,"row":7,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Source-Proportion","panelIndex":3,"row":7,"size_x":4,"size_y":3,"type":"visualization"},{"col":5,"id":"Errors-By-Source-Type","panelIndex":4,"row":10,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Type-Proportion","panelIndex":5,"row":10,"size_x":4,"size_y":3,"type":"visualization"},{"col":1,"id":"Errors-Over-Time","panelIndex":14,"row":3,"size_x":12,"size_y":4,"type":"visualization"},{"col":8,"id":"Unique-Error-Messages","panelIndex":19,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":3,"id":"Total-Error-Messages","panelIndex":20,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":1,"columns":["failed_sensor_type","error_type","exception","hostname","message","raw_message","error_hash"],"id":"Errors","panelIndex":21,"row":13,"size_x":12,"size_y":7,"sort":["timestamp","desc"],"type":"search"}] +p590 +sVoptionsJSON +p591 +V{"darkTheme":false} +p592 +sVversion +p593 +I1 +sVkibanaSavedObjectMeta +p594 +(dp595 +VsearchSourceJSON +p596 +V{"filter":[{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}}}]} +p597 +sssV_index +p598 +V.kibana +p599 +sa(dp600 +V_score +p601 +F1 +sV_type +p602 +Vindex-pattern +p603 +sV_id +p604 +Vsnort* +p605 sV_source -p513 -(dp514 +p606 +(dp607 Vfields -p515 +p608 V[{"name":"msg","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dgmlen","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentjoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpack","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"protocol","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:threatinteladapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"original_string","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:geoadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ethlen","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threat:triage:level","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:threatinteladapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"ip_dst_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatinteljoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_rev","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"ethsrc","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpseq","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichmentsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpwindow","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_dst_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tos","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ethdst","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"is_alert","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ttl","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"iplen","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_generator","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] -p516 +p609 sVtimeFieldName -p517 +p610 Vtimestamp -p518 +p611 sVtitle -p519 +p612 Vsnort* -p520 +p613 ssV_index -p521 +p614 V.kibana -p522 -sa(dp523 +p615 +sa(dp616 V_score -p524 +p617 F1 sV_type -p525 +p618 Vindex-pattern -p526 +p619 sV_id -p527 +p620 Vyaf* -p528 +p621 sV_source -p529 -(dp530 +p622 +(dp623 Vfields -p531 +p624 V[{"name":"enrichments:geo:ip_dst_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"isn","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentjoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dip","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dp","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"protocol","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"rpkt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"original_string","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:threatinteladapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tag","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"app","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"oct","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end_reason","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichmentsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"start_time","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"riflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"proto","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:threatinteladapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"iflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"ip_dst_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatinteljoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"uflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"duration","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_dst_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"pkt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ruflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"roct","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sip","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sp","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rtag","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end-reason","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"risn","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end_time","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rtt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] -p532 +p625 sVtimeFieldName -p533 +p626 Vtimestamp -p534 +p627 sVtitle -p535 +p628 Vyaf* -p536 +p629 ssV_index -p537 +p630 V.kibana -p538 -sa(dp539 +p631 +sa(dp632 V_score -p540 +p633 F1 sV_type -p541 +p634 +Vsearch +p635 +sV_id +p636 +Vweb-search +p637 +sV_source +p638 +(dp639 +Vsort +p640 +(lp641 +Vtimestamp +p642 +aVdesc +p643 +asVhits +p644 +I0 +sVdescription +p645 +V +sVtitle +p646 +VWeb Requests +p647 +sVversion +p648 +I1 +sVkibanaSavedObjectMeta +p649 +(dp650 +VsearchSourceJSON +p651 +V{"index":"bro*","query":{"query_string":{"query":"protocol: http OR protocol: https","analyze_wildcard":true}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} +p652 +ssVcolumns +p653 +(lp654 +Vmethod +p655 +aVhost +p656 +aVuri +p657 +aVreferrer +p658 +aVip_src_addr +p659 +aVip_dst_addr +p660 +assV_index +p661 +V.kibana +p662 +sa(dp663 +V_score +p664 +F1 +sV_type +p665 Vvisualization -p542 +p666 sV_id -p543 +p667 +VLocation-Header +p668 +sV_source +p669 +(dp670 +VvisState +p671 +V{"title":"Enrichment","type":"markdown","params":{"markdown":"Apache Metron can perform real-time enrichment of telemetry data as it is consumed. To highlight this feature, all of the IP address fields collected from the default sensor suite were used to perform geo-ip lookups. This data was then used to pinpoint each location on the map."},"aggs":[],"listeners":{}} +p672 +sVdescription +p673 +V +sVtitle +p674 +VEnrichment +p675 +sVuiStateJSON +p676 +V{} +p677 +sVversion +p678 +I1 +sVkibanaSavedObjectMeta +p679 +(dp680 +VsearchSourceJSON +p681 +V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p682 +sssV_index +p683 +V.kibana +p684 +sa(dp685 +V_score +p686 +F1 +sV_type +p687 +Vvisualization +p688 +sV_id +p689 +VSnort-Alert-Types +p690 +sV_source +p691 +(dp692 +VvisState +p693 +V{"title":"Snort Alert Types","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"cardinality","schema":"metric","params":{"field":"sig_id","customLabel":"Alert Type(s)"}}],"listeners":{}} +p694 +sVdescription +p695 +V +sVtitle +p696 +VSnort Alert Types +p697 +sVuiStateJSON +p698 +V{} +p699 +sVversion +p700 +I1 +sVkibanaSavedObjectMeta +p701 +(dp702 +VsearchSourceJSON +p703 +V{"index":"snort*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p704 +sssV_index +p705 +V.kibana +p706 +sa(dp707 +V_score +p708 +F1 +sV_type +p709 +Vvisualization +p710 +sV_id +p711 +VFrequent-DNS-Queries +p712 +sV_source +p713 +(dp714 +VvisState +p715 +V{"title":"Frequent DNS Requests","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} +p716 +sVdescription +p717 +V +sVtitle +p718 +VFrequent DNS Requests +p719 +sVuiStateJSON +p720 +V{} +p721 +sVversion +p722 +I1 +sVkibanaSavedObjectMeta +p723 +(dp724 +VsearchSourceJSON +p725 +V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p726 +sssV_index +p727 +V.kibana +p728 +sa(dp729 +V_score +p730 +F1 +sV_type +p731 +Vvisualization +p732 +sV_id +p733 VDNS-Request(s) -p544 +p734 sV_source -p545 -(dp546 +p735 +(dp736 VvisState -p547 +p737 V{"title":"DNS Requests","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} -p548 +p738 sVdescription -p549 +p739 V sVtitle -p550 +p740 VDNS Requests -p551 +p741 sVuiStateJSON -p552 +p742 V{} -p553 +p743 sVversion -p554 +p744 I1 sVsavedSearchId -p555 +p745 Vdns-search -p556 +p746 sVkibanaSavedObjectMeta -p557 -(dp558 +p747 +(dp748 VsearchSourceJSON -p559 +p749 V{"filter":[]} -p560 +p750 sssV_index -p561 +p751 V.kibana -p562 -sa(dp563 +p752 +sa(dp753 V_score -p564 +p754 F1 sV_type -p565 +p755 Vvisualization -p566 +p756 sV_id -p567 +p757 VHTTP(S)-Requests -p568 +p758 +sV_source +p759 +(dp760 +VvisState +p761 +V{"title":"Web Requests","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} +p762 +sVdescription +p763 +V +sVtitle +p764 +VWeb Requests +p765 +sVuiStateJSON +p766 +V{} +p767 +sVversion +p768 +I1 +sVsavedSearchId +p769 +Vweb-search +p770 +sVkibanaSavedObjectMeta +p771 +(dp772 +VsearchSourceJSON +p773 +V{"filter":[]} +p774 +sssV_index +p775 +V.kibana +p776 +sa(dp777 +V_score +p778 +F1 +sV_type +p779 +Vvisualization +p780 +sV_id +p781 +VErrors-Over-Time +p782 +sV_source +p783 +(dp784 +VvisState +p785 +V{"title":"Error Over Time","type":"line","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"showCircles":true,"smoothLines":false,"interpolate":"linear","scale":"linear","drawLinesBetweenPoints":true,"radiusRatio":9,"times":[],"addTimeMarker":true,"defaultYExtents":false,"setYExtents":false,"yAxis":{"min":0}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"date_histogram","schema":"segment","params":{"field":"timestamp","interval":"auto","customInterval":"2h","min_doc_count":1,"extended_bounds":{}}}],"listeners":{}} +p786 +sVdescription +p787 +V +sVtitle +p788 +VErrors Over Time +p789 +sVuiStateJSON +p790 +V{} +p791 +sVversion +p792 +I1 +sVkibanaSavedObjectMeta +p793 +(dp794 +VsearchSourceJSON +p795 +V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p796 +sssV_index +p797 +V.kibana +p798 +sa(dp799 +V_score +p800 +F1 +sV_type +p801 +Vvisualization +p802 +sV_id +p803 +VError-Source-Proportion +p804 sV_source -p569 -(dp570 +p805 +(dp806 VvisState -p571 -V{"title":"Web Requests","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} -p572 +p807 +V{\u000a "title": "Error Sensor Proportion",\u000a "type": "pie",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "isDonut": false\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {}\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "failed_sensor_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1",\u000a "customLabel": "Sensor"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +p808 sVdescription -p573 +p809 V sVtitle -p574 -VWeb Requests -p575 +p810 +VError Source Proportion +p811 sVuiStateJSON -p576 +p812 V{} -p577 +p813 sVversion -p578 +p814 I1 -sVsavedSearchId -p579 -Vweb-search -p580 sVkibanaSavedObjectMeta -p581 -(dp582 +p815 +(dp816 VsearchSourceJSON -p583 -V{"filter":[]} -p584 +p817 +V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "query": "*",\u000a "analyze_wildcard": true\u000a }\u000a },\u000a "filter": []\u000a} +p818 sssV_index -p585 +p819 V.kibana -p586 -sa(dp587 +p820 +sa(dp821 V_score -p588 +p822 F1 sV_type -p589 +p823 +Vindex-pattern +p824 +sV_id +p825 +Verror* +p826 +sV_source +p827 +(dp828 +Vfields +p829 +V[{"name":"exception","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"stack","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"error_hash","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"raw_message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"failed_sensor_type","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"hostname","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"error_type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"error_fields","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"raw_message_bytes","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] +p830 +sVtimeFieldName +p831 +Vtimestamp +p832 +sVtitle +p833 +Verror* +p834 +ssV_index +p835 +V.kibana +p836 +sa(dp837 +V_score +p838 +F1 +sV_type +p839 +Vconfig +p840 +sV_id +p841 +V4.5.3 +p842 +sV_source +p843 +(dp844 +VbuildNum +p845 +I9892 +sVdefaultIndex +p846 +Vbro* +p847 +ssV_index +p848 +V.kibana +p849 +sa(dp850 +V_score +p851 +F1 +sV_type +p852 Vsearch -p590 +p853 sV_id -p591 +p854 Vdns-search -p592 +p855 sV_source -p593 -(dp594 +p856 +(dp857 Vsort -p595 -(lp596 +p858 +(lp859 Vtimestamp -p597 +p860 aVdesc -p598 +p861 asVhits -p599 +p862 I0 sVdescription -p600 +p863 V sVtitle -p601 +p864 VDNS Requests -p602 +p865 sVversion -p603 +p866 I1 sVkibanaSavedObjectMeta -p604 -(dp605 +p867 +(dp868 VsearchSourceJSON -p606 +p869 V{"index":"bro*","query":{"query_string":{"query":"protocol: dns","analyze_wildcard":true}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} -p607 +p870 ssVcolumns -p608 -(lp609 +p871 +(lp872 Vquery -p610 +p873 aVqtype_name -p611 +p874 aVanswers -p612 +p875 aVip_src_addr -p613 +p876 aVip_dst_addr -p614 +p877 assV_index -p615 +p878 V.kibana -p616 -sa(dp617 +p879 +sa(dp880 V_score -p618 +p881 F1 sV_type -p619 +p882 Vvisualization -p620 +p883 sV_id -p621 -VFlow-Locations -p622 +p884 +VDNS-Requests-Header +p885 sV_source -p623 -(dp624 +p886 +(dp887 VvisState -p625 -V{"title":"New Visualization","type":"tile_map","params":{"mapType":"Scaled Circle Markers","isDesaturated":true,"addTooltip":true,"heatMaxZoom":16,"heatMinOpacity":0.1,"heatRadius":25,"heatBlur":15,"heatNormalizeData":true,"wms":{"enabled":false,"url":"https://basemap.nationalmap.gov/arcgis/services/USGSTopo/MapServer/WMSServer","options":{"version":"1.3.0","layers":"0","format":"image/png","transparent":true,"attribution":"Maps provided by USGS","styles":""}}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"geohash_grid","schema":"segment","params":{"field":"enrichments:geo:ip_dst_addr:location_point","autoPrecision":true,"precision":2}}],"listeners":{}} -p626 +p888 +V{"aggs":[],"listeners":{},"params":{"markdown":"[Bro](https://www.bro.org/) is extracting DNS requests and responses being made over the network. Understanding who is making those requests, the frequency, and types can provide a deep understanding of the actors present on the network."},"title":"DNS Requests","type":"markdown"} +p889 sVdescription -p627 +p890 V sVtitle -p628 -VFlow Locations -p629 +p891 +VDNS Requests +p892 sVuiStateJSON -p630 +p893 V{} -p631 +p894 sVversion -p632 +p895 I1 sVkibanaSavedObjectMeta -p633 -(dp634 +p896 +(dp897 VsearchSourceJSON -p635 -V{"index":["yaf*", "bro*", "snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p636 +p898 +V{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} +p899 sssV_index -p637 +p900 V.kibana -p638 -sa(dp639 +p901 +sa(dp902 V_score -p640 +p903 F1 sV_type -p641 +p904 Vvisualization -p642 +p905 sV_id -p643 -VUnusual-Referrers -p644 +p906 +VYAF-Flows-Header +p907 sV_source -p645 -(dp646 +p908 +(dp909 VvisState -p647 -V{"title":"Unusual Referrers","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"significant_terms","schema":"bucket","params":{"field":"referrer","size":5,"customLabel":"Top 5 Unusual Referrers"}}],"listeners":{}} -p648 +p910 +V{"title":"YAF","type":"markdown","params":{"markdown":"[YAF](https://tools.netsa.cert.org/yaf/yaf.html) can be used to generate Netflow-like flow records. These flow records provide significant visibility of the actors communicating over the target network."},"aggs":[],"listeners":{}} +p911 sVdescription -p649 +p912 V sVtitle -p650 -VUnusual Referrers -p651 +p913 +VYAF +p914 sVuiStateJSON -p652 +p915 V{} -p653 +p916 sVversion -p654 +p917 I1 -sVsavedSearchId -p655 -Vweb-search -p656 sVkibanaSavedObjectMeta -p657 -(dp658 +p918 +(dp919 VsearchSourceJSON -p659 -V{"filter":[]} -p660 +p920 +V{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} +p921 sssV_index -p661 +p922 V.kibana -p662 -sa(dp663 +p923 +sa(dp924 V_score -p664 +p925 F1 sV_type -p665 +p926 Vvisualization -p666 +p927 sV_id -p667 +p928 +VTop-5-Exceptions +p929 +sV_source +p930 +(dp931 +VvisState +p932 +V{"title":"Top-5 Exceptions","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"exception","size":5,"order":"desc","orderBy":"1","customLabel":"Exceptions"}}],"listeners":{}} +p933 +sVdescription +p934 +V +sVtitle +p935 +VTop-5 Exceptions +p936 +sVuiStateJSON +p937 +V{} +p938 +sVversion +p939 +I1 +sVkibanaSavedObjectMeta +p940 +(dp941 +VsearchSourceJSON +p942 +V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p943 +sssV_index +p944 +V.kibana +p945 +sa(dp946 +V_score +p947 +F1 +sV_type +p948 +Vvisualization +p949 +sV_id +p950 VFrequent-DNS-Requests -p668 +p951 sV_source -p669 -(dp670 +p952 +(dp953 VvisState -p671 +p954 V{"title":"Frequent DNS Requests","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":5,"order":"desc","orderBy":"1","customLabel":"DNS Query"}}],"listeners":{}} -p672 +p955 sVdescription -p673 +p956 V sVtitle -p674 +p957 VFrequent DNS Requests -p675 +p958 sVuiStateJSON -p676 +p959 V{} -p677 +p960 sVversion -p678 +p961 I1 sVkibanaSavedObjectMeta -p679 -(dp680 +p962 +(dp963 VsearchSourceJSON -p681 +p964 V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p682 +p965 sssV_index -p683 +p966 V.kibana -p684 -sa(dp685 +p967 +sa(dp968 V_score -p686 +p969 F1 sV_type -p687 +p970 Vvisualization -p688 +p971 sV_id -p689 +p972 VCountry -p690 +p973 sV_source -p691 -(dp692 +p974 +(dp975 VvisState -p693 +p976 V{"title":"By Country","type":"pie","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"isDonut":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"enrichments:geo:ip_src_addr:country","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} -p694 +p977 sVdescription -p695 +p978 V sVtitle -p696 +p979 VBy Country -p697 +p980 sVuiStateJSON -p698 +p981 V{} -p699 +p982 sVversion -p700 +p983 I1 sVkibanaSavedObjectMeta -p701 -(dp702 +p984 +(dp985 VsearchSourceJSON -p703 -V{"index":["yaf*", "bro*", "snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p704 +p986 +V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p987 sssV_index -p705 +p988 V.kibana -p706 -sa(dp707 +p989 +sa(dp990 V_score -p708 +p991 F1 sV_type -p709 +p992 Vvisualization -p710 +p993 sV_id -p711 -VTop-Destinations -p712 +p994 +VFlow-Locations +p995 sV_source -p713 -(dp714 +p996 +(dp997 VvisState -p715 -V{"title":"Top Destinations","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"ip_dst_addr","size":10,"order":"desc","orderBy":"1","customLabel":"Destination IP"}}],"listeners":{}} -p716 +p998 +V{"title":"Flow Locations","type":"tile_map","params":{"mapType":"Scaled Circle Markers","isDesaturated":true,"addTooltip":true,"heatMaxZoom":16,"heatMinOpacity":0.1,"heatRadius":25,"heatBlur":15,"heatNormalizeData":true,"wms":{"enabled":false,"url":"https://basemap.nationalmap.gov/arcgis/services/USGSTopo/MapServer/WMSServer","options":{"version":"1.3.0","layers":"0","format":"image/png","transparent":true,"attribution":"Maps provided by USGS","styles":""}}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"geohash_grid","schema":"segment","params":{"field":"enrichments:geo:ip_dst_addr:location_point","autoPrecision":true,"precision":2}}],"listeners":{}} +p999 sVdescription -p717 +p1000 V sVtitle -p718 -VTop Destinations -p719 +p1001 +VFlow Locations +p1002 sVuiStateJSON -p720 +p1003 V{} -p721 +p1004 sVversion -p722 +p1005 I1 sVkibanaSavedObjectMeta -p723 -(dp724 +p1006 +(dp1007 VsearchSourceJSON -p725 -V{"index":["yaf*", "bro*", "snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p726 +p1008 +V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p1009 sssV_index -p727 +p1010 V.kibana -p728 -sa(dp729 +p1011 +sa(dp1012 V_score -p730 +p1013 F1 sV_type -p731 +p1014 Vvisualization -p732 +p1015 sV_id -p733 -VDNS-Requests-Header -p734 +p1016 +VTop-Destinations +p1017 sV_source -p735 -(dp736 +p1018 +(dp1019 VvisState -p737 -V{"aggs":[],"listeners":{},"params":{"markdown":"[Bro](https://www.bro.org/) is extracting DNS requests and responses being made over the network. Understanding who is making those requests, the frequency, and types can provide a deep understanding of the actors present on the network."},"title":"DNS Requests","type":"markdown"} -p738 +p1020 +V{"title":"Top Destinations","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"ip_dst_addr","size":10,"order":"desc","orderBy":"1","customLabel":"Destination IP"}}],"listeners":{}} +p1021 sVdescription -p739 +p1022 V sVtitle -p740 -VDNS Requests -p741 +p1023 +VTop Destinations +p1024 sVuiStateJSON -p742 +p1025 V{} -p743 +p1026 sVversion -p744 +p1027 I1 sVkibanaSavedObjectMeta -p745 -(dp746 +p1028 +(dp1029 VsearchSourceJSON -p747 -V{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} -p748 +p1030 +V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p1031 sssV_index -p749 +p1032 V.kibana -p750 -sa(dp751 +p1033 +sa(dp1034 V_score -p752 +p1035 F1 sV_type -p753 +p1036 Vvisualization -p754 +p1037 sV_id -p755 -VYAF-Flows-Header -p756 +p1038 +VUnusual-Referrers +p1039 sV_source -p757 -(dp758 +p1040 +(dp1041 VvisState -p759 -V{"title":"YAF","type":"markdown","params":{"markdown":"[YAF](https://tools.netsa.cert.org/yaf/yaf.html) can be used to generate Netflow-like flow records. These flow records provide significant visibility of the actors communicating over the target network."},"aggs":[],"listeners":{}} -p760 +p1042 +V{"title":"Unusual Referrers","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"significant_terms","schema":"bucket","params":{"field":"referrer","size":5,"customLabel":"Top 5 Unusual Referrers"}}],"listeners":{}} +p1043 sVdescription -p761 +p1044 V sVtitle -p762 -VYAF -p763 +p1045 +VUnusual Referrers +p1046 sVuiStateJSON -p764 +p1047 V{} -p765 +p1048 sVversion -p766 +p1049 I1 +sVsavedSearchId +p1050 +Vweb-search +p1051 sVkibanaSavedObjectMeta -p767 -(dp768 +p1052 +(dp1053 VsearchSourceJSON -p769 -V{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} -p770 +p1054 +V{"filter":[]} +p1055 sssV_index -p771 +p1056 V.kibana -p772 +p1057 sa. \ No newline at end of file diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template index 070c90f369..3bb4633704 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template @@ -18,7 +18,7 @@ "type": "string", "index": "not_analyzed" }, - "time": { + "timestamp": { "type": "date", "format": "epoch_millis" }, @@ -35,11 +35,15 @@ "type": "binary", "index": "no" }, - "raw_message_hash": { + "error_fields": { "type": "string", "index": "not_analyzed" }, - "source_type": { + "error_hash": { + "type": "string", + "index": "not_analyzed" + }, + "failed_sensor_type": { "type": "string", "index": "not_analyzed" }, From 5ffb501c81942977d00e38435d635fc29cd9145b Mon Sep 17 00:00:00 2001 From: justinjleet Date: Thu, 9 Feb 2017 11:43:51 -0500 Subject: [PATCH 12/16] Change dashboard name --- .../package/scripts/dashboard/dashboard.p | 566 +++++++++--------- 1 file changed, 283 insertions(+), 283 deletions(-) diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p index 6e9b9aa5cd..924ec6b1c4 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p @@ -1147,521 +1147,521 @@ p575 F1 sV_type p576 -Vdashboard +Vindex-pattern p577 sV_id p578 -VError-Dashboard +Vsnort* p579 sV_source p580 (dp581 -Vhits -p582 -I0 -sVtimeRestore -p583 -I00 -sVdescription -p584 -V -sVtitle -p585 -VError Dashboard -p586 -sVuiStateJSON -p587 -V{"P-2":{"vis":{"legendOpen":true}}} -p588 -sVpanelsJSON -p589 -V[{"col":5,"id":"Errors-By-Error-Type","panelIndex":2,"row":7,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Source-Proportion","panelIndex":3,"row":7,"size_x":4,"size_y":3,"type":"visualization"},{"col":5,"id":"Errors-By-Source-Type","panelIndex":4,"row":10,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Type-Proportion","panelIndex":5,"row":10,"size_x":4,"size_y":3,"type":"visualization"},{"col":1,"id":"Errors-Over-Time","panelIndex":14,"row":3,"size_x":12,"size_y":4,"type":"visualization"},{"col":8,"id":"Unique-Error-Messages","panelIndex":19,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":3,"id":"Total-Error-Messages","panelIndex":20,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":1,"columns":["failed_sensor_type","error_type","exception","hostname","message","raw_message","error_hash"],"id":"Errors","panelIndex":21,"row":13,"size_x":12,"size_y":7,"sort":["timestamp","desc"],"type":"search"}] -p590 -sVoptionsJSON -p591 -V{"darkTheme":false} -p592 -sVversion -p593 -I1 -sVkibanaSavedObjectMeta -p594 -(dp595 -VsearchSourceJSON -p596 -V{"filter":[{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}}}]} -p597 -sssV_index -p598 -V.kibana -p599 -sa(dp600 -V_score -p601 -F1 -sV_type -p602 -Vindex-pattern -p603 -sV_id -p604 -Vsnort* -p605 -sV_source -p606 -(dp607 Vfields -p608 +p582 V[{"name":"msg","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dgmlen","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentjoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpack","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"protocol","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:threatinteladapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"original_string","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:geoadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ethlen","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threat:triage:level","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:threatinteladapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"ip_dst_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatinteljoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_rev","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"ethsrc","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpseq","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichmentsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpwindow","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_dst_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tos","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ethdst","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"is_alert","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ttl","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"iplen","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_generator","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] -p609 +p583 sVtimeFieldName -p610 +p584 Vtimestamp -p611 +p585 sVtitle -p612 +p586 Vsnort* -p613 +p587 ssV_index -p614 +p588 V.kibana -p615 -sa(dp616 +p589 +sa(dp590 V_score -p617 +p591 F1 sV_type -p618 +p592 Vindex-pattern -p619 +p593 sV_id -p620 +p594 Vyaf* -p621 +p595 sV_source -p622 -(dp623 +p596 +(dp597 Vfields -p624 +p598 V[{"name":"enrichments:geo:ip_dst_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"isn","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentjoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dip","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dp","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"protocol","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"rpkt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"original_string","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:threatinteladapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tag","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"app","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"oct","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end_reason","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichmentsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"start_time","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"riflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"proto","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:threatinteladapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"iflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"ip_dst_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatinteljoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"uflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"duration","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_dst_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"pkt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ruflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"roct","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sip","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sp","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rtag","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end-reason","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"risn","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end_time","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rtt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] -p625 +p599 sVtimeFieldName -p626 +p600 Vtimestamp -p627 +p601 sVtitle -p628 +p602 Vyaf* -p629 +p603 ssV_index -p630 +p604 V.kibana -p631 -sa(dp632 +p605 +sa(dp606 V_score -p633 +p607 F1 sV_type -p634 +p608 Vsearch -p635 +p609 sV_id -p636 +p610 Vweb-search -p637 +p611 sV_source -p638 -(dp639 +p612 +(dp613 Vsort -p640 -(lp641 +p614 +(lp615 Vtimestamp -p642 +p616 aVdesc -p643 +p617 asVhits -p644 +p618 I0 sVdescription -p645 +p619 V sVtitle -p646 +p620 VWeb Requests -p647 +p621 sVversion -p648 +p622 I1 sVkibanaSavedObjectMeta -p649 -(dp650 +p623 +(dp624 VsearchSourceJSON -p651 +p625 V{"index":"bro*","query":{"query_string":{"query":"protocol: http OR protocol: https","analyze_wildcard":true}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} -p652 +p626 ssVcolumns -p653 -(lp654 +p627 +(lp628 Vmethod -p655 +p629 aVhost -p656 +p630 aVuri -p657 +p631 aVreferrer -p658 +p632 aVip_src_addr -p659 +p633 aVip_dst_addr -p660 +p634 assV_index -p661 +p635 V.kibana -p662 -sa(dp663 +p636 +sa(dp637 V_score -p664 +p638 F1 sV_type -p665 +p639 Vvisualization -p666 +p640 sV_id -p667 +p641 VLocation-Header -p668 +p642 sV_source -p669 -(dp670 +p643 +(dp644 VvisState -p671 +p645 V{"title":"Enrichment","type":"markdown","params":{"markdown":"Apache Metron can perform real-time enrichment of telemetry data as it is consumed. To highlight this feature, all of the IP address fields collected from the default sensor suite were used to perform geo-ip lookups. This data was then used to pinpoint each location on the map."},"aggs":[],"listeners":{}} -p672 +p646 sVdescription -p673 +p647 V sVtitle -p674 +p648 VEnrichment -p675 +p649 sVuiStateJSON -p676 +p650 V{} -p677 +p651 sVversion -p678 +p652 I1 sVkibanaSavedObjectMeta -p679 -(dp680 +p653 +(dp654 VsearchSourceJSON -p681 +p655 V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p682 +p656 sssV_index -p683 +p657 V.kibana -p684 -sa(dp685 +p658 +sa(dp659 V_score -p686 +p660 F1 sV_type -p687 +p661 Vvisualization -p688 +p662 sV_id -p689 +p663 VSnort-Alert-Types -p690 +p664 sV_source -p691 -(dp692 +p665 +(dp666 VvisState -p693 +p667 V{"title":"Snort Alert Types","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"cardinality","schema":"metric","params":{"field":"sig_id","customLabel":"Alert Type(s)"}}],"listeners":{}} -p694 +p668 sVdescription -p695 +p669 V sVtitle -p696 +p670 VSnort Alert Types -p697 +p671 sVuiStateJSON -p698 +p672 V{} -p699 +p673 sVversion -p700 +p674 I1 sVkibanaSavedObjectMeta -p701 -(dp702 +p675 +(dp676 VsearchSourceJSON -p703 +p677 V{"index":"snort*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p704 +p678 sssV_index -p705 +p679 V.kibana -p706 -sa(dp707 +p680 +sa(dp681 V_score -p708 +p682 F1 sV_type -p709 +p683 Vvisualization -p710 +p684 sV_id -p711 +p685 VFrequent-DNS-Queries -p712 +p686 sV_source -p713 -(dp714 +p687 +(dp688 VvisState -p715 +p689 V{"title":"Frequent DNS Requests","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} -p716 +p690 sVdescription -p717 +p691 V sVtitle -p718 +p692 VFrequent DNS Requests -p719 +p693 sVuiStateJSON -p720 +p694 V{} -p721 +p695 sVversion -p722 +p696 I1 sVkibanaSavedObjectMeta -p723 -(dp724 +p697 +(dp698 VsearchSourceJSON -p725 +p699 V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p726 +p700 sssV_index -p727 +p701 V.kibana -p728 -sa(dp729 +p702 +sa(dp703 V_score -p730 +p704 F1 sV_type -p731 +p705 Vvisualization -p732 +p706 sV_id -p733 +p707 VDNS-Request(s) -p734 +p708 sV_source -p735 -(dp736 +p709 +(dp710 VvisState -p737 +p711 V{"title":"DNS Requests","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} -p738 +p712 sVdescription -p739 +p713 V sVtitle -p740 +p714 VDNS Requests -p741 +p715 sVuiStateJSON -p742 +p716 V{} -p743 +p717 sVversion -p744 +p718 I1 sVsavedSearchId -p745 +p719 Vdns-search -p746 +p720 sVkibanaSavedObjectMeta -p747 -(dp748 +p721 +(dp722 VsearchSourceJSON -p749 +p723 V{"filter":[]} -p750 +p724 sssV_index -p751 +p725 V.kibana -p752 -sa(dp753 +p726 +sa(dp727 V_score -p754 +p728 F1 sV_type -p755 +p729 Vvisualization -p756 +p730 sV_id -p757 +p731 VHTTP(S)-Requests -p758 +p732 sV_source -p759 -(dp760 +p733 +(dp734 VvisState -p761 +p735 V{"title":"Web Requests","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} -p762 +p736 sVdescription -p763 +p737 V sVtitle -p764 +p738 VWeb Requests -p765 +p739 sVuiStateJSON -p766 +p740 V{} -p767 +p741 sVversion -p768 +p742 I1 sVsavedSearchId -p769 +p743 Vweb-search -p770 +p744 sVkibanaSavedObjectMeta -p771 -(dp772 +p745 +(dp746 VsearchSourceJSON -p773 +p747 V{"filter":[]} -p774 +p748 sssV_index -p775 +p749 V.kibana -p776 -sa(dp777 +p750 +sa(dp751 V_score -p778 +p752 F1 sV_type -p779 +p753 Vvisualization -p780 +p754 sV_id -p781 +p755 VErrors-Over-Time -p782 +p756 sV_source -p783 -(dp784 +p757 +(dp758 VvisState -p785 +p759 V{"title":"Error Over Time","type":"line","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"showCircles":true,"smoothLines":false,"interpolate":"linear","scale":"linear","drawLinesBetweenPoints":true,"radiusRatio":9,"times":[],"addTimeMarker":true,"defaultYExtents":false,"setYExtents":false,"yAxis":{"min":0}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"date_histogram","schema":"segment","params":{"field":"timestamp","interval":"auto","customInterval":"2h","min_doc_count":1,"extended_bounds":{}}}],"listeners":{}} -p786 +p760 sVdescription -p787 +p761 V sVtitle -p788 +p762 VErrors Over Time -p789 +p763 sVuiStateJSON -p790 +p764 V{} -p791 +p765 sVversion -p792 +p766 I1 sVkibanaSavedObjectMeta -p793 -(dp794 +p767 +(dp768 VsearchSourceJSON -p795 +p769 V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p796 +p770 sssV_index -p797 +p771 V.kibana -p798 -sa(dp799 +p772 +sa(dp773 V_score -p800 +p774 F1 sV_type -p801 +p775 Vvisualization -p802 +p776 sV_id -p803 +p777 VError-Source-Proportion -p804 +p778 sV_source -p805 -(dp806 +p779 +(dp780 VvisState -p807 +p781 V{\u000a "title": "Error Sensor Proportion",\u000a "type": "pie",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "isDonut": false\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {}\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "failed_sensor_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1",\u000a "customLabel": "Sensor"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} -p808 +p782 sVdescription -p809 +p783 V sVtitle -p810 +p784 VError Source Proportion -p811 +p785 sVuiStateJSON -p812 +p786 V{} -p813 +p787 sVversion -p814 +p788 I1 sVkibanaSavedObjectMeta -p815 -(dp816 +p789 +(dp790 VsearchSourceJSON -p817 +p791 V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "query": "*",\u000a "analyze_wildcard": true\u000a }\u000a },\u000a "filter": []\u000a} -p818 +p792 sssV_index -p819 +p793 V.kibana -p820 -sa(dp821 +p794 +sa(dp795 V_score -p822 +p796 F1 sV_type -p823 +p797 Vindex-pattern -p824 +p798 sV_id -p825 +p799 Verror* -p826 +p800 sV_source -p827 -(dp828 +p801 +(dp802 Vfields -p829 +p803 V[{"name":"exception","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"stack","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"error_hash","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"raw_message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"failed_sensor_type","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"hostname","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"error_type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"error_fields","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"raw_message_bytes","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] -p830 +p804 sVtimeFieldName -p831 +p805 Vtimestamp -p832 +p806 sVtitle -p833 +p807 Verror* -p834 +p808 ssV_index +p809 +V.kibana +p810 +sa(dp811 +V_score +p812 +F1 +sV_type +p813 +Vdashboard +p814 +sV_id +p815 +VMetron-Error-Dashboard +p816 +sV_source +p817 +(dp818 +Vhits +p819 +I0 +sVtimeRestore +p820 +I00 +sVdescription +p821 +V +sVtitle +p822 +VMetron Error Dashboard +p823 +sVuiStateJSON +p824 +V{"P-2":{"vis":{"legendOpen":true}}} +p825 +sVpanelsJSON +p826 +V[{"col":5,"id":"Errors-By-Error-Type","panelIndex":2,"row":7,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Source-Proportion","panelIndex":3,"row":7,"size_x":4,"size_y":3,"type":"visualization"},{"col":5,"id":"Errors-By-Source-Type","panelIndex":4,"row":10,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Type-Proportion","panelIndex":5,"row":10,"size_x":4,"size_y":3,"type":"visualization"},{"col":1,"id":"Errors-Over-Time","panelIndex":14,"row":3,"size_x":12,"size_y":4,"type":"visualization"},{"col":8,"id":"Unique-Error-Messages","panelIndex":19,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":3,"id":"Total-Error-Messages","panelIndex":20,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":1,"columns":["failed_sensor_type","error_type","exception","hostname","message","raw_message","error_hash"],"id":"Errors","panelIndex":21,"row":13,"size_x":12,"size_y":7,"sort":["timestamp","desc"],"type":"search"}] +p827 +sVoptionsJSON +p828 +V{"darkTheme":false} +p829 +sVversion +p830 +I1 +sVkibanaSavedObjectMeta +p831 +(dp832 +VsearchSourceJSON +p833 +V{"filter":[{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}}}]} +p834 +sssV_index p835 V.kibana p836 From 89bf70756cc963f3e45d1f59d94a5dd2a2012634 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Mon, 27 Feb 2017 09:23:56 -0500 Subject: [PATCH 13/16] Updating template with more final field names, and updating dashboard appropriately --- .../package/scripts/dashboard/dashboard.p | 1308 +++++++++-------- .../package/files/error_index.template | 10 +- 2 files changed, 701 insertions(+), 617 deletions(-) diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p index 924ec6b1c4..49f6806988 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p @@ -326,25 +326,25 @@ Vvisualization p165 sV_id p166 -VWeb-Request-Header +VErrors-By-Hostname p167 sV_source p168 (dp169 VvisState p170 -V{"title":"Web Request Header","type":"markdown","params":{"markdown":"The [Bro Network Security Monitor](https://www.bro.org/) is extracting application-level information from raw network packets. In this example, Bro is extracting HTTP(S) requests being made over the network. "},"aggs":[],"listeners":{}} +V{"title":"Errors By Error Type","type":"histogram","params":{"addLegend":true,"addTimeMarker":false,"addTooltip":true,"defaultYExtents":false,"mode":"grouped","scale":"linear","setYExtents":false,"shareYAxis":true,"times":[],"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{"customLabel":"Count"}},{"id":"2","type":"terms","schema":"segment","params":{"field":"hostname","size":5,"order":"desc","orderBy":"1"}},{"id":"4","type":"cardinality","schema":"metric","params":{"field":"raw_message_hash","customLabel":"Unique Datapoint Count"}}],"listeners":{}} p171 sVdescription p172 V sVtitle p173 -VWeb Request Header +VErrors By Hostname p174 sVuiStateJSON p175 -V{} +V{"vis":{"colors":{"Unique Datapoint Count":"#9AC48A","Count":"#629E51"}}} p176 sVversion p177 @@ -354,7 +354,7 @@ p178 (dp179 VsearchSourceJSON p180 -V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"index":"error*","query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} p181 sssV_index p182 @@ -370,21 +370,21 @@ Vvisualization p187 sV_id p188 -VError-Type-Proportion +VWeb-Request-Header p189 sV_source p190 (dp191 VvisState p192 -V{"title":"Error Type Proportion","type":"pie","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"isDonut":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"error_type","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} +V{"title":"Web Request Header","type":"markdown","params":{"markdown":"The [Bro Network Security Monitor](https://www.bro.org/) is extracting application-level information from raw network packets. In this example, Bro is extracting HTTP(S) requests being made over the network. "},"aggs":[],"listeners":{}} p193 sVdescription p194 V sVtitle p195 -VError Type Proportion +VWeb Request Header p196 sVuiStateJSON p197 @@ -398,7 +398,7 @@ p200 (dp201 VsearchSourceJSON p202 -V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p203 sssV_index p204 @@ -414,25 +414,25 @@ Vvisualization p209 sV_id p210 -VFlow-Duration +VError-Type-Proportion p211 sV_source p212 (dp213 VvisState p214 -V{"title":"Flow Duration","type":"area","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"smoothLines":false,"scale":"linear","interpolate":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"histogram","schema":"segment","params":{"field":"duration","interval":10,"extended_bounds":{},"customLabel":"Flow Duration (seconds)"}}],"listeners":{}} +V{"title":"Error Type Proportion","type":"pie","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"isDonut":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"error_type","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} p215 sVdescription p216 V sVtitle p217 -VFlow Duration +VError Type Proportion p218 sVuiStateJSON p219 -V{"vis":{"legendOpen":false}} +V{} p220 sVversion p221 @@ -442,7 +442,7 @@ p222 (dp223 VsearchSourceJSON p224 -V{"index":"yaf*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p225 sssV_index p226 @@ -458,25 +458,25 @@ Vvisualization p231 sV_id p232 -VErrors-By-Source +VFlow-Duration p233 sV_source p234 (dp235 VvisState p236 -V{"title":"Errors By Source","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"source_type","size":5,"order":"desc","orderBy":"1","customLabel":"Source"}}],"listeners":{}} +V{"title":"Flow Duration","type":"area","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"smoothLines":false,"scale":"linear","interpolate":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"histogram","schema":"segment","params":{"field":"duration","interval":10,"extended_bounds":{},"customLabel":"Flow Duration (seconds)"}}],"listeners":{}} p237 sVdescription p238 V sVtitle p239 -VErrors By Source +VFlow Duration p240 sVuiStateJSON p241 -V{} +V{"vis":{"legendOpen":false}} p242 sVversion p243 @@ -486,7 +486,7 @@ p244 (dp245 VsearchSourceJSON p246 -V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"index":"yaf*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p247 sssV_index p248 @@ -502,25 +502,25 @@ Vvisualization p253 sV_id p254 -VEvents +VErrors-By-Source p255 sV_source p256 (dp257 VvisState p258 -V{"title":"Events","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"date_histogram","schema":"segment","params":{"field":"timestamp","interval":"auto","customInterval":"2h","min_doc_count":1,"extended_bounds":{}}},{"id":"3","type":"terms","schema":"group","params":{"field":"source:type","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} +V{"title":"Errors By Source","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"source_type","size":5,"order":"desc","orderBy":"1","customLabel":"Source"}}],"listeners":{}} p259 sVdescription p260 V sVtitle p261 -VEvents +VErrors By Source p262 sVuiStateJSON p263 -V{"vis":{"legendOpen":false}} +V{} p264 sVversion p265 @@ -530,7 +530,7 @@ p266 (dp267 VsearchSourceJSON p268 -V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p269 sssV_index p270 @@ -546,25 +546,25 @@ Vvisualization p275 sV_id p276 -VUnique-Error-Messages +VEvents p277 sV_source p278 (dp279 VvisState p280 -V{\u000a "title": "Total Unique Error Messages",\u000a "type": "metric",\u000a "params": {\u000a "handleNoResults": true,\u000a "fontSize": 60\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "error_hash",\u000a "customLabel": "Unique Error Messages"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +V{"title":"Events","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"date_histogram","schema":"segment","params":{"field":"timestamp","interval":"auto","customInterval":"2h","min_doc_count":1,"extended_bounds":{}}},{"id":"3","type":"terms","schema":"group","params":{"field":"source:type","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} p281 sVdescription p282 V sVtitle p283 -VUnique Error Messages +VEvents p284 sVuiStateJSON p285 -V{} +V{"vis":{"legendOpen":false}} p286 sVversion p287 @@ -574,7 +574,7 @@ p288 (dp289 VsearchSourceJSON p290 -V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "query": "*",\u000a "analyze_wildcard": true\u000a }\u000a },\u000a "filter": []\u000a} +V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p291 sssV_index p292 @@ -590,25 +590,25 @@ Vvisualization p297 sV_id p298 -VErrors-By-Error-Type +VUnique-Error-Messages p299 sV_source p300 (dp301 VvisState p302 -V{\u000a "title": "Errors By Error Type",\u000a "type": "histogram",\u000a "params": {\u000a "addLegend": true,\u000a "addTimeMarker": false,\u000a "addTooltip": true,\u000a "defaultYExtents": false,\u000a "mode": "grouped",\u000a "scale": "linear",\u000a "setYExtents": false,\u000a "shareYAxis": true,\u000a "times": [],\u000a "yAxis": {}\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {\u000a "customLabel": "Count"\u000a }\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "error_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1"\u000a }\u000a },\u000a {\u000a "id": "4",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "error_hash",\u000a "customLabel": "Unique Datapoint Count"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +V{\u000a "title": "Total Unique Error Messages",\u000a "type": "metric",\u000a "params": {\u000a "handleNoResults": true,\u000a "fontSize": 60\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "raw_message_hash",\u000a "customLabel": "Unique Error Messages"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} p303 sVdescription p304 V sVtitle p305 -VErrors By Error Type +VUnique Error Messages p306 sVuiStateJSON p307 -V{\u000a "vis": {\u000a "colors": {\u000a "Unique Datapoint Count": "#806EB7",\u000a "Count": "#614D93"\u000a }\u000a }\u000a} +V{} p308 sVversion p309 @@ -618,7 +618,7 @@ p310 (dp311 VsearchSourceJSON p312 -V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "analyze_wildcard": true,\u000a "query": "*"\u000a }\u000a },\u000a "filter": []\u000a} +V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "query": "*",\u000a "analyze_wildcard": true\u000a }\u000a },\u000a "filter": []\u000a} p313 sssV_index p314 @@ -630,190 +630,190 @@ p317 F1 sV_type p318 -Vsearch +Vvisualization p319 sV_id p320 -VErrors +VErrors-By-Error-Type p321 sV_source p322 (dp323 -Vsort +VvisState p324 -(lp325 -Vtimestamp +V{\u000a "title": "Errors By Error Type",\u000a "type": "histogram",\u000a "params": {\u000a "addLegend": true,\u000a "addTimeMarker": false,\u000a "addTooltip": true,\u000a "defaultYExtents": false,\u000a "mode": "grouped",\u000a "scale": "linear",\u000a "setYExtents": false,\u000a "shareYAxis": true,\u000a "times": [],\u000a "yAxis": {}\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {\u000a "customLabel": "Count"\u000a }\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "error_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1"\u000a }\u000a },\u000a {\u000a "id": "4",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "raw_message_hash",\u000a "customLabel": "Unique Datapoint Count"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +p325 +sVdescription p326 -aVdesc +V +sVtitle p327 -asVhits +VErrors By Error Type p328 -I0 -sVdescription +sVuiStateJSON p329 -V -sVtitle +V{\u000a "vis": {\u000a "colors": {\u000a "Unique Datapoint Count": "#806EB7",\u000a "Count": "#614D93"\u000a }\u000a }\u000a} p330 -VErrors -p331 sVversion -p332 +p331 I1 sVkibanaSavedObjectMeta -p333 -(dp334 +p332 +(dp333 VsearchSourceJSON +p334 +V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "analyze_wildcard": true,\u000a "query": "*"\u000a }\u000a },\u000a "filter": []\u000a} p335 -V{"index":"error*","query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} +sssV_index p336 -ssVcolumns +V.kibana p337 -(lp338 -Vfailed_sensor_type +sa(dp338 +V_score p339 -aVerror_type +F1 +sV_type p340 -aVexception +Vvisualization p341 -aVhostname +sV_id p342 -aVmessage +VError-Hostname-Proportion p343 -aVraw_message +sV_source p344 -aVerror_hash -p345 -assV_index +(dp345 +VvisState p346 -V.kibana +V{"aggs":[{"id":"1","params":{},"schema":"metric","type":"count"},{"id":"2","params":{"customLabel":"Sensor","field":"hostname","order":"desc","orderBy":"1","size":5},"schema":"segment","type":"terms"}],"listeners":{},"params":{"addLegend":true,"addTooltip":true,"isDonut":false,"shareYAxis":true},"title":"Error Source Proportion","type":"pie"} p347 -sa(dp348 -V_score +sVdescription +p348 +V +sVtitle p349 -F1 -sV_type +VError Hostname Proportion p350 -Vdashboard +sVuiStateJSON p351 -sV_id +V{"vis":{"colors":{"host":"#629E51","host2":"#9AC48A","hostAnother":"#7EB26D","hostNew":"#B7DBAB"}}} p352 -VMetron-Dashboard +sVversion p353 -sV_source +I1 +sVkibanaSavedObjectMeta p354 (dp355 -Vhits +VsearchSourceJSON p356 -I0 -sVtimeRestore +V{"index":"error*","query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} p357 -I00 -sVdescription +sssV_index p358 -V -sVtitle +V.kibana p359 -VMetron Dashboard -p360 -sVuiStateJSON +sa(dp360 +V_score p361 -V{"P-23":{"spy":{"mode":{"name":null,"fill":false}}},"P-34":{"vis":{"legendOpen":false}}} +F1 +sV_type p362 -sVpanelsJSON +Vsearch p363 -V[{"col":1,"id":"Welcome","panelIndex":30,"row":1,"size_x":11,"size_y":2,"type":"visualization"},{"col":1,"id":"Total-Events","panelIndex":6,"row":3,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"id":"Events","panelIndex":16,"row":3,"size_x":8,"size_y":4,"type":"visualization"},{"col":1,"id":"Event-Types","panelIndex":15,"row":5,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Location-Header","panelIndex":24,"row":7,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Unique-Location(s)","panelIndex":23,"row":9,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"id":"Flow-Locations","panelIndex":32,"row":7,"size_x":8,"size_y":6,"type":"visualization"},{"col":1,"id":"Country","panelIndex":8,"row":11,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"YAF-Flows-Header","panelIndex":27,"row":13,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"YAF-Flow(s)","panelIndex":21,"row":15,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["ip_src_addr","ip_src_port","ip_dst_addr","ip_dst_port","protocol","duration","pkt"],"id":"yaf-search","panelIndex":20,"row":13,"size_x":8,"size_y":6,"sort":["duration","desc"],"type":"search"},{"col":1,"id":"Flow-Duration","panelIndex":31,"row":17,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Snort-Header","panelIndex":25,"row":19,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["msg","sig_id","ip_src_addr","ip_src_port","ip_dst_addr","ip_dst_port"],"id":"snort-search","panelIndex":3,"row":19,"size_x":8,"size_y":6,"sort":["timestamp","desc"],"type":"search"},{"col":1,"id":"Snort-Alert-Types","panelIndex":10,"row":21,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Top-Alerts-By-Host","panelIndex":19,"row":23,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Web-Request-Header","panelIndex":26,"row":25,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["method","host","uri","referrer","user_agent","ip_src_addr","ip_dst_addr"],"id":"web-search","panelIndex":4,"row":25,"size_x":8,"size_y":6,"sort":["timestamp","desc"],"type":"search"},{"col":1,"id":"HTTP(S)-Requests","panelIndex":17,"row":27,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"DNS-Requests-Header","panelIndex":29,"row":31,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["query","qtype_name","answers","ip_src_addr","ip_dst_addr"],"id":"dns-search","panelIndex":5,"row":31,"size_x":8,"size_y":6,"sort":["timestamp","desc"],"type":"search"},{"col":1,"id":"DNS-Request(s)","panelIndex":14,"row":33,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Web-Request-Type","panelIndex":33,"row":29,"size_x":3,"size_y":2,"type":"visualization"}] +sV_id p364 -sVoptionsJSON +VErrors p365 -V{"darkTheme":false} +sV_source p366 -sVversion -p367 -I1 -sVkibanaSavedObjectMeta +(dp367 +Vsort p368 -(dp369 -VsearchSourceJSON +(lp369 +Vtime p370 -V{"filter":[{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}}}]} +aVdesc p371 -sssV_index +asVhits p372 -V.kibana +I0 +sVdescription p373 -sa(dp374 -V_score +V +sVtitle +p374 +VErrors p375 -F1 -sV_type +sVversion p376 -Vvisualization +I1 +sVkibanaSavedObjectMeta p377 -sV_id -p378 -VSnort-Header +(dp378 +VsearchSourceJSON p379 -sV_source +V{"index":"error*","query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} p380 -(dp381 -VvisState -p382 -V{"title":"Snort","type":"markdown","params":{"markdown":"[Snort](https://www.snort.org/) is a Network Intrusion Detection System (NIDS) that is being used to generate alerts identifying known bad events. Snort relies on a fixed set of rules that act as signatures for identifying abnormal events."},"aggs":[],"listeners":{}} +ssVcolumns +p381 +(lp382 +Vsource_type p383 -sVdescription +aVerror_type p384 -V -sVtitle +aVexception p385 -VSnort +aVhostname p386 -sVuiStateJSON +aVmessage p387 -V{} +aVraw_message p388 -sVversion +aVraw_message_hash p389 -I1 -sVkibanaSavedObjectMeta +assV_index p390 -(dp391 -VsearchSourceJSON -p392 -V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p393 -sssV_index -p394 V.kibana -p395 -sa(dp396 +p391 +sa(dp392 V_score -p397 +p393 F1 sV_type -p398 -Vvisualization -p399 +p394 +Vdashboard +p395 sV_id +p396 +VMetron-Dashboard +p397 +sV_source +p398 +(dp399 +Vhits p400 -VYAF-Flow(s) +I0 +sVtimeRestore p401 -sV_source +I00 +sVdescription p402 -(dp403 -VvisState +V +sVtitle +p403 +VMetron Dashboard p404 -V{"title":"YAF Flows","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} +sVuiStateJSON p405 -sVdescription +V{"P-23":{"spy":{"mode":{"name":null,"fill":false}}},"P-34":{"vis":{"legendOpen":false}}} p406 -V -sVtitle +sVpanelsJSON p407 -VYAF Flows +V[{"col":1,"id":"Welcome","panelIndex":30,"row":1,"size_x":11,"size_y":2,"type":"visualization"},{"col":1,"id":"Total-Events","panelIndex":6,"row":3,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"id":"Events","panelIndex":16,"row":3,"size_x":8,"size_y":4,"type":"visualization"},{"col":1,"id":"Event-Types","panelIndex":15,"row":5,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Location-Header","panelIndex":24,"row":7,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Unique-Location(s)","panelIndex":23,"row":9,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"id":"Flow-Locations","panelIndex":32,"row":7,"size_x":8,"size_y":6,"type":"visualization"},{"col":1,"id":"Country","panelIndex":8,"row":11,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"YAF-Flows-Header","panelIndex":27,"row":13,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"YAF-Flow(s)","panelIndex":21,"row":15,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["ip_src_addr","ip_src_port","ip_dst_addr","ip_dst_port","protocol","duration","pkt"],"id":"yaf-search","panelIndex":20,"row":13,"size_x":8,"size_y":6,"sort":["duration","desc"],"type":"search"},{"col":1,"id":"Flow-Duration","panelIndex":31,"row":17,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Snort-Header","panelIndex":25,"row":19,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["msg","sig_id","ip_src_addr","ip_src_port","ip_dst_addr","ip_dst_port"],"id":"snort-search","panelIndex":3,"row":19,"size_x":8,"size_y":6,"sort":["timestamp","desc"],"type":"search"},{"col":1,"id":"Snort-Alert-Types","panelIndex":10,"row":21,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Top-Alerts-By-Host","panelIndex":19,"row":23,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Web-Request-Header","panelIndex":26,"row":25,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["method","host","uri","referrer","user_agent","ip_src_addr","ip_dst_addr"],"id":"web-search","panelIndex":4,"row":25,"size_x":8,"size_y":6,"sort":["timestamp","desc"],"type":"search"},{"col":1,"id":"HTTP(S)-Requests","panelIndex":17,"row":27,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"DNS-Requests-Header","panelIndex":29,"row":31,"size_x":3,"size_y":2,"type":"visualization"},{"col":4,"columns":["query","qtype_name","answers","ip_src_addr","ip_dst_addr"],"id":"dns-search","panelIndex":5,"row":31,"size_x":8,"size_y":6,"sort":["timestamp","desc"],"type":"search"},{"col":1,"id":"DNS-Request(s)","panelIndex":14,"row":33,"size_x":3,"size_y":2,"type":"visualization"},{"col":1,"id":"Web-Request-Type","panelIndex":33,"row":29,"size_x":3,"size_y":2,"type":"visualization"}] p408 -sVuiStateJSON +sVoptionsJSON p409 -V{} +V{"darkTheme":false} p410 sVversion p411 @@ -823,7 +823,7 @@ p412 (dp413 VsearchSourceJSON p414 -V{"index":"yaf*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"filter":[{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}}}]} p415 sssV_index p416 @@ -839,21 +839,21 @@ Vvisualization p421 sV_id p422 -VTop-DNS-Query +VSnort-Header p423 sV_source p424 (dp425 VvisState p426 -V{"title":"Top DNS Query","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":10,"order":"desc","orderBy":"1"}}],"listeners":{}} +V{"title":"Snort","type":"markdown","params":{"markdown":"[Snort](https://www.snort.org/) is a Network Intrusion Detection System (NIDS) that is being used to generate alerts identifying known bad events. Snort relies on a fixed set of rules that act as signatures for identifying abnormal events."},"aggs":[],"listeners":{}} p427 sVdescription p428 V sVtitle p429 -VTop DNS Query +VSnort p430 sVuiStateJSON p431 @@ -867,7 +867,7 @@ p434 (dp435 VsearchSourceJSON p436 -V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p437 sssV_index p438 @@ -883,21 +883,21 @@ Vvisualization p443 sV_id p444 -VEvent-Types +VYAF-Flow(s) p445 sV_source p446 (dp447 VvisState p448 -V{"title":"Event Sources","type":"pie","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"isDonut":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"source:type","size":10,"order":"desc","orderBy":"1"}}],"listeners":{}} +V{"title":"YAF Flows","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} p449 sVdescription p450 V sVtitle p451 -VEvent Sources +VYAF Flows p452 sVuiStateJSON p453 @@ -911,7 +911,7 @@ p456 (dp457 VsearchSourceJSON p458 -V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"index":"yaf*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p459 sssV_index p460 @@ -927,21 +927,21 @@ Vvisualization p465 sV_id p466 -VTotal-Events +VTop-DNS-Query p467 sV_source p468 (dp469 VvisState p470 -V{"title":"Event Count","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{"customLabel":"Events"}}],"listeners":{}} +V{"title":"Top DNS Query","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":10,"order":"desc","orderBy":"1"}}],"listeners":{}} p471 sVdescription p472 V sVtitle p473 -VEvent Count +VTop DNS Query p474 sVuiStateJSON p475 @@ -955,7 +955,7 @@ p478 (dp479 VsearchSourceJSON p480 -V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p481 sssV_index p482 @@ -971,21 +971,21 @@ Vvisualization p487 sV_id p488 -VUnique-Location(s) +VEvent-Types p489 sV_source p490 (dp491 VvisState p492 -V{"title":"Geo-IP Locations","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"cardinality","schema":"metric","params":{"field":"enrichments:geo:ip_src_addr:locID","customLabel":"Unique Location(s)"}}],"listeners":{}} +V{"title":"Event Sources","type":"pie","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"isDonut":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"source:type","size":10,"order":"desc","orderBy":"1"}}],"listeners":{}} p493 sVdescription p494 V sVtitle p495 -VGeo-IP Locations +VEvent Sources p496 sVuiStateJSON p497 @@ -1015,21 +1015,21 @@ Vvisualization p509 sV_id p510 -VTop-Alerts-By-Host +VTotal-Events p511 sV_source p512 (dp513 VvisState p514 -V{"title":"Top Alerts By Host","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"ip_src_addr","size":5,"order":"desc","orderBy":"1","customLabel":"Source"}},{"id":"3","type":"terms","schema":"bucket","params":{"field":"ip_dst_addr","size":5,"order":"desc","orderBy":"1","customLabel":"Destination"}}],"listeners":{}} +V{"title":"Event Count","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{"customLabel":"Events"}}],"listeners":{}} p515 sVdescription p516 V sVtitle p517 -VTop Alerts By Host +VEvent Count p518 sVuiStateJSON p519 @@ -1038,104 +1038,104 @@ p520 sVversion p521 I1 -sVsavedSearchId -p522 -Vsnort-search -p523 sVkibanaSavedObjectMeta -p524 -(dp525 +p522 +(dp523 VsearchSourceJSON -p526 -V{"filter":[]} -p527 +p524 +V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p525 sssV_index -p528 +p526 V.kibana -p529 -sa(dp530 +p527 +sa(dp528 V_score -p531 +p529 F1 sV_type -p532 +p530 Vvisualization -p533 +p531 sV_id -p534 -VTotal-Error-Messages -p535 +p532 +VUnique-Location(s) +p533 sV_source -p536 -(dp537 +p534 +(dp535 VvisState -p538 -V{"title":"Total Errored Messages","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{"customLabel":"Total Error Messages"}}],"listeners":{}} -p539 +p536 +V{"title":"Geo-IP Locations","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"cardinality","schema":"metric","params":{"field":"enrichments:geo:ip_src_addr:locID","customLabel":"Unique Location(s)"}}],"listeners":{}} +p537 sVdescription -p540 +p538 V sVtitle -p541 -VTotal Error Messages -p542 +p539 +VGeo-IP Locations +p540 sVuiStateJSON -p543 +p541 V{} -p544 +p542 sVversion -p545 +p543 I1 sVkibanaSavedObjectMeta -p546 -(dp547 +p544 +(dp545 VsearchSourceJSON -p548 -V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p549 +p546 +V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p547 sssV_index -p550 +p548 V.kibana -p551 -sa(dp552 +p549 +sa(dp550 V_score -p553 +p551 F1 sV_type -p554 +p552 Vvisualization -p555 +p553 sV_id -p556 -VErrors-By-Source-Type -p557 +p554 +VTop-Alerts-By-Host +p555 sV_source -p558 -(dp559 +p556 +(dp557 VvisState -p560 -V{\u000a "title": "Errors By Source Type",\u000a "type": "histogram",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "scale": "linear",\u000a "mode": "grouped",\u000a "times": [],\u000a "addTimeMarker": false,\u000a "defaultYExtents": false,\u000a "setYExtents": false,\u000a "yAxis": {}\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {\u000a "customLabel": "Count"\u000a }\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "failed_sensor_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1"\u000a }\u000a },\u000a {\u000a "id": "4",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "error_hash",\u000a "customLabel": "Unique Datapoint Count"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} -p561 +p558 +V{"title":"Top Alerts By Host","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"ip_src_addr","size":5,"order":"desc","orderBy":"1","customLabel":"Source"}},{"id":"3","type":"terms","schema":"bucket","params":{"field":"ip_dst_addr","size":5,"order":"desc","orderBy":"1","customLabel":"Destination"}}],"listeners":{}} +p559 sVdescription -p562 +p560 V sVtitle +p561 +VTop Alerts By Host +p562 +sVuiStateJSON p563 -VErrors By Source Type +V{} p564 -sVuiStateJSON +sVversion p565 -V{\u000a "vis": {\u000a "colors": {\u000a "Unique Datapoint Count": "#0A50A1",\u000a "Count": "#5195CE"\u000a }\u000a }\u000a} +I1 +sVsavedSearchId p566 -sVversion +Vsnort-search p567 -I1 sVkibanaSavedObjectMeta p568 (dp569 VsearchSourceJSON p570 -V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "analyze_wildcard": true,\u000a "query": "*"\u000a }\u000a },\u000a "filter": []\u000a} +V{"filter":[]} p571 sssV_index p572 @@ -1147,208 +1147,208 @@ p575 F1 sV_type p576 -Vindex-pattern +Vvisualization p577 sV_id p578 -Vsnort* +VTotal-Error-Messages p579 sV_source p580 (dp581 -Vfields +VvisState p582 -V[{"name":"msg","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dgmlen","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentjoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpack","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"protocol","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:threatinteladapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"original_string","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:geoadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ethlen","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threat:triage:level","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:threatinteladapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"ip_dst_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatinteljoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_rev","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"ethsrc","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpseq","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichmentsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpwindow","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_dst_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tos","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ethdst","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"is_alert","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ttl","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"iplen","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_generator","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] +V{"title":"Total Errored Messages","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{"customLabel":"Total Error Messages"}}],"listeners":{}} p583 -sVtimeFieldName +sVdescription p584 -Vtimestamp -p585 +V sVtitle +p585 +VTotal Error Messages p586 -Vsnort* +sVuiStateJSON p587 -ssV_index +V{} p588 -V.kibana +sVversion p589 -sa(dp590 -V_score -p591 -F1 -sV_type +I1 +sVkibanaSavedObjectMeta +p590 +(dp591 +VsearchSourceJSON p592 -Vindex-pattern +V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p593 -sV_id +sssV_index p594 -Vyaf* +V.kibana p595 -sV_source -p596 -(dp597 -Vfields +sa(dp596 +V_score +p597 +F1 +sV_type p598 -V[{"name":"enrichments:geo:ip_dst_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"isn","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentjoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dip","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dp","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"protocol","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"rpkt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"original_string","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:threatinteladapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tag","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"app","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"oct","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end_reason","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichmentsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"start_time","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"riflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"proto","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:threatinteladapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"iflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"ip_dst_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatinteljoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"uflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"duration","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_dst_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"pkt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ruflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"roct","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sip","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sp","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rtag","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end-reason","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"risn","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end_time","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rtt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] +Vvisualization p599 -sVtimeFieldName +sV_id p600 -Vtimestamp +VErrors-By-Source-Type p601 -sVtitle +sV_source p602 -Vyaf* -p603 -ssV_index +(dp603 +VvisState p604 -V.kibana +V{\u000a "title": "Errors By Source Type",\u000a "type": "histogram",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "scale": "linear",\u000a "mode": "grouped",\u000a "times": [],\u000a "addTimeMarker": false,\u000a "defaultYExtents": false,\u000a "setYExtents": false,\u000a "yAxis": {}\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {\u000a "customLabel": "Count"\u000a }\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "source_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1"\u000a }\u000a },\u000a {\u000a "id": "4",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "raw_message_hash",\u000a "customLabel": "Unique Datapoint Count"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} p605 -sa(dp606 -V_score +sVdescription +p606 +V +sVtitle p607 -F1 -sV_type +VErrors By Source Type p608 -Vsearch +sVuiStateJSON p609 -sV_id +V{\u000a "vis": {\u000a "colors": {\u000a "Unique Datapoint Count": "#0A50A1",\u000a "Count": "#5195CE"\u000a }\u000a }\u000a} p610 -Vweb-search +sVversion p611 -sV_source +I1 +sVkibanaSavedObjectMeta p612 (dp613 -Vsort +VsearchSourceJSON p614 -(lp615 -Vtimestamp +V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "analyze_wildcard": true,\u000a "query": "*"\u000a }\u000a },\u000a "filter": []\u000a} +p615 +sssV_index p616 -aVdesc +V.kibana p617 -asVhits -p618 -I0 -sVdescription +sa(dp618 +V_score p619 -V -sVtitle +F1 +sV_type p620 -VWeb Requests +Vindex-pattern p621 -sVversion +sV_id p622 -I1 -sVkibanaSavedObjectMeta +Vsnort* p623 -(dp624 -VsearchSourceJSON -p625 -V{"index":"bro*","query":{"query_string":{"query":"protocol: http OR protocol: https","analyze_wildcard":true}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} +sV_source +p624 +(dp625 +Vfields p626 -ssVcolumns +V[{"name":"msg","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dgmlen","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentjoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpack","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"protocol","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:threatinteladapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"original_string","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:geoadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ethlen","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threat:triage:level","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:threatinteladapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"ip_dst_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatinteljoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_rev","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"ethsrc","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpseq","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichmentsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpwindow","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_dst_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tos","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ethdst","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"is_alert","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ttl","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"iplen","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_generator","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] p627 -(lp628 -Vmethod +sVtimeFieldName +p628 +Vtimestamp p629 -aVhost +sVtitle p630 -aVuri +Vsnort* p631 -aVreferrer +ssV_index p632 -aVip_src_addr -p633 -aVip_dst_addr -p634 -assV_index -p635 V.kibana -p636 -sa(dp637 +p633 +sa(dp634 V_score -p638 +p635 F1 sV_type +p636 +Vindex-pattern +p637 +sV_id +p638 +Vyaf* p639 -Vvisualization +sV_source p640 -sV_id -p641 -VLocation-Header +(dp641 +Vfields p642 -sV_source +V[{"name":"enrichments:geo:ip_dst_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"isn","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentjoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dip","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dp","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"protocol","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"rpkt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"original_string","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:threatinteladapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tag","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"app","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"oct","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end_reason","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichmentsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"start_time","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"riflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"proto","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:threatinteladapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"iflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"ip_dst_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatinteljoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"uflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"duration","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_dst_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"pkt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ruflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"roct","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sip","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sp","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rtag","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end-reason","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"risn","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end_time","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rtt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] p643 -(dp644 -VvisState +sVtimeFieldName +p644 +Vtimestamp p645 -V{"title":"Enrichment","type":"markdown","params":{"markdown":"Apache Metron can perform real-time enrichment of telemetry data as it is consumed. To highlight this feature, all of the IP address fields collected from the default sensor suite were used to perform geo-ip lookups. This data was then used to pinpoint each location on the map."},"aggs":[],"listeners":{}} +sVtitle p646 -sVdescription +Vyaf* p647 -V -sVtitle +ssV_index p648 -VEnrichment +V.kibana p649 -sVuiStateJSON -p650 -V{} +sa(dp650 +V_score p651 -sVversion +F1 +sV_type p652 -I1 -sVkibanaSavedObjectMeta +Vsearch p653 -(dp654 -VsearchSourceJSON +sV_id +p654 +Vweb-search p655 -V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +sV_source p656 -sssV_index -p657 -V.kibana +(dp657 +Vsort p658 -sa(dp659 -V_score +(lp659 +Vtimestamp p660 -F1 -sV_type +aVdesc p661 -Vvisualization +asVhits p662 -sV_id +I0 +sVdescription p663 -VSnort-Alert-Types +V +sVtitle p664 -sV_source +VWeb Requests p665 -(dp666 -VvisState +sVversion +p666 +I1 +sVkibanaSavedObjectMeta p667 -V{"title":"Snort Alert Types","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"cardinality","schema":"metric","params":{"field":"sig_id","customLabel":"Alert Type(s)"}}],"listeners":{}} -p668 -sVdescription +(dp668 +VsearchSourceJSON p669 -V -sVtitle +V{"index":"bro*","query":{"query_string":{"query":"protocol: http OR protocol: https","analyze_wildcard":true}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} p670 -VSnort Alert Types +ssVcolumns p671 -sVuiStateJSON -p672 -V{} +(lp672 +Vmethod p673 -sVversion +aVhost p674 -I1 -sVkibanaSavedObjectMeta +aVuri p675 -(dp676 -VsearchSourceJSON +aVreferrer +p676 +aVip_src_addr p677 -V{"index":"snort*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +aVip_dst_addr p678 -sssV_index +assV_index p679 V.kibana p680 @@ -1362,21 +1362,21 @@ Vvisualization p684 sV_id p685 -VFrequent-DNS-Queries +VLocation-Header p686 sV_source p687 (dp688 VvisState p689 -V{"title":"Frequent DNS Requests","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} +V{"title":"Enrichment","type":"markdown","params":{"markdown":"Apache Metron can perform real-time enrichment of telemetry data as it is consumed. To highlight this feature, all of the IP address fields collected from the default sensor suite were used to perform geo-ip lookups. This data was then used to pinpoint each location on the map."},"aggs":[],"listeners":{}} p690 sVdescription p691 V sVtitle p692 -VFrequent DNS Requests +VEnrichment p693 sVuiStateJSON p694 @@ -1390,7 +1390,7 @@ p697 (dp698 VsearchSourceJSON p699 -V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p700 sssV_index p701 @@ -1406,21 +1406,21 @@ Vvisualization p706 sV_id p707 -VDNS-Request(s) +VSnort-Alert-Types p708 sV_source p709 (dp710 VvisState p711 -V{"title":"DNS Requests","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} +V{"title":"Snort Alert Types","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"cardinality","schema":"metric","params":{"field":"sig_id","customLabel":"Alert Type(s)"}}],"listeners":{}} p712 sVdescription p713 V sVtitle p714 -VDNS Requests +VSnort Alert Types p715 sVuiStateJSON p716 @@ -1429,152 +1429,152 @@ p717 sVversion p718 I1 -sVsavedSearchId -p719 -Vdns-search -p720 sVkibanaSavedObjectMeta -p721 -(dp722 +p719 +(dp720 VsearchSourceJSON -p723 -V{"filter":[]} -p724 +p721 +V{"index":"snort*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p722 sssV_index -p725 +p723 V.kibana -p726 -sa(dp727 +p724 +sa(dp725 V_score -p728 +p726 F1 sV_type -p729 +p727 Vvisualization -p730 +p728 sV_id -p731 -VHTTP(S)-Requests -p732 +p729 +VFrequent-DNS-Queries +p730 sV_source -p733 -(dp734 +p731 +(dp732 VvisState -p735 -V{"title":"Web Requests","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} -p736 +p733 +V{"title":"Frequent DNS Requests","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} +p734 sVdescription -p737 +p735 V sVtitle -p738 -VWeb Requests -p739 +p736 +VFrequent DNS Requests +p737 sVuiStateJSON -p740 +p738 V{} -p741 +p739 sVversion -p742 +p740 I1 -sVsavedSearchId -p743 -Vweb-search -p744 sVkibanaSavedObjectMeta -p745 -(dp746 +p741 +(dp742 VsearchSourceJSON -p747 -V{"filter":[]} -p748 +p743 +V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p744 sssV_index -p749 +p745 V.kibana -p750 -sa(dp751 +p746 +sa(dp747 V_score -p752 +p748 F1 sV_type -p753 +p749 Vvisualization -p754 +p750 sV_id -p755 -VErrors-Over-Time -p756 +p751 +VDNS-Request(s) +p752 sV_source -p757 -(dp758 +p753 +(dp754 VvisState -p759 -V{"title":"Error Over Time","type":"line","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"showCircles":true,"smoothLines":false,"interpolate":"linear","scale":"linear","drawLinesBetweenPoints":true,"radiusRatio":9,"times":[],"addTimeMarker":true,"defaultYExtents":false,"setYExtents":false,"yAxis":{"min":0}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"date_histogram","schema":"segment","params":{"field":"timestamp","interval":"auto","customInterval":"2h","min_doc_count":1,"extended_bounds":{}}}],"listeners":{}} -p760 +p755 +V{"title":"DNS Requests","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} +p756 sVdescription -p761 +p757 V sVtitle -p762 -VErrors Over Time -p763 +p758 +VDNS Requests +p759 sVuiStateJSON -p764 +p760 V{} -p765 +p761 sVversion -p766 +p762 I1 +sVsavedSearchId +p763 +Vdns-search +p764 sVkibanaSavedObjectMeta -p767 -(dp768 +p765 +(dp766 VsearchSourceJSON -p769 -V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p770 +p767 +V{"filter":[]} +p768 sssV_index -p771 +p769 V.kibana -p772 -sa(dp773 +p770 +sa(dp771 V_score -p774 +p772 F1 sV_type -p775 +p773 Vvisualization -p776 +p774 sV_id -p777 -VError-Source-Proportion -p778 +p775 +VHTTP(S)-Requests +p776 sV_source -p779 -(dp780 +p777 +(dp778 VvisState -p781 -V{\u000a "title": "Error Sensor Proportion",\u000a "type": "pie",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "isDonut": false\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {}\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "failed_sensor_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1",\u000a "customLabel": "Sensor"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} -p782 +p779 +V{"title":"Web Requests","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} +p780 sVdescription -p783 +p781 V sVtitle +p782 +VWeb Requests +p783 +sVuiStateJSON p784 -VError Source Proportion +V{} p785 -sVuiStateJSON +sVversion p786 -V{} +I1 +sVsavedSearchId p787 -sVversion +Vweb-search p788 -I1 sVkibanaSavedObjectMeta p789 (dp790 VsearchSourceJSON p791 -V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "query": "*",\u000a "analyze_wildcard": true\u000a }\u000a },\u000a "filter": []\u000a} +V{"filter":[]} p792 sssV_index p793 @@ -1586,255 +1586,255 @@ p796 F1 sV_type p797 -Vindex-pattern +Vvisualization p798 sV_id p799 -Verror* +VError-Source-Proportion p800 sV_source p801 (dp802 -Vfields +VvisState p803 -V[{"name":"exception","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"stack","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"error_hash","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"raw_message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"failed_sensor_type","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"hostname","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"error_type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"error_fields","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"raw_message_bytes","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] +V{\u000a "title": "Sensor Type Proportion",\u000a "type": "pie",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "isDonut": false\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {}\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "source_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1",\u000a "customLabel": "Sensor"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} p804 -sVtimeFieldName +sVdescription p805 -Vtimestamp -p806 +V sVtitle +p806 +VError Source Proportion p807 -Verror* +sVuiStateJSON p808 -ssV_index +V{} p809 -V.kibana +sVversion p810 -sa(dp811 -V_score -p812 -F1 -sV_type +I1 +sVkibanaSavedObjectMeta +p811 +(dp812 +VsearchSourceJSON p813 -Vdashboard +V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "query": "*",\u000a "analyze_wildcard": true\u000a }\u000a },\u000a "filter": []\u000a} p814 -sV_id +sssV_index p815 -VMetron-Error-Dashboard +V.kibana p816 -sV_source -p817 -(dp818 -Vhits +sa(dp817 +V_score +p818 +F1 +sV_type p819 -I0 -sVtimeRestore +Vvisualization p820 -I00 -sVdescription +sV_id p821 -V -sVtitle +VErrors-Over-Time p822 -VMetron Error Dashboard +sV_source p823 -sVuiStateJSON -p824 -V{"P-2":{"vis":{"legendOpen":true}}} +(dp824 +VvisState p825 -sVpanelsJSON +V{\u000a "title": "Error Over Time",\u000a "type": "line",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "showCircles": true,\u000a "smoothLines": false,\u000a "interpolate": "linear",\u000a "scale": "linear",\u000a "drawLinesBetweenPoints": true,\u000a "radiusRatio": 9,\u000a "times": [],\u000a "addTimeMarker": true,\u000a "defaultYExtents": false,\u000a "setYExtents": false,\u000a "yAxis": {\u000a "min": 0\u000a }\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {}\u000a },\u000a {\u000a "id": "2",\u000a "type": "date_histogram",\u000a "schema": "segment",\u000a "params": {\u000a "field": "time",\u000a "interval": "auto",\u000a "customInterval": "2h",\u000a "min_doc_count": 1,\u000a "extended_bounds": {}\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} p826 -V[{"col":5,"id":"Errors-By-Error-Type","panelIndex":2,"row":7,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Source-Proportion","panelIndex":3,"row":7,"size_x":4,"size_y":3,"type":"visualization"},{"col":5,"id":"Errors-By-Source-Type","panelIndex":4,"row":10,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Type-Proportion","panelIndex":5,"row":10,"size_x":4,"size_y":3,"type":"visualization"},{"col":1,"id":"Errors-Over-Time","panelIndex":14,"row":3,"size_x":12,"size_y":4,"type":"visualization"},{"col":8,"id":"Unique-Error-Messages","panelIndex":19,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":3,"id":"Total-Error-Messages","panelIndex":20,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":1,"columns":["failed_sensor_type","error_type","exception","hostname","message","raw_message","error_hash"],"id":"Errors","panelIndex":21,"row":13,"size_x":12,"size_y":7,"sort":["timestamp","desc"],"type":"search"}] +sVdescription p827 -sVoptionsJSON +V +sVtitle p828 -V{"darkTheme":false} +VErrors Over Time p829 -sVversion +sVuiStateJSON p830 +V{} +p831 +sVversion +p832 I1 sVkibanaSavedObjectMeta -p831 -(dp832 -VsearchSourceJSON p833 -V{"filter":[{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}}}]} -p834 -sssV_index +(dp834 +VsearchSourceJSON p835 +V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "query": "*",\u000a "analyze_wildcard": true\u000a }\u000a },\u000a "filter": []\u000a} +p836 +sssV_index +p837 V.kibana -p836 -sa(dp837 -V_score p838 +sa(dp839 +V_score +p840 F1 sV_type -p839 -Vconfig -p840 -sV_id p841 -V4.5.3 +Vindex-pattern p842 -sV_source +sV_id p843 -(dp844 -VbuildNum +Verror* +p844 +sV_source p845 -I9892 -sVdefaultIndex -p846 -Vbro* +(dp846 +Vfields p847 -ssV_index +V[{"name":"exception","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"stack","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"raw_message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source_type","type":"string","count":2,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"hostname","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"error_type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"raw_message_bytes","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"time","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"raw_message_hash","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] p848 -V.kibana +sVtimeFieldName p849 -sa(dp850 -V_score +Vtime +p850 +sVtitle p851 -F1 -sV_type +Verror* p852 -Vsearch +ssV_index p853 -sV_id +V.kibana p854 -Vdns-search -p855 -sV_source +sa(dp855 +V_score p856 -(dp857 -Vsort +F1 +sV_type +p857 +Vdashboard p858 -(lp859 -Vtimestamp +sV_id +p859 +VMetron-Error-Dashboard p860 -aVdesc +sV_source p861 -asVhits -p862 +(dp862 +Vhits +p863 I0 +sVtimeRestore +p864 +I00 sVdescription -p863 +p865 V sVtitle -p864 -VDNS Requests -p865 -sVversion p866 -I1 -sVkibanaSavedObjectMeta +VMetron Error Dashboard p867 -(dp868 -VsearchSourceJSON +sVuiStateJSON +p868 +V{"P-2":{"vis":{"legendOpen":true}},"P-5":{"vis":{"colors":{"another_new_parser_error":"#806EB7","new_parser_error":"#AEA2E0","parser_error":"#614D93"}}},"P-3":{"vis":{"colors":{"fourth":"#1F78C1","test_error":"#82B5D8","new_error":"#BADFF4"}}}} p869 -V{"index":"bro*","query":{"query_string":{"query":"protocol: dns","analyze_wildcard":true}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} +sVpanelsJSON p870 -ssVcolumns +V[{"col":5,"id":"Errors-By-Error-Type","panelIndex":2,"row":10,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Source-Proportion","panelIndex":3,"row":7,"size_x":4,"size_y":3,"type":"visualization"},{"col":5,"id":"Errors-By-Source-Type","panelIndex":4,"row":7,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Type-Proportion","panelIndex":5,"row":10,"size_x":4,"size_y":3,"type":"visualization"},{"col":1,"id":"Errors-Over-Time","panelIndex":14,"row":3,"size_x":12,"size_y":4,"type":"visualization"},{"col":8,"id":"Unique-Error-Messages","panelIndex":19,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":3,"id":"Total-Error-Messages","panelIndex":20,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":5,"id":"Errors-By-Hostname","panelIndex":22,"row":13,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Hostname-Proportion","panelIndex":23,"row":13,"size_x":4,"size_y":3,"type":"visualization"},{"col":1,"columns":["source_type","error_type","exception","hostname","message","raw_message","raw_message_hash"],"id":"Errors","panelIndex":24,"row":16,"size_x":12,"size_y":7,"sort":["time","desc"],"type":"search"}] p871 -(lp872 -Vquery +sVoptionsJSON +p872 +V{"darkTheme":false} p873 -aVqtype_name +sVversion p874 -aVanswers +I1 +sVkibanaSavedObjectMeta p875 -aVip_src_addr -p876 -aVip_dst_addr +(dp876 +VsearchSourceJSON p877 -assV_index +V{"filter":[{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}}}]} p878 -V.kibana +sssV_index p879 -sa(dp880 +V.kibana +p880 +sa(dp881 V_score -p881 +p882 F1 sV_type -p882 -Vvisualization p883 -sV_id +Vconfig p884 -VDNS-Requests-Header +sV_id p885 -sV_source +V4.5.3 p886 -(dp887 -VvisState -p888 -V{"aggs":[],"listeners":{},"params":{"markdown":"[Bro](https://www.bro.org/) is extracting DNS requests and responses being made over the network. Understanding who is making those requests, the frequency, and types can provide a deep understanding of the actors present on the network."},"title":"DNS Requests","type":"markdown"} +sV_source +p887 +(dp888 +VbuildNum p889 -sVdescription +I9892 +sVdefaultIndex p890 -V -sVtitle +Vbro* p891 -VDNS Requests +ssV_index p892 -sVuiStateJSON +V.kibana p893 -V{} -p894 -sVversion +sa(dp894 +V_score p895 -I1 -sVkibanaSavedObjectMeta +F1 +sV_type p896 -(dp897 -VsearchSourceJSON +Vsearch +p897 +sV_id p898 -V{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} +Vdns-search p899 -sssV_index +sV_source p900 -V.kibana -p901 -sa(dp902 -V_score -p903 -F1 -sV_type +(dp901 +Vsort +p902 +(lp903 +Vtimestamp p904 -Vvisualization +aVdesc p905 -sV_id +asVhits p906 -VYAF-Flows-Header +I0 +sVdescription p907 -sV_source +V +sVtitle p908 -(dp909 -VvisState +VDNS Requests +p909 +sVversion p910 -V{"title":"YAF","type":"markdown","params":{"markdown":"[YAF](https://tools.netsa.cert.org/yaf/yaf.html) can be used to generate Netflow-like flow records. These flow records provide significant visibility of the actors communicating over the target network."},"aggs":[],"listeners":{}} +I1 +sVkibanaSavedObjectMeta p911 -sVdescription -p912 -V -sVtitle +(dp912 +VsearchSourceJSON p913 -VYAF +V{"index":"bro*","query":{"query_string":{"query":"protocol: dns","analyze_wildcard":true}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} p914 -sVuiStateJSON +ssVcolumns p915 -V{} -p916 -sVversion +(lp916 +Vquery p917 -I1 -sVkibanaSavedObjectMeta +aVqtype_name p918 -(dp919 -VsearchSourceJSON +aVanswers +p919 +aVip_src_addr p920 -V{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} +aVip_dst_addr p921 -sssV_index +assV_index p922 V.kibana p923 @@ -1848,21 +1848,21 @@ Vvisualization p927 sV_id p928 -VTop-5-Exceptions +VDNS-Requests-Header p929 sV_source p930 (dp931 VvisState p932 -V{"title":"Top-5 Exceptions","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"exception","size":5,"order":"desc","orderBy":"1","customLabel":"Exceptions"}}],"listeners":{}} +V{"aggs":[],"listeners":{},"params":{"markdown":"[Bro](https://www.bro.org/) is extracting DNS requests and responses being made over the network. Understanding who is making those requests, the frequency, and types can provide a deep understanding of the actors present on the network."},"title":"DNS Requests","type":"markdown"} p933 sVdescription p934 V sVtitle p935 -VTop-5 Exceptions +VDNS Requests p936 sVuiStateJSON p937 @@ -1876,7 +1876,7 @@ p940 (dp941 VsearchSourceJSON p942 -V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} p943 sssV_index p944 @@ -1892,21 +1892,21 @@ Vvisualization p949 sV_id p950 -VFrequent-DNS-Requests +VYAF-Flows-Header p951 sV_source p952 (dp953 VvisState p954 -V{"title":"Frequent DNS Requests","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":5,"order":"desc","orderBy":"1","customLabel":"DNS Query"}}],"listeners":{}} +V{"title":"YAF","type":"markdown","params":{"markdown":"[YAF](https://tools.netsa.cert.org/yaf/yaf.html) can be used to generate Netflow-like flow records. These flow records provide significant visibility of the actors communicating over the target network."},"aggs":[],"listeners":{}} p955 sVdescription p956 V sVtitle p957 -VFrequent DNS Requests +VYAF p958 sVuiStateJSON p959 @@ -1920,7 +1920,7 @@ p962 (dp963 VsearchSourceJSON p964 -V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} p965 sssV_index p966 @@ -1936,21 +1936,21 @@ Vvisualization p971 sV_id p972 -VCountry +VTop-5-Exceptions p973 sV_source p974 (dp975 VvisState p976 -V{"title":"By Country","type":"pie","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"isDonut":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"enrichments:geo:ip_src_addr:country","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} +V{"title":"Top-5 Exceptions","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"exception","size":5,"order":"desc","orderBy":"1","customLabel":"Exceptions"}}],"listeners":{}} p977 sVdescription p978 V sVtitle p979 -VBy Country +VTop-5 Exceptions p980 sVuiStateJSON p981 @@ -1964,7 +1964,7 @@ p984 (dp985 VsearchSourceJSON p986 -V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p987 sssV_index p988 @@ -1980,21 +1980,21 @@ Vvisualization p993 sV_id p994 -VFlow-Locations +VFrequent-DNS-Requests p995 sV_source p996 (dp997 VvisState p998 -V{"title":"Flow Locations","type":"tile_map","params":{"mapType":"Scaled Circle Markers","isDesaturated":true,"addTooltip":true,"heatMaxZoom":16,"heatMinOpacity":0.1,"heatRadius":25,"heatBlur":15,"heatNormalizeData":true,"wms":{"enabled":false,"url":"https://basemap.nationalmap.gov/arcgis/services/USGSTopo/MapServer/WMSServer","options":{"version":"1.3.0","layers":"0","format":"image/png","transparent":true,"attribution":"Maps provided by USGS","styles":""}}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"geohash_grid","schema":"segment","params":{"field":"enrichments:geo:ip_dst_addr:location_point","autoPrecision":true,"precision":2}}],"listeners":{}} +V{"title":"Frequent DNS Requests","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":5,"order":"desc","orderBy":"1","customLabel":"DNS Query"}}],"listeners":{}} p999 sVdescription p1000 V sVtitle p1001 -VFlow Locations +VFrequent DNS Requests p1002 sVuiStateJSON p1003 @@ -2008,7 +2008,7 @@ p1006 (dp1007 VsearchSourceJSON p1008 -V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p1009 sssV_index p1010 @@ -2024,21 +2024,21 @@ Vvisualization p1015 sV_id p1016 -VTop-Destinations +VCountry p1017 sV_source p1018 (dp1019 VvisState p1020 -V{"title":"Top Destinations","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"ip_dst_addr","size":10,"order":"desc","orderBy":"1","customLabel":"Destination IP"}}],"listeners":{}} +V{"title":"By Country","type":"pie","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"isDonut":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"enrichments:geo:ip_src_addr:country","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} p1021 sVdescription p1022 V sVtitle p1023 -VTop Destinations +VBy Country p1024 sVuiStateJSON p1025 @@ -2068,21 +2068,21 @@ Vvisualization p1037 sV_id p1038 -VUnusual-Referrers +VFlow-Locations p1039 sV_source p1040 (dp1041 VvisState p1042 -V{"title":"Unusual Referrers","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"significant_terms","schema":"bucket","params":{"field":"referrer","size":5,"customLabel":"Top 5 Unusual Referrers"}}],"listeners":{}} +V{"title":"Flow Locations","type":"tile_map","params":{"mapType":"Scaled Circle Markers","isDesaturated":true,"addTooltip":true,"heatMaxZoom":16,"heatMinOpacity":0.1,"heatRadius":25,"heatBlur":15,"heatNormalizeData":true,"wms":{"enabled":false,"url":"https://basemap.nationalmap.gov/arcgis/services/USGSTopo/MapServer/WMSServer","options":{"version":"1.3.0","layers":"0","format":"image/png","transparent":true,"attribution":"Maps provided by USGS","styles":""}}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"geohash_grid","schema":"segment","params":{"field":"enrichments:geo:ip_dst_addr:location_point","autoPrecision":true,"precision":2}}],"listeners":{}} p1043 sVdescription p1044 V sVtitle p1045 -VUnusual Referrers +VFlow Locations p1046 sVuiStateJSON p1047 @@ -2091,19 +2091,107 @@ p1048 sVversion p1049 I1 -sVsavedSearchId +sVkibanaSavedObjectMeta p1050 +(dp1051 +VsearchSourceJSON +p1052 +V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p1053 +sssV_index +p1054 +V.kibana +p1055 +sa(dp1056 +V_score +p1057 +F1 +sV_type +p1058 +Vvisualization +p1059 +sV_id +p1060 +VTop-Destinations +p1061 +sV_source +p1062 +(dp1063 +VvisState +p1064 +V{"title":"Top Destinations","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"ip_dst_addr","size":10,"order":"desc","orderBy":"1","customLabel":"Destination IP"}}],"listeners":{}} +p1065 +sVdescription +p1066 +V +sVtitle +p1067 +VTop Destinations +p1068 +sVuiStateJSON +p1069 +V{} +p1070 +sVversion +p1071 +I1 +sVkibanaSavedObjectMeta +p1072 +(dp1073 +VsearchSourceJSON +p1074 +V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} +p1075 +sssV_index +p1076 +V.kibana +p1077 +sa(dp1078 +V_score +p1079 +F1 +sV_type +p1080 +Vvisualization +p1081 +sV_id +p1082 +VUnusual-Referrers +p1083 +sV_source +p1084 +(dp1085 +VvisState +p1086 +V{"title":"Unusual Referrers","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"significant_terms","schema":"bucket","params":{"field":"referrer","size":5,"customLabel":"Top 5 Unusual Referrers"}}],"listeners":{}} +p1087 +sVdescription +p1088 +V +sVtitle +p1089 +VUnusual Referrers +p1090 +sVuiStateJSON +p1091 +V{} +p1092 +sVversion +p1093 +I1 +sVsavedSearchId +p1094 Vweb-search -p1051 +p1095 sVkibanaSavedObjectMeta -p1052 -(dp1053 +p1096 +(dp1097 VsearchSourceJSON -p1054 +p1098 V{"filter":[]} -p1055 +p1099 sssV_index -p1056 +p1100 V.kibana -p1057 +p1101 sa. \ No newline at end of file diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template index 3bb4633704..070c90f369 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template @@ -18,7 +18,7 @@ "type": "string", "index": "not_analyzed" }, - "timestamp": { + "time": { "type": "date", "format": "epoch_millis" }, @@ -35,15 +35,11 @@ "type": "binary", "index": "no" }, - "error_fields": { + "raw_message_hash": { "type": "string", "index": "not_analyzed" }, - "error_hash": { - "type": "string", - "index": "not_analyzed" - }, - "failed_sensor_type": { + "source_type": { "type": "string", "index": "not_analyzed" }, From 08963b2682123cd3b9d5287dd5b76355ebe07515 Mon Sep 17 00:00:00 2001 From: justinjleet Date: Wed, 1 Mar 2017 15:08:21 -0500 Subject: [PATCH 14/16] fixing things --- .../packaging/ambari/metron-mpack/README.md | 18 ++++++++++++++++++ .../CURRENT/package/files/error_index.template | 10 +++++++--- 2 files changed, 25 insertions(+), 3 deletions(-) create mode 100644 metron-deployment/packaging/ambari/metron-mpack/README.md diff --git a/metron-deployment/packaging/ambari/metron-mpack/README.md b/metron-deployment/packaging/ambari/metron-mpack/README.md new file mode 100644 index 0000000000..64c240420f --- /dev/null +++ b/metron-deployment/packaging/ambari/metron-mpack/README.md @@ -0,0 +1,18 @@ +# Overview + +Contains the definitions of the various services for the Ambari Management Pack. + +More info can be found at https://cwiki.apache.org/confluence/display/AMBARI/Management+Packs + +## Kibana + +The dashboards installed by the Kibana custom action are managed by the dashboard.p file. This file is created by exporting existing dashboards from a running Kibana instance. + +To create a new version of the file, make any necessary changes to Kibana (e.g. on quick-dev), and export with the appropriate command. +``` +python src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboardindex.py $KIBANA_URL 9200 dashboard.p -s +mv dashboard.p /Users/jleet/Documents/workspace/incubator-metron/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p +``` + +Build the Ambari Mpack and the changes will + diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template index 070c90f369..3bb4633704 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/error_index.template @@ -18,7 +18,7 @@ "type": "string", "index": "not_analyzed" }, - "time": { + "timestamp": { "type": "date", "format": "epoch_millis" }, @@ -35,11 +35,15 @@ "type": "binary", "index": "no" }, - "raw_message_hash": { + "error_fields": { "type": "string", "index": "not_analyzed" }, - "source_type": { + "error_hash": { + "type": "string", + "index": "not_analyzed" + }, + "failed_sensor_type": { "type": "string", "index": "not_analyzed" }, From ad7191d2a0408cb49ab6df8cb3e8f0b11f1403de Mon Sep 17 00:00:00 2001 From: justinjleet Date: Thu, 2 Mar 2017 09:26:10 -0500 Subject: [PATCH 15/16] fixes + readme update --- .../packaging/ambari/metron-mpack/README.md | 8 ++- .../package/scripts/dashboard/dashboard.p | 64 +++++++++---------- 2 files changed, 37 insertions(+), 35 deletions(-) diff --git a/metron-deployment/packaging/ambari/metron-mpack/README.md b/metron-deployment/packaging/ambari/metron-mpack/README.md index 64c240420f..08f28b015d 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/README.md +++ b/metron-deployment/packaging/ambari/metron-mpack/README.md @@ -4,15 +4,17 @@ Contains the definitions of the various services for the Ambari Management Pack. More info can be found at https://cwiki.apache.org/confluence/display/AMBARI/Management+Packs -## Kibana +## Kibana Dashboards The dashboards installed by the Kibana custom action are managed by the dashboard.p file. This file is created by exporting existing dashboards from a running Kibana instance. To create a new version of the file, make any necessary changes to Kibana (e.g. on quick-dev), and export with the appropriate command. ``` python src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboardindex.py $KIBANA_URL 9200 dashboard.p -s -mv dashboard.p /Users/jleet/Documents/workspace/incubator-metron/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p +mv dashboard.p src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p ``` -Build the Ambari Mpack and the changes will +Build the Ambari Mpack to get the dashboard updated appropriately. + +Once the MPack is installed, run the Kibana service's action "Load Template" to install dashboards. This will completely overwrite the .kibana in Elasticsearch, so use with caution. diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p index 49f6806988..31e31bcc1b 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p @@ -333,7 +333,7 @@ p168 (dp169 VvisState p170 -V{"title":"Errors By Error Type","type":"histogram","params":{"addLegend":true,"addTimeMarker":false,"addTooltip":true,"defaultYExtents":false,"mode":"grouped","scale":"linear","setYExtents":false,"shareYAxis":true,"times":[],"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{"customLabel":"Count"}},{"id":"2","type":"terms","schema":"segment","params":{"field":"hostname","size":5,"order":"desc","orderBy":"1"}},{"id":"4","type":"cardinality","schema":"metric","params":{"field":"raw_message_hash","customLabel":"Unique Datapoint Count"}}],"listeners":{}} +V{\u000a "title": "Errors By Error Type",\u000a "type": "histogram",\u000a "params": {\u000a "addLegend": true,\u000a "addTimeMarker": false,\u000a "addTooltip": true,\u000a "defaultYExtents": false,\u000a "mode": "grouped",\u000a "scale": "linear",\u000a "setYExtents": false,\u000a "shareYAxis": true,\u000a "times": [],\u000a "yAxis": {}\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {\u000a "customLabel": "Count"\u000a }\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "hostname",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1"\u000a }\u000a },\u000a {\u000a "id": "4",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "error_hash",\u000a "customLabel": "Unique Datapoint Count"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} p171 sVdescription p172 @@ -344,7 +344,7 @@ VErrors By Hostname p174 sVuiStateJSON p175 -V{"vis":{"colors":{"Unique Datapoint Count":"#9AC48A","Count":"#629E51"}}} +V{\u000a "vis": {\u000a "colors": {\u000a "Unique Datapoint Count": "#9AC48A",\u000a "Count": "#629E51"\u000a }\u000a }\u000a} p176 sVversion p177 @@ -354,7 +354,7 @@ p178 (dp179 VsearchSourceJSON p180 -V{"index":"error*","query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} +V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "analyze_wildcard": true,\u000a "query": "*"\u000a }\u000a },\u000a "filter": []\u000a} p181 sssV_index p182 @@ -590,25 +590,25 @@ Vvisualization p297 sV_id p298 -VUnique-Error-Messages +VError-Hostname-Proportion p299 sV_source p300 (dp301 VvisState p302 -V{\u000a "title": "Total Unique Error Messages",\u000a "type": "metric",\u000a "params": {\u000a "handleNoResults": true,\u000a "fontSize": 60\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "raw_message_hash",\u000a "customLabel": "Unique Error Messages"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +V{"aggs":[{"id":"1","params":{},"schema":"metric","type":"count"},{"id":"2","params":{"customLabel":"Sensor","field":"hostname","order":"desc","orderBy":"1","size":5},"schema":"segment","type":"terms"}],"listeners":{},"params":{"addLegend":true,"addTooltip":true,"isDonut":false,"shareYAxis":true},"title":"Error Source Proportion","type":"pie"} p303 sVdescription p304 V sVtitle p305 -VUnique Error Messages +VError Hostname Proportion p306 sVuiStateJSON p307 -V{} +V{"vis":{"colors":{"host":"#629E51","host2":"#9AC48A","hostAnother":"#7EB26D","hostNew":"#B7DBAB"}}} p308 sVversion p309 @@ -618,7 +618,7 @@ p310 (dp311 VsearchSourceJSON p312 -V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "query": "*",\u000a "analyze_wildcard": true\u000a }\u000a },\u000a "filter": []\u000a} +V{"index":"error*","query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} p313 sssV_index p314 @@ -634,25 +634,25 @@ Vvisualization p319 sV_id p320 -VErrors-By-Error-Type +VUnique-Error-Messages p321 sV_source p322 (dp323 VvisState p324 -V{\u000a "title": "Errors By Error Type",\u000a "type": "histogram",\u000a "params": {\u000a "addLegend": true,\u000a "addTimeMarker": false,\u000a "addTooltip": true,\u000a "defaultYExtents": false,\u000a "mode": "grouped",\u000a "scale": "linear",\u000a "setYExtents": false,\u000a "shareYAxis": true,\u000a "times": [],\u000a "yAxis": {}\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {\u000a "customLabel": "Count"\u000a }\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "error_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1"\u000a }\u000a },\u000a {\u000a "id": "4",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "raw_message_hash",\u000a "customLabel": "Unique Datapoint Count"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +V{\u000a "title": "Total Unique Error Messages",\u000a "type": "metric",\u000a "params": {\u000a "handleNoResults": true,\u000a "fontSize": 60\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "error_hash",\u000a "customLabel": "Unique Error Messages"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} p325 sVdescription p326 V sVtitle p327 -VErrors By Error Type +VUnique Error Messages p328 sVuiStateJSON p329 -V{\u000a "vis": {\u000a "colors": {\u000a "Unique Datapoint Count": "#806EB7",\u000a "Count": "#614D93"\u000a }\u000a }\u000a} +V{} p330 sVversion p331 @@ -662,7 +662,7 @@ p332 (dp333 VsearchSourceJSON p334 -V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "analyze_wildcard": true,\u000a "query": "*"\u000a }\u000a },\u000a "filter": []\u000a} +V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "query": "*",\u000a "analyze_wildcard": true\u000a }\u000a },\u000a "filter": []\u000a} p335 sssV_index p336 @@ -678,25 +678,25 @@ Vvisualization p341 sV_id p342 -VError-Hostname-Proportion +VErrors-By-Error-Type p343 sV_source p344 (dp345 VvisState p346 -V{"aggs":[{"id":"1","params":{},"schema":"metric","type":"count"},{"id":"2","params":{"customLabel":"Sensor","field":"hostname","order":"desc","orderBy":"1","size":5},"schema":"segment","type":"terms"}],"listeners":{},"params":{"addLegend":true,"addTooltip":true,"isDonut":false,"shareYAxis":true},"title":"Error Source Proportion","type":"pie"} +V{\u000a "title": "Errors By Error Type",\u000a "type": "histogram",\u000a "params": {\u000a "addLegend": true,\u000a "addTimeMarker": false,\u000a "addTooltip": true,\u000a "defaultYExtents": false,\u000a "mode": "grouped",\u000a "scale": "linear",\u000a "setYExtents": false,\u000a "shareYAxis": true,\u000a "times": [],\u000a "yAxis": {}\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {\u000a "customLabel": "Count"\u000a }\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "error_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1"\u000a }\u000a },\u000a {\u000a "id": "4",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "error_hash",\u000a "customLabel": "Unique Datapoint Count"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} p347 sVdescription p348 V sVtitle p349 -VError Hostname Proportion +VErrors By Error Type p350 sVuiStateJSON p351 -V{"vis":{"colors":{"host":"#629E51","host2":"#9AC48A","hostAnother":"#7EB26D","hostNew":"#B7DBAB"}}} +V{\u000a "vis": {\u000a "colors": {\u000a "Unique Datapoint Count": "#806EB7",\u000a "Count": "#614D93"\u000a }\u000a }\u000a} p352 sVversion p353 @@ -706,7 +706,7 @@ p354 (dp355 VsearchSourceJSON p356 -V{"index":"error*","query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} +V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "analyze_wildcard": true,\u000a "query": "*"\u000a }\u000a },\u000a "filter": []\u000a} p357 sssV_index p358 @@ -730,7 +730,7 @@ p366 Vsort p368 (lp369 -Vtime +Vtimestamp p370 aVdesc p371 @@ -757,7 +757,7 @@ p380 ssVcolumns p381 (lp382 -Vsource_type +Vfailed_sensor_type p383 aVerror_type p384 @@ -769,7 +769,7 @@ aVmessage p387 aVraw_message p388 -aVraw_message_hash +aVerror_hash p389 assV_index p390 @@ -1202,7 +1202,7 @@ p602 (dp603 VvisState p604 -V{\u000a "title": "Errors By Source Type",\u000a "type": "histogram",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "scale": "linear",\u000a "mode": "grouped",\u000a "times": [],\u000a "addTimeMarker": false,\u000a "defaultYExtents": false,\u000a "setYExtents": false,\u000a "yAxis": {}\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {\u000a "customLabel": "Count"\u000a }\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "source_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1"\u000a }\u000a },\u000a {\u000a "id": "4",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "raw_message_hash",\u000a "customLabel": "Unique Datapoint Count"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +V{\u000a "title": "Errors By Source Type",\u000a "type": "histogram",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "scale": "linear",\u000a "mode": "grouped",\u000a "times": [],\u000a "addTimeMarker": false,\u000a "defaultYExtents": false,\u000a "setYExtents": false,\u000a "yAxis": {}\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {\u000a "customLabel": "Count"\u000a }\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "failed_sensor_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1"\u000a }\u000a },\u000a {\u000a "id": "4",\u000a "type": "cardinality",\u000a "schema": "metric",\u000a "params": {\u000a "field": "error_hash",\u000a "customLabel": "Unique Datapoint Count"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} p605 sVdescription p606 @@ -1590,21 +1590,21 @@ Vvisualization p798 sV_id p799 -VError-Source-Proportion +VErrors-Over-Time p800 sV_source p801 (dp802 VvisState p803 -V{\u000a "title": "Sensor Type Proportion",\u000a "type": "pie",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "isDonut": false\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {}\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "source_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1",\u000a "customLabel": "Sensor"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +V{\u000a "title": "Error Over Time",\u000a "type": "line",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "showCircles": true,\u000a "smoothLines": false,\u000a "interpolate": "linear",\u000a "scale": "linear",\u000a "drawLinesBetweenPoints": true,\u000a "radiusRatio": 9,\u000a "times": [],\u000a "addTimeMarker": true,\u000a "defaultYExtents": false,\u000a "setYExtents": false,\u000a "yAxis": {\u000a "min": 0\u000a }\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {}\u000a },\u000a {\u000a "id": "2",\u000a "type": "date_histogram",\u000a "schema": "segment",\u000a "params": {\u000a "field": "timestamp",\u000a "interval": "auto",\u000a "customInterval": "2h",\u000a "min_doc_count": 1,\u000a "extended_bounds": {}\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} p804 sVdescription p805 V sVtitle p806 -VError Source Proportion +VErrors Over Time p807 sVuiStateJSON p808 @@ -1634,21 +1634,21 @@ Vvisualization p820 sV_id p821 -VErrors-Over-Time +VError-Source-Proportion p822 sV_source p823 (dp824 VvisState p825 -V{\u000a "title": "Error Over Time",\u000a "type": "line",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "showCircles": true,\u000a "smoothLines": false,\u000a "interpolate": "linear",\u000a "scale": "linear",\u000a "drawLinesBetweenPoints": true,\u000a "radiusRatio": 9,\u000a "times": [],\u000a "addTimeMarker": true,\u000a "defaultYExtents": false,\u000a "setYExtents": false,\u000a "yAxis": {\u000a "min": 0\u000a }\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {}\u000a },\u000a {\u000a "id": "2",\u000a "type": "date_histogram",\u000a "schema": "segment",\u000a "params": {\u000a "field": "time",\u000a "interval": "auto",\u000a "customInterval": "2h",\u000a "min_doc_count": 1,\u000a "extended_bounds": {}\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +V{\u000a "title": "Sensor Type Proportion",\u000a "type": "pie",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "isDonut": false\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {}\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "failed_sensor_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1",\u000a "customLabel": "Sensor"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} p826 sVdescription p827 V sVtitle p828 -VErrors Over Time +VError Source Proportion p829 sVuiStateJSON p830 @@ -1685,11 +1685,11 @@ p845 (dp846 Vfields p847 -V[{"name":"exception","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"stack","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"raw_message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source_type","type":"string","count":2,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"hostname","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"error_type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"raw_message_bytes","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"time","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"raw_message_hash","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] +V[{"name":"exception","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"stack","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"error_hash","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"raw_message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"failed_sensor_type","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"hostname","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"error_type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"error_fields","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"raw_message_bytes","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"timestamp","type":"date","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] p848 sVtimeFieldName p849 -Vtime +Vtimestamp p850 sVtitle p851 @@ -1729,11 +1729,11 @@ VMetron Error Dashboard p867 sVuiStateJSON p868 -V{"P-2":{"vis":{"legendOpen":true}},"P-5":{"vis":{"colors":{"another_new_parser_error":"#806EB7","new_parser_error":"#AEA2E0","parser_error":"#614D93"}}},"P-3":{"vis":{"colors":{"fourth":"#1F78C1","test_error":"#82B5D8","new_error":"#BADFF4"}}}} +V{"P-2":{"vis":{"legendOpen":true}},"P-23":{"vis":{"colors":{"amb3.service.consul":"#629E51","host":"#629E51","host2":"#9AC48A","hostAnother":"#7EB26D","hostNew":"#B7DBAB"}}},"P-3":{"vis":{"colors":{"fourth":"#1F78C1","new_error":"#BADFF4","test_error":"#82B5D8"}}},"P-5":{"vis":{"colors":{"another_new_parser_error":"#806EB7","new_parser_error":"#AEA2E0","parser_error":"#614D93"}}}} p869 sVpanelsJSON p870 -V[{"col":5,"id":"Errors-By-Error-Type","panelIndex":2,"row":10,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Source-Proportion","panelIndex":3,"row":7,"size_x":4,"size_y":3,"type":"visualization"},{"col":5,"id":"Errors-By-Source-Type","panelIndex":4,"row":7,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Type-Proportion","panelIndex":5,"row":10,"size_x":4,"size_y":3,"type":"visualization"},{"col":1,"id":"Errors-Over-Time","panelIndex":14,"row":3,"size_x":12,"size_y":4,"type":"visualization"},{"col":8,"id":"Unique-Error-Messages","panelIndex":19,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":3,"id":"Total-Error-Messages","panelIndex":20,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":5,"id":"Errors-By-Hostname","panelIndex":22,"row":13,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Hostname-Proportion","panelIndex":23,"row":13,"size_x":4,"size_y":3,"type":"visualization"},{"col":1,"columns":["source_type","error_type","exception","hostname","message","raw_message","raw_message_hash"],"id":"Errors","panelIndex":24,"row":16,"size_x":12,"size_y":7,"sort":["time","desc"],"type":"search"}] +V[{"col":5,"id":"Errors-By-Error-Type","panelIndex":2,"row":7,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Source-Proportion","panelIndex":3,"row":7,"size_x":4,"size_y":3,"type":"visualization"},{"col":5,"id":"Errors-By-Source-Type","panelIndex":4,"row":10,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Type-Proportion","panelIndex":5,"row":10,"size_x":4,"size_y":3,"type":"visualization"},{"col":1,"id":"Errors-Over-Time","panelIndex":14,"row":3,"size_x":12,"size_y":4,"type":"visualization"},{"col":8,"id":"Unique-Error-Messages","panelIndex":19,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":3,"id":"Total-Error-Messages","panelIndex":20,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":5,"id":"Errors-By-Hostname","panelIndex":22,"row":13,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Hostname-Proportion","panelIndex":23,"row":13,"size_x":4,"size_y":3,"type":"visualization"},{"col":1,"columns":["failed_sensor_type","error_type","exception","hostname","message","raw_message","error_hash"],"id":"Errors","panelIndex":25,"row":23,"size_x":12,"size_y":7,"sort":["timestamp","desc"],"type":"search"}] p871 sVoptionsJSON p872 From 4fe93403e9987d6677092aa93a332645ffa2737b Mon Sep 17 00:00:00 2001 From: justinjleet Date: Mon, 6 Mar 2017 15:15:39 -0500 Subject: [PATCH 16/16] Updating dashboard to be more reasonable --- .../package/scripts/dashboard/dashboard.p | 1074 ++++++++++------- 1 file changed, 609 insertions(+), 465 deletions(-) diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p index 31e31bcc1b..ed5b2bdacd 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/KIBANA/4.5.1/package/scripts/dashboard/dashboard.p @@ -1235,298 +1235,298 @@ p619 F1 sV_type p620 -Vindex-pattern +Vvisualization p621 sV_id p622 -Vsnort* +VError-Histogram-By-Sensor-Type p623 sV_source p624 (dp625 -Vfields +VvisState p626 -V[{"name":"msg","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dgmlen","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentjoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpack","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"protocol","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:threatinteladapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"original_string","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:geoadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ethlen","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threat:triage:level","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:threatinteladapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"ip_dst_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatinteljoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_rev","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"ethsrc","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpseq","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichmentsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpwindow","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_dst_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tos","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ethdst","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"is_alert","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ttl","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"iplen","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_generator","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] +V{"title":"Error Histogram By Sensor Type","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"grouped","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"date_histogram","schema":"segment","params":{"field":"timestamp","interval":"auto","customInterval":"2h","min_doc_count":1,"extended_bounds":{},"customLabel":"Time"}},{"id":"3","type":"terms","schema":"group","params":{"field":"failed_sensor_type","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} p627 -sVtimeFieldName +sVdescription p628 -Vtimestamp -p629 +V sVtitle +p629 +VError Histogram By Sensor Type p630 -Vsnort* +sVuiStateJSON p631 -ssV_index +V{} p632 -V.kibana +sVversion p633 -sa(dp634 -V_score +I1 +sVsavedSearchId +p634 +VErrors p635 +sVkibanaSavedObjectMeta +p636 +(dp637 +VsearchSourceJSON +p638 +V{"filter":[]} +p639 +sssV_index +p640 +V.kibana +p641 +sa(dp642 +V_score +p643 F1 sV_type -p636 +p644 Vindex-pattern -p637 +p645 sV_id -p638 +p646 +Vsnort* +p647 +sV_source +p648 +(dp649 +Vfields +p650 +V[{"name":"msg","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dgmlen","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentjoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpack","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"protocol","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:threatinteladapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"original_string","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:geoadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ethlen","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threat:triage:level","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:threatinteladapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"ip_dst_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatinteljoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_rev","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"ethsrc","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpseq","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichmentsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tcpwindow","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_dst_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tos","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ethdst","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"is_alert","type":"boolean","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ttl","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"iplen","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_id","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sig_generator","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_src_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] +p651 +sVtimeFieldName +p652 +Vtimestamp +p653 +sVtitle +p654 +Vsnort* +p655 +ssV_index +p656 +V.kibana +p657 +sa(dp658 +V_score +p659 +F1 +sV_type +p660 +Vindex-pattern +p661 +sV_id +p662 Vyaf* -p639 +p663 sV_source -p640 -(dp641 +p664 +(dp665 Vfields -p642 +p666 V[{"name":"enrichments:geo:ip_dst_addr:location_point","type":"geo_point","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"isn","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentjoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dip","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"dp","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"protocol","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"rpkt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"original_string","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"adapter:threatinteladapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:geoadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"tag","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"app","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"oct","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end_reason","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"enrichmentsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:city","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"start_time","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"riflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"proto","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:threatinteladapter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"enrichments:geo:ip_dst_addr:country","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:locID","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"iflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"ip_dst_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:dmaCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatinteljoinbolt:joiner:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"uflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichmentsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:latitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"duration","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_dst_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"pkt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"adapter:hostfromjsonlistadapter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ruflags","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"roct","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sip","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"sp","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_addr","type":"ip","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rtag","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:end:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:longitude","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"timestamp","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end-reason","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"risn","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"end_time","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"enrichments:geo:ip_dst_addr:postalCode","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"rtt","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"ip_src_port","type":"number","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"threatintelsplitterbolt:splitter:begin:ts","type":"date","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] -p643 +p667 sVtimeFieldName -p644 +p668 Vtimestamp -p645 +p669 sVtitle -p646 +p670 Vyaf* -p647 +p671 ssV_index -p648 +p672 V.kibana -p649 -sa(dp650 +p673 +sa(dp674 V_score -p651 +p675 F1 sV_type -p652 +p676 Vsearch -p653 +p677 sV_id -p654 +p678 Vweb-search -p655 +p679 sV_source -p656 -(dp657 +p680 +(dp681 Vsort -p658 -(lp659 +p682 +(lp683 Vtimestamp -p660 +p684 aVdesc -p661 +p685 asVhits -p662 +p686 I0 sVdescription -p663 +p687 V sVtitle -p664 +p688 VWeb Requests -p665 +p689 sVversion -p666 +p690 I1 sVkibanaSavedObjectMeta -p667 -(dp668 +p691 +(dp692 VsearchSourceJSON -p669 +p693 V{"index":"bro*","query":{"query_string":{"query":"protocol: http OR protocol: https","analyze_wildcard":true}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} -p670 +p694 ssVcolumns -p671 -(lp672 +p695 +(lp696 Vmethod -p673 +p697 aVhost -p674 +p698 aVuri -p675 +p699 aVreferrer -p676 +p700 aVip_src_addr -p677 +p701 aVip_dst_addr -p678 +p702 assV_index -p679 +p703 V.kibana -p680 -sa(dp681 +p704 +sa(dp705 V_score -p682 +p706 F1 sV_type -p683 +p707 Vvisualization -p684 +p708 sV_id -p685 +p709 VLocation-Header -p686 +p710 sV_source -p687 -(dp688 +p711 +(dp712 VvisState -p689 +p713 V{"title":"Enrichment","type":"markdown","params":{"markdown":"Apache Metron can perform real-time enrichment of telemetry data as it is consumed. To highlight this feature, all of the IP address fields collected from the default sensor suite were used to perform geo-ip lookups. This data was then used to pinpoint each location on the map."},"aggs":[],"listeners":{}} -p690 +p714 sVdescription -p691 +p715 V sVtitle -p692 +p716 VEnrichment -p693 +p717 sVuiStateJSON -p694 +p718 V{} -p695 +p719 sVversion -p696 +p720 I1 sVkibanaSavedObjectMeta -p697 -(dp698 +p721 +(dp722 VsearchSourceJSON -p699 +p723 V{"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p700 +p724 sssV_index -p701 +p725 V.kibana -p702 -sa(dp703 +p726 +sa(dp727 V_score -p704 +p728 F1 sV_type -p705 +p729 Vvisualization -p706 +p730 sV_id -p707 +p731 VSnort-Alert-Types -p708 +p732 sV_source -p709 -(dp710 +p733 +(dp734 VvisState -p711 +p735 V{"title":"Snort Alert Types","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"cardinality","schema":"metric","params":{"field":"sig_id","customLabel":"Alert Type(s)"}}],"listeners":{}} -p712 +p736 sVdescription -p713 +p737 V sVtitle -p714 +p738 VSnort Alert Types -p715 +p739 sVuiStateJSON -p716 +p740 V{} -p717 +p741 sVversion -p718 +p742 I1 sVkibanaSavedObjectMeta -p719 -(dp720 +p743 +(dp744 VsearchSourceJSON -p721 +p745 V{"index":"snort*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p722 +p746 sssV_index -p723 +p747 V.kibana -p724 -sa(dp725 +p748 +sa(dp749 V_score -p726 +p750 F1 sV_type -p727 +p751 Vvisualization -p728 +p752 sV_id -p729 +p753 VFrequent-DNS-Queries -p730 +p754 sV_source -p731 -(dp732 +p755 +(dp756 VvisState -p733 +p757 V{"title":"Frequent DNS Requests","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} -p734 -sVdescription -p735 -V -sVtitle -p736 -VFrequent DNS Requests -p737 -sVuiStateJSON -p738 -V{} -p739 -sVversion -p740 -I1 -sVkibanaSavedObjectMeta -p741 -(dp742 -VsearchSourceJSON -p743 -V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p744 -sssV_index -p745 -V.kibana -p746 -sa(dp747 -V_score -p748 -F1 -sV_type -p749 -Vvisualization -p750 -sV_id -p751 -VDNS-Request(s) -p752 -sV_source -p753 -(dp754 -VvisState -p755 -V{"title":"DNS Requests","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} -p756 +p758 sVdescription -p757 +p759 V sVtitle -p758 -VDNS Requests -p759 -sVuiStateJSON p760 -V{} +VFrequent DNS Requests p761 -sVversion +sVuiStateJSON p762 -I1 -sVsavedSearchId +V{} p763 -Vdns-search +sVversion p764 +I1 sVkibanaSavedObjectMeta p765 (dp766 VsearchSourceJSON p767 -V{"filter":[]} +V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} p768 sssV_index p769 @@ -1542,21 +1542,21 @@ Vvisualization p774 sV_id p775 -VHTTP(S)-Requests +VDNS-Request(s) p776 sV_source p777 (dp778 VvisState p779 -V{"title":"Web Requests","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} +V{"title":"DNS Requests","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} p780 sVdescription p781 V sVtitle p782 -VWeb Requests +VDNS Requests p783 sVuiStateJSON p784 @@ -1567,7 +1567,7 @@ p786 I1 sVsavedSearchId p787 -Vweb-search +Vdns-search p788 sVkibanaSavedObjectMeta p789 @@ -1590,21 +1590,21 @@ Vvisualization p798 sV_id p799 -VErrors-Over-Time +VHTTP(S)-Requests p800 sV_source p801 (dp802 VvisState p803 -V{\u000a "title": "Error Over Time",\u000a "type": "line",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "showCircles": true,\u000a "smoothLines": false,\u000a "interpolate": "linear",\u000a "scale": "linear",\u000a "drawLinesBetweenPoints": true,\u000a "radiusRatio": 9,\u000a "times": [],\u000a "addTimeMarker": true,\u000a "defaultYExtents": false,\u000a "setYExtents": false,\u000a "yAxis": {\u000a "min": 0\u000a }\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {}\u000a },\u000a {\u000a "id": "2",\u000a "type": "date_histogram",\u000a "schema": "segment",\u000a "params": {\u000a "field": "timestamp",\u000a "interval": "auto",\u000a "customInterval": "2h",\u000a "min_doc_count": 1,\u000a "extended_bounds": {}\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +V{"title":"Web Requests","type":"metric","params":{"handleNoResults":true,"fontSize":60},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}}],"listeners":{}} p804 sVdescription p805 V sVtitle p806 -VErrors Over Time +VWeb Requests p807 sVuiStateJSON p808 @@ -1613,585 +1613,729 @@ p809 sVversion p810 I1 -sVkibanaSavedObjectMeta +sVsavedSearchId p811 -(dp812 -VsearchSourceJSON +Vweb-search +p812 +sVkibanaSavedObjectMeta p813 +(dp814 +VsearchSourceJSON +p815 +V{"filter":[]} +p816 +sssV_index +p817 +V.kibana +p818 +sa(dp819 +V_score +p820 +F1 +sV_type +p821 +Vvisualization +p822 +sV_id +p823 +VErrors-Over-Time +p824 +sV_source +p825 +(dp826 +VvisState +p827 +V{\u000a "title": "Error Over Time",\u000a "type": "line",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "showCircles": true,\u000a "smoothLines": false,\u000a "interpolate": "linear",\u000a "scale": "linear",\u000a "drawLinesBetweenPoints": true,\u000a "radiusRatio": 9,\u000a "times": [],\u000a "addTimeMarker": true,\u000a "defaultYExtents": false,\u000a "setYExtents": false,\u000a "yAxis": {\u000a "min": 0\u000a }\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {}\u000a },\u000a {\u000a "id": "2",\u000a "type": "date_histogram",\u000a "schema": "segment",\u000a "params": {\u000a "field": "timestamp",\u000a "interval": "auto",\u000a "customInterval": "2h",\u000a "min_doc_count": 1,\u000a "extended_bounds": {}\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} +p828 +sVdescription +p829 +V +sVtitle +p830 +VErrors Over Time +p831 +sVuiStateJSON +p832 +V{} +p833 +sVversion +p834 +I1 +sVkibanaSavedObjectMeta +p835 +(dp836 +VsearchSourceJSON +p837 V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "query": "*",\u000a "analyze_wildcard": true\u000a }\u000a },\u000a "filter": []\u000a} -p814 +p838 sssV_index -p815 +p839 V.kibana -p816 -sa(dp817 +p840 +sa(dp841 V_score -p818 +p842 F1 sV_type -p819 +p843 Vvisualization -p820 +p844 sV_id -p821 +p845 VError-Source-Proportion -p822 +p846 sV_source -p823 -(dp824 +p847 +(dp848 VvisState -p825 +p849 V{\u000a "title": "Sensor Type Proportion",\u000a "type": "pie",\u000a "params": {\u000a "shareYAxis": true,\u000a "addTooltip": true,\u000a "addLegend": true,\u000a "isDonut": false\u000a },\u000a "aggs": [\u000a {\u000a "id": "1",\u000a "type": "count",\u000a "schema": "metric",\u000a "params": {}\u000a },\u000a {\u000a "id": "2",\u000a "type": "terms",\u000a "schema": "segment",\u000a "params": {\u000a "field": "failed_sensor_type",\u000a "size": 5,\u000a "order": "desc",\u000a "orderBy": "1",\u000a "customLabel": "Sensor"\u000a }\u000a }\u000a ],\u000a "listeners": {}\u000a} -p826 +p850 sVdescription -p827 +p851 V sVtitle -p828 +p852 VError Source Proportion -p829 +p853 sVuiStateJSON -p830 +p854 V{} -p831 +p855 sVversion -p832 +p856 I1 sVkibanaSavedObjectMeta -p833 -(dp834 +p857 +(dp858 VsearchSourceJSON -p835 +p859 V{\u000a "index": "error*",\u000a "query": {\u000a "query_string": {\u000a "query": "*",\u000a "analyze_wildcard": true\u000a }\u000a },\u000a "filter": []\u000a} -p836 +p860 sssV_index -p837 +p861 V.kibana -p838 -sa(dp839 +p862 +sa(dp863 V_score -p840 +p864 F1 sV_type -p841 +p865 Vindex-pattern -p842 +p866 sV_id -p843 +p867 Verror* -p844 +p868 sV_source -p845 -(dp846 +p869 +(dp870 Vfields -p847 +p871 V[{"name":"exception","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"stack","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_index","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"error_hash","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"raw_message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"message","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"failed_sensor_type","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"hostname","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"source:type","type":"string","count":1,"scripted":false,"indexed":true,"analyzed":true,"doc_values":false},{"name":"error_type","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"error_fields","type":"string","count":0,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_source","type":"_source","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"raw_message_bytes","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"timestamp","type":"date","count":1,"scripted":false,"indexed":true,"analyzed":false,"doc_values":true},{"name":"_id","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_type","type":"string","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false},{"name":"_score","type":"number","count":0,"scripted":false,"indexed":false,"analyzed":false,"doc_values":false}] -p848 +p872 sVtimeFieldName -p849 +p873 Vtimestamp -p850 +p874 sVtitle -p851 +p875 Verror* -p852 +p876 ssV_index -p853 +p877 V.kibana -p854 -sa(dp855 +p878 +sa(dp879 V_score -p856 +p880 F1 sV_type -p857 +p881 +Vvisualization +p882 +sV_id +p883 +VError-Date-Histogram +p884 +sV_source +p885 +(dp886 +VvisState +p887 +V{"title":"New Visualization","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"date_histogram","schema":"segment","params":{"field":"timestamp","interval":"auto","customInterval":"2h","min_doc_count":1,"extended_bounds":{},"customLabel":"Time"}}],"listeners":{}} +p888 +sVdescription +p889 +V +sVtitle +p890 +VError Date Histogram +p891 +sVuiStateJSON +p892 +V{} +p893 +sVversion +p894 +I1 +sVsavedSearchId +p895 +VErrors +p896 +sVkibanaSavedObjectMeta +p897 +(dp898 +VsearchSourceJSON +p899 +V{"filter":[]} +p900 +sssV_index +p901 +V.kibana +p902 +sa(dp903 +V_score +p904 +F1 +sV_type +p905 Vdashboard -p858 +p906 sV_id -p859 +p907 VMetron-Error-Dashboard -p860 +p908 sV_source -p861 -(dp862 +p909 +(dp910 Vhits -p863 +p911 I0 sVtimeRestore -p864 +p912 I00 sVdescription -p865 +p913 V sVtitle -p866 +p914 VMetron Error Dashboard -p867 +p915 sVuiStateJSON -p868 +p916 V{"P-2":{"vis":{"legendOpen":true}},"P-23":{"vis":{"colors":{"amb3.service.consul":"#629E51","host":"#629E51","host2":"#9AC48A","hostAnother":"#7EB26D","hostNew":"#B7DBAB"}}},"P-3":{"vis":{"colors":{"fourth":"#1F78C1","new_error":"#BADFF4","test_error":"#82B5D8"}}},"P-5":{"vis":{"colors":{"another_new_parser_error":"#806EB7","new_parser_error":"#AEA2E0","parser_error":"#614D93"}}}} -p869 +p917 sVpanelsJSON -p870 -V[{"col":5,"id":"Errors-By-Error-Type","panelIndex":2,"row":7,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Source-Proportion","panelIndex":3,"row":7,"size_x":4,"size_y":3,"type":"visualization"},{"col":5,"id":"Errors-By-Source-Type","panelIndex":4,"row":10,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Type-Proportion","panelIndex":5,"row":10,"size_x":4,"size_y":3,"type":"visualization"},{"col":1,"id":"Errors-Over-Time","panelIndex":14,"row":3,"size_x":12,"size_y":4,"type":"visualization"},{"col":8,"id":"Unique-Error-Messages","panelIndex":19,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":3,"id":"Total-Error-Messages","panelIndex":20,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":5,"id":"Errors-By-Hostname","panelIndex":22,"row":13,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Hostname-Proportion","panelIndex":23,"row":13,"size_x":4,"size_y":3,"type":"visualization"},{"col":1,"columns":["failed_sensor_type","error_type","exception","hostname","message","raw_message","error_hash"],"id":"Errors","panelIndex":25,"row":23,"size_x":12,"size_y":7,"sort":["timestamp","desc"],"type":"search"}] -p871 +p918 +V[{"col":5,"id":"Errors-By-Error-Type","panelIndex":2,"row":9,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Source-Proportion","panelIndex":3,"row":9,"size_x":4,"size_y":3,"type":"visualization"},{"col":5,"id":"Errors-By-Source-Type","panelIndex":4,"row":12,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Type-Proportion","panelIndex":5,"row":12,"size_x":4,"size_y":3,"type":"visualization"},{"col":8,"id":"Unique-Error-Messages","panelIndex":19,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":3,"id":"Total-Error-Messages","panelIndex":20,"row":1,"size_x":4,"size_y":2,"type":"visualization"},{"col":5,"id":"Errors-By-Hostname","panelIndex":22,"row":15,"size_x":8,"size_y":3,"type":"visualization"},{"col":1,"id":"Error-Hostname-Proportion","panelIndex":23,"row":15,"size_x":4,"size_y":3,"type":"visualization"},{"col":1,"columns":["failed_sensor_type","error_type","exception","hostname","message","raw_message","error_hash"],"id":"Errors","panelIndex":25,"row":18,"size_x":12,"size_y":7,"sort":["timestamp","desc"],"type":"search"},{"col":1,"id":"Error-Histogram-By-Sensor-Type","panelIndex":27,"row":3,"size_x":12,"size_y":3,"type":"visualization"},{"id":"Unique-Error-Histogram-By-Sensor-Type","type":"visualization","panelIndex":28,"size_x":12,"size_y":3,"col":1,"row":6}] +p919 sVoptionsJSON -p872 +p920 V{"darkTheme":false} -p873 +p921 sVversion -p874 +p922 I1 sVkibanaSavedObjectMeta -p875 -(dp876 +p923 +(dp924 VsearchSourceJSON -p877 +p925 V{"filter":[{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}}}]} -p878 +p926 sssV_index -p879 +p927 V.kibana -p880 -sa(dp881 +p928 +sa(dp929 V_score -p882 +p930 F1 sV_type -p883 +p931 Vconfig -p884 +p932 sV_id -p885 +p933 V4.5.3 -p886 +p934 sV_source -p887 -(dp888 +p935 +(dp936 VbuildNum -p889 +p937 I9892 sVdefaultIndex -p890 +p938 Vbro* -p891 +p939 ssV_index -p892 +p940 V.kibana -p893 -sa(dp894 +p941 +sa(dp942 V_score -p895 +p943 F1 sV_type -p896 +p944 Vsearch -p897 +p945 sV_id -p898 +p946 Vdns-search -p899 +p947 sV_source -p900 -(dp901 +p948 +(dp949 Vsort -p902 -(lp903 +p950 +(lp951 Vtimestamp -p904 +p952 aVdesc -p905 +p953 asVhits -p906 +p954 I0 sVdescription -p907 +p955 V sVtitle -p908 +p956 VDNS Requests -p909 +p957 sVversion -p910 +p958 I1 sVkibanaSavedObjectMeta -p911 -(dp912 +p959 +(dp960 VsearchSourceJSON -p913 +p961 V{"index":"bro*","query":{"query_string":{"query":"protocol: dns","analyze_wildcard":true}},"filter":[],"highlight":{"pre_tags":["@kibana-highlighted-field@"],"post_tags":["@/kibana-highlighted-field@"],"fields":{"*":{}},"require_field_match":false,"fragment_size":2147483647}} -p914 +p962 ssVcolumns -p915 -(lp916 +p963 +(lp964 Vquery -p917 +p965 aVqtype_name -p918 +p966 aVanswers -p919 +p967 aVip_src_addr -p920 +p968 aVip_dst_addr -p921 +p969 assV_index -p922 +p970 V.kibana -p923 -sa(dp924 +p971 +sa(dp972 V_score -p925 +p973 F1 sV_type -p926 +p974 Vvisualization -p927 +p975 sV_id -p928 +p976 VDNS-Requests-Header -p929 +p977 sV_source -p930 -(dp931 +p978 +(dp979 VvisState -p932 +p980 V{"aggs":[],"listeners":{},"params":{"markdown":"[Bro](https://www.bro.org/) is extracting DNS requests and responses being made over the network. Understanding who is making those requests, the frequency, and types can provide a deep understanding of the actors present on the network."},"title":"DNS Requests","type":"markdown"} -p933 +p981 sVdescription -p934 +p982 V sVtitle -p935 +p983 VDNS Requests -p936 +p984 sVuiStateJSON -p937 +p985 V{} -p938 +p986 sVversion -p939 +p987 I1 sVkibanaSavedObjectMeta -p940 -(dp941 +p988 +(dp989 VsearchSourceJSON -p942 +p990 V{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} -p943 +p991 sssV_index -p944 +p992 V.kibana -p945 -sa(dp946 +p993 +sa(dp994 V_score -p947 +p995 F1 sV_type -p948 +p996 Vvisualization -p949 +p997 sV_id -p950 +p998 VYAF-Flows-Header -p951 +p999 sV_source -p952 -(dp953 +p1000 +(dp1001 VvisState -p954 +p1002 V{"title":"YAF","type":"markdown","params":{"markdown":"[YAF](https://tools.netsa.cert.org/yaf/yaf.html) can be used to generate Netflow-like flow records. These flow records provide significant visibility of the actors communicating over the target network."},"aggs":[],"listeners":{}} -p955 +p1003 sVdescription -p956 +p1004 V sVtitle -p957 +p1005 VYAF -p958 +p1006 sVuiStateJSON -p959 +p1007 V{} -p960 +p1008 sVversion -p961 +p1009 I1 sVkibanaSavedObjectMeta -p962 -(dp963 +p1010 +(dp1011 VsearchSourceJSON -p964 +p1012 V{"query":{"query_string":{"analyze_wildcard":true,"query":"*"}},"filter":[]} -p965 +p1013 sssV_index -p966 +p1014 V.kibana -p967 -sa(dp968 +p1015 +sa(dp1016 V_score -p969 +p1017 F1 sV_type -p970 +p1018 Vvisualization -p971 +p1019 sV_id -p972 +p1020 VTop-5-Exceptions -p973 +p1021 sV_source -p974 -(dp975 +p1022 +(dp1023 VvisState -p976 +p1024 V{"title":"Top-5 Exceptions","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"stacked","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"exception","size":5,"order":"desc","orderBy":"1","customLabel":"Exceptions"}}],"listeners":{}} -p977 +p1025 sVdescription -p978 +p1026 V sVtitle -p979 +p1027 VTop-5 Exceptions -p980 +p1028 sVuiStateJSON -p981 +p1029 V{} -p982 +p1030 sVversion -p983 +p1031 I1 sVkibanaSavedObjectMeta -p984 -(dp985 +p1032 +(dp1033 VsearchSourceJSON -p986 +p1034 V{"index":"error*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p987 +p1035 sssV_index -p988 +p1036 V.kibana -p989 -sa(dp990 +p1037 +sa(dp1038 V_score -p991 +p1039 F1 sV_type -p992 +p1040 Vvisualization -p993 +p1041 sV_id -p994 +p1042 VFrequent-DNS-Requests -p995 +p1043 sV_source -p996 -(dp997 +p1044 +(dp1045 VvisState -p998 +p1046 V{"title":"Frequent DNS Requests","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"query","size":5,"order":"desc","orderBy":"1","customLabel":"DNS Query"}}],"listeners":{}} -p999 +p1047 sVdescription -p1000 +p1048 V sVtitle -p1001 +p1049 VFrequent DNS Requests -p1002 +p1050 sVuiStateJSON -p1003 +p1051 V{} -p1004 +p1052 sVversion -p1005 +p1053 I1 sVkibanaSavedObjectMeta -p1006 -(dp1007 +p1054 +(dp1055 VsearchSourceJSON -p1008 +p1056 V{"index":"bro*","query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p1009 +p1057 sssV_index -p1010 +p1058 V.kibana -p1011 -sa(dp1012 +p1059 +sa(dp1060 V_score -p1013 +p1061 F1 sV_type -p1014 +p1062 Vvisualization -p1015 +p1063 sV_id -p1016 +p1064 VCountry -p1017 +p1065 sV_source -p1018 -(dp1019 +p1066 +(dp1067 VvisState -p1020 +p1068 V{"title":"By Country","type":"pie","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"isDonut":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"segment","params":{"field":"enrichments:geo:ip_src_addr:country","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} -p1021 +p1069 sVdescription -p1022 +p1070 V sVtitle -p1023 +p1071 VBy Country -p1024 +p1072 sVuiStateJSON -p1025 +p1073 V{} -p1026 +p1074 sVversion -p1027 +p1075 I1 sVkibanaSavedObjectMeta -p1028 -(dp1029 +p1076 +(dp1077 VsearchSourceJSON -p1030 +p1078 V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p1031 +p1079 sssV_index -p1032 +p1080 V.kibana -p1033 -sa(dp1034 +p1081 +sa(dp1082 V_score -p1035 +p1083 F1 sV_type -p1036 +p1084 Vvisualization -p1037 +p1085 sV_id -p1038 +p1086 VFlow-Locations -p1039 +p1087 sV_source -p1040 -(dp1041 +p1088 +(dp1089 VvisState -p1042 +p1090 V{"title":"Flow Locations","type":"tile_map","params":{"mapType":"Scaled Circle Markers","isDesaturated":true,"addTooltip":true,"heatMaxZoom":16,"heatMinOpacity":0.1,"heatRadius":25,"heatBlur":15,"heatNormalizeData":true,"wms":{"enabled":false,"url":"https://basemap.nationalmap.gov/arcgis/services/USGSTopo/MapServer/WMSServer","options":{"version":"1.3.0","layers":"0","format":"image/png","transparent":true,"attribution":"Maps provided by USGS","styles":""}}},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"geohash_grid","schema":"segment","params":{"field":"enrichments:geo:ip_dst_addr:location_point","autoPrecision":true,"precision":2}}],"listeners":{}} -p1043 +p1091 sVdescription -p1044 +p1092 V sVtitle -p1045 +p1093 VFlow Locations -p1046 +p1094 sVuiStateJSON -p1047 +p1095 V{} -p1048 +p1096 sVversion -p1049 +p1097 I1 sVkibanaSavedObjectMeta -p1050 -(dp1051 +p1098 +(dp1099 VsearchSourceJSON -p1052 +p1100 V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p1053 +p1101 sssV_index -p1054 +p1102 V.kibana -p1055 -sa(dp1056 +p1103 +sa(dp1104 V_score -p1057 +p1105 F1 sV_type -p1058 +p1106 Vvisualization -p1059 +p1107 sV_id -p1060 +p1108 VTop-Destinations -p1061 +p1109 sV_source -p1062 -(dp1063 +p1110 +(dp1111 VvisState -p1064 +p1112 V{"title":"Top Destinations","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"terms","schema":"bucket","params":{"field":"ip_dst_addr","size":10,"order":"desc","orderBy":"1","customLabel":"Destination IP"}}],"listeners":{}} -p1065 +p1113 sVdescription -p1066 +p1114 V sVtitle -p1067 +p1115 VTop Destinations -p1068 +p1116 sVuiStateJSON -p1069 +p1117 V{} -p1070 +p1118 sVversion -p1071 +p1119 I1 sVkibanaSavedObjectMeta -p1072 -(dp1073 +p1120 +(dp1121 VsearchSourceJSON -p1074 +p1122 V{"index":["yaf*","bro*","snort*"],"query":{"query_string":{"query":"*","analyze_wildcard":true}},"filter":[]} -p1075 +p1123 sssV_index -p1076 +p1124 V.kibana -p1077 -sa(dp1078 +p1125 +sa(dp1126 V_score -p1079 +p1127 F1 sV_type -p1080 +p1128 Vvisualization -p1081 +p1129 sV_id -p1082 +p1130 VUnusual-Referrers -p1083 +p1131 sV_source -p1084 -(dp1085 +p1132 +(dp1133 VvisState -p1086 +p1134 V{"title":"Unusual Referrers","type":"table","params":{"perPage":10,"showPartialRows":false,"showMeticsAtAllLevels":false},"aggs":[{"id":"1","type":"count","schema":"metric","params":{}},{"id":"2","type":"significant_terms","schema":"bucket","params":{"field":"referrer","size":5,"customLabel":"Top 5 Unusual Referrers"}}],"listeners":{}} -p1087 +p1135 sVdescription -p1088 +p1136 V sVtitle -p1089 +p1137 VUnusual Referrers -p1090 +p1138 sVuiStateJSON -p1091 +p1139 V{} -p1092 +p1140 sVversion -p1093 +p1141 I1 sVsavedSearchId -p1094 +p1142 Vweb-search -p1095 +p1143 sVkibanaSavedObjectMeta -p1096 -(dp1097 +p1144 +(dp1145 VsearchSourceJSON -p1098 +p1146 V{"filter":[]} -p1099 +p1147 sssV_index -p1100 +p1148 V.kibana -p1101 +p1149 +sa(dp1150 +V_score +p1151 +F1 +sV_type +p1152 +Vvisualization +p1153 +sV_id +p1154 +VUnique-Error-Histogram-By-Sensor-Type +p1155 +sV_source +p1156 +(dp1157 +VvisState +p1158 +V{"title":"Error Histogram By Sensor Type","type":"histogram","params":{"shareYAxis":true,"addTooltip":true,"addLegend":true,"scale":"linear","mode":"grouped","times":[],"addTimeMarker":false,"defaultYExtents":false,"setYExtents":false,"yAxis":{}},"aggs":[{"id":"1","type":"cardinality","schema":"metric","params":{"field":"error_hash"}},{"id":"2","type":"date_histogram","schema":"segment","params":{"field":"timestamp","interval":"auto","customInterval":"2h","min_doc_count":1,"extended_bounds":{},"customLabel":"Time"}},{"id":"3","type":"terms","schema":"group","params":{"field":"failed_sensor_type","size":5,"order":"desc","orderBy":"1"}}],"listeners":{}} +p1159 +sVdescription +p1160 +V +sVtitle +p1161 +VUnique Error Histogram By Sensor Type +p1162 +sVuiStateJSON +p1163 +V{} +p1164 +sVversion +p1165 +I1 +sVsavedSearchId +p1166 +VErrors +p1167 +sVkibanaSavedObjectMeta +p1168 +(dp1169 +VsearchSourceJSON +p1170 +V{"filter":[]} +p1171 +sssV_index +p1172 +V.kibana +p1173 sa. \ No newline at end of file