From a7c7dc287b4f9c99c6780b934a0b6f433a03aa04 Mon Sep 17 00:00:00 2001 From: cstella Date: Mon, 8 Oct 2018 18:06:52 -0600 Subject: [PATCH 01/41] Casey Stella - elasticsearch rest client migration base work --- dependencies_with_url.csv | 33 ++-- .../CURRENT/configuration/metron-env.xml | 9 - .../CURRENT/package/scripts/metron_service.py | 2 - .../package/scripts/params/params_linux.py | 3 +- .../METRON/CURRENT/themes/metron_theme.json | 10 -- .../service/impl/MetaAlertServiceImpl.java | 2 +- metron-platform/elasticsearch-shaded/pom.xml | 28 +++- .../META-INF/log4j-provider.properties | 18 -- metron-platform/metron-elasticsearch/pom.xml | 29 +++- .../dao/ElasticsearchColumnMetadataDao.java | 82 +++++---- .../elasticsearch/dao/ElasticsearchDao.java | 17 +- .../dao/ElasticsearchMetaAlertDao.java | 2 +- .../dao/ElasticsearchMetaAlertSearchDao.java | 6 +- .../dao/ElasticsearchMetaAlertUpdateDao.java | 4 +- .../dao/ElasticsearchRequestSubmitter.java | 13 +- .../dao/ElasticsearchRetrieveLatestDao.java | 27 +-- .../dao/ElasticsearchSearchDao.java | 7 +- .../dao/ElasticsearchUpdateDao.java | 18 +- .../utils/ElasticsearchClient.java | 156 ++++++++++++++++++ .../utils/ElasticsearchUtils.java | 95 ++++++++--- .../elasticsearch/utils/FieldMapping.java | 29 ++++ .../elasticsearch/utils/FieldProperties.java | 33 ++++ .../writer/ElasticsearchWriter.java | 22 ++- .../ElasticsearchColumnMetadataDaoTest.java | 50 +++--- .../dao/ElasticsearchDaoTest.java | 7 +- .../ElasticsearchRequestSubmitterTest.java | 20 ++- ...ElasticsearchMetaAlertIntegrationTest.java | 9 +- .../ElasticsearchSearchIntegrationTest.java | 15 +- .../ElasticsearchUpdateIntegrationTest.java | 2 +- .../components/ElasticSearchComponent.java | 6 +- .../dao/metaalert/MetaAlertSearchDao.java | 4 +- .../metaalert/MetaAlertIntegrationTest.java | 2 +- .../src/main/config/zookeeper/global.json | 2 +- pom.xml | 2 +- 34 files changed, 532 insertions(+), 232 deletions(-) delete mode 100644 metron-platform/elasticsearch-shaded/src/main/resources/META-INF/log4j-provider.properties create mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchClient.java create mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldMapping.java create mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldProperties.java diff --git a/dependencies_with_url.csv b/dependencies_with_url.csv index 53977f3f8b..66497c314f 100644 --- a/dependencies_with_url.csv +++ b/dependencies_with_url.csv @@ -256,12 +256,8 @@ io.dropwizard.metrics:metrics-json:jar:3.1.5:compile,ASLv2,https://github.com/dr io.dropwizard.metrics:metrics-jvm:jar:3.1.5:compile,ASLv2,https://github.com/dropwizard/metrics io.netty:netty-all:jar:4.0.23.Final:compile,ASLv2, io.netty:netty-all:jar:4.0.23.Final:provided,ASLv2, -<<<<<<< HEAD io.netty:netty-all:jar:4.1.17.Final:compile,ASLv2, -======= io.netty:netty-all:jar:4.1.23.Final:compile,ASLv2, -io.netty:netty:jar:3.10.5.Final:compile,Apache License, Version 2.0,http://netty.io/ ->>>>>>> apache/master io.netty:netty:jar:3.6.2.Final:compile,Apache License, Version 2.0,http://netty.io/ io.netty:netty:jar:3.7.0.Final:compile,Apache License, Version 2.0,http://netty.io/ io.netty:netty:jar:3.9.9.Final:compile,Apache License, Version 2.0,http://netty.io/ @@ -472,20 +468,21 @@ org.eclipse.persistence:org.eclipse.persistence.jpa:jar:2.6.4:compile,EPL 1.0,ht com.github.ben-manes.caffeine:caffeine:jar:2.6.2:compile,ASLv2,https://github.com/ben-manes/caffeine/blob/v2.6.2/LICENSE com.google.code.gson:gson:jar:2.2:compile,ASLv2,https://github.com/google/gson com.google.code.gson:gson:jar:2.8.2:compile,ASLv2,https://github.com/google/gson - org.codehaus.plexus:plexus-classworlds:jar:2.4:compile - org.codehaus.plexus:plexus-component-annotations:jar:1.5.5:compile - org.codehaus.plexus:plexus-interpolation:jar:1.14:compile - org.codehaus.plexus:plexus-utils:jar:2.0.7:compile - org.jsoup:jsoup:jar:1.6.1:compile - org.sonatype.aether:aether-api:jar:1.12:compile - org.sonatype.aether:aether-connector-file:jar:1.12:compile - org.sonatype.aether:aether-connector-wagon:jar:1.12:compile - org.sonatype.aether:aether-impl:jar:1.12:compile - org.sonatype.aether:aether-spi:jar:1.12:compile - org.sonatype.aether:aether-util:jar:1.12:compile - org.sonatype.sisu:sisu-guice:jar:no_aop:3.0.2:compile - org.sonatype.sisu:sisu-inject-bean:jar:2.2.2:compile - org.sonatype.sisu:sisu-inject-plexus:jar:2.2.2:compile +org.codehaus.plexus:plexus-classworlds:jar:2.4:compile +org.codehaus.plexus:plexus-component-annotations:jar:1.5.5:compile +org.codehaus.plexus:plexus-interpolation:jar:1.14:compile +org.codehaus.plexus:plexus-utils:jar:2.0.7:compile +org.jsoup:jsoup:jar:1.6.1:compile +org.sonatype.aether:aether-api:jar:1.12:compile +org.sonatype.aether:aether-connector-file:jar:1.12:compile +org.sonatype.aether:aether-connector-wagon:jar:1.12:compile +org.sonatype.aether:aether-impl:jar:1.12:compile +org.sonatype.aether:aether-spi:jar:1.12:compile +org.sonatype.aether:aether-util:jar:1.12:compile +org.sonatype.sisu:sisu-guice:jar:no_aop:3.0.2:compile +org.sonatype.sisu:sisu-inject-bean:jar:2.2.2:compile +org.sonatype.sisu:sisu-inject-plexus:jar:2.2.2:compile com.zaxxer:HikariCP:jar:2.7.8:compile,ASLv2,https://github.com/brettwooldridge/HikariCP org.hibernate.validator:hibernate-validator:jar:6.0.9.Final:compile,ASLv2,https://github.com/hibernate/hibernate-validator com.github.palindromicity:simple-syslog-5424:jar:0.0.8:compile,ASLv2,https://github.com/palindromicity/simple-syslog-5424 +org.elasticsearch.client:elasticsearch-rest-high-level-client:jar:5.6.2:compile,ASLv2,https://github.com/elastic/elasticsearch/blob/master/LICENSE.txt 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 81dda6c149..e644b313b2 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 @@ -94,15 +94,6 @@ true - - es_binary_port - 9300 - Elasticsearch binary port. (9300) - Elasticsearch Binary Port - - true - - es_http_port 9200 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 9d15e93f63..a7074dafd4 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 @@ -583,8 +583,6 @@ def check_indexer_parameters(): missing.append("metron-env/es_cluster_name") if not config['configurations']['metron-env']['es_hosts']: missing.append("metron-env/es_hosts") - if not config['configurations']['metron-env']['es_binary_port']: - missing.append("metron-env/es_binary_port") if not config['configurations']['metron-env']['es_date_format']: missing.append("metron-env/es_date_format") 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 458a7bedea..dd00e9ce89 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 @@ -96,9 +96,8 @@ es_cluster_name = config['configurations']['metron-env']['es_cluster_name'] es_hosts = config['configurations']['metron-env']['es_hosts'] es_host_list = es_hosts.split(",") -es_binary_port = config['configurations']['metron-env']['es_binary_port'] -es_url = ",".join([host + ":" + es_binary_port for host in es_host_list]) es_http_port = config['configurations']['metron-env']['es_http_port'] +es_url = ",".join([host + ":" + es_http_port for host in es_host_list]) es_http_url = es_host_list[0] + ":" + es_http_port es_date_format = config['configurations']['metron-env']['es_date_format'] diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/themes/metron_theme.json b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/themes/metron_theme.json index 7e6c83a756..26c7f4eba2 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/themes/metron_theme.json +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/themes/metron_theme.json @@ -459,10 +459,6 @@ "config": "metron-env/es_hosts", "subsection-name": "subsection-index-settings" }, - { - "config": "metron-env/es_binary_port", - "subsection-name": "subsection-index-settings" - }, { "config": "metron-env/es_http_port", "subsection-name": "subsection-index-settings" @@ -924,12 +920,6 @@ "type": "text-field" } }, - { - "config": "metron-env/es_binary_port", - "widget": { - "type": "text-field" - } - }, { "config": "metron-env/es_http_port", "widget": { diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/MetaAlertServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/MetaAlertServiceImpl.java index bd8419fe9b..7581ef3378 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/MetaAlertServiceImpl.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/MetaAlertServiceImpl.java @@ -59,7 +59,7 @@ public Document create(MetaAlertCreateRequest createRequest) throws RestExceptio public SearchResponse getAllMetaAlertsForAlert(String guid) throws RestException { try { return dao.getAllMetaAlertsForAlert(guid); - } catch (InvalidSearchException ise) { + } catch (IOException|InvalidSearchException ise) { throw new RestException(ise.getMessage(), ise); } } diff --git a/metron-platform/elasticsearch-shaded/pom.xml b/metron-platform/elasticsearch-shaded/pom.xml index ccad3cb3ae..d9002e4bfc 100644 --- a/metron-platform/elasticsearch-shaded/pom.xml +++ b/metron-platform/elasticsearch-shaded/pom.xml @@ -30,6 +30,11 @@ 18.0 + io.netty + netty-common + 4.1.13.Final + + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${global_elasticsearch_version} + + + + org.elasticsearch.plugin + aggs-matrix-stats-client + + + org.apache.logging.log4j log4j-core @@ -154,10 +176,6 @@ io.netty org.apache.metron.io.netty - - org.apache.logging.log4j - org.apache.metron.logging.log4j - com.google.common org.apache.metron.guava.elasticsearch-shaded diff --git a/metron-platform/elasticsearch-shaded/src/main/resources/META-INF/log4j-provider.properties b/metron-platform/elasticsearch-shaded/src/main/resources/META-INF/log4j-provider.properties deleted file mode 100644 index c4bd3f06b6..0000000000 --- a/metron-platform/elasticsearch-shaded/src/main/resources/META-INF/log4j-provider.properties +++ /dev/null @@ -1,18 +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. - -LoggerContextFactory = org.apache.metron.logging.log4j.core.impl.Log4jContextFactory -Log4jAPIVersion = 2.6.0 -FactoryPriority= 10 \ No newline at end of file diff --git a/metron-platform/metron-elasticsearch/pom.xml b/metron-platform/metron-elasticsearch/pom.xml index adc601aa17..593e80b0f8 100644 --- a/metron-platform/metron-elasticsearch/pom.xml +++ b/metron-platform/metron-elasticsearch/pom.xml @@ -72,6 +72,17 @@ + + org.apache.httpcomponents + httpcore + 4.4.9 + + + org.elasticsearch.plugin + transport-netty4-client + ${global_elasticsearch_version} + test + org.apache.metron metron-hbase @@ -209,13 +220,25 @@ org.apache.logging.log4j log4j-api - ${global_log4j_core_version} + 2.8.2 + test org.apache.logging.log4j log4j-core + 2.8.2 + test + + com.google.guava guava-testlib @@ -297,9 +320,9 @@ + + implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/--> diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java index 6a8cad8cc1..64a641f5e2 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java @@ -18,10 +18,17 @@ package org.apache.metron.elasticsearch.dao; +import org.apache.metron.elasticsearch.utils.ElasticsearchClient; +import org.apache.metron.elasticsearch.utils.FieldMapping; +import org.apache.metron.elasticsearch.utils.FieldProperties; import org.apache.metron.indexing.dao.ColumnMetadataDao; import org.apache.metron.indexing.dao.search.FieldType; +import org.elasticsearch.action.admin.indices.get.GetIndexRequest; import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.client.AdminClient; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.slf4j.Logger; @@ -64,12 +71,12 @@ public class ElasticsearchColumnMetadataDao implements ColumnMetadataDao { /** * An Elasticsearch administrative client. */ - private transient AdminClient adminClient; + private transient ElasticsearchClient adminClient; /** * @param adminClient The Elasticsearch admin client. */ - public ElasticsearchColumnMetadataDao(AdminClient adminClient) { + public ElasticsearchColumnMetadataDao(ElasticsearchClient adminClient) { this.adminClient = adminClient; } @@ -82,51 +89,40 @@ public Map getColumnMetadata(List indices) throws IOE String[] latestIndices = getLatestIndices(indices); if (latestIndices.length > 0) { - ImmutableOpenMap> mappings = adminClient - .indices() - .getMappings(new GetMappingsRequest().indices(latestIndices)) - .actionGet() - .getMappings(); + + Map mappings = adminClient.getMappings(latestIndices); // for each index - for (Object key : mappings.keys().toArray()) { - String indexName = key.toString(); - ImmutableOpenMap mapping = mappings.get(indexName); + for (Map.Entry kv : mappings.entrySet()) { + String indexName = kv.getKey(); + FieldMapping mapping = kv.getValue(); // for each mapping in the index - Iterator mappingIterator = mapping.keysIt(); - while (mappingIterator.hasNext()) { - MappingMetaData mappingMetaData = mapping.get(mappingIterator.next()); - Map sourceAsMap = mappingMetaData.getSourceAsMap(); - if (sourceAsMap.containsKey("properties")) { - Map> map = (Map>) sourceAsMap.get("properties"); - - // for each field in the mapping - for (String field : map.keySet()) { - if (!fieldBlackList.contains(field)) { - FieldType type = toFieldType(map.get(field).get("type")); - - if(!indexColumnMetadata.containsKey(field)) { - indexColumnMetadata.put(field, type); - - // record the last index in which a field exists, to be able to print helpful error message on type mismatch - previousIndices.put(field, indexName); - - } else { - FieldType previousType = indexColumnMetadata.get(field); - if (!type.equals(previousType)) { - String previousIndexName = previousIndices.get(field); - LOG.error(String.format( + for(Map.Entry fieldToProperties : mapping.entrySet()) { + String field = fieldToProperties.getKey(); + FieldProperties properties = fieldToProperties.getValue(); + if (!fieldBlackList.contains(field)) { + FieldType type = toFieldType((String) properties.get("type")); + + if(!indexColumnMetadata.containsKey(field)) { + indexColumnMetadata.put(field, type); + + // record the last index in which a field exists, to be able to print helpful error message on type mismatch + previousIndices.put(field, indexName); + + } else { + FieldType previousType = indexColumnMetadata.get(field); + if (!type.equals(previousType)) { + String previousIndexName = previousIndices.get(field); + LOG.error(String.format( "Field type mismatch: %s.%s has type %s while %s.%s has type %s. Defaulting type to %s.", indexName, field, type.getFieldType(), previousIndexName, field, previousType.getFieldType(), FieldType.OTHER.getFieldType())); - indexColumnMetadata.put(field, FieldType.OTHER); + indexColumnMetadata.put(field, FieldType.OTHER); - // the field is defined in multiple indices with different types; ignore the field as type has been set to OTHER - fieldBlackList.add(field); - } - } + // the field is defined in multiple indices with different types; ignore the field as type has been set to OTHER + fieldBlackList.add(field); } } } @@ -166,15 +162,11 @@ public Map getColumnMetadata(List indices) throws IOE * @param includeIndices The base names of the indices to include * @return The latest version of a set of indices. */ - String[] getLatestIndices(List includeIndices) { + String[] getLatestIndices(List includeIndices) throws IOException { LOG.debug("Getting latest indices; indices={}", includeIndices); Map latestIndices = new HashMap<>(); - String[] indices = adminClient - .indices() - .prepareGetIndex() - .setFeatures() - .get() - .getIndices(); + + String[] indices = adminClient.getIndices(); for (String index : indices) { int prefixEnd = index.indexOf(INDEX_NAME_DELIMITER); diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java index 9f6e1a1cf8..fa04610b79 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java @@ -22,6 +22,8 @@ import java.util.List; import java.util.Map; import java.util.Optional; + +import org.apache.metron.elasticsearch.utils.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.IndexDao; @@ -38,6 +40,7 @@ import org.apache.metron.indexing.dao.update.OriginalNotFoundException; import org.apache.metron.indexing.dao.update.PatchRequest; import org.apache.metron.indexing.dao.update.ReplaceRequest; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.index.query.QueryBuilder; import org.slf4j.Logger; @@ -47,7 +50,7 @@ public class ElasticsearchDao implements IndexDao { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private transient TransportClient client; + private transient ElasticsearchClient client; private ElasticsearchSearchDao searchDao; private ElasticsearchUpdateDao updateDao; private ElasticsearchRetrieveLatestDao retrieveLatestDao; @@ -64,7 +67,7 @@ public class ElasticsearchDao implements IndexDao { private AccessConfig accessConfig; - protected ElasticsearchDao(TransportClient client, + protected ElasticsearchDao(ElasticsearchClient client, AccessConfig config, ElasticsearchSearchDao searchDao, ElasticsearchUpdateDao updateDao, @@ -99,7 +102,7 @@ public synchronized void init(AccessConfig config) { this.client = ElasticsearchUtils .getClient(config.getGlobalConfigSupplier().get()); this.accessConfig = config; - this.columnMetadataDao = new ElasticsearchColumnMetadataDao(this.client.admin()); + this.columnMetadataDao = new ElasticsearchColumnMetadataDao(this.client); this.requestSubmitter = new ElasticsearchRequestSubmitter(this.client); this.searchDao = new ElasticsearchSearchDao(client, accessConfig, columnMetadataDao, requestSubmitter); @@ -127,13 +130,13 @@ public GroupResponse group(GroupRequest groupRequest) throws InvalidSearchExcept } @Override - public Document getLatest(final String guid, final String sensorType) { + public Document getLatest(final String guid, final String sensorType) throws IOException { return retrieveLatestDao.getLatest(guid, sensorType); } @Override public Iterable getAllLatest( - final List getRequests) { + final List getRequests) throws IOException { return retrieveLatestDao.getAllLatest(getRequests); } @@ -188,7 +191,7 @@ public Document removeCommentFromAlert(CommentAddRemoveRequest request, Document return this.updateDao.removeCommentFromAlert(request, latest); } - protected Optional getIndexName(String guid, String sensorType) { + protected Optional getIndexName(String guid, String sensorType) throws IOException { return updateDao.getIndexName(guid, sensorType); } @@ -202,7 +205,7 @@ protected GroupResponse group(GroupRequest groupRequest, QueryBuilder queryBuild return searchDao.group(groupRequest, queryBuilder); } - public TransportClient getClient() { + public ElasticsearchClient getClient() { return this.client; } } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDao.java index fc0b20c14a..ac5417e153 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDao.java @@ -176,7 +176,7 @@ public Iterable getAllLatest(List getRequests) throws IOEx } @Override - public SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException { + public SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException, IOException { return metaAlertSearchDao.getAllMetaAlertsForAlert(guid); } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertSearchDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertSearchDao.java index 00fc9d02f6..65bfa206f8 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertSearchDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertSearchDao.java @@ -41,6 +41,8 @@ import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryStringQueryBuilder; +import java.io.IOException; + public class ElasticsearchMetaAlertSearchDao implements MetaAlertSearchDao { protected ElasticsearchDao elasticsearchDao; @@ -89,7 +91,7 @@ public GroupResponse group(GroupRequest groupRequest) throws InvalidSearchExcept } @Override - public SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException { + public SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException, IOException { if (guid == null || guid.trim().isEmpty()) { throw new InvalidSearchException("Guid cannot be empty"); } @@ -104,7 +106,7 @@ public SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearch ).innerHit(new InnerHitBuilder()) ) .must(termQuery(MetaAlertConstants.STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString())); - return queryAllResults(elasticsearchDao.getClient(), qb, config.getMetaAlertIndex(), + return queryAllResults(elasticsearchDao.getClient().getHighLevelClient(), qb, config.getMetaAlertIndex(), pageSize); } } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertUpdateDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertUpdateDao.java index 3b6789118f..2e9c855694 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertUpdateDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertUpdateDao.java @@ -199,7 +199,7 @@ public Document removeCommentFromAlert(CommentAddRemoveRequest request, Document * @param alertGuid The GUID of the child alert * @return The Elasticsearch response containing the meta alerts */ - protected SearchResponse getMetaAlertsForAlert(String alertGuid) { + protected SearchResponse getMetaAlertsForAlert(String alertGuid) throws IOException { QueryBuilder qb = boolQuery() .must( nestedQuery( @@ -212,7 +212,7 @@ protected SearchResponse getMetaAlertsForAlert(String alertGuid) { ) .must(termQuery(MetaAlertConstants.STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString())); return ElasticsearchUtils - .queryAllResults(elasticsearchDao.getClient(), qb, getConfig().getMetaAlertIndex(), + .queryAllResults(elasticsearchDao.getClient().getHighLevelClient(), qb, getConfig().getMetaAlertIndex(), pageSize); } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRequestSubmitter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRequestSubmitter.java index 0e0df21fb3..64d9200548 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRequestSubmitter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRequestSubmitter.java @@ -20,17 +20,20 @@ import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.metron.elasticsearch.utils.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.search.InvalidSearchException; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.rest.RestStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.lang.invoke.MethodHandles; /** @@ -43,9 +46,9 @@ public class ElasticsearchRequestSubmitter { /** * The Elasticsearch client. */ - private TransportClient client; + private ElasticsearchClient client; - public ElasticsearchRequestSubmitter(TransportClient client) { + public ElasticsearchRequestSubmitter(ElasticsearchClient client) { this.client = client; } @@ -60,12 +63,10 @@ public SearchResponse submitSearch(SearchRequest request) throws InvalidSearchEx // submit the search request org.elasticsearch.action.search.SearchResponse esResponse; try { - esResponse = client - .search(request) - .actionGet(); + esResponse = client.getHighLevelClient().search(request); LOG.debug("Got Elasticsearch response; response={}", esResponse.toString()); - } catch (SearchPhaseExecutionException e) { + } catch (Exception e) { String msg = String.format( "Failed to execute search; error='%s', search='%s'", ExceptionUtils.getRootCauseMessage(e), diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java index f6bfedafaa..ff1189c793 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java @@ -28,33 +28,38 @@ import java.util.List; import java.util.Optional; import java.util.function.Function; + +import org.apache.metron.elasticsearch.utils.ElasticsearchClient; import org.apache.metron.indexing.dao.RetrieveLatestDao; import org.apache.metron.indexing.dao.search.GetRequest; import org.apache.metron.indexing.dao.update.Document; +import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.index.query.IdsQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; +import org.elasticsearch.search.builder.SearchSourceBuilder; public class ElasticsearchRetrieveLatestDao implements RetrieveLatestDao { - private TransportClient transportClient; + private ElasticsearchClient transportClient; - public ElasticsearchRetrieveLatestDao(TransportClient transportClient) { + public ElasticsearchRetrieveLatestDao(ElasticsearchClient transportClient) { this.transportClient = transportClient; } @Override - public Document getLatest(String guid, String sensorType) { + public Document getLatest(String guid, String sensorType) throws IOException { Optional doc = searchByGuid(guid, sensorType, hit -> toDocument(guid, hit)); return doc.orElse(null); } @Override - public Iterable getAllLatest(List getRequests) { + public Iterable getAllLatest(List getRequests) throws IOException { Collection guids = new HashSet<>(); Collection sensorTypes = new HashSet<>(); for (GetRequest getRequest : getRequests) { @@ -80,7 +85,7 @@ public Iterable getAllLatest(List getRequests) { } Optional searchByGuid(String guid, String sensorType, - Function> callback) { + Function> callback) throws IOException { Collection sensorTypes = sensorType != null ? Collections.singleton(sensorType) : null; List results = searchByGuids(Collections.singleton(guid), sensorTypes, callback); if (results.size() > 0) { @@ -96,7 +101,7 @@ Optional searchByGuid(String guid, String sensorType, * If more than one hit happens, the first one will be returned. */ List searchByGuids(Collection guids, Collection sensorTypes, - Function> callback) { + Function> callback) throws IOException { if (guids == null || guids.isEmpty()) { return Collections.emptyList(); } @@ -113,11 +118,13 @@ List searchByGuids(Collection guids, Collection sensorTyp for (String guid : guids) { query = idsQuery.addIds(guid); } + SearchRequest request = new SearchRequest(); + SearchSourceBuilder builder = new SearchSourceBuilder(); + builder.query(query); + builder.size(guids.size()); + request.source(builder); - SearchRequestBuilder request = transportClient.prepareSearch() - .setQuery(query) - .setSize(guids.size()); - org.elasticsearch.action.search.SearchResponse response = request.get(); + org.elasticsearch.action.search.SearchResponse response = transportClient.getHighLevelClient().search(request); SearchHits hits = response.getHits(); List results = new ArrayList<>(); for (SearchHit hit : hits) { diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchDao.java index 5cd0a4d477..32cefe0c82 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchDao.java @@ -33,6 +33,8 @@ import java.util.Map; import java.util.Optional; import java.util.function.Function; + +import org.apache.metron.elasticsearch.utils.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.search.FieldType; @@ -52,6 +54,7 @@ import org.apache.metron.indexing.dao.search.SortOrder; import org.apache.metron.indexing.dao.update.Document; import org.elasticsearch.action.search.SearchRequestBuilder; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.index.mapper.LegacyIpFieldMapper; import org.elasticsearch.index.query.IdsQueryBuilder; @@ -88,12 +91,12 @@ public class ElasticsearchSearchDao implements SearchDao { */ private static final String SORT_MISSING_FIRST = "_first"; - private transient TransportClient client; + private transient ElasticsearchClient client; private AccessConfig accessConfig; private ElasticsearchColumnMetadataDao columnMetadataDao; private ElasticsearchRequestSubmitter requestSubmitter; - public ElasticsearchSearchDao(TransportClient client, + public ElasticsearchSearchDao(ElasticsearchClient client, AccessConfig accessConfig, ElasticsearchColumnMetadataDao columnMetadataDao, ElasticsearchRequestSubmitter requestSubmitter) { diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java index 6843ac75f3..75300ea576 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java @@ -28,17 +28,21 @@ import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; + +import org.apache.metron.elasticsearch.utils.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.search.AlertComment; import org.apache.metron.indexing.dao.update.CommentAddRemoveRequest; import org.apache.metron.indexing.dao.update.Document; import org.apache.metron.indexing.dao.update.UpdateDao; +import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.transport.TransportClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,11 +51,11 @@ public class ElasticsearchUpdateDao implements UpdateDao { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private transient TransportClient client; + private transient ElasticsearchClient client; private AccessConfig accessConfig; private ElasticsearchRetrieveLatestDao retrieveLatestDao; - public ElasticsearchUpdateDao(TransportClient client, + public ElasticsearchUpdateDao(ElasticsearchClient client, AccessConfig accessConfig, ElasticsearchRetrieveLatestDao searchDao) { this.client = client; @@ -68,7 +72,7 @@ public Document update(Document update, Optional index) throws IOExcepti IndexRequest indexRequest = buildIndexRequest(update, sensorType, indexName); try { - IndexResponse response = client.index(indexRequest).get(); + IndexResponse response = client.getHighLevelClient().index(indexRequest); ShardInfo shardInfo = response.getShardInfo(); int failed = shardInfo.getFailed(); @@ -87,7 +91,7 @@ public Map> batchUpdate(Map> updateEntry : updates.entrySet()) { @@ -103,7 +107,7 @@ public Map> batchUpdate(Map index, String indexPostFix) { + protected String getIndexName(Document update, Optional index, String indexPostFix) throws IOException { return index.orElse(getIndexName(update.getGuid(), update.getSensorType()) .orElse(ElasticsearchUtils.getIndexName(update.getSensorType(), indexPostFix, null)) ); } - protected Optional getIndexName(String guid, String sensorType) { + protected Optional getIndexName(String guid, String sensorType) throws IOException { return retrieveLatestDao.searchByGuid(guid, sensorType, hit -> Optional.ofNullable(hit.getIndex()) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchClient.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchClient.java new file mode 100644 index 0000000000..669ac10840 --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchClient.java @@ -0,0 +1,156 @@ +/** + * 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.elasticsearch.utils; + +import com.google.common.base.Joiner; +import com.google.common.base.Splitter; +import com.google.common.collect.Iterables; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.http.HttpEntity; +import org.apache.http.entity.BasicHttpEntity; +import org.apache.http.entity.StringEntity; +import org.apache.metron.common.utils.JSONUtils; +import org.elasticsearch.action.admin.indices.get.GetIndexRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.cluster.metadata.MappingMetaData; +import org.elasticsearch.common.collect.ImmutableOpenMap; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ElasticsearchClient implements AutoCloseable{ + private RestClient lowLevelClient; + private RestHighLevelClient highLevelClient; + + public ElasticsearchClient(RestClient lowLevelClient, RestHighLevelClient highLevelClient) { + this.lowLevelClient = lowLevelClient; + this.highLevelClient = highLevelClient; + } + + public RestClient getLowLevelClient() { + return lowLevelClient; + } + + public RestHighLevelClient getHighLevelClient() { + return highLevelClient; + } + + @Override + public void close() throws IOException { + if(lowLevelClient != null) { + lowLevelClient.close(); + } + } + + public void putMapping(String index, String type, String source) throws IOException { + HttpEntity entity = new StringEntity(source); + Response response = lowLevelClient.performRequest("PUT" + , "/" + index + "/_mapping/" + type + , Collections.emptyMap() + , entity + ); + + if(response.getStatusLine().getStatusCode() != 200) { + String responseStr = IOUtils.toString(response.getEntity().getContent()); + throw new IllegalStateException("Got a " + response.getStatusLine().getStatusCode() + " due to " + responseStr); + } + /** + * ((ElasticsearchDao) esDao).getClient().admin().indices().preparePutMapping(INDEX) + .setType("test_doc") + .setSource(nestedAlertMapping) + .get(); + */ + } + + public String[] getIndices() throws IOException { + Response response = lowLevelClient.performRequest("GET", "/_cat/indices"); + if(response.getStatusLine().getStatusCode() == 200) { + String responseStr = IOUtils.toString(response.getEntity().getContent()); + List indices = new ArrayList<>(); + for(String line : Splitter.on("\n").split(responseStr)) { + Iterable splits = Splitter.on(" ").split(line.replaceAll("\\s+", " ").trim()); + if(Iterables.size(splits) > 3) { + String index = Iterables.get(splits, 2, ""); + if(!StringUtils.isEmpty(index)) { + indices.add(index.trim()); + } + } + } + String[] ret = new String[indices.size()]; + ret=indices.toArray(ret); + return ret; + } + return null; + } + + private Map getInnerMap(Map outerMap, String... keys) { + Map ret = outerMap; + if(keys.length == 0) { + return outerMap; + } + for(String key : keys) { + ret = (Map)ret.get(key); + if(ret == null) { + return ret; + } + } + return ret; + } + + public Map getMappings(String[] indices) throws IOException { + Map ret = new HashMap<>(); + String indicesCsv = Joiner.on(",").join(indices); + Response response = lowLevelClient.performRequest("GET", "/" + indicesCsv + "/_mapping"); + if(response.getStatusLine().getStatusCode() == 200) { + String responseStr = IOUtils.toString(response.getEntity().getContent()); + Map indexToMapping = JSONUtils.INSTANCE.load(responseStr, JSONUtils.MAP_SUPPLIER); + for(Map.Entry index2Mapping : indexToMapping.entrySet()) { + String index = index2Mapping.getKey(); + Map mappings = getInnerMap((Map)index2Mapping.getValue(), "mappings"); + if(mappings.size() > 0) { + Map.Entry docMap = Iterables.getFirst(mappings.entrySet(), null); + if(docMap != null) { + Map fieldPropertiesMap = getInnerMap((Map)docMap.getValue(), "properties"); + if(fieldPropertiesMap != null) { + FieldMapping mapping = new FieldMapping(); + for (Map.Entry field2PropsKV : fieldPropertiesMap.entrySet()) { + if(field2PropsKV.getValue() != null) { + FieldProperties props = new FieldProperties((Map) field2PropsKV.getValue()); + mapping.put(field2PropsKV.getKey(), props); + } + } + ret.put(index, mapping); + } + } + } + } + } + return ret; + } + +} diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java index 98dc66d813..838f8c74ba 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java @@ -27,6 +27,7 @@ import java.net.InetAddress; import java.net.UnknownHostException; import java.text.SimpleDateFormat; +import java.util.AbstractMap; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -37,23 +38,35 @@ import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; + import org.apache.commons.lang.StringUtils; +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.common.utils.HDFSUtils; import org.apache.metron.common.utils.ReflectionUtils; +import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; import org.apache.metron.indexing.dao.search.SearchResult; import org.apache.metron.netty.utils.NettyRuntimeWrapper; import org.apache.metron.stellar.common.utils.ConversionUtils; import org.codehaus.jackson.map.ObjectMapper; import org.elasticsearch.action.search.SearchRequestBuilder; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.transport.client.PreBuiltTransportClient; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.query.QuerySearchRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -130,11 +143,36 @@ public static String getBaseIndexName(String indexName) { * @param globalConfiguration Metron global config * @return */ - public static TransportClient getClient(Map globalConfiguration) { + public static ElasticsearchClient getClient(Map globalConfiguration) { + Map esSettings = getEsSettings(globalConfiguration); + Optional> credentials = getCredentials(esSettings); Set customESSettings = new HashSet<>(); - customESSettings.addAll(Arrays.asList("es.client.class", USERNAME_CONFIG_KEY, PWD_FILE_CONFIG_KEY)); + + + RestClientBuilder builder = null; + List hps = getIps(globalConfiguration); + { + HttpHost[] posts = new HttpHost[hps.size()]; + int i = 0; + for (HostnamePort hp : hps) { + posts[i++] = new HttpHost(hp.hostname, hp.port); + } + builder = RestClient.builder(posts); + } + if(credentials.isPresent()) { + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials(AuthScope.ANY, + new UsernamePasswordCredentials(credentials.get().getKey(), credentials.get().getValue())); + builder = builder.setHttpClientConfigCallback( + httpAsyncClientBuilder -> httpAsyncClientBuilder.setDefaultCredentialsProvider(credentialsProvider) + ); + } + RestClient lowLevelClient = builder.build(); + RestHighLevelClient client = new RestHighLevelClient(lowLevelClient); + return new ElasticsearchClient(lowLevelClient, client); + + /*customESSettings.addAll(Arrays.asList("es.client.class", USERNAME_CONFIG_KEY, PWD_FILE_CONFIG_KEY)); Settings.Builder settingsBuilder = Settings.builder(); - Map esSettings = getEsSettings(globalConfiguration); for (Map.Entry entry : esSettings.entrySet()) { String key = entry.getKey(); String value = entry.getValue(); @@ -162,7 +200,7 @@ public static TransportClient getClient(Map globalConfiguration) return client; } catch (UnknownHostException exception) { throw new RuntimeException(exception); - } + }*/ } private static Map getEsSettings(Map config) { @@ -171,6 +209,22 @@ private static Map getEsSettings(Map config) { String.class); } + private static Optional> getCredentials(Map esSettings) { + Optional> ret = Optional.empty(); + if (esSettings.containsKey(PWD_FILE_CONFIG_KEY)) { + + if (!esSettings.containsKey(USERNAME_CONFIG_KEY) || StringUtils.isEmpty(esSettings.get(USERNAME_CONFIG_KEY))) { + throw new IllegalArgumentException("X-pack username is required and cannot be empty"); + } + String user = esSettings.get(USERNAME_CONFIG_KEY); + String password = esSettings.containsKey(PWD_FILE_CONFIG_KEY)?esSettings.get(getPasswordFromFile(esSettings.get(PWD_FILE_CONFIG_KEY))):null; + if(user != null && password != null) { + return Optional.of(new AbstractMap.SimpleImmutableEntry(user, password)); + } + } + return ret; + } + /* * Append Xpack security settings (if any) */ @@ -335,30 +389,29 @@ public static Optional toJSON(Object request) { * @param qb A QueryBuilder that provides the query to be run. * @return A SearchResponse containing the appropriate results. */ - public static SearchResponse queryAllResults(TransportClient transportClient, + public static SearchResponse queryAllResults(RestHighLevelClient transportClient, QueryBuilder qb, String index, int pageSize - ) { - SearchRequestBuilder searchRequestBuilder = transportClient - .prepareSearch(index) - .addStoredField("*") - .setFetchSource(true) - .setQuery(qb) - .setSize(pageSize); - org.elasticsearch.action.search.SearchResponse esResponse = searchRequestBuilder - .execute() - .actionGet(); + ) throws IOException { + org.elasticsearch.action.search.SearchRequest request = new org.elasticsearch.action.search.SearchRequest(); + SearchSourceBuilder builder = new SearchSourceBuilder(); + builder.query(qb); + builder.size(pageSize); + builder.fetchSource(true); + builder.storedField("*"); + request.source(builder); + request.indices(index); + + org.elasticsearch.action.search.SearchResponse esResponse = transportClient.search(request); List allResults = getSearchResults(esResponse); long total = esResponse.getHits().getTotalHits(); if (total > pageSize) { int pages = (int) (total / pageSize) + 1; for (int i = 1; i < pages; i++) { int from = i * pageSize; - searchRequestBuilder.setFrom(from); - esResponse = searchRequestBuilder - .execute() - .actionGet(); + builder.from(from); + esResponse = transportClient.search(request); allResults.addAll(getSearchResults(esResponse)); } } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldMapping.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldMapping.java new file mode 100644 index 0000000000..101e28812e --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldMapping.java @@ -0,0 +1,29 @@ +/** + * 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.elasticsearch.utils; + +import org.apache.commons.collections4.map.AbstractMapDecorator; + +import java.util.HashMap; + +public class FieldMapping extends AbstractMapDecorator{ + public FieldMapping() { + super(new HashMap()); + } + +} diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldProperties.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldProperties.java new file mode 100644 index 0000000000..82aca42134 --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldProperties.java @@ -0,0 +1,33 @@ +/** + * 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.elasticsearch.utils; + +import org.apache.commons.collections4.map.AbstractMapDecorator; + +import java.util.HashMap; +import java.util.Map; + +public class FieldProperties extends AbstractMapDecorator { + public FieldProperties() { + super(new HashMap<>()); + } + + public FieldProperties(Map m) { + super(m); + } +} diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java index 4b8dd083e9..20f387f84a 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java @@ -23,13 +23,17 @@ import org.apache.metron.common.field.FieldNameConverters; import org.apache.metron.common.writer.BulkMessageWriter; import org.apache.metron.common.writer.BulkWriterResponse; +import org.apache.metron.elasticsearch.utils.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.storm.task.TopologyContext; import org.apache.storm.tuple.Tuple; import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.transport.TransportClient; import org.json.simple.JSONObject; import org.slf4j.Logger; @@ -53,7 +57,7 @@ public class ElasticsearchWriter implements BulkMessageWriter, Seria /** * The Elasticsearch client. */ - private transient TransportClient client; + private transient ElasticsearchClient client; /** * A simple data formatter used to build the appropriate Elasticsearch index name. @@ -76,7 +80,8 @@ public BulkWriterResponse write(String sensorType, WriterConfiguration configura FieldNameConverter fieldNameConverter = FieldNameConverters.create(sensorType, configurations); final String indexPostfix = dateFormat.format(new Date()); - BulkRequestBuilder bulkRequest = client.prepareBulk(); + BulkRequest bulkRequest = new BulkRequest(); + //BulkRequestBuilder bulkRequest = client.prepareBulk(); for(JSONObject message: messages) { JSONObject esDoc = new JSONObject(); @@ -85,22 +90,21 @@ public BulkWriterResponse write(String sensorType, WriterConfiguration configura } String indexName = ElasticsearchUtils.getIndexName(sensorType, indexPostfix, configurations); - IndexRequestBuilder indexRequestBuilder = client.prepareIndex(indexName, sensorType + "_doc"); - indexRequestBuilder = indexRequestBuilder.setSource(esDoc.toJSONString()); + IndexRequest indexRequest = new IndexRequest(indexName, sensorType + "_doc"); + indexRequest.source(esDoc.toJSONString()); String guid = (String)esDoc.get(Constants.GUID); if(guid != null) { - indexRequestBuilder.setId(guid); + indexRequest.id(guid); } Object ts = esDoc.get("timestamp"); if(ts != null) { - indexRequestBuilder = indexRequestBuilder.setTimestamp(ts.toString()); + indexRequest.timestamp(ts.toString()); } - - bulkRequest.add(indexRequestBuilder); + bulkRequest.add(indexRequest); } - BulkResponse bulkResponse = bulkRequest.execute().actionGet(); + BulkResponse bulkResponse = client.getHighLevelClient().bulk(bulkRequest); return buildWriteReponse(tuples, bulkResponse); } diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDaoTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDaoTest.java index 0a83ee0918..e2a675fc89 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDaoTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDaoTest.java @@ -18,19 +18,26 @@ package org.apache.metron.elasticsearch.dao; +import org.apache.metron.elasticsearch.utils.ElasticsearchClient; +import org.apache.metron.elasticsearch.utils.FieldMapping; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.admin.indices.get.GetIndexRequestBuilder; import org.elasticsearch.action.admin.indices.get.GetIndexResponse; import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse; import org.elasticsearch.client.AdminClient; import org.elasticsearch.client.IndicesAdminClient; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.junit.Test; +import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static org.junit.Assert.assertArrayEquals; import static org.mockito.Matchers.any; @@ -47,7 +54,7 @@ public class ElasticsearchColumnMetadataDaoTest { * @return An object to test. */ public ElasticsearchColumnMetadataDao setup(String[] indices) { - return setup(indices, ImmutableOpenMap.of()); + return setup(indices, new HashMap<>()); } /** @@ -57,32 +64,23 @@ public ElasticsearchColumnMetadataDao setup(String[] indices) { */ public ElasticsearchColumnMetadataDao setup( String[] indices, - ImmutableOpenMap> mappings) { - - AdminClient adminClient = mock(AdminClient.class); - IndicesAdminClient indicesAdminClient = mock(IndicesAdminClient.class); - GetIndexRequestBuilder getIndexRequestBuilder = mock(GetIndexRequestBuilder.class); - GetIndexResponse getIndexResponse = mock(GetIndexResponse.class); - ActionFuture getMappingsActionFuture = mock(ActionFuture.class); - GetMappingsResponse getMappingsResponse = mock(GetMappingsResponse.class); - - // setup the mocks so that a set of indices are available to the DAO - when(adminClient.indices()).thenReturn(indicesAdminClient); - when(indicesAdminClient.prepareGetIndex()).thenReturn(getIndexRequestBuilder); - when(getIndexRequestBuilder.setFeatures()).thenReturn(getIndexRequestBuilder); - when(getIndexRequestBuilder.get()).thenReturn(getIndexResponse); - when(getIndexResponse.getIndices()).thenReturn(indices); - - // setup the mocks so that a set of mappings are available to the DAO - when(indicesAdminClient.getMappings(any())).thenReturn(getMappingsActionFuture); - when(getMappingsActionFuture.actionGet()).thenReturn(getMappingsResponse); - when(getMappingsResponse.getMappings()).thenReturn(mappings); - - return new ElasticsearchColumnMetadataDao(adminClient); + Map mappings) { + ElasticsearchClient client = new ElasticsearchClient(mock(RestClient.class), mock(RestHighLevelClient.class)) { + @Override + public String[] getIndices() throws IOException { + return indices; + } + + @Override + public Map getMappings(String[] indices) throws IOException { + return mappings; + } + }; + return new ElasticsearchColumnMetadataDao(client); } @Test - public void testGetOneLatestIndex() { + public void testGetOneLatestIndex() throws IOException { // setup String[] existingIndices = new String[] { @@ -105,7 +103,7 @@ public void testGetOneLatestIndex() { } @Test - public void testGetLatestIndices() { + public void testGetLatestIndices() throws IOException { // setup String[] existingIndices = new String[] { "bro_index_2017.10.03.19", @@ -127,7 +125,7 @@ public void testGetLatestIndices() { } @Test - public void testLatestIndicesWhereNoneExist() { + public void testLatestIndicesWhereNoneExist() throws IOException { // setup - there are no existing indices String[] existingIndices = new String[] {}; diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchDaoTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchDaoTest.java index 6c3c3270da..2855bbc8c7 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchDaoTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchDaoTest.java @@ -29,6 +29,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; + +import org.apache.metron.elasticsearch.utils.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.search.FieldType; @@ -37,6 +39,8 @@ import org.apache.metron.indexing.dao.search.SearchResponse; import org.apache.metron.indexing.dao.search.SortField; import org.apache.metron.indexing.dao.search.SortOrder; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchHit; @@ -90,7 +94,8 @@ private void setup(RestStatus status, int maxSearchResults, Map globalConfig = new HashMap() { { put("es.clustername", "metron"); - put("es.port", "9300"); + put("es.port", "9200"); put("es.ip", "localhost"); put("es.date.format", DATE_FORMAT); } @@ -334,11 +334,8 @@ protected void addRecords(List> inputData, String index, Str } @Override - protected void setupTypings() { - ((ElasticsearchDao) esDao).getClient().admin().indices().preparePutMapping(INDEX) - .setType("test_doc") - .setSource(nestedAlertMapping) - .get(); + protected void setupTypings() throws IOException { + ((ElasticsearchDao) esDao).getClient().putMapping(INDEX, "test_doc", nestedAlertMapping); } @Override diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java index 8071e68fc9..61dd0f6a56 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java @@ -39,6 +39,7 @@ import org.apache.metron.indexing.dao.search.SearchResponse; import org.apache.metron.indexing.dao.search.SearchResult; import org.apache.metron.integration.InMemoryComponent; +import org.apache.metron.integration.utils.TestUtils; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequestBuilder; @@ -200,7 +201,7 @@ protected static IndexDao createDao() { config.setGlobalConfigSupplier( () -> new HashMap() {{ put("es.clustername", "metron"); - put("es.port", "9300"); + put("es.port", "9200"); put("es.ip", "localhost"); put("es.date.format", dateFormat); }} @@ -272,8 +273,10 @@ public void bad_facet_query_throws_exception() throws Exception { public void returns_column_metadata_for_specified_indices() throws Exception { // getColumnMetadata with only bro { + //TODO: It shouldn't require an assertEventually() here as it should be synchronous. + // Before merging, please figure out why. + TestUtils.assertEventually(() -> Assert.assertEquals(13, dao.getColumnMetadata(Collections.singletonList("bro")).size())); Map fieldTypes = dao.getColumnMetadata(Collections.singletonList("bro")); - Assert.assertEquals(13, fieldTypes.size()); Assert.assertEquals(FieldType.TEXT, fieldTypes.get("bro_field")); Assert.assertEquals(FieldType.TEXT, fieldTypes.get("ttl")); Assert.assertEquals(FieldType.KEYWORD, fieldTypes.get("guid")); @@ -292,8 +295,10 @@ public void returns_column_metadata_for_specified_indices() throws Exception { } // getColumnMetadata with only snort { + //TODO: It shouldn't require an assertEventually() here as it should be synchronous. + // Before merging, please figure out why. + TestUtils.assertEventually(() -> Assert.assertEquals(14, dao.getColumnMetadata(Collections.singletonList("snort")).size())); Map fieldTypes = dao.getColumnMetadata(Collections.singletonList("snort")); - Assert.assertEquals(14, fieldTypes.size()); Assert.assertEquals(FieldType.INTEGER, fieldTypes.get("snort_field")); Assert.assertEquals(FieldType.INTEGER, fieldTypes.get("ttl")); Assert.assertEquals(FieldType.KEYWORD, fieldTypes.get("guid")); @@ -313,8 +318,10 @@ public void returns_column_metadata_for_specified_indices() throws Exception { @Override public void returns_column_data_for_multiple_indices() throws Exception { + //TODO: It shouldn't require an assertEventually() here as it should be synchronous. + // Before merging, please figure out why. + TestUtils.assertEventually(() -> Assert.assertEquals(15, dao.getColumnMetadata(Arrays.asList("bro", "snort")).size())); Map fieldTypes = dao.getColumnMetadata(Arrays.asList("bro", "snort")); - Assert.assertEquals(15, fieldTypes.size()); Assert.assertEquals(FieldType.KEYWORD, fieldTypes.get("guid")); Assert.assertEquals(FieldType.TEXT, fieldTypes.get("source:type")); Assert.assertEquals(FieldType.IP, fieldTypes.get("ip_src_addr")); diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java index c5c0bc1509..6f367908cb 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java @@ -104,7 +104,7 @@ public static void teardown() { protected static Map createGlobalConfig() { return new HashMap() {{ put("es.clustername", "metron"); - put("es.port", "9300"); + put("es.port", "9200"); put("es.ip", "localhost"); put("es.date.format", dateFormat); }}; diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java index 45b4d60397..3e14c000d8 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java @@ -135,7 +135,7 @@ public void start() throws UnableToStartException { .put("path.data",dataDir.getAbsolutePath()) .put("path.home", indexDir.getAbsoluteFile()) .put("transport.type", "netty4") - .put("http.enabled", "false"); + .put("http.enabled", "true"); if (extraElasticSearchSettings != null) { settingsBuilder = settingsBuilder.put(extraElasticSearchSettings); @@ -277,7 +277,9 @@ public boolean hasIndex(String indexName) { @Override public void stop() { try { - node.close(); + if(node != null) { + node.close(); + } } catch (IOException e) { throw new RuntimeException("Unable to stop node." , e); } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java index e8b9f268c1..cbbe9eefaf 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java @@ -22,6 +22,8 @@ import org.apache.metron.indexing.dao.search.SearchDao; import org.apache.metron.indexing.dao.search.SearchResponse; +import java.io.IOException; + public interface MetaAlertSearchDao extends SearchDao { /** @@ -30,6 +32,6 @@ public interface MetaAlertSearchDao extends SearchDao { * @return All meta alerts with a child alert having the GUID * @throws InvalidSearchException If a problem occurs with the search */ - SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException; + SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException, IOException; } diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java index 7e288531a2..24989b4a68 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java @@ -1095,7 +1095,7 @@ protected abstract void addRecords(List> inputData, String i protected abstract long getMatchingMetaAlertCount(String fieldName, String fieldValue) throws IOException, InterruptedException; - protected abstract void setupTypings(); + protected abstract void setupTypings() throws IOException; // Get the base index name without any adjustments (e.g. without ES's "_index") protected abstract String getTestIndexName(); diff --git a/metron-platform/metron-integration-test/src/main/config/zookeeper/global.json b/metron-platform/metron-integration-test/src/main/config/zookeeper/global.json index 9292f729fa..f7d45a7787 100644 --- a/metron-platform/metron-integration-test/src/main/config/zookeeper/global.json +++ b/metron-platform/metron-integration-test/src/main/config/zookeeper/global.json @@ -1,7 +1,7 @@ { "es.clustername": "metron", "es.ip": "localhost", - "es.port": 9300, + "es.port": 9200, "es.date.format": "yyyy.MM.dd.HH", "solr.zookeeper": "localhost:2181", diff --git a/pom.xml b/pom.xml index a98de7221d..8fe178699d 100644 --- a/pom.xml +++ b/pom.xml @@ -126,7 +126,7 @@ 6.6.2 1.10.19 1.7.0 - 2.4.3 + 3.1.1 2.7.4 2.0.14 3.0.2 From 10410ea9718a2a1b1d287fb4f22a6c98efb1fdaa Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Mon, 8 Oct 2018 18:07:22 -0600 Subject: [PATCH 02/41] Update shade plugin version --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8fe178699d..1e6adb0b88 100644 --- a/pom.xml +++ b/pom.xml @@ -126,7 +126,7 @@ 6.6.2 1.10.19 1.7.0 - 3.1.1 + 3.2.0 2.7.4 2.0.14 3.0.2 From a33a16872118175ed35729df6ddde2959e49ae2f Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Tue, 9 Oct 2018 09:56:08 -0600 Subject: [PATCH 03/41] Fix es update dao test --- .../dao/ElasticsearchUpdateDaoTest.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDaoTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDaoTest.java index 3b48a60408..5fc331b352 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDaoTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDaoTest.java @@ -18,30 +18,32 @@ package org.apache.metron.elasticsearch.dao; +import static org.mockito.Mockito.mock; + +import org.apache.metron.elasticsearch.utils.ElasticsearchClient; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.UpdateDaoTest; import org.apache.metron.indexing.dao.update.UpdateDao; -import org.elasticsearch.client.transport.TransportClient; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; import org.junit.Before; -import static org.mockito.Mockito.mock; - /** * This class returns the ElasticsearchUpdateDao implementation to be used in UpdateDaoTest. UpdateDaoTest contains a * common set of tests that all Dao implementations must pass. */ public class ElasticsearchUpdateDaoTest extends UpdateDaoTest { - private TransportClient client; private AccessConfig accessConfig; private ElasticsearchRetrieveLatestDao retrieveLatestDao; private ElasticsearchUpdateDao updateDao; @Before public void setup() { - client = mock(TransportClient.class); accessConfig = new AccessConfig(); retrieveLatestDao = mock(ElasticsearchRetrieveLatestDao.class); + RestHighLevelClient highLevel = mock(RestHighLevelClient.class); + ElasticsearchClient client = new ElasticsearchClient(mock(RestClient.class), highLevel); updateDao = new ElasticsearchUpdateDao(client, accessConfig, retrieveLatestDao); } From 43809968320e586fd70411776140f3aa13a60195 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Thu, 11 Oct 2018 17:59:25 -0600 Subject: [PATCH 04/41] Get shade plugin working with the new ES client and the ClassIndexTransformer Shade plugin transformer. --- metron-platform/elasticsearch-shaded/pom.xml | 51 -------------------- metron-platform/metron-elasticsearch/pom.xml | 7 +-- pom.xml | 2 +- 3 files changed, 5 insertions(+), 55 deletions(-) diff --git a/metron-platform/elasticsearch-shaded/pom.xml b/metron-platform/elasticsearch-shaded/pom.xml index d9002e4bfc..7766e3d47b 100644 --- a/metron-platform/elasticsearch-shaded/pom.xml +++ b/metron-platform/elasticsearch-shaded/pom.xml @@ -34,57 +34,10 @@ netty-common 4.1.13.Final - org.elasticsearch.client elasticsearch-rest-high-level-client ${global_elasticsearch_version} - - - - org.elasticsearch.plugin - aggs-matrix-stats-client - - org.apache.logging.log4j @@ -117,10 +70,6 @@ com.fasterxml.jackson.core jackson-core - - org.apache.logging.log4j - log4j-api - diff --git a/metron-platform/metron-elasticsearch/pom.xml b/metron-platform/metron-elasticsearch/pom.xml index 593e80b0f8..44362ad5fb 100644 --- a/metron-platform/metron-elasticsearch/pom.xml +++ b/metron-platform/metron-elasticsearch/pom.xml @@ -306,7 +306,6 @@ - @@ -320,13 +319,15 @@ + implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/> + + diff --git a/pom.xml b/pom.xml index 1e6adb0b88..90e52bd429 100644 --- a/pom.xml +++ b/pom.xml @@ -104,7 +104,7 @@ 4.5 3.7 2.7.1 - 3.3 + 3.4 1.0.3 1.2.2 ${base_flux_version} From af03f6f036e96c742db39733bf8ebc2cbf229129 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Thu, 18 Oct 2018 20:56:52 -0600 Subject: [PATCH 05/41] Introduce config classes for managing ES client configuration. Translate properties for new client. --- .../common/configuration/ConfigOption.java | 7 + .../ElasticsearchClient.java | 33 +-- .../config/ElasticsearchClientConfig.java | 176 ++++++++++++ .../config/ElasticsearchClientOptions.java | 60 ++++ .../dao/ElasticsearchColumnMetadataDao.java | 11 +- .../elasticsearch/dao/ElasticsearchDao.java | 4 +- .../dao/ElasticsearchRequestSubmitter.java | 6 +- .../dao/ElasticsearchRetrieveLatestDao.java | 5 +- .../dao/ElasticsearchSearchDao.java | 16 +- .../dao/ElasticsearchUpdateDao.java | 5 +- .../utils/ElasticsearchUtils.java | 259 +++++++++--------- .../writer/ElasticsearchWriter.java | 6 +- .../ElasticsearchColumnMetadataDaoTest.java | 11 +- .../dao/ElasticsearchDaoTest.java | 3 +- .../ElasticsearchRequestSubmitterTest.java | 5 +- .../dao/ElasticsearchUpdateDaoTest.java | 2 +- 16 files changed, 393 insertions(+), 216 deletions(-) rename metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/{utils => client}/ElasticsearchClient.java (90%) create mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientConfig.java create mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientOptions.java diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigOption.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigOption.java index 6308f0a185..14d5b69c6d 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigOption.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/ConfigOption.java @@ -30,6 +30,13 @@ default BiFunction transform() { return (s, o) -> o; } + /** + * Returns true if the map contains the key for the defined config option + */ + default boolean containsOption(Map map) { + return map.containsKey(getKey()); + } + default void put(Map map, Object value) { map.put(getKey(), value); } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchClient.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClient.java similarity index 90% rename from metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchClient.java rename to metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClient.java index 669ac10840..6778b0e77d 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchClient.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClient.java @@ -15,34 +15,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.metron.elasticsearch.utils; +package org.apache.metron.elasticsearch.client; import com.google.common.base.Joiner; import com.google.common.base.Splitter; import com.google.common.collect.Iterables; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.http.HttpEntity; -import org.apache.http.entity.BasicHttpEntity; import org.apache.http.entity.StringEntity; import org.apache.metron.common.utils.JSONUtils; -import org.elasticsearch.action.admin.indices.get.GetIndexRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.client.Request; +import org.apache.metron.elasticsearch.utils.FieldMapping; +import org.apache.metron.elasticsearch.utils.FieldProperties; import org.elasticsearch.client.Response; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.cluster.metadata.MappingMetaData; -import org.elasticsearch.common.collect.ImmutableOpenMap; - -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +/** + * Wrapper around the Elasticsearch REST clients. Exposes capabilities of the low and high-level clients. + */ public class ElasticsearchClient implements AutoCloseable{ private RestClient lowLevelClient; private RestHighLevelClient highLevelClient; @@ -79,12 +76,6 @@ public void putMapping(String index, String type, String source) throws IOExcept String responseStr = IOUtils.toString(response.getEntity().getContent()); throw new IllegalStateException("Got a " + response.getStatusLine().getStatusCode() + " due to " + responseStr); } - /** - * ((ElasticsearchDao) esDao).getClient().admin().indices().preparePutMapping(INDEX) - .setType("test_doc") - .setSource(nestedAlertMapping) - .get(); - */ } public String[] getIndices() throws IOException { diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientConfig.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientConfig.java new file mode 100644 index 0000000000..2067f09e63 --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientConfig.java @@ -0,0 +1,176 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.metron.elasticsearch.config; + +import static java.lang.String.format; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.AbstractMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.commons.collections4.map.AbstractMapDecorator; +import org.apache.commons.lang.StringUtils; +import org.apache.metron.common.utils.HDFSUtils; + +public class ElasticsearchClientConfig extends AbstractMapDecorator { + + private static Integer THIRTY_SECONDS_IN_MILLIS = 30_000; + private static Integer ONE_SECONDS_IN_MILLIS = 1_000; + private static String DEFAULT_KEYSTORE_TYPE = "JKS"; + + /** + * Initialize config from provided settings Map. + * + * @param settings Map of config options from which to initialize. + */ + public ElasticsearchClientConfig(Map settings) { + super(settings); + } + + /** + * @return Connection timeout as specified by user, or default 1s as defined by the ES client. + */ + public Integer getConnectTimeoutMillis() { + return ElasticsearchClientOptions.CONNECTION_TIMEOUT_MILLIS + .getOrDefault(this, Integer.class, ONE_SECONDS_IN_MILLIS); + } + + /** + * @return socket timeout specified by user, or default 30s as defined by the ES client. + */ + public Integer getSocketTimeoutMillis() { + return ElasticsearchClientOptions.SOCKET_TIMEOUT_MILLIS + .getOrDefault(this, Integer.class, THIRTY_SECONDS_IN_MILLIS); + } + + /** + * @return max retry timeout specified by user, or default 30s as defined by the ES client. + */ + public Integer getMaxRetryTimeoutMillis() { + return ElasticsearchClientOptions.MAX_RETRY_TIMEOUT_MILLIS + .getOrDefault(this, Integer.class, THIRTY_SECONDS_IN_MILLIS); + } + + /** + * Elasticsearch X-Pack credentials. + * + * @return Username, password + */ + public Optional> getCredentials() { + if (ElasticsearchClientOptions.XPACK_PASSWORD_FILE.containsOption(this)) { + if (!ElasticsearchClientOptions.XPACK_USERNAME.containsOption(this) || + StringUtils.isEmpty(ElasticsearchClientOptions.XPACK_USERNAME.get(this, String.class))) { + throw new IllegalArgumentException( + "X-pack username is required when password supplied and cannot be empty"); + } + String user = ElasticsearchClientOptions.XPACK_USERNAME.get(this, String.class); + String password = getPasswordFromFile( + ElasticsearchClientOptions.XPACK_PASSWORD_FILE.get(this, String.class)); + if (user != null && password != null) { + return Optional.of(new AbstractMap.SimpleImmutableEntry(user, password)); + } + } + return Optional.empty(); + } + + /** + * Expects single password on first line. + */ + private static String getPasswordFromFile(String hdfsPath) { + List lines = readLines(hdfsPath); + if (lines.size() == 0) { + throw new IllegalArgumentException(format("No password found in file '%s'", hdfsPath)); + } + return lines.get(0); + } + + /** + * Read all lines from HDFS file. + * + * @param hdfsPath path to file + * @return lines + */ + private static List readLines(String hdfsPath) { + try { + return HDFSUtils.readFile(hdfsPath); + } catch (IOException e) { + throw new IllegalStateException( + format("Unable to read XPack password file from HDFS location '%s'", hdfsPath), e); + } + } + + /** + * Determines if SSL is enabled from user-supplied config ssl.enabled. + */ + public boolean isSSLEnabled() { + return ElasticsearchClientOptions.SSL_ENABLED.getOrDefault(this, Boolean.class, false); + } + + /** + * @return Number of threads to use for client connection. + */ + public Optional getNumClientConnectionThreads() { + if (ElasticsearchClientOptions.NUM_CLIENT_CONNECTION_THREADS.containsOption(this)) { + return Optional + .of(ElasticsearchClientOptions.NUM_CLIENT_CONNECTION_THREADS.get(this, Integer.class)); + } + return Optional.empty(); + } + + /** + * @return User-defined keystore type. Defaults to "JKS" if not defined. + */ + public String getKeyStoreType() { + if (ElasticsearchClientOptions.KEYSTORE_TYPE.containsOption(this) + && StringUtils + .isNotEmpty(ElasticsearchClientOptions.KEYSTORE_TYPE.get(this, String.class))) { + return ElasticsearchClientOptions.KEYSTORE_TYPE.get(this, String.class); + } + return DEFAULT_KEYSTORE_TYPE; + } + + /** + * Reads keystore password from the HDFS file defined by setting "keystore.password.file", if it + * exists. + * + * @return password if it exists, empty optional otherwise. + */ + public Optional getKeyStorePassword() { + if (ElasticsearchClientOptions.KEYSTORE_PASSWORD_FILE.containsOption(this)) { + String password = getPasswordFromFile( + ElasticsearchClientOptions.KEYSTORE_PASSWORD_FILE.get(this, String.class)); + if (StringUtils.isNotEmpty(password)) { + return Optional.of(password); + } + } + return Optional.empty(); + } + + /** + * @return keystore path. + */ + public Optional getKeyStorePath() { + if (ElasticsearchClientOptions.KEYSTORE_PATH.containsOption(this)) { + return Optional.of(ElasticsearchClientOptions.KEYSTORE_PATH.get(this, Path.class)); + } + return Optional.empty(); + } +} diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientOptions.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientOptions.java new file mode 100644 index 0000000000..c92a34fb8e --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientOptions.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.metron.elasticsearch.config; + +import org.apache.metron.common.configuration.ConfigOption; + +public enum ElasticsearchClientOptions implements ConfigOption { + CONNECTION_TIMEOUT_MILLIS("connection.timeout.millis"), + SOCKET_TIMEOUT_MILLIS("socket.timeout.millis"), + MAX_RETRY_TIMEOUT_MILLIS("max.retry.timeout.millis"), + NUM_CLIENT_CONNECTION_THREADS("num.client.connection.threads"), + // authentication + XPACK_USERNAME("xpack.username"), + XPACK_PASSWORD_FILE("xpack.password.file"), + // security/encryption + SSL_ENABLED("ssl.enabled"), + KEYSTORE_TYPE("keystore.type"), + KEYSTORE_PATH("keystore.path"), + KEYSTORE_PASSWORD_FILE("keystore.password.file"); + + private final String key; + + ElasticsearchClientOptions(String key) { + this.key = key; + } + + @Override + public String getKey() { + return key; + } + + /** + * Convenience method for printing all options as their key representation. + */ + public static void printOptions() { + String newLine = ""; + for (ElasticsearchClientOptions opt : ElasticsearchClientOptions.values()) { + System.out.print(newLine); + System.out.print(opt.getKey()); + newLine = System.lineSeparator(); + } + } + +} diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java index 64a641f5e2..7c2a9bc451 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java @@ -18,19 +18,11 @@ package org.apache.metron.elasticsearch.dao; -import org.apache.metron.elasticsearch.utils.ElasticsearchClient; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.FieldMapping; import org.apache.metron.elasticsearch.utils.FieldProperties; import org.apache.metron.indexing.dao.ColumnMetadataDao; import org.apache.metron.indexing.dao.search.FieldType; -import org.elasticsearch.action.admin.indices.get.GetIndexRequest; -import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.client.AdminClient; -import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.cluster.metadata.MappingMetaData; -import org.elasticsearch.common.collect.ImmutableOpenMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,7 +31,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java index fa04610b79..bf6cae221a 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java @@ -23,7 +23,7 @@ import java.util.Map; import java.util.Optional; -import org.apache.metron.elasticsearch.utils.ElasticsearchClient; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.IndexDao; @@ -40,8 +40,6 @@ import org.apache.metron.indexing.dao.update.OriginalNotFoundException; import org.apache.metron.indexing.dao.update.PatchRequest; import org.apache.metron.indexing.dao.update.ReplaceRequest; -import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.index.query.QueryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRequestSubmitter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRequestSubmitter.java index 64d9200548..c63532e8e5 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRequestSubmitter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRequestSubmitter.java @@ -20,20 +20,16 @@ import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.exception.ExceptionUtils; -import org.apache.metron.elasticsearch.utils.ElasticsearchClient; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.search.InvalidSearchException; -import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; -import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.rest.RestStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.lang.invoke.MethodHandles; /** diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java index ff1189c793..0c91007943 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java @@ -29,14 +29,11 @@ import java.util.Optional; import java.util.function.Function; -import org.apache.metron.elasticsearch.utils.ElasticsearchClient; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.indexing.dao.RetrieveLatestDao; import org.apache.metron.indexing.dao.search.GetRequest; import org.apache.metron.indexing.dao.update.Document; import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchRequestBuilder; -import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.index.query.IdsQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchDao.java index 32cefe0c82..0b87e5645a 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchDao.java @@ -19,26 +19,19 @@ import static org.apache.metron.elasticsearch.utils.ElasticsearchUtils.INDEX_NAME_DELIMITER; -import com.google.common.base.Splitter; -import com.google.common.collect.Iterables; import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.function.Function; -import org.apache.metron.elasticsearch.utils.ElasticsearchClient; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.search.FieldType; -import org.apache.metron.indexing.dao.search.GetRequest; import org.apache.metron.indexing.dao.search.Group; import org.apache.metron.indexing.dao.search.GroupOrder; import org.apache.metron.indexing.dao.search.GroupOrderType; @@ -52,17 +45,10 @@ import org.apache.metron.indexing.dao.search.SearchResult; import org.apache.metron.indexing.dao.search.SortField; import org.apache.metron.indexing.dao.search.SortOrder; -import org.apache.metron.indexing.dao.update.Document; -import org.elasticsearch.action.search.SearchRequestBuilder; -import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.index.mapper.LegacyIpFieldMapper; -import org.elasticsearch.index.query.IdsQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryStringQueryBuilder; import org.elasticsearch.search.SearchHit; -import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.aggregations.Aggregation; import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.Aggregations; diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java index 75300ea576..c769b2f56e 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java @@ -29,7 +29,7 @@ import java.util.Optional; import java.util.stream.Collectors; -import org.apache.metron.elasticsearch.utils.ElasticsearchClient; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.search.AlertComment; @@ -37,13 +37,10 @@ import org.apache.metron.indexing.dao.update.Document; import org.apache.metron.indexing.dao.update.UpdateDao; import org.elasticsearch.action.bulk.BulkRequest; -import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo; -import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.client.transport.TransportClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java index 838f8c74ba..ab4d566085 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java @@ -23,62 +23,54 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import java.io.IOException; +import java.io.InputStream; import java.lang.invoke.MethodHandles; -import java.net.InetAddress; -import java.net.UnknownHostException; +import java.nio.file.Files; +import java.security.KeyManagementException; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.cert.CertificateException; import java.text.SimpleDateFormat; -import java.util.AbstractMap; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Optional; -import java.util.Set; import java.util.stream.Collectors; - import org.apache.commons.lang.StringUtils; import org.apache.http.HttpHost; import org.apache.http.auth.AuthScope; import org.apache.http.auth.UsernamePasswordCredentials; import org.apache.http.client.CredentialsProvider; -import org.apache.http.client.methods.HttpPost; +import org.apache.http.client.config.RequestConfig.Builder; import org.apache.http.impl.client.BasicCredentialsProvider; import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; +import org.apache.http.impl.nio.reactor.IOReactorConfig; +import org.apache.http.ssl.SSLContextBuilder; +import org.apache.http.ssl.SSLContexts; import org.apache.metron.common.configuration.writer.WriterConfiguration; -import org.apache.metron.common.utils.HDFSUtils; -import org.apache.metron.common.utils.ReflectionUtils; -import org.apache.metron.indexing.dao.search.SearchRequest; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; +import org.apache.metron.elasticsearch.config.ElasticsearchClientConfig; import org.apache.metron.indexing.dao.search.SearchResponse; import org.apache.metron.indexing.dao.search.SearchResult; -import org.apache.metron.netty.utils.NettyRuntimeWrapper; -import org.apache.metron.stellar.common.utils.ConversionUtils; import org.codehaus.jackson.map.ObjectMapper; -import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; -import org.elasticsearch.search.query.QuerySearchRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class ElasticsearchUtils { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private static final String ES_CLIENT_CLASS_DEFAULT = "org.elasticsearch.transport.client.PreBuiltTransportClient"; - private static final String PWD_FILE_CONFIG_KEY = "es.xpack.password.file"; - private static final String USERNAME_CONFIG_KEY = "es.xpack.username"; - private static final String TRANSPORT_CLIENT_USER_KEY = "xpack.security.user"; - - private static ThreadLocal> DATE_FORMAT_CACHE = ThreadLocal.withInitial(() -> new HashMap<>()); @@ -137,143 +129,146 @@ public static String getBaseIndexName(String indexName) { } /** - * Instantiates an Elasticsearch client based on es.client.class, if set. Defaults to - * org.elasticsearch.transport.client.PreBuiltTransportClient. + * Instantiates an Elasticsearch client * * @param globalConfiguration Metron global config - * @return + * @return new es client */ public static ElasticsearchClient getClient(Map globalConfiguration) { - Map esSettings = getEsSettings(globalConfiguration); - Optional> credentials = getCredentials(esSettings); - Set customESSettings = new HashSet<>(); + ElasticsearchClientConfig esClientConfig = new ElasticsearchClientConfig(getEsSettings(globalConfiguration)); + + String scheme = esClientConfig.isSSLEnabled() ? "https" : "http"; + RestClientBuilder builder = getRestClientBuilder(globalConfiguration, scheme); + + RestClientBuilder.RequestConfigCallback reqCallback = reqConfigBuilder -> { + setupConnectionTimeouts(reqConfigBuilder, esClientConfig); + return reqConfigBuilder; + }; + builder.setRequestConfigCallback(reqCallback); + builder.setMaxRetryTimeoutMillis(esClientConfig.getMaxRetryTimeoutMillis()); + + RestClientBuilder.HttpClientConfigCallback httpClientConfigCallback = clientBuilder -> { + setupNumConnectionThreads(clientBuilder, esClientConfig); + setupAuthentication(clientBuilder, esClientConfig); + setupConnectionEncryption(clientBuilder, esClientConfig); + return clientBuilder; + }; + builder.setHttpClientConfigCallback(httpClientConfigCallback); - - RestClientBuilder builder = null; - List hps = getIps(globalConfiguration); - { - HttpHost[] posts = new HttpHost[hps.size()]; - int i = 0; - for (HostnamePort hp : hps) { - posts[i++] = new HttpHost(hp.hostname, hp.port); - } - builder = RestClient.builder(posts); - } - if(credentials.isPresent()) { - final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); - credentialsProvider.setCredentials(AuthScope.ANY, - new UsernamePasswordCredentials(credentials.get().getKey(), credentials.get().getValue())); - builder = builder.setHttpClientConfigCallback( - httpAsyncClientBuilder -> httpAsyncClientBuilder.setDefaultCredentialsProvider(credentialsProvider) - ); - } RestClient lowLevelClient = builder.build(); RestHighLevelClient client = new RestHighLevelClient(lowLevelClient); return new ElasticsearchClient(lowLevelClient, client); - - /*customESSettings.addAll(Arrays.asList("es.client.class", USERNAME_CONFIG_KEY, PWD_FILE_CONFIG_KEY)); - Settings.Builder settingsBuilder = Settings.builder(); - for (Map.Entry entry : esSettings.entrySet()) { - String key = entry.getKey(); - String value = entry.getValue(); - if (!customESSettings.contains(key)) { - settingsBuilder.put(key, value); - } - } - settingsBuilder.put("cluster.name", globalConfiguration.get("es.clustername")); - settingsBuilder.put("client.transport.ping_timeout", esSettings.getOrDefault("client.transport.ping_timeout","500s")); - setXPackSecurityOrNone(settingsBuilder, esSettings); - - try { - LOG.info("Number of available processors in Netty: {}", NettyRuntimeWrapper.availableProcessors()); - // Netty sets available processors statically and if an attempt is made to set it more than - // once an IllegalStateException is thrown by NettyRuntime.setAvailableProcessors(NettyRuntime.java:87) - // https://discuss.elastic.co/t/getting-availableprocessors-is-already-set-to-1-rejecting-1-illegalstateexception-exception/103082 - // https://discuss.elastic.co/t/elasticsearch-5-4-1-availableprocessors-is-already-set/88036 - System.setProperty("es.set.netty.runtime.available.processors", "false"); - TransportClient client = createTransportClient(settingsBuilder.build(), esSettings); - for (HostnamePort hp : getIps(globalConfiguration)) { - client.addTransportAddress( - new InetSocketTransportAddress(InetAddress.getByName(hp.hostname), hp.port) - ); - } - return client; - } catch (UnknownHostException exception) { - throw new RuntimeException(exception); - }*/ } - private static Map getEsSettings(Map config) { - return ConversionUtils - .convertMap((Map) config.getOrDefault("es.client.settings", new HashMap()), - String.class); + private static Map getEsSettings(Map globalConfig) { + return (Map) globalConfig.getOrDefault("es.client.settings", new HashMap()); } - private static Optional> getCredentials(Map esSettings) { - Optional> ret = Optional.empty(); - if (esSettings.containsKey(PWD_FILE_CONFIG_KEY)) { - - if (!esSettings.containsKey(USERNAME_CONFIG_KEY) || StringUtils.isEmpty(esSettings.get(USERNAME_CONFIG_KEY))) { - throw new IllegalArgumentException("X-pack username is required and cannot be empty"); - } - String user = esSettings.get(USERNAME_CONFIG_KEY); - String password = esSettings.containsKey(PWD_FILE_CONFIG_KEY)?esSettings.get(getPasswordFromFile(esSettings.get(PWD_FILE_CONFIG_KEY))):null; - if(user != null && password != null) { - return Optional.of(new AbstractMap.SimpleImmutableEntry(user, password)); - } + private static RestClientBuilder getRestClientBuilder(Map globalConfiguration, + String scheme) { + List hps = getIps(globalConfiguration); + HttpHost[] posts = new HttpHost[hps.size()]; + int i = 0; + for (HostnamePort hp : hps) { + posts[i++] = new HttpHost(hp.hostname, hp.port, scheme); } - return ret; + return RestClient.builder(posts); } - /* - * Append Xpack security settings (if any) + /** + * Modifies request config builder with connection and socket timeouts. + * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_timeouts.html + * + * @param reqConfigBuilder builder to modify + * @param esClientConfig pull timeout settings from this config */ - private static void setXPackSecurityOrNone(Settings.Builder settingsBuilder, Map esSettings) { - - if (esSettings.containsKey(PWD_FILE_CONFIG_KEY)) { - - if (!esSettings.containsKey(USERNAME_CONFIG_KEY) || StringUtils.isEmpty(esSettings.get(USERNAME_CONFIG_KEY))) { - throw new IllegalArgumentException("X-pack username is required and cannot be empty"); - } + private static void setupConnectionTimeouts(Builder reqConfigBuilder, + ElasticsearchClientConfig esClientConfig) { + reqConfigBuilder.setConnectTimeout(esClientConfig.getConnectTimeoutMillis()); + reqConfigBuilder.setSocketTimeout(esClientConfig.getSocketTimeoutMillis()); + } - settingsBuilder.put( - TRANSPORT_CLIENT_USER_KEY, - esSettings.get(USERNAME_CONFIG_KEY) + ":" + getPasswordFromFile(esSettings.get(PWD_FILE_CONFIG_KEY)) - ); + /** + * Modifies client builder with setting for num connection threads. Default is ES client default, + * which is 1 to num processors per the documentation. + * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_number_of_threads.html + * + * @param clientBuilder builder to modify + * @param esClientConfig pull num threads property from config + */ + private static void setupNumConnectionThreads(HttpAsyncClientBuilder clientBuilder, + ElasticsearchClientConfig esClientConfig) { + if (esClientConfig.getNumClientConnectionThreads().isPresent()) { + Integer numThreads = esClientConfig.getNumClientConnectionThreads().get(); + LOG.info("Setting number of client connection threads: {}", numThreads); + clientBuilder.setDefaultIOReactorConfig(IOReactorConfig.custom() + .setIoThreadCount(numThreads).build()); } } - /* - * Single password on first line + /** + * Modifies client builder with settings for authentication with X-Pack. + * Note, we do not expose the ability to disable preemptive authentication. + * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_basic_authentication.html + * + * @param clientBuilder builder to modify + * @param esClientConfig pull credentials property from config */ - private static String getPasswordFromFile(String hdfsPath) { - List lines = null; - try { - lines = HDFSUtils.readFile(hdfsPath); - } catch (IOException e) { - throw new IllegalArgumentException( - format("Unable to read XPack password file from HDFS location '%s'", hdfsPath), e); - } - if (lines.size() == 0) { - throw new IllegalArgumentException(format("No password found in file '%s'", hdfsPath)); + private static void setupAuthentication(HttpAsyncClientBuilder clientBuilder, ElasticsearchClientConfig esClientConfig) { + Optional> credentials = esClientConfig.getCredentials(); + if (credentials.isPresent()) { + LOG.info( + "Found auth credentials - setting up user/pass authenticated client connection for ES."); + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + UsernamePasswordCredentials upcredentials = new UsernamePasswordCredentials( + credentials.get().getKey(), credentials.get().getValue()); + credentialsProvider.setCredentials(AuthScope.ANY, upcredentials); + clientBuilder.setDefaultCredentialsProvider(credentialsProvider); + } else { + LOG.info( + "Elasticsearch client credentials not provided. Defaulting to non-authenticated client connection."); } - return lines.get(0); } /** - * Constructs ES transport client from the provided ES settings additional es config + * Modify client builder with connection encryption details (SSL) if applicable. + * If ssl.enabled=true, sets up SSL connection. If enabled, keystore.path is required. User can + * also optionally set keystore.password and keystore.type. + * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_encrypted_communication.html * - * @param settings client settings - * @param esSettings client type to instantiate - * @return client with provided settings + * @param clientBuilder builder to modify + * @param esClientConfig pull connection encryption details from config */ - private static TransportClient createTransportClient(Settings settings, - Map esSettings) { - String esClientClassName = (String) esSettings - .getOrDefault("es.client.class", ES_CLIENT_CLASS_DEFAULT); - return ReflectionUtils - .createInstance(esClientClassName, new Class[]{Settings.class, Class[].class}, - new Object[]{settings, new Class[0]}); + private static void setupConnectionEncryption(HttpAsyncClientBuilder clientBuilder, + ElasticsearchClientConfig esClientConfig) { + if (esClientConfig.isSSLEnabled()) { + LOG.info("Configuring client for SSL connection."); + if (!esClientConfig.getKeyStorePath().isPresent()) { + throw new IllegalStateException("KeyStore path must be provided for SSL connection."); + } + KeyStore truststore; + try { + truststore = KeyStore.getInstance(esClientConfig.getKeyStoreType()); + } catch (KeyStoreException e) { + throw new IllegalStateException( + "Unable to get keystore type '" + esClientConfig.getKeyStoreType() + "'", e); + } + Optional optKeyStorePass = esClientConfig.getKeyStorePassword(); + char[] keyStorePass = optKeyStorePass.map(String::toCharArray).orElse(null); + try (InputStream is = Files.newInputStream(esClientConfig.getKeyStorePath().get())) { + truststore.load(is, keyStorePass); + } catch (IOException | NoSuchAlgorithmException | CertificateException e) { + throw new IllegalStateException( + "Unable to load keystore from path '" + esClientConfig.getKeyStorePath().get() + "'", + e); + } + try { + SSLContextBuilder sslBuilder = SSLContexts.custom().loadTrustMaterial(truststore, null); + clientBuilder.setSSLContext(sslBuilder.build()); + } catch (NoSuchAlgorithmException | KeyStoreException | KeyManagementException e) { + throw new IllegalStateException("Unable to load truststore.", e); + } + } } public static class HostnamePort { diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java index 20f387f84a..fa090e5069 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java @@ -23,18 +23,14 @@ import org.apache.metron.common.field.FieldNameConverters; import org.apache.metron.common.writer.BulkMessageWriter; import org.apache.metron.common.writer.BulkWriterResponse; -import org.apache.metron.elasticsearch.utils.ElasticsearchClient; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.storm.task.TopologyContext; import org.apache.storm.tuple.Tuple; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; -import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.client.transport.TransportClient; import org.json.simple.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDaoTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDaoTest.java index e2a675fc89..2652f226ce 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDaoTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDaoTest.java @@ -18,18 +18,10 @@ package org.apache.metron.elasticsearch.dao; -import org.apache.metron.elasticsearch.utils.ElasticsearchClient; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.FieldMapping; -import org.elasticsearch.action.ActionFuture; -import org.elasticsearch.action.admin.indices.get.GetIndexRequestBuilder; -import org.elasticsearch.action.admin.indices.get.GetIndexResponse; -import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse; -import org.elasticsearch.client.AdminClient; -import org.elasticsearch.client.IndicesAdminClient; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.cluster.metadata.MappingMetaData; -import org.elasticsearch.common.collect.ImmutableOpenMap; import org.junit.Test; import java.io.IOException; @@ -42,7 +34,6 @@ import static org.junit.Assert.assertArrayEquals; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; /** * Tests the ElasticsearchColumnMetadata class. diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchDaoTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchDaoTest.java index 2855bbc8c7..6dc01a47c6 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchDaoTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchDaoTest.java @@ -30,7 +30,7 @@ import java.util.List; import java.util.Map; -import org.apache.metron.elasticsearch.utils.ElasticsearchClient; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.search.FieldType; @@ -41,7 +41,6 @@ import org.apache.metron.indexing.dao.search.SortOrder; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchRequestSubmitterTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchRequestSubmitterTest.java index 8cf39dd759..7a84588c0a 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchRequestSubmitterTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchRequestSubmitterTest.java @@ -18,20 +18,17 @@ package org.apache.metron.elasticsearch.dao; -import org.apache.metron.elasticsearch.utils.ElasticsearchClient; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.indexing.dao.search.InvalidSearchException; -import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.index.Index; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchShardTarget; import org.junit.Test; -import org.mockito.Mockito; import java.io.IOException; diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDaoTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDaoTest.java index 5fc331b352..3b7f132004 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDaoTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDaoTest.java @@ -20,7 +20,7 @@ import static org.mockito.Mockito.mock; -import org.apache.metron.elasticsearch.utils.ElasticsearchClient; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.UpdateDaoTest; import org.apache.metron.indexing.dao.update.UpdateDao; From 1a47ded7a36f9d391227973c7da2921305373283 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Tue, 23 Oct 2018 16:23:04 -0600 Subject: [PATCH 06/41] Remove extra deps in metron-elasticsearch around log4j. --- metron-platform/metron-elasticsearch/pom.xml | 28 -------------------- 1 file changed, 28 deletions(-) diff --git a/metron-platform/metron-elasticsearch/pom.xml b/metron-platform/metron-elasticsearch/pom.xml index 44362ad5fb..e3cf840ee8 100644 --- a/metron-platform/metron-elasticsearch/pom.xml +++ b/metron-platform/metron-elasticsearch/pom.xml @@ -217,34 +217,6 @@ test-jar test - - org.apache.logging.log4j - log4j-api - 2.8.2 - test - - - org.apache.logging.log4j - log4j-core - 2.8.2 - test - - - - com.google.guava - guava-testlib - ${global_guava_version} - test - From 54870d68e6f43e859367879bff7537f97c11d0bd Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Tue, 23 Oct 2018 18:33:25 -0600 Subject: [PATCH 07/41] Fixes for dep version issues. --- dependencies_with_url.csv | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dependencies_with_url.csv b/dependencies_with_url.csv index a29e634594..349c002d71 100644 --- a/dependencies_with_url.csv +++ b/dependencies_with_url.csv @@ -484,6 +484,6 @@ org.sonatype.sisu:sisu-inject-bean:jar:2.2.2:compile org.sonatype.sisu:sisu-inject-plexus:jar:2.2.2:compile com.zaxxer:HikariCP:jar:2.7.8:compile,ASLv2,https://github.com/brettwooldridge/HikariCP org.hibernate.validator:hibernate-validator:jar:6.0.9.Final:compile,ASLv2,https://github.com/hibernate/hibernate-validator -com.github.palindromicity:simple-syslog-5424:jar:0.0.8:compile,ASLv2,https://github.com/palindromicity/simple-syslog-5424 com.github.palindromicity:simple-syslog-5424:jar:0.0.9:compile,ASLv2,https://github.com/palindromicity/simple-syslog-5424 org.elasticsearch.client:elasticsearch-rest-high-level-client:jar:5.6.2:compile,ASLv2,https://github.com/elastic/elasticsearch/blob/master/LICENSE.txt +org.elasticsearch.plugin:aggs-matrix-stats-client:jar:5.6.2:compile,ASLv2,https://github.com/elastic/elasticsearch/blob/master/LICENSE.txt diff --git a/pom.xml b/pom.xml index 79f2392d75..f412036b74 100644 --- a/pom.xml +++ b/pom.xml @@ -104,7 +104,7 @@ 4.5 3.7 2.7.1 - 3.4 + 3.3 1.0.3 1.2.2 ${base_flux_version} From 554de87ae160aae1ae85afb3dbb01a220a9d6838 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 25 Oct 2018 14:54:41 -0400 Subject: [PATCH 08/41] METRON-1845 Correct Test Data Load in Elasticsearch Integration Tests --- ...ElasticsearchMetaAlertIntegrationTest.java | 48 +++--- .../ElasticsearchSearchIntegrationTest.java | 160 ++++++++++++------ .../ElasticsearchUpdateIntegrationTest.java | 2 +- .../components/ElasticSearchComponent.java | 74 +++++--- .../indexing/dao/UpdateIntegrationTest.java | 8 +- .../metaalert/MetaAlertIntegrationTest.java | 3 +- 6 files changed, 187 insertions(+), 108 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMetaAlertIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMetaAlertIntegrationTest.java index 03b163930f..fe76f88aef 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMetaAlertIntegrationTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMetaAlertIntegrationTest.java @@ -19,28 +19,7 @@ package org.apache.metron.elasticsearch.integration; -import static org.apache.metron.elasticsearch.dao.ElasticsearchMetaAlertDao.METAALERTS_INDEX; -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.ALERT_FIELD; -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_DOC; -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_FIELD; -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_TYPE; - import com.fasterxml.jackson.core.JsonProcessingException; -import java.io.File; -import java.io.IOException; -import java.text.SimpleDateFormat; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.function.Function; -import java.util.stream.Collectors; - import com.google.common.collect.ImmutableList; import org.adrianwalker.multilinestring.Multiline; import org.apache.metron.common.Constants; @@ -50,13 +29,13 @@ import org.apache.metron.elasticsearch.integration.components.ElasticSearchComponent; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.IndexDao; -import org.apache.metron.indexing.dao.metaalert.MetaAlertDao; import org.apache.metron.indexing.dao.metaalert.MetaAlertIntegrationTest; import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus; import org.apache.metron.indexing.dao.search.GetRequest; import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; import org.apache.metron.indexing.dao.search.SortField; +import org.json.simple.parser.ParseException; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -66,6 +45,27 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import java.io.File; +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.metron.elasticsearch.dao.ElasticsearchMetaAlertDao.METAALERTS_INDEX; +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.ALERT_FIELD; +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_DOC; +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_FIELD; +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_TYPE; + @RunWith(Parameterized.class) public class ElasticsearchMetaAlertIntegrationTest extends MetaAlertIntegrationTest { @@ -321,8 +321,8 @@ protected long getMatchingMetaAlertCount(String fieldName, String fieldValue) @Override protected void addRecords(List> inputData, String index, String docType) - throws IOException { - es.add(index, docType, inputData.stream().map(m -> { + throws IOException, ParseException { + es.add(esDao, index, docType, inputData.stream().map(m -> { try { return JSONUtils.INSTANCE.toJSON(m, true); } catch (JsonProcessingException e) { diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java index f5416fc806..663b5328fc 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java @@ -18,17 +18,18 @@ package org.apache.metron.elasticsearch.integration; -import java.io.File; -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import com.google.common.util.concurrent.Uninterruptibles; import org.apache.metron.common.Constants; import org.apache.metron.common.utils.JSONUtils; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; +import org.apache.metron.elasticsearch.dao.ElasticsearchColumnMetadataDao; import org.apache.metron.elasticsearch.dao.ElasticsearchDao; +import org.apache.metron.elasticsearch.dao.ElasticsearchRequestSubmitter; +import org.apache.metron.elasticsearch.dao.ElasticsearchRetrieveLatestDao; +import org.apache.metron.elasticsearch.dao.ElasticsearchSearchDao; +import org.apache.metron.elasticsearch.dao.ElasticsearchUpdateDao; import org.apache.metron.elasticsearch.integration.components.ElasticSearchComponent; +import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.IndexDao; import org.apache.metron.indexing.dao.SearchIntegrationTest; @@ -38,12 +39,10 @@ import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; import org.apache.metron.indexing.dao.search.SearchResult; +import org.apache.metron.indexing.dao.update.Document; import org.apache.metron.integration.InMemoryComponent; import org.apache.metron.integration.utils.TestUtils; -import org.elasticsearch.action.bulk.BulkRequestBuilder; -import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.support.WriteRequest; +import org.apache.metron.stellar.common.utils.ConversionUtils; import org.json.simple.JSONArray; import org.json.simple.JSONObject; import org.json.simple.parser.JSONParser; @@ -51,9 +50,26 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.function.BooleanSupplier; + +import static org.apache.metron.integration.utils.TestUtils.assertEventually; public class ElasticsearchSearchIntegrationTest extends SearchIntegrationTest { + private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private static String indexDir = "target/elasticsearch_search"; private static String dateFormat = "yyyy.MM.dd.HH"; private static String broTemplatePath = "../../metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/files/bro_index.template"; @@ -70,21 +86,26 @@ public static void setup() throws Exception { loadTestData(); } - protected static IndexDao createDao() { + protected static Map createGlobalConfig() { + return new HashMap() {{ + put("es.clustername", "metron"); + put("es.port", "9200"); + put("es.ip", "localhost"); + put("es.date.format", dateFormat); + }}; + } + + protected static AccessConfig createAccessConfig() { AccessConfig config = new AccessConfig(); config.setMaxSearchResults(100); config.setMaxSearchGroups(100); - config.setGlobalConfigSupplier( () -> - new HashMap() {{ - put("es.clustername", "metron"); - put("es.port", "9200"); - put("es.ip", "localhost"); - put("es.date.format", dateFormat); - }} - ); + config.setGlobalConfigSupplier(() -> createGlobalConfig()); + return config; + } + protected static IndexDao createDao() { IndexDao dao = new ElasticsearchDao(); - dao.init(config); + dao.init(createAccessConfig()); return dao; } @@ -97,48 +118,81 @@ protected static InMemoryComponent startIndex() throws Exception { return es; } - protected static void loadTestData() throws ParseException, IOException { + protected static void loadTestData() throws Exception { ElasticSearchComponent es = (ElasticSearchComponent) indexComponent; + // define the bro index template + String broIndex = "bro_index_2017.01.01.01"; JSONObject broTemplate = JSONUtils.INSTANCE.load(new File(broTemplatePath), JSONObject.class); addTestFieldMappings(broTemplate, "bro_doc"); - es.getClient().admin().indices().prepareCreate("bro_index_2017.01.01.01") - .addMapping("bro_doc", JSONUtils.INSTANCE.toJSON(broTemplate.get("mappings"), false)).get(); + es.getClient().admin().indices().prepareCreate(broIndex) + .addMapping("bro_doc", JSONUtils.INSTANCE.toJSON(broTemplate.get("mappings"), false)).get(); + + // define the snort index template + String snortIndex = "snort_index_2017.01.01.02"; JSONObject snortTemplate = JSONUtils.INSTANCE.load(new File(snortTemplatePath), JSONObject.class); addTestFieldMappings(snortTemplate, "snort_doc"); - es.getClient().admin().indices().prepareCreate("snort_index_2017.01.01.02") - .addMapping("snort_doc", JSONUtils.INSTANCE.toJSON(snortTemplate.get("mappings"), false)).get(); - - BulkRequestBuilder bulkRequest = es.getClient().prepareBulk() - .setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL); - JSONArray broArray = (JSONArray) new JSONParser().parse(broData); - for (Object o : broArray) { - JSONObject jsonObject = (JSONObject) o; - IndexRequestBuilder indexRequestBuilder = es.getClient() - .prepareIndex("bro_index_2017.01.01.01", "bro_doc"); - indexRequestBuilder = indexRequestBuilder.setId((String) jsonObject.get("guid")); - indexRequestBuilder = indexRequestBuilder.setSource(jsonObject.toJSONString()); - indexRequestBuilder = indexRequestBuilder - .setTimestamp(jsonObject.get("timestamp").toString()); - bulkRequest.add(indexRequestBuilder); + es.getClient().admin().indices().prepareCreate(snortIndex) + .addMapping("snort_doc", JSONUtils.INSTANCE.toJSON(snortTemplate.get("mappings"), false)).get(); + + // setup the classes required to write the test data + AccessConfig accessConfig = createAccessConfig(); + ElasticsearchClient client = ElasticsearchUtils.getClient(createGlobalConfig()); + ElasticsearchRetrieveLatestDao retrieveLatestDao = new ElasticsearchRetrieveLatestDao(client); + ElasticsearchColumnMetadataDao columnMetadataDao = new ElasticsearchColumnMetadataDao(client); + ElasticsearchRequestSubmitter requestSubmitter = new ElasticsearchRequestSubmitter(client); + ElasticsearchUpdateDao updateDao = new ElasticsearchUpdateDao(client, accessConfig, retrieveLatestDao); + ElasticsearchSearchDao searchDao = new ElasticsearchSearchDao(client, accessConfig, columnMetadataDao, requestSubmitter); + + // write the test documents for Bro + List broDocuments = new ArrayList<>(); + for (Object broObject: (JSONArray) new JSONParser().parse(broData)) { + broDocuments.add(((JSONObject) broObject).toJSONString()); } - JSONArray snortArray = (JSONArray) new JSONParser().parse(snortData); - for (Object o : snortArray) { - JSONObject jsonObject = (JSONObject) o; - IndexRequestBuilder indexRequestBuilder = es.getClient() - .prepareIndex("snort_index_2017.01.01.02", "snort_doc"); - indexRequestBuilder = indexRequestBuilder.setId((String) jsonObject.get("guid")); - indexRequestBuilder = indexRequestBuilder.setSource(jsonObject.toJSONString()); - indexRequestBuilder = indexRequestBuilder - .setTimestamp(jsonObject.get("timestamp").toString()); - bulkRequest.add(indexRequestBuilder); + es.add(updateDao, broIndex, "bro", broDocuments); + + // write the test documents for Snort + List snortDocuments = new ArrayList<>(); + for (Object snortObject: (JSONArray) new JSONParser().parse(snortData)) { + snortDocuments.add(((JSONObject) snortObject).toJSONString()); } - BulkResponse bulkResponse = bulkRequest.execute().actionGet(); - if (bulkResponse.hasFailures()) { - throw new RuntimeException("Failed to index test data"); + es.add(updateDao, snortIndex, "snort", snortDocuments); + + // wait until the test documents are visible + assertEventually(() -> Assert.assertEquals(10, findAll(searchDao).getTotal())); + } + + /** + * Finds all documents that are indexed. + * + * @param searchDao The {@link ElasticsearchSearchDao} that is used to search for documents. + * @return The search response. + */ + private static SearchResponse findAll(ElasticsearchSearchDao searchDao) { + try { + SearchRequest request = JSONUtils.INSTANCE.load(allQuery, SearchRequest.class); + return searchDao.search(request); + + } catch(IOException | InvalidSearchException e) { + throw new RuntimeException(e); } } + /** + * Create an indexable Document from a JSON message. + * + * @param message The JSON message that needs indexed. + * @param docType The document type to write. + * @return The {@link Document} that was written. + * @throws IOException + */ + private static Document createDocument(JSONObject message, String docType) throws IOException { + Long timestamp = ConversionUtils.convert(message.get("timestamp"), Long.class); + String source = message.toJSONString(); + String guid = (String) message.get("guid"); + return new Document(source, guid, docType, timestamp); + } + /** * Add test fields to a template with defined types in case they are not defined in the sensor template shipped with Metron. * This is useful for testing certain cases, for example faceting on fields of various types. @@ -168,8 +222,6 @@ public void bad_facet_query_throws_exception() throws Exception { dao.search(request); } - - @Override public void returns_column_metadata_for_specified_indices() throws Exception { // getColumnMetadata with only bro diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java index 6f367908cb..61ab652da7 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java @@ -117,7 +117,7 @@ protected static IndexDao createDao() { @Override protected void addTestData(String indexName, String sensorType, List> docs) throws Exception { - es.add(index, SENSOR_NAME + es.add(getDao(), index, SENSOR_NAME , Iterables.transform(docs, m -> { try { diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java index 3e14c000d8..045ed12e5a 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java @@ -24,15 +24,32 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.function.BooleanSupplier; + +import com.google.common.util.concurrent.Uninterruptibles; import org.apache.commons.io.FileUtils; import org.apache.metron.common.Constants; import org.apache.metron.common.utils.JSONUtils; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; +import org.apache.metron.elasticsearch.dao.ElasticsearchColumnMetadataDao; +import org.apache.metron.elasticsearch.dao.ElasticsearchRequestSubmitter; +import org.apache.metron.elasticsearch.dao.ElasticsearchRetrieveLatestDao; +import org.apache.metron.elasticsearch.dao.ElasticsearchSearchDao; +import org.apache.metron.elasticsearch.dao.ElasticsearchUpdateDao; +import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; +import org.apache.metron.indexing.dao.search.InvalidSearchException; +import org.apache.metron.indexing.dao.search.SearchRequest; +import org.apache.metron.indexing.dao.update.Document; +import org.apache.metron.indexing.dao.update.UpdateDao; import org.apache.metron.integration.InMemoryComponent; import org.apache.metron.integration.UnableToStartException; +import org.apache.metron.stellar.common.utils.ConversionUtils; import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction; import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; @@ -54,6 +71,10 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.search.SearchHit; import org.elasticsearch.transport.Netty4Plugin; +import org.json.simple.JSONArray; +import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; +import org.json.simple.parser.ParseException; public class ElasticSearchComponent implements InMemoryComponent { @@ -194,35 +215,41 @@ public Client getClient() { return client; } - public BulkResponse add(String indexName, String sensorType, String... docs) throws IOException { + public void add(UpdateDao updateDao, String indexName, String sensorType, String... docs) + throws IOException, ParseException { List d = new ArrayList<>(); Collections.addAll(d, docs); - return add(indexName, sensorType, d); + add(updateDao, indexName, sensorType, d); } - public BulkResponse add(String indexName, String sensorType, Iterable docs) - throws IOException { - BulkRequestBuilder bulkRequest = getClient().prepareBulk(); - for (String doc : docs) { - IndexRequestBuilder indexRequestBuilder = getClient() - .prepareIndex(indexName, sensorType + "_doc"); - - indexRequestBuilder = indexRequestBuilder.setSource(doc); - Map esDoc = JSONUtils.INSTANCE - .load(doc, JSONUtils.MAP_SUPPLIER); - indexRequestBuilder.setId((String) esDoc.get(Constants.GUID)); - Object ts = esDoc.get("timestamp"); - if (ts != null) { - indexRequestBuilder = indexRequestBuilder.setTimestamp(ts.toString()); - } - bulkRequest.add(indexRequestBuilder); - } + public void add(UpdateDao updateDao, String indexName, String sensorType, Iterable docs) + throws IOException, ParseException { - BulkResponse response = bulkRequest.execute().actionGet(); - if (response.hasFailures()) { - throw new IOException(response.buildFailureMessage()); + // create a collection of indexable documents + JSONParser parser = new JSONParser(); + Map> documents = new HashMap<>(); + for(String json: docs) { + JSONObject message = (JSONObject) parser.parse(json); + documents.put(createDocument(message, sensorType), Optional.of(indexName)); } - return response; + + // write the documents + updateDao.batchUpdate(documents); + } + + /** + * Create an indexable Document from a JSON message. + * + * @param message The JSON message that needs indexed. + * @param docType The document type to write. + * @return The {@link Document} that was written. + * @throws IOException + */ + private static Document createDocument(JSONObject message, String docType) throws IOException { + Long timestamp = ConversionUtils.convert(message.get("timestamp"), Long.class); + String source = message.toJSONString(); + String guid = (String) message.get("guid"); + return new Document(source, guid, docType, timestamp); } public void createIndexWithMapping(String indexName, String mappingType, String mappingSource) @@ -246,7 +273,6 @@ public List> getAllIndexedDocs(String index, String sourceTy getClient().admin().indices().refresh(new RefreshRequest()); SearchResponse response = getClient().prepareSearch(index) .setTypes(sourceType) -// .setSource("message") ?? .setFrom(0) .setSize(1000) .execute().actionGet(); diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java index ef9714edc1..a1d9c0e720 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java @@ -106,14 +106,14 @@ public void test() throws Exception { }}, Optional.empty()); Assert.assertEquals(message0, update.getDocument()); - Assert.assertEquals(1, getMockHTable().size()); + Assert.assertEquals(10, getMockHTable().size()); findUpdatedDoc(message0, guid, SENSOR_NAME); { //ensure hbase is up to date Get g = new Get(HBaseDao.Key.toBytes(new HBaseDao.Key(guid, SENSOR_NAME))); Result r = getMockHTable().get(g); NavigableMap columns = r.getFamilyMap(CF.getBytes()); - Assert.assertEquals(1, columns.size()); + Assert.assertEquals(2, columns.size()); Assert.assertEquals(message0 , JSONUtils.INSTANCE.load(new String(columns.lastEntry().getValue()) , JSONUtils.MAP_SUPPLIER) @@ -145,7 +145,7 @@ public void test() throws Exception { setIndex(getIndexName()); }}, Optional.empty()); Assert.assertEquals(message0, update.getDocument()); - Assert.assertEquals(1, getMockHTable().size()); + Assert.assertEquals(10, getMockHTable().size()); Document doc = getDao().getLatest(guid, SENSOR_NAME); Assert.assertEquals(message0, doc.getDocument()); findUpdatedDoc(message0, guid, SENSOR_NAME); @@ -154,7 +154,7 @@ public void test() throws Exception { Get g = new Get(HBaseDao.Key.toBytes(new HBaseDao.Key(guid, SENSOR_NAME))); Result r = getMockHTable().get(g); NavigableMap columns = r.getFamilyMap(CF.getBytes()); - Assert.assertEquals(2, columns.size()); + Assert.assertEquals(3, columns.size()); Assert.assertEquals(message0, JSONUtils.INSTANCE.load(new String(columns.lastEntry().getValue()) , JSONUtils.MAP_SUPPLIER) ); diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java index 24989b4a68..90bee80f7a 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java @@ -55,6 +55,7 @@ import org.apache.metron.indexing.dao.update.Document; import org.apache.metron.indexing.dao.update.OriginalNotFoundException; import org.apache.metron.indexing.dao.update.PatchRequest; +import org.json.simple.parser.ParseException; import org.junit.Assert; import org.junit.Test; @@ -1090,7 +1091,7 @@ protected abstract long getMatchingAlertCount(String fieldName, Object fieldValu throws IOException, InterruptedException; protected abstract void addRecords(List> inputData, String index, - String docType) throws IOException; + String docType) throws IOException, ParseException; protected abstract long getMatchingMetaAlertCount(String fieldName, String fieldValue) throws IOException, InterruptedException; From 3b95d1ece278bf517642a0b0a8eee5a7ea021b9e Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 25 Oct 2018 15:06:39 -0400 Subject: [PATCH 09/41] Removed dead function --- .../ElasticsearchSearchIntegrationTest.java | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java index 663b5328fc..268fd0e6c6 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java @@ -178,21 +178,6 @@ private static SearchResponse findAll(ElasticsearchSearchDao searchDao) { } } - /** - * Create an indexable Document from a JSON message. - * - * @param message The JSON message that needs indexed. - * @param docType The document type to write. - * @return The {@link Document} that was written. - * @throws IOException - */ - private static Document createDocument(JSONObject message, String docType) throws IOException { - Long timestamp = ConversionUtils.convert(message.get("timestamp"), Long.class); - String source = message.toJSONString(); - String guid = (String) message.get("guid"); - return new Document(source, guid, docType, timestamp); - } - /** * Add test fields to a template with defined types in case they are not defined in the sensor template shipped with Metron. * This is useful for testing certain cases, for example faceting on fields of various types. From 7d9ee2563fdcc0e6b0a3d5106ea568577f21d303 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 26 Oct 2018 14:55:03 -0400 Subject: [PATCH 10/41] Part way through Solr changes that I am going to roll-back --- .../dao/metaalert/MetaAlertSearchDao.java | 7 +- .../metaalert/MetaAlertIntegrationTest.java | 203 ++++++++++-------- .../solr/dao/SolrMetaAlertUpdateDao.java | 8 +- .../SolrMetaAlertIntegrationTest.java | 39 ++-- .../SolrRetrieveLatestIntegrationTest.java | 5 +- .../SolrSearchIntegrationTest.java | 4 +- .../SolrUpdateIntegrationTest.java | 2 +- .../integration/components/SolrComponent.java | 115 +++++++--- 8 files changed, 243 insertions(+), 140 deletions(-) diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java index cbbe9eefaf..671b03a996 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java @@ -27,8 +27,11 @@ public interface MetaAlertSearchDao extends SearchDao { /** - * Given an alert GUID, retrieve all associated meta alerts. - * @param guid The alert GUID to be searched for + * Returns all active meta-alerts associated with an alert. + * + *

An alert can be associated with multiple meta-alerts. + * + * @param guid The GUID of the alert to search for. * @return All meta alerts with a child alert having the GUID * @throws InvalidSearchException If a problem occurs with the search */ diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java index 90bee80f7a..0adae587a6 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java @@ -18,26 +18,8 @@ package org.apache.metron.indexing.dao.metaalert; -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.ALERT_FIELD; -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_FIELD; -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_TYPE; -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.STATUS_FIELD; -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.THREAT_FIELD_DEFAULT; - import com.google.common.base.Joiner; import com.google.common.collect.Iterables; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; import org.adrianwalker.multilinestring.Multiline; import org.apache.metron.common.Constants; import org.apache.metron.common.utils.JSONUtils; @@ -59,6 +41,25 @@ import org.junit.Assert; import org.junit.Test; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.ALERT_FIELD; +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_FIELD; +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_TYPE; +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.STATUS_FIELD; +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.THREAT_FIELD_DEFAULT; + public abstract class MetaAlertIntegrationTest { private static final String META_INDEX_FLAG = "%META_INDEX%"; @@ -148,50 +149,68 @@ public abstract class MetaAlertIntegrationTest { @Test public void shouldGetAllMetaAlertsForAlert() throws Exception { - // Load alerts - List> alerts = buildAlerts(3); - addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); - - // Load metaAlerts - List> metaAlerts = buildMetaAlerts(12, MetaAlertStatus.ACTIVE, - Optional.of(Collections.singletonList(alerts.get(0)))); - metaAlerts.add(buildMetaAlert("meta_active_12", MetaAlertStatus.ACTIVE, - Optional.of(Arrays.asList(alerts.get(0), alerts.get(2))))); - metaAlerts.add(buildMetaAlert("meta_inactive", MetaAlertStatus.INACTIVE, - Optional.of(Arrays.asList(alerts.get(0), alerts.get(2))))); - // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. - addRecords(metaAlerts, getMetaAlertIndex(), METAALERT_TYPE); + // create 3 'standard' alerts + List> alerts = new ArrayList<>(); + alerts.add(buildAlert("message_0", SENSOR_NAME)); + alerts.add(buildAlert("message_1", SENSOR_NAME)); + alerts.add(buildAlert("message_2", SENSOR_NAME)); + addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); + + // create 5 active meta-alerts; each linked to "message_0" + Optional>> associatedAlerts = Optional.of(Collections.singletonList(alerts.get(0))); + Map meta = addMetaAlert("meta_0", MetaAlertStatus.ACTIVE, associatedAlerts); + addMetaAlert("meta_1", MetaAlertStatus.ACTIVE, associatedAlerts); + addMetaAlert("meta_2", MetaAlertStatus.ACTIVE, associatedAlerts); + addMetaAlert("meta_3", MetaAlertStatus.ACTIVE, associatedAlerts); + addMetaAlert("meta_4", MetaAlertStatus.ACTIVE, associatedAlerts); + + // create 1 meta-alert linked to "message_0" and "message_2" + addMetaAlert("meta_5", MetaAlertStatus.ACTIVE, + Optional.of(Arrays.asList(alerts.get(0), alerts.get(2)))); + + // create 1 "inactive" meta-alert that includes "message_0" and "message_2" + addMetaAlert("meta_6_inactive", MetaAlertStatus.INACTIVE, + Optional.of(Arrays.asList(alerts.get(0), alerts.get(2)))); // Verify load was successful - List createdDocs = metaAlerts.stream().map(metaAlert -> - new GetRequest((String) metaAlert.get(Constants.GUID), METAALERT_TYPE)) - .collect(Collectors.toList()); - createdDocs.addAll(alerts.stream().map(alert -> - new GetRequest((String) alert.get(Constants.GUID), SENSOR_NAME)) - .collect(Collectors.toList())); - findCreatedDocs(createdDocs); + List getRequests = new ArrayList<>(); + getRequests.add(new GetRequest("message_0", SENSOR_NAME)); + getRequests.add(new GetRequest("message_1", SENSOR_NAME)); + getRequests.add(new GetRequest("message_2", SENSOR_NAME)); + getRequests.add(new GetRequest("meta_0", METAALERT_TYPE)); + getRequests.add(new GetRequest("meta_1", METAALERT_TYPE)); + getRequests.add(new GetRequest("meta_2", METAALERT_TYPE)); + getRequests.add(new GetRequest("meta_3", METAALERT_TYPE)); + getRequests.add(new GetRequest("meta_4", METAALERT_TYPE)); + getRequests.add(new GetRequest("meta_5", METAALERT_TYPE)); + getRequests.add(new GetRequest("meta_6_inactive", METAALERT_TYPE)); + findCreatedDocs(getRequests); { - // Verify searches successfully return more than 10 results + // verify there are 13 "active" alerts linked to "alert_0" SearchResponse searchResponse0 = metaDao.getAllMetaAlertsForAlert("message_0"); List searchResults0 = searchResponse0.getResults(); Assert.assertEquals(13, searchResults0.size()); + + // verify that the search results contained the first meta-alert Set> resultSet = new HashSet<>(); Iterables.addAll(resultSet, Iterables.transform(searchResults0, r -> r.getSource())); - StringBuffer reason = new StringBuffer("Unable to find " + metaAlerts.get(0) + "\n"); + StringBuffer reason = new StringBuffer("Unable to find message0\n"); reason.append(Joiner.on("\n").join(resultSet)); - Assert.assertTrue(reason.toString(), resultSet.contains(metaAlerts.get(0))); - - // Verify no meta alerts are returned because message_1 was not added to any + Assert.assertTrue(reason.toString(), resultSet.contains(meta)); + } + { + // Verify that there are no meta-alerts linked to "message_1" SearchResponse searchResponse1 = metaDao.getAllMetaAlertsForAlert("message_1"); List searchResults1 = searchResponse1.getResults(); Assert.assertEquals(0, searchResults1.size()); - - // Verify only the meta alert message_2 was added to is returned + } + { + // Verify that there is only 1 "active" meta-alert linked to "message_2" SearchResponse searchResponse2 = metaDao.getAllMetaAlertsForAlert("message_2"); List searchResults2 = searchResponse2.getResults(); Assert.assertEquals(1, searchResults2.size()); - Assert.assertEquals(metaAlerts.get(12), searchResults2.get(0).getSource()); + Assert.assertEquals("meta_5", searchResults2.get(0).getId()); } } @@ -200,13 +219,10 @@ public void shouldSortByThreatTriageScore() throws Exception { // Load alerts List> alerts = buildAlerts(2); alerts.get(0).put(METAALERT_FIELD, "meta_active_0"); - addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); + addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); // Load metaAlerts - List> metaAlerts = buildMetaAlerts(1, MetaAlertStatus.ACTIVE, - Optional.of(Collections.singletonList(alerts.get(0)))); - // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. - addRecords(metaAlerts, getMetaAlertIndex(), METAALERT_TYPE); + addMetaAlert("meta_1", MetaAlertStatus.ACTIVE, Optional.of(Collections.singletonList(alerts.get(0)))); // Verify load was successful List createdDocs = metaAlerts.stream().map(metaAlert -> @@ -263,11 +279,11 @@ public void getAllMetaAlertsForAlertShouldThrowExceptionForEmptyGuid() throws Ex public void shouldCreateMetaAlert() throws Exception { // Load alerts List> alerts = buildAlerts(3); - addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); + addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); // Verify load was successful findCreatedDocs(Arrays.asList( - new GetRequest("message_0", SENSOR_NAME), + new GetRequest("alert_0", SENSOR_NAME), new GetRequest("message_1", SENSOR_NAME), new GetRequest("message_2", SENSOR_NAME))); @@ -342,12 +358,12 @@ public void shouldAddAlertsToMetaAlert() throws Exception { // Load alerts List> alerts = buildAlerts(4); alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_alert")); - addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); + addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); // Load metaAlert Map metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE, Optional.of(Collections.singletonList(alerts.get(0)))); - addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), METAALERT_TYPE); + addMetaAlerts(Collections.singletonList(metaAlert)); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -434,12 +450,12 @@ public void shouldRemoveAlertsFromMetaAlert() throws Exception { alerts.get(1).put(METAALERT_FIELD, Collections.singletonList("meta_alert")); alerts.get(2).put(METAALERT_FIELD, Collections.singletonList("meta_alert")); alerts.get(3).put(METAALERT_FIELD, Collections.singletonList("meta_alert")); - addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); + addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); // Load metaAlert Map metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE, Optional.of(Arrays.asList(alerts.get(0), alerts.get(1), alerts.get(2), alerts.get(3)))); - addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), METAALERT_TYPE); + addMetaAlerts(Collections.singletonList(metaAlert)); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -552,12 +568,12 @@ public void addRemoveAlertsShouldThrowExceptionForInactiveMetaAlert() throws Exc // Load alerts List> alerts = buildAlerts(2); alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_alert")); - addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); + addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); // Load metaAlert Map metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.INACTIVE, Optional.of(Collections.singletonList(alerts.get(0)))); - addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), METAALERT_TYPE); + addMetaAlerts(Collections.singletonList(metaAlert)); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -603,14 +619,12 @@ public void shouldUpdateMetaAlertStatus() throws Exception { for (Map alert : childAlerts) { alert.put(METAALERT_FIELD, Collections.singletonList("meta_alert")); } - addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); + addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); // Load metaAlerts Map metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE, Optional.of(childAlerts)); - // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. - addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), - METAALERT_TYPE); + addMetaAlerts(Collections.singletonList(metaAlert)); List requests = new ArrayList<>(); for (int i = 0; i < numChildAlerts; ++i) { @@ -706,10 +720,7 @@ public void shouldSearchByStatus() throws Exception { Map inactiveMetaAlert = buildMetaAlert("meta_inactive", MetaAlertStatus.INACTIVE, Optional.empty()); - - // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. - addRecords(Arrays.asList(activeMetaAlert, inactiveMetaAlert), getMetaAlertIndex(), - METAALERT_TYPE); + addMetaAlerts(Arrays.asList(activeMetaAlert, inactiveMetaAlert)); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -744,7 +755,7 @@ public void shouldHidesAlertsOnGroup() throws Exception { alerts.get(0).put("score", 1); alerts.get(1).put("ip_src_addr", "192.168.1.1"); alerts.get(1).put("score", 10); - addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); + addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); // Put the nested type into the test index, so that it'll match appropriately setupTypings(); @@ -789,7 +800,7 @@ public void shouldUpdateMetaAlertOnAlertUpdate() throws Exception { // Load alerts List> alerts = buildAlerts(2); alerts.get(0).put(METAALERT_FIELD, Arrays.asList("meta_active", "meta_inactive")); - addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); + addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); // Load metaAlerts Map activeMetaAlert = buildMetaAlert("meta_active", MetaAlertStatus.ACTIVE, @@ -797,9 +808,7 @@ public void shouldUpdateMetaAlertOnAlertUpdate() throws Exception { Map inactiveMetaAlert = buildMetaAlert("meta_inactive", MetaAlertStatus.INACTIVE, Optional.of(Collections.singletonList(alerts.get(0)))); - // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. - addRecords(Arrays.asList(activeMetaAlert, inactiveMetaAlert), getMetaAlertIndex(), - METAALERT_TYPE); + addMetaAlerts(Arrays.asList(activeMetaAlert, inactiveMetaAlert)); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -889,7 +898,7 @@ public void shouldPatchAllowedMetaAlerts() throws Exception { List> alerts = buildAlerts(2); alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_active")); alerts.get(1).put(METAALERT_FIELD, Collections.singletonList("meta_active")); - addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); + addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); // Put the nested type into the test index, so that it'll match appropriately setupTypings(); @@ -898,7 +907,7 @@ public void shouldPatchAllowedMetaAlerts() throws Exception { Map metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE, Optional.of(Arrays.asList(alerts.get(0), alerts.get(1)))); // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. - addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), METAALERT_TYPE); + addMetaAlerts(Collections.singletonList(metaAlert)); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -996,8 +1005,7 @@ protected void convertAlertsFieldToList(Map document) { } } - protected boolean findCreatedDoc(String guid, String sensorType) - throws InterruptedException, IOException, OriginalNotFoundException { + protected boolean findCreatedDoc(String guid, String sensorType) throws Exception { for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) { Document doc = metaDao.getLatest(guid, sensorType); if (doc != null) { @@ -1053,16 +1061,20 @@ protected List> buildAlerts(int count) { List> inputData = new ArrayList<>(); for (int i = 0; i < count; ++i) { final String guid = "message_" + i; - Map alerts = new HashMap<>(); - alerts.put(Constants.GUID, guid); - alerts.put(getSourceTypeField(), SENSOR_NAME); - alerts.put(THREAT_FIELD_DEFAULT, (double) i); - alerts.put("timestamp", System.currentTimeMillis()); - inputData.add(alerts); + inputData.add(buildAlert(guid, SENSOR_NAME)); } return inputData; } + protected Map buildAlert(String guid, String sensorName) { + Map alert = new HashMap<>(); + alert.put(Constants.GUID, guid); + alert.put(getSourceTypeField(), sensorName); + alert.put(THREAT_FIELD_DEFAULT, (double) i); + alert.put(Constants.Fields.TIMESTAMP.getName(), System.currentTimeMillis()); + return alert; + } + protected List> buildMetaAlerts(int count, MetaAlertStatus status, Optional>> alerts) { List> inputData = new ArrayList<>(); @@ -1073,8 +1085,7 @@ protected List> buildMetaAlerts(int count, MetaAlertStatus s return inputData; } - protected Map buildMetaAlert(String guid, MetaAlertStatus status, - Optional>> alerts) { + protected Map buildMetaAlert(String guid, MetaAlertStatus status, Optional>> alerts) { Map metaAlert = new HashMap<>(); metaAlert.put(Constants.GUID, guid); metaAlert.put(getSourceTypeField(), METAALERT_TYPE); @@ -1090,8 +1101,28 @@ protected Map buildMetaAlert(String guid, MetaAlertStatus status protected abstract long getMatchingAlertCount(String fieldName, Object fieldValue) throws IOException, InterruptedException; - protected abstract void addRecords(List> inputData, String index, - String docType) throws IOException, ParseException; + /** + * Add an alert. + * + * @param alerts The alerts to add. + * @param index The index to add the alert to. + * @param docType The document type of the alert. + * @throws IOException + * @throws ParseException + */ + protected abstract void addAlerts(List> alerts, String index, String docType) throws Exception; + + /** + * Create a meta-alert. + * + * @param guid The GUID of the meta-alert. + * @param status The status of the meta-alert. + * @param alerts The alerts to associate with the meta-alert. + * @throws IOException + * @throws ParseException + */ + protected abstract Map addMetaAlert(String guid, MetaAlertStatus status, + Optional>> alerts) throws Exception; protected abstract long getMatchingMetaAlertCount(String fieldName, String fieldValue) throws IOException, InterruptedException; diff --git a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertUpdateDao.java b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertUpdateDao.java index 124d4be140..a2189a3404 100644 --- a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertUpdateDao.java +++ b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertUpdateDao.java @@ -135,10 +135,10 @@ public Document createMetaAlert(MetaAlertCreateRequest request) */ @Override public Document update(Document update, Optional collection) throws IOException { - if (MetaAlertConstants.METAALERT_TYPE.equals(update.getSensorType())) { - // We've been passed an update to the meta alert. - throw new UnsupportedOperationException("Meta alerts cannot be directly updated"); - } +// if (MetaAlertConstants.METAALERT_TYPE.equals(update.getSensorType())) { +// // We've been passed an update to the meta alert. +// throw new UnsupportedOperationException("Meta alerts cannot be directly updated"); +// } // Index can't be optional, or it won't be committed Map> updates = new HashMap<>(); diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrMetaAlertIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrMetaAlertIntegrationTest.java index 6687e9af57..620724bfee 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrMetaAlertIntegrationTest.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrMetaAlertIntegrationTest.java @@ -33,6 +33,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; import com.google.common.collect.ImmutableMap; import org.apache.metron.common.Constants; @@ -53,6 +54,7 @@ import org.apache.metron.solr.integration.components.SolrComponent; import org.apache.solr.client.solrj.SolrServerException; import org.apache.zookeeper.KeeperException; +import org.json.simple.parser.ParseException; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -159,7 +161,7 @@ public void shouldSearchByNestedAlert() throws Exception { alerts.get(2).put("ip_src_port", 8008); alerts.get(3).put("ip_src_addr", "192.168.1.4"); alerts.get(3).put("ip_src_port", 8007); - addRecords(alerts, getTestIndexName(), SENSOR_NAME); + addAlerts(alerts, getTestIndexName(), SENSOR_NAME); // Put the nested type into the test index, so that it'll match appropriately setupTypings(); @@ -170,9 +172,7 @@ public void shouldSearchByNestedAlert() throws Exception { Map inactiveMetaAlert = buildMetaAlert("meta_inactive", MetaAlertStatus.INACTIVE, Optional.of(Arrays.asList(alerts.get(2), alerts.get(3)))); - // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. - addRecords(Arrays.asList(activeMetaAlert, inactiveMetaAlert), METAALERTS_COLLECTION, - METAALERT_TYPE); + addMetaAlerts(Arrays.asList(activeMetaAlert, inactiveMetaAlert)); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -269,7 +269,7 @@ public void shouldNotRetrieveFullChildrenWithoutSourceType() throws Exception { alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_active")); alerts.get(0).put("ip_src_addr", "192.168.1.1"); alerts.get(0).put("ip_src_port", 8010); - addRecords(alerts, getTestIndexName(), SENSOR_NAME); + addAlerts(alerts, getTestIndexName(), SENSOR_NAME); // Put the nested type into the test index, so that it'll match appropriately setupTypings(); @@ -278,7 +278,7 @@ public void shouldNotRetrieveFullChildrenWithoutSourceType() throws Exception { Map activeMetaAlert = buildMetaAlert("meta_active", MetaAlertStatus.ACTIVE, Optional.of(Arrays.asList(alerts.get(0)))); // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. - addRecords(Collections.singletonList(activeMetaAlert), METAALERTS_COLLECTION, METAALERT_TYPE); + addMetaAlerts(Collections.singletonList(activeMetaAlert)); // Verify load was successful findCreatedDocs(Collections.singletonList(new GetRequest("meta_active", METAALERT_TYPE))); @@ -353,13 +353,28 @@ protected long getMatchingMetaAlertCount(String fieldName, String fieldValue) } @Override - protected void addRecords(List> inputData, String index, String docType) - throws IOException { + protected void addAlerts(List> alerts, String index, String docType) throws Exception { + // Ignore docType for Solr. It's unused. - try { - solr.addDocs(index, inputData); - } catch (SolrServerException e) { - throw new IOException("Unable to load Solr Docs", e); + solr.addAlerts(metaDao, index, alerts); + + // ensure that the alerts were added; exception thrown if not found after a number of retries + for(Map alert: alerts) { + String guid = String.class.cast(alert.get(Constants.GUID)); + findCreatedDoc(guid, docType); + } + } + + @Override + protected void addMetaAlerts(List> metaAlerts) throws Exception { + // Ignore docType for Solr. It's unused. + solr.addAlerts(metaDao, index, metaAlerts); + solr.addMetaAlert(metaDao, metaAlerts); + + // ensure that the alerts were added; exception thrown if not found after a number of retries + for(Map alert: alerts) { + String guid = String.class.cast(alert.get(Constants.GUID)); + findCreatedDoc(guid, docType); } } diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java index f7c2e86a20..fc5192775a 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java @@ -188,8 +188,7 @@ protected GetRequest buildGetRequest(String sensor, int i) { return requestOne; } - protected static void addData(String collection, String sensorName) - throws IOException, SolrServerException { + protected static void addData(String collection, String sensorName) throws IOException { List> inputData = new ArrayList<>(); for (int i = 0; i < 3; ++i) { final String name = buildGuid(sensorName, i); @@ -198,7 +197,7 @@ protected static void addData(String collection, String sensorName) inputMap.put(Constants.GUID, name); inputData.add(inputMap); } - solrComponent.addDocs(collection, inputData); + solrComponent.addDocs(dao, collection, inputData); } protected static String buildGuid(String sensorName, int i) { diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrSearchIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrSearchIntegrationTest.java index 84f222258f..a3fc198493 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrSearchIntegrationTest.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrSearchIntegrationTest.java @@ -99,9 +99,9 @@ protected static InMemoryComponent startIndex() throws Exception { @SuppressWarnings("unchecked") protected static void loadTestData() throws ParseException, IOException, SolrServerException { JSONArray broArray = (JSONArray) new JSONParser().parse(broData); - solrComponent.addDocs("bro", broArray); + solrComponent.addDocs(dao,"bro", broArray); JSONArray snortArray = (JSONArray) new JSONParser().parse(snortData); - solrComponent.addDocs("snort", snortArray); + solrComponent.addDocs(dao,"snort", snortArray); } @Override diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java index 5b965590c7..d2256cfdf4 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java @@ -128,7 +128,7 @@ private static Map createGlobalConfig() { @Override protected void addTestData(String indexName, String sensorType, List> docs) throws Exception { - solrComponent.addDocs(indexName, docs); + solrComponent.addDocs(getDao(), indexName, docs); } @Override diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java index 4bc9f8a469..e92cc780f7 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java @@ -19,14 +19,23 @@ import com.google.common.base.Function; import java.util.Collection; +import java.util.Collections; import java.util.Map.Entry; +import java.util.Optional; import java.util.stream.Collectors; import org.apache.metron.common.Constants; import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants; +import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest; +import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus; +import org.apache.metron.indexing.dao.metaalert.MetaAlertUpdateDao; +import org.apache.metron.indexing.dao.search.GetRequest; +import org.apache.metron.indexing.dao.update.Document; +import org.apache.metron.indexing.dao.update.UpdateDao; import org.apache.metron.integration.InMemoryComponent; import org.apache.metron.integration.UnableToStartException; import org.apache.metron.solr.dao.SolrUtilities; import org.apache.metron.solr.writer.MetronSolrClient; +import org.apache.metron.stellar.common.utils.ConversionUtils; import org.apache.solr.client.solrj.SolrQuery; import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.embedded.JettyConfig; @@ -46,6 +55,9 @@ import java.util.Map; import org.apache.solr.common.SolrInputDocument; import org.apache.zookeeper.KeeperException; +import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; +import org.json.simple.parser.ParseException; public class SolrComponent implements InMemoryComponent { @@ -191,37 +203,80 @@ public List> getAllIndexedDocs(String collection) { return docs; } - public void addDocs(String collection, List> docs) - throws IOException, SolrServerException { - CloudSolrClient solr = miniSolrCloudCluster.getSolrClient(); - solr.setDefaultCollection(collection); - Collection solrInputDocuments = docs.stream().map(doc -> { - SolrInputDocument solrInputDocument = new SolrInputDocument(); - for (Entry entry : doc.entrySet()) { - // If the entry itself is a map, add it as a child document. Handle one level of nesting. - if (entry.getValue() instanceof List && !entry.getKey().equals( - MetaAlertConstants.METAALERT_FIELD)) { - for (Object entryItem : (List)entry.getValue()) { - if (entryItem instanceof Map) { - @SuppressWarnings("unchecked") - Map childDoc = (Map) entryItem; - SolrInputDocument childInputDoc = new SolrInputDocument(); - for (Entry childEntry : childDoc.entrySet()) { - childInputDoc.addField(childEntry.getKey(), childEntry.getValue()); - } - solrInputDocument.addChildDocument(childInputDoc); - } - } - } else { - solrInputDocument.addField(entry.getKey(), entry.getValue()); - } - } - return solrInputDocument; - }).collect(Collectors.toList()); + public void addAlerts(UpdateDao updateDao, String collection, List> messages) throws IOException { + // transform each input message to a Document that can be indexed by the UpdateDao + for(Map msg: messages) { + updateDao.update(createDocument(msg, collection), Optional.of(collection)); + } + + // TODO do we need to wait until the documents are visible? + try { + Thread.sleep(2000); + } catch(Exception e) { + } + +// +// checkUpdateResponse(solr.add(collection, solrInputDocuments)); +// // Make sure to commit so things show up +// checkUpdateResponse(solr.commit(true, true)); + } + + private List getRequest(List> alerts) { + List requests = new ArrayList<>(); + for(Map alert: alerts) { + String guid = String.class.cast(alert.get(Constants.GUID)); + requests.add(new GetRequest(guid, MetaAlertConstants.METAALERT_TYPE)); + } + return requests; + } + + public void addMetaAlert(MetaAlertUpdateDao updateDao, String guid, MetaAlertStatus status, + Optional>> alerts) throws IOException { + + MetaAlertCreateRequest request = new MetaAlertCreateRequest(); + alerts.ifPresent(theAlerts -> request.setAlerts(getRequest(theAlerts))); + + // transform each input message to a Document that can be indexed by the UpdateDao + for(Map msg: messages) { + + + + {{ + setAlerts(new ArrayList() {{ + add(new GetRequest("message_1", SENSOR_NAME)); + add(new GetRequest("message_2", SENSOR_NAME, getTestIndexFullName())); + }}); + setGroups(Collections.singletonList("group")); + }}; + + MetaAlertCreateRequest create = new MetaAlertCreateRequest(); + create.setAlerts(); + updateDao.createMetaAlert(createDocument(msg, collection), Optional.of(collection)); + } + + try { + Thread.sleep(2000); + } catch(Exception e) { + } + // TODO do we need to wait until the documents are visible? +// +// checkUpdateResponse(solr.add(collection, solrInputDocuments)); +// // Make sure to commit so things show up +// checkUpdateResponse(solr.commit(true, true)); + } - checkUpdateResponse(solr.add(collection, solrInputDocuments)); - // Make sure to commit so things show up - checkUpdateResponse(solr.commit(true, true)); + /** + * Create an indexable Document from a JSON message. + * + * @param message The message that needs indexed. + * @param docType The document type to write. + * @return The {@link Document} that was written. + * @throws IOException + */ + private static Document createDocument(Map message, String docType) throws IOException { + Long timestamp = ConversionUtils.convert(message.get("timestamp"), Long.class); + String guid = (String) message.get("guid"); + return new Document(message, guid, docType, timestamp); } protected void checkUpdateResponse(UpdateResponse result) throws IOException { From e6993867ebd00a5c09ef1a4da772664a11d1f212 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 26 Oct 2018 14:55:15 -0400 Subject: [PATCH 11/41] Revert "Part way through Solr changes that I am going to roll-back" This reverts commit 7d9ee2563fdcc0e6b0a3d5106ea568577f21d303. --- .../dao/metaalert/MetaAlertSearchDao.java | 7 +- .../metaalert/MetaAlertIntegrationTest.java | 203 ++++++++---------- .../solr/dao/SolrMetaAlertUpdateDao.java | 8 +- .../SolrMetaAlertIntegrationTest.java | 39 ++-- .../SolrRetrieveLatestIntegrationTest.java | 5 +- .../SolrSearchIntegrationTest.java | 4 +- .../SolrUpdateIntegrationTest.java | 2 +- .../integration/components/SolrComponent.java | 115 +++------- 8 files changed, 140 insertions(+), 243 deletions(-) diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java index 671b03a996..cbbe9eefaf 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java @@ -27,11 +27,8 @@ public interface MetaAlertSearchDao extends SearchDao { /** - * Returns all active meta-alerts associated with an alert. - * - *

An alert can be associated with multiple meta-alerts. - * - * @param guid The GUID of the alert to search for. + * Given an alert GUID, retrieve all associated meta alerts. + * @param guid The alert GUID to be searched for * @return All meta alerts with a child alert having the GUID * @throws InvalidSearchException If a problem occurs with the search */ diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java index 0adae587a6..90bee80f7a 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java @@ -18,8 +18,26 @@ package org.apache.metron.indexing.dao.metaalert; +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.ALERT_FIELD; +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_FIELD; +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_TYPE; +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.STATUS_FIELD; +import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.THREAT_FIELD_DEFAULT; + import com.google.common.base.Joiner; import com.google.common.collect.Iterables; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; import org.adrianwalker.multilinestring.Multiline; import org.apache.metron.common.Constants; import org.apache.metron.common.utils.JSONUtils; @@ -41,25 +59,6 @@ import org.junit.Assert; import org.junit.Test; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; - -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.ALERT_FIELD; -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_FIELD; -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_TYPE; -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.STATUS_FIELD; -import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.THREAT_FIELD_DEFAULT; - public abstract class MetaAlertIntegrationTest { private static final String META_INDEX_FLAG = "%META_INDEX%"; @@ -149,68 +148,50 @@ public abstract class MetaAlertIntegrationTest { @Test public void shouldGetAllMetaAlertsForAlert() throws Exception { - // create 3 'standard' alerts - List> alerts = new ArrayList<>(); - alerts.add(buildAlert("message_0", SENSOR_NAME)); - alerts.add(buildAlert("message_1", SENSOR_NAME)); - alerts.add(buildAlert("message_2", SENSOR_NAME)); - addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); - - // create 5 active meta-alerts; each linked to "message_0" - Optional>> associatedAlerts = Optional.of(Collections.singletonList(alerts.get(0))); - Map meta = addMetaAlert("meta_0", MetaAlertStatus.ACTIVE, associatedAlerts); - addMetaAlert("meta_1", MetaAlertStatus.ACTIVE, associatedAlerts); - addMetaAlert("meta_2", MetaAlertStatus.ACTIVE, associatedAlerts); - addMetaAlert("meta_3", MetaAlertStatus.ACTIVE, associatedAlerts); - addMetaAlert("meta_4", MetaAlertStatus.ACTIVE, associatedAlerts); - - // create 1 meta-alert linked to "message_0" and "message_2" - addMetaAlert("meta_5", MetaAlertStatus.ACTIVE, - Optional.of(Arrays.asList(alerts.get(0), alerts.get(2)))); - - // create 1 "inactive" meta-alert that includes "message_0" and "message_2" - addMetaAlert("meta_6_inactive", MetaAlertStatus.INACTIVE, - Optional.of(Arrays.asList(alerts.get(0), alerts.get(2)))); + // Load alerts + List> alerts = buildAlerts(3); + addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); + + // Load metaAlerts + List> metaAlerts = buildMetaAlerts(12, MetaAlertStatus.ACTIVE, + Optional.of(Collections.singletonList(alerts.get(0)))); + metaAlerts.add(buildMetaAlert("meta_active_12", MetaAlertStatus.ACTIVE, + Optional.of(Arrays.asList(alerts.get(0), alerts.get(2))))); + metaAlerts.add(buildMetaAlert("meta_inactive", MetaAlertStatus.INACTIVE, + Optional.of(Arrays.asList(alerts.get(0), alerts.get(2))))); + // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. + addRecords(metaAlerts, getMetaAlertIndex(), METAALERT_TYPE); // Verify load was successful - List getRequests = new ArrayList<>(); - getRequests.add(new GetRequest("message_0", SENSOR_NAME)); - getRequests.add(new GetRequest("message_1", SENSOR_NAME)); - getRequests.add(new GetRequest("message_2", SENSOR_NAME)); - getRequests.add(new GetRequest("meta_0", METAALERT_TYPE)); - getRequests.add(new GetRequest("meta_1", METAALERT_TYPE)); - getRequests.add(new GetRequest("meta_2", METAALERT_TYPE)); - getRequests.add(new GetRequest("meta_3", METAALERT_TYPE)); - getRequests.add(new GetRequest("meta_4", METAALERT_TYPE)); - getRequests.add(new GetRequest("meta_5", METAALERT_TYPE)); - getRequests.add(new GetRequest("meta_6_inactive", METAALERT_TYPE)); - findCreatedDocs(getRequests); + List createdDocs = metaAlerts.stream().map(metaAlert -> + new GetRequest((String) metaAlert.get(Constants.GUID), METAALERT_TYPE)) + .collect(Collectors.toList()); + createdDocs.addAll(alerts.stream().map(alert -> + new GetRequest((String) alert.get(Constants.GUID), SENSOR_NAME)) + .collect(Collectors.toList())); + findCreatedDocs(createdDocs); { - // verify there are 13 "active" alerts linked to "alert_0" + // Verify searches successfully return more than 10 results SearchResponse searchResponse0 = metaDao.getAllMetaAlertsForAlert("message_0"); List searchResults0 = searchResponse0.getResults(); Assert.assertEquals(13, searchResults0.size()); - - // verify that the search results contained the first meta-alert Set> resultSet = new HashSet<>(); Iterables.addAll(resultSet, Iterables.transform(searchResults0, r -> r.getSource())); - StringBuffer reason = new StringBuffer("Unable to find message0\n"); + StringBuffer reason = new StringBuffer("Unable to find " + metaAlerts.get(0) + "\n"); reason.append(Joiner.on("\n").join(resultSet)); - Assert.assertTrue(reason.toString(), resultSet.contains(meta)); - } - { - // Verify that there are no meta-alerts linked to "message_1" + Assert.assertTrue(reason.toString(), resultSet.contains(metaAlerts.get(0))); + + // Verify no meta alerts are returned because message_1 was not added to any SearchResponse searchResponse1 = metaDao.getAllMetaAlertsForAlert("message_1"); List searchResults1 = searchResponse1.getResults(); Assert.assertEquals(0, searchResults1.size()); - } - { - // Verify that there is only 1 "active" meta-alert linked to "message_2" + + // Verify only the meta alert message_2 was added to is returned SearchResponse searchResponse2 = metaDao.getAllMetaAlertsForAlert("message_2"); List searchResults2 = searchResponse2.getResults(); Assert.assertEquals(1, searchResults2.size()); - Assert.assertEquals("meta_5", searchResults2.get(0).getId()); + Assert.assertEquals(metaAlerts.get(12), searchResults2.get(0).getSource()); } } @@ -219,10 +200,13 @@ public void shouldSortByThreatTriageScore() throws Exception { // Load alerts List> alerts = buildAlerts(2); alerts.get(0).put(METAALERT_FIELD, "meta_active_0"); - addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); + addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); // Load metaAlerts - addMetaAlert("meta_1", MetaAlertStatus.ACTIVE, Optional.of(Collections.singletonList(alerts.get(0)))); + List> metaAlerts = buildMetaAlerts(1, MetaAlertStatus.ACTIVE, + Optional.of(Collections.singletonList(alerts.get(0)))); + // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. + addRecords(metaAlerts, getMetaAlertIndex(), METAALERT_TYPE); // Verify load was successful List createdDocs = metaAlerts.stream().map(metaAlert -> @@ -279,11 +263,11 @@ public void getAllMetaAlertsForAlertShouldThrowExceptionForEmptyGuid() throws Ex public void shouldCreateMetaAlert() throws Exception { // Load alerts List> alerts = buildAlerts(3); - addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); + addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); // Verify load was successful findCreatedDocs(Arrays.asList( - new GetRequest("alert_0", SENSOR_NAME), + new GetRequest("message_0", SENSOR_NAME), new GetRequest("message_1", SENSOR_NAME), new GetRequest("message_2", SENSOR_NAME))); @@ -358,12 +342,12 @@ public void shouldAddAlertsToMetaAlert() throws Exception { // Load alerts List> alerts = buildAlerts(4); alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_alert")); - addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); + addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); // Load metaAlert Map metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE, Optional.of(Collections.singletonList(alerts.get(0)))); - addMetaAlerts(Collections.singletonList(metaAlert)); + addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), METAALERT_TYPE); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -450,12 +434,12 @@ public void shouldRemoveAlertsFromMetaAlert() throws Exception { alerts.get(1).put(METAALERT_FIELD, Collections.singletonList("meta_alert")); alerts.get(2).put(METAALERT_FIELD, Collections.singletonList("meta_alert")); alerts.get(3).put(METAALERT_FIELD, Collections.singletonList("meta_alert")); - addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); + addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); // Load metaAlert Map metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE, Optional.of(Arrays.asList(alerts.get(0), alerts.get(1), alerts.get(2), alerts.get(3)))); - addMetaAlerts(Collections.singletonList(metaAlert)); + addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), METAALERT_TYPE); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -568,12 +552,12 @@ public void addRemoveAlertsShouldThrowExceptionForInactiveMetaAlert() throws Exc // Load alerts List> alerts = buildAlerts(2); alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_alert")); - addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); + addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); // Load metaAlert Map metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.INACTIVE, Optional.of(Collections.singletonList(alerts.get(0)))); - addMetaAlerts(Collections.singletonList(metaAlert)); + addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), METAALERT_TYPE); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -619,12 +603,14 @@ public void shouldUpdateMetaAlertStatus() throws Exception { for (Map alert : childAlerts) { alert.put(METAALERT_FIELD, Collections.singletonList("meta_alert")); } - addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); + addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); // Load metaAlerts Map metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE, Optional.of(childAlerts)); - addMetaAlerts(Collections.singletonList(metaAlert)); + // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. + addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), + METAALERT_TYPE); List requests = new ArrayList<>(); for (int i = 0; i < numChildAlerts; ++i) { @@ -720,7 +706,10 @@ public void shouldSearchByStatus() throws Exception { Map inactiveMetaAlert = buildMetaAlert("meta_inactive", MetaAlertStatus.INACTIVE, Optional.empty()); - addMetaAlerts(Arrays.asList(activeMetaAlert, inactiveMetaAlert)); + + // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. + addRecords(Arrays.asList(activeMetaAlert, inactiveMetaAlert), getMetaAlertIndex(), + METAALERT_TYPE); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -755,7 +744,7 @@ public void shouldHidesAlertsOnGroup() throws Exception { alerts.get(0).put("score", 1); alerts.get(1).put("ip_src_addr", "192.168.1.1"); alerts.get(1).put("score", 10); - addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); + addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); // Put the nested type into the test index, so that it'll match appropriately setupTypings(); @@ -800,7 +789,7 @@ public void shouldUpdateMetaAlertOnAlertUpdate() throws Exception { // Load alerts List> alerts = buildAlerts(2); alerts.get(0).put(METAALERT_FIELD, Arrays.asList("meta_active", "meta_inactive")); - addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); + addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); // Load metaAlerts Map activeMetaAlert = buildMetaAlert("meta_active", MetaAlertStatus.ACTIVE, @@ -808,7 +797,9 @@ public void shouldUpdateMetaAlertOnAlertUpdate() throws Exception { Map inactiveMetaAlert = buildMetaAlert("meta_inactive", MetaAlertStatus.INACTIVE, Optional.of(Collections.singletonList(alerts.get(0)))); - addMetaAlerts(Arrays.asList(activeMetaAlert, inactiveMetaAlert)); + // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. + addRecords(Arrays.asList(activeMetaAlert, inactiveMetaAlert), getMetaAlertIndex(), + METAALERT_TYPE); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -898,7 +889,7 @@ public void shouldPatchAllowedMetaAlerts() throws Exception { List> alerts = buildAlerts(2); alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_active")); alerts.get(1).put(METAALERT_FIELD, Collections.singletonList("meta_active")); - addAlerts(alerts, getTestIndexFullName(), SENSOR_NAME); + addRecords(alerts, getTestIndexFullName(), SENSOR_NAME); // Put the nested type into the test index, so that it'll match appropriately setupTypings(); @@ -907,7 +898,7 @@ public void shouldPatchAllowedMetaAlerts() throws Exception { Map metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE, Optional.of(Arrays.asList(alerts.get(0), alerts.get(1)))); // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. - addMetaAlerts(Collections.singletonList(metaAlert)); + addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), METAALERT_TYPE); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -1005,7 +996,8 @@ protected void convertAlertsFieldToList(Map document) { } } - protected boolean findCreatedDoc(String guid, String sensorType) throws Exception { + protected boolean findCreatedDoc(String guid, String sensorType) + throws InterruptedException, IOException, OriginalNotFoundException { for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) { Document doc = metaDao.getLatest(guid, sensorType); if (doc != null) { @@ -1061,20 +1053,16 @@ protected List> buildAlerts(int count) { List> inputData = new ArrayList<>(); for (int i = 0; i < count; ++i) { final String guid = "message_" + i; - inputData.add(buildAlert(guid, SENSOR_NAME)); + Map alerts = new HashMap<>(); + alerts.put(Constants.GUID, guid); + alerts.put(getSourceTypeField(), SENSOR_NAME); + alerts.put(THREAT_FIELD_DEFAULT, (double) i); + alerts.put("timestamp", System.currentTimeMillis()); + inputData.add(alerts); } return inputData; } - protected Map buildAlert(String guid, String sensorName) { - Map alert = new HashMap<>(); - alert.put(Constants.GUID, guid); - alert.put(getSourceTypeField(), sensorName); - alert.put(THREAT_FIELD_DEFAULT, (double) i); - alert.put(Constants.Fields.TIMESTAMP.getName(), System.currentTimeMillis()); - return alert; - } - protected List> buildMetaAlerts(int count, MetaAlertStatus status, Optional>> alerts) { List> inputData = new ArrayList<>(); @@ -1085,7 +1073,8 @@ protected List> buildMetaAlerts(int count, MetaAlertStatus s return inputData; } - protected Map buildMetaAlert(String guid, MetaAlertStatus status, Optional>> alerts) { + protected Map buildMetaAlert(String guid, MetaAlertStatus status, + Optional>> alerts) { Map metaAlert = new HashMap<>(); metaAlert.put(Constants.GUID, guid); metaAlert.put(getSourceTypeField(), METAALERT_TYPE); @@ -1101,28 +1090,8 @@ protected Map buildMetaAlert(String guid, MetaAlertStatus status protected abstract long getMatchingAlertCount(String fieldName, Object fieldValue) throws IOException, InterruptedException; - /** - * Add an alert. - * - * @param alerts The alerts to add. - * @param index The index to add the alert to. - * @param docType The document type of the alert. - * @throws IOException - * @throws ParseException - */ - protected abstract void addAlerts(List> alerts, String index, String docType) throws Exception; - - /** - * Create a meta-alert. - * - * @param guid The GUID of the meta-alert. - * @param status The status of the meta-alert. - * @param alerts The alerts to associate with the meta-alert. - * @throws IOException - * @throws ParseException - */ - protected abstract Map addMetaAlert(String guid, MetaAlertStatus status, - Optional>> alerts) throws Exception; + protected abstract void addRecords(List> inputData, String index, + String docType) throws IOException, ParseException; protected abstract long getMatchingMetaAlertCount(String fieldName, String fieldValue) throws IOException, InterruptedException; diff --git a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertUpdateDao.java b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertUpdateDao.java index a2189a3404..124d4be140 100644 --- a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertUpdateDao.java +++ b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertUpdateDao.java @@ -135,10 +135,10 @@ public Document createMetaAlert(MetaAlertCreateRequest request) */ @Override public Document update(Document update, Optional collection) throws IOException { -// if (MetaAlertConstants.METAALERT_TYPE.equals(update.getSensorType())) { -// // We've been passed an update to the meta alert. -// throw new UnsupportedOperationException("Meta alerts cannot be directly updated"); -// } + if (MetaAlertConstants.METAALERT_TYPE.equals(update.getSensorType())) { + // We've been passed an update to the meta alert. + throw new UnsupportedOperationException("Meta alerts cannot be directly updated"); + } // Index can't be optional, or it won't be committed Map> updates = new HashMap<>(); diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrMetaAlertIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrMetaAlertIntegrationTest.java index 620724bfee..6687e9af57 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrMetaAlertIntegrationTest.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrMetaAlertIntegrationTest.java @@ -33,7 +33,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.stream.Collectors; import com.google.common.collect.ImmutableMap; import org.apache.metron.common.Constants; @@ -54,7 +53,6 @@ import org.apache.metron.solr.integration.components.SolrComponent; import org.apache.solr.client.solrj.SolrServerException; import org.apache.zookeeper.KeeperException; -import org.json.simple.parser.ParseException; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -161,7 +159,7 @@ public void shouldSearchByNestedAlert() throws Exception { alerts.get(2).put("ip_src_port", 8008); alerts.get(3).put("ip_src_addr", "192.168.1.4"); alerts.get(3).put("ip_src_port", 8007); - addAlerts(alerts, getTestIndexName(), SENSOR_NAME); + addRecords(alerts, getTestIndexName(), SENSOR_NAME); // Put the nested type into the test index, so that it'll match appropriately setupTypings(); @@ -172,7 +170,9 @@ public void shouldSearchByNestedAlert() throws Exception { Map inactiveMetaAlert = buildMetaAlert("meta_inactive", MetaAlertStatus.INACTIVE, Optional.of(Arrays.asList(alerts.get(2), alerts.get(3)))); - addMetaAlerts(Arrays.asList(activeMetaAlert, inactiveMetaAlert)); + // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. + addRecords(Arrays.asList(activeMetaAlert, inactiveMetaAlert), METAALERTS_COLLECTION, + METAALERT_TYPE); // Verify load was successful findCreatedDocs(Arrays.asList( @@ -269,7 +269,7 @@ public void shouldNotRetrieveFullChildrenWithoutSourceType() throws Exception { alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_active")); alerts.get(0).put("ip_src_addr", "192.168.1.1"); alerts.get(0).put("ip_src_port", 8010); - addAlerts(alerts, getTestIndexName(), SENSOR_NAME); + addRecords(alerts, getTestIndexName(), SENSOR_NAME); // Put the nested type into the test index, so that it'll match appropriately setupTypings(); @@ -278,7 +278,7 @@ public void shouldNotRetrieveFullChildrenWithoutSourceType() throws Exception { Map activeMetaAlert = buildMetaAlert("meta_active", MetaAlertStatus.ACTIVE, Optional.of(Arrays.asList(alerts.get(0)))); // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically. - addMetaAlerts(Collections.singletonList(activeMetaAlert)); + addRecords(Collections.singletonList(activeMetaAlert), METAALERTS_COLLECTION, METAALERT_TYPE); // Verify load was successful findCreatedDocs(Collections.singletonList(new GetRequest("meta_active", METAALERT_TYPE))); @@ -353,28 +353,13 @@ protected long getMatchingMetaAlertCount(String fieldName, String fieldValue) } @Override - protected void addAlerts(List> alerts, String index, String docType) throws Exception { - + protected void addRecords(List> inputData, String index, String docType) + throws IOException { // Ignore docType for Solr. It's unused. - solr.addAlerts(metaDao, index, alerts); - - // ensure that the alerts were added; exception thrown if not found after a number of retries - for(Map alert: alerts) { - String guid = String.class.cast(alert.get(Constants.GUID)); - findCreatedDoc(guid, docType); - } - } - - @Override - protected void addMetaAlerts(List> metaAlerts) throws Exception { - // Ignore docType for Solr. It's unused. - solr.addAlerts(metaDao, index, metaAlerts); - solr.addMetaAlert(metaDao, metaAlerts); - - // ensure that the alerts were added; exception thrown if not found after a number of retries - for(Map alert: alerts) { - String guid = String.class.cast(alert.get(Constants.GUID)); - findCreatedDoc(guid, docType); + try { + solr.addDocs(index, inputData); + } catch (SolrServerException e) { + throw new IOException("Unable to load Solr Docs", e); } } diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java index fc5192775a..f7c2e86a20 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java @@ -188,7 +188,8 @@ protected GetRequest buildGetRequest(String sensor, int i) { return requestOne; } - protected static void addData(String collection, String sensorName) throws IOException { + protected static void addData(String collection, String sensorName) + throws IOException, SolrServerException { List> inputData = new ArrayList<>(); for (int i = 0; i < 3; ++i) { final String name = buildGuid(sensorName, i); @@ -197,7 +198,7 @@ protected static void addData(String collection, String sensorName) throws IOExc inputMap.put(Constants.GUID, name); inputData.add(inputMap); } - solrComponent.addDocs(dao, collection, inputData); + solrComponent.addDocs(collection, inputData); } protected static String buildGuid(String sensorName, int i) { diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrSearchIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrSearchIntegrationTest.java index a3fc198493..84f222258f 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrSearchIntegrationTest.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrSearchIntegrationTest.java @@ -99,9 +99,9 @@ protected static InMemoryComponent startIndex() throws Exception { @SuppressWarnings("unchecked") protected static void loadTestData() throws ParseException, IOException, SolrServerException { JSONArray broArray = (JSONArray) new JSONParser().parse(broData); - solrComponent.addDocs(dao,"bro", broArray); + solrComponent.addDocs("bro", broArray); JSONArray snortArray = (JSONArray) new JSONParser().parse(snortData); - solrComponent.addDocs(dao,"snort", snortArray); + solrComponent.addDocs("snort", snortArray); } @Override diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java index d2256cfdf4..5b965590c7 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java @@ -128,7 +128,7 @@ private static Map createGlobalConfig() { @Override protected void addTestData(String indexName, String sensorType, List> docs) throws Exception { - solrComponent.addDocs(getDao(), indexName, docs); + solrComponent.addDocs(indexName, docs); } @Override diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java index e92cc780f7..4bc9f8a469 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java @@ -19,23 +19,14 @@ import com.google.common.base.Function; import java.util.Collection; -import java.util.Collections; import java.util.Map.Entry; -import java.util.Optional; import java.util.stream.Collectors; import org.apache.metron.common.Constants; import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants; -import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest; -import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus; -import org.apache.metron.indexing.dao.metaalert.MetaAlertUpdateDao; -import org.apache.metron.indexing.dao.search.GetRequest; -import org.apache.metron.indexing.dao.update.Document; -import org.apache.metron.indexing.dao.update.UpdateDao; import org.apache.metron.integration.InMemoryComponent; import org.apache.metron.integration.UnableToStartException; import org.apache.metron.solr.dao.SolrUtilities; import org.apache.metron.solr.writer.MetronSolrClient; -import org.apache.metron.stellar.common.utils.ConversionUtils; import org.apache.solr.client.solrj.SolrQuery; import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.embedded.JettyConfig; @@ -55,9 +46,6 @@ import java.util.Map; import org.apache.solr.common.SolrInputDocument; import org.apache.zookeeper.KeeperException; -import org.json.simple.JSONObject; -import org.json.simple.parser.JSONParser; -import org.json.simple.parser.ParseException; public class SolrComponent implements InMemoryComponent { @@ -203,80 +191,37 @@ public List> getAllIndexedDocs(String collection) { return docs; } - public void addAlerts(UpdateDao updateDao, String collection, List> messages) throws IOException { - // transform each input message to a Document that can be indexed by the UpdateDao - for(Map msg: messages) { - updateDao.update(createDocument(msg, collection), Optional.of(collection)); - } - - // TODO do we need to wait until the documents are visible? - try { - Thread.sleep(2000); - } catch(Exception e) { - } - -// -// checkUpdateResponse(solr.add(collection, solrInputDocuments)); -// // Make sure to commit so things show up -// checkUpdateResponse(solr.commit(true, true)); - } - - private List getRequest(List> alerts) { - List requests = new ArrayList<>(); - for(Map alert: alerts) { - String guid = String.class.cast(alert.get(Constants.GUID)); - requests.add(new GetRequest(guid, MetaAlertConstants.METAALERT_TYPE)); - } - return requests; - } - - public void addMetaAlert(MetaAlertUpdateDao updateDao, String guid, MetaAlertStatus status, - Optional>> alerts) throws IOException { - - MetaAlertCreateRequest request = new MetaAlertCreateRequest(); - alerts.ifPresent(theAlerts -> request.setAlerts(getRequest(theAlerts))); - - // transform each input message to a Document that can be indexed by the UpdateDao - for(Map msg: messages) { - - - - {{ - setAlerts(new ArrayList() {{ - add(new GetRequest("message_1", SENSOR_NAME)); - add(new GetRequest("message_2", SENSOR_NAME, getTestIndexFullName())); - }}); - setGroups(Collections.singletonList("group")); - }}; - - MetaAlertCreateRequest create = new MetaAlertCreateRequest(); - create.setAlerts(); - updateDao.createMetaAlert(createDocument(msg, collection), Optional.of(collection)); - } - - try { - Thread.sleep(2000); - } catch(Exception e) { - } - // TODO do we need to wait until the documents are visible? -// -// checkUpdateResponse(solr.add(collection, solrInputDocuments)); -// // Make sure to commit so things show up -// checkUpdateResponse(solr.commit(true, true)); - } + public void addDocs(String collection, List> docs) + throws IOException, SolrServerException { + CloudSolrClient solr = miniSolrCloudCluster.getSolrClient(); + solr.setDefaultCollection(collection); + Collection solrInputDocuments = docs.stream().map(doc -> { + SolrInputDocument solrInputDocument = new SolrInputDocument(); + for (Entry entry : doc.entrySet()) { + // If the entry itself is a map, add it as a child document. Handle one level of nesting. + if (entry.getValue() instanceof List && !entry.getKey().equals( + MetaAlertConstants.METAALERT_FIELD)) { + for (Object entryItem : (List)entry.getValue()) { + if (entryItem instanceof Map) { + @SuppressWarnings("unchecked") + Map childDoc = (Map) entryItem; + SolrInputDocument childInputDoc = new SolrInputDocument(); + for (Entry childEntry : childDoc.entrySet()) { + childInputDoc.addField(childEntry.getKey(), childEntry.getValue()); + } + solrInputDocument.addChildDocument(childInputDoc); + } + } + } else { + solrInputDocument.addField(entry.getKey(), entry.getValue()); + } + } + return solrInputDocument; + }).collect(Collectors.toList()); - /** - * Create an indexable Document from a JSON message. - * - * @param message The message that needs indexed. - * @param docType The document type to write. - * @return The {@link Document} that was written. - * @throws IOException - */ - private static Document createDocument(Map message, String docType) throws IOException { - Long timestamp = ConversionUtils.convert(message.get("timestamp"), Long.class); - String guid = (String) message.get("guid"); - return new Document(message, guid, docType, timestamp); + checkUpdateResponse(solr.add(collection, solrInputDocuments)); + // Make sure to commit so things show up + checkUpdateResponse(solr.commit(true, true)); } protected void checkUpdateResponse(UpdateResponse result) throws IOException { From 4a7a8370e61e427ac790d2e21bc43a55ac45636b Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 26 Oct 2018 15:29:09 -0400 Subject: [PATCH 12/41] Resolved integration test differences with Solr --- .../indexing/dao/UpdateIntegrationTest.java | 4 +- .../SolrUpdateIntegrationTest.java | 135 ++++++++++++++++-- 2 files changed, 129 insertions(+), 10 deletions(-) diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java index a1d9c0e720..5590b6fa3d 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java @@ -62,8 +62,8 @@ public abstract class UpdateIntegrationTest { @Multiline protected String commentTwo; - private static final int MAX_RETRIES = 10; - private static final int SLEEP_MS = 500; + protected static final int MAX_RETRIES = 10; + protected static final int SLEEP_MS = 500; protected static final String SENSOR_NAME = "test"; private static final String CF = "p"; diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java index 5b965590c7..3c2e628b7b 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java @@ -17,19 +17,15 @@ */ package org.apache.metron.solr.integration; -import static org.apache.metron.solr.SolrConstants.SOLR_ZOOKEEPER; -import static org.junit.Assert.assertEquals; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; import org.apache.commons.lang3.StringUtils; import org.apache.curator.framework.CuratorFramework; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.ConfigurationsUtils; +import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.common.zookeeper.ZKConfigurationsCache; import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.apache.metron.hbase.mock.MockHTable; @@ -39,17 +35,30 @@ import org.apache.metron.indexing.dao.MultiIndexDao; import org.apache.metron.indexing.dao.UpdateIntegrationTest; import org.apache.metron.indexing.dao.update.Document; +import org.apache.metron.indexing.dao.update.ReplaceRequest; import org.apache.metron.indexing.util.IndexingCacheUtil; import org.apache.metron.solr.dao.SolrDao; import org.apache.metron.solr.integration.components.SolrComponent; import org.junit.After; import org.junit.AfterClass; +import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Optional; + +import static org.apache.metron.solr.SolrConstants.SOLR_ZOOKEEPER; +import static org.junit.Assert.assertEquals; + public class SolrUpdateIntegrationTest extends UpdateIntegrationTest { @Rule public final ExpectedException exception = ExpectedException.none(); @@ -186,4 +195,114 @@ public void testHugeErrorFields() throws Exception { exception.expectMessage("Document contains at least one immense term in field=\"error_hash\""); getDao().update(errorDoc, Optional.of("error")); } + + @Test + @Override + public void test() throws Exception { + List> inputData = new ArrayList<>(); + for(int i = 0; i < 10;++i) { + final String name = "message" + i; + inputData.add( + new HashMap() {{ + put("source.type", SENSOR_NAME); + put("name" , name); + put("timestamp", System.currentTimeMillis()); + put(Constants.GUID, name); + }} + ); + } + addTestData(getIndexName(), SENSOR_NAME, inputData); + List> docs = null; + for(int t = 0;t < MAX_RETRIES;++t, Thread.sleep(SLEEP_MS)) { + docs = getIndexedTestData(getIndexName(), SENSOR_NAME); + if(docs.size() >= 10) { + break; + } + } + Assert.assertEquals(10, docs.size()); + //modify the first message and add a new field + { + Map message0 = new HashMap(inputData.get(0)) {{ + put("new-field", "metron"); + }}; + String guid = "" + message0.get(Constants.GUID); + Document update = getDao().replace(new ReplaceRequest(){{ + setReplacement(message0); + setGuid(guid); + setSensorType(SENSOR_NAME); + setIndex(getIndexName()); + }}, Optional.empty()); + + Assert.assertEquals(message0, update.getDocument()); + Assert.assertEquals(1, getMockHTable().size()); + findUpdatedDoc(message0, guid, SENSOR_NAME); + { + //ensure hbase is up to date + Get g = new Get(HBaseDao.Key.toBytes(new HBaseDao.Key(guid, SENSOR_NAME))); + Result r = getMockHTable().get(g); + NavigableMap columns = r.getFamilyMap(CF.getBytes()); + Assert.assertEquals(1, columns.size()); + Assert.assertEquals(message0 + , JSONUtils.INSTANCE.load(new String(columns.lastEntry().getValue()) + , JSONUtils.MAP_SUPPLIER) + ); + } + { + //ensure ES is up-to-date + long cnt = 0; + for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t, Thread.sleep(SLEEP_MS)) { + docs = getIndexedTestData(getIndexName(), SENSOR_NAME); + cnt = docs + .stream() + .filter(d -> message0.get("new-field").equals(d.get("new-field"))) + .count(); + } + Assert.assertNotEquals("Data store is not updated!", cnt, 0); + } + } + //modify the same message and modify the new field + { + Map message0 = new HashMap(inputData.get(0)) {{ + put("new-field", "metron2"); + }}; + String guid = "" + message0.get(Constants.GUID); + Document update = getDao().replace(new ReplaceRequest(){{ + setReplacement(message0); + setGuid(guid); + setSensorType(SENSOR_NAME); + setIndex(getIndexName()); + }}, Optional.empty()); + Assert.assertEquals(message0, update.getDocument()); + Assert.assertEquals(1, getMockHTable().size()); + Document doc = getDao().getLatest(guid, SENSOR_NAME); + Assert.assertEquals(message0, doc.getDocument()); + findUpdatedDoc(message0, guid, SENSOR_NAME); + { + //ensure hbase is up to date + Get g = new Get(HBaseDao.Key.toBytes(new HBaseDao.Key(guid, SENSOR_NAME))); + Result r = getMockHTable().get(g); + NavigableMap columns = r.getFamilyMap(CF.getBytes()); + Assert.assertEquals(2, columns.size()); + Assert.assertEquals(message0, JSONUtils.INSTANCE.load(new String(columns.lastEntry().getValue()) + , JSONUtils.MAP_SUPPLIER) + ); + Assert.assertNotEquals(message0, JSONUtils.INSTANCE.load(new String(columns.firstEntry().getValue()) + , JSONUtils.MAP_SUPPLIER) + ); + } + { + //ensure ES is up-to-date + long cnt = 0; + for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t,Thread.sleep(SLEEP_MS)) { + docs = getIndexedTestData(getIndexName(), SENSOR_NAME); + cnt = docs + .stream() + .filter(d -> message0.get("new-field").equals(d.get("new-field"))) + .count(); + } + + Assert.assertNotEquals("Data store is not updated!", cnt, 0); + } + } + } } From ebb7ef57c9fb20fb957b86516734fed1c18cda84 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Fri, 26 Oct 2018 17:10:54 -0400 Subject: [PATCH 13/41] Start addressing pr feedback. --- .../client/ElasticsearchClient.java | 69 +++++++++++++++++-- .../config/ElasticsearchClientConfig.java | 3 + .../dao/ElasticsearchColumnMetadataDao.java | 32 ++++----- 3 files changed, 82 insertions(+), 22 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClient.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClient.java index 6778b0e77d..11d60751a3 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClient.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClient.java @@ -39,6 +39,8 @@ /** * Wrapper around the Elasticsearch REST clients. Exposes capabilities of the low and high-level clients. + * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/java-rest-overview.html. Most, if not + * all of use in Metron would be focused through the high-level client. It handles marshaling/unmarshaling. */ public class ElasticsearchClient implements AutoCloseable{ private RestClient lowLevelClient; @@ -49,25 +51,84 @@ public ElasticsearchClient(RestClient lowLevelClient, RestHighLevelClient highLe this.highLevelClient = highLevelClient; } + /** + * Exposes an Elasticsearch low-level client. Prefer the high level client. + */ public RestClient getLowLevelClient() { return lowLevelClient; } + /** + *

+ * Exposes an Elasticsearch high-level client. Prefer to use this client over the low-level client where possible. This client wraps the low-level + * client and exposes some additional sugar on top of the low level methods including marshaling/unmarshaling. + *

+ *

+ * Note, as of 5.6.2 it does NOT support index or cluster management operations. + * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_changing_the_application_8217_s_code.html + *
+ *     Does not provide indices or cluster management APIs. Management operations can be executed by external scripts or using the low-level client. + *

+ *

+ * Current supported ES API's seen here - https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/java-rest-high-supported-apis.html + *

+ * + *
    + *
  • Single document APIs + *
      + *
    • Index API
    • + *
    • Get API
    • + *
    • Delete API
    • + *
    • Update API
    • + *
    + *
  • + *
  • Multi document APIs + *
      + *
    • Bulk API
    • + *
    + *
  • + *
  • Search APIs + *
      + *
    • Search API
    • + *
    • Search Scroll API
    • + *
    • Clear Scroll API
    • + *
    + *
  • + *
  • Miscellaneous APIs + *
      + *
    • Info API
    • + *
    + *
  • + *
+ */ public RestHighLevelClient getHighLevelClient() { return highLevelClient; } + /** + * Included as part of AutoCloseable because Elasticsearch recommends closing the client when not + * being used. + * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_changing_the_client_8217_s_initialization_code.html + * @throws IOException + */ @Override public void close() throws IOException { - if(lowLevelClient != null) { + if (lowLevelClient != null) { lowLevelClient.close(); } } - public void putMapping(String index, String type, String source) throws IOException { - HttpEntity entity = new StringEntity(source); + /** + * https://www.elastic.co/guide/en/elasticsearch/reference/5.6/indices-put-mapping.html + * @param index + * @param mappingType https://www.elastic.co/guide/en/elasticsearch/reference/5.6/mapping.html#mapping-type + * @param mapping + * @throws IOException + */ + public void putMapping(String index, String mappingType, String mapping) throws IOException { + HttpEntity entity = new StringEntity(mapping); Response response = lowLevelClient.performRequest("PUT" - , "/" + index + "/_mapping/" + type + , "/" + index + "/_mapping/" + mappingType , Collections.emptyMap() , entity ); diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientConfig.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientConfig.java index 2067f09e63..b1b7a46c34 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientConfig.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientConfig.java @@ -30,6 +30,9 @@ import org.apache.commons.lang.StringUtils; import org.apache.metron.common.utils.HDFSUtils; +/** + * Access configuration options for the ES client. + */ public class ElasticsearchClientConfig extends AbstractMapDecorator { private static Integer THIRTY_SECONDS_IN_MILLIS = 30_000; diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java index 7c2a9bc451..c85681c650 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java @@ -18,13 +18,7 @@ package org.apache.metron.elasticsearch.dao; -import org.apache.metron.elasticsearch.client.ElasticsearchClient; -import org.apache.metron.elasticsearch.utils.FieldMapping; -import org.apache.metron.elasticsearch.utils.FieldProperties; -import org.apache.metron.indexing.dao.ColumnMetadataDao; -import org.apache.metron.indexing.dao.search.FieldType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import static org.apache.metron.elasticsearch.utils.ElasticsearchUtils.INDEX_NAME_DELIMITER; import java.io.IOException; import java.lang.invoke.MethodHandles; @@ -34,8 +28,13 @@ import java.util.List; import java.util.Map; import java.util.Set; - -import static org.apache.metron.elasticsearch.utils.ElasticsearchUtils.INDEX_NAME_DELIMITER; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; +import org.apache.metron.elasticsearch.utils.FieldMapping; +import org.apache.metron.elasticsearch.utils.FieldProperties; +import org.apache.metron.indexing.dao.ColumnMetadataDao; +import org.apache.metron.indexing.dao.search.FieldType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Responsible for retrieving column-level metadata for Elasticsearch search indices. @@ -59,16 +58,13 @@ public class ElasticsearchColumnMetadataDao implements ColumnMetadataDao { elasticsearchTypeMap = Collections.unmodifiableMap(fieldTypeMap); } - /** - * An Elasticsearch administrative client. - */ - private transient ElasticsearchClient adminClient; + private transient ElasticsearchClient esClient; /** - * @param adminClient The Elasticsearch admin client. + * @param esClient The Elasticsearch client. */ - public ElasticsearchColumnMetadataDao(ElasticsearchClient adminClient) { - this.adminClient = adminClient; + public ElasticsearchColumnMetadataDao(ElasticsearchClient esClient) { + this.esClient = esClient; } @SuppressWarnings("unchecked") @@ -81,7 +77,7 @@ public Map getColumnMetadata(List indices) throws IOE String[] latestIndices = getLatestIndices(indices); if (latestIndices.length > 0) { - Map mappings = adminClient.getMappings(latestIndices); + Map mappings = esClient.getMappings(latestIndices); // for each index for (Map.Entry kv : mappings.entrySet()) { @@ -157,7 +153,7 @@ String[] getLatestIndices(List includeIndices) throws IOException { LOG.debug("Getting latest indices; indices={}", includeIndices); Map latestIndices = new HashMap<>(); - String[] indices = adminClient.getIndices(); + String[] indices = esClient.getIndices(); for (String index : indices) { int prefixEnd = index.indexOf(INDEX_NAME_DELIMITER); From 35811175824415c1e37418bb2efb347e6f7a801a Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Mon, 29 Oct 2018 12:10:05 -0400 Subject: [PATCH 14/41] METRON-1849 Added integration test for UpdateDao.update() and UpdateDao.batchUpdate() --- .../indexing/dao/UpdateIntegrationTest.java | 64 ++++++++++++++++++- 1 file changed, 62 insertions(+), 2 deletions(-) diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java index 5590b6fa3d..fdfa5af7f5 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.NavigableMap; import java.util.Optional; +import java.util.UUID; import java.util.stream.Collectors; import org.adrianwalker.multilinestring.Multiline; import org.apache.commons.collections.MapUtils; @@ -178,6 +179,56 @@ public void test() throws Exception { } } + @Test + public void testUpdate() throws Exception { + // create a document to update + final String guid = UUID.randomUUID().toString(); + final Long timestamp = 1526306463050L; + Document toUpdate = createDocument(guid, timestamp); + + // update the document and validate + Document updated = getDao().update(toUpdate, Optional.of(SENSOR_NAME)); + Assert.assertEquals(toUpdate, updated); + + // ensure the document is updated in the index + Document indexed = findUpdatedDoc(toUpdate.getDocument(), guid, SENSOR_NAME); + Assert.assertEquals(toUpdate, indexed); + } + + @Test + public void testBatchUpdate() throws Exception { + Map> toUpdate = new HashMap<>(); + + // create the first document to update + final String guid1 = UUID.randomUUID().toString(); + final Long timestamp1 = 1526306463050L; + Document document1 = createDocument(guid1, timestamp1); + toUpdate.put(document1, Optional.of(SENSOR_NAME)); + + // create the second document to update + final String guid2 = UUID.randomUUID().toString(); + final Long timestamp2 = 1526306463100L; + Document document2 = createDocument(guid2, timestamp2); + toUpdate.put(document2, Optional.of(SENSOR_NAME)); + + // create the third document to update + final String guid3 = UUID.randomUUID().toString(); + final Long timestamp3 = 1526306463300L; + Document document3 = createDocument(guid3, timestamp3); + toUpdate.put(document3, Optional.of(SENSOR_NAME)); + + // update the documents as a batch and validate + Map> updated = getDao().batchUpdate(toUpdate); + Assert.assertTrue(updated.containsKey(document1)); + Assert.assertTrue(updated.containsKey(document2)); + Assert.assertTrue(updated.containsKey(document3)); + + // ensure the documents were written to the index + Assert.assertEquals(document1, findUpdatedDoc(document1.getDocument(), guid1, SENSOR_NAME)); + Assert.assertEquals(document2, findUpdatedDoc(document2.getDocument(), guid2, SENSOR_NAME)); + Assert.assertEquals(document3, findUpdatedDoc(document3.getDocument(), guid3, SENSOR_NAME)); + } + @Test public void testAddCommentAndPatch() throws Exception { Map fields = new HashMap<>(); @@ -296,12 +347,21 @@ private CommentAddRemoveRequest buildAlertRequest(String guid, String comment, S return request; } - protected void findUpdatedDoc(Map message0, String guid, String sensorType) + protected Document createDocument(String guid, Long timestamp) { + Map message1 = new HashMap<>(); + message1.put(Constants.GUID, guid); + message1.put(Constants.SENSOR_TYPE, SENSOR_NAME); + message1.put(Constants.Fields.TIMESTAMP.getName(), timestamp); + + return new Document(message1, guid, SENSOR_NAME, timestamp); + } + + protected Document findUpdatedDoc(Map message0, String guid, String sensorType) throws InterruptedException, IOException, OriginalNotFoundException { for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) { Document doc = getDao().getLatest(guid, sensorType); if (doc != null && message0.equals(doc.getDocument())) { - return; + return doc; } if (t == MAX_RETRIES -1) { MapUtils.debugPrint(System.out, "Expected", message0); From b7ac957164418d733e6c8fb8aa896186bbc5176f Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Tue, 30 Oct 2018 17:30:08 -0600 Subject: [PATCH 15/41] Addressing review feedback --- .../client/ElasticsearchClient.java | 67 ++++-- .../client/ElasticsearchClientFactory.java | 204 ++++++++++++++++++ .../config/ElasticsearchClientConfig.java | 13 +- .../dao/ElasticsearchColumnMetadataDao.java | 2 +- .../elasticsearch/dao/ElasticsearchDao.java | 6 +- .../utils/ElasticsearchUtils.java | 198 +---------------- .../elasticsearch/utils/FieldMapping.java | 3 + .../elasticsearch/utils/FieldProperties.java | 3 + .../writer/ElasticsearchWriter.java | 3 +- .../ElasticsearchColumnMetadataDaoTest.java | 2 +- 10 files changed, 281 insertions(+), 220 deletions(-) create mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClientFactory.java diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClient.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClient.java index 11d60751a3..d62a7c0bcd 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClient.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClient.java @@ -46,6 +46,12 @@ public class ElasticsearchClient implements AutoCloseable{ private RestClient lowLevelClient; private RestHighLevelClient highLevelClient; + /** + * Instantiate with ElasticsearchClientFactory. + * + * @param lowLevelClient + * @param highLevelClient + */ public ElasticsearchClient(RestClient lowLevelClient, RestHighLevelClient highLevelClient) { this.lowLevelClient = lowLevelClient; this.highLevelClient = highLevelClient; @@ -139,6 +145,9 @@ public void putMapping(String index, String mappingType, String mapping) throws } } + /** + * Gets ALL Elasticsearch indices, or null if status code returned is not OK 200. + */ public String[] getIndices() throws IOException { Response response = lowLevelClient.performRequest("GET", "/_cat/indices"); if(response.getStatusLine().getStatusCode() == 200) { @@ -160,21 +169,13 @@ public String[] getIndices() throws IOException { return null; } - private Map getInnerMap(Map outerMap, String... keys) { - Map ret = outerMap; - if(keys.length == 0) { - return outerMap; - } - for(String key : keys) { - ret = (Map)ret.get(key); - if(ret == null) { - return ret; - } - } - return ret; - } - - public Map getMappings(String[] indices) throws IOException { + /** + * Gets FieldMapping detail for a list of indices. + * + * @param indices get field mapppings for the provided indices + * @return mapping of index name to FieldMapping + */ + public Map getMappingByIndex(String[] indices) throws IOException { Map ret = new HashMap<>(); String indicesCsv = Joiner.on(",").join(indices); Response response = lowLevelClient.performRequest("GET", "/" + indicesCsv + "/_mapping"); @@ -205,4 +206,40 @@ public Map getMappings(String[] indices) throws IOExceptio return ret; } + /** + * Traverses the outer map to retrieve a leaf map by iteratively calling get(key) using the provided keys in order. e.g. + * for an outer map provided as follows: + *
+   * {
+   *   "foo" : {
+   *     "bar" : {
+   *       "baz" : {
+   *         "hello" : "world"
+   *       }
+   *     }
+   *   }
+   * }
+   * 
+ * calling getInnerMap(outerMap, new String[] { "foo", "bar", "baz" }) would return the following: + *
+   * {hello=world}
+   * 
+ * @param outerMap Complex map of nested keys/values + * @param keys ordered list of keys to iterate over to grab a leaf mapping. + * @return leaf node, or innermost matching node from outerMap if no leaf exists + */ + private Map getInnerMap(Map outerMap, String... keys) { + Map ret = outerMap; + if(keys.length == 0) { + return outerMap; + } + for(String key : keys) { + ret = (Map)ret.get(key); + if(ret == null) { + return ret; + } + } + return ret; + } + } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClientFactory.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClientFactory.java new file mode 100644 index 0000000000..452c2b8cdc --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClientFactory.java @@ -0,0 +1,204 @@ +/** + * 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.elasticsearch.client; + +import java.io.IOException; +import java.io.InputStream; +import java.lang.invoke.MethodHandles; +import java.nio.file.Files; +import java.security.KeyManagementException; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.cert.CertificateException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.client.config.RequestConfig.Builder; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; +import org.apache.http.impl.nio.reactor.IOReactorConfig; +import org.apache.http.ssl.SSLContextBuilder; +import org.apache.http.ssl.SSLContexts; +import org.apache.metron.elasticsearch.config.ElasticsearchClientConfig; +import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; +import org.apache.metron.elasticsearch.utils.ElasticsearchUtils.HostnamePort; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Main entry point to create the ES client. + */ +public class ElasticsearchClientFactory { + + private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + /** + * Creates an Elasticsearch client from settings provided via the global config. + * + * @return new client + */ + public static ElasticsearchClient create(Map globalConfig) { + ElasticsearchClientConfig esClientConfig = new ElasticsearchClientConfig( + getEsSettings(globalConfig)); + + HttpHost[] httpHosts = getHttpHosts(globalConfig, esClientConfig.getConnectionScheme()); + RestClientBuilder builder = RestClient.builder(httpHosts); + + RestClientBuilder.RequestConfigCallback reqCallback = reqConfigBuilder -> { + setupConnectionTimeouts(reqConfigBuilder, esClientConfig); + return reqConfigBuilder; + }; + builder.setRequestConfigCallback(reqCallback); + builder.setMaxRetryTimeoutMillis(esClientConfig.getMaxRetryTimeoutMillis()); + + RestClientBuilder.HttpClientConfigCallback httpClientConfigCallback = clientBuilder -> { + setupNumConnectionThreads(clientBuilder, esClientConfig); + setupAuthentication(clientBuilder, esClientConfig); + setupConnectionEncryption(clientBuilder, esClientConfig); + return clientBuilder; + }; + builder.setHttpClientConfigCallback(httpClientConfigCallback); + + RestClient lowLevelClient = builder.build(); + RestHighLevelClient client = new RestHighLevelClient(lowLevelClient); + return new ElasticsearchClient(lowLevelClient, client); + } + + private static Map getEsSettings(Map globalConfig) { + return (Map) globalConfig.getOrDefault("es.client.settings", new HashMap<>()); + } + + private static HttpHost[] getHttpHosts(Map globalConfiguration, String scheme) { + List hps = ElasticsearchUtils.getIps(globalConfiguration); + HttpHost[] httpHosts = new HttpHost[hps.size()]; + int i = 0; + for (HostnamePort hp : hps) { + httpHosts[i++] = new HttpHost(hp.hostname, hp.port, scheme); + } + return httpHosts; + } + + /** + * Modifies request config builder with connection and socket timeouts. + * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_timeouts.html + * + * @param reqConfigBuilder builder to modify + * @param esClientConfig pull timeout settings from this config + */ + private static void setupConnectionTimeouts(Builder reqConfigBuilder, + ElasticsearchClientConfig esClientConfig) { + reqConfigBuilder.setConnectTimeout(esClientConfig.getConnectTimeoutMillis()); + reqConfigBuilder.setSocketTimeout(esClientConfig.getSocketTimeoutMillis()); + } + + /** + * Modifies client builder with setting for num connection threads. Default is ES client default, + * which is 1 to num processors per the documentation. + * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_number_of_threads.html + * + * @param clientBuilder builder to modify + * @param esClientConfig pull num threads property from config + */ + private static void setupNumConnectionThreads(HttpAsyncClientBuilder clientBuilder, + ElasticsearchClientConfig esClientConfig) { + if (esClientConfig.getNumClientConnectionThreads().isPresent()) { + Integer numThreads = esClientConfig.getNumClientConnectionThreads().get(); + LOG.info("Setting number of client connection threads: {}", numThreads); + clientBuilder.setDefaultIOReactorConfig(IOReactorConfig.custom() + .setIoThreadCount(numThreads).build()); + } + } + + /** + * Modifies client builder with settings for authentication with X-Pack. + * Note, we do not expose the ability to disable preemptive authentication. + * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_basic_authentication.html + * + * @param clientBuilder builder to modify + * @param esClientConfig pull credentials property from config + */ + private static void setupAuthentication(HttpAsyncClientBuilder clientBuilder, + ElasticsearchClientConfig esClientConfig) { + Optional> credentials = esClientConfig.getCredentials(); + if (credentials.isPresent()) { + LOG.info( + "Found auth credentials - setting up user/pass authenticated client connection for ES."); + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + UsernamePasswordCredentials upcredentials = new UsernamePasswordCredentials( + credentials.get().getKey(), credentials.get().getValue()); + credentialsProvider.setCredentials(AuthScope.ANY, upcredentials); + clientBuilder.setDefaultCredentialsProvider(credentialsProvider); + } else { + LOG.info( + "Elasticsearch client credentials not provided. Defaulting to non-authenticated client connection."); + } + } + + /** + * Modify client builder with connection encryption details (SSL) if applicable. + * If ssl.enabled=true, sets up SSL connection. If enabled, keystore.path is required. User can + * also optionally set keystore.password and keystore.type. + * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_encrypted_communication.html + * + * @param clientBuilder builder to modify + * @param esClientConfig pull connection encryption details from config + */ + private static void setupConnectionEncryption(HttpAsyncClientBuilder clientBuilder, + ElasticsearchClientConfig esClientConfig) { + if (esClientConfig.isSSLEnabled()) { + LOG.info("Configuring client for SSL connection."); + if (!esClientConfig.getKeyStorePath().isPresent()) { + throw new IllegalStateException("KeyStore path must be provided for SSL connection."); + } + KeyStore truststore; + try { + truststore = KeyStore.getInstance(esClientConfig.getKeyStoreType()); + } catch (KeyStoreException e) { + throw new IllegalStateException( + "Unable to get keystore type '" + esClientConfig.getKeyStoreType() + "'", e); + } + Optional optKeyStorePass = esClientConfig.getKeyStorePassword(); + char[] keyStorePass = optKeyStorePass.map(String::toCharArray).orElse(null); + try (InputStream is = Files.newInputStream(esClientConfig.getKeyStorePath().get())) { + truststore.load(is, keyStorePass); + } catch (IOException | NoSuchAlgorithmException | CertificateException e) { + throw new IllegalStateException( + "Unable to load keystore from path '" + esClientConfig.getKeyStorePath().get() + "'", + e); + } + try { + SSLContextBuilder sslBuilder = SSLContexts.custom().loadTrustMaterial(truststore, null); + clientBuilder.setSSLContext(sslBuilder.build()); + } catch (NoSuchAlgorithmException | KeyStoreException | KeyManagementException e) { + throw new IllegalStateException("Unable to load truststore.", e); + } + } + } + +} diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientConfig.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientConfig.java index b1b7a46c34..6b1796e861 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientConfig.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/config/ElasticsearchClientConfig.java @@ -35,9 +35,9 @@ */ public class ElasticsearchClientConfig extends AbstractMapDecorator { - private static Integer THIRTY_SECONDS_IN_MILLIS = 30_000; - private static Integer ONE_SECONDS_IN_MILLIS = 1_000; - private static String DEFAULT_KEYSTORE_TYPE = "JKS"; + private static final Integer THIRTY_SECONDS_IN_MILLIS = 30_000; + private static final Integer ONE_SECONDS_IN_MILLIS = 1_000; + private static final String DEFAULT_KEYSTORE_TYPE = "JKS"; /** * Initialize config from provided settings Map. @@ -127,6 +127,13 @@ public boolean isSSLEnabled() { return ElasticsearchClientOptions.SSL_ENABLED.getOrDefault(this, Boolean.class, false); } + /** + * http by default, https if ssl is enabled. + */ + public String getConnectionScheme() { + return isSSLEnabled() ? "https" : "http"; + } + /** * @return Number of threads to use for client connection. */ diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java index c85681c650..cb446945fd 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDao.java @@ -77,7 +77,7 @@ public Map getColumnMetadata(List indices) throws IOE String[] latestIndices = getLatestIndices(indices); if (latestIndices.length > 0) { - Map mappings = esClient.getMappings(latestIndices); + Map mappings = esClient.getMappingByIndex(latestIndices); // for each index for (Map.Entry kv : mappings.entrySet()) { diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java index bf6cae221a..210e1ce3fd 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java @@ -22,9 +22,8 @@ import java.util.List; import java.util.Map; import java.util.Optional; - import org.apache.metron.elasticsearch.client.ElasticsearchClient; -import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; +import org.apache.metron.elasticsearch.client.ElasticsearchClientFactory; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.IndexDao; import org.apache.metron.indexing.dao.RetrieveLatestDao; @@ -97,8 +96,7 @@ public void setAccessConfig(AccessConfig accessConfig) { @Override public synchronized void init(AccessConfig config) { if (this.client == null) { - this.client = ElasticsearchUtils - .getClient(config.getGlobalConfigSupplier().get()); + this.client = ElasticsearchClientFactory.create(config.getGlobalConfigSupplier().get()); this.accessConfig = config; this.columnMetadataDao = new ElasticsearchColumnMetadataDao(this.client); this.requestSubmitter = new ElasticsearchRequestSubmitter(this.client); diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java index ab4d566085..47cbd9870f 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java @@ -17,20 +17,11 @@ */ package org.apache.metron.elasticsearch.utils; -import static java.lang.String.format; - import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import java.io.IOException; -import java.io.InputStream; import java.lang.invoke.MethodHandles; -import java.nio.file.Files; -import java.security.KeyManagementException; -import java.security.KeyStore; -import java.security.KeyStoreException; -import java.security.NoSuchAlgorithmException; -import java.security.cert.CertificateException; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; @@ -38,28 +29,12 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Optional; import java.util.stream.Collectors; -import org.apache.commons.lang.StringUtils; -import org.apache.http.HttpHost; -import org.apache.http.auth.AuthScope; -import org.apache.http.auth.UsernamePasswordCredentials; -import org.apache.http.client.CredentialsProvider; -import org.apache.http.client.config.RequestConfig.Builder; -import org.apache.http.impl.client.BasicCredentialsProvider; -import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; -import org.apache.http.impl.nio.reactor.IOReactorConfig; -import org.apache.http.ssl.SSLContextBuilder; -import org.apache.http.ssl.SSLContexts; import org.apache.metron.common.configuration.writer.WriterConfiguration; -import org.apache.metron.elasticsearch.client.ElasticsearchClient; -import org.apache.metron.elasticsearch.config.ElasticsearchClientConfig; import org.apache.metron.indexing.dao.search.SearchResponse; import org.apache.metron.indexing.dao.search.SearchResult; import org.codehaus.jackson.map.ObjectMapper; -import org.elasticsearch.client.RestClient; -import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.xcontent.XContentHelper; @@ -84,10 +59,6 @@ public class ElasticsearchUtils { */ public static final String INDEX_NAME_DELIMITER = "_index"; - public static SimpleDateFormat getIndexFormat(WriterConfiguration configurations) { - return getIndexFormat(configurations.getGlobalConfig()); - } - public static SimpleDateFormat getIndexFormat(Map globalConfig) { String format = (String) globalConfig.get("es.date.format"); return DATE_FORMAT_CACHE.get().computeIfAbsent(format, SimpleDateFormat::new); @@ -108,179 +79,16 @@ public static String getIndexName(String sensorType, String indexPostfix, Writer return indexName; } - /** - * Extracts the base index name from a full index name. - * - * For example, given an index named 'bro_index_2017.01.01.01', the base - * index name is 'bro'. - * - * @param indexName The full index name including delimiter and date postfix. - * @return The base index name. - */ - public static String getBaseIndexName(String indexName) { - - String[] parts = indexName.split(INDEX_NAME_DELIMITER); - if(parts.length < 1 || StringUtils.isEmpty(parts[0])) { - String msg = format("Unexpected index name; index=%s, delimiter=%s", indexName, INDEX_NAME_DELIMITER); - throw new IllegalStateException(msg); - } - - return parts[0]; - } - - /** - * Instantiates an Elasticsearch client - * - * @param globalConfiguration Metron global config - * @return new es client - */ - public static ElasticsearchClient getClient(Map globalConfiguration) { - ElasticsearchClientConfig esClientConfig = new ElasticsearchClientConfig(getEsSettings(globalConfiguration)); - - String scheme = esClientConfig.isSSLEnabled() ? "https" : "http"; - RestClientBuilder builder = getRestClientBuilder(globalConfiguration, scheme); - - RestClientBuilder.RequestConfigCallback reqCallback = reqConfigBuilder -> { - setupConnectionTimeouts(reqConfigBuilder, esClientConfig); - return reqConfigBuilder; - }; - builder.setRequestConfigCallback(reqCallback); - builder.setMaxRetryTimeoutMillis(esClientConfig.getMaxRetryTimeoutMillis()); - - RestClientBuilder.HttpClientConfigCallback httpClientConfigCallback = clientBuilder -> { - setupNumConnectionThreads(clientBuilder, esClientConfig); - setupAuthentication(clientBuilder, esClientConfig); - setupConnectionEncryption(clientBuilder, esClientConfig); - return clientBuilder; - }; - builder.setHttpClientConfigCallback(httpClientConfigCallback); - - RestClient lowLevelClient = builder.build(); - RestHighLevelClient client = new RestHighLevelClient(lowLevelClient); - return new ElasticsearchClient(lowLevelClient, client); - } - - private static Map getEsSettings(Map globalConfig) { - return (Map) globalConfig.getOrDefault("es.client.settings", new HashMap()); - } - - private static RestClientBuilder getRestClientBuilder(Map globalConfiguration, - String scheme) { - List hps = getIps(globalConfiguration); - HttpHost[] posts = new HttpHost[hps.size()]; - int i = 0; - for (HostnamePort hp : hps) { - posts[i++] = new HttpHost(hp.hostname, hp.port, scheme); - } - return RestClient.builder(posts); - } - - /** - * Modifies request config builder with connection and socket timeouts. - * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_timeouts.html - * - * @param reqConfigBuilder builder to modify - * @param esClientConfig pull timeout settings from this config - */ - private static void setupConnectionTimeouts(Builder reqConfigBuilder, - ElasticsearchClientConfig esClientConfig) { - reqConfigBuilder.setConnectTimeout(esClientConfig.getConnectTimeoutMillis()); - reqConfigBuilder.setSocketTimeout(esClientConfig.getSocketTimeoutMillis()); - } - - /** - * Modifies client builder with setting for num connection threads. Default is ES client default, - * which is 1 to num processors per the documentation. - * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_number_of_threads.html - * - * @param clientBuilder builder to modify - * @param esClientConfig pull num threads property from config - */ - private static void setupNumConnectionThreads(HttpAsyncClientBuilder clientBuilder, - ElasticsearchClientConfig esClientConfig) { - if (esClientConfig.getNumClientConnectionThreads().isPresent()) { - Integer numThreads = esClientConfig.getNumClientConnectionThreads().get(); - LOG.info("Setting number of client connection threads: {}", numThreads); - clientBuilder.setDefaultIOReactorConfig(IOReactorConfig.custom() - .setIoThreadCount(numThreads).build()); - } - } - - /** - * Modifies client builder with settings for authentication with X-Pack. - * Note, we do not expose the ability to disable preemptive authentication. - * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_basic_authentication.html - * - * @param clientBuilder builder to modify - * @param esClientConfig pull credentials property from config - */ - private static void setupAuthentication(HttpAsyncClientBuilder clientBuilder, ElasticsearchClientConfig esClientConfig) { - Optional> credentials = esClientConfig.getCredentials(); - if (credentials.isPresent()) { - LOG.info( - "Found auth credentials - setting up user/pass authenticated client connection for ES."); - final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); - UsernamePasswordCredentials upcredentials = new UsernamePasswordCredentials( - credentials.get().getKey(), credentials.get().getValue()); - credentialsProvider.setCredentials(AuthScope.ANY, upcredentials); - clientBuilder.setDefaultCredentialsProvider(credentialsProvider); - } else { - LOG.info( - "Elasticsearch client credentials not provided. Defaulting to non-authenticated client connection."); - } - } - - /** - * Modify client builder with connection encryption details (SSL) if applicable. - * If ssl.enabled=true, sets up SSL connection. If enabled, keystore.path is required. User can - * also optionally set keystore.password and keystore.type. - * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_encrypted_communication.html - * - * @param clientBuilder builder to modify - * @param esClientConfig pull connection encryption details from config - */ - private static void setupConnectionEncryption(HttpAsyncClientBuilder clientBuilder, - ElasticsearchClientConfig esClientConfig) { - if (esClientConfig.isSSLEnabled()) { - LOG.info("Configuring client for SSL connection."); - if (!esClientConfig.getKeyStorePath().isPresent()) { - throw new IllegalStateException("KeyStore path must be provided for SSL connection."); - } - KeyStore truststore; - try { - truststore = KeyStore.getInstance(esClientConfig.getKeyStoreType()); - } catch (KeyStoreException e) { - throw new IllegalStateException( - "Unable to get keystore type '" + esClientConfig.getKeyStoreType() + "'", e); - } - Optional optKeyStorePass = esClientConfig.getKeyStorePassword(); - char[] keyStorePass = optKeyStorePass.map(String::toCharArray).orElse(null); - try (InputStream is = Files.newInputStream(esClientConfig.getKeyStorePath().get())) { - truststore.load(is, keyStorePass); - } catch (IOException | NoSuchAlgorithmException | CertificateException e) { - throw new IllegalStateException( - "Unable to load keystore from path '" + esClientConfig.getKeyStorePath().get() + "'", - e); - } - try { - SSLContextBuilder sslBuilder = SSLContexts.custom().loadTrustMaterial(truststore, null); - clientBuilder.setSSLContext(sslBuilder.build()); - } catch (NoSuchAlgorithmException | KeyStoreException | KeyManagementException e) { - throw new IllegalStateException("Unable to load truststore.", e); - } - } - } - public static class HostnamePort { - String hostname; - Integer port; + public String hostname; + public Integer port; public HostnamePort(String hostname, Integer port) { this.hostname = hostname; this.port = port; } } - protected static List getIps(Map globalConfiguration) { + public static List getIps(Map globalConfiguration) { Object ipObj = globalConfiguration.get("es.ip"); Object portObj = globalConfiguration.get("es.port"); if(ipObj == null) { diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldMapping.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldMapping.java index 101e28812e..15bcb4c70f 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldMapping.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldMapping.java @@ -21,6 +21,9 @@ import java.util.HashMap; +/** + * Typedef that maps Elasticsearch index name to properties. + */ public class FieldMapping extends AbstractMapDecorator{ public FieldMapping() { super(new HashMap()); diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldProperties.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldProperties.java index 82aca42134..d116b40580 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldProperties.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/FieldProperties.java @@ -22,6 +22,9 @@ import java.util.HashMap; import java.util.Map; +/** + * Typedef that maps Elasticsearch field names to types. + */ public class FieldProperties extends AbstractMapDecorator { public FieldProperties() { super(new HashMap<>()); diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java index fa090e5069..112a1cca3c 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java @@ -24,6 +24,7 @@ import org.apache.metron.common.writer.BulkMessageWriter; import org.apache.metron.common.writer.BulkWriterResponse; import org.apache.metron.elasticsearch.client.ElasticsearchClient; +import org.apache.metron.elasticsearch.client.ElasticsearchClientFactory; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.storm.task.TopologyContext; import org.apache.storm.tuple.Tuple; @@ -65,7 +66,7 @@ public class ElasticsearchWriter implements BulkMessageWriter, Seria public void init(Map stormConf, TopologyContext topologyContext, WriterConfiguration configurations) { Map globalConfiguration = configurations.getGlobalConfig(); - client = ElasticsearchUtils.getClient(globalConfiguration); + client = ElasticsearchClientFactory.create(globalConfiguration); dateFormat = ElasticsearchUtils.getIndexFormat(globalConfiguration); } diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDaoTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDaoTest.java index 2652f226ce..c9389c0043 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDaoTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchColumnMetadataDaoTest.java @@ -63,7 +63,7 @@ public String[] getIndices() throws IOException { } @Override - public Map getMappings(String[] indices) throws IOException { + public Map getMappingByIndex(String[] indices) throws IOException { return mappings; } }; From c0faab2aaf04475ffc9e5558018b538fb62a3c7b Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Wed, 31 Oct 2018 11:43:47 -0400 Subject: [PATCH 16/41] METRON-1849 Progress on shared write logic --- .../bulk/BulkDocumentWriter.java | 39 +++++ .../bulk/ElasticsearchDocumentWriter.java | 152 ++++++++++++++++++ .../dao/ElasticsearchUpdateDao.java | 79 +++++---- .../writer/ElasticsearchWriter.java | 114 +++++++------ .../writer/TupleBasedDocument.java | 43 +++++ .../metron/indexing/dao/update/Document.java | 72 +++++---- 6 files changed, 378 insertions(+), 121 deletions(-) create mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java create mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchDocumentWriter.java create mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java new file mode 100644 index 0000000000..aac825d430 --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java @@ -0,0 +1,39 @@ +/** + * 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.elasticsearch.bulk; + +import org.apache.metron.indexing.dao.update.Document; + +import java.util.List; + +public interface BulkDocumentWriter { + + interface SuccessCallback { + void onSuccess(List documents); + } + + interface FailureCallback { + void onFailure(D failedDocument, Throwable cause, String message); + } + + void onSuccess(SuccessCallback onSuccess); + + void onFailure(FailureCallback onFailure); + + void write(List documents); +} diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchDocumentWriter.java new file mode 100644 index 0000000000..c2a77cc841 --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchDocumentWriter.java @@ -0,0 +1,152 @@ +/** + * 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.elasticsearch.bulk; + +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; +import org.apache.metron.indexing.dao.update.Document; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +public class ElasticsearchDocumentWriter implements BulkDocumentWriter { + + private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private SuccessCallback onSuccess; + private FailureCallback onFailure; + private ElasticsearchClient client; + + public ElasticsearchDocumentWriter(ElasticsearchClient client) { + this.client = client; + } + + @Override + public void onSuccess(SuccessCallback onSuccess) { + this.onSuccess = onSuccess; + } + + @Override + public void onFailure(FailureCallback onFailure) { + this.onFailure = onFailure; + } + + @Override + public void write(List documents) { + try { + // create an index request for each document + List requests = createIndexRequests(documents); + + // create one bulk request for all the documents + BulkRequest bulkRequest = new BulkRequest(); + bulkRequest.add(requests); + + // handle the bulk response + BulkResponse bulkResponse = client.getHighLevelClient().bulk(bulkRequest); + List successful = handleBulkResponse(bulkResponse, documents); + + // notify the success callback + onSuccess.onSuccess(successful); + LOG.debug("Wrote document(s) to Elasticsearch; batchSize={}, success={}, failed={}, tookInMillis={}", + documents.size(), successful.size(), documents.size() - successful.size(), bulkResponse.getTookInMillis()); + + } catch(IOException e) { + // failed to submit bulk request; all documents failed + for(Document failed: documents) { + onFailure.onFailure(failed, e, ExceptionUtils.getRootCauseMessage(e)); + } + LOG.error("Failed to submit bulk request; all documents failed", e); + } + } + + /** + * Handles the {@link BulkResponse} received from Elasticsearch. + * + * @param bulkResponse The response received from Elasticsearch. + * @param documents The documents that are being written. + * @return The documents that were successfully written. Failed documents are excluded. + */ + private List handleBulkResponse(BulkResponse bulkResponse, List documents) { + List successful = new ArrayList<>(); + if (bulkResponse.hasFailures()) { + + // interrogate the response to distinguish between those that succeeded and those that failed + Iterator iterator = bulkResponse.iterator(); + while(iterator.hasNext()) { + BulkItemResponse response = iterator.next(); + if(response.isFailed()) { + // request failed + Document failed = documents.get(response.getItemId()); + Exception cause = response.getFailure().getCause(); + String message = response.getFailureMessage(); + onFailure.onFailure(failed, cause, message); + + } else { + // request succeeded + Document success = documents.get(response.getItemId()); + successful.add(success); + } + } + } else { + // all requests succeeded + successful.addAll(documents); + } + + return successful; + } + + /** + * Creates an {@link IndexRequest} for each {@link Document}. + * + * @param documents The list of documents to write. + * @return A list of requests; one for each document. + */ + private List createIndexRequests(List documents) { + List requests = new ArrayList<>(); + + // create a request for each document + for(Document document: documents) { + IndexRequest request = new IndexRequest() + .source(document.getDocument()) + .type(document.getSensorType() + "_doc") + .id(document.getGuid()); + + // the index name may not be defined + document.getIndex().ifPresent(name -> request.index(name)); + + // the timestamp may not be defined + if(document.getTimestamp() != null) { + request.timestamp(document.getTimestamp().toString()); + } + + requests.add(request); + } + + return requests; + } +} diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java index c769b2f56e..aa2507364c 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java @@ -23,12 +23,15 @@ import java.lang.invoke.MethodHandles; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Date; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; +import org.apache.metron.elasticsearch.bulk.BulkDocumentWriter; import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; @@ -51,6 +54,9 @@ public class ElasticsearchUpdateDao implements UpdateDao { private transient ElasticsearchClient client; private AccessConfig accessConfig; private ElasticsearchRetrieveLatestDao retrieveLatestDao; + private BulkDocumentWriter documentWriter; + private int failures; + private Throwable lastException; public ElasticsearchUpdateDao(ElasticsearchClient client, AccessConfig accessConfig, @@ -62,54 +68,45 @@ public ElasticsearchUpdateDao(ElasticsearchClient client, @Override public Document update(Document update, Optional index) throws IOException { - String indexPostfix = ElasticsearchUtils - .getIndexFormat(accessConfig.getGlobalConfigSupplier().get()).format(new Date()); - String sensorType = update.getSensorType(); - String indexName = getIndexName(update, index, indexPostfix); - - IndexRequest indexRequest = buildIndexRequest(update, sensorType, indexName); - try { - IndexResponse response = client.getHighLevelClient().index(indexRequest); - - ShardInfo shardInfo = response.getShardInfo(); - int failed = shardInfo.getFailed(); - if (failed > 0) { - throw new IOException( - "ElasticsearchDao index failed: " + Arrays.toString(shardInfo.getFailures())); - } - } catch (Exception e) { - throw new IOException(e.getMessage(), e); - } - return update; + Map> updates = new HashMap<>(); + updates.put(update, index); + + Map> results = batchUpdate(updates); + return results.keySet().iterator().next(); } @Override public Map> batchUpdate(Map> updates) throws IOException { - String indexPostfix = ElasticsearchUtils - .getIndexFormat(accessConfig.getGlobalConfigSupplier().get()).format(new Date()); - - BulkRequest bulkRequestBuilder = new BulkRequest(); - - // Get the indices we'll actually be using for each Document. - for (Map.Entry> updateEntry : updates.entrySet()) { - Document update = updateEntry.getKey(); - String sensorType = update.getSensorType(); - String indexName = getIndexName(update, updateEntry.getValue(), indexPostfix); - IndexRequest indexRequest = buildIndexRequest( - update, - sensorType, - indexName - ); - - bulkRequestBuilder.add(indexRequest); + Map globalConfig = accessConfig.getGlobalConfigSupplier().get(); + String indexPostfix = ElasticsearchUtils.getIndexFormat(globalConfig).format(new Date()); + + List documents = new ArrayList<>(); + for (Map.Entry> entry : updates.entrySet()) { + Document document = entry.getKey(); + + // set the index name since it is known + String indexName = getIndexName(document, entry.getValue(), indexPostfix); + document.setIndex(Optional.of(indexName)); + + documents.add(document); } - BulkResponse bulkResponse = client.getHighLevelClient().bulk(bulkRequestBuilder); - if (bulkResponse.hasFailures()) { - LOG.error("Bulk Request has failures: {}", bulkResponse.buildFailureMessage()); - throw new IOException( - "ElasticsearchDao upsert failed: " + bulkResponse.buildFailureMessage()); + // track if a failure occurs so that a checked exception can be thrown; cannot throw checked exception in lambda + failures = 0; + lastException = null; + documentWriter.onFailure((document, cause, message) -> { + failures++; + lastException = cause; + LOG.error(message, cause); + }); + + // write the documents. if any document fails, raise an exception + documentWriter.write(documents); + if(failures > 0) { + String msg = String.format("Failed to update all documents; %d of %d update(s) failed", failures, documents.size()); + throw new IOException(msg, lastException); } + return updates; } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java index fa090e5069..5508c997e4 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java @@ -17,20 +17,18 @@ */ package org.apache.metron.elasticsearch.writer; +import com.google.common.collect.Lists; import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.common.field.FieldNameConverter; import org.apache.metron.common.field.FieldNameConverters; import org.apache.metron.common.writer.BulkMessageWriter; import org.apache.metron.common.writer.BulkWriterResponse; +import org.apache.metron.elasticsearch.bulk.BulkDocumentWriter; import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.storm.task.TopologyContext; import org.apache.storm.tuple.Tuple; -import org.elasticsearch.action.bulk.BulkItemResponse; -import org.elasticsearch.action.bulk.BulkRequest; -import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.index.IndexRequest; import org.json.simple.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,10 +36,14 @@ import java.io.Serializable; import java.lang.invoke.MethodHandles; import java.text.SimpleDateFormat; +import java.util.ArrayList; import java.util.Date; -import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static java.lang.String.format; /** * A {@link BulkMessageWriter} that writes messages to Elasticsearch. @@ -55,53 +57,87 @@ public class ElasticsearchWriter implements BulkMessageWriter, Seria */ private transient ElasticsearchClient client; + private BulkDocumentWriter documentWriter; + /** * A simple data formatter used to build the appropriate Elasticsearch index name. */ private SimpleDateFormat dateFormat; - @Override public void init(Map stormConf, TopologyContext topologyContext, WriterConfiguration configurations) { + // TODO define the documentWriter? Or initialize if none? Need to pass in the client? + Map globalConfiguration = configurations.getGlobalConfig(); client = ElasticsearchUtils.getClient(globalConfiguration); dateFormat = ElasticsearchUtils.getIndexFormat(globalConfiguration); } @Override - public BulkWriterResponse write(String sensorType, WriterConfiguration configurations, Iterable tuples, List messages) throws Exception { + public BulkWriterResponse write(String sensorType, + WriterConfiguration configurations, + Iterable tuplesIter, + List messages) { // fetch the field name converter for this sensor type FieldNameConverter fieldNameConverter = FieldNameConverters.create(sensorType, configurations); + String indexPostfix = dateFormat.format(new Date()); + String indexName = ElasticsearchUtils.getIndexName(sensorType, indexPostfix, configurations); + + // the number of tuples must match the number of messages + List tuples = Lists.newArrayList(tuplesIter); + int batchSize = tuples.size(); + if(messages.size() != batchSize) { + throw new IllegalStateException(format("Expect same number of tuples and messages; |tuples|=%d, |messages|=%d", + tuples.size(), messages.size())); + } + LOG.debug("About to write {} document(s) to Elasticsearch", batchSize); - final String indexPostfix = dateFormat.format(new Date()); - BulkRequest bulkRequest = new BulkRequest(); - //BulkRequestBuilder bulkRequest = client.prepareBulk(); - for(JSONObject message: messages) { + // create a document from each message + List documents = new ArrayList<>(); + for(int i=0; i { + List successfulTuples = docs.stream().map(doc -> doc.getTuple()).collect(Collectors.toList()); + response.addAllSuccesses(successfulTuples); + }); + + // add any failed tuples to the response + documentWriter.onFailure((failedDocument, cause, message) -> { + Tuple failedTuple = failedDocument.getTuple(); + response.addError(cause, failedTuple); + }); + + // write the documents + documentWriter.write(documents); + return response; } @Override @@ -109,34 +145,6 @@ public String getName() { return "elasticsearch"; } - protected BulkWriterResponse buildWriteReponse(Iterable tuples, BulkResponse bulkResponse) throws Exception { - // Elasticsearch responses are in the same order as the request, giving us an implicit mapping with Tuples - BulkWriterResponse writerResponse = new BulkWriterResponse(); - if (bulkResponse.hasFailures()) { - Iterator respIter = bulkResponse.iterator(); - Iterator tupleIter = tuples.iterator(); - while (respIter.hasNext() && tupleIter.hasNext()) { - BulkItemResponse item = respIter.next(); - Tuple tuple = tupleIter.next(); - - if (item.isFailed()) { - writerResponse.addError(item.getFailure().getCause(), tuple); - } else { - writerResponse.addSuccess(tuple); - } - - // Should never happen, so fail the entire batch if it does. - if (respIter.hasNext() != tupleIter.hasNext()) { - throw new Exception(bulkResponse.buildFailureMessage()); - } - } - } else { - writerResponse.addAllSuccesses(tuples); - } - - return writerResponse; - } - @Override public void close() throws Exception { client.close(); diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java new file mode 100644 index 0000000000..95f5c2e027 --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java @@ -0,0 +1,43 @@ +/** + * 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.elasticsearch.writer; + +import org.apache.metron.indexing.dao.update.Document; +import org.apache.storm.tuple.Tuple; + +import java.util.Map; +import java.util.Optional; + +public class TupleBasedDocument extends Document { + + private Tuple tuple; + + public TupleBasedDocument(Map document, + String guid, + String sensorType, + Long timestamp, + Tuple tuple, + Optional index) { + super(document, guid, sensorType, timestamp, index); + this.tuple = tuple; + } + + public Tuple getTuple() { + return tuple; + } +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java index 3686b19d23..41562242b8 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java @@ -21,6 +21,10 @@ import java.io.IOException; import java.util.HashMap; import java.util.Map; +import java.util.Optional; + +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; import org.apache.metron.common.utils.JSONUtils; public class Document { @@ -28,6 +32,7 @@ public class Document { Map document; String guid; String sensorType; + Optional index; public Document(Map document, String guid, String sensorType, Long timestamp) { setDocument(document); @@ -36,6 +41,14 @@ public Document(Map document, String guid, String sensorType, Lo setSensorType(sensorType); } + public Document(Map document, String guid, String sensorType, Long timestamp, Optional index) { + setDocument(document); + setGuid(guid); + setTimestamp(timestamp); + setSensorType(sensorType); + setIndex(index); + } + public Document(String document, String guid, String sensorType, Long timestamp) throws IOException { this(convertDoc(document), guid, sensorType, timestamp); } @@ -89,14 +102,12 @@ public void setGuid(String guid) { this.guid = guid; } - @Override - public String toString() { - return "Document{" + - "timestamp=" + timestamp + - ", document=" + document + - ", guid='" + guid + '\'' + - ", sensorType='" + sensorType + '\'' + - '}'; + public Optional getIndex() { + return index; + } + + public void setIndex(Optional index) { + this.index = index; } @Override @@ -104,31 +115,38 @@ public boolean equals(Object o) { if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { + if (!(o instanceof Document)) { return false; } - Document document1 = (Document) o; - - if (timestamp != null ? !timestamp.equals(document1.timestamp) : document1.timestamp != null) { - return false; - } - if (document != null ? !document.equals(document1.document) : document1.document != null) { - return false; - } - if (guid != null ? !guid.equals(document1.guid) : document1.guid != null) { - return false; - } - return sensorType != null ? sensorType.equals(document1.sensorType) - : document1.sensorType == null; + return new EqualsBuilder() + .append(timestamp, document1.timestamp) + .append(document, document1.document) + .append(guid, document1.guid) + .append(sensorType, document1.sensorType) + .append(index, document1.index) + .isEquals(); } @Override public int hashCode() { - int result = timestamp != null ? timestamp.hashCode() : 0; - result = 31 * result + (document != null ? document.hashCode() : 0); - result = 31 * result + (guid != null ? guid.hashCode() : 0); - result = 31 * result + (sensorType != null ? sensorType.hashCode() : 0); - return result; + return new HashCodeBuilder(17, 37) + .append(timestamp) + .append(document) + .append(guid) + .append(sensorType) + .append(index) + .toHashCode(); + } + + @Override + public String toString() { + return "Document{" + + "timestamp=" + timestamp + + ", document=" + document + + ", guid='" + guid + '\'' + + ", sensorType='" + sensorType + '\'' + + ", index=" + index + + '}'; } } From 021810a0edb116c3b9bcf12cb8a77bf4197f24fb Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Wed, 31 Oct 2018 16:51:32 -0400 Subject: [PATCH 17/41] METRON-1849 Progress on tests, still integration tests to fix-up --- ...a => ElasticsearchBulkDocumentWriter.java} | 35 +-- .../dao/ElasticsearchUpdateDao.java | 8 +- .../writer/ElasticsearchWriter.java | 35 ++- .../ElasticsearchBulkDocumentWriterTest.java | 198 +++++++++++++ .../writer/BulkDocumentWriterStub.java | 86 ++++++ .../writer/ElasticsearchWriterTest.java | 264 ++++++++++-------- .../metron/indexing/dao/update/Document.java | 1 + .../src/test/resources/log4j.properties | 0 8 files changed, 480 insertions(+), 147 deletions(-) rename metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/{ElasticsearchDocumentWriter.java => ElasticsearchBulkDocumentWriter.java} (82%) create mode 100644 metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java create mode 100644 metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java create mode 100644 metron-platform/metron-indexing/src/test/resources/log4j.properties diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java similarity index 82% rename from metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchDocumentWriter.java rename to metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java index c2a77cc841..453fb4d9b1 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java @@ -33,27 +33,30 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.Optional; -public class ElasticsearchDocumentWriter implements BulkDocumentWriter { +public class ElasticsearchBulkDocumentWriter implements BulkDocumentWriter { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private SuccessCallback onSuccess; - private FailureCallback onFailure; + private Optional onSuccess; + private Optional onFailure; private ElasticsearchClient client; - public ElasticsearchDocumentWriter(ElasticsearchClient client) { + public ElasticsearchBulkDocumentWriter(ElasticsearchClient client) { this.client = client; + this.onSuccess = Optional.empty(); + this.onFailure = Optional.empty(); } @Override public void onSuccess(SuccessCallback onSuccess) { - this.onSuccess = onSuccess; + this.onSuccess = Optional.of(onSuccess); } @Override public void onFailure(FailureCallback onFailure) { - this.onFailure = onFailure; + this.onFailure = Optional.of(onFailure); } @Override @@ -71,14 +74,14 @@ public void write(List documents) { List successful = handleBulkResponse(bulkResponse, documents); // notify the success callback - onSuccess.onSuccess(successful); - LOG.debug("Wrote document(s) to Elasticsearch; batchSize={}, success={}, failed={}, tookInMillis={}", + onSuccess.ifPresent(callback -> callback.onSuccess(successful)); + LOG.debug("Wrote document(s) to Elasticsearch; batchSize={}, success={}, failed={}, took={} ms", documents.size(), successful.size(), documents.size() - successful.size(), bulkResponse.getTookInMillis()); } catch(IOException e) { // failed to submit bulk request; all documents failed for(Document failed: documents) { - onFailure.onFailure(failed, e, ExceptionUtils.getRootCauseMessage(e)); + onFailure.ifPresent(callback -> callback.onFailure(failed, e, ExceptionUtils.getRootCauseMessage(e))); } LOG.error("Failed to submit bulk request; all documents failed", e); } @@ -104,7 +107,7 @@ private List handleBulkResponse(BulkResponse bulkResponse, List doc Document failed = documents.get(response.getItemId()); Exception cause = response.getFailure().getCause(); String message = response.getFailureMessage(); - onFailure.onFailure(failed, cause, message); + onFailure.ifPresent(callback -> callback.onFailure(failed, cause, message)); } else { // request succeeded @@ -131,19 +134,19 @@ private List createIndexRequests(List documents) { // create a request for each document for(Document document: documents) { + if(document.getTimestamp() == null) { + throw new IllegalArgumentException("Document must contain the timestamp"); + } + IndexRequest request = new IndexRequest() .source(document.getDocument()) .type(document.getSensorType() + "_doc") - .id(document.getGuid()); + .id(document.getGuid()) + .timestamp(document.getTimestamp().toString()); // the index name may not be defined document.getIndex().ifPresent(name -> request.index(name)); - // the timestamp may not be defined - if(document.getTimestamp() != null) { - request.timestamp(document.getTimestamp().toString()); - } - requests.add(request); } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java index aa2507364c..92af5049d4 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java @@ -22,8 +22,6 @@ import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.List; @@ -32,6 +30,7 @@ import java.util.stream.Collectors; import org.apache.metron.elasticsearch.bulk.BulkDocumentWriter; +import org.apache.metron.elasticsearch.bulk.ElasticsearchBulkDocumentWriter; import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; @@ -39,11 +38,7 @@ import org.apache.metron.indexing.dao.update.CommentAddRemoveRequest; import org.apache.metron.indexing.dao.update.Document; import org.apache.metron.indexing.dao.update.UpdateDao; -import org.elasticsearch.action.bulk.BulkRequest; -import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -64,6 +59,7 @@ public ElasticsearchUpdateDao(ElasticsearchClient client, this.client = client; this.accessConfig = accessConfig; this.retrieveLatestDao = searchDao; + this.documentWriter = new ElasticsearchBulkDocumentWriter<>(client); } @Override diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java index 5508c997e4..b81ff7b7e4 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java @@ -25,6 +25,7 @@ import org.apache.metron.common.writer.BulkMessageWriter; import org.apache.metron.common.writer.BulkWriterResponse; import org.apache.metron.elasticsearch.bulk.BulkDocumentWriter; +import org.apache.metron.elasticsearch.bulk.ElasticsearchBulkDocumentWriter; import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.storm.task.TopologyContext; @@ -57,7 +58,14 @@ public class ElasticsearchWriter implements BulkMessageWriter, Seria */ private transient ElasticsearchClient client; - private BulkDocumentWriter documentWriter; + /** + * Responsible for writing documents. + * + *

Uses a {@link TupleBasedDocument} to maintain the relationship + * between a {@link Tuple} and the document created for that tuple. If + * a document cannot be written, the associated tuple needs to be failed. + */ + private transient BulkDocumentWriter documentWriter; /** * A simple data formatter used to build the appropriate Elasticsearch index name. @@ -66,12 +74,14 @@ public class ElasticsearchWriter implements BulkMessageWriter, Seria @Override public void init(Map stormConf, TopologyContext topologyContext, WriterConfiguration configurations) { - - // TODO define the documentWriter? Or initialize if none? Need to pass in the client? - Map globalConfiguration = configurations.getGlobalConfig(); - client = ElasticsearchUtils.getClient(globalConfiguration); dateFormat = ElasticsearchUtils.getIndexFormat(globalConfiguration); + + // only create the document writer, if one not already set. useful for testing + if(documentWriter == null) { + client = ElasticsearchUtils.getClient(globalConfiguration); + documentWriter = new ElasticsearchBulkDocumentWriter<>(client); + } } @Override @@ -92,7 +102,6 @@ public BulkWriterResponse write(String sensorType, throw new IllegalStateException(format("Expect same number of tuples and messages; |tuples|=%d, |messages|=%d", tuples.size(), messages.size())); } - LOG.debug("About to write {} document(s) to Elasticsearch", batchSize); // create a document from each message List documents = new ArrayList<>(); @@ -106,7 +115,7 @@ public BulkWriterResponse write(String sensorType, copyField(k.toString(), message, source, fieldNameConverter); } - // define the unique document id + // define the document id String guid = String.class.cast(source.get(Constants.GUID)); if(guid == null) { throw new IllegalStateException("What to do if no GUID?"); @@ -147,7 +156,9 @@ public String getName() { @Override public void close() throws Exception { - client.close(); + if(client != null) { + client.close(); + } } /** @@ -175,5 +186,13 @@ private void copyField( // copy the field destination.put(destinationFieldName, source.get(sourceFieldName)); } + + /** + * Set the document writer. Primarily used for testing. + * @param documentWriter The {@link BulkDocumentWriter} to use. + */ + public void setDocumentWriter(BulkDocumentWriter documentWriter) { + this.documentWriter = documentWriter; + } } diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java new file mode 100644 index 0000000000..86427907af --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java @@ -0,0 +1,198 @@ +package org.apache.metron.elasticsearch.bulk; + +import org.apache.metron.common.Constants; +import org.apache.metron.elasticsearch.client.ElasticsearchClient; +import org.apache.metron.indexing.dao.update.Document; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.RestHighLevelClient; +import org.json.simple.JSONObject; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.UUID; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class ElasticsearchBulkDocumentWriterTest { + + ElasticsearchBulkDocumentWriter writer; + ElasticsearchClient client; + RestHighLevelClient highLevelClient; + ArgumentCaptor argumentCaptor; + boolean onSuccessCalled; + boolean onFailureCalled; + + @Before + public void setup() { + // initial setup to mock Elasticsearch + highLevelClient = mock(RestHighLevelClient.class); + client = mock(ElasticsearchClient.class); + when(client.getHighLevelClient()).thenReturn(highLevelClient); + + writer = new ElasticsearchBulkDocumentWriter<>(client); + onFailureCalled = false; + onSuccessCalled = false; + } + + @Test + public void testSuccessCallback() throws IOException { + setupElasticsearchToSucceed(); + + // create a document to write + List documents = new ArrayList<>(); + documents.add(document(message())); + + // validate the "on success" callback + writer.onSuccess(successfulDocs -> { + onSuccessCalled = true; + assertEquals(documents, successfulDocs); + }); + + writer.write(documents); + assertTrue(onSuccessCalled); + assertFalse(onFailureCalled); + } + + @Test + public void testSuccessWithNoCallbacks() throws IOException { + setupElasticsearchToSucceed(); + + // create a document to write + List documents = new ArrayList<>(); + documents.add(document(message())); + + // no callbacks defined + writer.write(documents); + assertFalse(onSuccessCalled); + assertFalse(onFailureCalled); + } + + @Test + public void testFailureCallback() throws IOException { + setupElasticsearchToFail(); + + // create a document to write + List documents = new ArrayList<>(); + documents.add(document(message())); + + // validate the "on failure" callback + writer.onFailure((failedDoc, cause, msg) -> { + onFailureCalled = true; + assertEquals(documents.get(0), failedDoc); + }); + + // no callbacks defined + writer.write(documents); + assertFalse(onSuccessCalled); + assertTrue(onFailureCalled); + } + + @Test + public void testFailureWithNoCallbacks() throws IOException { + setupElasticsearchToFail(); + + // create a document to write + List documents = new ArrayList<>(); + documents.add(document(message())); + + // validate the "on failure" callback + writer.write(documents); + assertFalse(onSuccessCalled); + assertFalse(onFailureCalled); + } + + @Test + public void testDocumentWithIndex() throws IOException { + setupElasticsearchToSucceed(); + + // create a document that does not contain a timestamp + final String indexName = "test_index_foo"; + Document document = document(message()); + document.setIndex(Optional.of(indexName)); + + List documents = new ArrayList<>(); + documents.add(document); + + // validate the "on success" callback + writer.onSuccess(successfulDocs -> { + onSuccessCalled = true; + assertEquals(documents, successfulDocs); + }); + + writer.write(documents); + assertTrue(onSuccessCalled); + assertFalse(onFailureCalled); + + // capture the bulk request that is submitted to elasticsearch + argumentCaptor = ArgumentCaptor.forClass(BulkRequest.class); + verify(highLevelClient).bulk(argumentCaptor.capture()); + + // ensure the index name was set on the request + BulkRequest request = argumentCaptor.getValue(); + assertEquals(indexName, request.requests().get(0).index()); + } + + private void setupElasticsearchToFail() throws IOException { + // define the item failure + BulkItemResponse.Failure failure = mock(BulkItemResponse.Failure.class); + when(failure.getCause()).thenReturn(new Exception("test exception")); + when(failure.getMessage()).thenReturn("error message"); + + // define the item level response + BulkItemResponse itemResponse = mock(BulkItemResponse.class); + when(itemResponse.isFailed()).thenReturn(true); + when(itemResponse.getItemId()).thenReturn(0); + when(itemResponse.getFailure()).thenReturn(failure); + List itemsResponses = Collections.singletonList(itemResponse); + + // define the bulk response to indicate failure + BulkResponse response = mock(BulkResponse.class); + when(response.iterator()).thenReturn(itemsResponses.iterator()); + when(response.hasFailures()).thenReturn(true); + + // have the client return the mock response + when(highLevelClient.bulk(any(BulkRequest.class))).thenReturn(response); + } + + private void setupElasticsearchToSucceed() throws IOException { + // define the bulk response to indicate success + BulkResponse response = mock(BulkResponse.class); + when(response.hasFailures()).thenReturn(false); + + // have the client return the mock response + when(highLevelClient.bulk(any(BulkRequest.class))).thenReturn(response); + } + + private Document document(JSONObject message) { + String guid = UUID.randomUUID().toString(); + String sensorType = "bro"; + Long timestamp = System.currentTimeMillis(); + return new Document(message, guid, sensorType, timestamp); + } + + private JSONObject message() { + JSONObject message = new JSONObject(); + message.put(Constants.GUID, UUID.randomUUID().toString()); + message.put(Constants.Fields.TIMESTAMP.getName(), System.currentTimeMillis()); + message.put(Constants.Fields.SRC_ADDR.getName(), "192.168.1.1"); + return message; + } +} diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java new file mode 100644 index 0000000000..bd8e9473d1 --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java @@ -0,0 +1,86 @@ +/* + * 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.elasticsearch.writer; + +import org.apache.metron.elasticsearch.bulk.BulkDocumentWriter; +import org.apache.metron.indexing.dao.update.Document; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +/** + * A {@link BulkDocumentWriter} stub that can pretend that all documents + * have been written successfully, that all documents have failed, or something + * in between those two extremes. + * + * @param The type of {@link Document} to write. + */ +public class BulkDocumentWriterStub implements BulkDocumentWriter { + + private SuccessCallback onSuccess; + private FailureCallback onFailure; + private float probabilityOfSuccess; + private Exception exception; + + public BulkDocumentWriterStub(float probabilityOfSuccess) { + this.probabilityOfSuccess = probabilityOfSuccess; + this.exception = new IllegalStateException("Exception created by a stub for testing"); + } + + @Override + public void onSuccess(SuccessCallback onSuccess) { + this.onSuccess = onSuccess; + } + + @Override + public void onFailure(FailureCallback onFailure) { + this.onFailure = onFailure; + } + + @Override + public void write(List documents) { + Random random = new Random(); + + List successes = new ArrayList<>(); + for(D document: documents) { + boolean success = random.nextFloat() <= probabilityOfSuccess; + if(success) { + successes.add(document); + } else { + // notify on failure + onFailure.onFailure(document, exception, "error"); + } + } + + // notify on success + onSuccess.onSuccess(successes); + } + + /** + * Set the exception that is passed to the failure callback when a message fails to write. + * @param exception The exception passed to the failure callback. + */ + public void setException(Exception exception) { + this.exception = exception; + } + + public Exception getException() { + return exception; + } +} diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java index 6a3638b93c..18afa2ef3d 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java @@ -18,170 +18,200 @@ package org.apache.metron.elasticsearch.writer; -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import org.apache.metron.common.Constants; +import org.apache.metron.common.configuration.writer.WriterConfiguration; +import org.apache.metron.common.writer.BulkWriterResponse; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.tuple.Tuple; +import org.json.simple.JSONObject; +import org.junit.Before; +import org.junit.Test; -import com.google.common.collect.ImmutableList; +import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.List; import java.util.Map; -import org.apache.metron.common.writer.BulkWriterResponse; -import org.apache.storm.tuple.Tuple; -import org.elasticsearch.action.bulk.BulkItemResponse; -import org.elasticsearch.action.bulk.BulkResponse; -import org.junit.Test; +import java.util.UUID; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class ElasticsearchWriterTest { - @Test - public void testSingleSuccesses() throws Exception { - Tuple tuple1 = mock(Tuple.class); - BulkResponse response = mock(BulkResponse.class); - when(response.hasFailures()).thenReturn(false); + Map stormConf; + TopologyContext topologyContext; + WriterConfiguration writerConfiguration; - BulkWriterResponse expected = new BulkWriterResponse(); - expected.addSuccess(tuple1); + @Before + public void setup() { + topologyContext = mock(TopologyContext.class); - ElasticsearchWriter esWriter = new ElasticsearchWriter(); - BulkWriterResponse actual = esWriter.buildWriteReponse(ImmutableList.of(tuple1), response); + writerConfiguration = mock(WriterConfiguration.class); + when(writerConfiguration.getGlobalConfig()).thenReturn(globals()); - assertEquals("Response should have no errors and single success", expected, actual); + stormConf = new HashMap(); } @Test - public void testMultipleSuccesses() throws Exception { - Tuple tuple1 = mock(Tuple.class); - Tuple tuple2 = mock(Tuple.class); - - BulkResponse response = mock(BulkResponse.class); - when(response.hasFailures()).thenReturn(false); + public void testSuccess() { + // create a writer where all writes will be successful + float probabilityOfSuccess = 1.0F; + ElasticsearchWriter esWriter = new ElasticsearchWriter(); + esWriter.setDocumentWriter( new BulkDocumentWriterStub<>(probabilityOfSuccess)); + esWriter.init(stormConf, topologyContext, writerConfiguration); - BulkWriterResponse expected = new BulkWriterResponse(); - expected.addSuccess(tuple1); - expected.addSuccess(tuple2); + // create a tuple and a message associated with that tuple + List tuples = createTuples(1); + List messages = createMessages(1); - ElasticsearchWriter esWriter = new ElasticsearchWriter(); - BulkWriterResponse actual = esWriter.buildWriteReponse(ImmutableList.of(tuple1, tuple2), response); + BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); - assertEquals("Response should have no errors and two successes", expected, actual); + // response should only contain successes + assertFalse(response.hasErrors()); + assertTrue(response.getSuccesses().contains(tuples.get(0))); } @Test - public void testSingleFailure() throws Exception { - Tuple tuple1 = mock(Tuple.class); - - BulkResponse response = mock(BulkResponse.class); - when(response.hasFailures()).thenReturn(true); - - Exception e = new IllegalStateException(); - BulkItemResponse itemResponse = buildBulkItemFailure(e); - when(response.iterator()).thenReturn(ImmutableList.of(itemResponse).iterator()); + public void testSuccesses() { + // create a writer where all writes will be successful + float probabilityOfSuccess = 1.0F; + ElasticsearchWriter esWriter = new ElasticsearchWriter(); + esWriter.setDocumentWriter(new BulkDocumentWriterStub<>(probabilityOfSuccess)); + esWriter.init(stormConf, topologyContext, writerConfiguration); - BulkWriterResponse expected = new BulkWriterResponse(); - expected.addError(e, tuple1); + // create a few tuples and the messages associated with the tuples + List tuples = createTuples(3); + List messages = createMessages(3); - ElasticsearchWriter esWriter = new ElasticsearchWriter(); - BulkWriterResponse actual = esWriter.buildWriteReponse(ImmutableList.of(tuple1), response); + BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); - assertEquals("Response should have one error and zero successes", expected, actual); + // response should only contain successes + assertFalse(response.hasErrors()); + assertTrue(response.getSuccesses().contains(tuples.get(0))); + assertTrue(response.getSuccesses().contains(tuples.get(1))); + assertTrue(response.getSuccesses().contains(tuples.get(2))); } @Test - public void testTwoSameFailure() throws Exception { - Tuple tuple1 = mock(Tuple.class); - Tuple tuple2 = mock(Tuple.class); - - BulkResponse response = mock(BulkResponse.class); - when(response.hasFailures()).thenReturn(true); - - Exception e = new IllegalStateException(); - - BulkItemResponse itemResponse = buildBulkItemFailure(e); - BulkItemResponse itemResponse2 = buildBulkItemFailure(e); - - when(response.iterator()).thenReturn(ImmutableList.of(itemResponse, itemResponse2).iterator()); - - BulkWriterResponse expected = new BulkWriterResponse(); - expected.addError(e, tuple1); - expected.addError(e, tuple2); + public void testFailure() { + // create a writer where all writes will fail + float probabilityOfSuccess = 0.0F; + BulkDocumentWriterStub docWriter = new BulkDocumentWriterStub<>(probabilityOfSuccess); ElasticsearchWriter esWriter = new ElasticsearchWriter(); - BulkWriterResponse actual = esWriter.buildWriteReponse(ImmutableList.of(tuple1, tuple2), response); + esWriter.setDocumentWriter(docWriter); + esWriter.init(stormConf, topologyContext, writerConfiguration); + + // create a tuple and a message associated with that tuple + List tuples = createTuples(1); + List messages = createMessages(1); - assertEquals("Response should have two errors and no successes", expected, actual); + BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); - // Ensure the errors actually get collapsed together - Map> actualErrors = actual.getErrors(); - HashMap> expectedErrors = new HashMap<>(); - expectedErrors.put(e, ImmutableList.of(tuple1, tuple2)); - assertEquals("Errors should have collapsed together", expectedErrors, actualErrors); + // response should only contain failures + assertTrue(response.hasErrors()); + Collection errors = response.getErrors().get(docWriter.getException()); + assertTrue(errors.contains(tuples.get(0))); } @Test - public void testTwoDifferentFailure() throws Exception { - Tuple tuple1 = mock(Tuple.class); - Tuple tuple2 = mock(Tuple.class); - - BulkResponse response = mock(BulkResponse.class); - when(response.hasFailures()).thenReturn(true); - - Exception e = new IllegalStateException("Cause"); - Exception e2 = new IllegalStateException("Different Cause"); - BulkItemResponse itemResponse = buildBulkItemFailure(e); - BulkItemResponse itemResponse2 = buildBulkItemFailure(e2); - - when(response.iterator()).thenReturn(ImmutableList.of(itemResponse, itemResponse2).iterator()); - - BulkWriterResponse expected = new BulkWriterResponse(); - expected.addError(e, tuple1); - expected.addError(e2, tuple2); + public void testFailures() { + // create a writer where all writes will fail + float probabilityOfSuccess = 0.0F; + BulkDocumentWriterStub docWriter = new BulkDocumentWriterStub<>(probabilityOfSuccess); ElasticsearchWriter esWriter = new ElasticsearchWriter(); - BulkWriterResponse actual = esWriter.buildWriteReponse(ImmutableList.of(tuple1, tuple2), response); + esWriter.setDocumentWriter(docWriter); + esWriter.init(stormConf, topologyContext, writerConfiguration); + + // create a few tuples and the messages associated with the tuples + List tuples = createTuples(3); + List messages = createMessages(3); - assertEquals("Response should have two errors and no successes", expected, actual); + BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); - // Ensure the errors did not get collapsed together - Map> actualErrors = actual.getErrors(); - HashMap> expectedErrors = new HashMap<>(); - expectedErrors.put(e, ImmutableList.of(tuple1)); - expectedErrors.put(e2, ImmutableList.of(tuple2)); - assertEquals("Errors should not have collapsed together", expectedErrors, actualErrors); + // response should only contain failures + assertTrue(response.hasErrors()); + Collection errors = response.getErrors().get(docWriter.getException()); + assertTrue(errors.contains(tuples.get(0))); + assertTrue(errors.contains(tuples.get(1))); + assertTrue(errors.contains(tuples.get(2))); } @Test - public void testSuccessAndFailure() throws Exception { - Tuple tuple1 = mock(Tuple.class); - Tuple tuple2 = mock(Tuple.class); + public void testPartialFailures() { + // create a writer where some will fails and some will succeed + float probabilityOfSuccess = 0.5F; + BulkDocumentWriterStub docWriter = new BulkDocumentWriterStub<>(probabilityOfSuccess); - BulkResponse response = mock(BulkResponse.class); - when(response.hasFailures()).thenReturn(true); + ElasticsearchWriter esWriter = new ElasticsearchWriter(); + esWriter.setDocumentWriter(docWriter); + esWriter.init(stormConf, topologyContext, writerConfiguration); + + // create some tuples and the messages associated with the tuples + int count = 100; + List tuples = createTuples(count); + List messages = createMessages(count); + + BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); + + // response should contain some successes and some failures + int successes = response.getSuccesses().size(); + int failures = response.getErrors().get(docWriter.getException()).size(); + assertTrue(response.hasErrors()); + assertTrue(successes > 0); + assertTrue(failures > 0); + assertEquals(count, successes + failures); + } - Exception e = new IllegalStateException("Cause"); - BulkItemResponse itemResponse = buildBulkItemFailure(e); + @Test(expected = IllegalStateException.class) + public void testWhenNumberOfMessagesDoesNotMatchTuples() { + // create a writer where all writes will be successful + float probabilityOfSuccess = 1.0F; + ElasticsearchWriter esWriter = new ElasticsearchWriter(); + esWriter.setDocumentWriter( new BulkDocumentWriterStub<>(probabilityOfSuccess)); + esWriter.init(stormConf, topologyContext, writerConfiguration); - BulkItemResponse itemResponse2 = mock(BulkItemResponse.class); - when(itemResponse2.isFailed()).thenReturn(false); + // there are 5 tuples and only 1 message; there should be 5 messages to match the number of tuples + List tuples = createTuples(5); + List messages = createMessages(1); - when(response.iterator()).thenReturn(ImmutableList.of(itemResponse, itemResponse2).iterator()); + esWriter.write("bro", writerConfiguration, tuples, messages); + fail("expected exception"); + } - BulkWriterResponse expected = new BulkWriterResponse(); - expected.addError(e, tuple1); - expected.addSuccess(tuple2); + private JSONObject message() { + JSONObject message = new JSONObject(); + message.put(Constants.GUID, UUID.randomUUID().toString()); + message.put(Constants.Fields.TIMESTAMP.getName(), System.currentTimeMillis()); + message.put(Constants.Fields.SRC_ADDR.getName(), "192.168.1.1"); + return message; + } - ElasticsearchWriter esWriter = new ElasticsearchWriter(); - BulkWriterResponse actual = esWriter.buildWriteReponse(ImmutableList.of(tuple1, tuple2), response); + private Map globals() { + Map globals = new HashMap<>(); + globals.put("es.date.format", "yyyy.MM.dd.HH"); + return globals; + } - assertEquals("Response should have one error and one success", expected, actual); + private List createTuples(int count) { + List tuples = new ArrayList<>(); + for(int i=0; i createMessages(int count) { + List messages = new ArrayList<>(); + for(int i=0; i document, String guid, String sensorType, Lo setGuid(guid); setTimestamp(timestamp); setSensorType(sensorType); + index = Optional.empty(); } public Document(Map document, String guid, String sensorType, Long timestamp, Optional index) { diff --git a/metron-platform/metron-indexing/src/test/resources/log4j.properties b/metron-platform/metron-indexing/src/test/resources/log4j.properties new file mode 100644 index 0000000000..e69de29bb2 From 056497034f7d0b5ebba7d8a66f1b6ff66ebfd119 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 1 Nov 2018 09:57:06 -0400 Subject: [PATCH 18/41] METRON-1845 Added license header --- .../ElasticsearchBulkDocumentWriterTest.java | 27 ++++++++++++++----- 1 file changed, 20 insertions(+), 7 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java index 86427907af..19da00ee2d 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java @@ -1,3 +1,20 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.metron.elasticsearch.bulk; import org.apache.metron.common.Constants; @@ -6,7 +23,6 @@ import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.client.RestHighLevelClient; import org.json.simple.JSONObject; import org.junit.Before; @@ -22,10 +38,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; @@ -62,8 +75,8 @@ public void testSuccessCallback() throws IOException { // validate the "on success" callback writer.onSuccess(successfulDocs -> { - onSuccessCalled = true; assertEquals(documents, successfulDocs); + onSuccessCalled = true; }); writer.write(documents); @@ -95,8 +108,8 @@ public void testFailureCallback() throws IOException { // validate the "on failure" callback writer.onFailure((failedDoc, cause, msg) -> { - onFailureCalled = true; assertEquals(documents.get(0), failedDoc); + onFailureCalled = true; }); // no callbacks defined @@ -133,8 +146,8 @@ public void testDocumentWithIndex() throws IOException { // validate the "on success" callback writer.onSuccess(successfulDocs -> { - onSuccessCalled = true; assertEquals(documents, successfulDocs); + onSuccessCalled = true; }); writer.write(documents); From f1c8ce4eb5d404f2a1d99e51e3111586221e13a0 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 1 Nov 2018 17:36:56 -0400 Subject: [PATCH 19/41] Most tests working except for a few --- .../dao/ElasticsearchRetrieveLatestDao.java | 22 +- .../ElasticsearchBulkDocumentWriterTest.java | 4 +- .../metron/indexing/dao/update/Document.java | 33 ++- .../indexing/dao/UpdateIntegrationTest.java | 210 ++++++++++-------- 4 files changed, 149 insertions(+), 120 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java index 0c91007943..d6bb7bd22c 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java @@ -26,9 +26,11 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.function.Function; +import org.apache.metron.common.Constants; import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.indexing.dao.RetrieveLatestDao; import org.apache.metron.indexing.dao.search.GetRequest; @@ -71,7 +73,7 @@ public Iterable getAllLatest(List getRequests) throws IOEx String doc = hit.getSourceAsString(); String sourceType = Iterables.getFirst(Splitter.on("_doc").split(hit.getType()), null); try { - return Optional.of(new Document(doc, hit.getId(), sourceType, ts)); + return Optional.of(new Document(doc, hit.getId(), sourceType, ts, Optional.ofNullable(hit.getIndex()))); } catch (IOException e) { throw new IllegalStateException("Unable to retrieve latest: " + e.getMessage(), e); } @@ -133,12 +135,26 @@ List searchByGuids(Collection guids, Collection sensorTyp return results; } + private Optional getTimestamp(Map document) { + Optional timestamp = Optional.empty(); + + if(document != null && document.containsKey(Constants.Fields.TIMESTAMP.getName())) { + Object value = document.get(Constants.Fields.TIMESTAMP.getName()); + if(value instanceof Long) { + timestamp = Optional.of(Long.class.cast(value)); + } + } + + return timestamp; + } + private Optional toDocument(final String guid, SearchHit hit) { - Long ts = 0L; String doc = hit.getSourceAsString(); String sourceType = toSourceType(hit.getType()); try { - return Optional.of(new Document(doc, guid, sourceType, ts)); + Document document = new Document(doc, guid, sourceType, 0L, Optional.ofNullable(hit.getIndex())); + getTimestamp(document.getDocument()).ifPresent(ts -> document.setTimestamp(ts)); + return Optional.of(document); } catch (IOException e) { throw new IllegalStateException("Unable to retrieve latest: " + e.getMessage(), e); } diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java index 19da00ee2d..ec59269f61 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java @@ -126,7 +126,7 @@ public void testFailureWithNoCallbacks() throws IOException { List documents = new ArrayList<>(); documents.add(document(message())); - // validate the "on failure" callback + // no callbacks defined writer.write(documents); assertFalse(onSuccessCalled); assertFalse(onFailureCalled); @@ -136,7 +136,7 @@ public void testFailureWithNoCallbacks() throws IOException { public void testDocumentWithIndex() throws IOException { setupElasticsearchToSucceed(); - // create a document that does not contain a timestamp + // create a document that specifies the index final String indexName = "test_index_foo"; Document document = document(message()); document.setIndex(Optional.of(indexName)); diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java index 2b2fb31dd7..cc9b9711bb 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.Optional; import org.apache.commons.lang3.builder.EqualsBuilder; @@ -50,6 +51,10 @@ public Document(Map document, String guid, String sensorType, Lo setIndex(index); } + public Document(String document, String guid, String sensorType, Long timestamp, Optional index) throws IOException { + this(convertDoc(document), guid, sensorType, timestamp, index); + } + public Document(String document, String guid, String sensorType, Long timestamp) throws IOException { this(convertDoc(document), guid, sensorType, timestamp); } @@ -113,31 +118,19 @@ public void setIndex(Optional index) { @Override public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof Document)) { - return false; - } + if (this == o) return true; + if (!(o instanceof Document)) return false; Document document1 = (Document) o; - return new EqualsBuilder() - .append(timestamp, document1.timestamp) - .append(document, document1.document) - .append(guid, document1.guid) - .append(sensorType, document1.sensorType) - .append(index, document1.index) - .isEquals(); + return Objects.equals(timestamp, document1.timestamp) && + Objects.equals(document, document1.document) && + Objects.equals(guid, document1.guid) && + Objects.equals(sensorType, document1.sensorType) && + Objects.equals(index, document1.index); } @Override public int hashCode() { - return new HashCodeBuilder(17, 37) - .append(timestamp) - .append(document) - .append(guid) - .append(sensorType) - .append(index) - .toHashCode(); + return Objects.hash(timestamp, document, guid, sensorType, index); } @Override diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java index fdfa5af7f5..0e920714c0 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java @@ -38,9 +38,12 @@ import org.apache.metron.indexing.dao.update.OriginalNotFoundException; import org.apache.metron.indexing.dao.update.PatchRequest; import org.apache.metron.indexing.dao.update.ReplaceRequest; +import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; +import static org.hamcrest.CoreMatchers.*; + public abstract class UpdateIntegrationTest { /** @@ -107,7 +110,7 @@ public void test() throws Exception { }}, Optional.empty()); Assert.assertEquals(message0, update.getDocument()); - Assert.assertEquals(10, getMockHTable().size()); +// Assert.assertEquals(10, getMockHTable().size()); findUpdatedDoc(message0, guid, SENSOR_NAME); { //ensure hbase is up to date @@ -185,14 +188,16 @@ public void testUpdate() throws Exception { final String guid = UUID.randomUUID().toString(); final Long timestamp = 1526306463050L; Document toUpdate = createDocument(guid, timestamp); - - // update the document and validate - Document updated = getDao().update(toUpdate, Optional.of(SENSOR_NAME)); - Assert.assertEquals(toUpdate, updated); - - // ensure the document is updated in the index - Document indexed = findUpdatedDoc(toUpdate.getDocument(), guid, SENSOR_NAME); - Assert.assertEquals(toUpdate, indexed); + { + // update the document and validate + Document updated = getDao().update(toUpdate, Optional.of(SENSOR_NAME)); + Assert.assertEquals(toUpdate, updated); + } + { + // ensure the document is updated in the index + Document indexed = findUpdatedDoc(toUpdate.getDocument(), guid, SENSOR_NAME); + Assert.assertEquals(toUpdate, indexed); + } } @Test @@ -219,9 +224,9 @@ public void testBatchUpdate() throws Exception { // update the documents as a batch and validate Map> updated = getDao().batchUpdate(toUpdate); - Assert.assertTrue(updated.containsKey(document1)); - Assert.assertTrue(updated.containsKey(document2)); - Assert.assertTrue(updated.containsKey(document3)); + Assert.assertThat(updated.keySet(), hasItem(document1)); + Assert.assertThat(updated.keySet(), hasItem(document2)); + Assert.assertThat(updated.keySet(), hasItem(document3)); // ensure the documents were written to the index Assert.assertEquals(document1, findUpdatedDoc(document1.getDocument(), guid1, SENSOR_NAME)); @@ -230,98 +235,87 @@ public void testBatchUpdate() throws Exception { } @Test - public void testAddCommentAndPatch() throws Exception { - Map fields = new HashMap<>(); - fields.put("guid", "add_comment"); - fields.put("source.type", SENSOR_NAME); - - Document document = new Document(fields, "add_comment", SENSOR_NAME, 1526306463050L); + public void testAddComment() throws Exception { + Document document = createAndIndexDocument("testAddCommentAndPatch"); + + // comment on the document + String commentText = "New Comment"; + String commentUser = "test_user"; + long commentTimestamp = 152630493050L; + Document withComment = addAlertComment(document.getGuid(), commentText, commentUser, commentTimestamp); { - Document update = getDao().update(document, Optional.of(SENSOR_NAME)); - Assert.assertEquals(document, update); - findUpdatedDoc(document.getDocument(), "add_comment", SENSOR_NAME); + // validate that the comment was made on the returned document + List comments = getComments(withComment); + Assert.assertEquals(1, comments.size()); + Assert.assertEquals(commentText, comments.get(0).getComment()); + Assert.assertEquals(commentUser, comments.get(0).getUsername()); + Assert.assertEquals(commentTimestamp, comments.get(0).getTimestamp()); } - ArrayList comments = new ArrayList<>(); { - Document update = addAlertComment("add_comment", "New Comment", "test_user", 1526306463050L); - // Ensure we have the first comment - comments.add(new AlertComment("New Comment", "test_user", 1526306463050L)); - document.getDocument().put(COMMENTS_FIELD, comments.stream().map(AlertComment::asMap).collect( - Collectors.toList())); - Assert.assertEquals(document, update); - findUpdatedDoc(document.getDocument(), "add_comment", SENSOR_NAME); - } - { - List> patchList = new ArrayList<>(); - Map patch = new HashMap<>(); - patch.put("op", "add"); - patch.put("path", "/project"); - patch.put("value", "metron"); - patchList.add(patch); - - PatchRequest pr = new PatchRequest(); - pr.setGuid("add_comment"); - pr.setIndex(SENSOR_NAME); - pr.setSensorType(SENSOR_NAME); - pr.setPatch(patchList); - Document update = getDao().patch(getDao(), pr, Optional.of(1526306463050L)); - - document.getDocument().put("project", "metron"); - Assert.assertEquals(document, update); - findUpdatedDoc(document.getDocument(), "add_comment", SENSOR_NAME); + // validate that the comment was made on the indexed document + Document indexed = findUpdatedDoc(withComment.getDocument(), withComment.getGuid(), SENSOR_NAME); + List comments = getComments(indexed); + Assert.assertEquals(1, comments.size()); + Assert.assertEquals(commentText, comments.get(0).getComment()); + Assert.assertEquals(commentUser, comments.get(0).getUsername()); + Assert.assertEquals(commentTimestamp, comments.get(0).getTimestamp()); } } @Test - @SuppressWarnings("unchecked") + public void testPatchDocumentThatHasComment() throws Exception { + Document document = createAndIndexDocument("testPatchDocumentWithComment"); + + // comment on the document + String commentText = "New Comment"; + String commentUser = "test_user"; + long commentTimestamp = 152630493050L; + Document withComment = addAlertComment(document.getGuid(), commentText, commentUser, commentTimestamp); + + // create a patch + List> patches = new ArrayList<>(); + Map patch = new HashMap<>(); + patch.put("op", "add"); + patch.put("path", "/project"); + patch.put("value", "metron"); + patches.add(patch); + + PatchRequest pr = new PatchRequest(); + pr.setGuid(withComment.getGuid()); + pr.setIndex(SENSOR_NAME); + pr.setSensorType(SENSOR_NAME); + pr.setPatch(patches); + + // patch the document that has been commented on + Document patched = getDao().patch(getDao(), pr, Optional.of(withComment.getTimestamp())); + Assert.assertEquals("metron", patched.getDocument().get("project")); + + // ensure the patch was made on the indexed document + Document indexed = findUpdatedDoc(patched.getDocument(), patched.getGuid(), SENSOR_NAME); + Assert.assertEquals("metron", indexed.getDocument().get("project")); + } + + @Test public void testRemoveComments() throws Exception { - Map fields = new HashMap<>(); - fields.put("guid", "add_comment"); - fields.put("source.type", SENSOR_NAME); + String guid = "testRemoveComments"; + createAndIndexDocument(guid); - Document document = new Document(fields, "add_comment", SENSOR_NAME, 1526401584951L); - { - Document update = getDao().update(document, Optional.of(SENSOR_NAME)); - Assert.assertEquals(document, update); - findUpdatedDoc(document.getDocument(), "add_comment", SENSOR_NAME); - } - ArrayList comments = new ArrayList<>(); - { - Document update = addAlertComment("add_comment", "New Comment", "test_user", 1526401584951L); - // Ensure we have the first comment - - comments.add(new AlertComment("New Comment", "test_user", 1526401584951L)); - document.getDocument().put(COMMENTS_FIELD, comments.stream().map(AlertComment::asMap).collect( - Collectors.toList())); - Assert.assertEquals(document, update); - findUpdatedDoc(document.getDocument(), "add_comment", SENSOR_NAME); - } - { - Document update = addAlertComment("add_comment", "New Comment 2", "test_user_2", 1526401584952L); - // Ensure we have the second comment - comments.add(new AlertComment("New Comment 2", "test_user_2", 1526401584952L)); - document.getDocument().put(COMMENTS_FIELD, comments.stream().map(AlertComment::asMap).collect( - Collectors.toList())); - Assert.assertEquals(document, update); - findUpdatedDoc(document.getDocument(), "add_comment", SENSOR_NAME); - } - { - Document update = removeAlertComment("add_comment", "New Comment 2", "test_user_2", 1526401584952L); - // Ensure we only have the first comments - comments = new ArrayList<>(); - comments.add(new AlertComment(commentOne)); - document.getDocument().put(COMMENTS_FIELD, comments.stream().map(AlertComment::asMap).collect( - Collectors.toList())); - Assert.assertEquals(document, update); - findUpdatedDoc(document.getDocument(), "add_comment", SENSOR_NAME); - } - { - Document update = removeAlertComment("add_comment", "New Comment", "test_user", 1526401584951L); - // Ensure we have no comments - document.getDocument().remove(COMMENTS_FIELD); - Assert.assertEquals(document, update); - findUpdatedDoc(document.getDocument(), "add_comment", SENSOR_NAME); - } + // add a comment on the document + Document withComments = addAlertComment(guid, "comment", "user1", 1526401584951L); + Assert.assertEquals(1, getComments(withComments).size()); + + // ensure the comment was added to the document in the index + Document indexedWithComments = findUpdatedDoc(withComments.getDocument(), withComments.getGuid(), withComments.getSensorType()); + Assert.assertEquals(1, getComments(indexedWithComments).size()); + + // remove a comment from the document + AlertComment toRemove = getComments(withComments).get(0); + Document noComments = removeAlertComment(guid, toRemove.getComment(), toRemove.getUsername(), toRemove.getTimestamp()); + Assert.assertEquals(0, getComments(noComments).size()); + + // ensure the comment was removed from the index + Document indexedNoComments = findUpdatedDoc(withComments.getDocument(), withComments.getGuid(), withComments.getSensorType()); + Assert.assertEquals(1, getComments(indexedNoComments).size()); } protected Document addAlertComment(String guid, String comment, String username, long timestamp) @@ -347,6 +341,19 @@ private CommentAddRemoveRequest buildAlertRequest(String guid, String comment, S return request; } + private Document createAndIndexDocument(String guid) throws Exception { + // create the document + Long timestamp = 1526306463050L; + Document toCreate = createDocument(guid, timestamp); + + // index the document + Document created = getDao().update(toCreate, Optional.of(SENSOR_NAME)); + Assert.assertEquals(toCreate, created); + + // ensure the document is indexed + return findUpdatedDoc(toCreate.getDocument(), guid, SENSOR_NAME); + } + protected Document createDocument(String guid, Long timestamp) { Map message1 = new HashMap<>(); message1.put(Constants.GUID, guid); @@ -356,6 +363,19 @@ protected Document createDocument(String guid, Long timestamp) { return new Document(message1, guid, SENSOR_NAME, timestamp); } + private List getComments(Document withComment) { + List> commentsField = List.class.cast(withComment.getDocument().get(COMMENTS_FIELD)); + List comments = new ArrayList<>(); + if(commentsField != null) { + comments = commentsField + .stream() + .map(map -> new AlertComment(map)) + .collect(Collectors.toList()); + } + + return comments; + } + protected Document findUpdatedDoc(Map message0, String guid, String sensorType) throws InterruptedException, IOException, OriginalNotFoundException { for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) { From 3e5c12169e8cea5da5349cf699b5519cdc82bca1 Mon Sep 17 00:00:00 2001 From: Michael Miklavcic Date: Thu, 1 Nov 2018 22:56:55 -0600 Subject: [PATCH 20/41] Refactor clientfactory. Updates to metron-common, metron-elasticsearch, and kerberos/security documentation --- metron-deployment/Kerberos-manual-setup.md | 154 +----------------- metron-platform/metron-common/README.md | 48 +++--- .../src/main/config/zookeeper/global.json | 1 - .../metron-elasticsearch/README.md | 45 ++++- .../client/ElasticsearchClientFactory.java | 119 ++++++-------- 5 files changed, 125 insertions(+), 242 deletions(-) diff --git a/metron-deployment/Kerberos-manual-setup.md b/metron-deployment/Kerberos-manual-setup.md index 4bd4516643..6d40552f68 100644 --- a/metron-deployment/Kerberos-manual-setup.md +++ b/metron-deployment/Kerberos-manual-setup.md @@ -563,17 +563,16 @@ X-Pack sudo -u hdfs hdfs dfs -chown metron:metron /apps/metron/elasticsearch/xpack-password ``` -1. New settings have been added to configure the Elasticsearch client. By default the client will run as the normal ES prebuilt transport client. If you enable X-Pack you should set the es.client.class as shown below. +1. New settings have been added to configure the Elasticsearch client. - Add the `es.client.settings` to global.json + Modify the `es.client.settings` key in global.json ``` $METRON_HOME/config/zookeeper/global.json -> "es.client.settings" : { - "es.client.class" : "org.elasticsearch.xpack.client.PreBuiltXPackTransportClient", - "es.xpack.username" : "transport_client_user", - "es.xpack.password.file" : "/apps/metron/elasticsearch/xpack-password" + "xpack.username" : "transport_client_user", + "xpack.password.file" : "/apps/metron/elasticsearch/xpack-password" } ``` @@ -583,151 +582,6 @@ X-Pack $METRON_HOME/bin/zk_load_configs.sh -m PUSH -i $METRON_HOME/config/zookeeper/ -z $ZOOKEEPER ``` -1. The last step before restarting the topology is to create a custom X-Pack shaded and relocated jar. This is up to you because of licensing restrictions, but here is a sample Maven pom file that should help. - - ``` - - - - 4.0.0 - org.elasticsearch - elasticsearch-xpack-shaded - elasticsearch-xpack-shaded - jar - 5.6.2 - - - elasticsearch-releases - https://artifacts.elastic.co/maven - - true - - - false - - - - - - org.elasticsearch.client - x-pack-transport - 5.6.2 - - - com.fasterxml.jackson.dataformat - jackson-dataformat-yaml - - - com.fasterxml.jackson.dataformat - jackson-dataformat-cbor - - - com.fasterxml.jackson.core - jackson-core - - - org.slf4j - slf4j-api - - - org.slf4j - slf4j-log4j12 - - - log4j - log4j - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - 3.2.0 - - true - - - - package - - shade - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - io.netty - org.apache.metron.io.netty - - - org.apache.logging.log4j - org.apache.metron.logging.log4j - - - - - org.slf4j.impl* - org.slf4j:slf4j-log4j* - - - - - - .yaml - LICENSE.txt - ASL2.0 - NOTICE.txt - - - - - - - - - - - - - - - ``` - -1. Once you've built the `elasticsearch-xpack-shaded-5.6.2.jar`, it needs to be made available to Storm when you submit the topology. Create a contrib directory for indexing and put the jar file in this directory. - - ``` - mkdir $METRON_HOME/indexing_contrib - cp elasticsearch-xpack-shaded-5.6.2.jar $METRON_HOME/indexing_contrib/elasticsearch-xpack-shaded-5.6.2.jar - ``` - 1. Now you can restart the Elasticsearch topology. Note, you should perform this step manually, as follows. ``` diff --git a/metron-platform/metron-common/README.md b/metron-platform/metron-common/README.md index dac1974fe7..d68259a5f2 100644 --- a/metron-platform/metron-common/README.md +++ b/metron-platform/metron-common/README.md @@ -79,29 +79,30 @@ This configuration is stored in zookeeper, but looks something like Various parts of our stack uses the global config are documented throughout the Metron documentation, but a convenient index is provided here: -| Property Name | Subsystem | Type | Ambari Property | -|---------------------------------------------------------------------------------------------------------------------|---------------|------------|----------------------------| -| [`es.clustername`](../metron-elasticsearch#esclustername) | Indexing | String | `es_cluster_name` | -| [`es.ip`](../metron-elasticsearch#esip) | Indexing | String | `es_hosts` | -| [`es.port`](../metron-elasticsearch#esport) | Indexing | String | `es_port` | -| [`es.date.format`](../metron-elasticsearch#esdateformat) | Indexing | String | `es_date_format` | -| [`fieldValidations`](#validation-framework) | Parsing | Object | N/A | -| [`parser.error.topic`](../metron-parsers#parsererrortopic) | Parsing | String | N/A | -| [`stellar.function.paths`](../../metron-stellar/stellar-common#stellarfunctionpaths) | Stellar | CSV String | N/A | -| [`stellar.function.resolver.includes`](../../metron-stellar/stellar-common#stellarfunctionresolverincludesexcludes) | Stellar | CSV String | N/A | -| [`stellar.function.resolver.excludes`](../../metron-stellar/stellar-common#stellarfunctionresolverincludesexcludes) | Stellar | CSV String | N/A | -| [`profiler.period.duration`](../../metron-analytics/metron-profiler#profilerperiodduration) | Profiler | Integer | `profiler_period_duration` | -| [`profiler.period.duration.units`](../../metron-analytics/metron-profiler#profilerperioddurationunits) | Profiler | String | `profiler_period_units` | -| [`profiler.writer.batchSize`](../../metron-analytics/metron-profiler/#profilerwriterbatchsize) | Profiler | Integer | N/A | -| [`profiler.writer.batchTimeout`](../../metron-analytics/metron-profiler/#profilerwriterbatchtimeout) | Profiler | Integer | N/A | -| [`update.hbase.table`](../metron-indexing#updatehbasetable) | REST/Indexing | String | `update_hbase_table` | -| [`update.hbase.cf`](../metron-indexing#updatehbasecf) | REST/Indexing | String | `update_hbase_cf` | -| [`geo.hdfs.file`](../metron-enrichment#geohdfsfile) | Enrichment | String | `geo_hdfs_file` | -| [`enrichment.writer.batchSize`](../metron-enrichment#enrichmentwriterbatchsize) | Enrichment | Integer | N/A | -| [`enrichment.writer.batchTimeout`](../metron-enrichment#enrichmentwriterbatchtimeout) | Enrichment | Integer | N/A | -| [`geo.hdfs.file`](../metron-enrichment#geohdfsfile) | Enrichment | String | `geo_hdfs_file` | -| [`source.type.field`](../../metron-interface/metron-alerts#sourcetypefield) | UI | String | `source_type_field` | -| [`threat.triage.score.field`](../../metron-interface/metron-alerts#threattriagescorefield) | UI | String | `threat_triage_score_field` | +| Property Name | Subsystem | Type | Ambari Property | +|---------------------------------------------------------------------------------------------------------------------|---------------|------------|------------------------------| +| [`es.clustername`](../metron-elasticsearch#esclustername) | Indexing | String | `es_cluster_name` | +| [`es.ip`](../metron-elasticsearch#esip) | Indexing | String | `es_hosts` | +| [`es.port`](../metron-elasticsearch#esport) | Indexing | String | `es_port` | +| [`es.date.format`](../metron-elasticsearch#esdateformat) | Indexing | String | `es_date_format` | +| [`es.client.settings`](../metron-elasticsearch#esclientsettings) | Indexing | Object | N/A | +| [`fieldValidations`](#validation-framework) | Parsing | Object | N/A | +| [`parser.error.topic`](../metron-parsers#parsererrortopic) | Parsing | String | N/A | +| [`stellar.function.paths`](../../metron-stellar/stellar-common#stellarfunctionpaths) | Stellar | CSV String | N/A | +| [`stellar.function.resolver.includes`](../../metron-stellar/stellar-common#stellarfunctionresolverincludesexcludes) | Stellar | CSV String | N/A | +| [`stellar.function.resolver.excludes`](../../metron-stellar/stellar-common#stellarfunctionresolverincludesexcludes) | Stellar | CSV String | N/A | +| [`profiler.period.duration`](../../metron-analytics/metron-profiler#profilerperiodduration) | Profiler | Integer | `profiler_period_duration` | +| [`profiler.period.duration.units`](../../metron-analytics/metron-profiler#profilerperioddurationunits) | Profiler | String | `profiler_period_units` | +| [`profiler.writer.batchSize`](../../metron-analytics/metron-profiler/#profilerwriterbatchsize) | Profiler | Integer | N/A | +| [`profiler.writer.batchTimeout`](../../metron-analytics/metron-profiler/#profilerwriterbatchtimeout) | Profiler | Integer | N/A | +| [`update.hbase.table`](../metron-indexing#updatehbasetable) | REST/Indexing | String | `update_hbase_table` | +| [`update.hbase.cf`](../metron-indexing#updatehbasecf) | REST/Indexing | String | `update_hbase_cf` | +| [`geo.hdfs.file`](../metron-enrichment#geohdfsfile) | Enrichment | String | `geo_hdfs_file` | +| [`enrichment.writer.batchSize`](../metron-enrichment#enrichmentwriterbatchsize) | Enrichment | Integer | N/A | +| [`enrichment.writer.batchTimeout`](../metron-enrichment#enrichmentwriterbatchtimeout) | Enrichment | Integer | N/A | +| [`geo.hdfs.file`](../metron-enrichment#geohdfsfile) | Enrichment | String | `geo_hdfs_file` | +| [`source.type.field`](../../metron-interface/metron-alerts#sourcetypefield) | UI | String | `source_type_field` | +| [`threat.triage.score.field`](../../metron-interface/metron-alerts#threattriagescorefield) | UI | String | `threat_triage_score_field` | ## Note Configs in Ambari If a field is managed via ambari, you should change the field via @@ -439,3 +440,4 @@ Options: -p DIRECTORY, --hdp_home=DIRECTORY HDP home directory ``` +` diff --git a/metron-platform/metron-common/src/main/config/zookeeper/global.json b/metron-platform/metron-common/src/main/config/zookeeper/global.json index 9e5402efb4..b638ca3bb4 100644 --- a/metron-platform/metron-common/src/main/config/zookeeper/global.json +++ b/metron-platform/metron-common/src/main/config/zookeeper/global.json @@ -6,6 +6,5 @@ "update.hbase.table": "metron_update", "update.hbase.cf": "t", "es.client.settings": { - "client.transport.ping_timeout": "500s" } } diff --git a/metron-platform/metron-elasticsearch/README.md b/metron-platform/metron-elasticsearch/README.md index 177412e5fa..463a0b85cb 100644 --- a/metron-platform/metron-elasticsearch/README.md +++ b/metron-platform/metron-elasticsearch/README.md @@ -59,6 +59,49 @@ For instance, an `es.date.format` of `yyyy.MM.dd.HH` would have the consequence roll hourly, whereas an `es.date.format` of `yyyy.MM.dd` would have the consequence that the indices would roll daily. +### `es.client.settings` + +This field in global config allows you to specify Elasticsearch REST client options. These are used in conjunction with the previously mentioned Elasticsearch properties +when setting up client connections to an Elasticsearch cluster. The available properties should be supplied as an object map. Current available options are as follows: + +| Property Name | Type | Required? | Default Value | Description | +|-------------------------------------|-----------|-----------|----------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| connection.timeout.millis | Integer | No | 1000 | Sets connection timeout. | +| socket.timeout.millis | Integer | No | 30000 | Sets socket timeout. | +| max.retry.timeout.millis | Integer | No | 30000 | Sets the maximum timeout (in milliseconds) to honour in case of multiple retries of the same request. | +| num.client.connection.threads | Integer | No | 1 | Number of worker threads used by the connection manager. Defaults to Runtime.getRuntime().availableProcessors(). | +| xpack.username | String | No | null | X-Pack username. | +| xpack.password.file | String | No | null | 1-line HDFS file where the X-Pack password is set. | +| ssl.enabled | Boolean | No | false | Turn on SSL connections. | +| keystore.type | String | No | "jks" | Allows you to specify a keytstore type. See https://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#KeyStore for more details. | +| keystore.path | String | No | null | Path to the Trust Store that holds your Elasticsearch certificate authorities and certificate. | +| keystore.password.file | String | No | null | 1-line HDFS file where the keystore password is set. | + +__Note:__ The migration from Elasticsearch's TransportClient to the Java REST client has resulted in some existing properties to change. Below is a mapping of the old properties to the new ones: + +| Old Property Name | New Property Name | +|----------------------------------------|-------------------------------------| +| client.transport.ping_timeout | n/a | +| n/a | connection.timeout.millis | +| n/a | socket.timeout.millis | +| n/a | max.retry.timeout.millis | +| n/a | num.client.connection.threads | +| es.client.class | n/a | +| es.xpack.username | xpack.username | +| es.xpack.password.file | xpack.password.file | +| xpack.security.transport.ssl.enabled | ssl.enabled | +| xpack.ssl.key | n/a | +| xpack.ssl.certificate | n/a | +| xpack.ssl.certificate_authorities | n/a | +| n/a | keystore.type | +| keystore.path | keystore.path | +| n/a | keystore.password.file | + +__Notes:__ +* The transport client implementation provides for a 'xpack.security.user' property, however we never used this property directly. Rather, in order to secure the password we used custom properties for user/pass. These properties have been carried over as `xpack.username` and `xpack.password.file`. +* See [https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_common_configuration.html](https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_common_configuration.html) for more specifics on the new client properties. +* Other notes on JSSE - [https://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/JSSERefGuide.html](https://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/JSSERefGuide.html) + ## Upgrading to 5.6.2 Users should be prepared to re-index when migrating from Elasticsearch 2.3.3 to 5.6.2. There are a number of template changes, most notably around @@ -287,7 +330,7 @@ The following is a list of properties that need to be defined along with their t ## Using Metron with Elasticsearch 5.6.2 -Although infrequent sometimes an internal field is added in Metron and existing templates must be updated. The following steps outlines how to do this, using `metron_alert` as an example. +Although infrequent, sometimes an internal field is added in Metron and existing templates must be updated. The following steps outlines how to do this, using `metron_alert` as an example. With the addition of the meta alert feature, there is a requirement that all sensors templates have a nested `metron_alert` field defined. This field is a dummy field. See [Ignoring Unmapped Fields](https://www.elastic.co/guide/en/elasticsearch/reference/current/search-request-sort.html#_ignoring_unmapped_fields) for more information diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClientFactory.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClientFactory.java index 452c2b8cdc..4e0b2fe142 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClientFactory.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/client/ElasticsearchClientFactory.java @@ -22,6 +22,7 @@ import java.io.InputStream; import java.lang.invoke.MethodHandles; import java.nio.file.Files; +import java.nio.file.Path; import java.security.KeyManagementException; import java.security.KeyStore; import java.security.KeyStoreException; @@ -32,13 +33,12 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; +import javax.net.ssl.SSLContext; import org.apache.http.HttpHost; import org.apache.http.auth.AuthScope; import org.apache.http.auth.UsernamePasswordCredentials; import org.apache.http.client.CredentialsProvider; -import org.apache.http.client.config.RequestConfig.Builder; import org.apache.http.impl.client.BasicCredentialsProvider; -import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; import org.apache.http.impl.nio.reactor.IOReactorConfig; import org.apache.http.ssl.SSLContextBuilder; import org.apache.http.ssl.SSLContexts; @@ -57,6 +57,7 @@ public class ElasticsearchClientFactory { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + private static final String ES_SETTINGS_KEY = "es.client.settings"; // es config key in global config /** * Creates an Elasticsearch client from settings provided via the global config. @@ -66,24 +67,24 @@ public class ElasticsearchClientFactory { public static ElasticsearchClient create(Map globalConfig) { ElasticsearchClientConfig esClientConfig = new ElasticsearchClientConfig( getEsSettings(globalConfig)); - HttpHost[] httpHosts = getHttpHosts(globalConfig, esClientConfig.getConnectionScheme()); RestClientBuilder builder = RestClient.builder(httpHosts); - RestClientBuilder.RequestConfigCallback reqCallback = reqConfigBuilder -> { - setupConnectionTimeouts(reqConfigBuilder, esClientConfig); + builder.setRequestConfigCallback(reqConfigBuilder -> { + // Modifies request config builder with connection and socket timeouts. + // https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_timeouts.html + reqConfigBuilder.setConnectTimeout(esClientConfig.getConnectTimeoutMillis()); + reqConfigBuilder.setSocketTimeout(esClientConfig.getSocketTimeoutMillis()); return reqConfigBuilder; - }; - builder.setRequestConfigCallback(reqCallback); + }); builder.setMaxRetryTimeoutMillis(esClientConfig.getMaxRetryTimeoutMillis()); - RestClientBuilder.HttpClientConfigCallback httpClientConfigCallback = clientBuilder -> { - setupNumConnectionThreads(clientBuilder, esClientConfig); - setupAuthentication(clientBuilder, esClientConfig); - setupConnectionEncryption(clientBuilder, esClientConfig); + builder.setHttpClientConfigCallback(clientBuilder -> { + clientBuilder.setDefaultIOReactorConfig(getIOReactorConfig(esClientConfig)); + clientBuilder.setDefaultCredentialsProvider(getCredentialsProvider(esClientConfig)); + clientBuilder.setSSLContext(getSSLContext(esClientConfig)); return clientBuilder; - }; - builder.setHttpClientConfigCallback(httpClientConfigCallback); + }); RestClient lowLevelClient = builder.build(); RestHighLevelClient client = new RestHighLevelClient(lowLevelClient); @@ -91,7 +92,7 @@ public static ElasticsearchClient create(Map globalConfig) { } private static Map getEsSettings(Map globalConfig) { - return (Map) globalConfig.getOrDefault("es.client.settings", new HashMap<>()); + return (Map) globalConfig.getOrDefault(ES_SETTINGS_KEY, new HashMap<>()); } private static HttpHost[] getHttpHosts(Map globalConfiguration, String scheme) { @@ -105,45 +106,21 @@ private static HttpHost[] getHttpHosts(Map globalConfiguration, } /** - * Modifies request config builder with connection and socket timeouts. - * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_timeouts.html - * - * @param reqConfigBuilder builder to modify - * @param esClientConfig pull timeout settings from this config - */ - private static void setupConnectionTimeouts(Builder reqConfigBuilder, - ElasticsearchClientConfig esClientConfig) { - reqConfigBuilder.setConnectTimeout(esClientConfig.getConnectTimeoutMillis()); - reqConfigBuilder.setSocketTimeout(esClientConfig.getSocketTimeoutMillis()); - } - - /** - * Modifies client builder with setting for num connection threads. Default is ES client default, + * Creates config with setting for num connection threads. Default is ES client default, * which is 1 to num processors per the documentation. * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_number_of_threads.html - * - * @param clientBuilder builder to modify - * @param esClientConfig pull num threads property from config */ - private static void setupNumConnectionThreads(HttpAsyncClientBuilder clientBuilder, - ElasticsearchClientConfig esClientConfig) { + private static IOReactorConfig getIOReactorConfig(ElasticsearchClientConfig esClientConfig) { if (esClientConfig.getNumClientConnectionThreads().isPresent()) { Integer numThreads = esClientConfig.getNumClientConnectionThreads().get(); LOG.info("Setting number of client connection threads: {}", numThreads); - clientBuilder.setDefaultIOReactorConfig(IOReactorConfig.custom() - .setIoThreadCount(numThreads).build()); + return IOReactorConfig.custom().setIoThreadCount(numThreads).build(); + } else { + return IOReactorConfig.DEFAULT; } } - /** - * Modifies client builder with settings for authentication with X-Pack. - * Note, we do not expose the ability to disable preemptive authentication. - * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_basic_authentication.html - * - * @param clientBuilder builder to modify - * @param esClientConfig pull credentials property from config - */ - private static void setupAuthentication(HttpAsyncClientBuilder clientBuilder, + private static CredentialsProvider getCredentialsProvider( ElasticsearchClientConfig esClientConfig) { Optional> credentials = esClientConfig.getCredentials(); if (credentials.isPresent()) { @@ -153,52 +130,60 @@ private static void setupAuthentication(HttpAsyncClientBuilder clientBuilder, UsernamePasswordCredentials upcredentials = new UsernamePasswordCredentials( credentials.get().getKey(), credentials.get().getValue()); credentialsProvider.setCredentials(AuthScope.ANY, upcredentials); - clientBuilder.setDefaultCredentialsProvider(credentialsProvider); + return credentialsProvider; } else { LOG.info( "Elasticsearch client credentials not provided. Defaulting to non-authenticated client connection."); + return null; } } /** - * Modify client builder with connection encryption details (SSL) if applicable. + *

Setup connection encryption details (SSL) if applicable. * If ssl.enabled=true, sets up SSL connection. If enabled, keystore.path is required. User can * also optionally set keystore.password and keystore.type. * https://www.elastic.co/guide/en/elasticsearch/client/java-rest/5.6/_encrypted_communication.html - * - * @param clientBuilder builder to modify - * @param esClientConfig pull connection encryption details from config + *

+ *

Other guidance on the HTTP Component library and configuring SSL connections. + * http://www.robinhowlett.com/blog/2016/01/05/everything-you-ever-wanted-to-know-about-ssl-but-were-afraid-to-ask. + *

+ *

JSSE docs - https://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/JSSERefGuide.html + *

+ *

Additional guidance for configuring Elasticsearch for SSL can be found here - https://www.elastic.co/guide/en/x-pack/5.6/ssl-tls.html */ - private static void setupConnectionEncryption(HttpAsyncClientBuilder clientBuilder, - ElasticsearchClientConfig esClientConfig) { + private static SSLContext getSSLContext(ElasticsearchClientConfig esClientConfig) { if (esClientConfig.isSSLEnabled()) { LOG.info("Configuring client for SSL connection."); if (!esClientConfig.getKeyStorePath().isPresent()) { throw new IllegalStateException("KeyStore path must be provided for SSL connection."); } - KeyStore truststore; - try { - truststore = KeyStore.getInstance(esClientConfig.getKeyStoreType()); - } catch (KeyStoreException e) { - throw new IllegalStateException( - "Unable to get keystore type '" + esClientConfig.getKeyStoreType() + "'", e); - } Optional optKeyStorePass = esClientConfig.getKeyStorePassword(); char[] keyStorePass = optKeyStorePass.map(String::toCharArray).orElse(null); - try (InputStream is = Files.newInputStream(esClientConfig.getKeyStorePath().get())) { - truststore.load(is, keyStorePass); - } catch (IOException | NoSuchAlgorithmException | CertificateException e) { - throw new IllegalStateException( - "Unable to load keystore from path '" + esClientConfig.getKeyStorePath().get() + "'", - e); - } + KeyStore trustStore = getStore(esClientConfig.getKeyStoreType(), + esClientConfig.getKeyStorePath().get(), keyStorePass); try { - SSLContextBuilder sslBuilder = SSLContexts.custom().loadTrustMaterial(truststore, null); - clientBuilder.setSSLContext(sslBuilder.build()); + SSLContextBuilder sslBuilder = SSLContexts.custom().loadTrustMaterial(trustStore, null); + return sslBuilder.build(); } catch (NoSuchAlgorithmException | KeyStoreException | KeyManagementException e) { throw new IllegalStateException("Unable to load truststore.", e); } } + return null; + } + + private static KeyStore getStore(String type, Path path, char[] pass) { + KeyStore store; + try { + store = KeyStore.getInstance(type); + } catch (KeyStoreException e) { + throw new IllegalStateException("Unable to get keystore type '" + type + "'", e); + } + try (InputStream is = Files.newInputStream(path)) { + store.load(is, pass); + } catch (IOException | NoSuchAlgorithmException | CertificateException e) { + throw new IllegalStateException("Unable to load keystore from path '" + path + "'", e); + } + return store; } } From d4b02ce80a6c72cdf4595f9a47a4955f59b0a5e4 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 2 Nov 2018 16:12:52 -0400 Subject: [PATCH 21/41] Fixed all tests --- .../bulk/BulkDocumentWriter.java | 12 ++-- .../bulk/ElasticsearchBulkDocumentWriter.java | 57 ++++++++---------- .../elasticsearch/bulk/IndexedDocument.java | 45 ++++++++++++++ .../elasticsearch/dao/ElasticsearchDao.java | 2 +- .../dao/ElasticsearchRetrieveLatestDao.java | 24 ++++---- .../dao/ElasticsearchUpdateDao.java | 59 +++++++------------ .../writer/ElasticsearchWriter.java | 9 ++- .../writer/TupleBasedDocument.java | 7 ++- .../ElasticsearchBulkDocumentWriterTest.java | 53 +++-------------- .../writer/BulkDocumentWriterStub.java | 3 +- .../metron/indexing/dao/update/Document.java | 35 ++--------- .../indexing/dao/UpdateIntegrationTest.java | 7 +-- 12 files changed, 138 insertions(+), 175 deletions(-) create mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java index aac825d430..5bb01cf7b7 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java @@ -17,17 +17,19 @@ */ package org.apache.metron.elasticsearch.bulk; -import org.apache.metron.indexing.dao.update.Document; - import java.util.List; -public interface BulkDocumentWriter { +/** + * + * @param + */ +public interface BulkDocumentWriter { - interface SuccessCallback { + interface SuccessCallback { void onSuccess(List documents); } - interface FailureCallback { + interface FailureCallback { void onFailure(D failedDocument, Throwable cause, String message); } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java index 453fb4d9b1..713ac034bd 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java @@ -18,8 +18,8 @@ package org.apache.metron.elasticsearch.bulk; import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.elasticsearch.client.ElasticsearchClient; -import org.apache.metron.indexing.dao.update.Document; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; @@ -34,8 +34,9 @@ import java.util.Iterator; import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; -public class ElasticsearchBulkDocumentWriter implements BulkDocumentWriter { +public class ElasticsearchBulkDocumentWriter implements BulkDocumentWriter { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); @@ -63,7 +64,10 @@ public void onFailure(FailureCallback onFailure) { public void write(List documents) { try { // create an index request for each document - List requests = createIndexRequests(documents); + List requests = documents + .stream() + .map(doc -> createRequest(doc)) + .collect(Collectors.toList()); // create one bulk request for all the documents BulkRequest bulkRequest = new BulkRequest(); @@ -71,7 +75,7 @@ public void write(List documents) { // handle the bulk response BulkResponse bulkResponse = client.getHighLevelClient().bulk(bulkRequest); - List successful = handleBulkResponse(bulkResponse, documents); + List successful = handleBulkResponse(bulkResponse, documents); // notify the success callback onSuccess.ifPresent(callback -> callback.onSuccess(successful)); @@ -80,7 +84,7 @@ public void write(List documents) { } catch(IOException e) { // failed to submit bulk request; all documents failed - for(Document failed: documents) { + for(D failed: documents) { onFailure.ifPresent(callback -> callback.onFailure(failed, e, ExceptionUtils.getRootCauseMessage(e))); } LOG.error("Failed to submit bulk request; all documents failed", e); @@ -94,8 +98,8 @@ public void write(List documents) { * @param documents The documents that are being written. * @return The documents that were successfully written. Failed documents are excluded. */ - private List handleBulkResponse(BulkResponse bulkResponse, List documents) { - List successful = new ArrayList<>(); + private List handleBulkResponse(BulkResponse bulkResponse, List documents) { + List successful = new ArrayList<>(); if (bulkResponse.hasFailures()) { // interrogate the response to distinguish between those that succeeded and those that failed @@ -104,14 +108,14 @@ private List handleBulkResponse(BulkResponse bulkResponse, List doc BulkItemResponse response = iterator.next(); if(response.isFailed()) { // request failed - Document failed = documents.get(response.getItemId()); + D failed = documents.get(response.getItemId()); Exception cause = response.getFailure().getCause(); String message = response.getFailureMessage(); onFailure.ifPresent(callback -> callback.onFailure(failed, cause, message)); } else { // request succeeded - Document success = documents.get(response.getItemId()); + D success = documents.get(response.getItemId()); successful.add(success); } } @@ -124,32 +128,21 @@ private List handleBulkResponse(BulkResponse bulkResponse, List doc } /** - * Creates an {@link IndexRequest} for each {@link Document}. + * Creates an {@link IndexRequest} for a document. * - * @param documents The list of documents to write. - * @return A list of requests; one for each document. + * @param document The document to index. + * @return The {@link IndexRequest} for a document. */ - private List createIndexRequests(List documents) { - List requests = new ArrayList<>(); - - // create a request for each document - for(Document document: documents) { - if(document.getTimestamp() == null) { - throw new IllegalArgumentException("Document must contain the timestamp"); - } - - IndexRequest request = new IndexRequest() - .source(document.getDocument()) - .type(document.getSensorType() + "_doc") - .id(document.getGuid()) - .timestamp(document.getTimestamp().toString()); - - // the index name may not be defined - document.getIndex().ifPresent(name -> request.index(name)); - - requests.add(request); + private IndexRequest createRequest(D document) { + if(document.getTimestamp() == null) { + throw new IllegalArgumentException("Document must contain the timestamp"); } - return requests; + return new IndexRequest() + .source(document.getDocument()) + .type(document.getSensorType() + "_doc") + .id(document.getGuid()) + .index(document.getIndex()) + .timestamp(document.getTimestamp().toString()); } } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java new file mode 100644 index 0000000000..713b78ff33 --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java @@ -0,0 +1,45 @@ +/** + * 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.elasticsearch.bulk; + +import org.apache.metron.indexing.dao.update.Document; + +import java.util.Map; + +public class IndexedDocument extends Document { + + private String index; + + public IndexedDocument(Map document, String guid, String sensorType, Long timestamp, String index) { + super(document, guid, sensorType, timestamp); + this.index = index; + } + + public IndexedDocument(Document document, String index) { + super(document.getDocument(), document.getGuid(), document.getSensorType(), document.getTimestamp()); + this.index = index; + } + + public String getIndex() { + return index; + } + + public void setIndex(String index) { + this.index = index; + } +} diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java index bf6cae221a..6dacfef922 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java @@ -190,7 +190,7 @@ public Document removeCommentFromAlert(CommentAddRemoveRequest request, Document } protected Optional getIndexName(String guid, String sensorType) throws IOException { - return updateDao.getIndexName(guid, sensorType); + return updateDao.findIndexNameByGUID(guid, sensorType); } protected SearchResponse search(SearchRequest request, QueryBuilder queryBuilder) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java index d6bb7bd22c..a3788a555b 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java @@ -20,16 +20,6 @@ import com.google.common.base.Splitter; import com.google.common.collect.Iterables; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.function.Function; - import org.apache.metron.common.Constants; import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.indexing.dao.RetrieveLatestDao; @@ -43,6 +33,16 @@ import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.builder.SearchSourceBuilder; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; + public class ElasticsearchRetrieveLatestDao implements RetrieveLatestDao { private ElasticsearchClient transportClient; @@ -73,7 +73,7 @@ public Iterable getAllLatest(List getRequests) throws IOEx String doc = hit.getSourceAsString(); String sourceType = Iterables.getFirst(Splitter.on("_doc").split(hit.getType()), null); try { - return Optional.of(new Document(doc, hit.getId(), sourceType, ts, Optional.ofNullable(hit.getIndex()))); + return Optional.of(new Document(doc, hit.getId(), sourceType, ts)); } catch (IOException e) { throw new IllegalStateException("Unable to retrieve latest: " + e.getMessage(), e); } @@ -152,7 +152,7 @@ private Optional toDocument(final String guid, SearchHit hit) { String doc = hit.getSourceAsString(); String sourceType = toSourceType(hit.getType()); try { - Document document = new Document(doc, guid, sourceType, 0L, Optional.ofNullable(hit.getIndex())); + Document document = new Document(doc, guid, sourceType, 0L); getTimestamp(document.getDocument()).ifPresent(ts -> document.setTimestamp(ts)); return Optional.of(document); } catch (IOException e) { diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java index 92af5049d4..a715013eb1 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java @@ -17,20 +17,9 @@ */ package org.apache.metron.elasticsearch.dao; -import static org.apache.metron.indexing.dao.IndexDao.COMMENTS_FIELD; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.ArrayList; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; - import org.apache.metron.elasticsearch.bulk.BulkDocumentWriter; import org.apache.metron.elasticsearch.bulk.ElasticsearchBulkDocumentWriter; +import org.apache.metron.elasticsearch.bulk.IndexedDocument; import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; @@ -38,10 +27,21 @@ import org.apache.metron.indexing.dao.update.CommentAddRemoveRequest; import org.apache.metron.indexing.dao.update.Document; import org.apache.metron.indexing.dao.update.UpdateDao; -import org.elasticsearch.action.index.IndexRequest; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.metron.indexing.dao.IndexDao.COMMENTS_FIELD; + public class ElasticsearchUpdateDao implements UpdateDao { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); @@ -49,7 +49,7 @@ public class ElasticsearchUpdateDao implements UpdateDao { private transient ElasticsearchClient client; private AccessConfig accessConfig; private ElasticsearchRetrieveLatestDao retrieveLatestDao; - private BulkDocumentWriter documentWriter; + private BulkDocumentWriter documentWriter; private int failures; private Throwable lastException; @@ -76,15 +76,13 @@ public Map> batchUpdate(Map globalConfig = accessConfig.getGlobalConfigSupplier().get(); String indexPostfix = ElasticsearchUtils.getIndexFormat(globalConfig).format(new Date()); - List documents = new ArrayList<>(); + List documents = new ArrayList<>(); for (Map.Entry> entry : updates.entrySet()) { Document document = entry.getKey(); + Optional optionalIndex = entry.getValue(); - // set the index name since it is known - String indexName = getIndexName(document, entry.getValue(), indexPostfix); - document.setIndex(Optional.of(indexName)); - - documents.add(document); + String indexName = optionalIndex.orElse(getIndexName(document, indexPostfix)); + documents.add(new IndexedDocument(document, indexName)); } // track if a failure occurs so that a checked exception can be thrown; cannot throw checked exception in lambda @@ -175,28 +173,15 @@ public Document removeCommentFromAlert(CommentAddRemoveRequest request, Document return update(newVersion, Optional.empty()); } - protected String getIndexName(Document update, Optional index, String indexPostFix) throws IOException { - return index.orElse(getIndexName(update.getGuid(), update.getSensorType()) - .orElse(ElasticsearchUtils.getIndexName(update.getSensorType(), indexPostFix, null)) - ); + protected String getIndexName(Document update, String indexPostFix) throws IOException { + return findIndexNameByGUID(update.getGuid(), update.getSensorType()) + .orElse(ElasticsearchUtils.getIndexName(update.getSensorType(), indexPostFix, null)); } - protected Optional getIndexName(String guid, String sensorType) throws IOException { + protected Optional findIndexNameByGUID(String guid, String sensorType) throws IOException { return retrieveLatestDao.searchByGuid(guid, sensorType, hit -> Optional.ofNullable(hit.getIndex()) ); } - - protected IndexRequest buildIndexRequest(Document update, String sensorType, String indexName) { - String type = sensorType + "_doc"; - Object ts = update.getTimestamp(); - IndexRequest indexRequest = new IndexRequest(indexName, type, update.getGuid()) - .source(update.getDocument()); - if (ts != null) { - indexRequest = indexRequest.timestamp(ts.toString()); - } - - return indexRequest; - } } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java index b81ff7b7e4..c349f06366 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java @@ -118,17 +118,16 @@ public BulkWriterResponse write(String sensorType, // define the document id String guid = String.class.cast(source.get(Constants.GUID)); if(guid == null) { - throw new IllegalStateException("What to do if no GUID?"); + LOG.info("Missing '{}' field; document ID will be auto-generated.", Constants.GUID); } // define the document timestamp - // TODO what if the timestamp is a string? - Long timestamp = Long.class.cast(source.get("timestamp")); + Long timestamp = Long.class.cast(source.get(Constants.Fields.TIMESTAMP.getName())); if(timestamp == null) { - throw new IllegalStateException("What to do if no timestamp?"); + LOG.info("Missing '{}' field; timestamp will be set to system time.", Constants.Fields.TIMESTAMP.getName()); } - documents.add(new TupleBasedDocument(source, guid, sensorType, timestamp, tuple, Optional.of(indexName))); + documents.add(new TupleBasedDocument(source, guid, sensorType, timestamp, indexName, tuple)); } // add successful tuples to the response diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java index 95f5c2e027..baffaf46a3 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java @@ -17,13 +17,14 @@ */ package org.apache.metron.elasticsearch.writer; +import org.apache.metron.elasticsearch.bulk.IndexedDocument; import org.apache.metron.indexing.dao.update.Document; import org.apache.storm.tuple.Tuple; import java.util.Map; import java.util.Optional; -public class TupleBasedDocument extends Document { +public class TupleBasedDocument extends IndexedDocument { private Tuple tuple; @@ -31,8 +32,8 @@ public TupleBasedDocument(Map document, String guid, String sensorType, Long timestamp, - Tuple tuple, - Optional index) { + String index, + Tuple tuple) { super(document, guid, sensorType, timestamp, index); this.tuple = tuple; } diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java index ec59269f61..e1c9c34e35 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java @@ -19,7 +19,6 @@ import org.apache.metron.common.Constants; import org.apache.metron.elasticsearch.client.ElasticsearchClient; -import org.apache.metron.indexing.dao.update.Document; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; @@ -27,13 +26,11 @@ import org.json.simple.JSONObject; import org.junit.Before; import org.junit.Test; -import org.mockito.ArgumentCaptor; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.UUID; import static org.junit.Assert.assertEquals; @@ -41,21 +38,19 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class ElasticsearchBulkDocumentWriterTest { - ElasticsearchBulkDocumentWriter writer; + ElasticsearchBulkDocumentWriter writer; ElasticsearchClient client; RestHighLevelClient highLevelClient; - ArgumentCaptor argumentCaptor; boolean onSuccessCalled; boolean onFailureCalled; @Before public void setup() { - // initial setup to mock Elasticsearch + // mock Elasticsearch highLevelClient = mock(RestHighLevelClient.class); client = mock(ElasticsearchClient.class); when(client.getHighLevelClient()).thenReturn(highLevelClient); @@ -70,7 +65,7 @@ public void testSuccessCallback() throws IOException { setupElasticsearchToSucceed(); // create a document to write - List documents = new ArrayList<>(); + List documents = new ArrayList<>(); documents.add(document(message())); // validate the "on success" callback @@ -89,7 +84,7 @@ public void testSuccessWithNoCallbacks() throws IOException { setupElasticsearchToSucceed(); // create a document to write - List documents = new ArrayList<>(); + List documents = new ArrayList<>(); documents.add(document(message())); // no callbacks defined @@ -103,7 +98,7 @@ public void testFailureCallback() throws IOException { setupElasticsearchToFail(); // create a document to write - List documents = new ArrayList<>(); + List documents = new ArrayList<>(); documents.add(document(message())); // validate the "on failure" callback @@ -123,7 +118,7 @@ public void testFailureWithNoCallbacks() throws IOException { setupElasticsearchToFail(); // create a document to write - List documents = new ArrayList<>(); + List documents = new ArrayList<>(); documents.add(document(message())); // no callbacks defined @@ -132,37 +127,6 @@ public void testFailureWithNoCallbacks() throws IOException { assertFalse(onFailureCalled); } - @Test - public void testDocumentWithIndex() throws IOException { - setupElasticsearchToSucceed(); - - // create a document that specifies the index - final String indexName = "test_index_foo"; - Document document = document(message()); - document.setIndex(Optional.of(indexName)); - - List documents = new ArrayList<>(); - documents.add(document); - - // validate the "on success" callback - writer.onSuccess(successfulDocs -> { - assertEquals(documents, successfulDocs); - onSuccessCalled = true; - }); - - writer.write(documents); - assertTrue(onSuccessCalled); - assertFalse(onFailureCalled); - - // capture the bulk request that is submitted to elasticsearch - argumentCaptor = ArgumentCaptor.forClass(BulkRequest.class); - verify(highLevelClient).bulk(argumentCaptor.capture()); - - // ensure the index name was set on the request - BulkRequest request = argumentCaptor.getValue(); - assertEquals(indexName, request.requests().get(0).index()); - } - private void setupElasticsearchToFail() throws IOException { // define the item failure BulkItemResponse.Failure failure = mock(BulkItemResponse.Failure.class); @@ -194,11 +158,12 @@ private void setupElasticsearchToSucceed() throws IOException { when(highLevelClient.bulk(any(BulkRequest.class))).thenReturn(response); } - private Document document(JSONObject message) { + private IndexedDocument document(JSONObject message) { String guid = UUID.randomUUID().toString(); String sensorType = "bro"; Long timestamp = System.currentTimeMillis(); - return new Document(message, guid, sensorType, timestamp); + String index = "bro_index"; + return new IndexedDocument(message, guid, sensorType, timestamp, index); } private JSONObject message() { diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java index bd8e9473d1..c03bb4c10b 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java @@ -18,6 +18,7 @@ package org.apache.metron.elasticsearch.writer; import org.apache.metron.elasticsearch.bulk.BulkDocumentWriter; +import org.apache.metron.elasticsearch.bulk.IndexedDocument; import org.apache.metron.indexing.dao.update.Document; import java.util.ArrayList; @@ -31,7 +32,7 @@ * * @param The type of {@link Document} to write. */ -public class BulkDocumentWriterStub implements BulkDocumentWriter { +public class BulkDocumentWriterStub implements BulkDocumentWriter { private SuccessCallback onSuccess; private FailureCallback onFailure; diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java index cc9b9711bb..66b8996428 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java @@ -18,41 +18,24 @@ package org.apache.metron.indexing.dao.update; +import org.apache.metron.common.utils.JSONUtils; + import java.io.IOException; import java.util.HashMap; import java.util.Map; import java.util.Objects; -import java.util.Optional; - -import org.apache.commons.lang3.builder.EqualsBuilder; -import org.apache.commons.lang3.builder.HashCodeBuilder; -import org.apache.metron.common.utils.JSONUtils; public class Document { Long timestamp; Map document; String guid; String sensorType; - Optional index; public Document(Map document, String guid, String sensorType, Long timestamp) { setDocument(document); setGuid(guid); setTimestamp(timestamp); setSensorType(sensorType); - index = Optional.empty(); - } - - public Document(Map document, String guid, String sensorType, Long timestamp, Optional index) { - setDocument(document); - setGuid(guid); - setTimestamp(timestamp); - setSensorType(sensorType); - setIndex(index); - } - - public Document(String document, String guid, String sensorType, Long timestamp, Optional index) throws IOException { - this(convertDoc(document), guid, sensorType, timestamp, index); } public Document(String document, String guid, String sensorType, Long timestamp) throws IOException { @@ -108,14 +91,6 @@ public void setGuid(String guid) { this.guid = guid; } - public Optional getIndex() { - return index; - } - - public void setIndex(Optional index) { - this.index = index; - } - @Override public boolean equals(Object o) { if (this == o) return true; @@ -124,13 +99,12 @@ public boolean equals(Object o) { return Objects.equals(timestamp, document1.timestamp) && Objects.equals(document, document1.document) && Objects.equals(guid, document1.guid) && - Objects.equals(sensorType, document1.sensorType) && - Objects.equals(index, document1.index); + Objects.equals(sensorType, document1.sensorType); } @Override public int hashCode() { - return Objects.hash(timestamp, document, guid, sensorType, index); + return Objects.hash(timestamp, document, guid, sensorType); } @Override @@ -140,7 +114,6 @@ public String toString() { ", document=" + document + ", guid='" + guid + '\'' + ", sensorType='" + sensorType + '\'' + - ", index=" + index + '}'; } } diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java index 0e920714c0..3af1a1294d 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java @@ -38,7 +38,6 @@ import org.apache.metron.indexing.dao.update.OriginalNotFoundException; import org.apache.metron.indexing.dao.update.PatchRequest; import org.apache.metron.indexing.dao.update.ReplaceRequest; -import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; @@ -110,7 +109,7 @@ public void test() throws Exception { }}, Optional.empty()); Assert.assertEquals(message0, update.getDocument()); -// Assert.assertEquals(10, getMockHTable().size()); + Assert.assertEquals(10, getMockHTable().size()); findUpdatedDoc(message0, guid, SENSOR_NAME); { //ensure hbase is up to date @@ -314,8 +313,8 @@ public void testRemoveComments() throws Exception { Assert.assertEquals(0, getComments(noComments).size()); // ensure the comment was removed from the index - Document indexedNoComments = findUpdatedDoc(withComments.getDocument(), withComments.getGuid(), withComments.getSensorType()); - Assert.assertEquals(1, getComments(indexedNoComments).size()); + Document indexedNoComments = findUpdatedDoc(noComments.getDocument(), withComments.getGuid(), withComments.getSensorType()); + Assert.assertEquals(0, getComments(indexedNoComments).size()); } protected Document addAlertComment(String guid, String comment, String username, long timestamp) From c3b28363e87f75e97bdc9ab14d30156136080af7 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 2 Nov 2018 16:27:42 -0400 Subject: [PATCH 22/41] Fixed how client is created --- .../integration/ElasticsearchSearchIntegrationTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java index 268fd0e6c6..f9a5e746fa 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java @@ -22,6 +22,7 @@ import org.apache.metron.common.Constants; import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.elasticsearch.client.ElasticsearchClient; +import org.apache.metron.elasticsearch.client.ElasticsearchClientFactory; import org.apache.metron.elasticsearch.dao.ElasticsearchColumnMetadataDao; import org.apache.metron.elasticsearch.dao.ElasticsearchDao; import org.apache.metron.elasticsearch.dao.ElasticsearchRequestSubmitter; @@ -137,7 +138,7 @@ protected static void loadTestData() throws Exception { // setup the classes required to write the test data AccessConfig accessConfig = createAccessConfig(); - ElasticsearchClient client = ElasticsearchUtils.getClient(createGlobalConfig()); + ElasticsearchClient client = ElasticsearchClientFactory.create(createGlobalConfig()); ElasticsearchRetrieveLatestDao retrieveLatestDao = new ElasticsearchRetrieveLatestDao(client); ElasticsearchColumnMetadataDao columnMetadataDao = new ElasticsearchColumnMetadataDao(client); ElasticsearchRequestSubmitter requestSubmitter = new ElasticsearchRequestSubmitter(client); From 4539f43c43334696e33a861e7cbbb76a83c217b6 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 2 Nov 2018 17:55:12 -0400 Subject: [PATCH 23/41] Improved docs --- .../bulk/BulkDocumentWriter.java | 35 ++++++++++-- .../bulk/ElasticsearchBulkDocumentWriter.java | 53 +++++++++---------- .../elasticsearch/bulk/IndexedDocument.java | 6 +++ .../writer/ElasticsearchWriter.java | 4 +- .../writer/TupleBasedDocument.java | 3 ++ .../ElasticsearchSearchIntegrationTest.java | 7 --- .../writer/BulkDocumentWriterStub.java | 8 +-- 7 files changed, 69 insertions(+), 47 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java index 5bb01cf7b7..300da3c9ea 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java @@ -20,22 +20,47 @@ import java.util.List; /** + * Writes documents to an index in bulk. * - * @param + *

Partial failures within a batch can be handled individually by registering + * a {@link FailureListener}. + * + * @param The type of document to write. */ public interface BulkDocumentWriter { - interface SuccessCallback { + /** + * A listener that is notified when a set of documents have been + * written successfully. + * @param The type of document to write. + */ + interface SuccessListener { void onSuccess(List documents); } - interface FailureCallback { + /** + * A listener that is notified when a document has failed to write. + * @param The type of document to write. + */ + interface FailureListener { void onFailure(D failedDocument, Throwable cause, String message); } - void onSuccess(SuccessCallback onSuccess); + /** + * Register a listener that is notified when a document is successfully written. + * @param onSuccess The listener to notify. + */ + void onSuccess(SuccessListener onSuccess); - void onFailure(FailureCallback onFailure); + /** + * Register a listener that is notified when a document fails to write. + * @param onFailure The listener to notify. + */ + void onFailure(FailureListener onFailure); + /** + * Write documents to an index. + * @param documents The documents to write. + */ void write(List documents); } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java index 713ac034bd..9a32e5aa4b 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java @@ -18,7 +18,6 @@ package org.apache.metron.elasticsearch.bulk; import org.apache.commons.lang3.exception.ExceptionUtils; -import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.BulkItemResponse; @@ -36,12 +35,16 @@ import java.util.Optional; import java.util.stream.Collectors; +/** + * Writes documents to an Elasticsearch index in bulk. + * + * @param The type of document to write. + */ public class ElasticsearchBulkDocumentWriter implements BulkDocumentWriter { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - private Optional onSuccess; - private Optional onFailure; + private Optional onSuccess; + private Optional onFailure; private ElasticsearchClient client; public ElasticsearchBulkDocumentWriter(ElasticsearchClient client) { @@ -51,12 +54,12 @@ public ElasticsearchBulkDocumentWriter(ElasticsearchClient client) { } @Override - public void onSuccess(SuccessCallback onSuccess) { + public void onSuccess(SuccessListener onSuccess) { this.onSuccess = Optional.of(onSuccess); } @Override - public void onFailure(FailureCallback onFailure) { + public void onFailure(FailureListener onFailure) { this.onFailure = Optional.of(onFailure); } @@ -69,11 +72,11 @@ public void write(List documents) { .map(doc -> createRequest(doc)) .collect(Collectors.toList()); - // create one bulk request for all the documents + // create one bulk request to wrap each of the index requests BulkRequest bulkRequest = new BulkRequest(); bulkRequest.add(requests); - // handle the bulk response + // submit the request and handle the response BulkResponse bulkResponse = client.getHighLevelClient().bulk(bulkRequest); List successful = handleBulkResponse(bulkResponse, documents); @@ -91,11 +94,22 @@ public void write(List documents) { } } + private IndexRequest createRequest(D document) { + if(document.getTimestamp() == null) { + throw new IllegalArgumentException("Document must contain the timestamp"); + } + return new IndexRequest() + .source(document.getDocument()) + .type(document.getSensorType() + "_doc") + .id(document.getGuid()) + .index(document.getIndex()) + .timestamp(document.getTimestamp().toString()); + } + /** * Handles the {@link BulkResponse} received from Elasticsearch. - * * @param bulkResponse The response received from Elasticsearch. - * @param documents The documents that are being written. + * @param documents The documents included in the bulk request. * @return The documents that were successfully written. Failed documents are excluded. */ private List handleBulkResponse(BulkResponse bulkResponse, List documents) { @@ -126,23 +140,4 @@ private List handleBulkResponse(BulkResponse bulkResponse, List documents) return successful; } - - /** - * Creates an {@link IndexRequest} for a document. - * - * @param document The document to index. - * @return The {@link IndexRequest} for a document. - */ - private IndexRequest createRequest(D document) { - if(document.getTimestamp() == null) { - throw new IllegalArgumentException("Document must contain the timestamp"); - } - - return new IndexRequest() - .source(document.getDocument()) - .type(document.getSensorType() + "_doc") - .id(document.getGuid()) - .index(document.getIndex()) - .timestamp(document.getTimestamp().toString()); - } } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java index 713b78ff33..0d82844044 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java @@ -21,8 +21,14 @@ import java.util.Map; +/** + * A {@link Document} along with the index where it is stored. + */ public class IndexedDocument extends Document { + /** + * The name of the index where the {@link Document} is stored. + */ private String index; public IndexedDocument(Map document, String guid, String sensorType, Long timestamp, String index) { diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java index 3f3433df14..1b2268d47e 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java @@ -61,8 +61,8 @@ public class ElasticsearchWriter implements BulkMessageWriter, Seria /** * Responsible for writing documents. * - *

Uses a {@link TupleBasedDocument} to maintain the relationship - * between a {@link Tuple} and the document created for that tuple. If + *

Uses a {@link TupleBasedDocument} to maintain the relationship between + * a {@link Tuple} and the document created from the contents of that tuple. If * a document cannot be written, the associated tuple needs to be failed. */ private transient BulkDocumentWriter documentWriter; diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java index baffaf46a3..12d7587410 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java @@ -24,6 +24,9 @@ import java.util.Map; import java.util.Optional; +/** + * An {@link IndexedDocument} that is created from the contents of a {@link Tuple}. + */ public class TupleBasedDocument extends IndexedDocument { private Tuple tuple; diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java index f9a5e746fa..cb991b538f 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java @@ -18,7 +18,6 @@ package org.apache.metron.elasticsearch.integration; -import com.google.common.util.concurrent.Uninterruptibles; import org.apache.metron.common.Constants; import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.elasticsearch.client.ElasticsearchClient; @@ -30,7 +29,6 @@ import org.apache.metron.elasticsearch.dao.ElasticsearchSearchDao; import org.apache.metron.elasticsearch.dao.ElasticsearchUpdateDao; import org.apache.metron.elasticsearch.integration.components.ElasticSearchComponent; -import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.IndexDao; import org.apache.metron.indexing.dao.SearchIntegrationTest; @@ -40,14 +38,11 @@ import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; import org.apache.metron.indexing.dao.search.SearchResult; -import org.apache.metron.indexing.dao.update.Document; import org.apache.metron.integration.InMemoryComponent; import org.apache.metron.integration.utils.TestUtils; -import org.apache.metron.stellar.common.utils.ConversionUtils; import org.json.simple.JSONArray; import org.json.simple.JSONObject; import org.json.simple.parser.JSONParser; -import org.json.simple.parser.ParseException; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -63,8 +58,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.TimeUnit; -import java.util.function.BooleanSupplier; import static org.apache.metron.integration.utils.TestUtils.assertEventually; diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java index c03bb4c10b..1b3f59202f 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java @@ -34,8 +34,8 @@ */ public class BulkDocumentWriterStub implements BulkDocumentWriter { - private SuccessCallback onSuccess; - private FailureCallback onFailure; + private SuccessListener onSuccess; + private FailureListener onFailure; private float probabilityOfSuccess; private Exception exception; @@ -45,12 +45,12 @@ public BulkDocumentWriterStub(float probabilityOfSuccess) { } @Override - public void onSuccess(SuccessCallback onSuccess) { + public void onSuccess(SuccessListener onSuccess) { this.onSuccess = onSuccess; } @Override - public void onFailure(FailureCallback onFailure) { + public void onFailure(FailureListener onFailure) { this.onFailure = onFailure; } From 32467b375f6e6fa3ac268fc0e5f4a6a447a48f76 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Mon, 5 Nov 2018 11:59:26 -0500 Subject: [PATCH 24/41] Small cleanup --- .../org/apache/metron/elasticsearch/bulk/IndexedDocument.java | 2 +- .../apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java | 2 +- .../apache/metron/elasticsearch/writer/TupleBasedDocument.java | 2 -- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java index 0d82844044..561fbd34ae 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java @@ -22,7 +22,7 @@ import java.util.Map; /** - * A {@link Document} along with the index where it is stored. + * A {@link Document} that can be written to a search index. */ public class IndexedDocument extends Document { diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java index a715013eb1..d65a2f0f55 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java @@ -85,7 +85,7 @@ public Map> batchUpdate(Map { diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java index 12d7587410..7202e22f97 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java @@ -18,11 +18,9 @@ package org.apache.metron.elasticsearch.writer; import org.apache.metron.elasticsearch.bulk.IndexedDocument; -import org.apache.metron.indexing.dao.update.Document; import org.apache.storm.tuple.Tuple; import java.util.Map; -import java.util.Optional; /** * An {@link IndexedDocument} that is created from the contents of a {@link Tuple}. From 55afca49c2d110fb5ac7e565d36385c3b9804c33 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Mon, 5 Nov 2018 18:39:43 -0500 Subject: [PATCH 25/41] Removed the IndexedDocument class to simplify things --- .../bulk/BulkDocumentWriter.java | 20 +++++-- .../bulk/ElasticsearchBulkDocumentWriter.java | 57 ++++++++++++++----- .../elasticsearch/bulk/IndexedDocument.java | 51 ----------------- .../dao/ElasticsearchUpdateDao.java | 13 ++--- .../writer/ElasticsearchWriter.java | 9 +-- .../writer/TupleBasedDocument.java | 9 ++- .../ElasticsearchBulkDocumentWriterTest.java | 42 +++++++------- .../writer/BulkDocumentWriterStub.java | 12 +++- 8 files changed, 104 insertions(+), 109 deletions(-) delete mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java index 300da3c9ea..2c4e4ebea6 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java @@ -17,6 +17,8 @@ */ package org.apache.metron.elasticsearch.bulk; +import org.apache.metron.indexing.dao.update.Document; + import java.util.List; /** @@ -27,14 +29,14 @@ * * @param The type of document to write. */ -public interface BulkDocumentWriter { +public interface BulkDocumentWriter { /** * A listener that is notified when a set of documents have been * written successfully. * @param The type of document to write. */ - interface SuccessListener { + interface SuccessListener { void onSuccess(List documents); } @@ -42,7 +44,7 @@ interface SuccessListener { * A listener that is notified when a document has failed to write. * @param The type of document to write. */ - interface FailureListener { + interface FailureListener { void onFailure(D failedDocument, Throwable cause, String message); } @@ -59,8 +61,14 @@ interface FailureListener { void onFailure(FailureListener onFailure); /** - * Write documents to an index. - * @param documents The documents to write. + * Add a document to the batch. + * @param document The document to write. + * @param index The name of the index to write to. + */ + void addDocument(D document, String index); + + /** + * Write all documents in the batch. */ - void write(List documents); + void write(); } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java index 9a32e5aa4b..aeccba550e 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java @@ -19,6 +19,7 @@ import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.metron.elasticsearch.client.ElasticsearchClient; +import org.apache.metron.indexing.dao.update.Document; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; @@ -40,17 +41,32 @@ * * @param The type of document to write. */ -public class ElasticsearchBulkDocumentWriter implements BulkDocumentWriter { +public class ElasticsearchBulkDocumentWriter implements BulkDocumentWriter { + + /** + * A {@link Document} along with the index it will be written to. + */ + private class Indexable { + D document; + String index; + + public Indexable(D document, String index) { + this.document = document; + this.index = index; + } + } private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private Optional onSuccess; private Optional onFailure; private ElasticsearchClient client; + private List documents; public ElasticsearchBulkDocumentWriter(ElasticsearchClient client) { this.client = client; this.onSuccess = Optional.empty(); this.onFailure = Optional.empty(); + this.documents = new ArrayList<>(); } @Override @@ -64,12 +80,17 @@ public void onFailure(FailureListener onFailure) { } @Override - public void write(List documents) { + public void addDocument(D document, String index) { + documents.add(new Indexable(document, index)); + } + + @Override + public void write() { try { // create an index request for each document List requests = documents .stream() - .map(doc -> createRequest(doc)) + .map(ix -> createRequest(ix.document, ix.index)) .collect(Collectors.toList()); // create one bulk request to wrap each of the index requests @@ -81,20 +102,22 @@ public void write(List documents) { List successful = handleBulkResponse(bulkResponse, documents); // notify the success callback - onSuccess.ifPresent(callback -> callback.onSuccess(successful)); + onSuccess.ifPresent(listener -> listener.onSuccess(successful)); LOG.debug("Wrote document(s) to Elasticsearch; batchSize={}, success={}, failed={}, took={} ms", documents.size(), successful.size(), documents.size() - successful.size(), bulkResponse.getTookInMillis()); } catch(IOException e) { // failed to submit bulk request; all documents failed - for(D failed: documents) { - onFailure.ifPresent(callback -> callback.onFailure(failed, e, ExceptionUtils.getRootCauseMessage(e))); + if(onFailure.isPresent()) { + for(D failed: getDocuments()) { + onFailure.get().onFailure(failed, e, ExceptionUtils.getRootCauseMessage(e)); + } } LOG.error("Failed to submit bulk request; all documents failed", e); } } - private IndexRequest createRequest(D document) { + private IndexRequest createRequest(D document, String index) { if(document.getTimestamp() == null) { throw new IllegalArgumentException("Document must contain the timestamp"); } @@ -102,7 +125,7 @@ private IndexRequest createRequest(D document) { .source(document.getDocument()) .type(document.getSensorType() + "_doc") .id(document.getGuid()) - .index(document.getIndex()) + .index(index) .timestamp(document.getTimestamp().toString()); } @@ -112,7 +135,7 @@ private IndexRequest createRequest(D document) { * @param documents The documents included in the bulk request. * @return The documents that were successfully written. Failed documents are excluded. */ - private List handleBulkResponse(BulkResponse bulkResponse, List documents) { + private List handleBulkResponse(BulkResponse bulkResponse, List documents) { List successful = new ArrayList<>(); if (bulkResponse.hasFailures()) { @@ -122,22 +145,30 @@ private List handleBulkResponse(BulkResponse bulkResponse, List documents) BulkItemResponse response = iterator.next(); if(response.isFailed()) { // request failed - D failed = documents.get(response.getItemId()); + D failed = getDocument(response.getItemId()); Exception cause = response.getFailure().getCause(); String message = response.getFailureMessage(); - onFailure.ifPresent(callback -> callback.onFailure(failed, cause, message)); + onFailure.ifPresent(listener -> listener.onFailure(failed, cause, message)); } else { // request succeeded - D success = documents.get(response.getItemId()); + D success = getDocument(response.getItemId()); successful.add(success); } } } else { // all requests succeeded - successful.addAll(documents); + successful.addAll(getDocuments()); } return successful; } + + private List getDocuments() { + return documents.stream().map(ix -> ix.document).collect(Collectors.toList()); + } + + private D getDocument(int index) { + return documents.get(index).document; + } } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java deleted file mode 100644 index 561fbd34ae..0000000000 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/IndexedDocument.java +++ /dev/null @@ -1,51 +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.elasticsearch.bulk; - -import org.apache.metron.indexing.dao.update.Document; - -import java.util.Map; - -/** - * A {@link Document} that can be written to a search index. - */ -public class IndexedDocument extends Document { - - /** - * The name of the index where the {@link Document} is stored. - */ - private String index; - - public IndexedDocument(Map document, String guid, String sensorType, Long timestamp, String index) { - super(document, guid, sensorType, timestamp); - this.index = index; - } - - public IndexedDocument(Document document, String index) { - super(document.getDocument(), document.getGuid(), document.getSensorType(), document.getTimestamp()); - this.index = index; - } - - public String getIndex() { - return index; - } - - public void setIndex(String index) { - this.index = index; - } -} diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java index d65a2f0f55..63ee843821 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java @@ -19,7 +19,6 @@ import org.apache.metron.elasticsearch.bulk.BulkDocumentWriter; import org.apache.metron.elasticsearch.bulk.ElasticsearchBulkDocumentWriter; -import org.apache.metron.elasticsearch.bulk.IndexedDocument; import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; @@ -49,7 +48,7 @@ public class ElasticsearchUpdateDao implements UpdateDao { private transient ElasticsearchClient client; private AccessConfig accessConfig; private ElasticsearchRetrieveLatestDao retrieveLatestDao; - private BulkDocumentWriter documentWriter; + private BulkDocumentWriter documentWriter; private int failures; private Throwable lastException; @@ -76,13 +75,11 @@ public Map> batchUpdate(Map globalConfig = accessConfig.getGlobalConfigSupplier().get(); String indexPostfix = ElasticsearchUtils.getIndexFormat(globalConfig).format(new Date()); - List documents = new ArrayList<>(); for (Map.Entry> entry : updates.entrySet()) { Document document = entry.getKey(); Optional optionalIndex = entry.getValue(); - String indexName = optionalIndex.orElse(getIndexName(document, indexPostfix)); - documents.add(new IndexedDocument(document, indexName)); + documentWriter.addDocument(document, indexName); } // record failures so that a checked exception can be thrown later; cannot throw checked exception in listener @@ -94,10 +91,10 @@ public Map> batchUpdate(Map 0) { - String msg = String.format("Failed to update all documents; %d of %d update(s) failed", failures, documents.size()); + String msg = String.format("Failed to update all documents; %d of %d update(s) failed", failures, updates.entrySet().size()); throw new IOException(msg, lastException); } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java index 1b2268d47e..8abf309d64 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java @@ -127,7 +127,8 @@ public BulkWriterResponse write(String sensorType, LOG.info("Missing '{}' field; timestamp will be set to system time.", Constants.Fields.TIMESTAMP.getName()); } - documents.add(new TupleBasedDocument(source, guid, sensorType, timestamp, indexName, tuple)); + TupleBasedDocument document = new TupleBasedDocument(source, guid, sensorType, timestamp, tuple); + documentWriter.addDocument(document, indexName); } // add successful tuples to the response @@ -138,13 +139,13 @@ public BulkWriterResponse write(String sensorType, }); // add any failed tuples to the response - documentWriter.onFailure((failedDocument, cause, message) -> { - Tuple failedTuple = failedDocument.getTuple(); + documentWriter.onFailure((document, cause, message) -> { + Tuple failedTuple = document.getTuple(); response.addError(cause, failedTuple); }); // write the documents - documentWriter.write(documents); + documentWriter.write(); return response; } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java index 7202e22f97..ba44937b16 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/TupleBasedDocument.java @@ -17,15 +17,15 @@ */ package org.apache.metron.elasticsearch.writer; -import org.apache.metron.elasticsearch.bulk.IndexedDocument; +import org.apache.metron.indexing.dao.update.Document; import org.apache.storm.tuple.Tuple; import java.util.Map; /** - * An {@link IndexedDocument} that is created from the contents of a {@link Tuple}. + * An {@link Document} that is created from the contents of a {@link Tuple}. */ -public class TupleBasedDocument extends IndexedDocument { +public class TupleBasedDocument extends Document { private Tuple tuple; @@ -33,9 +33,8 @@ public TupleBasedDocument(Map document, String guid, String sensorType, Long timestamp, - String index, Tuple tuple) { - super(document, guid, sensorType, timestamp, index); + super(document, guid, sensorType, timestamp); this.tuple = tuple; } diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java index e1c9c34e35..155746b8b4 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java @@ -19,6 +19,7 @@ import org.apache.metron.common.Constants; import org.apache.metron.elasticsearch.client.ElasticsearchClient; +import org.apache.metron.indexing.dao.update.Document; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; @@ -28,7 +29,6 @@ import org.junit.Test; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.UUID; @@ -42,7 +42,7 @@ public class ElasticsearchBulkDocumentWriterTest { - ElasticsearchBulkDocumentWriter writer; + ElasticsearchBulkDocumentWriter writer; ElasticsearchClient client; RestHighLevelClient highLevelClient; boolean onSuccessCalled; @@ -65,16 +65,18 @@ public void testSuccessCallback() throws IOException { setupElasticsearchToSucceed(); // create a document to write - List documents = new ArrayList<>(); - documents.add(document(message())); + Document doc = document(message()); + String index = "bro_index"; + writer.addDocument(doc, index); // validate the "on success" callback writer.onSuccess(successfulDocs -> { - assertEquals(documents, successfulDocs); + assertEquals(1, successfulDocs.size()); + assertEquals(doc, successfulDocs.get(0)); onSuccessCalled = true; }); - writer.write(documents); + writer.write(); assertTrue(onSuccessCalled); assertFalse(onFailureCalled); } @@ -84,11 +86,12 @@ public void testSuccessWithNoCallbacks() throws IOException { setupElasticsearchToSucceed(); // create a document to write - List documents = new ArrayList<>(); - documents.add(document(message())); + Document doc = document(message()); + String index = "bro_index"; + writer.addDocument(doc, index); // no callbacks defined - writer.write(documents); + writer.write(); assertFalse(onSuccessCalled); assertFalse(onFailureCalled); } @@ -98,17 +101,18 @@ public void testFailureCallback() throws IOException { setupElasticsearchToFail(); // create a document to write - List documents = new ArrayList<>(); - documents.add(document(message())); + Document doc = document(message()); + String index = "bro_index"; + writer.addDocument(doc, index); // validate the "on failure" callback writer.onFailure((failedDoc, cause, msg) -> { - assertEquals(documents.get(0), failedDoc); + assertEquals(doc, failedDoc); onFailureCalled = true; }); // no callbacks defined - writer.write(documents); + writer.write(); assertFalse(onSuccessCalled); assertTrue(onFailureCalled); } @@ -118,11 +122,12 @@ public void testFailureWithNoCallbacks() throws IOException { setupElasticsearchToFail(); // create a document to write - List documents = new ArrayList<>(); - documents.add(document(message())); + Document doc = document(message()); + String index = "bro_index"; + writer.addDocument(doc, index); // no callbacks defined - writer.write(documents); + writer.write(); assertFalse(onSuccessCalled); assertFalse(onFailureCalled); } @@ -158,12 +163,11 @@ private void setupElasticsearchToSucceed() throws IOException { when(highLevelClient.bulk(any(BulkRequest.class))).thenReturn(response); } - private IndexedDocument document(JSONObject message) { + private Document document(JSONObject message) { String guid = UUID.randomUUID().toString(); String sensorType = "bro"; Long timestamp = System.currentTimeMillis(); - String index = "bro_index"; - return new IndexedDocument(message, guid, sensorType, timestamp, index); + return new Document(message, guid, sensorType, timestamp); } private JSONObject message() { diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java index 1b3f59202f..185c73b15f 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java @@ -18,7 +18,6 @@ package org.apache.metron.elasticsearch.writer; import org.apache.metron.elasticsearch.bulk.BulkDocumentWriter; -import org.apache.metron.elasticsearch.bulk.IndexedDocument; import org.apache.metron.indexing.dao.update.Document; import java.util.ArrayList; @@ -32,16 +31,18 @@ * * @param The type of {@link Document} to write. */ -public class BulkDocumentWriterStub implements BulkDocumentWriter { +public class BulkDocumentWriterStub implements BulkDocumentWriter { private SuccessListener onSuccess; private FailureListener onFailure; private float probabilityOfSuccess; private Exception exception; + private List documents; public BulkDocumentWriterStub(float probabilityOfSuccess) { this.probabilityOfSuccess = probabilityOfSuccess; this.exception = new IllegalStateException("Exception created by a stub for testing"); + this.documents = new ArrayList<>(); } @Override @@ -55,7 +56,12 @@ public void onFailure(FailureListener onFailure) { } @Override - public void write(List documents) { + public void addDocument(D document, String index) { + documents.add(document); + } + + @Override + public void write() { Random random = new Random(); List successes = new ArrayList<>(); From 5b6c6b0569d5191670a0aac11b15b5913cce0a2f Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 8 Nov 2018 15:26:16 -0500 Subject: [PATCH 26/41] ElasticsearchBulkDocumentWriter was not flushing the documents that were written. --- .../bulk/BulkDocumentWriter.java | 5 +++ .../bulk/ElasticsearchBulkDocumentWriter.java | 10 +++++ .../ElasticsearchBulkDocumentWriterTest.java | 38 +++++++++++++++++++ .../writer/BulkDocumentWriterStub.java | 5 +++ 4 files changed, 58 insertions(+) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java index 2c4e4ebea6..b083f5b492 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java @@ -67,6 +67,11 @@ interface FailureListener { */ void addDocument(D document, String index); + /** + * @return The number of documents waiting to be written. + */ + int size(); + /** * Write all documents in the batch. */ diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java index aeccba550e..bd579ad56e 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java @@ -82,6 +82,7 @@ public void onFailure(FailureListener onFailure) { @Override public void addDocument(D document, String index) { documents.add(new Indexable(document, index)); + LOG.debug("Adding document to batch; document={}, index={}", document, index); } @Override @@ -114,9 +115,18 @@ public void write() { } } LOG.error("Failed to submit bulk request; all documents failed", e); + + } finally { + // flush all documents no matter which ones succeeded or failed + documents.clear(); } } + @Override + public int size() { + return documents.size(); + } + private IndexRequest createRequest(D document, String index) { if(document.getTimestamp() == null) { throw new IllegalArgumentException("Document must contain the timestamp"); diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java index 155746b8b4..fa269ffb20 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java @@ -132,6 +132,44 @@ public void testFailureWithNoCallbacks() throws IOException { assertFalse(onFailureCalled); } + @Test + public void testFlushBatchOnSuccess() throws IOException { + setupElasticsearchToSucceed(); + assertEquals(0, writer.size()); + + // add some documents to write + String index = "bro_index"; + writer.addDocument(document(message()), index); + writer.addDocument(document(message()), index); + writer.addDocument(document(message()), index); + writer.addDocument(document(message()), index); + writer.addDocument(document(message()), index); + assertEquals(5, writer.size()); + + // after the write, all documents should have been flushed + writer.write(); + assertEquals(0, writer.size()); + } + + @Test + public void testFlushBatchOnFailure() throws IOException { + setupElasticsearchToFail(); + assertEquals(0, writer.size()); + + // add some documents to write + String index = "bro_index"; + writer.addDocument(document(message()), index); + writer.addDocument(document(message()), index); + writer.addDocument(document(message()), index); + writer.addDocument(document(message()), index); + writer.addDocument(document(message()), index); + assertEquals(5, writer.size()); + + // after the write, all documents should have been flushed + writer.write(); + assertEquals(0, writer.size()); + } + private void setupElasticsearchToFail() throws IOException { // define the item failure BulkItemResponse.Failure failure = mock(BulkItemResponse.Failure.class); diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java index 185c73b15f..93fc07991c 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java @@ -79,6 +79,11 @@ public void write() { onSuccess.onSuccess(successes); } + @Override + public int size() { + return documents.size(); + } + /** * Set the exception that is passed to the failure callback when a message fails to write. * @param exception The exception passed to the failure callback. From 6a506df616a63a6a8f44376106db259953e5b353 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Tue, 27 Nov 2018 16:00:52 -0500 Subject: [PATCH 27/41] Removed use of Java streams --- .../bulk/ElasticsearchBulkDocumentWriter.java | 11 ++++------- .../elasticsearch/writer/ElasticsearchWriter.java | 7 ++++--- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java index bd579ad56e..a2845c6702 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java @@ -89,14 +89,11 @@ public void addDocument(D document, String index) { public void write() { try { // create an index request for each document - List requests = documents - .stream() - .map(ix -> createRequest(ix.document, ix.index)) - .collect(Collectors.toList()); - - // create one bulk request to wrap each of the index requests BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.add(requests); + for(Indexable doc: documents) { + DocWriteRequest request = createRequest(doc.document, doc.index); + bulkRequest.add(request); + } // submit the request and handle the response BulkResponse bulkResponse = client.getHighLevelClient().bulk(bulkRequest); diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java index cb1fb022ac..a202d75b1a 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java @@ -133,9 +133,10 @@ public BulkWriterResponse write(String sensorType, // add successful tuples to the response BulkWriterResponse response = new BulkWriterResponse(); - documentWriter.onSuccess(docs -> { - List successfulTuples = docs.stream().map(doc -> doc.getTuple()).collect(Collectors.toList()); - response.addAllSuccesses(successfulTuples); + documentWriter.onSuccess(successes -> { + for(TupleBasedDocument doc: successes) { + response.addSuccess(doc.getTuple()); + } }); // add any failed tuples to the response From 84691e343d9a3509940ba70005364d432698726b Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 29 Nov 2018 09:00:12 -0500 Subject: [PATCH 28/41] Clean-up imports --- .../indexing/dao/UpdateIntegrationTest.java | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java index 2d5eda8cb8..a8bac73b69 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java @@ -14,17 +14,6 @@ */ package org.apache.metron.indexing.dao; -import static org.apache.metron.indexing.dao.IndexDao.COMMENTS_FIELD; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.Optional; -import java.util.UUID; -import java.util.stream.Collectors; import org.adrianwalker.multilinestring.Multiline; import org.apache.commons.collections.MapUtils; import org.apache.hadoop.hbase.client.Get; @@ -41,7 +30,18 @@ import org.junit.Assert; import org.junit.Test; -import static org.hamcrest.CoreMatchers.*; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Optional; +import java.util.UUID; +import java.util.stream.Collectors; + +import static org.apache.metron.indexing.dao.IndexDao.COMMENTS_FIELD; +import static org.hamcrest.CoreMatchers.hasItem; public abstract class UpdateIntegrationTest { From 414b5dca364380087b92a05fbf992a11651bf598 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 29 Nov 2018 09:02:11 -0500 Subject: [PATCH 29/41] No need to change this test. Missed in merge with master --- .../SolrUpdateIntegrationTest.java | 135 ++---------------- 1 file changed, 8 insertions(+), 127 deletions(-) diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java index 3c2e628b7b..5b965590c7 100644 --- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java +++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java @@ -17,15 +17,19 @@ */ package org.apache.metron.solr.integration; +import static org.apache.metron.solr.SolrConstants.SOLR_ZOOKEEPER; +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; import org.apache.commons.lang3.StringUtils; import org.apache.curator.framework.CuratorFramework; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Result; -import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.ConfigurationsUtils; -import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.common.zookeeper.ZKConfigurationsCache; import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.apache.metron.hbase.mock.MockHTable; @@ -35,30 +39,17 @@ import org.apache.metron.indexing.dao.MultiIndexDao; import org.apache.metron.indexing.dao.UpdateIntegrationTest; import org.apache.metron.indexing.dao.update.Document; -import org.apache.metron.indexing.dao.update.ReplaceRequest; import org.apache.metron.indexing.util.IndexingCacheUtil; import org.apache.metron.solr.dao.SolrDao; import org.apache.metron.solr.integration.components.SolrComponent; import org.junit.After; import org.junit.AfterClass; -import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.Optional; - -import static org.apache.metron.solr.SolrConstants.SOLR_ZOOKEEPER; -import static org.junit.Assert.assertEquals; - public class SolrUpdateIntegrationTest extends UpdateIntegrationTest { @Rule public final ExpectedException exception = ExpectedException.none(); @@ -195,114 +186,4 @@ public void testHugeErrorFields() throws Exception { exception.expectMessage("Document contains at least one immense term in field=\"error_hash\""); getDao().update(errorDoc, Optional.of("error")); } - - @Test - @Override - public void test() throws Exception { - List> inputData = new ArrayList<>(); - for(int i = 0; i < 10;++i) { - final String name = "message" + i; - inputData.add( - new HashMap() {{ - put("source.type", SENSOR_NAME); - put("name" , name); - put("timestamp", System.currentTimeMillis()); - put(Constants.GUID, name); - }} - ); - } - addTestData(getIndexName(), SENSOR_NAME, inputData); - List> docs = null; - for(int t = 0;t < MAX_RETRIES;++t, Thread.sleep(SLEEP_MS)) { - docs = getIndexedTestData(getIndexName(), SENSOR_NAME); - if(docs.size() >= 10) { - break; - } - } - Assert.assertEquals(10, docs.size()); - //modify the first message and add a new field - { - Map message0 = new HashMap(inputData.get(0)) {{ - put("new-field", "metron"); - }}; - String guid = "" + message0.get(Constants.GUID); - Document update = getDao().replace(new ReplaceRequest(){{ - setReplacement(message0); - setGuid(guid); - setSensorType(SENSOR_NAME); - setIndex(getIndexName()); - }}, Optional.empty()); - - Assert.assertEquals(message0, update.getDocument()); - Assert.assertEquals(1, getMockHTable().size()); - findUpdatedDoc(message0, guid, SENSOR_NAME); - { - //ensure hbase is up to date - Get g = new Get(HBaseDao.Key.toBytes(new HBaseDao.Key(guid, SENSOR_NAME))); - Result r = getMockHTable().get(g); - NavigableMap columns = r.getFamilyMap(CF.getBytes()); - Assert.assertEquals(1, columns.size()); - Assert.assertEquals(message0 - , JSONUtils.INSTANCE.load(new String(columns.lastEntry().getValue()) - , JSONUtils.MAP_SUPPLIER) - ); - } - { - //ensure ES is up-to-date - long cnt = 0; - for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t, Thread.sleep(SLEEP_MS)) { - docs = getIndexedTestData(getIndexName(), SENSOR_NAME); - cnt = docs - .stream() - .filter(d -> message0.get("new-field").equals(d.get("new-field"))) - .count(); - } - Assert.assertNotEquals("Data store is not updated!", cnt, 0); - } - } - //modify the same message and modify the new field - { - Map message0 = new HashMap(inputData.get(0)) {{ - put("new-field", "metron2"); - }}; - String guid = "" + message0.get(Constants.GUID); - Document update = getDao().replace(new ReplaceRequest(){{ - setReplacement(message0); - setGuid(guid); - setSensorType(SENSOR_NAME); - setIndex(getIndexName()); - }}, Optional.empty()); - Assert.assertEquals(message0, update.getDocument()); - Assert.assertEquals(1, getMockHTable().size()); - Document doc = getDao().getLatest(guid, SENSOR_NAME); - Assert.assertEquals(message0, doc.getDocument()); - findUpdatedDoc(message0, guid, SENSOR_NAME); - { - //ensure hbase is up to date - Get g = new Get(HBaseDao.Key.toBytes(new HBaseDao.Key(guid, SENSOR_NAME))); - Result r = getMockHTable().get(g); - NavigableMap columns = r.getFamilyMap(CF.getBytes()); - Assert.assertEquals(2, columns.size()); - Assert.assertEquals(message0, JSONUtils.INSTANCE.load(new String(columns.lastEntry().getValue()) - , JSONUtils.MAP_SUPPLIER) - ); - Assert.assertNotEquals(message0, JSONUtils.INSTANCE.load(new String(columns.firstEntry().getValue()) - , JSONUtils.MAP_SUPPLIER) - ); - } - { - //ensure ES is up-to-date - long cnt = 0; - for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t,Thread.sleep(SLEEP_MS)) { - docs = getIndexedTestData(getIndexName(), SENSOR_NAME); - cnt = docs - .stream() - .filter(d -> message0.get("new-field").equals(d.get("new-field"))) - .count(); - } - - Assert.assertNotEquals("Data store is not updated!", cnt, 0); - } - } - } } From 7b4f82fef7c1c3231e5eb6e6686d190790dc02d4 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Thu, 29 Nov 2018 11:17:05 -0500 Subject: [PATCH 30/41] Removing an unnecessary use of Java streams --- .../bulk/ElasticsearchBulkDocumentWriter.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java index dc84c016fa..a88c93e778 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java @@ -35,7 +35,6 @@ import java.util.Iterator; import java.util.List; import java.util.Optional; -import java.util.stream.Collectors; /** * Writes documents to an Elasticsearch index in bulk. @@ -103,15 +102,16 @@ public void write() { BulkResponse bulkResponse = client.getHighLevelClient().bulk(bulkRequest); List successful = handleBulkResponse(bulkResponse, documents); - // notify the success callback + // notify the success listeners onSuccess.ifPresent(listener -> listener.onSuccess(successful)); LOG.debug("Wrote document(s) to Elasticsearch; batchSize={}, success={}, failed={}, took={} ms", documents.size(), successful.size(), documents.size() - successful.size(), bulkResponse.getTookInMillis()); } catch(IOException e) { - // failed to submit bulk request; all documents failed + // assume all documents have failed. notify the failure listeners if(onFailure.isPresent()) { - for(D failed: getDocuments()) { + for(Indexable indexable: documents) { + D failed = indexable.document; onFailure.get().onFailure(failed, e, ExceptionUtils.getRootCauseMessage(e)); } } @@ -174,14 +174,20 @@ private List handleBulkResponse(BulkResponse bulkResponse, List do } } else { // all requests succeeded - successful.addAll(getDocuments()); + for(Indexable success: documents) { + successful.add(success.document); + } } return successful; } private List getDocuments() { - return documents.stream().map(ix -> ix.document).collect(Collectors.toList()); + List results = new ArrayList<>(); + for(Indexable indexable: documents) { + results.add(indexable.document); + } + return results; } private D getDocument(int index) { From bfd51383ced4b6e5e214ae32fe5f89c0090953a3 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 30 Nov 2018 13:09:59 -0500 Subject: [PATCH 31/41] Rename test method to be more descriptive --- .../writer/ElasticsearchWriterTest.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java index 18afa2ef3d..866809d2c2 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java @@ -58,7 +58,7 @@ public void setup() { } @Test - public void testSuccess() { + public void shouldWriteSuccessfully() { // create a writer where all writes will be successful float probabilityOfSuccess = 1.0F; ElasticsearchWriter esWriter = new ElasticsearchWriter(); @@ -77,7 +77,7 @@ public void testSuccess() { } @Test - public void testSuccesses() { + public void shouldWriteManySuccessfully() { // create a writer where all writes will be successful float probabilityOfSuccess = 1.0F; ElasticsearchWriter esWriter = new ElasticsearchWriter(); @@ -98,7 +98,7 @@ public void testSuccesses() { } @Test - public void testFailure() { + public void shouldHandleWriteFailure() { // create a writer where all writes will fail float probabilityOfSuccess = 0.0F; BulkDocumentWriterStub docWriter = new BulkDocumentWriterStub<>(probabilityOfSuccess); @@ -120,7 +120,7 @@ public void testFailure() { } @Test - public void testFailures() { + public void shouldHandleManyWriteFailures() { // create a writer where all writes will fail float probabilityOfSuccess = 0.0F; BulkDocumentWriterStub docWriter = new BulkDocumentWriterStub<>(probabilityOfSuccess); @@ -144,7 +144,7 @@ public void testFailures() { } @Test - public void testPartialFailures() { + public void shouldHandlePartialFailures() { // create a writer where some will fails and some will succeed float probabilityOfSuccess = 0.5F; BulkDocumentWriterStub docWriter = new BulkDocumentWriterStub<>(probabilityOfSuccess); @@ -170,7 +170,7 @@ public void testPartialFailures() { } @Test(expected = IllegalStateException.class) - public void testWhenNumberOfMessagesDoesNotMatchTuples() { + public void shouldCheckIfNumberOfMessagesMatchNumberOfTuples() { // create a writer where all writes will be successful float probabilityOfSuccess = 1.0F; ElasticsearchWriter esWriter = new ElasticsearchWriter(); From 213dc0f9d29bc511c461f117a325e63e7f3485b6 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 30 Nov 2018 13:36:59 -0500 Subject: [PATCH 32/41] ElasticsearchWriter should handle message timestamp expressed as a String to maintain backwards compatibility --- .../writer/ElasticsearchWriter.java | 13 ++++++++---- .../writer/ElasticsearchWriterTest.java | 20 +++++++++++++++++++ 2 files changed, 29 insertions(+), 4 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java index a202d75b1a..74471c7929 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java @@ -18,6 +18,7 @@ package org.apache.metron.elasticsearch.writer; import com.google.common.collect.Lists; +import org.apache.commons.lang3.math.NumberUtils; import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.common.field.FieldNameConverter; @@ -45,6 +46,7 @@ import java.util.stream.Collectors; import static java.lang.String.format; +import static org.apache.metron.stellar.common.Constants.Fields.TIMESTAMP; /** * A {@link BulkMessageWriter} that writes messages to Elasticsearch. @@ -118,13 +120,16 @@ public BulkWriterResponse write(String sensorType, // define the document id String guid = String.class.cast(source.get(Constants.GUID)); if(guid == null) { - LOG.info("Missing '{}' field; document ID will be auto-generated.", Constants.GUID); + LOG.warn("Missing '{}' field; document ID will be auto-generated.", Constants.GUID); } // define the document timestamp - Long timestamp = Long.class.cast(source.get(Constants.Fields.TIMESTAMP.getName())); - if(timestamp == null) { - LOG.info("Missing '{}' field; timestamp will be set to system time.", Constants.Fields.TIMESTAMP.getName()); + Long timestamp = null; + Object value = source.get(TIMESTAMP.getName()); + if(value != null) { + timestamp = Long.parseLong(value.toString()); + } else { + LOG.warn("Missing '{}' field; timestamp will be set to system time.", TIMESTAMP.getName()); } TupleBasedDocument document = new TupleBasedDocument(source, guid, sensorType, timestamp, tuple); diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java index 866809d2c2..c37f197039 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java @@ -185,6 +185,26 @@ public void shouldCheckIfNumberOfMessagesMatchNumberOfTuples() { fail("expected exception"); } + @Test + public void shouldWriteSuccessfullyWhenMessageTimestampIsString() { + // create a writer where all writes will be successful + float probabilityOfSuccess = 1.0F; + ElasticsearchWriter esWriter = new ElasticsearchWriter(); + esWriter.setDocumentWriter(new BulkDocumentWriterStub<>(probabilityOfSuccess)); + esWriter.init(stormConf, topologyContext, writerConfiguration); + + // create a message where the timestamp is expressed as a string + List tuples = createTuples(1); + List messages = createMessages(1); + messages.get(0).put(Constants.Fields.TIMESTAMP.getName(), new Long(System.currentTimeMillis()).toString()); + + BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); + + // response should only contain successes + assertFalse(response.hasErrors()); + assertTrue(response.getSuccesses().contains(tuples.get(0))); + } + private JSONObject message() { JSONObject message = new JSONObject(); message.put(Constants.GUID, UUID.randomUUID().toString()); From 260ccc366b79ef53595dbfd097066040444b4eda Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Fri, 30 Nov 2018 13:58:34 -0500 Subject: [PATCH 33/41] Added test for missing GUID --- .../writer/ElasticsearchWriterTest.java | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java index c37f197039..309588957d 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java @@ -36,6 +36,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; @@ -205,6 +206,26 @@ public void shouldWriteSuccessfullyWhenMessageTimestampIsString() { assertTrue(response.getSuccesses().contains(tuples.get(0))); } + @Test + public void shouldWriteSuccessfullyWhenMissingGUID() { + // create a writer where all writes will be successful + float probabilityOfSuccess = 1.0F; + ElasticsearchWriter esWriter = new ElasticsearchWriter(); + esWriter.setDocumentWriter(new BulkDocumentWriterStub<>(probabilityOfSuccess)); + esWriter.init(stormConf, topologyContext, writerConfiguration); + + // create a message where the GUID is missing + List tuples = createTuples(1); + List messages = createMessages(1); + assertNotNull(messages.get(0).remove(Constants.GUID)); + + BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); + + // response should only contain successes + assertFalse(response.hasErrors()); + assertTrue(response.getSuccesses().contains(tuples.get(0))); + } + private JSONObject message() { JSONObject message = new JSONObject(); message.put(Constants.GUID, UUID.randomUUID().toString()); From a30b750c6ee5c7c8f53b6a063cedc230266d32d4 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Wed, 5 Dec 2018 08:11:29 -0500 Subject: [PATCH 34/41] Restore original Document class from master --- .../metron/indexing/dao/update/Document.java | 55 ++++++++++++------- 1 file changed, 35 insertions(+), 20 deletions(-) diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java index 66b8996428..3686b19d23 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java @@ -18,12 +18,10 @@ package org.apache.metron.indexing.dao.update; -import org.apache.metron.common.utils.JSONUtils; - import java.io.IOException; import java.util.HashMap; import java.util.Map; -import java.util.Objects; +import org.apache.metron.common.utils.JSONUtils; public class Document { Long timestamp; @@ -92,28 +90,45 @@ public void setGuid(String guid) { } @Override - public boolean equals(Object o) { - if (this == o) return true; - if (!(o instanceof Document)) return false; - Document document1 = (Document) o; - return Objects.equals(timestamp, document1.timestamp) && - Objects.equals(document, document1.document) && - Objects.equals(guid, document1.guid) && - Objects.equals(sensorType, document1.sensorType); + public String toString() { + return "Document{" + + "timestamp=" + timestamp + + ", document=" + document + + ", guid='" + guid + '\'' + + ", sensorType='" + sensorType + '\'' + + '}'; } @Override - public int hashCode() { - return Objects.hash(timestamp, document, guid, sensorType); + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + Document document1 = (Document) o; + + if (timestamp != null ? !timestamp.equals(document1.timestamp) : document1.timestamp != null) { + return false; + } + if (document != null ? !document.equals(document1.document) : document1.document != null) { + return false; + } + if (guid != null ? !guid.equals(document1.guid) : document1.guid != null) { + return false; + } + return sensorType != null ? sensorType.equals(document1.sensorType) + : document1.sensorType == null; } @Override - public String toString() { - return "Document{" + - "timestamp=" + timestamp + - ", document=" + document + - ", guid='" + guid + '\'' + - ", sensorType='" + sensorType + '\'' + - '}'; + public int hashCode() { + int result = timestamp != null ? timestamp.hashCode() : 0; + result = 31 * result + (document != null ? document.hashCode() : 0); + result = 31 * result + (guid != null ? guid.hashCode() : 0); + result = 31 * result + (sensorType != null ? sensorType.hashCode() : 0); + return result; } } From 1c1e4705fe0a0d075f61f3971f943a8f01f39eb7 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Wed, 5 Dec 2018 08:18:24 -0500 Subject: [PATCH 35/41] Using ConversionUtils for String conversion --- .../metron/elasticsearch/writer/ElasticsearchWriter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java index 74471c7929..5c7cfefb6a 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java @@ -30,6 +30,7 @@ import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.client.ElasticsearchClientFactory; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; +import org.apache.metron.stellar.common.utils.ConversionUtils; import org.apache.storm.task.TopologyContext; import org.apache.storm.tuple.Tuple; import org.json.simple.JSONObject; @@ -118,7 +119,7 @@ public BulkWriterResponse write(String sensorType, } // define the document id - String guid = String.class.cast(source.get(Constants.GUID)); + String guid = ConversionUtils.convert(source.get(Constants.GUID), String.class); if(guid == null) { LOG.warn("Missing '{}' field; document ID will be auto-generated.", Constants.GUID); } From da65a45aae40c684bcff89f07c8eee36fc8a652b Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Wed, 5 Dec 2018 10:33:28 -0500 Subject: [PATCH 36/41] The BulkDocumentWriter now returns a list of write results, rather than using listeners. --- .../bulk/BulkDocumentWriter.java | 36 +------- .../bulk/BulkDocumentWriterResults.java | 68 +++++++++++++++ .../bulk/ElasticsearchBulkDocumentWriter.java | 63 ++++---------- .../elasticsearch/bulk/WriteFailure.java | 48 +++++++++++ .../elasticsearch/bulk/WriteSuccess.java | 36 ++++++++ .../dao/ElasticsearchUpdateDao.java | 33 ++++---- .../writer/ElasticsearchWriter.java | 83 ++++++++++--------- .../ElasticsearchBulkDocumentWriterTest.java | 80 +++++------------- .../writer/BulkDocumentWriterStub.java | 31 +++---- 9 files changed, 261 insertions(+), 217 deletions(-) create mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriterResults.java create mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/WriteFailure.java create mode 100644 metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/WriteSuccess.java diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java index b083f5b492..927c5a2747 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java @@ -19,47 +19,13 @@ import org.apache.metron.indexing.dao.update.Document; -import java.util.List; - /** * Writes documents to an index in bulk. * - *

Partial failures within a batch can be handled individually by registering - * a {@link FailureListener}. - * * @param The type of document to write. */ public interface BulkDocumentWriter { - /** - * A listener that is notified when a set of documents have been - * written successfully. - * @param The type of document to write. - */ - interface SuccessListener { - void onSuccess(List documents); - } - - /** - * A listener that is notified when a document has failed to write. - * @param The type of document to write. - */ - interface FailureListener { - void onFailure(D failedDocument, Throwable cause, String message); - } - - /** - * Register a listener that is notified when a document is successfully written. - * @param onSuccess The listener to notify. - */ - void onSuccess(SuccessListener onSuccess); - - /** - * Register a listener that is notified when a document fails to write. - * @param onFailure The listener to notify. - */ - void onFailure(FailureListener onFailure); - /** * Add a document to the batch. * @param document The document to write. @@ -75,5 +41,5 @@ interface FailureListener { /** * Write all documents in the batch. */ - void write(); + BulkDocumentWriterResults write(); } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriterResults.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriterResults.java new file mode 100644 index 0000000000..10be247149 --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriterResults.java @@ -0,0 +1,68 @@ +/** + * 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.elasticsearch.bulk; + +import org.apache.metron.indexing.dao.update.Document; + +import java.util.ArrayList; +import java.util.List; + +/** + * The result of writing documents in bulk using a {@link BulkDocumentWriter}. + * @param The type of documents to write. + */ +public class BulkDocumentWriterResults { + + private List successes; + private List failures; + + public BulkDocumentWriterResults() { + this.successes = new ArrayList<>(); + this.failures = new ArrayList<>(); + } + + public void add(WriteSuccess success) { + this.successes.add(success); + } + + public void addSuccess(D success) { + add(new WriteSuccess(success)); + } + + public void addSuccesses(List successes) { + for(D success: successes) { + addSuccess(success); + } + } + + public List getSuccesses() { + return successes; + } + + public void add(WriteFailure failure) { + this.failures.add(failure); + } + + public void addFailure(D document, Throwable cause, String message) { + add(new WriteFailure(document, cause, message)); + } + + public List getFailures() { + return failures; + } +} diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java index a88c93e778..a40dd97bca 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java @@ -34,7 +34,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.Optional; /** * Writes documents to an Elasticsearch index in bulk. @@ -57,38 +56,25 @@ public Indexable(D document, String index) { } private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private Optional onSuccess; - private Optional onFailure; private ElasticsearchClient client; private List documents; private WriteRequest.RefreshPolicy refreshPolicy; public ElasticsearchBulkDocumentWriter(ElasticsearchClient client) { this.client = client; - this.onSuccess = Optional.empty(); - this.onFailure = Optional.empty(); this.documents = new ArrayList<>(); this.refreshPolicy = WriteRequest.RefreshPolicy.NONE; } @Override - public void onSuccess(SuccessListener onSuccess) { - this.onSuccess = Optional.of(onSuccess); + public void addDocument(D document, String indexName) { + documents.add(new Indexable(document, indexName)); + LOG.debug("Adding document to batch; document={}, index={}", document, indexName); } @Override - public void onFailure(FailureListener onFailure) { - this.onFailure = Optional.of(onFailure); - } - - @Override - public void addDocument(D document, String index) { - documents.add(new Indexable(document, index)); - LOG.debug("Adding document to batch; document={}, index={}", document, index); - } - - @Override - public void write() { + public BulkDocumentWriterResults write() { + BulkDocumentWriterResults results = new BulkDocumentWriterResults<>(); try { // create an index request for each document BulkRequest bulkRequest = new BulkRequest(); @@ -100,20 +86,13 @@ public void write() { // submit the request and handle the response BulkResponse bulkResponse = client.getHighLevelClient().bulk(bulkRequest); - List successful = handleBulkResponse(bulkResponse, documents); - - // notify the success listeners - onSuccess.ifPresent(listener -> listener.onSuccess(successful)); - LOG.debug("Wrote document(s) to Elasticsearch; batchSize={}, success={}, failed={}, took={} ms", - documents.size(), successful.size(), documents.size() - successful.size(), bulkResponse.getTookInMillis()); + handleBulkResponse(bulkResponse, documents, results); } catch(IOException e) { // assume all documents have failed. notify the failure listeners - if(onFailure.isPresent()) { - for(Indexable indexable: documents) { - D failed = indexable.document; - onFailure.get().onFailure(failed, e, ExceptionUtils.getRootCauseMessage(e)); - } + for(Indexable indexable: documents) { + D failed = indexable.document; + results.addFailure(failed, e, ExceptionUtils.getRootCauseMessage(e)); } LOG.error("Failed to submit bulk request; all documents failed", e); @@ -121,6 +100,10 @@ public void write() { // flush all documents no matter which ones succeeded or failed documents.clear(); } + + LOG.debug("Wrote document(s) to Elasticsearch; batchSize={}, success={}, failed={}", + documents.size(), results.getSuccesses().size(), results.getFailures().size()); + return results; } @Override @@ -149,10 +132,10 @@ private IndexRequest createRequest(D document, String index) { * Handles the {@link BulkResponse} received from Elasticsearch. * @param bulkResponse The response received from Elasticsearch. * @param documents The documents included in the bulk request. + * @param results The writer results. * @return The documents that were successfully written. Failed documents are excluded. */ - private List handleBulkResponse(BulkResponse bulkResponse, List documents) { - List successful = new ArrayList<>(); + private void handleBulkResponse(BulkResponse bulkResponse, List documents, BulkDocumentWriterResults results) { if (bulkResponse.hasFailures()) { // interrogate the response to distinguish between those that succeeded and those that failed @@ -164,30 +147,20 @@ private List handleBulkResponse(BulkResponse bulkResponse, List do D failed = getDocument(response.getItemId()); Exception cause = response.getFailure().getCause(); String message = response.getFailureMessage(); - onFailure.ifPresent(listener -> listener.onFailure(failed, cause, message)); + results.addFailure(failed, cause, message); } else { // request succeeded D success = getDocument(response.getItemId()); - successful.add(success); + results.addSuccess(success); } } } else { // all requests succeeded for(Indexable success: documents) { - successful.add(success.document); + results.addSuccess(success.document); } } - - return successful; - } - - private List getDocuments() { - List results = new ArrayList<>(); - for(Indexable indexable: documents) { - results.add(indexable.document); - } - return results; } private D getDocument(int index) { diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/WriteFailure.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/WriteFailure.java new file mode 100644 index 0000000000..ac571c7c04 --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/WriteFailure.java @@ -0,0 +1,48 @@ +/** + * 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.elasticsearch.bulk; + +import org.apache.metron.indexing.dao.update.Document; + +/** + * Indicates that a document failed to be written by a {@link BulkDocumentWriter}. + * @param The type of document that failed to write. + */ +public class WriteFailure { + private D document; + private Throwable cause; + private String message; + + public WriteFailure(D document, Throwable cause, String message) { + this.document = document; + this.cause = cause; + this.message = message; + } + + public D getDocument() { + return document; + } + + public Throwable getCause() { + return cause; + } + + public String getMessage() { + return message; + } +} diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/WriteSuccess.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/WriteSuccess.java new file mode 100644 index 0000000000..a86325d297 --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/WriteSuccess.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.elasticsearch.bulk; + +import org.apache.metron.indexing.dao.update.Document; + +/** + * Indicates that a document was successfully written by a {@link BulkDocumentWriter}. + * @param The type of document written. + */ +public class WriteSuccess { + private D document; + + public WriteSuccess(D document) { + this.document = document; + } + + public D getDocument() { + return document; + } +} diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java index db9638005b..399030e95b 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java @@ -18,6 +18,8 @@ package org.apache.metron.elasticsearch.dao; import org.apache.metron.elasticsearch.bulk.ElasticsearchBulkDocumentWriter; +import org.apache.metron.elasticsearch.bulk.WriteFailure; +import org.apache.metron.elasticsearch.bulk.BulkDocumentWriterResults; import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; @@ -41,6 +43,8 @@ import static org.apache.metron.indexing.dao.IndexDao.COMMENTS_FIELD; +import static java.lang.String.format; + public class ElasticsearchUpdateDao implements UpdateDao { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); @@ -48,8 +52,6 @@ public class ElasticsearchUpdateDao implements UpdateDao { private AccessConfig accessConfig; private ElasticsearchRetrieveLatestDao retrieveLatestDao; private ElasticsearchBulkDocumentWriter documentWriter; - private int failures; - private Throwable lastException; public ElasticsearchUpdateDao(ElasticsearchClient client, AccessConfig accessConfig, @@ -79,23 +81,24 @@ public Map> batchUpdate(Map optionalIndex = entry.getValue(); String indexName = optionalIndex.orElse(getIndexName(document, indexPostfix)); documentWriter.addDocument(document, indexName); - } - // record failures so that a checked exception can be thrown later; cannot throw checked exception in listener - failures = 0; - lastException = null; - documentWriter.onFailure((document, cause, message) -> { - failures++; - lastException = cause; - LOG.error(message, cause); - }); - // write the documents. if any document fails, raise an exception. - documentWriter.write(); + BulkDocumentWriterResults results = documentWriter.write(); + int failures = results.getFailures().size(); if(failures > 0) { - String msg = String.format("Failed to update all; %d of %d update(s) failed", failures, updates.entrySet().size()); - throw new IOException(msg, lastException); + int successes = results.getSuccesses().size(); + String msg = format("Failed to update all documents; %d successes, %d failures", successes, failures); + LOG.error(msg); + + // log each individual failure + for(WriteFailure failure: results.getFailures()) { + LOG.error(failure.getMessage(), failure.getCause()); + } + + // raise an exception + Throwable cause = results.getFailures().get(0).getCause(); + throw new IOException(msg, cause); } return updates; diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java index 5c7cfefb6a..a3459d8ccd 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriter.java @@ -18,7 +18,6 @@ package org.apache.metron.elasticsearch.writer; import com.google.common.collect.Lists; -import org.apache.commons.lang3.math.NumberUtils; import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.common.field.FieldNameConverter; @@ -27,6 +26,9 @@ import org.apache.metron.common.writer.BulkWriterResponse; import org.apache.metron.elasticsearch.bulk.BulkDocumentWriter; import org.apache.metron.elasticsearch.bulk.ElasticsearchBulkDocumentWriter; +import org.apache.metron.elasticsearch.bulk.WriteFailure; +import org.apache.metron.elasticsearch.bulk.WriteSuccess; +import org.apache.metron.elasticsearch.bulk.BulkDocumentWriterResults; import org.apache.metron.elasticsearch.client.ElasticsearchClient; import org.apache.metron.elasticsearch.client.ElasticsearchClientFactory; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; @@ -44,7 +46,6 @@ import java.util.Date; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import static java.lang.String.format; import static org.apache.metron.stellar.common.Constants.Fields.TIMESTAMP; @@ -107,55 +108,55 @@ public BulkWriterResponse write(String sensorType, } // create a document from each message - List documents = new ArrayList<>(); for(int i=0; i { - for(TupleBasedDocument doc: successes) { - response.addSuccess(doc.getTuple()); - } - }); - - // add any failed tuples to the response - documentWriter.onFailure((document, cause, message) -> { - Tuple failedTuple = document.getTuple(); - response.addError(cause, failedTuple); - }); - // write the documents - documentWriter.write(); + BulkDocumentWriterResults results = documentWriter.write(); + + // build the response + BulkWriterResponse response = new BulkWriterResponse(); + for(WriteSuccess success: results.getSuccesses()) { + response.addSuccess(success.getDocument().getTuple()); + } + for(WriteFailure failure: results.getFailures()) { + response.addError(failure.getCause(), failure.getDocument().getTuple()); + } return response; } + private TupleBasedDocument createDocument(JSONObject message, + Tuple tuple, + String sensorType, + FieldNameConverter fieldNameConverter) { + // transform the message fields to the source fields of the indexed document + JSONObject source = new JSONObject(); + for(Object k : message.keySet()){ + copyField(k.toString(), message, source, fieldNameConverter); + } + + // define the document id + String guid = ConversionUtils.convert(source.get(Constants.GUID), String.class); + if(guid == null) { + LOG.warn("Missing '{}' field; document ID will be auto-generated.", Constants.GUID); + } + + // define the document timestamp + Long timestamp = null; + Object value = source.get(TIMESTAMP.getName()); + if(value != null) { + timestamp = Long.parseLong(value.toString()); + } else { + LOG.warn("Missing '{}' field; timestamp will be set to system time.", TIMESTAMP.getName()); + } + + return new TupleBasedDocument(source, guid, sensorType, timestamp, tuple); + } + @Override public String getName() { return "elasticsearch"; diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java index fa269ffb20..b313811fcf 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriterTest.java @@ -34,8 +34,7 @@ import java.util.UUID; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertNotNull; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -45,8 +44,6 @@ public class ElasticsearchBulkDocumentWriterTest { ElasticsearchBulkDocumentWriter writer; ElasticsearchClient client; RestHighLevelClient highLevelClient; - boolean onSuccessCalled; - boolean onFailureCalled; @Before public void setup() { @@ -56,84 +53,46 @@ public void setup() { when(client.getHighLevelClient()).thenReturn(highLevelClient); writer = new ElasticsearchBulkDocumentWriter<>(client); - onFailureCalled = false; - onSuccessCalled = false; } @Test - public void testSuccessCallback() throws IOException { + public void testWriteSuccess() throws IOException { setupElasticsearchToSucceed(); - // create a document to write + // write a document successfully Document doc = document(message()); String index = "bro_index"; writer.addDocument(doc, index); - // validate the "on success" callback - writer.onSuccess(successfulDocs -> { - assertEquals(1, successfulDocs.size()); - assertEquals(doc, successfulDocs.get(0)); - onSuccessCalled = true; - }); + BulkDocumentWriterResults results = writer.write(); + assertEquals(1, results.getSuccesses().size()); + assertEquals(0, results.getFailures().size()); - writer.write(); - assertTrue(onSuccessCalled); - assertFalse(onFailureCalled); - } - - @Test - public void testSuccessWithNoCallbacks() throws IOException { - setupElasticsearchToSucceed(); - - // create a document to write - Document doc = document(message()); - String index = "bro_index"; - writer.addDocument(doc, index); - - // no callbacks defined - writer.write(); - assertFalse(onSuccessCalled); - assertFalse(onFailureCalled); + WriteSuccess success = results.getSuccesses().get(0); + assertEquals(doc, success.getDocument()); } @Test - public void testFailureCallback() throws IOException { + public void testWriteFailure() throws IOException { setupElasticsearchToFail(); - // create a document to write + // the document will fail to write Document doc = document(message()); String index = "bro_index"; writer.addDocument(doc, index); - // validate the "on failure" callback - writer.onFailure((failedDoc, cause, msg) -> { - assertEquals(doc, failedDoc); - onFailureCalled = true; - }); + BulkDocumentWriterResults results = writer.write(); + assertEquals(0, results.getSuccesses().size()); + assertEquals(1, results.getFailures().size()); - // no callbacks defined - writer.write(); - assertFalse(onSuccessCalled); - assertTrue(onFailureCalled); - } - - @Test - public void testFailureWithNoCallbacks() throws IOException { - setupElasticsearchToFail(); - - // create a document to write - Document doc = document(message()); - String index = "bro_index"; - writer.addDocument(doc, index); - - // no callbacks defined - writer.write(); - assertFalse(onSuccessCalled); - assertFalse(onFailureCalled); + WriteFailure failure = results.getFailures().get(0); + assertEquals(doc, failure.getDocument()); + assertEquals("error message", failure.getMessage()); + assertNotNull(failure.getCause()); } @Test - public void testFlushBatchOnSuccess() throws IOException { + public void testSizeWhenWriteSuccessful() throws IOException { setupElasticsearchToSucceed(); assertEquals(0, writer.size()); @@ -152,7 +111,7 @@ public void testFlushBatchOnSuccess() throws IOException { } @Test - public void testFlushBatchOnFailure() throws IOException { + public void testSizeWhenWriteFails() throws IOException { setupElasticsearchToFail(); assertEquals(0, writer.size()); @@ -181,6 +140,7 @@ private void setupElasticsearchToFail() throws IOException { when(itemResponse.isFailed()).thenReturn(true); when(itemResponse.getItemId()).thenReturn(0); when(itemResponse.getFailure()).thenReturn(failure); + when(itemResponse.getFailureMessage()).thenReturn("error message"); List itemsResponses = Collections.singletonList(itemResponse); // define the bulk response to indicate failure diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java index 93fc07991c..aeafee9ebc 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java @@ -18,6 +18,7 @@ package org.apache.metron.elasticsearch.writer; import org.apache.metron.elasticsearch.bulk.BulkDocumentWriter; +import org.apache.metron.elasticsearch.bulk.BulkDocumentWriterResults; import org.apache.metron.indexing.dao.update.Document; import java.util.ArrayList; @@ -33,50 +34,38 @@ */ public class BulkDocumentWriterStub implements BulkDocumentWriter { - private SuccessListener onSuccess; - private FailureListener onFailure; private float probabilityOfSuccess; private Exception exception; private List documents; + /** + * @param probabilityOfSuccess The probability that a write will succeed. + * If 1.0, all writes succeed. If 0.0, all writes fail. + */ public BulkDocumentWriterStub(float probabilityOfSuccess) { this.probabilityOfSuccess = probabilityOfSuccess; this.exception = new IllegalStateException("Exception created by a stub for testing"); this.documents = new ArrayList<>(); } - @Override - public void onSuccess(SuccessListener onSuccess) { - this.onSuccess = onSuccess; - } - - @Override - public void onFailure(FailureListener onFailure) { - this.onFailure = onFailure; - } - @Override public void addDocument(D document, String index) { documents.add(document); } @Override - public void write() { + public BulkDocumentWriterResults write() { + BulkDocumentWriterResults results = new BulkDocumentWriterResults<>(); Random random = new Random(); - - List successes = new ArrayList<>(); for(D document: documents) { boolean success = random.nextFloat() <= probabilityOfSuccess; if(success) { - successes.add(document); + results.addSuccess(document); } else { - // notify on failure - onFailure.onFailure(document, exception, "error"); + results.addFailure(document, exception, "error message"); } } - - // notify on success - onSuccess.onSuccess(successes); + return results; } @Override From 57c14799c89f15829451fba4e33d8ebc83efe189 Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Tue, 11 Dec 2018 08:23:29 -0500 Subject: [PATCH 37/41] Fixed missing generic types per @jleet feedback --- .../elasticsearch/bulk/BulkDocumentWriter.java | 2 +- .../bulk/BulkDocumentWriterResults.java | 14 +++++++------- .../elasticsearch/dao/ElasticsearchUpdateDao.java | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java index 927c5a2747..34f543ec88 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriter.java @@ -41,5 +41,5 @@ public interface BulkDocumentWriter { /** * Write all documents in the batch. */ - BulkDocumentWriterResults write(); + BulkDocumentWriterResults write(); } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriterResults.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriterResults.java index 10be247149..90e5ce31f1 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriterResults.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/BulkDocumentWriterResults.java @@ -28,20 +28,20 @@ */ public class BulkDocumentWriterResults { - private List successes; - private List failures; + private List> successes; + private List> failures; public BulkDocumentWriterResults() { this.successes = new ArrayList<>(); this.failures = new ArrayList<>(); } - public void add(WriteSuccess success) { + public void add(WriteSuccess success) { this.successes.add(success); } public void addSuccess(D success) { - add(new WriteSuccess(success)); + add(new WriteSuccess(success)); } public void addSuccesses(List successes) { @@ -50,11 +50,11 @@ public void addSuccesses(List successes) { } } - public List getSuccesses() { + public List> getSuccesses() { return successes; } - public void add(WriteFailure failure) { + public void add(WriteFailure failure) { this.failures.add(failure); } @@ -62,7 +62,7 @@ public void addFailure(D document, Throwable cause, String message) { add(new WriteFailure(document, cause, message)); } - public List getFailures() { + public List> getFailures() { return failures; } } diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java index 399030e95b..fa02f8df4d 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java @@ -96,7 +96,7 @@ public Map> batchUpdate(Map Date: Tue, 11 Dec 2018 08:25:58 -0500 Subject: [PATCH 38/41] Fixed javadoc per @jleet --- .../elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java | 1 - 1 file changed, 1 deletion(-) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java index a40dd97bca..ae1d47af39 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java @@ -133,7 +133,6 @@ private IndexRequest createRequest(D document, String index) { * @param bulkResponse The response received from Elasticsearch. * @param documents The documents included in the bulk request. * @param results The writer results. - * @return The documents that were successfully written. Failed documents are excluded. */ private void handleBulkResponse(BulkResponse bulkResponse, List documents, BulkDocumentWriterResults results) { if (bulkResponse.hasFailures()) { From c32327183ffb1bf19d4d19670fcbd3b4a3b5222a Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Tue, 11 Dec 2018 09:15:59 -0500 Subject: [PATCH 39/41] Simplifying iteration over the bulk item responses per @jleet --- .../elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java index ae1d47af39..66404a49d4 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java @@ -138,9 +138,7 @@ private void handleBulkResponse(BulkResponse bulkResponse, List docum if (bulkResponse.hasFailures()) { // interrogate the response to distinguish between those that succeeded and those that failed - Iterator iterator = bulkResponse.iterator(); - while(iterator.hasNext()) { - BulkItemResponse response = iterator.next(); + for(BulkItemResponse response: bulkResponse) { if(response.isFailed()) { // request failed D failed = getDocument(response.getItemId()); From 30bc70af147467905a53835acc68861c90ea706f Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Tue, 11 Dec 2018 11:23:16 -0500 Subject: [PATCH 40/41] Was able to remove the BulkDocumentWriterStub and use a mock instead. Easier to maintain. Thanks @jleet --- .../writer/BulkDocumentWriterStub.java | 87 --------- .../writer/ElasticsearchWriterTest.java | 181 +++++++++++------- 2 files changed, 115 insertions(+), 153 deletions(-) delete mode 100644 metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java deleted file mode 100644 index aeafee9ebc..0000000000 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/BulkDocumentWriterStub.java +++ /dev/null @@ -1,87 +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.elasticsearch.writer; - -import org.apache.metron.elasticsearch.bulk.BulkDocumentWriter; -import org.apache.metron.elasticsearch.bulk.BulkDocumentWriterResults; -import org.apache.metron.indexing.dao.update.Document; - -import java.util.ArrayList; -import java.util.List; -import java.util.Random; - -/** - * A {@link BulkDocumentWriter} stub that can pretend that all documents - * have been written successfully, that all documents have failed, or something - * in between those two extremes. - * - * @param The type of {@link Document} to write. - */ -public class BulkDocumentWriterStub implements BulkDocumentWriter { - - private float probabilityOfSuccess; - private Exception exception; - private List documents; - - /** - * @param probabilityOfSuccess The probability that a write will succeed. - * If 1.0, all writes succeed. If 0.0, all writes fail. - */ - public BulkDocumentWriterStub(float probabilityOfSuccess) { - this.probabilityOfSuccess = probabilityOfSuccess; - this.exception = new IllegalStateException("Exception created by a stub for testing"); - this.documents = new ArrayList<>(); - } - - @Override - public void addDocument(D document, String index) { - documents.add(document); - } - - @Override - public BulkDocumentWriterResults write() { - BulkDocumentWriterResults results = new BulkDocumentWriterResults<>(); - Random random = new Random(); - for(D document: documents) { - boolean success = random.nextFloat() <= probabilityOfSuccess; - if(success) { - results.addSuccess(document); - } else { - results.addFailure(document, exception, "error message"); - } - } - return results; - } - - @Override - public int size() { - return documents.size(); - } - - /** - * Set the exception that is passed to the failure callback when a message fails to write. - * @param exception The exception passed to the failure callback. - */ - public void setException(Exception exception) { - this.exception = exception; - } - - public Exception getException() { - return exception; - } -} diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java index 309588957d..e5e85b090f 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/writer/ElasticsearchWriterTest.java @@ -21,6 +21,8 @@ import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.common.writer.BulkWriterResponse; +import org.apache.metron.elasticsearch.bulk.BulkDocumentWriter; +import org.apache.metron.elasticsearch.bulk.BulkDocumentWriterResults; import org.apache.storm.task.TopologyContext; import org.apache.storm.tuple.Tuple; import org.json.simple.JSONObject; @@ -60,16 +62,20 @@ public void setup() { @Test public void shouldWriteSuccessfully() { - // create a writer where all writes will be successful - float probabilityOfSuccess = 1.0F; - ElasticsearchWriter esWriter = new ElasticsearchWriter(); - esWriter.setDocumentWriter( new BulkDocumentWriterStub<>(probabilityOfSuccess)); - esWriter.init(stormConf, topologyContext, writerConfiguration); - // create a tuple and a message associated with that tuple List tuples = createTuples(1); List messages = createMessages(1); + // create a document writer which will successfully write all + BulkDocumentWriterResults results = new BulkDocumentWriterResults<>(); + results.addSuccess(createDocument(messages.get(0), tuples.get(0))); + BulkDocumentWriter docWriter = mock(BulkDocumentWriter.class); + when(docWriter.write()).thenReturn(results); + + // attempt to write + ElasticsearchWriter esWriter = new ElasticsearchWriter(); + esWriter.setDocumentWriter(docWriter); + esWriter.init(stormConf, topologyContext, writerConfiguration); BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); // response should only contain successes @@ -79,16 +85,22 @@ public void shouldWriteSuccessfully() { @Test public void shouldWriteManySuccessfully() { - // create a writer where all writes will be successful - float probabilityOfSuccess = 1.0F; - ElasticsearchWriter esWriter = new ElasticsearchWriter(); - esWriter.setDocumentWriter(new BulkDocumentWriterStub<>(probabilityOfSuccess)); - esWriter.init(stormConf, topologyContext, writerConfiguration); - // create a few tuples and the messages associated with the tuples List tuples = createTuples(3); List messages = createMessages(3); + // create a document writer which will successfully write all + BulkDocumentWriterResults results = new BulkDocumentWriterResults<>(); + results.addSuccess(createDocument(messages.get(0), tuples.get(0))); + results.addSuccess(createDocument(messages.get(1), tuples.get(1))); + results.addSuccess(createDocument(messages.get(2), tuples.get(2))); + BulkDocumentWriter docWriter = mock(BulkDocumentWriter.class); + when(docWriter.write()).thenReturn(results); + + // attempt to write + ElasticsearchWriter esWriter = new ElasticsearchWriter(); + esWriter.setDocumentWriter(docWriter); + esWriter.init(stormConf, topologyContext, writerConfiguration); BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); // response should only contain successes @@ -100,45 +112,56 @@ public void shouldWriteManySuccessfully() { @Test public void shouldHandleWriteFailure() { - // create a writer where all writes will fail - float probabilityOfSuccess = 0.0F; - BulkDocumentWriterStub docWriter = new BulkDocumentWriterStub<>(probabilityOfSuccess); - - ElasticsearchWriter esWriter = new ElasticsearchWriter(); - esWriter.setDocumentWriter(docWriter); - esWriter.init(stormConf, topologyContext, writerConfiguration); - // create a tuple and a message associated with that tuple List tuples = createTuples(1); List messages = createMessages(1); + Exception cause = new Exception(); + + // create a document writer which will fail all writes + BulkDocumentWriterResults results = new BulkDocumentWriterResults<>(); + results.addFailure(createDocument(messages.get(0), tuples.get(0)), cause, "error"); + BulkDocumentWriter docWriter = mock(BulkDocumentWriter.class); + when(docWriter.write()).thenReturn(results); + // attempt to write + ElasticsearchWriter esWriter = new ElasticsearchWriter(); + esWriter.setDocumentWriter(docWriter); + esWriter.init(stormConf, topologyContext, writerConfiguration); BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); - // response should only contain failures - assertTrue(response.hasErrors()); - Collection errors = response.getErrors().get(docWriter.getException()); + // the writer response should only contain failures + assertEquals(0, response.getSuccesses().size()); + assertEquals(1, response.getErrors().size()); + Collection errors = response.getErrors().get(cause); assertTrue(errors.contains(tuples.get(0))); } @Test public void shouldHandleManyWriteFailures() { - // create a writer where all writes will fail - float probabilityOfSuccess = 0.0F; - BulkDocumentWriterStub docWriter = new BulkDocumentWriterStub<>(probabilityOfSuccess); + // create a few tuples and the messages associated with the tuples + int count = 3; + List tuples = createTuples(count); + List messages = createMessages(count); + Exception cause = new Exception(); + // create a document writer which will fail all writes + BulkDocumentWriterResults results = new BulkDocumentWriterResults<>(); + results.addFailure(createDocument(messages.get(0), tuples.get(0)), cause, "error"); + results.addFailure(createDocument(messages.get(1), tuples.get(1)), cause, "error"); + results.addFailure(createDocument(messages.get(2), tuples.get(2)), cause, "error"); + BulkDocumentWriter docWriter = mock(BulkDocumentWriter.class); + when(docWriter.write()).thenReturn(results); + + // attempt to write ElasticsearchWriter esWriter = new ElasticsearchWriter(); esWriter.setDocumentWriter(docWriter); esWriter.init(stormConf, topologyContext, writerConfiguration); - - // create a few tuples and the messages associated with the tuples - List tuples = createTuples(3); - List messages = createMessages(3); - BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); - // response should only contain failures - assertTrue(response.hasErrors()); - Collection errors = response.getErrors().get(docWriter.getException()); + // the writer response should only contain failures + assertEquals(0, response.getSuccesses().size()); + assertEquals(1, response.getErrors().size()); + Collection errors = response.getErrors().get(cause); assertTrue(errors.contains(tuples.get(0))); assertTrue(errors.contains(tuples.get(1))); assertTrue(errors.contains(tuples.get(2))); @@ -146,36 +169,36 @@ public void shouldHandleManyWriteFailures() { @Test public void shouldHandlePartialFailures() { - // create a writer where some will fails and some will succeed - float probabilityOfSuccess = 0.5F; - BulkDocumentWriterStub docWriter = new BulkDocumentWriterStub<>(probabilityOfSuccess); + // create a few tuples and the messages associated with the tuples + int count = 2; + List tuples = createTuples(count); + List messages = createMessages(count); + Exception cause = new Exception(); + // create a document writer that will fail one and succeed the other + BulkDocumentWriterResults results = new BulkDocumentWriterResults<>(); + results.addFailure(createDocument(messages.get(0), tuples.get(0)), cause, "error"); + results.addSuccess(createDocument(messages.get(1), tuples.get(1))); + BulkDocumentWriter docWriter = mock(BulkDocumentWriter.class); + when(docWriter.write()).thenReturn(results); + + // attempt to write ElasticsearchWriter esWriter = new ElasticsearchWriter(); esWriter.setDocumentWriter(docWriter); esWriter.init(stormConf, topologyContext, writerConfiguration); - - // create some tuples and the messages associated with the tuples - int count = 100; - List tuples = createTuples(count); - List messages = createMessages(count); - BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); // response should contain some successes and some failures - int successes = response.getSuccesses().size(); - int failures = response.getErrors().get(docWriter.getException()).size(); - assertTrue(response.hasErrors()); - assertTrue(successes > 0); - assertTrue(failures > 0); - assertEquals(count, successes + failures); + assertEquals(1, response.getSuccesses().size()); + assertEquals(1, response.getErrors().size()); + assertTrue(response.getErrors().get(cause).contains(tuples.get(0))); + assertTrue(response.getSuccesses().contains(tuples.get(1))); } @Test(expected = IllegalStateException.class) public void shouldCheckIfNumberOfMessagesMatchNumberOfTuples() { - // create a writer where all writes will be successful - float probabilityOfSuccess = 1.0F; ElasticsearchWriter esWriter = new ElasticsearchWriter(); - esWriter.setDocumentWriter( new BulkDocumentWriterStub<>(probabilityOfSuccess)); + esWriter.setDocumentWriter(mock(BulkDocumentWriter.class)); esWriter.init(stormConf, topologyContext, writerConfiguration); // there are 5 tuples and only 1 message; there should be 5 messages to match the number of tuples @@ -188,17 +211,29 @@ public void shouldCheckIfNumberOfMessagesMatchNumberOfTuples() { @Test public void shouldWriteSuccessfullyWhenMessageTimestampIsString() { - // create a writer where all writes will be successful - float probabilityOfSuccess = 1.0F; - ElasticsearchWriter esWriter = new ElasticsearchWriter(); - esWriter.setDocumentWriter(new BulkDocumentWriterStub<>(probabilityOfSuccess)); - esWriter.init(stormConf, topologyContext, writerConfiguration); - - // create a message where the timestamp is expressed as a string List tuples = createTuples(1); List messages = createMessages(1); + + // the timestamp is a String, rather than a Long messages.get(0).put(Constants.Fields.TIMESTAMP.getName(), new Long(System.currentTimeMillis()).toString()); + // create the document + JSONObject message = messages.get(0); + String timestamp = (String) message.get(Constants.Fields.TIMESTAMP.getName()); + String guid = (String) message.get(Constants.GUID); + String sensorType = (String) message.get(Constants.SENSOR_TYPE); + TupleBasedDocument document = new TupleBasedDocument(message, guid, sensorType, Long.parseLong(timestamp), tuples.get(0)); + + // create a document writer which will successfully write that document + BulkDocumentWriterResults results = new BulkDocumentWriterResults<>(); + results.addSuccess(document); + BulkDocumentWriter docWriter = mock(BulkDocumentWriter.class); + when(docWriter.write()).thenReturn(results); + + // attempt to write + ElasticsearchWriter esWriter = new ElasticsearchWriter(); + esWriter.setDocumentWriter(docWriter); + esWriter.init(stormConf, topologyContext, writerConfiguration); BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); // response should only contain successes @@ -208,17 +243,23 @@ public void shouldWriteSuccessfullyWhenMessageTimestampIsString() { @Test public void shouldWriteSuccessfullyWhenMissingGUID() { - // create a writer where all writes will be successful - float probabilityOfSuccess = 1.0F; - ElasticsearchWriter esWriter = new ElasticsearchWriter(); - esWriter.setDocumentWriter(new BulkDocumentWriterStub<>(probabilityOfSuccess)); - esWriter.init(stormConf, topologyContext, writerConfiguration); - - // create a message where the GUID is missing + // create a tuple and a message associated with that tuple List tuples = createTuples(1); List messages = createMessages(1); + + // remove the GUID from the message assertNotNull(messages.get(0).remove(Constants.GUID)); + // create a document writer which will successfully write all + BulkDocumentWriterResults results = new BulkDocumentWriterResults<>(); + results.addSuccess(createDocument(messages.get(0), tuples.get(0))); + BulkDocumentWriter docWriter = mock(BulkDocumentWriter.class); + when(docWriter.write()).thenReturn(results); + + // attempt to write + ElasticsearchWriter esWriter = new ElasticsearchWriter(); + esWriter.setDocumentWriter(docWriter); + esWriter.init(stormConf, topologyContext, writerConfiguration); BulkWriterResponse response = esWriter.write("bro", writerConfiguration, tuples, messages); // response should only contain successes @@ -226,11 +267,19 @@ public void shouldWriteSuccessfullyWhenMissingGUID() { assertTrue(response.getSuccesses().contains(tuples.get(0))); } + private TupleBasedDocument createDocument(JSONObject message, Tuple tuple) { + Long timestamp = (Long) message.get(Constants.Fields.TIMESTAMP.getName()); + String guid = (String) message.get(Constants.GUID); + String sensorType = (String) message.get(Constants.SENSOR_TYPE); + return new TupleBasedDocument(message, guid, sensorType, timestamp, tuple); + } + private JSONObject message() { JSONObject message = new JSONObject(); message.put(Constants.GUID, UUID.randomUUID().toString()); message.put(Constants.Fields.TIMESTAMP.getName(), System.currentTimeMillis()); message.put(Constants.Fields.SRC_ADDR.getName(), "192.168.1.1"); + message.put(Constants.SENSOR_TYPE, "sensor"); return message; } From 2dadcd0eb2b8c22fb60cbc1d057235f13c62313f Mon Sep 17 00:00:00 2001 From: Nick Allen Date: Tue, 11 Dec 2018 11:52:53 -0500 Subject: [PATCH 41/41] Fixed errant comment --- .../elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java index 66404a49d4..9e6e568ea7 100644 --- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java +++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/bulk/ElasticsearchBulkDocumentWriter.java @@ -89,7 +89,7 @@ public BulkDocumentWriterResults write() { handleBulkResponse(bulkResponse, documents, results); } catch(IOException e) { - // assume all documents have failed. notify the failure listeners + // assume all documents have failed for(Indexable indexable: documents) { D failed = indexable.document; results.addFailure(failed, e, ExceptionUtils.getRootCauseMessage(e));