From f4380781540314a923dbc9b1ce61d6dd6a0afdf6 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 20 Jul 2017 13:38:34 +0100 Subject: [PATCH 01/38] METRON-1051: Add update capability. DAO Layer is complete, lacking REST layer. --- .../metron/rest/MetronRestConstants.java | 3 + .../metron/rest/config/IndexConfig.java | 37 ++- .../elasticsearch/dao/ElasticsearchDao.java | 76 +++++- .../utils/ElasticsearchUtils.java | 29 ++- .../ElasticsearchMutationIntegrationTest.java | 223 ++++++++++++++++++ .../components/ElasticSearchComponent.java | 37 +++ .../apache/metron/hbase/TableProvider.java | 11 +- metron-platform/metron-indexing/pom.xml | 46 ++++ .../metron/indexing/dao/AccessConfig.java | 30 +++ .../apache/metron/indexing/dao/Document.java | 69 ++++++ .../apache/metron/indexing/dao/HBaseDao.java | 89 +++++++ .../apache/metron/indexing/dao/IndexDao.java | 42 ++++ .../metron/indexing/dao/IndexDaoFactory.java | 39 ++- .../indexing/dao/IndexUpdateCallback.java | 25 ++ .../metron/indexing/dao/MultiIndexDao.java | 138 +++++++++++ .../metron/indexing/mutation/Mutation.java | 63 +++++ .../indexing/mutation/MutationException.java | 24 ++ .../indexing/mutation/MutationOperation.java | 42 ++++ .../metron/indexing/mutation/Mutator.java | 27 +++ .../metron/indexing/mutation/Update.java | 23 ++ .../indexing/mutation/mutators/Patch.java | 40 ++++ .../indexing/mutation/mutators/Replace.java | 32 +++ .../metron/indexing/dao/InMemoryDao.java | 32 +++ .../dao/IndexingDaoIntegrationTest.java | 18 +- .../indexing/mutation/MutationTest.java | 119 ++++++++++ 25 files changed, 1281 insertions(+), 33 deletions(-) create mode 100644 metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMutationIntegrationTest.java create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/Document.java create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexUpdateCallback.java create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutation.java create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationException.java create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationOperation.java create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutator.java create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Update.java create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Patch.java create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Replace.java create mode 100644 metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/mutation/MutationTest.java diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java index 43aaeaece6..1e118c1ca2 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java @@ -61,4 +61,7 @@ public class MetronRestConstants { public static final String SEARCH_MAX_RESULTS = "search.max.results"; public static final String INDEX_DAO_IMPL = "index.dao.impl"; + public static final String INDEX_HBASE_TABLE_PROVIDER_IMPL = "index.hbase.provider"; + public static final String INDEX_HBASE_TABLE = "index.hbase.table"; + public static final String INDEX_HBASE_CF = "index.hbase.cf"; } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java index 2bfafea823..fab11e745c 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java @@ -17,6 +17,8 @@ */ package org.apache.metron.rest.config; +import org.apache.metron.hbase.HTableProvider; +import org.apache.metron.hbase.TableProvider; import org.apache.metron.indexing.dao.AccessConfig; import org.apache.metron.indexing.dao.IndexDao; import org.apache.metron.indexing.dao.IndexDaoFactory; @@ -50,17 +52,32 @@ public IndexConfig(Environment environment) { @Bean public IndexDao indexDao() { - String indexDaoImpl = environment.getProperty(MetronRestConstants.INDEX_DAO_IMPL, String.class, null); - int searchMaxResults = environment.getProperty(MetronRestConstants.SEARCH_MAX_RESULTS, Integer.class, -1); - AccessConfig config = new AccessConfig(); - config.setMaxSearchResults(searchMaxResults); - if(indexDaoImpl == null) { - throw new IllegalStateException("You must provide an index DAO implementation via the " + INDEX_DAO_IMPL + " config"); - } try { - return IndexDaoFactory.create(indexDaoImpl, globalConfigService.get(), config); - } catch (Exception e) { - throw new IllegalStateException("Unable to instantiate " + indexDaoImpl + ": " + e.getMessage(), e); + String hbaseProviderImpl = environment.getProperty(MetronRestConstants.INDEX_HBASE_TABLE_PROVIDER_IMPL, String.class, null); + String hbaseTable = environment.getProperty(MetronRestConstants.INDEX_HBASE_TABLE, String.class, null); + String hbaseCf = environment.getProperty(MetronRestConstants.INDEX_HBASE_CF, String.class, null); + String indexDaoImpl = environment.getProperty(MetronRestConstants.INDEX_DAO_IMPL, String.class, null); + int searchMaxResults = environment.getProperty(MetronRestConstants.SEARCH_MAX_RESULTS, Integer.class, -1); + AccessConfig config = new AccessConfig(); + config.setMaxSearchResults(searchMaxResults); + config.setTable(hbaseTable); + config.setColumnFamily(hbaseCf); + config.setTableProvider(TableProvider.create(hbaseProviderImpl, () -> new HTableProvider())); + if (indexDaoImpl == null) { + throw new IllegalStateException("You must provide an index DAO implementation via the " + INDEX_DAO_IMPL + " config"); + } + IndexDao ret = IndexDaoFactory.combine(IndexDaoFactory.create(indexDaoImpl, globalConfigService.get(), config)); + if (ret == null) { + throw new IllegalStateException("IndexDao is unable to be created."); + } + return ret; + } + catch(RuntimeException re) { + throw re; + } + catch(Exception e) { + throw new IllegalStateException("Unable to create index DAO: " + e.getMessage(), e); } } + } \ No newline at end of file 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 a4838b5cfa..9f94326334 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 @@ -17,17 +17,34 @@ */ package org.apache.metron.elasticsearch.dao; +import com.google.common.base.Splitter; +import com.google.common.collect.Iterables; +import org.apache.metron.common.Constants; +import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; +import org.apache.metron.indexing.dao.Document; import org.apache.metron.indexing.dao.IndexDao; import org.apache.metron.indexing.dao.search.*; +import org.apache.metron.indexing.dao.search.SearchRequest; +import org.apache.metron.indexing.dao.search.SearchResponse; import org.apache.metron.indexing.dao.search.SortOrder; +import org.elasticsearch.action.search.*; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.index.query.QueryStringQueryBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.sort.*; - +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.client.transport.TransportClient; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHitField; +import org.elasticsearch.search.SearchHits; +import java.io.IOException; import java.util.Arrays; +import java.util.Date; import java.util.Map; import java.util.stream.Collectors; @@ -83,8 +100,59 @@ public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchEx } @Override - public void init(Map globalConfig, AccessConfig config) { - this.client = ElasticsearchUtils.getClient(globalConfig, config.getOptionalSettings()); - this.accessConfig = config; + public synchronized void init(Map globalConfig, AccessConfig config) { + if(this.client == null) { + this.client = ElasticsearchUtils.getClient(globalConfig, config.getOptionalSettings()); + this.accessConfig = config; + } + } + + @Override + public Document getLatest(String uuid, String sensorType) throws IOException { + QueryBuilder query = QueryBuilders.matchQuery(Constants.GUID, uuid); + SearchRequestBuilder request = client.prepareSearch() + .setTypes(sensorType + "_doc") + .setQuery(query) + .setSource("message") + ; + MultiSearchResponse response = client.prepareMultiSearch() + .add(request) + .get(); + //TODO: Fix this to + // * handle multiple responses + // * be more resilient to error + for(MultiSearchResponse.Item i : response) { + org.elasticsearch.action.search.SearchResponse resp = i.getResponse(); + SearchHits hits = resp.getHits(); + for(SearchHit hit : hits) { + Long ts = 0L; + String doc = hit.getSourceAsString(); + String sourceType = Iterables.getFirst(Splitter.on("_doc").split(hit.getType()), null); + Document d = new Document(doc, uuid, sourceType, ts); + return d; + } + } + return null; } + + @Override + public void update(Document update, WriterConfiguration configurations) throws IOException { + String indexPostfix = ElasticsearchUtils.getIndexFormat(configurations).format(new Date()); + String sensorType = update.getSensorType(); + String indexName = ElasticsearchUtils.getIndexName(sensorType, indexPostfix, configurations); + IndexRequestBuilder indexRequestBuilder = client.prepareIndex(indexName, + sensorType + "_doc"); + + indexRequestBuilder = indexRequestBuilder.setSource(update.getDocument()); + Object ts = update.getTimestamp(); + if(ts != null) { + indexRequestBuilder = indexRequestBuilder.setTimestamp(ts.toString()); + } + + BulkResponse bulkResponse = client.prepareBulk().add(indexRequestBuilder).execute().actionGet(); + if(bulkResponse.hasFailures()) { + throw new IOException(bulkResponse.buildFailureMessage()); + } + } + } 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 d1994039f4..c7c4d90bb8 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 @@ -20,6 +20,7 @@ import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.elasticsearch.writer.ElasticsearchWriter; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.common.settings.Settings; @@ -27,14 +28,32 @@ import java.net.InetAddress; import java.net.UnknownHostException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; +import java.text.SimpleDateFormat; +import java.util.*; public class ElasticsearchUtils { + private static ThreadLocal> DATE_FORMAT_CACHE + = ThreadLocal.withInitial(() -> new HashMap<>()); + + 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); + } + + public static String getIndexName(String sensorType, String indexPostfix, WriterConfiguration configurations) { + String indexName = sensorType; + if (configurations != null) { + indexName = configurations.getIndex(sensorType); + } + indexName = indexName + "_index_" + indexPostfix; + return indexName; + } + public static TransportClient getClient(Map globalConfiguration, Map optionalSettings) { Settings.Builder settingsBuilder = Settings.settingsBuilder(); settingsBuilder.put("cluster.name", globalConfiguration.get("es.clustername")); diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMutationIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMutationIntegrationTest.java new file mode 100644 index 0000000000..472d2aa92d --- /dev/null +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMutationIntegrationTest.java @@ -0,0 +1,223 @@ +/** + * 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.integration; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.google.common.collect.Iterables; +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.HTableInterface; +import org.apache.hadoop.hbase.client.Result; +import org.apache.metron.common.Constants; +import org.apache.metron.common.configuration.writer.WriterConfiguration; +import org.apache.metron.common.utils.JSONUtils; +import org.apache.metron.elasticsearch.dao.ElasticsearchDao; +import org.apache.metron.elasticsearch.integration.components.ElasticSearchComponent; +import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; +import org.apache.metron.elasticsearch.writer.ElasticsearchWriter; +import org.apache.metron.hbase.TableProvider; +import org.apache.metron.indexing.dao.*; +import org.apache.metron.test.mock.MockHTable; +import org.apache.metron.indexing.mutation.Mutation; +import org.apache.metron.indexing.mutation.MutationOperation; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.*; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class ElasticsearchMutationIntegrationTest { + private static final int MAX_RETRIES = 10; + private static final int SLEEP_MS = 500; + private static final String SENSOR_NAME= "test"; + private static final String TABLE_NAME = "modifications"; + private static final String CF = "p"; + private static String indexDir = "target/elasticsearch_mutation"; + private static String dateFormat = "yyyy.MM.dd.HH"; + private static String index = SENSOR_NAME + "_index_" + new SimpleDateFormat(dateFormat).format(new Date()); + private static MockHTable table; + private static IndexDao esDao; + private static IndexDao hbaseDao; + private static MultiIndexDao dao; + private static WriterConfiguration configurations; + private static ElasticSearchComponent es; + + @BeforeClass + public static void setup() throws Exception { + Configuration config = HBaseConfiguration.create(); + MockHTable.Provider tableProvider = new MockHTable.Provider(); + tableProvider.addToCache(TABLE_NAME, CF); + table = (MockHTable)tableProvider.getTable(config, TABLE_NAME); + // setup the client + es = new ElasticSearchComponent.Builder() + .withHttpPort(9211) + .withIndexDir(new File(indexDir)) + .build(); + es.start(); + + hbaseDao = new HBaseDao(); + AccessConfig accessConfig = new AccessConfig(); + accessConfig.setColumnFamily(CF); + accessConfig.setTable(TABLE_NAME); + accessConfig.setTableProvider((TableProvider) (config1, tableName) -> table); + Map globalConfig = new HashMap() {{ + put("es.clustername", "metron"); + put("es.port", "9300"); + put("es.ip", "localhost"); + put("es.date.format", dateFormat); + }}; + + esDao = new ElasticsearchDao(); + + dao = new MultiIndexDao(hbaseDao, esDao); + dao.init(globalConfig, accessConfig); + configurations = mock(WriterConfiguration.class); + when(configurations.getIndex(any())).thenReturn(SENSOR_NAME); + when(configurations.getGlobalConfig()).thenReturn(globalConfig); + } + + @AfterClass + public static void teardown() { + if(es != null) { + es.stop(); + } + } + + + + @Test + 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); + }} + ); + } + es.add(index, SENSOR_NAME + , Iterables.transform(inputData, + m -> { + try { + return JSONUtils.INSTANCE.toJSON(m, true); + } catch (JsonProcessingException e) { + throw new IllegalStateException(e.getMessage(), e); + } + } + ) + ); + List> docs = null; + for(int t = 0;t < MAX_RETRIES;++t, Thread.sleep(SLEEP_MS)) { + docs = es.getAllIndexedDocs(index, SENSOR_NAME + "_doc"); + 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 message0Json = JSONUtils.INSTANCE.toJSON(message0, true); + String uuid = "" + message0.get(Constants.GUID); + Mutation mutation = Mutation.of(MutationOperation.REPLACE, message0Json); + dao.update(uuid, SENSOR_NAME, mutation, Optional.empty(), configurations); + Assert.assertEquals(1, table.size()); + Document doc = dao.getLatest(uuid, SENSOR_NAME); + Assert.assertEquals(message0Json, doc.getDocument()); + { + //ensure hbase is up to date + Get g = new Get(uuid.getBytes()); + Result r = table.get(g); + NavigableMap columns = r.getFamilyMap(CF.getBytes()); + Assert.assertEquals(1, columns.size()); + Assert.assertEquals(message0Json, new String(columns.lastEntry().getValue())); + } + { + //ensure ES is up-to-date + long cnt = 0; + for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t, Thread.sleep(SLEEP_MS)) { + docs = es.getAllIndexedDocs(index, SENSOR_NAME + "_doc"); + cnt = docs + .stream() + .filter(d -> { + Object newfield = d.get("new-field"); + return newfield != null && newfield.equals(message0.get("new-field")); + }).count(); + } + if (cnt == 0) { + Assert.fail("Elasticsearch is not updated!"); + } + } + } + //modify the same message and modify the new field + { + Map message0 = new HashMap(inputData.get(0)) {{ + put("new-field", "metron2"); + }}; + String message0Json = JSONUtils.INSTANCE.toJSON(message0, true); + String uuid = "" + message0.get(Constants.GUID); + Mutation mutation = Mutation.of(MutationOperation.REPLACE, message0Json); + dao.update(uuid, SENSOR_NAME, mutation, Optional.empty(), configurations); + Assert.assertEquals(1, table.size()); + Document doc = dao.getLatest(uuid, SENSOR_NAME); + Assert.assertEquals(message0Json, doc.getDocument()); + { + //ensure hbase is up to date + Get g = new Get(uuid.getBytes()); + Result r = table.get(g); + NavigableMap columns = r.getFamilyMap(CF.getBytes()); + Assert.assertEquals(2, columns.size()); + Assert.assertEquals(message0Json, new String(columns.lastEntry().getValue())); + Assert.assertNotEquals(message0Json, new String(columns.firstEntry().getValue())); + } + { + //ensure ES is up-to-date + long cnt = 0; + for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t,Thread.sleep(SLEEP_MS)) { + docs = es.getAllIndexedDocs(index, SENSOR_NAME + "_doc"); + cnt = docs + .stream() + .filter(d -> { + Object newfield = d.get("new-field"); + return newfield != null && newfield.equals(message0.get("new-field")); + }).count(); + } + if (cnt == 0) { + Assert.fail("Elasticsearch is not updated!"); + } + } + + } + } + +} diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java index 7766fe30eb..abddf774a6 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 @@ -17,13 +17,18 @@ */ package org.apache.metron.elasticsearch.integration.components; +import com.fasterxml.jackson.core.type.TypeReference; import org.apache.commons.io.FileUtils; +import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.integration.InMemoryComponent; import org.apache.metron.integration.UnableToStartException; import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction; import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; @@ -90,6 +95,38 @@ private void cleanDir(File dir) throws IOException { } dir.mkdirs(); } + + public BulkResponse add(String indexName, String sensorType, String... docs) throws IOException { + List d = new ArrayList<>(); + for(String doc : docs) { + d.add(doc); + } + return add(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, new TypeReference>() { + }); + Object ts = esDoc.get("timestamp"); + if(ts != null) { + indexRequestBuilder = indexRequestBuilder.setTimestamp(ts.toString()); + } + bulkRequest.add(indexRequestBuilder); + } + + BulkResponse response = bulkRequest.execute().actionGet(); + if(response.hasFailures()) { + throw new IOException(response.buildFailureMessage()); + } + return response; + } + @Override public void start() throws UnableToStartException { File logDir= new File(indexDir, "/logs"); diff --git a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/TableProvider.java b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/TableProvider.java index dc0569e844..1804697fc7 100644 --- a/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/TableProvider.java +++ b/metron-platform/metron-hbase/src/main/java/org/apache/metron/hbase/TableProvider.java @@ -22,7 +22,16 @@ import java.io.IOException; import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; +import java.util.function.Supplier; public interface TableProvider extends Serializable { - HTableInterface getTable(Configuration config, String tableName) throws IOException; + HTableInterface getTable(Configuration config, String tableName) throws IOException; + static TableProvider create(String impl, Supplier defaultSupplier) throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException, InstantiationException { + if(impl == null) { + return defaultSupplier.get(); + } + Class clazz = (Class) Class.forName(impl); + return clazz.getConstructor().newInstance(); + } } diff --git a/metron-platform/metron-indexing/pom.xml b/metron-platform/metron-indexing/pom.xml index b0127bb35b..c64c374c06 100644 --- a/metron-platform/metron-indexing/pom.xml +++ b/metron-platform/metron-indexing/pom.xml @@ -36,6 +36,32 @@ metron-common ${project.parent.version} + + org.apache.metron + metron-hbase + ${project.parent.version} + + + org.apache.hbase + hbase-client + ${global_hbase_version} + provided + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + com.google.guava + guava + + + + org.apache.hadoop hadoop-hdfs @@ -47,7 +73,27 @@ provided + + + com.flipkart.zjsonpatch + zjsonpatch + 0.3.1 + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-core + + + org.apache.storm storm-core diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java index dd6848449d..ee713f7c48 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java @@ -17,12 +17,18 @@ */ package org.apache.metron.indexing.dao; +import org.apache.metron.hbase.HTableProvider; +import org.apache.metron.hbase.TableProvider; + import java.util.HashMap; import java.util.Map; public class AccessConfig { private Integer maxSearchResults; private Map optionalSettings = new HashMap<>(); + private TableProvider tableProvider = null; + private String columnFamily; + private String table; public Integer getMaxSearchResults() { return maxSearchResults; @@ -39,4 +45,28 @@ public Map getOptionalSettings() { public void setOptionalSettings(Map optionalSettings) { this.optionalSettings = optionalSettings; } + + public TableProvider getTableProvider() { + return tableProvider; + } + + public void setTableProvider(TableProvider tableProvider) { + this.tableProvider = tableProvider; + } + + public String getColumnFamily() { + return columnFamily; + } + + public void setColumnFamily(String columnFamily) { + this.columnFamily = columnFamily; + } + + public String getTable() { + return table; + } + + public void setTable(String table) { + this.table = table; + } } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/Document.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/Document.java new file mode 100644 index 0000000000..76b45ac471 --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/Document.java @@ -0,0 +1,69 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.metron.indexing.dao; + +public class Document { + Long timestamp; + String document; + String uuid; + String sensorType; + + public Document(String document, String uuid, String sensorType, Long timestamp) { + setDocument(document); + setUuid(uuid); + setTimestamp(timestamp); + setSensorType(sensorType); + } + + public Document(String document, String uuid, String sensorType) { + this( document, uuid, sensorType, null); + } + + public String getSensorType() { + return sensorType; + } + + public void setSensorType(String sensorType) { + this.sensorType = sensorType; + } + + public Long getTimestamp() { + return timestamp; + } + + public void setTimestamp(Long timestamp) { + this.timestamp = timestamp != null?timestamp:System.currentTimeMillis(); + } + + public String getDocument() { + return document; + } + + public void setDocument(String document) { + this.document = document; + } + + public String getUuid() { + return uuid; + } + + public void setUuid(String uuid) { + this.uuid = uuid; + } +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java new file mode 100644 index 0000000000..0a7c1590dd --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java @@ -0,0 +1,89 @@ +/** + * 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.indexing.dao; + +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.HTableInterface; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.metron.common.configuration.writer.WriterConfiguration; +import org.apache.metron.indexing.dao.search.InvalidSearchException; +import org.apache.metron.indexing.dao.search.SearchRequest; +import org.apache.metron.indexing.dao.search.SearchResponse; + +import java.io.IOException; +import java.util.Map; +import java.util.NavigableMap; + +public class HBaseDao implements IndexDao { + private HTableInterface tableInterface; + private byte[] cf; + + public HBaseDao() { + + } + + @Override + public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException { + return null; + } + + @Override + public synchronized void init(Map globalConfig, AccessConfig config) { + if(tableInterface == null) { + try { + tableInterface = config.getTableProvider().getTable(HBaseConfiguration.create(), config.getTable()); + cf = config.getColumnFamily().getBytes(); + } catch (IOException e) { + throw new IllegalStateException("Unable to initialize HBaseDao: " + e.getMessage(), e); + } + } + } + + @Override + public Document getLatest(String uuid, String sensorType) throws IOException { + Get get = new Get(uuid.getBytes()); + get.addFamily(cf); + Result result = tableInterface.get(get); + NavigableMap columns = result.getFamilyMap( cf); + if(columns == null || columns.size() == 0) { + return null; + } + Map.Entry entry= columns.lastEntry(); + Long ts = Bytes.toLong(entry.getKey()); + if(entry.getValue()!= null) { + String json = new String(entry.getValue()); + return new Document(json, uuid, sensorType, ts); + } + else { + return null; + } + } + + @Override + public void update(Document update, WriterConfiguration configurations) throws IOException { + Put put = new Put(update.getUuid().getBytes()); + long ts = update.getTimestamp() == null?System.currentTimeMillis():update.getTimestamp(); + byte[] columnQualifier = Bytes.toBytes(ts); + put.addColumn(cf, columnQualifier, Bytes.toBytes(update.getDocument())); + tableInterface.put(put); + } +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java index a835d65426..d53be2ef0d 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java @@ -17,13 +17,55 @@ */ package org.apache.metron.indexing.dao; +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.metron.common.configuration.writer.WriterConfiguration; +import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.indexing.dao.search.InvalidSearchException; import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; +import org.apache.metron.indexing.mutation.Mutation; +import org.apache.metron.indexing.mutation.MutationException; +import java.io.IOException; import java.util.Map; +import java.util.Optional; public interface IndexDao { SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException; void init(Map globalConfig, AccessConfig config); + Document getLatest(String uuid, String sensorType) throws IOException; + void update(Document update, WriterConfiguration configurations) throws IOException; + + default void update( final Document original + , Mutation mutation + , Optional timestamp + , WriterConfiguration configurations + ) throws IOException, MutationException + { + String mutated = null; + try { + mutated = + mutation.apply(() -> { + try { + return JSONUtils.INSTANCE.load(original.document, JsonNode.class); + } catch (IOException e) { + throw new IllegalStateException(e.getMessage(), e); + } + }); + } + catch(Exception ex) { + throw new MutationException(ex.getMessage(), ex); + } + Document updated = new Document(mutated, original.getUuid(), original.getSensorType(), timestamp.orElse(null)); + update(updated, configurations); + } + + default void update(String uuid, String sensorType, Mutation mutation, Optional timestamp, WriterConfiguration configurations) throws IOException, MutationException + { + Document latest = getLatest(uuid, sensorType); + if(latest == null) { + throw new IllegalStateException("Unable to retrieve message with UUID: " + uuid + " please use the update() method that specifies the document."); + } + update(latest, mutation, timestamp, configurations); + } } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDaoFactory.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDaoFactory.java index 9c2de0ee2d..946fd99e5d 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDaoFactory.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDaoFactory.java @@ -17,14 +17,43 @@ */ package org.apache.metron.indexing.dao; +import com.google.common.base.Splitter; +import com.google.common.collect.Iterables; + import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.List; import java.util.Map; +import java.util.function.Function; public class IndexDaoFactory { - public static IndexDao create(String daoImpl, Map globalConfig, AccessConfig config) throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException, InstantiationException { - Class clazz = (Class) Class.forName(daoImpl); - IndexDao instance = clazz.getConstructor().newInstance(); - instance.init(globalConfig, config); - return instance; + public static List create( String daoImpls + , Map globalConfig + , AccessConfig config + ) throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException, InstantiationException + { + List ret = new ArrayList<>(); + for(String daoImpl : Splitter.on(",").split(daoImpls)) { + Class clazz = (Class) Class.forName(daoImpl); + IndexDao instance = clazz.getConstructor().newInstance(); + instance.init(globalConfig, config); + ret.add(instance); + } + return ret; + } + + public static IndexDao combine(Iterable daos) throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException, InstantiationException { + return combine(daos, x -> x); + } + + public static IndexDao combine(Iterable daos, Function daoTransformation) throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException, InstantiationException { + int numDaos = Iterables.size(daos); + if(numDaos == 0) { + throw new IllegalArgumentException("Trying to combine 0 dao's into a DAO is not a supported configuration."); + } + if( numDaos == 1) { + return daoTransformation.apply(Iterables.getFirst(daos, null)); + } + return new MultiIndexDao(daos, daoTransformation); } } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexUpdateCallback.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexUpdateCallback.java new file mode 100644 index 0000000000..1022a23e1d --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexUpdateCallback.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.metron.indexing.dao; + +import java.io.IOException; + +public interface IndexUpdateCallback { + void postUpdate(IndexDao dao, Document doc) throws IOException; +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java new file mode 100644 index 0000000000..976d281419 --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java @@ -0,0 +1,138 @@ +/** + * 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.indexing.dao; + +import com.google.common.base.Joiner; +import com.google.common.collect.Iterables; +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.metron.common.configuration.writer.WriterConfiguration; +import org.apache.metron.indexing.dao.search.InvalidSearchException; +import org.apache.metron.indexing.dao.search.SearchRequest; +import org.apache.metron.indexing.dao.search.SearchResponse; + +import java.io.IOException; +import java.util.*; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class MultiIndexDao implements IndexDao { + private List indices; + + public MultiIndexDao( IndexDao... composedDao) { + indices = new ArrayList<>(); + for(IndexDao dao: composedDao) { + indices.add(dao); + } + } + + public MultiIndexDao(Iterable composedDao) { + this.indices = new ArrayList<>(); + Iterables.addAll(indices, composedDao); + } + + public MultiIndexDao(Iterable composedDao, Function decoratorTransformation) { + this(Iterables.transform(composedDao, x -> decoratorTransformation.apply(x))); + } + + @Override + public void update(final Document update, WriterConfiguration configurations) throws IOException { + List exceptions = + indices.parallelStream().map(dao -> { + try { + dao.update(update, configurations); + return null; + } catch (Throwable e) { + return dao.getClass() + ": " + e.getMessage() + "\n" + ExceptionUtils.getStackTrace(e); + } + }).filter(e -> e != null).collect(Collectors.toList()); + if(exceptions.size() > 0) { + throw new IOException(Joiner.on("\n").join(exceptions)); + } + } + + private static class DocumentContainer { + private Optional d = Optional.empty(); + private Optional t = Optional.empty(); + public DocumentContainer(Document d) { + this.d = Optional.ofNullable(d); + } + public DocumentContainer(Throwable t) { + this.t = Optional.ofNullable(t); + } + + public Optional getDocument() { + return d; + } + public Optional getException() { + return t; + } + + } + + @Override + public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException { + for(IndexDao dao : indices) { + SearchResponse s = dao.search(searchRequest); + if(s != null) { + return s; + } + } + return null; + } + + @Override + public void init(Map globalConfig, AccessConfig config) { + for(IndexDao dao : indices) { + dao.init(globalConfig, config); + } + } + + @Override + public Document getLatest(final String uuid, String sensorType) throws IOException { + Document ret = null; + List output = + indices.parallelStream().map(dao -> { + try { + return new DocumentContainer(dao.getLatest(uuid, sensorType)); + } catch (Throwable e) { + return new DocumentContainer(e); + } + }).collect(Collectors.toList()); + + List error = new ArrayList<>(); + for(DocumentContainer dc : output) { + if(dc.getException().isPresent()) { + Throwable e = dc.getException().get(); + error.add(e.getMessage() + "\n" + ExceptionUtils.getStackTrace(e)); + } + else { + if(dc.getDocument().isPresent()) { + Document d = dc.getDocument().get(); + if(ret == null || ret.getTimestamp() < d.getTimestamp()) { + ret = d; + } + } + } + } + if(error.size() > 0) { + throw new IOException(Joiner.on("\n").join(error)); + } + return ret; + } +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutation.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutation.java new file mode 100644 index 0000000000..a29736c96f --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutation.java @@ -0,0 +1,63 @@ +/** + * 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.indexing.mutation; + +import com.fasterxml.jackson.databind.JsonNode; + +import java.io.Serializable; +import java.util.function.Supplier; + +public class Mutation implements Serializable { + MutationOperation mutator; + String mutationArg; + + public Mutation(MutationOperation mutator, String mutationArg) { + this.mutator = mutator; + this.mutationArg = mutationArg; + } + + public static Mutation of(MutationOperation mutator, String mutationArg) { + return new Mutation(mutator, mutationArg); + } + + /** + * Applies this function to the given argument. + * + * @param jsonNodeSupplier the function argument + * @return the function result + */ + public String apply(Supplier jsonNodeSupplier) throws MutationException { + return mutator.mutate(jsonNodeSupplier, mutationArg); + } + + public MutationOperation getMutator() { + return mutator; + } + + public void setMutator(MutationOperation mutator) { + this.mutator = mutator; + } + + public String getMutationArg() { + return mutationArg; + } + + public void setMutationArg(String mutationArg) { + this.mutationArg = mutationArg; + } +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationException.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationException.java new file mode 100644 index 0000000000..b0ee817234 --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationException.java @@ -0,0 +1,24 @@ +/** + * 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.indexing.mutation; + +public class MutationException extends Exception { + public MutationException(String s, Exception e) { + super(s, e); + } +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationOperation.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationOperation.java new file mode 100644 index 0000000000..6beeac92ac --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationOperation.java @@ -0,0 +1,42 @@ +/** + * 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.indexing.mutation; + +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.metron.indexing.mutation.mutators.Patch; +import org.apache.metron.indexing.mutation.mutators.Replace; + +import java.util.function.Supplier; + +public enum MutationOperation implements Mutator{ + PATCH(new Patch()), + REPLACE(new Replace()) + ; + + Mutator mutator; + + MutationOperation(Mutator mutator) { + this.mutator = mutator; + } + + @Override + public String mutate(Supplier originalNode, String arg) throws MutationException { + return this.mutator.mutate(originalNode, arg); + } +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutator.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutator.java new file mode 100644 index 0000000000..db3b50a3ab --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutator.java @@ -0,0 +1,27 @@ +/** + * 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.indexing.mutation; + +import com.fasterxml.jackson.databind.JsonNode; + +import java.util.function.Supplier; + +public interface Mutator { + String mutate(Supplier original, String arg) throws MutationException; +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Update.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Update.java new file mode 100644 index 0000000000..e4995bef65 --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Update.java @@ -0,0 +1,23 @@ +/** + * 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.indexing.mutation; + +public class Update { + +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Patch.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Patch.java new file mode 100644 index 0000000000..b4be82c83f --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Patch.java @@ -0,0 +1,40 @@ +/** + * 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.indexing.mutation.mutators; + +import com.fasterxml.jackson.databind.JsonNode; +import com.flipkart.zjsonpatch.JsonPatch; +import org.apache.metron.common.utils.JSONUtils; +import org.apache.metron.indexing.mutation.MutationException; +import org.apache.metron.indexing.mutation.Mutator; + +import java.util.function.Supplier; + +public class Patch implements Mutator { + @Override + public String mutate(Supplier originalSupplier, String arg) throws MutationException { + final JsonNode orig = originalSupplier.get(); + try { + JsonNode out = JsonPatch.apply(JSONUtils.INSTANCE.load(arg, JsonNode.class), orig); + return new String(JSONUtils.INSTANCE.toJSON(out)); + + } catch (Exception e) { + throw new MutationException("Unable to mutate: " + orig.asText() + " with " + arg + " because " + e.getMessage(), e); + } + } +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Replace.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Replace.java new file mode 100644 index 0000000000..2503d0ba1b --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Replace.java @@ -0,0 +1,32 @@ +/** + * 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.indexing.mutation.mutators; + +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.metron.indexing.mutation.MutationException; +import org.apache.metron.indexing.mutation.Mutator; + +import java.util.function.Supplier; + +public class Replace implements Mutator { + + @Override + public String mutate(Supplier original, String arg) throws MutationException { + return arg; + } +} diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java index 444a9da2c7..07c4bcbc9e 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java @@ -22,6 +22,7 @@ import com.google.common.collect.ComparisonChain; import com.google.common.collect.Iterables; import org.apache.metron.common.Constants; +import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.indexing.dao.search.*; @@ -132,6 +133,37 @@ public void init(Map globalConfig, AccessConfig config) { this.config = config; } + @Override + public Document getLatest(String uuid, String sensorType) throws IOException { + for(Map.Entry> kv: BACKING_STORE.entrySet()) { + if(kv.getKey().startsWith(sensorType)) { + for(String doc : kv.getValue()) { + Map docParsed = parse(doc); + if(docParsed.getOrDefault(Constants.GUID, "").equals(uuid)) { + return new Document(doc, uuid, sensorType, 0L); + } + } + } + } + return null; + } + + @Override + public void update(Document update, WriterConfiguration configurations) throws IOException { + for(Map.Entry> kv: BACKING_STORE.entrySet()) { + if (kv.getKey().startsWith(update.getSensorType())) { + for(Iterator it = kv.getValue().iterator();it.hasNext();) { + String doc = it.next(); + Map docParsed = parse(doc); + if(docParsed.getOrDefault(Constants.GUID, "").equals(update.getUuid())) { + it.remove(); + } + } + kv.getValue().add(update.getDocument()); + } + } + } + public static void load(Map> backingStore) { BACKING_STORE = backingStore; } diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/IndexingDaoIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/IndexingDaoIntegrationTest.java index 8b5baef878..f469f1f95f 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/IndexingDaoIntegrationTest.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/IndexingDaoIntegrationTest.java @@ -156,14 +156,16 @@ public abstract class IndexingDaoIntegrationTest { @Multiline public static String exceededMaxResultsQuery; - protected IndexDao dao; - protected InMemoryComponent indexComponent; + protected static IndexDao dao; + protected static InMemoryComponent indexComponent; @Before - public void setup() throws Exception { - indexComponent = startIndex(); - loadTestData(); - dao = createDao(); + public synchronized void setup() throws Exception { + if(dao == null && indexComponent == null) { + indexComponent = startIndex(); + loadTestData(); + dao = createDao(); + } } @Test @@ -244,8 +246,8 @@ public void test() throws Exception { } } - @After - public void stop() throws Exception { + @AfterClass + public static void stop() throws Exception { indexComponent.stop(); } diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/mutation/MutationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/mutation/MutationTest.java new file mode 100644 index 0000000000..5d65f6f435 --- /dev/null +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/mutation/MutationTest.java @@ -0,0 +1,119 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.metron.indexing.mutation; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.Sets; +import org.adrianwalker.multilinestring.Multiline; +import org.apache.metron.common.utils.JSONUtils; +import org.json.simple.JSONObject; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class MutationTest { + + @Test + public void testReplace() throws Exception { + Mutation m = Mutation.of(MutationOperation.REPLACE, "{ \"a\" : 1 }"); + String out = m.apply( + () -> { + try { + return JSONUtils.INSTANCE.load("{ \"b\" : 1 }", JsonNode.class); + } catch (IOException e) { + throw new IllegalStateException(e); + } + } + + ); + Assert.assertEquals(m.mutationArg, out); + } + + /** + [ + { "op": "add" + , "path": "/b" + , "value": "metron" + } + ] + */ + @Multiline + public static String addElement; + + @Test + public void testPatch_add() throws Exception { + final Map orig = new HashMap() {{ + put("a", 1); + }}; + Map out = apply(Mutation.of(MutationOperation.PATCH, addElement), orig); + mapEquals(out, new HashMap() {{ + putAll(orig); + put("b", "metron"); + }}); + } + + + /** + [ + { "op": "add" + , "path": "/b" + , "value": "metron" + }, + { "op": "remove" + , "path": "/a" + } + ] + */ + @Multiline + public static String addAndThenRemoveElement; + + @Test + public void testPatch_addThenRemove() throws Exception { + final Map orig = new HashMap() {{ + put("a", 1); + }}; + Map out = apply(Mutation.of(MutationOperation.PATCH, addAndThenRemoveElement) , orig); + mapEquals(out, new HashMap() {{ + put("b", "metron"); + }}); + } + + public static Map apply(Mutation m, Map orig) throws Exception { + String out = m.apply( + () -> { + try { + String origStr = new JSONObject(orig).toJSONString(); + return JSONUtils.INSTANCE.load(origStr, JsonNode.class); + } catch (IOException e) { + throw new IllegalStateException(e); + } + } + + ); + return JSONUtils.INSTANCE.load(out, new TypeReference>() { + }); + } + + public static void mapEquals(Map m1, Map m2) { + Assert.assertEquals(0, Sets.symmetricDifference(m1.entrySet(), m2.entrySet()).size()); + } +} From a29945c704c375f5ac56e8d63ab73a34694790ce Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 20 Jul 2017 14:18:20 +0100 Subject: [PATCH 02/38] Updating REST and testing to consolidate one MockHTableProvider --- .../metron-profiler-client/pom.xml | 13 +++++++ .../profiler/client/GetProfileTest.java | 19 +++-------- .../client/HBaseProfilerClientTest.java | 4 +-- metron-analytics/metron-profiler/pom.xml | 13 +++++++ .../integration/ProfilerIntegrationTest.java | 25 ++++---------- metron-interface/metron-rest/pom.xml | 7 ++++ .../src/main/resources/application-test.yml | 7 +++- .../SearchControllerIntegrationTest.java | 2 ++ .../ElasticsearchMutationIntegrationTest.java | 4 --- metron-platform/metron-enrichment/pom.xml | 7 ++++ .../simplehbase/SimpleHBaseAdapterTest.java | 7 ++-- .../threatintel/ThreatIntelAdapterTest.java | 7 ++-- .../EnrichmentIntegrationTest.java | 20 ++++------- .../SimpleHBaseEnrichmentFunctionsTest.java | 13 +++---- metron-platform/metron-hbase/pom.xml | 16 +++++++++ .../apache/metron/hbase}/mock/MockHTable.java | 25 ++------------ .../metron/hbase/mock/MockProvider.java} | 34 +++++++++++-------- metron-platform/metron-parsers/pom.xml | 7 ++++ .../SimpleHBaseEnrichmentWriterTest.java | 11 +++--- ...eHbaseEnrichmentWriterIntegrationTest.java | 8 ++--- 20 files changed, 133 insertions(+), 116 deletions(-) rename metron-platform/{metron-test-utilities/src/main/java/org/apache/metron/test => metron-hbase/src/test/java/org/apache/metron/hbase}/mock/MockHTable.java (96%) rename metron-platform/{metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/mock/MockTableProvider.java => metron-hbase/src/test/java/org/apache/metron/hbase/mock/MockProvider.java} (63%) diff --git a/metron-analytics/metron-profiler-client/pom.xml b/metron-analytics/metron-profiler-client/pom.xml index 61d93f86dd..bba881d5db 100644 --- a/metron-analytics/metron-profiler-client/pom.xml +++ b/metron-analytics/metron-profiler-client/pom.xml @@ -80,6 +80,19 @@ + + org.apache.metron + metron-hbase + ${project.parent.version} + test + test-jar + + + org.slf4j + slf4j-log4j12 + + + org.apache.hadoop hadoop-mapreduce-client-core diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/GetProfileTest.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/GetProfileTest.java index 917a5cad8b..56fe85cc3a 100644 --- a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/GetProfileTest.java +++ b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/GetProfileTest.java @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.HTableInterface; +import org.apache.metron.hbase.mock.MockProvider; import org.apache.metron.stellar.dsl.Context; import org.apache.metron.stellar.dsl.functions.resolver.SimpleFunctionResolver; import org.apache.metron.stellar.dsl.functions.resolver.SingletonFunctionResolver; @@ -35,7 +36,6 @@ import org.apache.metron.profiler.hbase.ValueOnlyColumnBuilder; import org.apache.metron.stellar.common.DefaultStellarStatefulExecutor; import org.apache.metron.stellar.common.StellarStatefulExecutor; -import org.apache.metron.test.mock.MockHTable; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -69,18 +69,7 @@ public class GetProfileTest { private static final TimeUnit periodUnits2 = TimeUnit.HOURS; private static final int saltDivisor2 = 2050; - /** - * A TableProvider that allows us to mock HBase. - */ - public static class MockTableProvider implements TableProvider, Serializable { - MockHTable.Provider provider = new MockHTable.Provider(); - - @Override - public HTableInterface getTable(Configuration config, String tableName) throws IOException { - return provider.getTable(config, tableName); - } - } private T run(String expression, Class clazz) { return executor.execute(expression, state, clazz); @@ -100,7 +89,7 @@ private T run(String expression, Class clazz) { @Before public void setup() { state = new HashMap<>(); - final HTableInterface table = MockHTable.Provider.addToCache(tableName, columnFamily); + final HTableInterface table = MockProvider.addToCache(tableName, columnFamily); // used to write values to be read during testing RowKeyBuilder rowKeyBuilder = new SaltyRowKeyBuilder(); @@ -111,7 +100,7 @@ public void setup() { Map global = new HashMap() {{ put(PROFILER_HBASE_TABLE.getKey(), tableName); put(PROFILER_COLUMN_FAMILY.getKey(), columnFamily); - put(PROFILER_HBASE_TABLE_PROVIDER.getKey(), MockTableProvider.class.getName()); + put(PROFILER_HBASE_TABLE_PROVIDER.getKey(), MockProvider.class.getName()); put(PROFILER_PERIOD.getKey(), Long.toString(periodDuration)); put(PROFILER_PERIOD_UNITS.getKey(), periodUnits.toString()); put(PROFILER_SALT_DIVISOR.getKey(), Integer.toString(saltDivisor)); @@ -152,7 +141,7 @@ private Context setup2() { Map global = new HashMap() {{ put(PROFILER_HBASE_TABLE.getKey(), tableName); put(PROFILER_COLUMN_FAMILY.getKey(), columnFamily); - put(PROFILER_HBASE_TABLE_PROVIDER.getKey(), MockTableProvider.class.getName()); + put(PROFILER_HBASE_TABLE_PROVIDER.getKey(), MockProvider.class.getName()); put(PROFILER_PERIOD.getKey(), Long.toString(periodDuration2)); put(PROFILER_PERIOD_UNITS.getKey(), periodUnits2.toString()); put(PROFILER_SALT_DIVISOR.getKey(), Integer.toString(saltDivisor2)); diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java index 960e4d27b4..8519f100f4 100644 --- a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java +++ b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/HBaseProfilerClientTest.java @@ -20,6 +20,7 @@ package org.apache.metron.profiler.client; +import org.apache.metron.hbase.mock.MockHTable; import org.apache.metron.profiler.ProfileMeasurement; import org.apache.metron.profiler.hbase.ColumnBuilder; import org.apache.metron.profiler.hbase.RowKeyBuilder; @@ -27,11 +28,8 @@ import org.apache.metron.profiler.hbase.ValueOnlyColumnBuilder; import org.apache.metron.stellar.common.DefaultStellarStatefulExecutor; import org.apache.metron.stellar.common.StellarStatefulExecutor; -import org.apache.metron.test.mock.MockHTable; import org.junit.After; -import org.junit.AfterClass; import org.junit.Before; -import org.junit.BeforeClass; import org.junit.Test; import java.util.Arrays; diff --git a/metron-analytics/metron-profiler/pom.xml b/metron-analytics/metron-profiler/pom.xml index bd3ab9c31b..41888a1efe 100644 --- a/metron-analytics/metron-profiler/pom.xml +++ b/metron-analytics/metron-profiler/pom.xml @@ -129,6 +129,19 @@ + + org.apache.metron + metron-hbase + ${project.parent.version} + test + test-jar + + + org.slf4j + slf4j-log4j12 + + + com.esotericsoftware kryo-shaded diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java index b863ebcbc1..9e9e7a6bdc 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java @@ -31,6 +31,8 @@ import org.apache.metron.common.Constants; import org.apache.metron.common.utils.SerDeUtils; import org.apache.metron.hbase.TableProvider; +import org.apache.metron.hbase.mock.MockHTable; +import org.apache.metron.hbase.mock.MockProvider; import org.apache.metron.integration.BaseIntegrationTest; import org.apache.metron.integration.ComponentRunner; import org.apache.metron.integration.UnableToStartException; @@ -40,7 +42,6 @@ import org.apache.metron.profiler.hbase.ColumnBuilder; import org.apache.metron.profiler.hbase.ValueOnlyColumnBuilder; import org.apache.metron.statistics.OnlineStatisticsProvider; -import org.apache.metron.test.mock.MockHTable; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -119,18 +120,6 @@ public class ProfilerIntegrationTest extends BaseIntegrationTest { private static final String inputTopic = Constants.INDEXING_TOPIC; private static final String outputTopic = "profiles"; - /** - * A TableProvider that allows us to mock HBase. - */ - public static class MockTableProvider implements TableProvider, Serializable { - - MockHTable.Provider provider = new MockHTable.Provider(); - - @Override - public HTableInterface getTable(Configuration config, String tableName) throws IOException { - return provider.getTable(config, tableName); - } - } /** * Tests the first example contained within the README. @@ -316,13 +305,13 @@ public static void setupBeforeClass() throws UnableToStartException { setProperty("profiler.hbase.batch", "10"); setProperty("profiler.hbase.flush.interval.seconds", "1"); setProperty("profiler.profile.ttl", "20"); - setProperty("hbase.provider.impl", "" + MockTableProvider.class.getName()); + setProperty("hbase.provider.impl", "" + MockProvider.class.getName()); setProperty("storm.auto.credentials", "[]"); setProperty("kafka.security.protocol", "PLAINTEXT"); }}; // create the mock table - profilerTable = (MockHTable) MockHTable.Provider.addToCache(tableName, columnFamily); + profilerTable = (MockHTable) MockProvider.addToCache(tableName, columnFamily); zkComponent = getZKServerComponent(topologyProperties); @@ -363,7 +352,7 @@ public void update(String path) throws Exception { @AfterClass public static void tearDownAfterClass() throws Exception { - MockHTable.Provider.clear(); + MockProvider.clear(); if (runner != null) { runner.stop(); } @@ -372,12 +361,12 @@ public static void tearDownAfterClass() throws Exception { @Before public void setup() { // create the mock table - profilerTable = (MockHTable) MockHTable.Provider.addToCache(tableName, columnFamily); + profilerTable = (MockHTable) MockProvider.addToCache(tableName, columnFamily); } @After public void tearDown() throws Exception { - MockHTable.Provider.clear(); + MockProvider.clear(); profilerTable.clear(); if (runner != null) { runner.reset(); diff --git a/metron-interface/metron-rest/pom.xml b/metron-interface/metron-rest/pom.xml index 2f59433260..970ee8611e 100644 --- a/metron-interface/metron-rest/pom.xml +++ b/metron-interface/metron-rest/pom.xml @@ -145,6 +145,13 @@ + + org.apache.metron + metron-hbase + ${project.parent.version} + test + test-jar + org.apache.metron metron-parsers diff --git a/metron-interface/metron-rest/src/main/resources/application-test.yml b/metron-interface/metron-rest/src/main/resources/application-test.yml index 9dfcdf9aed..59b127f322 100644 --- a/metron-interface/metron-rest/src/main/resources/application-test.yml +++ b/metron-interface/metron-rest/src/main/resources/application-test.yml @@ -44,6 +44,11 @@ storm: search: max: results: 100 + index: dao: - impl: org.apache.metron.indexing.dao.InMemoryDao \ No newline at end of file + impl: org.apache.metron.indexing.dao.InMemoryDao,org.apache.metron.indexing.dao.HBaseDao + hbase: + provider: org.apache.metron.hbase.mock.MockProvider + table: updates + cf: t \ No newline at end of file diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java index f1eb1aea7f..5612a1e7ac 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java @@ -19,6 +19,7 @@ import com.google.common.collect.ImmutableMap; import org.adrianwalker.multilinestring.Multiline; +import org.apache.metron.hbase.mock.MockProvider; import org.apache.metron.indexing.dao.InMemoryDao; import org.apache.metron.indexing.dao.IndexingDaoIntegrationTest; import org.apache.metron.rest.service.SearchService; @@ -77,6 +78,7 @@ public class SearchControllerIntegrationTest { public void setup() throws Exception { this.mockMvc = MockMvcBuilders.webAppContextSetup(this.wac).apply(springSecurity()).build(); loadTestData(); + MockProvider.addToCache("updates", "t"); } @After diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMutationIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMutationIntegrationTest.java index 472d2aa92d..b6a721ed5d 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMutationIntegrationTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMutationIntegrationTest.java @@ -22,15 +22,12 @@ 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.HTableInterface; import org.apache.hadoop.hbase.client.Result; import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.elasticsearch.dao.ElasticsearchDao; import org.apache.metron.elasticsearch.integration.components.ElasticSearchComponent; -import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; -import org.apache.metron.elasticsearch.writer.ElasticsearchWriter; import org.apache.metron.hbase.TableProvider; import org.apache.metron.indexing.dao.*; import org.apache.metron.test.mock.MockHTable; @@ -42,7 +39,6 @@ import org.junit.Test; import java.io.File; -import java.io.IOException; import java.text.SimpleDateFormat; import java.util.*; diff --git a/metron-platform/metron-enrichment/pom.xml b/metron-platform/metron-enrichment/pom.xml index e2749c5421..37cb49f5be 100644 --- a/metron-platform/metron-enrichment/pom.xml +++ b/metron-platform/metron-enrichment/pom.xml @@ -60,6 +60,13 @@ metron-hbase ${project.parent.version} + + org.apache.metron + metron-hbase + ${project.parent.version} + test + test-jar + org.apache.metron metron-profiler-client diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapterTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapterTest.java index 35a90b7412..0c52bc94f8 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapterTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapterTest.java @@ -26,7 +26,8 @@ import org.apache.metron.enrichment.converter.EnrichmentValue; import org.apache.metron.enrichment.lookup.EnrichmentLookup; import org.apache.metron.enrichment.converter.EnrichmentHelper; -import org.apache.metron.test.mock.MockHTable; +import org.apache.metron.hbase.mock.MockHTable; +import org.apache.metron.hbase.mock.MockProvider; import org.apache.metron.enrichment.lookup.LookupKV; import org.apache.metron.enrichment.lookup.accesstracker.BloomAccessTracker; import org.apache.metron.enrichment.lookup.accesstracker.PersistentAccessTracker; @@ -102,8 +103,8 @@ public class SimpleHBaseAdapterTest { @Before public void setup() throws Exception { - final MockHTable trackerTable = (MockHTable) MockHTable.Provider.addToCache(atTableName, cf); - final MockHTable hbaseTable = (MockHTable) MockHTable.Provider.addToCache(hbaseTableName, cf); + final MockHTable trackerTable = (MockHTable) MockProvider.addToCache(atTableName, cf); + final MockHTable hbaseTable = (MockHTable) MockProvider.addToCache(hbaseTableName, cf); EnrichmentHelper.INSTANCE.load(hbaseTable, cf, new ArrayList>() {{ add(new LookupKV<>(new EnrichmentKey(PLAYFUL_CLASSIFICATION_TYPE, "10.0.2.3") , new EnrichmentValue(PLAYFUL_ENRICHMENT) diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapterTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapterTest.java index e96c7a76e9..e18de930e6 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapterTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapterTest.java @@ -28,7 +28,8 @@ import org.apache.metron.enrichment.converter.EnrichmentValue; import org.apache.metron.enrichment.lookup.EnrichmentLookup; import org.apache.metron.enrichment.converter.EnrichmentHelper; -import org.apache.metron.test.mock.MockHTable; +import org.apache.metron.hbase.mock.MockHTable; +import org.apache.metron.hbase.mock.MockProvider; import org.apache.metron.enrichment.lookup.LookupKV; import org.apache.metron.enrichment.lookup.accesstracker.BloomAccessTracker; import org.apache.metron.enrichment.lookup.accesstracker.PersistentAccessTracker; @@ -98,8 +99,8 @@ public HTableInterface getTable(Configuration config, String tableName) throws I @Before public void setup() throws Exception { - final MockHTable trackerTable = (MockHTable) MockHTable.Provider.addToCache(atTableName, cf); - final MockHTable threatIntelTable = (MockHTable) MockHTable.Provider.addToCache(threatIntelTableName, cf); + final MockHTable trackerTable = (MockHTable)MockProvider.addToCache(atTableName, cf); + final MockHTable threatIntelTable = (MockHTable)MockProvider.addToCache(threatIntelTableName, cf); EnrichmentHelper.INSTANCE.load(threatIntelTable, cf, new ArrayList>() {{ add(new LookupKV<>(new EnrichmentKey("10.0.2.3", "10.0.2.3"), new EnrichmentValue(new HashMap<>()))); }}); diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java index e798b721e1..285b08279c 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java @@ -39,6 +39,8 @@ import org.apache.metron.enrichment.lookup.accesstracker.PersistentBloomTrackerCreator; import org.apache.metron.enrichment.stellar.SimpleHBaseEnrichmentFunctions; import org.apache.metron.hbase.TableProvider; +import org.apache.metron.hbase.mock.MockHTable; +import org.apache.metron.hbase.mock.MockProvider; import org.apache.metron.integration.BaseIntegrationTest; import org.apache.metron.integration.ComponentRunner; import org.apache.metron.integration.ProcessorResult; @@ -48,7 +50,6 @@ import org.apache.metron.integration.processors.KafkaMessageSet; import org.apache.metron.integration.processors.KafkaProcessor; import org.apache.metron.integration.utils.TestUtils; -import org.apache.metron.test.mock.MockHTable; import org.apache.metron.test.utils.UnitTestHelper; import org.json.simple.parser.ParseException; import org.junit.Assert; @@ -95,13 +96,6 @@ public class EnrichmentIntegrationTest extends BaseIntegrationTest { private static File geoHdfsFile; - public static class Provider implements TableProvider, Serializable { - MockHTable.Provider provider = new MockHTable.Provider(); - @Override - public HTableInterface getTable(Configuration config, String tableName) throws IOException { - return provider.getTable(config, tableName); - } - } private static List getInputMessages(String path){ try{ @@ -142,7 +136,7 @@ public void test() throws Exception { "{\"ip\":\"10.1.128.237\", \"local\":\"UNKNOWN\", \"type\":\"unknown\", \"asset_value\" : \"important\"},\n" + "{\"ip\":\"10.60.10.254\", \"local\":\"YES\", \"type\":\"printer\", \"asset_value\" : \"important\"},\n" + "{\"ip\":\"10.0.2.15\", \"local\":\"YES\", \"type\":\"printer\", \"asset_value\" : \"important\"}]"); - setProperty("hbase.provider.impl", "" + Provider.class.getName()); + setProperty("hbase.provider.impl", "" + MockProvider.class.getName()); setProperty("threat.intel.tracker.table", trackerHBaseTableName); setProperty("threat.intel.tracker.cf", cf); setProperty("threat.intel.simple.hbase.table", threatIntelTableName); @@ -171,7 +165,7 @@ public void test() throws Exception { File globalConfig = new File(new File(TestConstants.SAMPLE_CONFIG_PATH), "global.json"); Map config = JSONUtils.INSTANCE.load(globalConfig, new TypeReference>() { }); - config.put(SimpleHBaseEnrichmentFunctions.TABLE_PROVIDER_TYPE_CONF, Provider.class.getName()); + config.put(SimpleHBaseEnrichmentFunctions.TABLE_PROVIDER_TYPE_CONF, MockProvider.class.getName()); config.put(SimpleHBaseEnrichmentFunctions.ACCESS_TRACKER_TYPE_CONF, "PERSISTENT_BLOOM"); config.put(PersistentBloomTrackerCreator.Config.PERSISTENT_BLOOM_TABLE, trackerHBaseTableName); config.put(PersistentBloomTrackerCreator.Config.PERSISTENT_BLOOM_CF, cf); @@ -184,12 +178,12 @@ public void test() throws Exception { .withEnrichmentConfigsPath(TestConstants.SAMPLE_CONFIG_PATH); //create MockHBaseTables - final MockHTable trackerTable = (MockHTable) MockHTable.Provider.addToCache(trackerHBaseTableName, cf); - final MockHTable threatIntelTable = (MockHTable) MockHTable.Provider.addToCache(threatIntelTableName, cf); + final MockHTable trackerTable = (MockHTable) MockProvider.addToCache(trackerHBaseTableName, cf); + final MockHTable threatIntelTable = (MockHTable) MockProvider.addToCache(threatIntelTableName, cf); EnrichmentHelper.INSTANCE.load(threatIntelTable, cf, new ArrayList>() {{ add(new LookupKV<>(new EnrichmentKey(MALICIOUS_IP_TYPE, "10.0.2.3"), new EnrichmentValue(new HashMap<>()))); }}); - final MockHTable enrichmentTable = (MockHTable) MockHTable.Provider.addToCache(enrichmentsTableName, cf); + final MockHTable enrichmentTable = (MockHTable) MockProvider.addToCache(enrichmentsTableName, cf); EnrichmentHelper.INSTANCE.load(enrichmentTable, cf, new ArrayList>() {{ add(new LookupKV<>(new EnrichmentKey(PLAYFUL_CLASSIFICATION_TYPE, "10.0.2.3") , new EnrichmentValue(PLAYFUL_ENRICHMENT) diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/stellar/SimpleHBaseEnrichmentFunctionsTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/stellar/SimpleHBaseEnrichmentFunctionsTest.java index b7668f3f9d..14f52f0d0b 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/stellar/SimpleHBaseEnrichmentFunctionsTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/stellar/SimpleHBaseEnrichmentFunctionsTest.java @@ -21,6 +21,8 @@ import com.google.common.collect.ImmutableMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.HTableInterface; +import org.apache.metron.hbase.mock.MockHTable; +import org.apache.metron.hbase.mock.MockProvider; import org.apache.metron.stellar.dsl.Context; import org.apache.metron.stellar.dsl.StellarFunctions; import org.apache.metron.stellar.common.StellarProcessor; @@ -29,7 +31,6 @@ import org.apache.metron.enrichment.converter.EnrichmentValue; import org.apache.metron.enrichment.lookup.LookupKV; import org.apache.metron.hbase.TableProvider; -import org.apache.metron.test.mock.MockHTable; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -46,18 +47,12 @@ public class SimpleHBaseEnrichmentFunctionsTest { private String cf = "cf"; private static Context context; - public static class TP implements TableProvider { - @Override - public HTableInterface getTable(Configuration config, String tableName) throws IOException { - return MockHTable.Provider.getFromCache(tableName); - } - } @Before public void setup() throws Exception { - final MockHTable hbaseTable = (MockHTable) MockHTable.Provider.addToCache(hbaseTableName, cf); + final MockHTable hbaseTable = (MockHTable) MockProvider.addToCache(hbaseTableName, cf); EnrichmentHelper.INSTANCE.load(hbaseTable, cf, new ArrayList>() {{ for(int i = 0;i < 5;++i) { add(new LookupKV<>(new EnrichmentKey(ENRICHMENT_TYPE, "indicator" + i) @@ -69,7 +64,7 @@ public void setup() throws Exception { context = new Context.Builder() .with( Context.Capabilities.GLOBAL_CONFIG , () -> ImmutableMap.of( SimpleHBaseEnrichmentFunctions.TABLE_PROVIDER_TYPE_CONF - , TP.class.getName() + , MockProvider.class.getName() ) ) .build(); diff --git a/metron-platform/metron-hbase/pom.xml b/metron-platform/metron-hbase/pom.xml index cd5f641ade..180134f995 100644 --- a/metron-platform/metron-hbase/pom.xml +++ b/metron-platform/metron-hbase/pom.xml @@ -232,4 +232,20 @@ provided + + + + org.apache.maven.plugins + maven-jar-plugin + ${global_jar_version} + + + + test-jar + + + + + + diff --git a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/mock/MockHTable.java b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/mock/MockHTable.java similarity index 96% rename from metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/mock/MockHTable.java rename to metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/mock/MockHTable.java index 0403d1b30a..e75b533e80 100644 --- a/metron-platform/metron-test-utilities/src/main/java/org/apache/metron/test/mock/MockHTable.java +++ b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/mock/MockHTable.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.metron.test.mock; +package org.apache.metron.hbase.mock; import com.google.common.collect.ImmutableList; @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.metron.hbase.TableProvider; import java.io.IOException; import java.io.Serializable; @@ -67,28 +68,6 @@ public class MockHTable implements HTableInterface { - public static class Provider implements Serializable { - private static Map _cache = new HashMap<>(); - public HTableInterface getTable(Configuration config, String tableName) throws IOException { - HTableInterface ret = _cache.get(tableName); - return ret; - } - - public static HTableInterface getFromCache(String tableName) { - return _cache.get(tableName); - } - - public static HTableInterface addToCache(String tableName, String... columnFamilies) { - MockHTable ret = new MockHTable(tableName, columnFamilies); - _cache.put(tableName, ret); - return ret; - } - - public static void clear() { - _cache.clear(); - } - } - private final String tableName; private final List columnFamilies = new ArrayList<>(); private HColumnDescriptor[] descriptors; diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/mock/MockTableProvider.java b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/mock/MockProvider.java similarity index 63% rename from metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/mock/MockTableProvider.java rename to metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/mock/MockProvider.java index ac2904a016..723ab4df5c 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/mock/MockTableProvider.java +++ b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/mock/MockProvider.java @@ -15,31 +15,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - -package org.apache.metron.enrichment.integration.mock; +package org.apache.metron.hbase.mock; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.metron.hbase.TableProvider; -import org.apache.metron.test.mock.MockHTable; import java.io.IOException; import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; -public class MockTableProvider implements TableProvider, Serializable { - static MockHTable.Provider provider = new MockHTable.Provider(); - @Override +public class MockProvider implements Serializable, TableProvider { + private static Map _cache = new HashMap<>(); public HTableInterface getTable(Configuration config, String tableName) throws IOException { - return provider.getTable(config, tableName); + HTableInterface ret = _cache.get(tableName); + return ret; } - public static void addTable(String tableName, String... cf) { - provider.addToCache(tableName, cf); + + public static HTableInterface getFromCache(String tableName) { + return _cache.get(tableName); } - public static MockHTable getTable(String tableName) { - try { - return (MockHTable) provider.getTable(null, tableName); - } catch (IOException e) { - throw new RuntimeException("Unable to get table: " + tableName); - } + + public static HTableInterface addToCache(String tableName, String... columnFamilies) { + MockHTable ret = new MockHTable(tableName, columnFamilies); + _cache.put(tableName, ret); + return ret; + } + + public static void clear() { + _cache.clear(); } } diff --git a/metron-platform/metron-parsers/pom.xml b/metron-platform/metron-parsers/pom.xml index c99fb8d8f7..b7c21ffab2 100644 --- a/metron-platform/metron-parsers/pom.xml +++ b/metron-platform/metron-parsers/pom.xml @@ -49,6 +49,13 @@ + + org.apache.metron + metron-hbase + ${project.parent.version} + test + test-jar + org.apache.metron metron-enrichment diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/SimpleHBaseEnrichmentWriterTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/SimpleHBaseEnrichmentWriterTest.java index b9b3246850..7095327829 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/SimpleHBaseEnrichmentWriterTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/SimpleHBaseEnrichmentWriterTest.java @@ -26,10 +26,10 @@ import org.apache.metron.enrichment.converter.EnrichmentConverter; import org.apache.metron.enrichment.converter.EnrichmentKey; import org.apache.metron.enrichment.converter.EnrichmentValue; -import org.apache.metron.enrichment.integration.mock.MockTableProvider; import org.apache.metron.enrichment.lookup.LookupKV; -import org.apache.metron.test.mock.MockHTable; import org.apache.metron.enrichment.writer.SimpleHbaseEnrichmentWriter; +import org.apache.metron.hbase.mock.MockHTable; +import org.apache.metron.hbase.mock.MockProvider; import org.json.simple.JSONObject; import org.junit.Assert; import org.junit.Before; @@ -50,11 +50,12 @@ public class SimpleHBaseEnrichmentWriterTest { put(SimpleHbaseEnrichmentWriter.Configurations.HBASE_TABLE.getKey(), TABLE_NAME); put(SimpleHbaseEnrichmentWriter.Configurations.HBASE_CF.getKey(), TABLE_CF); put(SimpleHbaseEnrichmentWriter.Configurations.ENRICHMENT_TYPE.getKey(), ENRICHMENT_TYPE); - put(SimpleHbaseEnrichmentWriter.Configurations.HBASE_PROVIDER.getKey(), MockTableProvider.class.getName()); + put(SimpleHbaseEnrichmentWriter.Configurations.HBASE_PROVIDER.getKey(), MockProvider.class.getName()); }}; + @Before public void setupMockTable() { - MockTableProvider.addTable(TABLE_NAME, TABLE_CF); + MockProvider.addToCache(TABLE_NAME, TABLE_CF); } @Test public void testBatchOneNormalPath() throws Exception { @@ -140,7 +141,7 @@ public void testFilteredKeys() throws Exception { Assert.assertEquals(2, values.get(0).getValue().getMetadata().size()); } public static List> getValues() throws IOException { - MockHTable table = MockTableProvider.getTable(TABLE_NAME); + MockHTable table = (MockHTable)MockProvider.getFromCache(TABLE_NAME); Assert.assertNotNull(table); List> ret = new ArrayList<>(); EnrichmentConverter converter = new EnrichmentConverter(); diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java index 4efe28bd22..6871247778 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java @@ -29,13 +29,13 @@ import org.apache.metron.enrichment.converter.EnrichmentKey; import org.apache.metron.enrichment.converter.EnrichmentValue; import org.apache.metron.enrichment.integration.components.ConfigUploadComponent; -import org.apache.metron.enrichment.integration.mock.MockTableProvider; import org.apache.metron.enrichment.lookup.LookupKV; +import org.apache.metron.hbase.mock.MockHTable; +import org.apache.metron.hbase.mock.MockProvider; import org.apache.metron.integration.*; import org.apache.metron.integration.components.KafkaComponent; import org.apache.metron.integration.components.ZKServerComponent; import org.apache.metron.parsers.integration.components.ParserTopologyComponent; -import org.apache.metron.test.mock.MockHTable; import org.junit.Assert; import org.junit.Test; @@ -75,7 +75,7 @@ public void test() throws UnableToStartException, IOException { add(Bytes.toBytes("col21,col22,col23")); add(Bytes.toBytes("col31,col32,col33")); }}; - MockTableProvider.addTable(sensorType, "cf"); + MockProvider.addToCache(sensorType, "cf"); final Properties topologyProperties = new Properties(); final ZKServerComponent zkServerComponent = getZKServerComponent(topologyProperties); final KafkaComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList() {{ @@ -112,7 +112,7 @@ public void test() throws UnableToStartException, IOException { @Override public ReadinessState process(ComponentRunner runner) { - MockHTable table = MockTableProvider.getTable(sensorType); + MockHTable table = (MockHTable)MockProvider.getFromCache(sensorType); if (table != null && table.size() == inputMessages.size()) { EnrichmentConverter converter = new EnrichmentConverter(); messages = new ArrayList<>(); From ad535c911196d7c24fd6fddc8c082ce2b0b4d859 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 20 Jul 2017 18:53:07 +0100 Subject: [PATCH 03/38] Updating REST api tests. --- .../metron/rest/config/IndexConfig.java | 7 + .../rest/controller/SearchController.java | 24 +++ .../rest/controller/UpdateController.java | 70 ++++++++ .../metron/rest/service/SearchService.java | 7 +- .../metron/rest/service/UpdateService.java | 12 ++ ...erviceImpl.java => SearchServiceImpl.java} | 19 +- .../rest/service/impl/UpdateServiceImpl.java | 41 +++++ .../rest/controller/DaoControllerTest.java | 41 +++++ .../SearchControllerIntegrationTest.java | 48 ++--- .../UpdateControllerIntegrationTest.java | 166 ++++++++++++++++++ .../apache/metron/common/utils/JSONUtils.java | 10 ++ metron-platform/metron-elasticsearch/pom.xml | 17 ++ .../elasticsearch/dao/ElasticsearchDao.java | 18 +- ...> ElasticsearchSearchIntegrationTest.java} | 20 +-- ...> ElasticsearchUpdateIntegrationTest.java} | 58 +++--- .../metron/indexing/dao/AccessConfig.java | 10 ++ .../apache/metron/indexing/dao/HBaseDao.java | 9 +- .../apache/metron/indexing/dao/IndexDao.java | 80 +++++---- .../metron/indexing/dao/IndexDaoFactory.java | 2 +- .../indexing/dao/IndexUpdateCallback.java | 2 + .../metron/indexing/dao/MultiIndexDao.java | 9 +- .../indexing/dao/search/GetRequest.java | 22 +++ .../indexing/dao/{ => update}/Document.java | 30 +++- .../update/OriginalNotFoundException.java} | 10 +- .../indexing/dao/update/PatchRequest.java | 44 +++++ .../indexing/dao/update/ReplaceRequest.java | 35 ++++ .../metron/indexing/mutation/Mutation.java | 63 ------- .../indexing/mutation/MutationOperation.java | 42 ----- .../metron/indexing/mutation/Mutator.java | 27 --- .../metron/indexing/mutation/Update.java | 23 --- .../indexing/mutation/mutators/Patch.java | 40 ----- .../indexing/mutation/mutators/Replace.java | 32 ---- .../metron/indexing/dao/InMemoryDao.java | 7 +- ...onTest.java => SearchIntegrationTest.java} | 2 +- .../indexing/mutation/MutationTest.java | 119 ------------- 35 files changed, 688 insertions(+), 478 deletions(-) create mode 100644 metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/UpdateController.java create mode 100644 metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/UpdateService.java rename metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/{IndexDaoSearchServiceImpl.java => SearchServiceImpl.java} (75%) create mode 100644 metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/UpdateServiceImpl.java create mode 100644 metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/DaoControllerTest.java create mode 100644 metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java rename metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/{ElasticsearchDaoIntegrationTest.java => ElasticsearchSearchIntegrationTest.java} (88%) rename metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/{ElasticsearchMutationIntegrationTest.java => ElasticsearchUpdateIntegrationTest.java} (81%) create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java rename metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/{ => update}/Document.java (66%) rename metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/{mutation/MutationException.java => dao/update/OriginalNotFoundException.java} (78%) create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java create mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java delete mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutation.java delete mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationOperation.java delete mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutator.java delete mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Update.java delete mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Patch.java delete mode 100644 metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Replace.java rename metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/{IndexingDaoIntegrationTest.java => SearchIntegrationTest.java} (99%) delete mode 100644 metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/mutation/MutationTest.java diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java index fab11e745c..56b563b5c7 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java @@ -62,6 +62,13 @@ public IndexDao indexDao() { config.setMaxSearchResults(searchMaxResults); config.setTable(hbaseTable); config.setColumnFamily(hbaseCf); + config.setGlobalConfigSupplier(() -> { + try { + return globalConfigService.get(); + } catch (RestException e) { + throw new IllegalStateException("Unable to retrieve the global config.", e); + } + }); config.setTableProvider(TableProvider.create(hbaseProviderImpl, () -> new HTableProvider())); if (indexDaoImpl == null) { throw new IllegalStateException("You must provide an index DAO implementation via the " + INDEX_DAO_IMPL + " config"); diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/SearchController.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/SearchController.java index 6915666cd0..cc4c00db6b 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/SearchController.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/SearchController.java @@ -20,6 +20,8 @@ import io.swagger.annotations.ApiOperation; import io.swagger.annotations.ApiParam; import io.swagger.annotations.ApiResponse; +import org.apache.metron.indexing.dao.search.GetRequest; +import org.apache.metron.indexing.dao.update.Document; import org.apache.metron.rest.RestException; import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; @@ -32,6 +34,9 @@ import org.springframework.web.bind.annotation.RequestMethod; import org.springframework.web.bind.annotation.RestController; +import java.util.Map; +import java.util.Optional; + @RestController @RequestMapping("/api/v1/search") public class SearchController { @@ -45,4 +50,23 @@ public class SearchController { ResponseEntity search(final @ApiParam(name = "searchRequest", value = "Search request", required = true) @RequestBody SearchRequest searchRequest) throws RestException { return new ResponseEntity<>(searchService.search(searchRequest), HttpStatus.OK); } + + @ApiOperation(value = "Returns latest document for a uuid and sensor") + @ApiResponse(message = "Document representing the output", code = 200) + @RequestMapping(value = "/findOne", method = RequestMethod.POST) + ResponseEntity> getLatest( + final @ApiParam(name = "getRequest", value = "Get Request", required = true) + @RequestBody + GetRequest request + ) throws RestException + { + Optional> latest = searchService.getLatest(request); + if(latest.isPresent()) { + return new ResponseEntity<>(latest.get(), HttpStatus.OK); + } + else { + return new ResponseEntity<>(HttpStatus.NOT_FOUND); + } + } + } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/UpdateController.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/UpdateController.java new file mode 100644 index 0000000000..0b6e1c5c22 --- /dev/null +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/UpdateController.java @@ -0,0 +1,70 @@ +/** + * 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.rest.controller; + +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiParam; +import io.swagger.annotations.ApiResponse; +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.apache.metron.rest.RestException; +import org.apache.metron.rest.service.UpdateService; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.http.HttpStatus; +import org.springframework.http.ResponseEntity; +import org.springframework.web.bind.annotation.RequestBody; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.RestController; + +@RestController +@RequestMapping("/api/v1/update") +public class UpdateController { + + @Autowired + private UpdateService service; + + @ApiOperation(value = "Update a document with a patch") + @ApiResponse(message = "Nothing", code = 200) + @RequestMapping(value = "/patch", method = RequestMethod.PATCH) + ResponseEntity patch( + final @ApiParam(name = "request", value = "Patch request", required = true) + @RequestBody + PatchRequest request + ) throws RestException { + try { + service.patch(request); + } catch (OriginalNotFoundException e) { + return new ResponseEntity<>(HttpStatus.NOT_FOUND); + } + return new ResponseEntity<>(HttpStatus.OK); + } + + @ApiOperation(value = "Update a document with a patch") + @ApiResponse(message = "Nothing", code = 200) + @RequestMapping(value = "/replace", method = RequestMethod.POST) + ResponseEntity replace( + final @ApiParam(name = "request", value = "Replacement request", required = true) + @RequestBody + ReplaceRequest request + ) throws RestException { + service.replace(request); + return new ResponseEntity<>(HttpStatus.OK); + } +} diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/SearchService.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/SearchService.java index df28d4a995..243d1af8c7 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/SearchService.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/SearchService.java @@ -17,12 +17,17 @@ */ package org.apache.metron.rest.service; +import org.apache.metron.indexing.dao.search.GetRequest; +import org.apache.metron.indexing.dao.update.Document; import org.apache.metron.rest.RestException; import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; +import java.util.Map; +import java.util.Optional; + public interface SearchService { SearchResponse search(SearchRequest searchRequest) throws RestException; - + Optional> getLatest(GetRequest request) throws RestException; } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/UpdateService.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/UpdateService.java new file mode 100644 index 0000000000..55c0698656 --- /dev/null +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/UpdateService.java @@ -0,0 +1,12 @@ +package org.apache.metron.rest.service; + +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.apache.metron.rest.RestException; + +public interface UpdateService { + + void patch(PatchRequest request) throws RestException, OriginalNotFoundException; + void replace(ReplaceRequest request) throws RestException; +} diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/IndexDaoSearchServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/SearchServiceImpl.java similarity index 75% rename from metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/IndexDaoSearchServiceImpl.java rename to metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/SearchServiceImpl.java index 123d6d063a..ab1d18de53 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/IndexDaoSearchServiceImpl.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/SearchServiceImpl.java @@ -18,20 +18,26 @@ package org.apache.metron.rest.service.impl; import org.apache.metron.indexing.dao.IndexDao; +import org.apache.metron.indexing.dao.search.GetRequest; import org.apache.metron.indexing.dao.search.InvalidSearchException; import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; +import org.apache.metron.indexing.dao.update.Document; import org.apache.metron.rest.RestException; import org.apache.metron.rest.service.SearchService; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Service; +import java.io.IOException; +import java.util.Map; +import java.util.Optional; + @Service -public class IndexDaoSearchServiceImpl implements SearchService { +public class SearchServiceImpl implements SearchService { private IndexDao dao; @Autowired - public IndexDaoSearchServiceImpl(IndexDao dao) { + public SearchServiceImpl(IndexDao dao) { this.dao = dao; } @@ -44,4 +50,13 @@ public SearchResponse search(SearchRequest searchRequest) throws RestException { throw new RestException(ise.getMessage(), ise); } } + + @Override + public Optional> getLatest(GetRequest request) throws RestException { + try { + return dao.getLatestResult(request); + } catch (IOException e) { + throw new RestException(e.getMessage(), e); + } + } } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/UpdateServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/UpdateServiceImpl.java new file mode 100644 index 0000000000..588a9679ab --- /dev/null +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/UpdateServiceImpl.java @@ -0,0 +1,41 @@ +package org.apache.metron.rest.service.impl; + +import org.apache.metron.indexing.dao.IndexDao; +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.apache.metron.rest.RestException; +import org.apache.metron.rest.service.UpdateService; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Service; + +import java.util.Optional; + +@Service +public class UpdateServiceImpl implements UpdateService { + private IndexDao dao; + + @Autowired + public UpdateServiceImpl(IndexDao dao) { + this.dao = dao; + } + + + @Override + public void patch(PatchRequest request) throws RestException, OriginalNotFoundException { + try { + dao.patch(request, Optional.of(System.currentTimeMillis())); + } catch (Exception e) { + throw new RestException(e.getMessage(), e); + } + } + + @Override + public void replace(ReplaceRequest request) throws RestException { + try { + dao.replace(request, Optional.of(System.currentTimeMillis())); + } catch (Exception e) { + throw new RestException(e.getMessage(), e); + } + } +} diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/DaoControllerTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/DaoControllerTest.java new file mode 100644 index 0000000000..45ce06af68 --- /dev/null +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/DaoControllerTest.java @@ -0,0 +1,41 @@ +package org.apache.metron.rest.controller; + +import com.google.common.collect.ImmutableMap; +import org.apache.metron.common.Constants; +import org.apache.metron.indexing.dao.InMemoryDao; +import org.apache.metron.indexing.dao.SearchIntegrationTest; +import org.json.simple.JSONArray; +import org.json.simple.JSONObject; +import org.json.simple.parser.JSONParser; +import org.json.simple.parser.ParseException; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class DaoControllerTest { + public static final String TABLE = "updates"; + public static final String CF = "t"; + public void loadTestData() throws ParseException { + Map> backingStore = new HashMap<>(); + for(Map.Entry indices : + ImmutableMap.of( + "bro_index_2017.01.01.01", SearchIntegrationTest.broData, + "snort_index_2017.01.01.01", SearchIntegrationTest.snortData + ).entrySet() + ) + { + List results = new ArrayList<>(); + backingStore.put(indices.getKey(), results); + JSONArray broArray = (JSONArray) new JSONParser().parse(indices.getValue()); + int i = 0; + for(Object o: broArray) { + JSONObject jsonObject = (JSONObject) o; + jsonObject.put(Constants.GUID, indices.getKey() + ":" + i++); + results.add(jsonObject.toJSONString()); + } + } + InMemoryDao.load(backingStore); + } +} diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java index 5612a1e7ac..f83c31f374 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java @@ -18,10 +18,9 @@ package org.apache.metron.rest.controller; import com.google.common.collect.ImmutableMap; -import org.adrianwalker.multilinestring.Multiline; import org.apache.metron.hbase.mock.MockProvider; import org.apache.metron.indexing.dao.InMemoryDao; -import org.apache.metron.indexing.dao.IndexingDaoIntegrationTest; +import org.apache.metron.indexing.dao.SearchIntegrationTest; import org.apache.metron.rest.service.SearchService; import org.json.simple.JSONArray; import org.json.simple.JSONObject; @@ -29,6 +28,7 @@ import org.json.simple.parser.ParseException; import org.junit.After; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.springframework.beans.factory.annotation.Autowired; @@ -58,7 +58,7 @@ @RunWith(SpringRunner.class) @SpringBootTest(webEnvironment = SpringBootTest.WebEnvironment.RANDOM_PORT) @ActiveProfiles(TEST_PROFILE) -public class SearchControllerIntegrationTest { +public class SearchControllerIntegrationTest extends DaoControllerTest { @@ -74,11 +74,15 @@ public class SearchControllerIntegrationTest { private String user = "user"; private String password = "password"; + @BeforeClass + public static void setupHbase() { + MockProvider.addToCache("updates", "t"); + } + @Before public void setup() throws Exception { this.mockMvc = MockMvcBuilders.webAppContextSetup(this.wac).apply(springSecurity()).build(); loadTestData(); - MockProvider.addToCache("updates", "t"); } @After @@ -88,14 +92,14 @@ public void cleanup() throws Exception { @Test public void testSecurity() throws Exception { - this.mockMvc.perform(post(searchUrl + "/search").with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(IndexingDaoIntegrationTest.allQuery)) + this.mockMvc.perform(post(searchUrl + "/search").with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(SearchIntegrationTest.allQuery)) .andExpect(status().isUnauthorized()); } @Test public void test() throws Exception { - this.mockMvc.perform(post(searchUrl + "/search").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(IndexingDaoIntegrationTest.allQuery)) + this.mockMvc.perform(post(searchUrl + "/search").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(SearchIntegrationTest.allQuery)) .andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) .andExpect(jsonPath("$.total").value(10)) @@ -120,7 +124,7 @@ public void test() throws Exception { .andExpect(jsonPath("$.results[9].source.source:type").value("bro")) .andExpect(jsonPath("$.results[9].source.timestamp").value(1)); - this.mockMvc.perform(post(searchUrl + "/search").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(IndexingDaoIntegrationTest.filterQuery)) + this.mockMvc.perform(post(searchUrl + "/search").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(SearchIntegrationTest.filterQuery)) .andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) .andExpect(jsonPath("$.total").value(3)) @@ -132,7 +136,7 @@ public void test() throws Exception { .andExpect(jsonPath("$.results[2].source.timestamp").value(1)); - this.mockMvc.perform(post(searchUrl + "/search").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(IndexingDaoIntegrationTest.sortQuery)) + this.mockMvc.perform(post(searchUrl + "/search").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(SearchIntegrationTest.sortQuery)) .andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) .andExpect(jsonPath("$.total").value(10)) @@ -147,7 +151,7 @@ public void test() throws Exception { .andExpect(jsonPath("$.results[8].source.ip_src_port").value(8009)) .andExpect(jsonPath("$.results[9].source.ip_src_port").value(8010)); - this.mockMvc.perform(post(searchUrl + "/search").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(IndexingDaoIntegrationTest.paginationQuery)) + this.mockMvc.perform(post(searchUrl + "/search").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(SearchIntegrationTest.paginationQuery)) .andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) .andExpect(jsonPath("$.total").value(10)) @@ -158,7 +162,7 @@ public void test() throws Exception { .andExpect(jsonPath("$.results[2].source.source:type").value("bro")) .andExpect(jsonPath("$.results[2].source.timestamp").value(4)); - this.mockMvc.perform(post(searchUrl + "/search").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(IndexingDaoIntegrationTest.indexQuery)) + this.mockMvc.perform(post(searchUrl + "/search").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(SearchIntegrationTest.indexQuery)) .andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) .andExpect(jsonPath("$.total").value(5)) @@ -173,32 +177,10 @@ public void test() throws Exception { .andExpect(jsonPath("$.results[4].source.source:type").value("bro")) .andExpect(jsonPath("$.results[4].source.timestamp").value(1)); - this.mockMvc.perform(post(searchUrl + "/search").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(IndexingDaoIntegrationTest.exceededMaxResultsQuery)) + this.mockMvc.perform(post(searchUrl + "/search").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(SearchIntegrationTest.exceededMaxResultsQuery)) .andExpect(status().isInternalServerError()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) .andExpect(jsonPath("$.responseCode").value(500)) .andExpect(jsonPath("$.message").value("Search result size must be less than 100")); } - - - - private void loadTestData() throws ParseException { - Map> backingStore = new HashMap<>(); - for(Map.Entry indices : - ImmutableMap.of( - "bro_index_2017.01.01.01", IndexingDaoIntegrationTest.broData, - "snort_index_2017.01.01.01", IndexingDaoIntegrationTest.snortData - ).entrySet() - ) - { - List results = new ArrayList<>(); - backingStore.put(indices.getKey(), results); - JSONArray broArray = (JSONArray) new JSONParser().parse(indices.getValue()); - for(Object o: broArray) { - JSONObject jsonObject = (JSONObject) o; - results.add(jsonObject.toJSONString()); - } - } - InMemoryDao.load(backingStore); - } } diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java new file mode 100644 index 0000000000..7fe194faf1 --- /dev/null +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java @@ -0,0 +1,166 @@ +package org.apache.metron.rest.controller; + +import org.adrianwalker.multilinestring.Multiline; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.metron.hbase.mock.MockHTable; +import org.apache.metron.hbase.mock.MockProvider; +import org.apache.metron.indexing.dao.SearchIntegrationTest; +import org.apache.metron.rest.service.UpdateService; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.boot.test.context.SpringBootTest; +import org.springframework.http.MediaType; +import org.springframework.test.context.ActiveProfiles; +import org.springframework.test.context.junit4.SpringRunner; +import org.springframework.test.web.servlet.MockMvc; +import org.springframework.test.web.servlet.setup.MockMvcBuilders; +import org.springframework.web.context.WebApplicationContext; + +import java.util.NavigableMap; + +import static org.apache.metron.rest.MetronRestConstants.TEST_PROFILE; +import static org.springframework.security.test.web.servlet.setup.SecurityMockMvcConfigurers.springSecurity; +import static org.springframework.security.test.web.servlet.request.SecurityMockMvcRequestPostProcessors.csrf; +import static org.springframework.security.test.web.servlet.request.SecurityMockMvcRequestPostProcessors.httpBasic; +import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post; +import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.patch; +import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content; +import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.jsonPath; +import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status; + +@RunWith(SpringRunner.class) +@SpringBootTest(webEnvironment = SpringBootTest.WebEnvironment.RANDOM_PORT) +@ActiveProfiles(TEST_PROFILE) +public class UpdateControllerIntegrationTest extends DaoControllerTest { + @Autowired + private UpdateService searchService; + + @Autowired + private WebApplicationContext wac; + + private MockMvc mockMvc; + + private String updateUrl = "/api/v1/update"; + private String searchUrl = "/api/v1/search"; + private String user = "user"; + private String password = "password"; + + /** + { + "uuid" : "bro_index_2017.01.01.01:1", + "sensorType" : "bro" + } + */ + @Multiline + public static String findMessage0; + + /** + { + "uuid" : "bro_index_2017.01.01.01:1", + "sensorType" : "bro", + "patch" : [ + { + "op": "add" + , "path": "/project" + , "value": "metron" + } + ] + } + */ + @Multiline + public static String patch; + + /** + { + "uuid" : "bro_index_2017.01.01.01:1", + "sensorType" : "bro", + "replacement" : { + "source:type": "bro", + "guid" : "bro_index_2017.01.01.01:1", + "ip_src_addr":"192.168.1.2", + "ip_src_port": 8009, + "timestamp":200, + "rejected":false + } + } + */ + @Multiline + public static String replace; + + + @BeforeClass + public static void setupHbase() { + MockProvider.addToCache(TABLE, CF); + } + + @Before + public void setup() throws Exception { + this.mockMvc = MockMvcBuilders.webAppContextSetup(this.wac).apply(springSecurity()).build(); + loadTestData(); + } + + @Test + public void test() throws Exception { + String uuid = "bro_index_2017.01.01.01:1"; + this.mockMvc.perform(post(searchUrl + "/findOne").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(findMessage0)) + .andExpect(status().isOk()) + .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) + .andExpect(jsonPath("$.source:type").value("bro")) + .andExpect(jsonPath("$.guid").value(uuid)) + .andExpect(jsonPath("$.project").doesNotExist()) + .andExpect(jsonPath("$.timestamp").value(2)) + ; + MockHTable table = (MockHTable) MockProvider.getFromCache(TABLE); + Assert.assertEquals(0,table.size()); + this.mockMvc.perform(patch(updateUrl+ "/patch").with(httpBasic(user, password)) + .with(csrf()) + .contentType(MediaType.parseMediaType("application/json;charset=UTF-8")) + .content(patch) + ) + .andExpect(status().isOk()); + this.mockMvc.perform(post(searchUrl + "/findOne").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(findMessage0)) + .andExpect(status().isOk()) + .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) + .andExpect(jsonPath("$.source:type").value("bro")) + .andExpect(jsonPath("$.guid").value(uuid)) + .andExpect(jsonPath("$.project").value("metron")) + .andExpect(jsonPath("$.timestamp").value(2)) + ; + Assert.assertEquals(1,table.size()); + { + //ensure hbase is up to date + Get g = new Get(uuid.getBytes()); + Result r = table.get(g); + NavigableMap columns = r.getFamilyMap(CF.getBytes()); + Assert.assertEquals(1, columns.size()); + } + this.mockMvc.perform(post(updateUrl+ "/replace").with(httpBasic(user, password)) + .with(csrf()) + .contentType(MediaType.parseMediaType("application/json;charset=UTF-8")) + .content(replace) + ) + .andExpect(status().isOk()); + this.mockMvc.perform(post(searchUrl + "/findOne").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(findMessage0)) + .andExpect(status().isOk()) + .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) + .andExpect(jsonPath("$.source:type").value("bro")) + .andExpect(jsonPath("$.guid").value(uuid)) + .andExpect(jsonPath("$.project").doesNotExist()) + .andExpect(jsonPath("$.timestamp").value(200)) + ; + Assert.assertEquals(1,table.size()); + { + //ensure hbase is up to date + Get g = new Get(uuid.getBytes()); + Result r = table.get(g); + NavigableMap columns = r.getFamilyMap(CF.getBytes()); + Assert.assertEquals(2, columns.size()); + } + } + +} diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/JSONUtils.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/JSONUtils.java index c61efc5127..60d009f332 100644 --- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/JSONUtils.java +++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/utils/JSONUtils.java @@ -27,6 +27,7 @@ import org.json.simple.parser.ParseException; import java.io.*; +import java.util.Map; public enum JSONUtils { INSTANCE; @@ -37,6 +38,15 @@ public enum JSONUtils { private static ThreadLocal _mapper = ThreadLocal.withInitial(() -> new ObjectMapper().setSerializationInclusion(JsonInclude.Include.NON_NULL)); + public T convert(Object original, Class targetClass) { + return _mapper.get().convertValue(original, targetClass); + } + + public ObjectMapper getMapper() { + return _mapper.get(); + } + + public T load(InputStream is, TypeReference ref) throws IOException { return _mapper.get().readValue(is, ref); } diff --git a/metron-platform/metron-elasticsearch/pom.xml b/metron-platform/metron-elasticsearch/pom.xml index 6cef1e9a2c..40989c616b 100644 --- a/metron-platform/metron-elasticsearch/pom.xml +++ b/metron-platform/metron-elasticsearch/pom.xml @@ -72,6 +72,23 @@ + + org.apache.metron + metron-hbase + ${parent.version} + test + test-jar + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + org.apache.hbase hbase-client 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 9f94326334..e99e1b5364 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 @@ -21,14 +21,14 @@ import com.google.common.collect.Iterables; import org.apache.metron.common.Constants; import org.apache.metron.common.configuration.writer.WriterConfiguration; +import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.metron.indexing.dao.AccessConfig; -import org.apache.metron.indexing.dao.Document; +import org.apache.metron.indexing.dao.update.Document; import org.apache.metron.indexing.dao.IndexDao; import org.apache.metron.indexing.dao.search.*; import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; -import org.apache.metron.indexing.dao.search.SortOrder; import org.elasticsearch.action.search.*; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.index.query.QueryStringQueryBuilder; @@ -36,11 +36,9 @@ import org.elasticsearch.search.sort.*; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.SearchHit; -import org.elasticsearch.search.SearchHitField; import org.elasticsearch.search.SearchHits; import java.io.IOException; import java.util.Arrays; @@ -100,9 +98,9 @@ public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchEx } @Override - public synchronized void init(Map globalConfig, AccessConfig config) { + public synchronized void init(AccessConfig config) { if(this.client == null) { - this.client = ElasticsearchUtils.getClient(globalConfig, config.getOptionalSettings()); + this.client = ElasticsearchUtils.getClient(config.getGlobalConfigSupplier().get(), config.getOptionalSettings()); this.accessConfig = config; } } @@ -136,14 +134,14 @@ public Document getLatest(String uuid, String sensorType) throws IOException { } @Override - public void update(Document update, WriterConfiguration configurations) throws IOException { - String indexPostfix = ElasticsearchUtils.getIndexFormat(configurations).format(new Date()); + public void update(Document update) throws IOException { + String indexPostfix = ElasticsearchUtils.getIndexFormat(accessConfig.getGlobalConfigSupplier().get()).format(new Date()); String sensorType = update.getSensorType(); - String indexName = ElasticsearchUtils.getIndexName(sensorType, indexPostfix, configurations); + String indexName = ElasticsearchUtils.getIndexName(sensorType, indexPostfix, null); IndexRequestBuilder indexRequestBuilder = client.prepareIndex(indexName, sensorType + "_doc"); - indexRequestBuilder = indexRequestBuilder.setSource(update.getDocument()); + indexRequestBuilder = indexRequestBuilder.setSource(new String(JSONUtils.INSTANCE.toJSON(update.getDocument()))); Object ts = update.getTimestamp(); if(ts != null) { indexRequestBuilder = indexRequestBuilder.setTimestamp(ts.toString()); diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchDaoIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java similarity index 88% rename from metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchDaoIntegrationTest.java rename to metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java index d937fffec8..83d6456d39 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchDaoIntegrationTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java @@ -22,7 +22,7 @@ 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.IndexingDaoIntegrationTest; +import org.apache.metron.indexing.dao.SearchIntegrationTest; import org.apache.metron.integration.InMemoryComponent; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; @@ -31,13 +31,11 @@ import org.json.simple.JSONObject; import org.json.simple.parser.JSONParser; import org.json.simple.parser.ParseException; -import org.junit.Before; -import org.junit.BeforeClass; import java.io.File; import java.util.HashMap; -public class ElasticsearchDaoIntegrationTest extends IndexingDaoIntegrationTest { +public class ElasticsearchSearchIntegrationTest extends SearchIntegrationTest { private static String indexDir = "target/elasticsearch_search"; private static String dateFormat = "yyyy.MM.dd.HH"; @@ -46,14 +44,16 @@ public class ElasticsearchDaoIntegrationTest extends IndexingDaoIntegrationTest protected IndexDao createDao() throws Exception { IndexDao ret = new ElasticsearchDao(); ret.init( - new HashMap() {{ - put("es.clustername", "metron"); - put("es.port", "9300"); - put("es.ip", "localhost"); - put("es.date.format", dateFormat); - }}, new AccessConfig() {{ setMaxSearchResults(100); + setGlobalConfigSupplier( () -> + new HashMap() {{ + put("es.clustername", "metron"); + put("es.port", "9300"); + put("es.ip", "localhost"); + put("es.date.format", dateFormat); + }} + ); }} ); return ret; diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMutationIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java similarity index 81% rename from metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMutationIntegrationTest.java rename to metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java index b6a721ed5d..1ef88fb991 100644 --- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMutationIntegrationTest.java +++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java @@ -18,6 +18,7 @@ package org.apache.metron.elasticsearch.integration; import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.Iterables; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -29,10 +30,11 @@ import org.apache.metron.elasticsearch.dao.ElasticsearchDao; import org.apache.metron.elasticsearch.integration.components.ElasticSearchComponent; import org.apache.metron.hbase.TableProvider; +import org.apache.metron.hbase.mock.MockHTable; +import org.apache.metron.hbase.mock.MockProvider; import org.apache.metron.indexing.dao.*; -import org.apache.metron.test.mock.MockHTable; -import org.apache.metron.indexing.mutation.Mutation; -import org.apache.metron.indexing.mutation.MutationOperation; +import org.apache.metron.indexing.dao.update.Document; +import org.apache.metron.indexing.dao.update.ReplaceRequest; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -42,11 +44,8 @@ import java.text.SimpleDateFormat; import java.util.*; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -public class ElasticsearchMutationIntegrationTest { +public class ElasticsearchUpdateIntegrationTest { private static final int MAX_RETRIES = 10; private static final int SLEEP_MS = 500; private static final String SENSOR_NAME= "test"; @@ -59,13 +58,12 @@ public class ElasticsearchMutationIntegrationTest { private static IndexDao esDao; private static IndexDao hbaseDao; private static MultiIndexDao dao; - private static WriterConfiguration configurations; private static ElasticSearchComponent es; @BeforeClass public static void setup() throws Exception { Configuration config = HBaseConfiguration.create(); - MockHTable.Provider tableProvider = new MockHTable.Provider(); + MockProvider tableProvider = new MockProvider(); tableProvider.addToCache(TABLE_NAME, CF); table = (MockHTable)tableProvider.getTable(config, TABLE_NAME); // setup the client @@ -79,21 +77,20 @@ public static void setup() throws Exception { AccessConfig accessConfig = new AccessConfig(); accessConfig.setColumnFamily(CF); accessConfig.setTable(TABLE_NAME); - accessConfig.setTableProvider((TableProvider) (config1, tableName) -> table); + accessConfig.setTableProvider(tableProvider); Map globalConfig = new HashMap() {{ put("es.clustername", "metron"); put("es.port", "9300"); put("es.ip", "localhost"); put("es.date.format", dateFormat); }}; + accessConfig.setGlobalConfigSupplier(() -> globalConfig); esDao = new ElasticsearchDao(); dao = new MultiIndexDao(hbaseDao, esDao); - dao.init(globalConfig, accessConfig); - configurations = mock(WriterConfiguration.class); - when(configurations.getIndex(any())).thenReturn(SENSOR_NAME); - when(configurations.getGlobalConfig()).thenReturn(globalConfig); + dao.init(accessConfig); + } @AfterClass @@ -143,20 +140,25 @@ public void test() throws Exception { Map message0 = new HashMap(inputData.get(0)) {{ put("new-field", "metron"); }}; - String message0Json = JSONUtils.INSTANCE.toJSON(message0, true); String uuid = "" + message0.get(Constants.GUID); - Mutation mutation = Mutation.of(MutationOperation.REPLACE, message0Json); - dao.update(uuid, SENSOR_NAME, mutation, Optional.empty(), configurations); + dao.replace(new ReplaceRequest(){{ + setReplacement(message0); + setUuid(uuid); + setSensorType(SENSOR_NAME); + }}, Optional.empty()); Assert.assertEquals(1, table.size()); Document doc = dao.getLatest(uuid, SENSOR_NAME); - Assert.assertEquals(message0Json, doc.getDocument()); + Assert.assertEquals(message0, doc.getDocument()); { //ensure hbase is up to date Get g = new Get(uuid.getBytes()); Result r = table.get(g); NavigableMap columns = r.getFamilyMap(CF.getBytes()); Assert.assertEquals(1, columns.size()); - Assert.assertEquals(message0Json, new String(columns.lastEntry().getValue())); + Assert.assertEquals(message0 + , JSONUtils.INSTANCE.load(new String(columns.lastEntry().getValue()) + , new TypeReference>() {}) + ); } { //ensure ES is up-to-date @@ -180,21 +182,27 @@ public void test() throws Exception { Map message0 = new HashMap(inputData.get(0)) {{ put("new-field", "metron2"); }}; - String message0Json = JSONUtils.INSTANCE.toJSON(message0, true); String uuid = "" + message0.get(Constants.GUID); - Mutation mutation = Mutation.of(MutationOperation.REPLACE, message0Json); - dao.update(uuid, SENSOR_NAME, mutation, Optional.empty(), configurations); + dao.replace(new ReplaceRequest(){{ + setReplacement(message0); + setUuid(uuid); + setSensorType(SENSOR_NAME); + }}, Optional.empty()); Assert.assertEquals(1, table.size()); Document doc = dao.getLatest(uuid, SENSOR_NAME); - Assert.assertEquals(message0Json, doc.getDocument()); + Assert.assertEquals(message0, doc.getDocument()); { //ensure hbase is up to date Get g = new Get(uuid.getBytes()); Result r = table.get(g); NavigableMap columns = r.getFamilyMap(CF.getBytes()); Assert.assertEquals(2, columns.size()); - Assert.assertEquals(message0Json, new String(columns.lastEntry().getValue())); - Assert.assertNotEquals(message0Json, new String(columns.firstEntry().getValue())); + Assert.assertEquals(message0, JSONUtils.INSTANCE.load(new String(columns.lastEntry().getValue()) + , new TypeReference>() {}) + ); + Assert.assertNotEquals(message0, JSONUtils.INSTANCE.load(new String(columns.firstEntry().getValue()) + , new TypeReference>() {}) + ); } { //ensure ES is up-to-date diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java index ee713f7c48..7dc03b8a91 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java @@ -22,14 +22,24 @@ import java.util.HashMap; import java.util.Map; +import java.util.function.Supplier; public class AccessConfig { private Integer maxSearchResults; + private Supplier> globalConfigSupplier; private Map optionalSettings = new HashMap<>(); private TableProvider tableProvider = null; private String columnFamily; private String table; + public Supplier> getGlobalConfigSupplier() { + return globalConfigSupplier; + } + + public void setGlobalConfigSupplier(Supplier> globalConfigSupplier) { + this.globalConfigSupplier = globalConfigSupplier; + } + public Integer getMaxSearchResults() { return maxSearchResults; } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java index 0a7c1590dd..2fbdf83b87 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java @@ -25,9 +25,11 @@ import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.util.Bytes; import org.apache.metron.common.configuration.writer.WriterConfiguration; +import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.indexing.dao.search.InvalidSearchException; import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; +import org.apache.metron.indexing.dao.update.Document; import java.io.IOException; import java.util.Map; @@ -47,7 +49,7 @@ public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchEx } @Override - public synchronized void init(Map globalConfig, AccessConfig config) { + public synchronized void init(AccessConfig config) { if(tableInterface == null) { try { tableInterface = config.getTableProvider().getTable(HBaseConfiguration.create(), config.getTable()); @@ -79,11 +81,12 @@ public Document getLatest(String uuid, String sensorType) throws IOException { } @Override - public void update(Document update, WriterConfiguration configurations) throws IOException { + public void update(Document update) throws IOException { Put put = new Put(update.getUuid().getBytes()); long ts = update.getTimestamp() == null?System.currentTimeMillis():update.getTimestamp(); byte[] columnQualifier = Bytes.toBytes(ts); - put.addColumn(cf, columnQualifier, Bytes.toBytes(update.getDocument())); + byte[] doc = JSONUtils.INSTANCE.toJSON(update.getDocument()); + put.addColumn(cf, columnQualifier, doc); tableInterface.put(put); } } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java index d53be2ef0d..8196fa4d3d 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java @@ -17,14 +17,18 @@ */ package org.apache.metron.indexing.dao; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JsonNode; -import org.apache.metron.common.configuration.writer.WriterConfiguration; +import com.flipkart.zjsonpatch.JsonPatch; import org.apache.metron.common.utils.JSONUtils; +import org.apache.metron.indexing.dao.search.GetRequest; import org.apache.metron.indexing.dao.search.InvalidSearchException; import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; -import org.apache.metron.indexing.mutation.Mutation; -import org.apache.metron.indexing.mutation.MutationException; +import org.apache.metron.indexing.dao.update.Document; +import org.apache.metron.indexing.dao.update.PatchRequest; +import org.apache.metron.indexing.dao.update.ReplaceRequest; +import org.apache.metron.indexing.dao.update.OriginalNotFoundException; import java.io.IOException; import java.util.Map; @@ -32,40 +36,56 @@ public interface IndexDao { SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException; - void init(Map globalConfig, AccessConfig config); + void init(AccessConfig config); Document getLatest(String uuid, String sensorType) throws IOException; - void update(Document update, WriterConfiguration configurations) throws IOException; - default void update( final Document original - , Mutation mutation - , Optional timestamp - , WriterConfiguration configurations - ) throws IOException, MutationException - { - String mutated = null; - try { - mutated = - mutation.apply(() -> { - try { - return JSONUtils.INSTANCE.load(original.document, JsonNode.class); - } catch (IOException e) { - throw new IllegalStateException(e.getMessage(), e); - } - }); + default Optional> getLatestResult(GetRequest request) throws IOException { + Document ret = getLatest(request.getUuid(), request.getSensorType()); + if(ret == null) { + return Optional.empty(); } - catch(Exception ex) { - throw new MutationException(ex.getMessage(), ex); + else { + return Optional.ofNullable(ret.getDocument()); } - Document updated = new Document(mutated, original.getUuid(), original.getSensorType(), timestamp.orElse(null)); - update(updated, configurations); } - default void update(String uuid, String sensorType, Mutation mutation, Optional timestamp, WriterConfiguration configurations) throws IOException, MutationException - { - Document latest = getLatest(uuid, sensorType); + void update(Document update) throws IOException; + + + default void patch( PatchRequest request + , Optional timestamp + ) throws OriginalNotFoundException, IOException { + Map latest = request.getSource(); if(latest == null) { - throw new IllegalStateException("Unable to retrieve message with UUID: " + uuid + " please use the update() method that specifies the document."); + Document latestDoc = getLatest(request.getUuid(), request.getSensorType()); + if(latestDoc.getDocument() != null) { + latest = latestDoc.getDocument(); + } + else { + throw new OriginalNotFoundException("Unable to patch an document that doesn't exist and isn't specified."); + } } - update(latest, mutation, timestamp, configurations); + JsonNode originalNode = JSONUtils.INSTANCE.convert(latest, JsonNode.class); + JsonNode patched = JsonPatch.apply(request.getPatch(), originalNode); + Map updated = JSONUtils.INSTANCE.getMapper() + .convertValue(patched, new TypeReference>() {}); + Document d = new Document(updated + , request.getUuid() + , request.getSensorType() + , timestamp.orElse(System.currentTimeMillis()) + ); + update(d); } + + default void replace( ReplaceRequest request + , Optional timestamp + ) throws IOException { + Document d = new Document(request.getReplacement() + , request.getUuid() + , request.getSensorType() + , timestamp.orElse(System.currentTimeMillis()) + ); + update(d); + } + } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDaoFactory.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDaoFactory.java index 946fd99e5d..25eab93e91 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDaoFactory.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDaoFactory.java @@ -36,7 +36,7 @@ public static List create( String daoImpls for(String daoImpl : Splitter.on(",").split(daoImpls)) { Class clazz = (Class) Class.forName(daoImpl); IndexDao instance = clazz.getConstructor().newInstance(); - instance.init(globalConfig, config); + instance.init(config); ret.add(instance); } return ret; diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexUpdateCallback.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexUpdateCallback.java index 1022a23e1d..4cb4d7e985 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexUpdateCallback.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexUpdateCallback.java @@ -18,6 +18,8 @@ package org.apache.metron.indexing.dao; +import org.apache.metron.indexing.dao.update.Document; + import java.io.IOException; public interface IndexUpdateCallback { diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java index 976d281419..3a14c2540d 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java @@ -25,6 +25,7 @@ import org.apache.metron.indexing.dao.search.InvalidSearchException; import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; +import org.apache.metron.indexing.dao.update.Document; import java.io.IOException; import java.util.*; @@ -51,11 +52,11 @@ public MultiIndexDao(Iterable composedDao, Function exceptions = indices.parallelStream().map(dao -> { try { - dao.update(update, configurations); + dao.update(update); return null; } catch (Throwable e) { return dao.getClass() + ": " + e.getMessage() + "\n" + ExceptionUtils.getStackTrace(e); @@ -97,9 +98,9 @@ public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchEx } @Override - public void init(Map globalConfig, AccessConfig config) { + public void init(AccessConfig config) { for(IndexDao dao : indices) { - dao.init(globalConfig, config); + dao.init(config); } } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java new file mode 100644 index 0000000000..4cf055f40e --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java @@ -0,0 +1,22 @@ +package org.apache.metron.indexing.dao.search; + +public class GetRequest { + String uuid; + String sensorType; + + public String getUuid() { + return uuid; + } + + public void setUuid(String uuid) { + this.uuid = uuid; + } + + public String getSensorType() { + return sensorType; + } + + public void setSensorType(String sensorType) { + this.sensorType = sensorType; + } +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/Document.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java similarity index 66% rename from metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/Document.java rename to metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java index 76b45ac471..20cc6bdf5a 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/Document.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/Document.java @@ -16,25 +16,43 @@ * limitations under the License. */ -package org.apache.metron.indexing.dao; +package org.apache.metron.indexing.dao.update; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.metron.common.utils.JSONUtils; + +import java.io.IOException; +import java.util.Map; public class Document { Long timestamp; - String document; + Map document; String uuid; String sensorType; - public Document(String document, String uuid, String sensorType, Long timestamp) { + public Document(Map document, String uuid, String sensorType, Long timestamp) { setDocument(document); setUuid(uuid); setTimestamp(timestamp); setSensorType(sensorType); } - public Document(String document, String uuid, String sensorType) { + + public Document(String document, String uuid, String sensorType, Long timestamp) throws IOException { + this(convertDoc(document), uuid, sensorType, timestamp); + } + + public Document(String document, String uuid, String sensorType) throws IOException { this( document, uuid, sensorType, null); } + private static Map convertDoc(String document) throws IOException { + return JSONUtils.INSTANCE.load(document, new TypeReference>() { + }); + } + public String getSensorType() { return sensorType; } @@ -51,11 +69,11 @@ public void setTimestamp(Long timestamp) { this.timestamp = timestamp != null?timestamp:System.currentTimeMillis(); } - public String getDocument() { + public Map getDocument() { return document; } - public void setDocument(String document) { + public void setDocument(Map document) { this.document = document; } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationException.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/OriginalNotFoundException.java similarity index 78% rename from metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationException.java rename to metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/OriginalNotFoundException.java index b0ee817234..87a03ae383 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationException.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/OriginalNotFoundException.java @@ -15,10 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.metron.indexing.mutation; +package org.apache.metron.indexing.dao.update; -public class MutationException extends Exception { - public MutationException(String s, Exception e) { +public class OriginalNotFoundException extends Exception { + public OriginalNotFoundException(String s) { + super(s); + } + + public OriginalNotFoundException(String s, Exception e) { super(s, e); } } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java new file mode 100644 index 0000000000..c2cb81bcbc --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java @@ -0,0 +1,44 @@ +package org.apache.metron.indexing.dao.update; + +import com.fasterxml.jackson.databind.JsonNode; + +import java.util.Map; + +public class PatchRequest { + JsonNode patch; + Map source; + String uuid; + String sensorType; + + public JsonNode getPatch() { + return patch; + } + + public void setPatch(JsonNode patch) { + this.patch = patch; + } + + public Map getSource() { + return source; + } + + public void setSource(Map source) { + this.source = source; + } + + public String getUuid() { + return uuid; + } + + public void setUuid(String uuid) { + this.uuid = uuid; + } + + public String getSensorType() { + return sensorType; + } + + public void setSensorType(String sensorType) { + this.sensorType = sensorType; + } +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java new file mode 100644 index 0000000000..3c438d95ad --- /dev/null +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java @@ -0,0 +1,35 @@ +package org.apache.metron.indexing.dao.update; + +import com.fasterxml.jackson.databind.JsonNode; + +import java.util.Map; + +public class ReplaceRequest { + Map replacement; + String uuid; + String sensorType; + + public String getSensorType() { + return sensorType; + } + + public void setSensorType(String sensorType) { + this.sensorType = sensorType; + } + + public Map getReplacement() { + return replacement; + } + + public void setReplacement(Map replacement) { + this.replacement = replacement; + } + + public String getUuid() { + return uuid; + } + + public void setUuid(String uuid) { + this.uuid = uuid; + } +} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutation.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutation.java deleted file mode 100644 index a29736c96f..0000000000 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutation.java +++ /dev/null @@ -1,63 +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.indexing.mutation; - -import com.fasterxml.jackson.databind.JsonNode; - -import java.io.Serializable; -import java.util.function.Supplier; - -public class Mutation implements Serializable { - MutationOperation mutator; - String mutationArg; - - public Mutation(MutationOperation mutator, String mutationArg) { - this.mutator = mutator; - this.mutationArg = mutationArg; - } - - public static Mutation of(MutationOperation mutator, String mutationArg) { - return new Mutation(mutator, mutationArg); - } - - /** - * Applies this function to the given argument. - * - * @param jsonNodeSupplier the function argument - * @return the function result - */ - public String apply(Supplier jsonNodeSupplier) throws MutationException { - return mutator.mutate(jsonNodeSupplier, mutationArg); - } - - public MutationOperation getMutator() { - return mutator; - } - - public void setMutator(MutationOperation mutator) { - this.mutator = mutator; - } - - public String getMutationArg() { - return mutationArg; - } - - public void setMutationArg(String mutationArg) { - this.mutationArg = mutationArg; - } -} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationOperation.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationOperation.java deleted file mode 100644 index 6beeac92ac..0000000000 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/MutationOperation.java +++ /dev/null @@ -1,42 +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.indexing.mutation; - -import com.fasterxml.jackson.databind.JsonNode; -import org.apache.metron.indexing.mutation.mutators.Patch; -import org.apache.metron.indexing.mutation.mutators.Replace; - -import java.util.function.Supplier; - -public enum MutationOperation implements Mutator{ - PATCH(new Patch()), - REPLACE(new Replace()) - ; - - Mutator mutator; - - MutationOperation(Mutator mutator) { - this.mutator = mutator; - } - - @Override - public String mutate(Supplier originalNode, String arg) throws MutationException { - return this.mutator.mutate(originalNode, arg); - } -} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutator.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutator.java deleted file mode 100644 index db3b50a3ab..0000000000 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Mutator.java +++ /dev/null @@ -1,27 +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.indexing.mutation; - -import com.fasterxml.jackson.databind.JsonNode; - -import java.util.function.Supplier; - -public interface Mutator { - String mutate(Supplier original, String arg) throws MutationException; -} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Update.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Update.java deleted file mode 100644 index e4995bef65..0000000000 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/Update.java +++ /dev/null @@ -1,23 +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.indexing.mutation; - -public class Update { - -} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Patch.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Patch.java deleted file mode 100644 index b4be82c83f..0000000000 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Patch.java +++ /dev/null @@ -1,40 +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.indexing.mutation.mutators; - -import com.fasterxml.jackson.databind.JsonNode; -import com.flipkart.zjsonpatch.JsonPatch; -import org.apache.metron.common.utils.JSONUtils; -import org.apache.metron.indexing.mutation.MutationException; -import org.apache.metron.indexing.mutation.Mutator; - -import java.util.function.Supplier; - -public class Patch implements Mutator { - @Override - public String mutate(Supplier originalSupplier, String arg) throws MutationException { - final JsonNode orig = originalSupplier.get(); - try { - JsonNode out = JsonPatch.apply(JSONUtils.INSTANCE.load(arg, JsonNode.class), orig); - return new String(JSONUtils.INSTANCE.toJSON(out)); - - } catch (Exception e) { - throw new MutationException("Unable to mutate: " + orig.asText() + " with " + arg + " because " + e.getMessage(), e); - } - } -} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Replace.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Replace.java deleted file mode 100644 index 2503d0ba1b..0000000000 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/mutation/mutators/Replace.java +++ /dev/null @@ -1,32 +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.indexing.mutation.mutators; - -import com.fasterxml.jackson.databind.JsonNode; -import org.apache.metron.indexing.mutation.MutationException; -import org.apache.metron.indexing.mutation.Mutator; - -import java.util.function.Supplier; - -public class Replace implements Mutator { - - @Override - public String mutate(Supplier original, String arg) throws MutationException { - return arg; - } -} diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java index 07c4bcbc9e..8078dab206 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java @@ -25,6 +25,7 @@ import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.indexing.dao.search.*; +import org.apache.metron.indexing.dao.update.Document; import java.io.IOException; import java.util.*; @@ -129,7 +130,7 @@ private static Map parse(String doc) { } @Override - public void init(Map globalConfig, AccessConfig config) { + public void init(AccessConfig config) { this.config = config; } @@ -149,7 +150,7 @@ public Document getLatest(String uuid, String sensorType) throws IOException { } @Override - public void update(Document update, WriterConfiguration configurations) throws IOException { + public void update(Document update) throws IOException { for(Map.Entry> kv: BACKING_STORE.entrySet()) { if (kv.getKey().startsWith(update.getSensorType())) { for(Iterator it = kv.getValue().iterator();it.hasNext();) { @@ -159,7 +160,7 @@ public void update(Document update, WriterConfiguration configurations) throws I it.remove(); } } - kv.getValue().add(update.getDocument()); + kv.getValue().add(JSONUtils.INSTANCE.toJSON(update.getDocument(), true)); } } } diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/IndexingDaoIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/SearchIntegrationTest.java similarity index 99% rename from metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/IndexingDaoIntegrationTest.java rename to metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/SearchIntegrationTest.java index f469f1f95f..7813d25630 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/IndexingDaoIntegrationTest.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/SearchIntegrationTest.java @@ -29,7 +29,7 @@ import java.util.List; -public abstract class IndexingDaoIntegrationTest { +public abstract class SearchIntegrationTest { /** * [ * {"source:type": "bro", "ip_src_addr":"192.168.1.1", "ip_src_port": 8010, "timestamp":1, "rejected":true}, diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/mutation/MutationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/mutation/MutationTest.java deleted file mode 100644 index 5d65f6f435..0000000000 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/mutation/MutationTest.java +++ /dev/null @@ -1,119 +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.indexing.mutation; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.JsonNode; -import com.google.common.collect.Sets; -import org.adrianwalker.multilinestring.Multiline; -import org.apache.metron.common.utils.JSONUtils; -import org.json.simple.JSONObject; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -public class MutationTest { - - @Test - public void testReplace() throws Exception { - Mutation m = Mutation.of(MutationOperation.REPLACE, "{ \"a\" : 1 }"); - String out = m.apply( - () -> { - try { - return JSONUtils.INSTANCE.load("{ \"b\" : 1 }", JsonNode.class); - } catch (IOException e) { - throw new IllegalStateException(e); - } - } - - ); - Assert.assertEquals(m.mutationArg, out); - } - - /** - [ - { "op": "add" - , "path": "/b" - , "value": "metron" - } - ] - */ - @Multiline - public static String addElement; - - @Test - public void testPatch_add() throws Exception { - final Map orig = new HashMap() {{ - put("a", 1); - }}; - Map out = apply(Mutation.of(MutationOperation.PATCH, addElement), orig); - mapEquals(out, new HashMap() {{ - putAll(orig); - put("b", "metron"); - }}); - } - - - /** - [ - { "op": "add" - , "path": "/b" - , "value": "metron" - }, - { "op": "remove" - , "path": "/a" - } - ] - */ - @Multiline - public static String addAndThenRemoveElement; - - @Test - public void testPatch_addThenRemove() throws Exception { - final Map orig = new HashMap() {{ - put("a", 1); - }}; - Map out = apply(Mutation.of(MutationOperation.PATCH, addAndThenRemoveElement) , orig); - mapEquals(out, new HashMap() {{ - put("b", "metron"); - }}); - } - - public static Map apply(Mutation m, Map orig) throws Exception { - String out = m.apply( - () -> { - try { - String origStr = new JSONObject(orig).toJSONString(); - return JSONUtils.INSTANCE.load(origStr, JsonNode.class); - } catch (IOException e) { - throw new IllegalStateException(e); - } - } - - ); - return JSONUtils.INSTANCE.load(out, new TypeReference>() { - }); - } - - public static void mapEquals(Map m1, Map m2) { - Assert.assertEquals(0, Sets.symmetricDifference(m1.entrySet(), m2.entrySet()).size()); - } -} From 87c446b1fe41316a087ba2abac4faa5342507a01 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 20 Jul 2017 19:44:38 +0100 Subject: [PATCH 04/38] Added licenses --- .../metron/rest/service/UpdateService.java | 17 +++++++++++++++++ .../rest/service/impl/UpdateServiceImpl.java | 17 +++++++++++++++++ .../rest/controller/DaoControllerTest.java | 17 +++++++++++++++++ .../UpdateControllerIntegrationTest.java | 17 +++++++++++++++++ .../metron/indexing/dao/search/GetRequest.java | 17 +++++++++++++++++ .../indexing/dao/update/PatchRequest.java | 17 +++++++++++++++++ .../indexing/dao/update/ReplaceRequest.java | 17 +++++++++++++++++ 7 files changed, 119 insertions(+) diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/UpdateService.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/UpdateService.java index 55c0698656..4cdf4b3c0c 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/UpdateService.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/UpdateService.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.rest.service; import org.apache.metron.indexing.dao.update.OriginalNotFoundException; diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/UpdateServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/UpdateServiceImpl.java index 588a9679ab..847173e716 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/UpdateServiceImpl.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/UpdateServiceImpl.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.rest.service.impl; import org.apache.metron.indexing.dao.IndexDao; diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/DaoControllerTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/DaoControllerTest.java index 45ce06af68..096f1be687 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/DaoControllerTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/DaoControllerTest.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.rest.controller; import com.google.common.collect.ImmutableMap; diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java index 7fe194faf1..603d1a83f7 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.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.rest.controller; import org.adrianwalker.multilinestring.Multiline; diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java index 4cf055f40e..cc847b6dc9 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.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.indexing.dao.search; public class GetRequest { diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java index c2cb81bcbc..1195fb71bf 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.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.indexing.dao.update; import com.fasterxml.jackson.databind.JsonNode; diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java index 3c438d95ad..be78591154 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.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.indexing.dao.update; import com.fasterxml.jackson.databind.JsonNode; From c36c12c80a64d9575395475114695746885e3f52 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 20 Jul 2017 20:59:10 +0100 Subject: [PATCH 05/38] Updating to add guid as the proper ID. --- .../metron/elasticsearch/writer/ElasticsearchWriter.java | 6 ++++++ .../integration/ElasticsearchUpdateIntegrationTest.java | 3 ++- 2 files changed, 8 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 cba2be6d27..af606de3e7 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,6 +17,7 @@ */ package org.apache.metron.elasticsearch.writer; +import org.apache.metron.common.Constants; import org.apache.metron.elasticsearch.utils.ElasticsearchUtils; import org.apache.storm.task.TopologyContext; import org.apache.storm.tuple.Tuple; @@ -91,6 +92,11 @@ public BulkWriterResponse write(String sensorType, WriterConfiguration configura sensorType + "_doc"); indexRequestBuilder = indexRequestBuilder.setSource(esDoc.toJSONString()); + String guid = (String)esDoc.get(Constants.GUID); + if(guid != null) { + indexRequestBuilder.setId(guid); + } + Object ts = esDoc.get("timestamp"); if(ts != null) { indexRequestBuilder = indexRequestBuilder.setTimestamp(ts.toString()); 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 1ef88fb991..5914c87119 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 @@ -220,8 +220,9 @@ public void test() throws Exception { Assert.fail("Elasticsearch is not updated!"); } } - } } + + } From b1989dee7810f0f0e8f5bc5fbd45879e7f4b63d2 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 20 Jul 2017 21:51:49 +0100 Subject: [PATCH 06/38] Updating to include upsert and optional search for index. --- .../elasticsearch/dao/ElasticsearchDao.java | 70 +++++++++++++++---- .../apache/metron/indexing/dao/HBaseDao.java | 3 +- .../apache/metron/indexing/dao/IndexDao.java | 9 +-- .../metron/indexing/dao/MultiIndexDao.java | 4 +- .../indexing/dao/search/SearchResult.java | 9 +++ .../metron/indexing/dao/update/Document.java | 1 + .../indexing/dao/update/PatchRequest.java | 9 +++ .../indexing/dao/update/ReplaceRequest.java | 9 +++ .../metron/indexing/dao/InMemoryDao.java | 2 +- 9 files changed, 93 insertions(+), 23 deletions(-) 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 e99e1b5364..7740cd873a 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 @@ -29,7 +29,12 @@ import org.apache.metron.indexing.dao.search.*; import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; +import org.elasticsearch.action.get.GetRequestBuilder; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.search.*; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.index.query.QueryStringQueryBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; @@ -44,6 +49,9 @@ import java.util.Arrays; import java.util.Date; import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.function.Function; import java.util.stream.Collectors; public class ElasticsearchDao implements IndexDao { @@ -92,6 +100,7 @@ public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchEx searchResult.setId(searchHit.getId()); searchResult.setSource(searchHit.getSource()); searchResult.setScore(searchHit.getScore()); + searchResult.setId(searchHit.getIndex()); return searchResult; }).collect(Collectors.toList())); return searchResponse; @@ -106,7 +115,25 @@ public synchronized void init(AccessConfig config) { } @Override - public Document getLatest(String uuid, String sensorType) throws IOException { + public Document getLatest(final String uuid, final String sensorType) throws IOException { + Optional ret = searchByUuid( + uuid + , sensorType + , hit -> { + Long ts = 0L; + String doc = hit.getSourceAsString(); + String sourceType = Iterables.getFirst(Splitter.on("_doc").split(hit.getType()), null); + try { + return Optional.of(new Document(doc, uuid, sourceType, ts)); + } catch (IOException e) { + throw new IllegalStateException("Unable to retrieve latest: " + e.getMessage(), e); + } + } + ); + return ret.orElse(null); + } + + Optional searchByUuid(String uuid, String sensorType, Function> callback) throws IOException{ QueryBuilder query = QueryBuilders.matchQuery(Constants.GUID, uuid); SearchRequestBuilder request = client.prepareSearch() .setTypes(sensorType + "_doc") @@ -123,33 +150,46 @@ public Document getLatest(String uuid, String sensorType) throws IOException { org.elasticsearch.action.search.SearchResponse resp = i.getResponse(); SearchHits hits = resp.getHits(); for(SearchHit hit : hits) { - Long ts = 0L; - String doc = hit.getSourceAsString(); - String sourceType = Iterables.getFirst(Splitter.on("_doc").split(hit.getType()), null); - Document d = new Document(doc, uuid, sourceType, ts); - return d; + Optional ret = callback.apply(hit); + if(ret.isPresent()) { + return ret; + } } } - return null; + return Optional.empty(); + } @Override - public void update(Document update) throws IOException { + public void update(Document update, Optional index) throws IOException { String indexPostfix = ElasticsearchUtils.getIndexFormat(accessConfig.getGlobalConfigSupplier().get()).format(new Date()); String sensorType = update.getSensorType(); String indexName = ElasticsearchUtils.getIndexName(sensorType, indexPostfix, null); - IndexRequestBuilder indexRequestBuilder = client.prepareIndex(indexName, - sensorType + "_doc"); - indexRequestBuilder = indexRequestBuilder.setSource(new String(JSONUtils.INSTANCE.toJSON(update.getDocument()))); + String type = sensorType + "_doc"; + byte[] source = JSONUtils.INSTANCE.toJSON(update.getDocument()); Object ts = update.getTimestamp(); + IndexRequest indexRequest = new IndexRequest(indexName, type, update.getUuid()) + .source(source) + ; if(ts != null) { - indexRequestBuilder = indexRequestBuilder.setTimestamp(ts.toString()); + indexRequest = indexRequest.timestamp(ts.toString()); } + String existingIndex = index.orElse( + searchByUuid(update.getUuid() + , sensorType + , hit -> Optional.ofNullable(hit.getIndex()) + ).orElse(indexName) + ); + UpdateRequest updateRequest = new UpdateRequest(existingIndex, type, update.getUuid()) + .doc(source) + .upsert(indexRequest) + ; - BulkResponse bulkResponse = client.prepareBulk().add(indexRequestBuilder).execute().actionGet(); - if(bulkResponse.hasFailures()) { - throw new IOException(bulkResponse.buildFailureMessage()); + try { + client.update(updateRequest).get(); + } catch (Exception e) { + throw new IOException(e.getMessage(), e); } } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java index 2fbdf83b87..eb4953fccc 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java @@ -34,6 +34,7 @@ import java.io.IOException; import java.util.Map; import java.util.NavigableMap; +import java.util.Optional; public class HBaseDao implements IndexDao { private HTableInterface tableInterface; @@ -81,7 +82,7 @@ public Document getLatest(String uuid, String sensorType) throws IOException { } @Override - public void update(Document update) throws IOException { + public void update(Document update, Optional index) throws IOException { Put put = new Put(update.getUuid().getBytes()); long ts = update.getTimestamp() == null?System.currentTimeMillis():update.getTimestamp(); byte[] columnQualifier = Bytes.toBytes(ts); diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java index 8196fa4d3d..d5713884fd 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java @@ -35,6 +35,7 @@ import java.util.Optional; public interface IndexDao { + SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException; void init(AccessConfig config); Document getLatest(String uuid, String sensorType) throws IOException; @@ -49,7 +50,7 @@ default Optional> getLatestResult(GetRequest request) throws } } - void update(Document update) throws IOException; + void update(Document update, Optional index) throws IOException; default void patch( PatchRequest request @@ -69,12 +70,12 @@ default void patch( PatchRequest request JsonNode patched = JsonPatch.apply(request.getPatch(), originalNode); Map updated = JSONUtils.INSTANCE.getMapper() .convertValue(patched, new TypeReference>() {}); - Document d = new Document(updated + Document d = new Document( updated , request.getUuid() , request.getSensorType() , timestamp.orElse(System.currentTimeMillis()) ); - update(d); + update(d, Optional.ofNullable(request.getIndex())); } default void replace( ReplaceRequest request @@ -85,7 +86,7 @@ default void replace( ReplaceRequest request , request.getSensorType() , timestamp.orElse(System.currentTimeMillis()) ); - update(d); + update(d, Optional.ofNullable(request.getIndex())); } } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java index 3a14c2540d..8b5ab0fe8a 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java @@ -52,11 +52,11 @@ public MultiIndexDao(Iterable composedDao, Function index) throws IOException { List exceptions = indices.parallelStream().map(dao -> { try { - dao.update(update); + dao.update(update, index); return null; } catch (Throwable e) { return dao.getClass() + ": " + e.getMessage() + "\n" + ExceptionUtils.getStackTrace(e); diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchResult.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchResult.java index ae4f9bd37c..4df07813c7 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchResult.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchResult.java @@ -24,6 +24,15 @@ public class SearchResult { private String id; private Map source; private float score; + private String index; + + public String getIndex() { + return index; + } + + public void setIndex(String index) { + this.index = index; + } public String getId() { return id; 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 20cc6bdf5a..a28dfd57d5 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 @@ -25,6 +25,7 @@ import java.io.IOException; import java.util.Map; +import java.util.Optional; public class Document { Long timestamp; diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java index 1195fb71bf..2f8cd15e5c 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java @@ -26,6 +26,15 @@ public class PatchRequest { Map source; String uuid; String sensorType; + String index; + + public String getIndex() { + return index; + } + + public void setIndex(String index) { + this.index = index; + } public JsonNode getPatch() { return patch; diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java index be78591154..d7ec921f3a 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java @@ -25,6 +25,15 @@ public class ReplaceRequest { Map replacement; String uuid; String sensorType; + String index; + + public String getIndex() { + return index; + } + + public void setIndex(String index) { + this.index = index; + } public String getSensorType() { return sensorType; diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java index 8078dab206..287fb2b722 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java @@ -150,7 +150,7 @@ public Document getLatest(String uuid, String sensorType) throws IOException { } @Override - public void update(Document update) throws IOException { + public void update(Document update, Optional index) throws IOException { for(Map.Entry> kv: BACKING_STORE.entrySet()) { if (kv.getKey().startsWith(update.getSensorType())) { for(Iterator it = kv.getValue().iterator();it.hasNext();) { From 9b1c738a8aee880f0b500cd16cb76f684acdd981 Mon Sep 17 00:00:00 2001 From: cstella Date: Tue, 25 Jul 2017 10:34:24 +0100 Subject: [PATCH 07/38] Adjusting tests for table provider. --- metron-platform/metron-data-management/pom.xml | 7 +++++++ .../dataloads/nonbulk/taxii/TaxiiIntegrationTest.java | 10 ++++++---- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/metron-platform/metron-data-management/pom.xml b/metron-platform/metron-data-management/pom.xml index fbfd71aae1..90c2c52728 100644 --- a/metron-platform/metron-data-management/pom.xml +++ b/metron-platform/metron-data-management/pom.xml @@ -114,6 +114,13 @@ metron-hbase ${project.parent.version} + + org.apache.metron + metron-hbase + ${project.parent.version} + test + test-jar + org.mitre.taxii taxii diff --git a/metron-platform/metron-data-management/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiIntegrationTest.java b/metron-platform/metron-data-management/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiIntegrationTest.java index 0223514bfa..4af7e2183b 100644 --- a/metron-platform/metron-data-management/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiIntegrationTest.java +++ b/metron-platform/metron-data-management/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiIntegrationTest.java @@ -31,8 +31,10 @@ import org.apache.metron.enrichment.converter.EnrichmentConverter; import org.apache.metron.enrichment.converter.EnrichmentKey; import org.apache.metron.enrichment.converter.EnrichmentValue; -import org.apache.metron.test.mock.MockHTable; import org.apache.metron.enrichment.lookup.LookupKV; +import org.apache.metron.hbase.HTableProvider; +import org.apache.metron.hbase.mock.MockHTable; +import org.apache.metron.hbase.mock.MockProvider; import org.junit.*; import java.io.IOException; @@ -49,7 +51,7 @@ public static void setup() throws IOException { @AfterClass public static void teardown() { MockTaxiiService.shutdown(); - MockHTable.Provider.clear(); + MockProvider.clear(); } /** @@ -91,7 +93,7 @@ public void testCommandLine() throws Exception { @Test public void testTaxii() throws Exception { - final MockHTable.Provider provider = new MockHTable.Provider(); + final MockProvider provider = new MockProvider(); final Configuration config = HBaseConfiguration.create(); TaxiiHandler handler = new TaxiiHandler(TaxiiConnectionConfig.load(taxiiConnectionConfig), new StixExtractor(), config ) { @Override @@ -115,7 +117,7 @@ protected synchronized HTableInterface createHTable(String tableInfo) throws IOE } Assert.assertTrue(maliciousAddresses.contains("94.102.53.142")); Assert.assertEquals(numStringsMatch(MockTaxiiService.pollMsg, "AddressObj:Address_Value condition=\"Equal\""), maliciousAddresses.size()); - MockHTable.Provider.clear(); + MockProvider.clear(); // Ensure that the handler can be run multiple times without connection issues. handler.run(); From 49898ffc5da2ea7c59912feaa50d2c7fd263bf6f Mon Sep 17 00:00:00 2001 From: cstella Date: Tue, 25 Jul 2017 10:53:22 +0100 Subject: [PATCH 08/38] Search index, not search id..duh --- .../org/apache/metron/elasticsearch/dao/ElasticsearchDao.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 7740cd873a..c4146fb669 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 @@ -100,7 +100,7 @@ public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchEx searchResult.setId(searchHit.getId()); searchResult.setSource(searchHit.getSource()); searchResult.setScore(searchHit.getScore()); - searchResult.setId(searchHit.getIndex()); + searchResult.setIndex(searchHit.getIndex()); return searchResult; }).collect(Collectors.toList())); return searchResponse; From b31485d5599f6458649f8e4361dcfdc2e3f29a14 Mon Sep 17 00:00:00 2001 From: cstella Date: Tue, 25 Jul 2017 11:16:25 +0100 Subject: [PATCH 09/38] Missed a rename. --- .../integration/SimpleHbaseEnrichmentWriterIntegrationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java index 6871247778..75140c09a1 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java @@ -55,7 +55,7 @@ public class SimpleHbaseEnrichmentWriterIntegrationTest extends BaseIntegrationT ,"shew.cf" : "cf" ,"shew.keyColumns" : "col2" ,"shew.enrichmentType" : "et" - ,"shew.hbaseProvider" : "org.apache.metron.enrichment.integration.mock.MockTableProvider" + ,"shew.hbaseProvider" : "org.apache.metron.hbase.mock.MockProvider" ,"columns" : { "col1" : 0 ,"col2" : 1 From 9114bbaa9570fa826e7c5ed24fe4fb92d8f2680b Mon Sep 17 00:00:00 2001 From: cstella Date: Tue, 25 Jul 2017 11:49:43 +0100 Subject: [PATCH 10/38] Updating intermittent test. --- .../UpdateControllerIntegrationTest.java | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java index 603d1a83f7..ff101f5bc2 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java @@ -35,6 +35,8 @@ import org.springframework.test.context.ActiveProfiles; import org.springframework.test.context.junit4.SpringRunner; import org.springframework.test.web.servlet.MockMvc; +import org.springframework.test.web.servlet.ResultActions; +import org.springframework.test.web.servlet.request.MockHttpServletRequestBuilder; import org.springframework.test.web.servlet.setup.MockMvcBuilders; import org.springframework.web.context.WebApplicationContext; @@ -124,14 +126,20 @@ public void setup() throws Exception { @Test public void test() throws Exception { String uuid = "bro_index_2017.01.01.01:1"; - this.mockMvc.perform(post(searchUrl + "/findOne").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(findMessage0)) - .andExpect(status().isOk()) + ResultActions result = this.mockMvc.perform(post(searchUrl + "/findOne").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(findMessage0)); + try { + result.andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) .andExpect(jsonPath("$.source:type").value("bro")) .andExpect(jsonPath("$.guid").value(uuid)) .andExpect(jsonPath("$.project").doesNotExist()) .andExpect(jsonPath("$.timestamp").value(2)) - ; + ; + } + catch(Throwable t) { + System.err.println(result.andReturn().getResponse().getContentAsString()); + throw t; + } MockHTable table = (MockHTable) MockProvider.getFromCache(TABLE); Assert.assertEquals(0,table.size()); this.mockMvc.perform(patch(updateUrl+ "/patch").with(httpBasic(user, password)) From e1300e3c9f7bd9345c805834a02de5591326b6f7 Mon Sep 17 00:00:00 2001 From: cstella Date: Tue, 25 Jul 2017 12:57:19 +0100 Subject: [PATCH 11/38] Cleaning some things up. --- .../org/apache/metron/rest/config/IndexConfig.java | 2 +- .../java/org/apache/metron/indexing/dao/HBaseDao.java | 11 ++++++----- .../apache/metron/indexing/dao/IndexDaoFactory.java | 1 - 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java index 56b563b5c7..3734f70299 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java @@ -73,7 +73,7 @@ public IndexDao indexDao() { if (indexDaoImpl == null) { throw new IllegalStateException("You must provide an index DAO implementation via the " + INDEX_DAO_IMPL + " config"); } - IndexDao ret = IndexDaoFactory.combine(IndexDaoFactory.create(indexDaoImpl, globalConfigService.get(), config)); + IndexDao ret = IndexDaoFactory.combine(IndexDaoFactory.create(indexDaoImpl, config)); if (ret == null) { throw new IllegalStateException("IndexDao is unable to be created."); } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java index eb4953fccc..83bf1e92ec 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java @@ -39,19 +39,20 @@ public class HBaseDao implements IndexDao { private HTableInterface tableInterface; private byte[] cf; - + private AccessConfig config; public HBaseDao() { } @Override - public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException { + public synchronized SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException { return null; } @Override public synchronized void init(AccessConfig config) { - if(tableInterface == null) { + if(config == null) { + this.config = config; try { tableInterface = config.getTableProvider().getTable(HBaseConfiguration.create(), config.getTable()); cf = config.getColumnFamily().getBytes(); @@ -62,7 +63,7 @@ public synchronized void init(AccessConfig config) { } @Override - public Document getLatest(String uuid, String sensorType) throws IOException { + public synchronized Document getLatest(String uuid, String sensorType) throws IOException { Get get = new Get(uuid.getBytes()); get.addFamily(cf); Result result = tableInterface.get(get); @@ -82,7 +83,7 @@ public Document getLatest(String uuid, String sensorType) throws IOException { } @Override - public void update(Document update, Optional index) throws IOException { + public synchronized void update(Document update, Optional index) throws IOException { Put put = new Put(update.getUuid().getBytes()); long ts = update.getTimestamp() == null?System.currentTimeMillis():update.getTimestamp(); byte[] columnQualifier = Bytes.toBytes(ts); diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDaoFactory.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDaoFactory.java index 25eab93e91..e8df0b7a91 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDaoFactory.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDaoFactory.java @@ -28,7 +28,6 @@ public class IndexDaoFactory { public static List create( String daoImpls - , Map globalConfig , AccessConfig config ) throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException, InstantiationException { From 9a9438ca63540a3f89a29ff2961e9daf85840056 Mon Sep 17 00:00:00 2001 From: cstella Date: Tue, 25 Jul 2017 13:51:18 +0100 Subject: [PATCH 12/38] Adding better logging. --- .../src/main/java/org/apache/metron/indexing/dao/HBaseDao.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java index 83bf1e92ec..1f93b1fc94 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java @@ -52,10 +52,12 @@ public synchronized SearchResponse search(SearchRequest searchRequest) throws In @Override public synchronized void init(AccessConfig config) { if(config == null) { + System.out.println("Initializing " + config.getTable()); this.config = config; try { tableInterface = config.getTableProvider().getTable(HBaseConfiguration.create(), config.getTable()); cf = config.getColumnFamily().getBytes(); + System.out.println("Initialized " + config.getTable()); } catch (IOException e) { throw new IllegalStateException("Unable to initialize HBaseDao: " + e.getMessage(), e); } From cb33aefae57ad0d532b109e77474378f7330dba7 Mon Sep 17 00:00:00 2001 From: cstella Date: Tue, 25 Jul 2017 14:21:14 +0100 Subject: [PATCH 13/38] Stupid local variable vs global variable. --- .../src/main/java/org/apache/metron/indexing/dao/HBaseDao.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java index 1f93b1fc94..bb4b58b624 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java @@ -51,7 +51,7 @@ public synchronized SearchResponse search(SearchRequest searchRequest) throws In @Override public synchronized void init(AccessConfig config) { - if(config == null) { + if(this.config == null) { System.out.println("Initializing " + config.getTable()); this.config = config; try { From cbd23e4db4d9d64841904393675935f293ab632b Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 26 Jul 2017 11:18:56 +0100 Subject: [PATCH 14/38] Fixing dao. --- .../org/apache/metron/indexing/dao/HBaseDao.java | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java index bb4b58b624..1fb50e231f 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java @@ -51,24 +51,29 @@ public synchronized SearchResponse search(SearchRequest searchRequest) throws In @Override public synchronized void init(AccessConfig config) { - if(this.config == null) { - System.out.println("Initializing " + config.getTable()); + if(this.tableInterface == null) { this.config = config; try { tableInterface = config.getTableProvider().getTable(HBaseConfiguration.create(), config.getTable()); cf = config.getColumnFamily().getBytes(); - System.out.println("Initialized " + config.getTable()); } catch (IOException e) { throw new IllegalStateException("Unable to initialize HBaseDao: " + e.getMessage(), e); } } } + public HTableInterface getTableInterface() { + if(tableInterface == null) { + init(config); + } + return tableInterface; + } + @Override public synchronized Document getLatest(String uuid, String sensorType) throws IOException { Get get = new Get(uuid.getBytes()); get.addFamily(cf); - Result result = tableInterface.get(get); + Result result = getTableInterface().get(get); NavigableMap columns = result.getFamilyMap( cf); if(columns == null || columns.size() == 0) { return null; @@ -91,6 +96,6 @@ public synchronized void update(Document update, Optional index) throws byte[] columnQualifier = Bytes.toBytes(ts); byte[] doc = JSONUtils.INSTANCE.toJSON(update.getDocument()); put.addColumn(cf, columnQualifier, doc); - tableInterface.put(put); + getTableInterface().put(put); } } From b43e574a075d765d62190d09956e13f297280eeb Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 26 Jul 2017 18:08:13 +0100 Subject: [PATCH 15/38] Moving config of hbase table and cf for updates to global config --- .../configuration/metron-indexing-env.xml | 12 ++++ .../package/scripts/indexing_commands.py | 64 +++++++++++++++++++ .../package/scripts/indexing_master.py | 5 ++ .../package/scripts/params/params_linux.py | 3 + .../package/scripts/params/status_params.py | 2 + .../CURRENT/package/templates/global.json.j2 | 4 +- .../metron/rest/MetronRestConstants.java | 2 - .../metron/rest/config/IndexConfig.java | 4 -- .../src/main/resources/application-test.yml | 2 - .../src/main/resources/application.yml | 2 +- .../apache/metron/rest/config/TestConfig.java | 19 +++++- .../UpdateControllerIntegrationTest.java | 15 +++++ .../src/test/resources/zookeeper/global.json | 4 ++ .../ElasticsearchUpdateIntegrationTest.java | 4 +- .../components/ConfigUploadComponent.java | 12 +++- .../metron/indexing/dao/AccessConfig.java | 17 ----- .../apache/metron/indexing/dao/HBaseDao.java | 15 ++++- .../src/main/config/zookeeper/global.json | 4 +- 18 files changed, 156 insertions(+), 34 deletions(-) create mode 100644 metron-interface/metron-rest/src/test/resources/zookeeper/global.json diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-indexing-env.xml b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-indexing-env.xml index e28e32829c..e36730a1c6 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-indexing-env.xml +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/configuration/metron-indexing-env.xml @@ -59,6 +59,18 @@ Indexing Writer Class Name org.apache.metron.elasticsearch.writer.ElasticsearchWriter Indexing Writer Class Name + + + update_table + The HBase table which will hold edits to indexed data + metron_update + Indexing Update Table + + + update_cf + The HBase column family which will hold edits to indexed data + t + Indexing Update Column Family indexing_workers diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/indexing_commands.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/indexing_commands.py index 711d4fcd79..1d8e914070 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/indexing_commands.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/indexing_commands.py @@ -33,6 +33,8 @@ class IndexingCommands: __configured = False __acl_configured = False __hdfs_perm_configured = False + __hbase_configured = False + __hbase_acl_configured = False def __init__(self, params): if params is None: @@ -42,6 +44,8 @@ def __init__(self, params): self.__indexing_topic = params.indexing_input_topic self.__configured = os.path.isfile(self.__params.indexing_configured_flag_file) self.__acl_configured = os.path.isfile(self.__params.indexing_acl_configured_flag_file) + self.__hbase_configured = os.path.isfile(self.__params.indexing_hbase_configured_flag_file) + self.__hbase_acl_configured = os.path.isfile(self.__params.indexing_hbase_acl_configured_flag_file) def is_configured(self): return self.__configured @@ -58,6 +62,66 @@ def set_configured(self): owner=self.__params.metron_user, mode=0755) + def is_hbase_configured(self): + return self.__hbase_configured + + def is_hbase_acl_configured(self): + return self.__hbase_acl_configured + + def set_hbase_configured(self): + Logger.info("Setting HBase Configured to True") + File(self.__params.indexing_hbase_configured_flag_file, + content="", + owner=self.__params.metron_user, + mode=0755) + + def set_hbase_acl_configured(self): + Logger.info("Setting HBase ACL Configured to True") + File(self.__params.indexing_hbase_acl_configured_flag_file, + content="", + owner=self.__params.metron_user, + mode=0755) + + def create_hbase_tables(self): + Logger.info("Creating HBase Tables") + if self.__params.security_enabled: + kinit(self.__params.kinit_path_local, + self.__params.hbase_keytab_path, + self.__params.hbase_principal_name, + execute_user=self.__params.hbase_user) + cmd = "echo \"create '{0}','{1}'\" | hbase shell -n" + add_update_cmd = cmd.format(self.__params.update_table, self.__params.update_cf) + Execute(add_update_cmd, + tries=3, + try_sleep=5, + logoutput=False, + path='/usr/sbin:/sbin:/usr/local/bin:/bin:/usr/bin', + user=self.__params.hbase_user + ) + + Logger.info("Done creating HBase Tables") + self.set_hbase_configured() + + def set_hbase_acls(self): + Logger.info("Setting HBase ACLs") + if self.__params.security_enabled: + kinit(self.__params.kinit_path_local, + self.__params.hbase_keytab_path, + self.__params.hbase_principal_name, + execute_user=self.__params.hbase_user) + cmd = "echo \"grant '{0}', 'RW', '{1}'\" | hbase shell -n" + add_update_acl_cmd = cmd.format(self.__params.metron_user, self.__params.update_table) + Execute(add_update_acl_cmd, + tries=3, + try_sleep=5, + logoutput=False, + path='/usr/sbin:/sbin:/usr/local/bin:/bin:/usr/bin', + user=self.__params.hbase_user + ) + + Logger.info("Done setting HBase ACLs") + self.set_hbase_acl_configured() + def set_acl_configured(self): File(self.__params.indexing_acl_configured_flag_file, content="", diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/indexing_master.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/indexing_master.py index 7e111cf319..71dcc74366 100755 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/indexing_master.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/indexing_master.py @@ -65,6 +65,11 @@ def configure(self, env, upgrade_type=None, config_dir=None): commands.init_kafka_acls() commands.set_acl_configured() + if not commands.is_hbase_configured(): + commands.create_hbase_tables() + if params.security_enabled and not commands.is_hbase_acl_configured(): + commands.set_hbase_acls() + Logger.info("Calling security setup") storm_security_setup(params) diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/params_linux.py index 3f84ef5f2d..b6206885c2 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 @@ -167,6 +167,9 @@ enrichment_hbase_provider_impl = 'org.apache.metron.hbase.HTableProvider' enrichment_table = status_params.enrichment_table enrichment_cf = status_params.enrichment_cf +update_table = status_params.update_table +update_cf = status_params.update_cf + threatintel_table = status_params.threatintel_table threatintel_cf = status_params.threatintel_cf diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py index 072a371b7d..eb349e2606 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py @@ -49,6 +49,8 @@ enrichment_cf = 't' threatintel_table = 'threatintel' threatintel_cf = 't' +update_table = 'metron_update' +update_cf = 't' # Indexing metron_indexing_topology = 'indexing' diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/global.json.j2 b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/global.json.j2 index 61e1416776..67226ff3a4 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/global.json.j2 +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/templates/global.json.j2 @@ -2,5 +2,7 @@ "es.clustername": "{{ es_cluster_name }}", "es.ip": "{{ es_url }}", "es.date.format": "{{es_date_format}}", - "parser.error.topic": "{{parser_error_topic}}" + "parser.error.topic": "{{parser_error_topic}}", + "update.hbase.table": "{{update_table}}", + "update.hbase.cf": "{{update_cf}}" } \ No newline at end of file diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java index 1e118c1ca2..5ff8314732 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/MetronRestConstants.java @@ -62,6 +62,4 @@ public class MetronRestConstants { public static final String SEARCH_MAX_RESULTS = "search.max.results"; public static final String INDEX_DAO_IMPL = "index.dao.impl"; public static final String INDEX_HBASE_TABLE_PROVIDER_IMPL = "index.hbase.provider"; - public static final String INDEX_HBASE_TABLE = "index.hbase.table"; - public static final String INDEX_HBASE_CF = "index.hbase.cf"; } diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java index 3734f70299..63851168a3 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.java @@ -54,14 +54,10 @@ public IndexConfig(Environment environment) { public IndexDao indexDao() { try { String hbaseProviderImpl = environment.getProperty(MetronRestConstants.INDEX_HBASE_TABLE_PROVIDER_IMPL, String.class, null); - String hbaseTable = environment.getProperty(MetronRestConstants.INDEX_HBASE_TABLE, String.class, null); - String hbaseCf = environment.getProperty(MetronRestConstants.INDEX_HBASE_CF, String.class, null); String indexDaoImpl = environment.getProperty(MetronRestConstants.INDEX_DAO_IMPL, String.class, null); int searchMaxResults = environment.getProperty(MetronRestConstants.SEARCH_MAX_RESULTS, Integer.class, -1); AccessConfig config = new AccessConfig(); config.setMaxSearchResults(searchMaxResults); - config.setTable(hbaseTable); - config.setColumnFamily(hbaseCf); config.setGlobalConfigSupplier(() -> { try { return globalConfigService.get(); diff --git a/metron-interface/metron-rest/src/main/resources/application-test.yml b/metron-interface/metron-rest/src/main/resources/application-test.yml index 59b127f322..f1a52b2d88 100644 --- a/metron-interface/metron-rest/src/main/resources/application-test.yml +++ b/metron-interface/metron-rest/src/main/resources/application-test.yml @@ -50,5 +50,3 @@ index: impl: org.apache.metron.indexing.dao.InMemoryDao,org.apache.metron.indexing.dao.HBaseDao hbase: provider: org.apache.metron.hbase.mock.MockProvider - table: updates - cf: t \ No newline at end of file diff --git a/metron-interface/metron-rest/src/main/resources/application.yml b/metron-interface/metron-rest/src/main/resources/application.yml index 473d29d195..a2b1dcb4c2 100644 --- a/metron-interface/metron-rest/src/main/resources/application.yml +++ b/metron-interface/metron-rest/src/main/resources/application.yml @@ -47,4 +47,4 @@ search: index: dao: - impl: org.apache.metron.elasticsearch.dao.ElasticsearchDao \ No newline at end of file + impl: org.apache.metron.elasticsearch.dao.ElasticsearchDao,org.apache.metron.indexing.dao.HBaseDao \ No newline at end of file diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java index 8d0fe42979..9c75f2f5ec 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/config/TestConfig.java @@ -21,10 +21,14 @@ import kafka.utils.ZKStringSerializer$; import kafka.utils.ZkUtils; import org.I0Itec.zkclient.ZkClient; +import org.apache.commons.io.IOUtils; +import org.apache.curator.CuratorZookeeperClient; import org.apache.curator.RetryPolicy; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.metron.TestConstants; +import org.apache.metron.common.configuration.ConfigurationsUtils; import org.apache.metron.integration.ComponentRunner; import org.apache.metron.integration.UnableToStartException; import org.apache.metron.integration.components.KafkaComponent; @@ -32,6 +36,7 @@ import org.apache.metron.rest.mock.MockStormCLIClientWrapper; import org.apache.metron.rest.mock.MockStormRestTemplate; import org.apache.metron.rest.service.impl.StormCLIWrapper; +import org.apache.zookeeper.KeeperException; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; import org.springframework.context.annotation.Profile; @@ -39,10 +44,16 @@ import org.springframework.kafka.core.DefaultKafkaConsumerFactory; import org.springframework.web.client.RestTemplate; +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; +import static org.apache.metron.common.configuration.ConfigurationsUtils.getClient; import static org.apache.metron.rest.MetronRestConstants.TEST_PROFILE; @Configuration @@ -65,7 +76,6 @@ public KafkaComponent kafkaWithZKComponent(Properties zkProperties) { return new KafkaComponent().withTopologyProperties(zkProperties); } - @Bean(destroyMethod = "stop") public ComponentRunner componentRunner(ZKServerComponent zkServerComponent, KafkaComponent kafkaWithZKComponent) { ComponentRunner runner = new ComponentRunner.Builder() @@ -74,6 +84,13 @@ public ComponentRunner componentRunner(ZKServerComponent zkServerComponent, Kafk .build(); try { runner.start(); + File globalConfigFile = new File("src/test/resources/zookeeper/global.json"); + try(BufferedReader r = new BufferedReader(new FileReader(globalConfigFile))){ + String globalConfig = IOUtils.toString(r); + ConfigurationsUtils.writeGlobalConfigToZookeeper(globalConfig.getBytes(), zkServerComponent.getConnectionString()); + } catch (Exception e) { + throw new IllegalStateException("Unable to upload global config", e); + } } catch (UnableToStartException e) { e.printStackTrace(); } diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java index ff101f5bc2..8a66737556 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java @@ -17,9 +17,12 @@ */ package org.apache.metron.rest.controller; +import org.I0Itec.zkclient.ZkClient; import org.adrianwalker.multilinestring.Multiline; +import org.apache.curator.framework.CuratorFramework; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Result; +import org.apache.metron.common.configuration.ConfigurationsUtils; import org.apache.metron.hbase.mock.MockHTable; import org.apache.metron.hbase.mock.MockProvider; import org.apache.metron.indexing.dao.SearchIntegrationTest; @@ -58,17 +61,29 @@ public class UpdateControllerIntegrationTest extends DaoControllerTest { @Autowired private UpdateService searchService; + @Autowired + public CuratorFramework client; @Autowired private WebApplicationContext wac; private MockMvc mockMvc; + private String globalConfigUrl = "/api/v1/global/config"; private String updateUrl = "/api/v1/update"; private String searchUrl = "/api/v1/search"; private String user = "user"; private String password = "password"; + /** + { + "update.hbase.table" : "updates", + "update.hbase.cf" : "t" + } + */ + @Multiline + public static String globalJson; + /** { "uuid" : "bro_index_2017.01.01.01:1", diff --git a/metron-interface/metron-rest/src/test/resources/zookeeper/global.json b/metron-interface/metron-rest/src/test/resources/zookeeper/global.json new file mode 100644 index 0000000000..396896ff0e --- /dev/null +++ b/metron-interface/metron-rest/src/test/resources/zookeeper/global.json @@ -0,0 +1,4 @@ +{ + "update.hbase.table" : "updates", + "update.hbase.cf" : "t" +} \ No newline at end of file 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 5914c87119..0fef4fae2a 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 @@ -75,14 +75,14 @@ public static void setup() throws Exception { hbaseDao = new HBaseDao(); AccessConfig accessConfig = new AccessConfig(); - accessConfig.setColumnFamily(CF); - accessConfig.setTable(TABLE_NAME); accessConfig.setTableProvider(tableProvider); Map globalConfig = new HashMap() {{ put("es.clustername", "metron"); put("es.port", "9300"); put("es.ip", "localhost"); put("es.date.format", dateFormat); + put(HBaseDao.HBASE_TABLE, TABLE_NAME); + put(HBaseDao.HBASE_CF, CF); }}; accessConfig.setGlobalConfigSupplier(() -> globalConfig); diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/components/ConfigUploadComponent.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/components/ConfigUploadComponent.java index 5901d9f53d..e4625ac7aa 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/components/ConfigUploadComponent.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/components/ConfigUploadComponent.java @@ -18,10 +18,12 @@ package org.apache.metron.enrichment.integration.components; import org.apache.curator.framework.CuratorFramework; +import org.apache.metron.common.configuration.ConfigurationsUtils; import org.apache.metron.common.configuration.SensorParserConfig; import org.apache.metron.integration.InMemoryComponent; import org.apache.metron.integration.UnableToStartException; import org.apache.metron.integration.components.ZKServerComponent; +import org.apache.zookeeper.KeeperException; import java.util.HashMap; import java.util.Map; @@ -34,6 +36,7 @@ public class ConfigUploadComponent implements InMemoryComponent { + private String connectionString; private Properties topologyProperties; private String globalConfigPath; private String parserConfigsPath; @@ -43,6 +46,12 @@ public class ConfigUploadComponent implements InMemoryComponent { private Optional> postStartCallback = Optional.empty(); private Optional globalConfig = Optional.empty(); private Map parserSensorConfigs = new HashMap<>(); + + public ConfigUploadComponent withConnectionString(String connectionString) { + this.connectionString = connectionString; + return this; + } + public ConfigUploadComponent withTopologyProperties(Properties topologyProperties) { this.topologyProperties = topologyProperties; return this; @@ -129,7 +138,7 @@ public void start() throws UnableToStartException { public void update() throws UnableToStartException { try { - final String zookeeperUrl = topologyProperties.getProperty(ZKServerComponent.ZOOKEEPER_PROPERTY); + final String zookeeperUrl = connectionString == null?topologyProperties.getProperty(ZKServerComponent.ZOOKEEPER_PROPERTY):connectionString; if(globalConfigPath != null || parserConfigsPath != null @@ -157,6 +166,7 @@ public void update() throws UnableToStartException { } + public SensorParserConfig getSensorParserConfig(String sensorType) { SensorParserConfig sensorParserConfig = new SensorParserConfig(); CuratorFramework client = getClient(topologyProperties.getProperty(ZKServerComponent.ZOOKEEPER_PROPERTY)); diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java index 7dc03b8a91..972ef00e0f 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java @@ -29,8 +29,6 @@ public class AccessConfig { private Supplier> globalConfigSupplier; private Map optionalSettings = new HashMap<>(); private TableProvider tableProvider = null; - private String columnFamily; - private String table; public Supplier> getGlobalConfigSupplier() { return globalConfigSupplier; @@ -64,19 +62,4 @@ public void setTableProvider(TableProvider tableProvider) { this.tableProvider = tableProvider; } - public String getColumnFamily() { - return columnFamily; - } - - public void setColumnFamily(String columnFamily) { - this.columnFamily = columnFamily; - } - - public String getTable() { - return table; - } - - public void setTable(String table) { - this.table = table; - } } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java index 1fb50e231f..c903d409f9 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java @@ -37,6 +37,8 @@ import java.util.Optional; public class HBaseDao implements IndexDao { + public static String HBASE_TABLE = "update.hbase.table"; + public static String HBASE_CF = "update.hbase.cf"; private HTableInterface tableInterface; private byte[] cf; private AccessConfig config; @@ -53,9 +55,18 @@ public synchronized SearchResponse search(SearchRequest searchRequest) throws In public synchronized void init(AccessConfig config) { if(this.tableInterface == null) { this.config = config; + Map globalConfig = config.getGlobalConfigSupplier().get(); + if(globalConfig == null) { + throw new IllegalStateException("Cannot find the global config."); + } + String table = (String)globalConfig.get(HBASE_TABLE); + String cf = (String) config.getGlobalConfigSupplier().get().get(HBASE_CF); + if(table == null || cf == null) { + throw new IllegalStateException("You must configure " + HBASE_TABLE + " and " + HBASE_CF + " in the global config."); + } try { - tableInterface = config.getTableProvider().getTable(HBaseConfiguration.create(), config.getTable()); - cf = config.getColumnFamily().getBytes(); + tableInterface = config.getTableProvider().getTable(HBaseConfiguration.create(), table); + this.cf = cf.getBytes(); } catch (IOException e) { throw new IllegalStateException("Unable to initialize HBaseDao: " + e.getMessage(), e); } 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 8d3005fbe8..9292f729fa 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 @@ -23,5 +23,7 @@ "profiler.client.salt.divisor": "1000", "hbase.provider.impl": "org.apache.metron.hbase.HTableProvider", - "geo.hdfs.file": "src/test/resources/GeoLite/GeoIP2-City-Test.mmdb.gz" + "geo.hdfs.file": "src/test/resources/GeoLite/GeoIP2-City-Test.mmdb.gz", + "update.hbase.table" : "updates", + "update.hbase.cf" : "t" } \ No newline at end of file From a5431626c595395b5bac52273e721eb066fafc4e Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 26 Jul 2017 18:18:25 +0100 Subject: [PATCH 16/38] Don't need the global config in the test anymore. --- .../controller/UpdateControllerIntegrationTest.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java index 8a66737556..aa0adb724d 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java @@ -69,21 +69,11 @@ public class UpdateControllerIntegrationTest extends DaoControllerTest { private MockMvc mockMvc; - private String globalConfigUrl = "/api/v1/global/config"; private String updateUrl = "/api/v1/update"; private String searchUrl = "/api/v1/search"; private String user = "user"; private String password = "password"; - /** - { - "update.hbase.table" : "updates", - "update.hbase.cf" : "t" - } - */ - @Multiline - public static String globalJson; - /** { "uuid" : "bro_index_2017.01.01.01:1", From a48443a9276826d81bd5a11ec2f1e49f8c6238d2 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 27 Jul 2017 10:38:49 +0100 Subject: [PATCH 17/38] Forgot to add the update config to the theme. --- .../METRON/CURRENT/themes/metron_theme.json | 27 +++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) 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 e7ac630987..0e5457d730 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 @@ -153,6 +153,25 @@ } ] }, + { + "name": "section-indexing-update", + "row-index": "0", + "column-index": "0", + "row-span": "1", + "column-span": "1", + "section-columns": "1", + "section-rows": "1", + "subsections": [ + { + "name": "subsection-indexing-update", + "display-name": "Index Updates", + "row-index": "0", + "column-index": "0", + "row-span": "1", + "column-span": "1" + } + ] + }, { "name": "section-indexing-storm", "row-index": "1", @@ -378,8 +397,12 @@ "subsection-name": "subsection-indexing-kafka" }, { - "config": "metron-indexing-env/indexing_error_topic", - "subsection-name": "subsection-indexing-kafka" + "config": "metron-indexing-env/update_table", + "subsection-name": "subsection-indexing-update" + }, + { + "config": "metron-indexing-env/update_cf", + "subsection-name": "subsection-indexing-update" }, { "config": "metron-indexing-env/indexing_writer_class_name", From d01a9ac74e515e08088c355a6dc8d22eda9067b7 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 27 Jul 2017 15:16:12 +0100 Subject: [PATCH 18/38] typo --- .../METRON/CURRENT/package/scripts/params/params_linux.py | 2 ++ .../METRON/CURRENT/package/scripts/params/status_params.py | 2 ++ 2 files changed, 4 insertions(+) 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 b6206885c2..e8f1a663f3 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 @@ -71,6 +71,8 @@ enrichment_geo_configured_flag_file = status_params.enrichment_geo_configured_flag_file indexing_configured_flag_file = status_params.indexing_configured_flag_file indexing_acl_configured_flag_file = status_params.indexing_acl_configured_flag_file +indexing_hbase_configured_flag_file = status_params.indexing_hbase_configured_flag_file +indexing_hbase_acl_configured_flag_file = status_params.indexing_hbase_acl_configured_flag_file indexing_hdfs_perm_configured_flag_file = status_params.indexing_hdfs_perm_configured_flag_file global_json_template = config['configurations']['metron-env']['global-json'] global_properties_template = config['configurations']['metron-env']['elasticsearch-properties'] diff --git a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py index eb349e2606..d156db8403 100644 --- a/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py +++ b/metron-deployment/packaging/ambari/metron-mpack/src/main/resources/common-services/METRON/CURRENT/package/scripts/params/status_params.py @@ -58,6 +58,8 @@ indexing_configured_flag_file = metron_zookeeper_config_path + '/../metron_indexing_configured' indexing_acl_configured_flag_file = metron_zookeeper_config_path + '/../metron_indexing_acl_configured' indexing_hdfs_perm_configured_flag_file = metron_zookeeper_config_path + '/../metron_indexing_hdfs_perm_configured' +indexing_hbase_configured_flag_file = metron_zookeeper_config_path + '/../metron_indexing_hbase_configured' +indexing_hbase_acl_configured_flag_file = metron_zookeeper_config_path + '/../metron_indexing_hbase_acl_configured' # Storm storm_rest_addr = config['configurations']['metron-env']['storm_rest_addr'] From 748ea2bca5ed957899708e0b16f572bb243456e9 Mon Sep 17 00:00:00 2001 From: cstella Date: Fri, 28 Jul 2017 21:32:51 +0100 Subject: [PATCH 19/38] Fixing shell script --- metron-interface/metron-rest/src/main/scripts/metron-rest | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/metron-interface/metron-rest/src/main/scripts/metron-rest b/metron-interface/metron-rest/src/main/scripts/metron-rest index 9c5b1d4d03..d1203557e2 100644 --- a/metron-interface/metron-rest/src/main/scripts/metron-rest +++ b/metron-interface/metron-rest/src/main/scripts/metron-rest @@ -41,10 +41,11 @@ if [ -f "$METRON_SYSCONFIG" ]; then . "$METRON_SYSCONFIG" fi +HBASE_HOME=${HBASE_HOME:-/usr/hdp/current/hbase-client} PIDFILE="$METRON_PID_DIR/$NAME.pid" LOCKFILE=/var/lock/subsys/$NAME -METRON_REST_CLASSPATH="$METRON_HOME/lib/metron-rest-$METRON_VERSION.jar" +METRON_REST_CLASSPATH="${HBASE_HOME}/conf:$METRON_HOME/lib/metron-rest-$METRON_VERSION.jar" # the vagrant Spring profile provides configuration values, otherwise configuration is provided by rest_application.yml if [[ !($METRON_SPRING_PROFILES_ACTIVE == *"vagrant"*) ]]; then From 38437b32db2a5c3262bc88a9b9cff4355e8df8cd Mon Sep 17 00:00:00 2001 From: cstella Date: Tue, 1 Aug 2017 15:38:33 +0100 Subject: [PATCH 20/38] Adding metron-hbase-client project so the REST api's spring dependencies and hbase-client can coexist. --- metron-interface/metron-rest/pom.xml | 9 +++ metron-platform/metron-hbase-client/pom.xml | 90 +++++++++++++++++++++ metron-platform/pom.xml | 1 + 3 files changed, 100 insertions(+) create mode 100644 metron-platform/metron-hbase-client/pom.xml diff --git a/metron-interface/metron-rest/pom.xml b/metron-interface/metron-rest/pom.xml index 970ee8611e..44e24438dd 100644 --- a/metron-interface/metron-rest/pom.xml +++ b/metron-interface/metron-rest/pom.xml @@ -145,6 +145,11 @@ + + org.apache.metron + metron-hbase-client + ${project.parent.version} + org.apache.metron metron-hbase @@ -157,6 +162,10 @@ metron-parsers ${project.parent.version} + + org.apache.hbase + hbase-client + com.fasterxml.jackson.core jackson-databind diff --git a/metron-platform/metron-hbase-client/pom.xml b/metron-platform/metron-hbase-client/pom.xml new file mode 100644 index 0000000000..e12043fbd5 --- /dev/null +++ b/metron-platform/metron-hbase-client/pom.xml @@ -0,0 +1,90 @@ + + + + + 4.0.0 + + org.apache.metron + metron-platform + 0.4.1 + + metron-hbase-client + metron-hbase-client + https://metron.apache.org/ + + UTF-8 + UTF-8 + 1.7.7 + 0.1.2 + ${global_hbase_guava_version} + + + + org.apache.hbase + hbase-shaded-client + 1.1.2 + + + + + + org.apache.maven.plugins + maven-jar-plugin + ${global_jar_version} + + + + test-jar + + + + + + org.apache.maven.plugins + maven-shade-plugin + ${global_shade_version} + + true + + + *slf4j* + + + + + + package + + shade + + + + + org.apache.commons.logging + org.apache.hadoop.hbase.shaded.org.apache.commons.logging + + + org.apache.log4j + org.apache.hadoop.hbase.shaded.org.apache.log4j + + + + + + + + + + diff --git a/metron-platform/pom.xml b/metron-platform/pom.xml index 7c616206b8..93ced81ca9 100644 --- a/metron-platform/pom.xml +++ b/metron-platform/pom.xml @@ -41,6 +41,7 @@ + metron-hbase-client metron-common metron-enrichment metron-solr From e3b37df23fe68dca536070c1134994fb585fb9c7 Mon Sep 17 00:00:00 2001 From: cstella Date: Tue, 1 Aug 2017 15:42:35 +0100 Subject: [PATCH 21/38] Cleared up some documentation. --- .../org/apache/metron/rest/controller/UpdateController.java | 2 +- metron-platform/metron-hbase-client/pom.xml | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/UpdateController.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/UpdateController.java index 0b6e1c5c22..56b0b7b241 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/UpdateController.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/UpdateController.java @@ -56,7 +56,7 @@ ResponseEntity patch( return new ResponseEntity<>(HttpStatus.OK); } - @ApiOperation(value = "Update a document with a patch") + @ApiOperation(value = "Replace a document with a full replacement") @ApiResponse(message = "Nothing", code = 200) @RequestMapping(value = "/replace", method = RequestMethod.POST) ResponseEntity replace( diff --git a/metron-platform/metron-hbase-client/pom.xml b/metron-platform/metron-hbase-client/pom.xml index e12043fbd5..6076db60b9 100644 --- a/metron-platform/metron-hbase-client/pom.xml +++ b/metron-platform/metron-hbase-client/pom.xml @@ -34,6 +34,7 @@ org.apache.hbase hbase-shaded-client + 1.1.2 From 9c0dfd8487fa73c4e19798c60c994583c743f940 Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 2 Aug 2017 10:05:40 +0100 Subject: [PATCH 22/38] Updating api changes coming in from master --- .../apache/metron/indexing/dao/HBaseDao.java | 12 ++++++++++ .../apache/metron/indexing/dao/IndexDao.java | 2 -- .../metron/indexing/dao/MultiIndexDao.java | 24 ++++++++++++++++++- .../metron/indexing/dao/InMemoryDao.java | 7 +++--- 4 files changed, 39 insertions(+), 6 deletions(-) diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java index c903d409f9..028491d609 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java @@ -26,12 +26,14 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.metron.common.configuration.writer.WriterConfiguration; import org.apache.metron.common.utils.JSONUtils; +import org.apache.metron.indexing.dao.search.FieldType; import org.apache.metron.indexing.dao.search.InvalidSearchException; import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; import org.apache.metron.indexing.dao.update.Document; import java.io.IOException; +import java.util.List; import java.util.Map; import java.util.NavigableMap; import java.util.Optional; @@ -109,4 +111,14 @@ public synchronized void update(Document update, Optional index) throws put.addColumn(cf, columnQualifier, doc); getTableInterface().put(put); } + + @Override + public Map> getColumnMetadata(List indices) throws IOException { + return null; + } + + @Override + public Map getCommonColumnMetadata(List indices) throws IOException { + return null; + } } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java index 3c0a420a3f..812ec0b875 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java @@ -31,12 +31,10 @@ import org.apache.metron.indexing.dao.update.OriginalNotFoundException; import java.io.IOException; -======= import org.apache.metron.indexing.dao.search.FieldType; import java.io.IOException; import java.util.List; ->>>>>>> master import java.util.Map; import java.util.Optional; diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java index 8b5ab0fe8a..1559ece721 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java @@ -21,7 +21,7 @@ import com.google.common.base.Joiner; import com.google.common.collect.Iterables; import org.apache.commons.lang3.exception.ExceptionUtils; -import org.apache.metron.common.configuration.writer.WriterConfiguration; +import org.apache.metron.indexing.dao.search.FieldType; import org.apache.metron.indexing.dao.search.InvalidSearchException; import org.apache.metron.indexing.dao.search.SearchRequest; import org.apache.metron.indexing.dao.search.SearchResponse; @@ -67,6 +67,28 @@ public void update(final Document update, Optional index) throws IOExcep } } + @Override + public Map> getColumnMetadata(List in) throws IOException { + for(IndexDao dao : indices) { + Map> r = dao.getColumnMetadata(in); + if(r != null) { + return r; + } + } + return null; + } + + @Override + public Map getCommonColumnMetadata(List in) throws IOException { + for(IndexDao dao : indices) { + Map r = dao.getCommonColumnMetadata(in); + if(r != null) { + return r; + } + } + return null; + } + private static class DocumentContainer { private Optional d = Optional.empty(); private Optional t = Optional.empty(); diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java index b71ff5b14e..b545018cb8 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java @@ -152,18 +152,19 @@ public Document getLatest(String uuid, String sensorType) throws IOException { @Override public void update(Document update, Optional index) throws IOException { - for(Map.Entry> kv: BACKING_STORE.entrySet()) { + for (Map.Entry> kv : BACKING_STORE.entrySet()) { if (kv.getKey().startsWith(update.getSensorType())) { - for(Iterator it = kv.getValue().iterator();it.hasNext();) { + for (Iterator it = kv.getValue().iterator(); it.hasNext(); ) { String doc = it.next(); Map docParsed = parse(doc); - if(docParsed.getOrDefault(Constants.GUID, "").equals(update.getUuid())) { + if (docParsed.getOrDefault(Constants.GUID, "").equals(update.getUuid())) { it.remove(); } } kv.getValue().add(JSONUtils.INSTANCE.toJSON(update.getDocument(), true)); } } + } public Map> getColumnMetadata(List indices) throws IOException { Map> columnMetadata = new HashMap<>(); From 042c3f7eb39b8ecda5858a54786978643f8ea67c Mon Sep 17 00:00:00 2001 From: cstella Date: Wed, 2 Aug 2017 10:28:05 +0100 Subject: [PATCH 23/38] some test refactoring. --- .../SearchControllerIntegrationTest.java | 20 ------------------- 1 file changed, 20 deletions(-) diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java index a08e78e5e7..848e211836 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java @@ -21,7 +21,6 @@ import org.apache.metron.hbase.mock.MockProvider; import org.apache.metron.indexing.dao.InMemoryDao; import org.apache.metron.indexing.dao.SearchIntegrationTest; -import org.apache.metron.indexing.dao.IndexingDaoIntegrationTest; import org.apache.metron.indexing.dao.search.FieldType; import org.apache.metron.rest.service.SearchService; import org.json.simple.JSONArray; @@ -246,25 +245,6 @@ public void test() throws Exception { - private void loadTestData() throws ParseException { - Map> backingStore = new HashMap<>(); - for(Map.Entry indices : - ImmutableMap.of( - "bro_index_2017.01.01.01", IndexingDaoIntegrationTest.broData, - "snort_index_2017.01.01.01", IndexingDaoIntegrationTest.snortData - ).entrySet() - ) - { - List results = new ArrayList<>(); - backingStore.put(indices.getKey(), results); - JSONArray broArray = (JSONArray) new JSONParser().parse(indices.getValue()); - for(Object o: broArray) { - JSONObject jsonObject = (JSONObject) o; - results.add(jsonObject.toJSONString()); - } - } - InMemoryDao.load(backingStore); - } private void loadColumnTypes() throws ParseException { Map> columnTypes = new HashMap<>(); From d9c57fe60c21a13c0c67539fe581ee5a84fcf329 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 3 Aug 2017 09:57:15 +0100 Subject: [PATCH 24/38] Updating README to document REST calls and change uuids to guids. --- metron-interface/metron-rest/README.md | 70 +++++++++++++++++++ .../elasticsearch/dao/ElasticsearchDao.java | 19 ++--- .../apache/metron/indexing/dao/HBaseDao.java | 8 +-- .../apache/metron/indexing/dao/IndexDao.java | 8 +-- .../metron/indexing/dao/MultiIndexDao.java | 4 +- .../indexing/dao/search/GetRequest.java | 10 +-- .../metron/indexing/dao/update/Document.java | 22 +++--- .../indexing/dao/update/PatchRequest.java | 10 +-- .../indexing/dao/update/ReplaceRequest.java | 10 +-- 9 files changed, 117 insertions(+), 44 deletions(-) diff --git a/metron-interface/metron-rest/README.md b/metron-interface/metron-rest/README.md index 46a3fc0d5b..b76712bca5 100644 --- a/metron-interface/metron-rest/README.md +++ b/metron-interface/metron-rest/README.md @@ -200,6 +200,7 @@ Request and Response objects are JSON formatted. The JSON schemas are available | [ `DELETE /api/v1/kafka/topic/{name}`](#delete-apiv1kafkatopicname)| | [ `GET /api/v1/kafka/topic/{name}/sample`](#get-apiv1kafkatopicnamesample)| | [ `GET /api/v1/search/search`](#get-apiv1searchsearch)| +| [ `GET /api/v1/search/findOne`](#get-apiv1searchfindone)| | [ `GET /api/v1/search/search`](#get-apiv1searchcolumnmetadata)| | [ `GET /api/v1/search/search`](#get-apiv1searchcolumnmetadatacommon)| | [ `GET /api/v1/sensor/enrichment/config`](#get-apiv1sensorenrichmentconfig)| @@ -242,6 +243,8 @@ Request and Response objects are JSON formatted. The JSON schemas are available | [ `GET /api/v1/storm/parser/stop/{name}`](#get-apiv1stormparserstopname)| | [ `GET /api/v1/storm/{name}`](#get-apiv1stormname)| | [ `GET /api/v1/storm/supervisors`](#get-apiv1stormsupervisors)| +| [ `PATCH /api/v1/update/patch`](#patch-apiv1updatepatch)| +| [ `PUT /api/v1/update/replace`](#patch-apiv1updatereplace)| | [ `GET /api/v1/user`](#get-apiv1user)| ### `GET /api/v1/global/config` @@ -350,6 +353,23 @@ Request and Response objects are JSON formatted. The JSON schemas are available * 200 - Returns sample message * 404 - Either Kafka topic is missing or contains no messages +### `GET /api/v1/search/findOne` + * Description: Returns latest document for a guid and sensor + * Input: + * getRequest - Get request + * guid - message UUID + * sensorType - Sensor Type + * Example: Return `bro` document with UUID of `000-000-0000` +``` +{ + "guid" : "000-000-0000", + "sensorType" : "bro" +} +``` + * Returns: + * 200 - Document representing the output + * 404 - Document with UUID and sensor type not found + ### `GET /api/v1/search/search` * Description: Searches the indexing store * Input: @@ -624,6 +644,56 @@ Request and Response objects are JSON formatted. The JSON schemas are available * Returns: * 200 - Returns a list of the status of all Storm Supervisors +### `PATCH /api/v1/update/patch` + * Description: Update a document with a patch + * Input: + * request - Patch Request + * guid - The Patch UUID + * sensorType - The sensor type + * patch - An array of [RFC 6902](https://tools.ietf.org/html/rfc6902) patches. + * Example adding a field called `project` with value `metron` to the `bro` message with UUID of `000-000-0000` : + ``` + { + "guid" : "000-000-0000", + "sensorType" : "bro", + "patch" : [ + { + "op": "add" + , "path": "/project" + , "value": "metron" + } + ] + } + ``` + * Returns: + * 200 - nothing + * 404 - document not found + +### `PUT /api/v1/update/replace` + * Description: Replace a document + * Input: + * request - Replacement request + * guid - The Patch UUID + * sensorType - The sensor type + * replacement - A Map representing the replaced document + * Example replacing a `bro` message with guid of `000-000-0000` +``` + { + "guid" : "000-000-0000", + "sensorType" : "bro", + "replacement" : { + "source:type": "bro", + "guid" : "bro_index_2017.01.01.01:1", + "ip_src_addr":"192.168.1.2", + "ip_src_port": 8009, + "timestamp":200, + "rejected":false + } + } +``` + * Returns: + * 200 - Current user + ### `GET /api/v1/user` * Description: Retrieves the current user * Returns: 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 5d14b3dd7c..2af7920cad 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 @@ -143,16 +143,16 @@ public synchronized void init(AccessConfig config) { } @Override - public Document getLatest(final String uuid, final String sensorType) throws IOException { + public Document getLatest(final String guid, final String sensorType) throws IOException { Optional ret = searchByUuid( - uuid + guid , sensorType , hit -> { Long ts = 0L; String doc = hit.getSourceAsString(); String sourceType = Iterables.getFirst(Splitter.on("_doc").split(hit.getType()), null); try { - return Optional.of(new Document(doc, uuid, sourceType, ts)); + return Optional.of(new Document(doc, guid, sourceType, ts)); } catch (IOException e) { throw new IllegalStateException("Unable to retrieve latest: " + e.getMessage(), e); } @@ -161,8 +161,14 @@ public Document getLatest(final String uuid, final String sensorType) throws IOE return ret.orElse(null); } - Optional searchByUuid(String uuid, String sensorType, Function> callback) throws IOException{ - QueryBuilder query = QueryBuilders.matchQuery(Constants.GUID, uuid); + /** + * Return the search hit based on the UUID and sensor type. + * A callback can be specified to transform the hit into a type T. + * If more than one hit happens, the first one will be returned. + * @throws IOException + */ + Optional searchByUuid(String guid, String sensorType, Function> callback) throws IOException{ + QueryBuilder query = QueryBuilders.matchQuery(Constants.GUID, guid); SearchRequestBuilder request = client.prepareSearch() .setTypes(sensorType + "_doc") .setQuery(query) @@ -171,9 +177,6 @@ Optional searchByUuid(String uuid, String sensorType, Function columns = result.getFamilyMap( cf); @@ -95,7 +95,7 @@ public synchronized Document getLatest(String uuid, String sensorType) throws IO Long ts = Bytes.toLong(entry.getKey()); if(entry.getValue()!= null) { String json = new String(entry.getValue()); - return new Document(json, uuid, sensorType, ts); + return new Document(json, guid, sensorType, ts); } else { return null; @@ -104,7 +104,7 @@ public synchronized Document getLatest(String uuid, String sensorType) throws IO @Override public synchronized void update(Document update, Optional index) throws IOException { - Put put = new Put(update.getUuid().getBytes()); + Put put = new Put(update.getGuid().getBytes()); long ts = update.getTimestamp() == null?System.currentTimeMillis():update.getTimestamp(); byte[] columnQualifier = Bytes.toBytes(ts); byte[] doc = JSONUtils.INSTANCE.toJSON(update.getDocument()); diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java index 812ec0b875..d46e36a0c0 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java @@ -45,7 +45,7 @@ public interface IndexDao { Document getLatest(String uuid, String sensorType) throws IOException; default Optional> getLatestResult(GetRequest request) throws IOException { - Document ret = getLatest(request.getUuid(), request.getSensorType()); + Document ret = getLatest(request.getGuid(), request.getSensorType()); if(ret == null) { return Optional.empty(); } @@ -62,7 +62,7 @@ default void patch( PatchRequest request ) throws OriginalNotFoundException, IOException { Map latest = request.getSource(); if(latest == null) { - Document latestDoc = getLatest(request.getUuid(), request.getSensorType()); + Document latestDoc = getLatest(request.getGuid(), request.getSensorType()); if(latestDoc.getDocument() != null) { latest = latestDoc.getDocument(); } @@ -75,7 +75,7 @@ default void patch( PatchRequest request Map updated = JSONUtils.INSTANCE.getMapper() .convertValue(patched, new TypeReference>() {}); Document d = new Document( updated - , request.getUuid() + , request.getGuid() , request.getSensorType() , timestamp.orElse(System.currentTimeMillis()) ); @@ -86,7 +86,7 @@ default void replace( ReplaceRequest request , Optional timestamp ) throws IOException { Document d = new Document(request.getReplacement() - , request.getUuid() + , request.getGuid() , request.getSensorType() , timestamp.orElse(System.currentTimeMillis()) ); diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java index 1559ece721..7481aa0d9e 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java @@ -127,12 +127,12 @@ public void init(AccessConfig config) { } @Override - public Document getLatest(final String uuid, String sensorType) throws IOException { + public Document getLatest(final String guid, String sensorType) throws IOException { Document ret = null; List output = indices.parallelStream().map(dao -> { try { - return new DocumentContainer(dao.getLatest(uuid, sensorType)); + return new DocumentContainer(dao.getLatest(guid, sensorType)); } catch (Throwable e) { return new DocumentContainer(e); } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java index cc847b6dc9..8a0aeaa62a 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java @@ -18,15 +18,15 @@ package org.apache.metron.indexing.dao.search; public class GetRequest { - String uuid; + String guid; String sensorType; - public String getUuid() { - return uuid; + public String getGuid() { + return guid; } - public void setUuid(String uuid) { - this.uuid = uuid; + public void setGuid(String guid) { + this.guid = guid; } public String getSensorType() { 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 a28dfd57d5..85c079f9cd 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 @@ -30,23 +30,23 @@ public class Document { Long timestamp; Map document; - String uuid; + String guid; String sensorType; - public Document(Map document, String uuid, String sensorType, Long timestamp) { + public Document(Map document, String guid, String sensorType, Long timestamp) { setDocument(document); - setUuid(uuid); + setGuid(guid); setTimestamp(timestamp); setSensorType(sensorType); } - public Document(String document, String uuid, String sensorType, Long timestamp) throws IOException { - this(convertDoc(document), uuid, sensorType, timestamp); + public Document(String document, String guid, String sensorType, Long timestamp) throws IOException { + this(convertDoc(document), guid, sensorType, timestamp); } - public Document(String document, String uuid, String sensorType) throws IOException { - this( document, uuid, sensorType, null); + public Document(String document, String guid, String sensorType) throws IOException { + this( document, guid, sensorType, null); } private static Map convertDoc(String document) throws IOException { @@ -78,11 +78,11 @@ public void setDocument(Map document) { this.document = document; } - public String getUuid() { - return uuid; + public String getGuid() { + return guid; } - public void setUuid(String uuid) { - this.uuid = uuid; + public void setGuid(String guid) { + this.guid = guid; } } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java index 2f8cd15e5c..12ecaf8352 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java @@ -24,7 +24,7 @@ public class PatchRequest { JsonNode patch; Map source; - String uuid; + String guid; String sensorType; String index; @@ -52,12 +52,12 @@ public void setSource(Map source) { this.source = source; } - public String getUuid() { - return uuid; + public String getGuid() { + return guid; } - public void setUuid(String uuid) { - this.uuid = uuid; + public void setGuid(String guid) { + this.guid = guid; } public String getSensorType() { diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java index d7ec921f3a..42d6cd6aa5 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java @@ -23,7 +23,7 @@ public class ReplaceRequest { Map replacement; - String uuid; + String guid; String sensorType; String index; @@ -51,11 +51,11 @@ public void setReplacement(Map replacement) { this.replacement = replacement; } - public String getUuid() { - return uuid; + public String getGuid() { + return guid; } - public void setUuid(String uuid) { - this.uuid = uuid; + public void setGuid(String guid) { + this.guid = guid; } } From c2dfb66fef4159b1271536c82a11f0ee61280119 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 3 Aug 2017 10:11:42 +0100 Subject: [PATCH 25/38] Updating documentation and better commenting. --- .../src/main/resources/application-test.yml | 2 ++ .../src/main/resources/application.yml | 1 + metron-platform/metron-indexing/README.md | 24 +++++++++++++++++++ .../metron/indexing/dao/InMemoryDao.java | 8 +++---- 4 files changed, 31 insertions(+), 4 deletions(-) diff --git a/metron-interface/metron-rest/src/main/resources/application-test.yml b/metron-interface/metron-rest/src/main/resources/application-test.yml index f1a52b2d88..18d3e54ffd 100644 --- a/metron-interface/metron-rest/src/main/resources/application-test.yml +++ b/metron-interface/metron-rest/src/main/resources/application-test.yml @@ -47,6 +47,8 @@ search: index: dao: + # By default, we use the InMemoryDao for our tests and HBaseDao for backing updates. impl: org.apache.metron.indexing.dao.InMemoryDao,org.apache.metron.indexing.dao.HBaseDao hbase: + # HBase is provided via a mock provider, so no actual HBase infrastructure is started. provider: org.apache.metron.hbase.mock.MockProvider diff --git a/metron-interface/metron-rest/src/main/resources/application.yml b/metron-interface/metron-rest/src/main/resources/application.yml index a2b1dcb4c2..4aff769c14 100644 --- a/metron-interface/metron-rest/src/main/resources/application.yml +++ b/metron-interface/metron-rest/src/main/resources/application.yml @@ -47,4 +47,5 @@ search: index: dao: + # By default, we use the ElasticsearchDao and HBaseDao for backing updates. impl: org.apache.metron.elasticsearch.dao.ElasticsearchDao,org.apache.metron.indexing.dao.HBaseDao \ No newline at end of file diff --git a/metron-platform/metron-indexing/README.md b/metron-platform/metron-indexing/README.md index 0cf3a66aae..2095d0fa83 100644 --- a/metron-platform/metron-indexing/README.md +++ b/metron-platform/metron-indexing/README.md @@ -111,6 +111,30 @@ Storm console. e.g.: * hdfs writer * disabled +# Updates to Indexed Data + +There are clear usecases where we would want to incorporate the capability to update indexed data. +Thus far, we have limited capabilities provided to support this use-case: +* Updates to the random access index (e.g. Elasticsearch and Solr) should be supported +* Updates to the cold storage index (e.g. HDFS) is not supported currently, however to support the batch +use-case updated documents will be provided in a NoSQL write-ahead log (e.g. a HBase table) and an Java API +will be provided to retrieve those updates scalably (i.e. a scan-free architecture). + +Put simply, the random access index will be always up-to-date, but the HDFS index will need to be +joined to the NoSQL write-ahead log to get current updates. + +## The `IndexDao` Abstraction + +The indices mentioned above as part of Update should be pluggable by the developer so that +new write-ahead logs or real-time indices can be supported by providing an implementation supporting +the data access patterns. + +To support a new index, one would need to implement the `org.apache.metron.indexing.dao.IndexDao` abstraction +and provide update and search capabilities. IndexDaos may be composed and updates will be performed +in parallel. This enables a flexible strategy for specifying your backing store for updates at runtime. +For instance, currently the REST API supports the update functionality and may be configured with a list of +IndexDao implementations to use to support the updates. + # Notes on Performance Tuning Default installed Metron is untuned for production deployment. By far diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java index b545018cb8..2d146e0a8e 100644 --- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java +++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryDao.java @@ -136,13 +136,13 @@ public void init(AccessConfig config) { } @Override - public Document getLatest(String uuid, String sensorType) throws IOException { + public Document getLatest(String guid, String sensorType) throws IOException { for(Map.Entry> kv: BACKING_STORE.entrySet()) { if(kv.getKey().startsWith(sensorType)) { for(String doc : kv.getValue()) { Map docParsed = parse(doc); - if(docParsed.getOrDefault(Constants.GUID, "").equals(uuid)) { - return new Document(doc, uuid, sensorType, 0L); + if(docParsed.getOrDefault(Constants.GUID, "").equals(guid)) { + return new Document(doc, guid, sensorType, 0L); } } } @@ -157,7 +157,7 @@ public void update(Document update, Optional index) throws IOException { for (Iterator it = kv.getValue().iterator(); it.hasNext(); ) { String doc = it.next(); Map docParsed = parse(doc); - if (docParsed.getOrDefault(Constants.GUID, "").equals(update.getUuid())) { + if (docParsed.getOrDefault(Constants.GUID, "").equals(update.getGuid())) { it.remove(); } } From aa1e8a0e38bfb24e9d9a0bed7ba1a3668d87560a Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 3 Aug 2017 10:15:16 +0100 Subject: [PATCH 26/38] Better documentation for the hbase dao. --- .../java/org/apache/metron/indexing/dao/HBaseDao.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java index 186bf77aa0..a1cf39884a 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/HBaseDao.java @@ -38,6 +38,17 @@ import java.util.NavigableMap; import java.util.Optional; +/** + * The HBaseDao is an index dao which only supports the following actions: + * * Update + * * Get document + * + * The mechanism here is that updates to documents will be added to a HBase Table as a write-ahead log. + * The Key for a row supporting a given document will be the GUID, which should be sufficiently distributed. + * Every new update will have a column added (column qualifier will be the timestamp of the update). + * Upon retrieval, the most recent column will be returned. + * + */ public class HBaseDao implements IndexDao { public static String HBASE_TABLE = "update.hbase.table"; public static String HBASE_CF = "update.hbase.cf"; From b4291432d5ef588028783e7dc7b2a598d2ce78b4 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 3 Aug 2017 10:26:25 +0100 Subject: [PATCH 27/38] Lingering s/uuid/guid/g changes. --- .../metron/elasticsearch/dao/ElasticsearchDao.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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 2af7920cad..fd9cb8ed2f 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 @@ -144,7 +144,7 @@ public synchronized void init(AccessConfig config) { @Override public Document getLatest(final String guid, final String sensorType) throws IOException { - Optional ret = searchByUuid( + Optional ret = searchByGuid( guid , sensorType , hit -> { @@ -167,7 +167,7 @@ public Document getLatest(final String guid, final String sensorType) throws IOE * If more than one hit happens, the first one will be returned. * @throws IOException */ - Optional searchByUuid(String guid, String sensorType, Function> callback) throws IOException{ + Optional searchByGuid(String guid, String sensorType, Function> callback) throws IOException{ QueryBuilder query = QueryBuilders.matchQuery(Constants.GUID, guid); SearchRequestBuilder request = client.prepareSearch() .setTypes(sensorType + "_doc") @@ -200,19 +200,19 @@ public void update(Document update, Optional index) throws IOException { String type = sensorType + "_doc"; byte[] source = JSONUtils.INSTANCE.toJSON(update.getDocument()); Object ts = update.getTimestamp(); - IndexRequest indexRequest = new IndexRequest(indexName, type, update.getUuid()) + IndexRequest indexRequest = new IndexRequest(indexName, type, update.getGuid()) .source(source) ; if(ts != null) { indexRequest = indexRequest.timestamp(ts.toString()); } String existingIndex = index.orElse( - searchByUuid(update.getUuid() + searchByGuid(update.getGuid() , sensorType , hit -> Optional.ofNullable(hit.getIndex()) ).orElse(indexName) ); - UpdateRequest updateRequest = new UpdateRequest(existingIndex, type, update.getUuid()) + UpdateRequest updateRequest = new UpdateRequest(existingIndex, type, update.getGuid()) .doc(source) .upsert(indexRequest) ; From 52aa91a926a716818c9b43579ee5e5088893b4b9 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 3 Aug 2017 10:29:00 +0100 Subject: [PATCH 28/38] s/MockProvider/MockHBaseTableProvider/g --- .../metron/profiler/client/GetProfileTest.java | 12 ++++-------- .../integration/ProfilerIntegrationTest.java | 17 ++++++----------- .../SearchControllerIntegrationTest.java | 10 ++-------- .../UpdateControllerIntegrationTest.java | 10 +++------- .../nonbulk/taxii/TaxiiIntegrationTest.java | 9 ++++----- .../ElasticsearchUpdateIntegrationTest.java | 6 ++---- .../simplehbase/SimpleHBaseAdapterTest.java | 6 +++--- .../threatintel/ThreatIntelAdapterTest.java | 6 +++--- .../integration/EnrichmentIntegrationTest.java | 16 ++++++---------- .../SimpleHBaseEnrichmentFunctionsTest.java | 10 +++------- ...rovider.java => MockHBaseTableProvider.java} | 2 +- .../SimpleHBaseEnrichmentWriterTest.java | 8 ++++---- ...pleHbaseEnrichmentWriterIntegrationTest.java | 6 +++--- 13 files changed, 44 insertions(+), 74 deletions(-) rename metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/mock/{MockProvider.java => MockHBaseTableProvider.java} (95%) diff --git a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/GetProfileTest.java b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/GetProfileTest.java index 56fe85cc3a..00d842c6b6 100644 --- a/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/GetProfileTest.java +++ b/metron-analytics/metron-profiler-client/src/test/java/org/apache/metron/profiler/client/GetProfileTest.java @@ -20,13 +20,11 @@ package org.apache.metron.profiler.client; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.HTableInterface; -import org.apache.metron.hbase.mock.MockProvider; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.apache.metron.stellar.dsl.Context; import org.apache.metron.stellar.dsl.functions.resolver.SimpleFunctionResolver; import org.apache.metron.stellar.dsl.functions.resolver.SingletonFunctionResolver; -import org.apache.metron.hbase.TableProvider; import org.apache.metron.profiler.ProfileMeasurement; import org.apache.metron.profiler.client.stellar.FixedLookback; import org.apache.metron.profiler.client.stellar.GetProfile; @@ -40,8 +38,6 @@ import org.junit.Before; import org.junit.Test; -import java.io.IOException; -import java.io.Serializable; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -89,7 +85,7 @@ private T run(String expression, Class clazz) { @Before public void setup() { state = new HashMap<>(); - final HTableInterface table = MockProvider.addToCache(tableName, columnFamily); + final HTableInterface table = MockHBaseTableProvider.addToCache(tableName, columnFamily); // used to write values to be read during testing RowKeyBuilder rowKeyBuilder = new SaltyRowKeyBuilder(); @@ -100,7 +96,7 @@ public void setup() { Map global = new HashMap() {{ put(PROFILER_HBASE_TABLE.getKey(), tableName); put(PROFILER_COLUMN_FAMILY.getKey(), columnFamily); - put(PROFILER_HBASE_TABLE_PROVIDER.getKey(), MockProvider.class.getName()); + put(PROFILER_HBASE_TABLE_PROVIDER.getKey(), MockHBaseTableProvider.class.getName()); put(PROFILER_PERIOD.getKey(), Long.toString(periodDuration)); put(PROFILER_PERIOD_UNITS.getKey(), periodUnits.toString()); put(PROFILER_SALT_DIVISOR.getKey(), Integer.toString(saltDivisor)); @@ -141,7 +137,7 @@ private Context setup2() { Map global = new HashMap() {{ put(PROFILER_HBASE_TABLE.getKey(), tableName); put(PROFILER_COLUMN_FAMILY.getKey(), columnFamily); - put(PROFILER_HBASE_TABLE_PROVIDER.getKey(), MockProvider.class.getName()); + put(PROFILER_HBASE_TABLE_PROVIDER.getKey(), MockHBaseTableProvider.class.getName()); put(PROFILER_PERIOD.getKey(), Long.toString(periodDuration2)); put(PROFILER_PERIOD_UNITS.getKey(), periodUnits2.toString()); put(PROFILER_SALT_DIVISOR.getKey(), Integer.toString(saltDivisor2)); diff --git a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java index 9e9e7a6bdc..ad968573d0 100644 --- a/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java +++ b/metron-analytics/metron-profiler/src/test/java/org/apache/metron/profiler/integration/ProfilerIntegrationTest.java @@ -23,16 +23,13 @@ import com.google.common.base.Joiner; import org.adrianwalker.multilinestring.Multiline; import org.apache.commons.math.util.MathUtils; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.util.Bytes; import org.apache.metron.common.Constants; import org.apache.metron.common.utils.SerDeUtils; -import org.apache.metron.hbase.TableProvider; import org.apache.metron.hbase.mock.MockHTable; -import org.apache.metron.hbase.mock.MockProvider; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.apache.metron.integration.BaseIntegrationTest; import org.apache.metron.integration.ComponentRunner; import org.apache.metron.integration.UnableToStartException; @@ -50,8 +47,6 @@ import org.junit.Test; import java.io.File; -import java.io.IOException; -import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -305,13 +300,13 @@ public static void setupBeforeClass() throws UnableToStartException { setProperty("profiler.hbase.batch", "10"); setProperty("profiler.hbase.flush.interval.seconds", "1"); setProperty("profiler.profile.ttl", "20"); - setProperty("hbase.provider.impl", "" + MockProvider.class.getName()); + setProperty("hbase.provider.impl", "" + MockHBaseTableProvider.class.getName()); setProperty("storm.auto.credentials", "[]"); setProperty("kafka.security.protocol", "PLAINTEXT"); }}; // create the mock table - profilerTable = (MockHTable) MockProvider.addToCache(tableName, columnFamily); + profilerTable = (MockHTable) MockHBaseTableProvider.addToCache(tableName, columnFamily); zkComponent = getZKServerComponent(topologyProperties); @@ -352,7 +347,7 @@ public void update(String path) throws Exception { @AfterClass public static void tearDownAfterClass() throws Exception { - MockProvider.clear(); + MockHBaseTableProvider.clear(); if (runner != null) { runner.stop(); } @@ -361,12 +356,12 @@ public static void tearDownAfterClass() throws Exception { @Before public void setup() { // create the mock table - profilerTable = (MockHTable) MockProvider.addToCache(tableName, columnFamily); + profilerTable = (MockHTable) MockHBaseTableProvider.addToCache(tableName, columnFamily); } @After public void tearDown() throws Exception { - MockProvider.clear(); + MockHBaseTableProvider.clear(); profilerTable.clear(); if (runner != null) { runner.reset(); diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java index 848e211836..1c5310bdd6 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/SearchControllerIntegrationTest.java @@ -17,15 +17,11 @@ */ package org.apache.metron.rest.controller; -import com.google.common.collect.ImmutableMap; -import org.apache.metron.hbase.mock.MockProvider; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.apache.metron.indexing.dao.InMemoryDao; import org.apache.metron.indexing.dao.SearchIntegrationTest; import org.apache.metron.indexing.dao.search.FieldType; import org.apache.metron.rest.service.SearchService; -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.After; import org.junit.Before; @@ -41,9 +37,7 @@ import org.springframework.test.web.servlet.setup.MockMvcBuilders; import org.springframework.web.context.WebApplicationContext; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; import static org.apache.metron.rest.MetronRestConstants.TEST_PROFILE; @@ -78,7 +72,7 @@ public class SearchControllerIntegrationTest extends DaoControllerTest { @BeforeClass public static void setupHbase() { - MockProvider.addToCache("updates", "t"); + MockHBaseTableProvider.addToCache("updates", "t"); } @Before diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java index aa0adb724d..6037c3f1b5 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java @@ -17,15 +17,12 @@ */ package org.apache.metron.rest.controller; -import org.I0Itec.zkclient.ZkClient; import org.adrianwalker.multilinestring.Multiline; import org.apache.curator.framework.CuratorFramework; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Result; -import org.apache.metron.common.configuration.ConfigurationsUtils; import org.apache.metron.hbase.mock.MockHTable; -import org.apache.metron.hbase.mock.MockProvider; -import org.apache.metron.indexing.dao.SearchIntegrationTest; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.apache.metron.rest.service.UpdateService; import org.junit.Assert; import org.junit.Before; @@ -39,7 +36,6 @@ import org.springframework.test.context.junit4.SpringRunner; import org.springframework.test.web.servlet.MockMvc; import org.springframework.test.web.servlet.ResultActions; -import org.springframework.test.web.servlet.request.MockHttpServletRequestBuilder; import org.springframework.test.web.servlet.setup.MockMvcBuilders; import org.springframework.web.context.WebApplicationContext; @@ -119,7 +115,7 @@ public class UpdateControllerIntegrationTest extends DaoControllerTest { @BeforeClass public static void setupHbase() { - MockProvider.addToCache(TABLE, CF); + MockHBaseTableProvider.addToCache(TABLE, CF); } @Before @@ -145,7 +141,7 @@ public void test() throws Exception { System.err.println(result.andReturn().getResponse().getContentAsString()); throw t; } - MockHTable table = (MockHTable) MockProvider.getFromCache(TABLE); + MockHTable table = (MockHTable) MockHBaseTableProvider.getFromCache(TABLE); Assert.assertEquals(0,table.size()); this.mockMvc.perform(patch(updateUrl+ "/patch").with(httpBasic(user, password)) .with(csrf()) diff --git a/metron-platform/metron-data-management/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiIntegrationTest.java b/metron-platform/metron-data-management/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiIntegrationTest.java index 4af7e2183b..f94a02abbb 100644 --- a/metron-platform/metron-data-management/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiIntegrationTest.java +++ b/metron-platform/metron-data-management/src/test/java/org/apache/metron/dataloads/nonbulk/taxii/TaxiiIntegrationTest.java @@ -32,9 +32,8 @@ import org.apache.metron.enrichment.converter.EnrichmentKey; import org.apache.metron.enrichment.converter.EnrichmentValue; import org.apache.metron.enrichment.lookup.LookupKV; -import org.apache.metron.hbase.HTableProvider; import org.apache.metron.hbase.mock.MockHTable; -import org.apache.metron.hbase.mock.MockProvider; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.junit.*; import java.io.IOException; @@ -51,7 +50,7 @@ public static void setup() throws IOException { @AfterClass public static void teardown() { MockTaxiiService.shutdown(); - MockProvider.clear(); + MockHBaseTableProvider.clear(); } /** @@ -93,7 +92,7 @@ public void testCommandLine() throws Exception { @Test public void testTaxii() throws Exception { - final MockProvider provider = new MockProvider(); + final MockHBaseTableProvider provider = new MockHBaseTableProvider(); final Configuration config = HBaseConfiguration.create(); TaxiiHandler handler = new TaxiiHandler(TaxiiConnectionConfig.load(taxiiConnectionConfig), new StixExtractor(), config ) { @Override @@ -117,7 +116,7 @@ protected synchronized HTableInterface createHTable(String tableInfo) throws IOE } Assert.assertTrue(maliciousAddresses.contains("94.102.53.142")); Assert.assertEquals(numStringsMatch(MockTaxiiService.pollMsg, "AddressObj:Address_Value condition=\"Equal\""), maliciousAddresses.size()); - MockProvider.clear(); + MockHBaseTableProvider.clear(); // Ensure that the handler can be run multiple times without connection issues. handler.run(); 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 0fef4fae2a..9ad32fcfd4 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 @@ -25,13 +25,11 @@ 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.writer.WriterConfiguration; import org.apache.metron.common.utils.JSONUtils; import org.apache.metron.elasticsearch.dao.ElasticsearchDao; import org.apache.metron.elasticsearch.integration.components.ElasticSearchComponent; -import org.apache.metron.hbase.TableProvider; import org.apache.metron.hbase.mock.MockHTable; -import org.apache.metron.hbase.mock.MockProvider; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.apache.metron.indexing.dao.*; import org.apache.metron.indexing.dao.update.Document; import org.apache.metron.indexing.dao.update.ReplaceRequest; @@ -63,7 +61,7 @@ public class ElasticsearchUpdateIntegrationTest { @BeforeClass public static void setup() throws Exception { Configuration config = HBaseConfiguration.create(); - MockProvider tableProvider = new MockProvider(); + MockHBaseTableProvider tableProvider = new MockHBaseTableProvider(); tableProvider.addToCache(TABLE_NAME, CF); table = (MockHTable)tableProvider.getTable(config, TABLE_NAME); // setup the client diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapterTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapterTest.java index 0c52bc94f8..974f8ab4bf 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapterTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/simplehbase/SimpleHBaseAdapterTest.java @@ -27,7 +27,7 @@ import org.apache.metron.enrichment.lookup.EnrichmentLookup; import org.apache.metron.enrichment.converter.EnrichmentHelper; import org.apache.metron.hbase.mock.MockHTable; -import org.apache.metron.hbase.mock.MockProvider; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.apache.metron.enrichment.lookup.LookupKV; import org.apache.metron.enrichment.lookup.accesstracker.BloomAccessTracker; import org.apache.metron.enrichment.lookup.accesstracker.PersistentAccessTracker; @@ -103,8 +103,8 @@ public class SimpleHBaseAdapterTest { @Before public void setup() throws Exception { - final MockHTable trackerTable = (MockHTable) MockProvider.addToCache(atTableName, cf); - final MockHTable hbaseTable = (MockHTable) MockProvider.addToCache(hbaseTableName, cf); + final MockHTable trackerTable = (MockHTable) MockHBaseTableProvider.addToCache(atTableName, cf); + final MockHTable hbaseTable = (MockHTable) MockHBaseTableProvider.addToCache(hbaseTableName, cf); EnrichmentHelper.INSTANCE.load(hbaseTable, cf, new ArrayList>() {{ add(new LookupKV<>(new EnrichmentKey(PLAYFUL_CLASSIFICATION_TYPE, "10.0.2.3") , new EnrichmentValue(PLAYFUL_ENRICHMENT) diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapterTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapterTest.java index e18de930e6..e421edcf31 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapterTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/adapters/threatintel/ThreatIntelAdapterTest.java @@ -29,7 +29,7 @@ import org.apache.metron.enrichment.lookup.EnrichmentLookup; import org.apache.metron.enrichment.converter.EnrichmentHelper; import org.apache.metron.hbase.mock.MockHTable; -import org.apache.metron.hbase.mock.MockProvider; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.apache.metron.enrichment.lookup.LookupKV; import org.apache.metron.enrichment.lookup.accesstracker.BloomAccessTracker; import org.apache.metron.enrichment.lookup.accesstracker.PersistentAccessTracker; @@ -99,8 +99,8 @@ public HTableInterface getTable(Configuration config, String tableName) throws I @Before public void setup() throws Exception { - final MockHTable trackerTable = (MockHTable)MockProvider.addToCache(atTableName, cf); - final MockHTable threatIntelTable = (MockHTable)MockProvider.addToCache(threatIntelTableName, cf); + final MockHTable trackerTable = (MockHTable) MockHBaseTableProvider.addToCache(atTableName, cf); + final MockHTable threatIntelTable = (MockHTable) MockHBaseTableProvider.addToCache(threatIntelTableName, cf); EnrichmentHelper.INSTANCE.load(threatIntelTable, cf, new ArrayList>() {{ add(new LookupKV<>(new EnrichmentKey("10.0.2.3", "10.0.2.3"), new EnrichmentValue(new HashMap<>()))); }}); diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java index 62def32fc0..f77f16e406 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/integration/EnrichmentIntegrationTest.java @@ -25,8 +25,6 @@ import com.google.common.base.Splitter; import com.google.common.collect.Iterables; import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.metron.TestConstants; import org.apache.metron.common.Constants; import org.apache.metron.common.utils.JSONUtils; @@ -38,9 +36,8 @@ import org.apache.metron.enrichment.lookup.LookupKV; import org.apache.metron.enrichment.lookup.accesstracker.PersistentBloomTrackerCreator; import org.apache.metron.enrichment.stellar.SimpleHBaseEnrichmentFunctions; -import org.apache.metron.hbase.TableProvider; import org.apache.metron.hbase.mock.MockHTable; -import org.apache.metron.hbase.mock.MockProvider; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.apache.metron.integration.BaseIntegrationTest; import org.apache.metron.integration.ComponentRunner; import org.apache.metron.integration.ProcessorResult; @@ -59,7 +56,6 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -132,7 +128,7 @@ public void test() throws Exception { setProperty("threatintel_error_topic", ERROR_TOPIC); setProperty("enrichment_join_cache_size", "1000"); setProperty("threatintel_join_cache_size", "1000"); - setProperty("enrichment_hbase_provider_impl", "" + MockProvider.class.getName()); + setProperty("enrichment_hbase_provider_impl", "" + MockHBaseTableProvider.class.getName()); setProperty("enrichment_table", enrichmentsTableName); setProperty("enrichment_cf", cf); setProperty("enrichment_host_known_hosts", "[{\"ip\":\"10.1.128.236\", \"local\":\"YES\", \"type\":\"webserver\", \"asset_value\" : \"important\"}," + @@ -165,7 +161,7 @@ public void test() throws Exception { File globalConfig = new File(new File(TestConstants.SAMPLE_CONFIG_PATH), "global.json"); Map config = JSONUtils.INSTANCE.load(globalConfig, new TypeReference>() { }); - config.put(SimpleHBaseEnrichmentFunctions.TABLE_PROVIDER_TYPE_CONF, MockProvider.class.getName()); + config.put(SimpleHBaseEnrichmentFunctions.TABLE_PROVIDER_TYPE_CONF, MockHBaseTableProvider.class.getName()); config.put(SimpleHBaseEnrichmentFunctions.ACCESS_TRACKER_TYPE_CONF, "PERSISTENT_BLOOM"); config.put(PersistentBloomTrackerCreator.Config.PERSISTENT_BLOOM_TABLE, trackerHBaseTableName); config.put(PersistentBloomTrackerCreator.Config.PERSISTENT_BLOOM_CF, cf); @@ -178,12 +174,12 @@ public void test() throws Exception { .withEnrichmentConfigsPath(TestConstants.SAMPLE_CONFIG_PATH); //create MockHBaseTables - final MockHTable trackerTable = (MockHTable) MockProvider.addToCache(trackerHBaseTableName, cf); - final MockHTable threatIntelTable = (MockHTable) MockProvider.addToCache(threatIntelTableName, cf); + final MockHTable trackerTable = (MockHTable) MockHBaseTableProvider.addToCache(trackerHBaseTableName, cf); + final MockHTable threatIntelTable = (MockHTable) MockHBaseTableProvider.addToCache(threatIntelTableName, cf); EnrichmentHelper.INSTANCE.load(threatIntelTable, cf, new ArrayList>() {{ add(new LookupKV<>(new EnrichmentKey(MALICIOUS_IP_TYPE, "10.0.2.3"), new EnrichmentValue(new HashMap<>()))); }}); - final MockHTable enrichmentTable = (MockHTable) MockProvider.addToCache(enrichmentsTableName, cf); + final MockHTable enrichmentTable = (MockHTable) MockHBaseTableProvider.addToCache(enrichmentsTableName, cf); EnrichmentHelper.INSTANCE.load(enrichmentTable, cf, new ArrayList>() {{ add(new LookupKV<>(new EnrichmentKey(PLAYFUL_CLASSIFICATION_TYPE, "10.0.2.3") , new EnrichmentValue(PLAYFUL_ENRICHMENT) diff --git a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/stellar/SimpleHBaseEnrichmentFunctionsTest.java b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/stellar/SimpleHBaseEnrichmentFunctionsTest.java index bf5f1dfc3b..b39d72b8d1 100644 --- a/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/stellar/SimpleHBaseEnrichmentFunctionsTest.java +++ b/metron-platform/metron-enrichment/src/test/java/org/apache/metron/enrichment/stellar/SimpleHBaseEnrichmentFunctionsTest.java @@ -19,10 +19,8 @@ package org.apache.metron.enrichment.stellar; import com.google.common.collect.ImmutableMap; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.client.HTableInterface; import org.apache.metron.hbase.mock.MockHTable; -import org.apache.metron.hbase.mock.MockProvider; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.apache.metron.stellar.dsl.Context; import org.apache.metron.stellar.dsl.ParseException; import org.apache.metron.stellar.dsl.StellarFunctions; @@ -32,12 +30,10 @@ import org.apache.metron.enrichment.converter.EnrichmentKey; import org.apache.metron.enrichment.converter.EnrichmentValue; import org.apache.metron.enrichment.lookup.LookupKV; -import org.apache.metron.hbase.TableProvider; import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -54,7 +50,7 @@ public class SimpleHBaseEnrichmentFunctionsTest { @Before public void setup() throws Exception { - final MockHTable hbaseTable = (MockHTable) MockProvider.addToCache(hbaseTableName, cf); + final MockHTable hbaseTable = (MockHTable) MockHBaseTableProvider.addToCache(hbaseTableName, cf); EnrichmentHelper.INSTANCE.load(hbaseTable, cf, new ArrayList>() {{ for(int i = 0;i < 5;++i) { add(new LookupKV<>(new EnrichmentKey(ENRICHMENT_TYPE, "indicator" + i) @@ -66,7 +62,7 @@ public void setup() throws Exception { context = new Context.Builder() .with( Context.Capabilities.GLOBAL_CONFIG , () -> ImmutableMap.of( SimpleHBaseEnrichmentFunctions.TABLE_PROVIDER_TYPE_CONF - , MockProvider.class.getName() + , MockHBaseTableProvider.class.getName() ) ) .build(); diff --git a/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/mock/MockProvider.java b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/mock/MockHBaseTableProvider.java similarity index 95% rename from metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/mock/MockProvider.java rename to metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/mock/MockHBaseTableProvider.java index 723ab4df5c..57981aca53 100644 --- a/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/mock/MockProvider.java +++ b/metron-platform/metron-hbase/src/test/java/org/apache/metron/hbase/mock/MockHBaseTableProvider.java @@ -26,7 +26,7 @@ import java.util.HashMap; import java.util.Map; -public class MockProvider implements Serializable, TableProvider { +public class MockHBaseTableProvider implements Serializable, TableProvider { private static Map _cache = new HashMap<>(); public HTableInterface getTable(Configuration config, String tableName) throws IOException { HTableInterface ret = _cache.get(tableName); diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/SimpleHBaseEnrichmentWriterTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/SimpleHBaseEnrichmentWriterTest.java index 7095327829..5f4b3fd0d3 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/SimpleHBaseEnrichmentWriterTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/SimpleHBaseEnrichmentWriterTest.java @@ -29,7 +29,7 @@ import org.apache.metron.enrichment.lookup.LookupKV; import org.apache.metron.enrichment.writer.SimpleHbaseEnrichmentWriter; import org.apache.metron.hbase.mock.MockHTable; -import org.apache.metron.hbase.mock.MockProvider; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.json.simple.JSONObject; import org.junit.Assert; import org.junit.Before; @@ -50,12 +50,12 @@ public class SimpleHBaseEnrichmentWriterTest { put(SimpleHbaseEnrichmentWriter.Configurations.HBASE_TABLE.getKey(), TABLE_NAME); put(SimpleHbaseEnrichmentWriter.Configurations.HBASE_CF.getKey(), TABLE_CF); put(SimpleHbaseEnrichmentWriter.Configurations.ENRICHMENT_TYPE.getKey(), ENRICHMENT_TYPE); - put(SimpleHbaseEnrichmentWriter.Configurations.HBASE_PROVIDER.getKey(), MockProvider.class.getName()); + put(SimpleHbaseEnrichmentWriter.Configurations.HBASE_PROVIDER.getKey(), MockHBaseTableProvider.class.getName()); }}; @Before public void setupMockTable() { - MockProvider.addToCache(TABLE_NAME, TABLE_CF); + MockHBaseTableProvider.addToCache(TABLE_NAME, TABLE_CF); } @Test public void testBatchOneNormalPath() throws Exception { @@ -141,7 +141,7 @@ public void testFilteredKeys() throws Exception { Assert.assertEquals(2, values.get(0).getValue().getMetadata().size()); } public static List> getValues() throws IOException { - MockHTable table = (MockHTable)MockProvider.getFromCache(TABLE_NAME); + MockHTable table = (MockHTable) MockHBaseTableProvider.getFromCache(TABLE_NAME); Assert.assertNotNull(table); List> ret = new ArrayList<>(); EnrichmentConverter converter = new EnrichmentConverter(); diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java index 75140c09a1..52a6c468b2 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java @@ -31,7 +31,7 @@ import org.apache.metron.enrichment.integration.components.ConfigUploadComponent; import org.apache.metron.enrichment.lookup.LookupKV; import org.apache.metron.hbase.mock.MockHTable; -import org.apache.metron.hbase.mock.MockProvider; +import org.apache.metron.hbase.mock.MockHBaseTableProvider; import org.apache.metron.integration.*; import org.apache.metron.integration.components.KafkaComponent; import org.apache.metron.integration.components.ZKServerComponent; @@ -75,7 +75,7 @@ public void test() throws UnableToStartException, IOException { add(Bytes.toBytes("col21,col22,col23")); add(Bytes.toBytes("col31,col32,col33")); }}; - MockProvider.addToCache(sensorType, "cf"); + MockHBaseTableProvider.addToCache(sensorType, "cf"); final Properties topologyProperties = new Properties(); final ZKServerComponent zkServerComponent = getZKServerComponent(topologyProperties); final KafkaComponent kafkaComponent = getKafkaComponent(topologyProperties, new ArrayList() {{ @@ -112,7 +112,7 @@ public void test() throws UnableToStartException, IOException { @Override public ReadinessState process(ComponentRunner runner) { - MockHTable table = (MockHTable)MockProvider.getFromCache(sensorType); + MockHTable table = (MockHTable) MockHBaseTableProvider.getFromCache(sensorType); if (table != null && table.size() == inputMessages.size()) { EnrichmentConverter converter = new EnrichmentConverter(); messages = new ArrayList<>(); From 56ccec70aa85bbb957b0d9f158ba11dea1fe319a Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 3 Aug 2017 10:39:25 +0100 Subject: [PATCH 29/38] Updating uuid. --- .../rest/controller/SearchController.java | 2 +- .../UpdateControllerIntegrationTest.java | 18 +++++++++--------- .../ElasticsearchUpdateIntegrationTest.java | 16 ++++++++-------- metron-platform/metron-hbase-client/pom.xml | 19 ++++++++++++++----- .../apache/metron/indexing/dao/IndexDao.java | 2 +- 5 files changed, 33 insertions(+), 24 deletions(-) diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/SearchController.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/SearchController.java index 97ea5421e2..dea628c285 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/SearchController.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/controller/SearchController.java @@ -54,7 +54,7 @@ ResponseEntity search(final @ApiParam(name = "searchRequest", va return new ResponseEntity<>(searchService.search(searchRequest), HttpStatus.OK); } - @ApiOperation(value = "Returns latest document for a uuid and sensor") + @ApiOperation(value = "Returns latest document for a guid and sensor") @ApiResponse(message = "Document representing the output", code = 200) @RequestMapping(value = "/findOne", method = RequestMethod.POST) ResponseEntity> getLatest( diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java index 6037c3f1b5..0d7fde7225 100644 --- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java +++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java @@ -72,7 +72,7 @@ public class UpdateControllerIntegrationTest extends DaoControllerTest { /** { - "uuid" : "bro_index_2017.01.01.01:1", + "guid" : "bro_index_2017.01.01.01:1", "sensorType" : "bro" } */ @@ -81,7 +81,7 @@ public class UpdateControllerIntegrationTest extends DaoControllerTest { /** { - "uuid" : "bro_index_2017.01.01.01:1", + "guid" : "bro_index_2017.01.01.01:1", "sensorType" : "bro", "patch" : [ { @@ -97,7 +97,7 @@ public class UpdateControllerIntegrationTest extends DaoControllerTest { /** { - "uuid" : "bro_index_2017.01.01.01:1", + "guid" : "bro_index_2017.01.01.01:1", "sensorType" : "bro", "replacement" : { "source:type": "bro", @@ -126,13 +126,13 @@ public void setup() throws Exception { @Test public void test() throws Exception { - String uuid = "bro_index_2017.01.01.01:1"; + String guid = "bro_index_2017.01.01.01:1"; ResultActions result = this.mockMvc.perform(post(searchUrl + "/findOne").with(httpBasic(user, password)).with(csrf()).contentType(MediaType.parseMediaType("application/json;charset=UTF-8")).content(findMessage0)); try { result.andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) .andExpect(jsonPath("$.source:type").value("bro")) - .andExpect(jsonPath("$.guid").value(uuid)) + .andExpect(jsonPath("$.guid").value(guid)) .andExpect(jsonPath("$.project").doesNotExist()) .andExpect(jsonPath("$.timestamp").value(2)) ; @@ -153,14 +153,14 @@ public void test() throws Exception { .andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) .andExpect(jsonPath("$.source:type").value("bro")) - .andExpect(jsonPath("$.guid").value(uuid)) + .andExpect(jsonPath("$.guid").value(guid)) .andExpect(jsonPath("$.project").value("metron")) .andExpect(jsonPath("$.timestamp").value(2)) ; Assert.assertEquals(1,table.size()); { //ensure hbase is up to date - Get g = new Get(uuid.getBytes()); + Get g = new Get(guid.getBytes()); Result r = table.get(g); NavigableMap columns = r.getFamilyMap(CF.getBytes()); Assert.assertEquals(1, columns.size()); @@ -175,14 +175,14 @@ public void test() throws Exception { .andExpect(status().isOk()) .andExpect(content().contentType(MediaType.parseMediaType("application/json;charset=UTF-8"))) .andExpect(jsonPath("$.source:type").value("bro")) - .andExpect(jsonPath("$.guid").value(uuid)) + .andExpect(jsonPath("$.guid").value(guid)) .andExpect(jsonPath("$.project").doesNotExist()) .andExpect(jsonPath("$.timestamp").value(200)) ; Assert.assertEquals(1,table.size()); { //ensure hbase is up to date - Get g = new Get(uuid.getBytes()); + Get g = new Get(guid.getBytes()); Result r = table.get(g); NavigableMap columns = r.getFamilyMap(CF.getBytes()); Assert.assertEquals(2, columns.size()); 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 9ad32fcfd4..542a36e789 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 @@ -138,18 +138,18 @@ public void test() throws Exception { Map message0 = new HashMap(inputData.get(0)) {{ put("new-field", "metron"); }}; - String uuid = "" + message0.get(Constants.GUID); + String guid = "" + message0.get(Constants.GUID); dao.replace(new ReplaceRequest(){{ setReplacement(message0); - setUuid(uuid); + setGuid(guid); setSensorType(SENSOR_NAME); }}, Optional.empty()); Assert.assertEquals(1, table.size()); - Document doc = dao.getLatest(uuid, SENSOR_NAME); + Document doc = dao.getLatest(guid, SENSOR_NAME); Assert.assertEquals(message0, doc.getDocument()); { //ensure hbase is up to date - Get g = new Get(uuid.getBytes()); + Get g = new Get(guid.getBytes()); Result r = table.get(g); NavigableMap columns = r.getFamilyMap(CF.getBytes()); Assert.assertEquals(1, columns.size()); @@ -180,18 +180,18 @@ public void test() throws Exception { Map message0 = new HashMap(inputData.get(0)) {{ put("new-field", "metron2"); }}; - String uuid = "" + message0.get(Constants.GUID); + String guid = "" + message0.get(Constants.GUID); dao.replace(new ReplaceRequest(){{ setReplacement(message0); - setUuid(uuid); + setGuid(guid); setSensorType(SENSOR_NAME); }}, Optional.empty()); Assert.assertEquals(1, table.size()); - Document doc = dao.getLatest(uuid, SENSOR_NAME); + Document doc = dao.getLatest(guid, SENSOR_NAME); Assert.assertEquals(message0, doc.getDocument()); { //ensure hbase is up to date - Get g = new Get(uuid.getBytes()); + Get g = new Get(guid.getBytes()); Result r = table.get(g); NavigableMap columns = r.getFamilyMap(CF.getBytes()); Assert.assertEquals(2, columns.size()); diff --git a/metron-platform/metron-hbase-client/pom.xml b/metron-platform/metron-hbase-client/pom.xml index 6076db60b9..5dd6127796 100644 --- a/metron-platform/metron-hbase-client/pom.xml +++ b/metron-platform/metron-hbase-client/pom.xml @@ -20,22 +20,31 @@ metron-platform 0.4.1 + metron-hbase-client metron-hbase-client https://metron.apache.org/ UTF-8 UTF-8 - 1.7.7 - 0.1.2 - ${global_hbase_guava_version} + + + 1.1.2 org.apache.hbase hbase-shaded-client - - 1.1.2 + ${shaded.client.version} diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java index d46e36a0c0..bb3b1d6754 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java @@ -42,7 +42,7 @@ public interface IndexDao { SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException; void init(AccessConfig config); - Document getLatest(String uuid, String sensorType) throws IOException; + Document getLatest(String guid, String sensorType) throws IOException; default Optional> getLatestResult(GetRequest request) throws IOException { Document ret = getLatest(request.getGuid(), request.getSensorType()); From 27ef4f213554774939f52669cd88cc3421587537 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 3 Aug 2017 11:03:48 +0100 Subject: [PATCH 30/38] Missed a string in a class name change. --- .../integration/SimpleHbaseEnrichmentWriterIntegrationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java index 52a6c468b2..4f513beba5 100644 --- a/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java +++ b/metron-platform/metron-parsers/src/test/java/org/apache/metron/writers/integration/SimpleHbaseEnrichmentWriterIntegrationTest.java @@ -55,7 +55,7 @@ public class SimpleHbaseEnrichmentWriterIntegrationTest extends BaseIntegrationT ,"shew.cf" : "cf" ,"shew.keyColumns" : "col2" ,"shew.enrichmentType" : "et" - ,"shew.hbaseProvider" : "org.apache.metron.hbase.mock.MockProvider" + ,"shew.hbaseProvider" : "org.apache.metron.hbase.mock.MockHBaseTableProvider" ,"columns" : { "col1" : 0 ,"col2" : 1 From 1c0e677d3e952fc70e7a2841b140403b7cf2a580 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 3 Aug 2017 11:47:22 +0100 Subject: [PATCH 31/38] One more miss. --- .../metron-rest/src/main/resources/application-test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metron-interface/metron-rest/src/main/resources/application-test.yml b/metron-interface/metron-rest/src/main/resources/application-test.yml index 18d3e54ffd..9793840332 100644 --- a/metron-interface/metron-rest/src/main/resources/application-test.yml +++ b/metron-interface/metron-rest/src/main/resources/application-test.yml @@ -51,4 +51,4 @@ index: impl: org.apache.metron.indexing.dao.InMemoryDao,org.apache.metron.indexing.dao.HBaseDao hbase: # HBase is provided via a mock provider, so no actual HBase infrastructure is started. - provider: org.apache.metron.hbase.mock.MockProvider + provider: org.apache.metron.hbase.mock.MockHBaseTableProvider From 3aca48f6a408121d75835336785e40ac9dc1bd02 Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 3 Aug 2017 13:31:13 +0100 Subject: [PATCH 32/38] Better javadocs. --- .../metron/indexing/dao/AccessConfig.java | 16 +++++++ .../apache/metron/indexing/dao/IndexDao.java | 47 +++++++++++++++++++ .../indexing/dao/search/GetRequest.java | 8 ++++ .../indexing/dao/search/SearchRequest.java | 20 ++++++++ .../indexing/dao/search/SearchResponse.java | 8 ++++ .../indexing/dao/search/SearchResult.java | 16 +++++++ .../indexing/dao/update/PatchRequest.java | 31 ++++++++++++ .../indexing/dao/update/ReplaceRequest.java | 16 +++++++ 8 files changed, 162 insertions(+) diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java index 972ef00e0f..ddb88e5895 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java @@ -30,6 +30,10 @@ public class AccessConfig { private Map optionalSettings = new HashMap<>(); private TableProvider tableProvider = null; + /** + * A supplier which will return the current global config. + * @return + */ public Supplier> getGlobalConfigSupplier() { return globalConfigSupplier; } @@ -38,6 +42,10 @@ public void setGlobalConfigSupplier(Supplier> globalConfigSu this.globalConfigSupplier = globalConfigSupplier; } + /** + * The maximum search result. + * @return + */ public Integer getMaxSearchResults() { return maxSearchResults; } @@ -46,6 +54,10 @@ public void setMaxSearchResults(Integer maxSearchResults) { this.maxSearchResults = maxSearchResults; } + /** + * Get optional settings for initializing indices. + * @return + */ public Map getOptionalSettings() { return optionalSettings; } @@ -54,6 +66,10 @@ public void setOptionalSettings(Map optionalSettings) { this.optionalSettings = optionalSettings; } + /** + * Return the table provider to use for NoSql DAOs + * @return + */ public TableProvider getTableProvider() { return tableProvider; } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java index bb3b1d6754..350e402ed5 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java @@ -40,10 +40,37 @@ public interface IndexDao { + /** + * Return search response based on the search request + * + * @param searchRequest + * @return + * @throws InvalidSearchException + */ SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException; + + /** + * Initialize the DAO with the AccessConfig object. + * @param config + */ void init(AccessConfig config); + + /** + * Return the latest version of a document given the GUID and the sensor type. + * + * @param guid The GUID for the document + * @param sensorType The sensor type of the document + * @return The Document matching or null if not available. + * @throws IOException + */ Document getLatest(String guid, String sensorType) throws IOException; + /** + * Return the latest version of a document given a GetRequest. + * @param request The GetRequest which indicates the GUID and sensor type. + * @return Optionally the document (dependent upon existence in the index). + * @throws IOException + */ default Optional> getLatestResult(GetRequest request) throws IOException { Document ret = getLatest(request.getGuid(), request.getSensorType()); if(ret == null) { @@ -54,9 +81,23 @@ default Optional> getLatestResult(GetRequest request) throws } } + /** + * Update given a Document and optionally the index where the document exists. + * + * @param update The document to replace from the index. + * @param index The index where the document lives. + * @throws IOException + */ void update(Document update, Optional index) throws IOException; + /** + * Update a document in an index given a JSON Patch (see RFC 6902 at https://tools.ietf.org/html/rfc6902) + * @param request The patch request + * @param timestamp Optionally a timestamp to set. If not specified then current time is used. + * @throws OriginalNotFoundException If the original is not found, then it cannot be patched. + * @throws IOException + */ default void patch( PatchRequest request , Optional timestamp ) throws OriginalNotFoundException, IOException { @@ -82,6 +123,12 @@ default void patch( PatchRequest request update(d, Optional.ofNullable(request.getIndex())); } + /** + * Replace a document in an index. + * @param request The replacement request. + * @param timestamp The timestamp (optional) of the update. If not specified, then current time will be used. + * @throws IOException + */ default void replace( ReplaceRequest request , Optional timestamp ) throws IOException { diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java index 8a0aeaa62a..eb255dc25e 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/GetRequest.java @@ -21,6 +21,10 @@ public class GetRequest { String guid; String sensorType; + /** + * The GUID of the document + * @return + */ public String getGuid() { return guid; } @@ -29,6 +33,10 @@ public void setGuid(String guid) { this.guid = guid; } + /** + * The sensor type of the indices that you're searching. + * @return + */ public String getSensorType() { return sensorType; } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchRequest.java index ecf6b57f4f..8c5fecaeab 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchRequest.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchRequest.java @@ -36,6 +36,10 @@ public SearchRequest() { sort.add(defaultSortField); } + /** + * The list of indices to search. + * @return + */ public List getIndices() { return indices; } @@ -44,6 +48,10 @@ public void setIndices(List indices) { this.indices = indices; } + /** + * The query to use to search the index + * @return + */ public String getQuery() { return query; } @@ -52,6 +60,10 @@ public void setQuery(String query) { this.query = query; } + /** + * The size of the results returned. + * @return + */ public int getSize() { return size; } @@ -60,6 +72,10 @@ public void setSize(int size) { this.size = size; } + /** + * The index to start the search from. + * @return + */ public int getFrom() { return from; } @@ -68,6 +84,10 @@ public void setFrom(int from) { this.from = from; } + /** + * The search order by field. + * @return + */ public List getSort() { return sort; } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchResponse.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchResponse.java index 7f616940b6..159d906095 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchResponse.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchResponse.java @@ -25,6 +25,10 @@ public class SearchResponse { private long total; private List results = new ArrayList<>(); + /** + * The total number of results + * @return + */ public long getTotal() { return total; } @@ -33,6 +37,10 @@ public void setTotal(long total) { this.total = total; } + /** + * The list of results + * @return + */ public List getResults() { return results; } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchResult.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchResult.java index 4df07813c7..9c00beac55 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchResult.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchResult.java @@ -26,6 +26,10 @@ public class SearchResult { private float score; private String index; + /** + * The index that the result comes from + * @return + */ public String getIndex() { return index; } @@ -34,6 +38,10 @@ public void setIndex(String index) { this.index = index; } + /** + * The ID of the document from the index. + * @return + */ public String getId() { return id; } @@ -42,6 +50,10 @@ public void setId(String id) { this.id = id; } + /** + * The source (the actual result). + * @return + */ public Map getSource() { return source; } @@ -50,6 +62,10 @@ public void setSource(Map source) { this.source = source; } + /** + * The score from the index. + * @return + */ public float getScore() { return score; } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java index 12ecaf8352..77f59581f0 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchRequest.java @@ -28,6 +28,10 @@ public class PatchRequest { String sensorType; String index; + /** + * The index of the document to be updated. This is optional, but could result in a performance gain if specified. + * @return + */ public String getIndex() { return index; } @@ -36,6 +40,20 @@ public void setIndex(String index) { this.index = index; } + /** + * The patch. This is in the form of a list of RFC 6902 patches. + * For example: + *
+   * [
+   *   {
+   *             "op": "add"
+   *            , "path": "/project"
+   *            , "value": "metron"
+   *   }
+   *           ]
+   * 
+ * @return + */ public JsonNode getPatch() { return patch; } @@ -44,6 +62,11 @@ public void setPatch(JsonNode patch) { this.patch = patch; } + /** + * The source document. If this is specified, then it will be used as the basis of the patch rather than the current + * document in the index. + * @return + */ public Map getSource() { return source; } @@ -52,6 +75,10 @@ public void setSource(Map source) { this.source = source; } + /** + * The GUID of the document to be patched. + * @return + */ public String getGuid() { return guid; } @@ -60,6 +87,10 @@ public void setGuid(String guid) { this.guid = guid; } + /** + * The sensor type of the document. + * @return + */ public String getSensorType() { return sensorType; } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java index 42d6cd6aa5..96bca3028e 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/ReplaceRequest.java @@ -27,6 +27,10 @@ public class ReplaceRequest { String sensorType; String index; + /** + * Return the index of the request. This is optional, but could result in better performance if specified. + * @return + */ public String getIndex() { return index; } @@ -35,6 +39,10 @@ public void setIndex(String index) { this.index = index; } + /** + * The sensor type of the request. This is mandatory. + * @return + */ public String getSensorType() { return sensorType; } @@ -43,6 +51,10 @@ public void setSensorType(String sensorType) { this.sensorType = sensorType; } + /** + * The replacement document. This is mandatory. + * @return + */ public Map getReplacement() { return replacement; } @@ -51,6 +63,10 @@ public void setReplacement(Map replacement) { this.replacement = replacement; } + /** + * The GUID of the document to replace. This is mandatory. + * @return + */ public String getGuid() { return guid; } From 99a1866cb7985d6ab40debfa9c217e35fe87196e Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 3 Aug 2017 13:38:30 +0100 Subject: [PATCH 33/38] Not required to convert to byte[] for ES --- .../apache/metron/elasticsearch/dao/ElasticsearchDao.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) 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 fd9cb8ed2f..261dd31549 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 @@ -198,10 +198,9 @@ public void update(Document update, Optional index) throws IOException { String indexName = ElasticsearchUtils.getIndexName(sensorType, indexPostfix, null); String type = sensorType + "_doc"; - byte[] source = JSONUtils.INSTANCE.toJSON(update.getDocument()); Object ts = update.getTimestamp(); IndexRequest indexRequest = new IndexRequest(indexName, type, update.getGuid()) - .source(source) + .source(update.getDocument()) ; if(ts != null) { indexRequest = indexRequest.timestamp(ts.toString()); @@ -213,7 +212,7 @@ public void update(Document update, Optional index) throws IOException { ).orElse(indexName) ); UpdateRequest updateRequest = new UpdateRequest(existingIndex, type, update.getGuid()) - .doc(source) + .doc(update.getDocument()) .upsert(indexRequest) ; From e55447d62b68295355fee509e29cece21980d9cb Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 3 Aug 2017 16:21:04 +0100 Subject: [PATCH 34/38] Reacted to @justinleet's comments. --- .../ElasticsearchUpdateIntegrationTest.java | 10 +++------- .../integration/components/ElasticSearchComponent.java | 9 ++------- .../org/apache/metron/indexing/dao/MultiIndexDao.java | 4 +--- 3 files changed, 6 insertions(+), 17 deletions(-) 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 542a36e789..0af795f94d 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 @@ -165,14 +165,10 @@ public void test() throws Exception { docs = es.getAllIndexedDocs(index, SENSOR_NAME + "_doc"); cnt = docs .stream() - .filter(d -> { - Object newfield = d.get("new-field"); - return newfield != null && newfield.equals(message0.get("new-field")); - }).count(); - } - if (cnt == 0) { - Assert.fail("Elasticsearch is not updated!"); + .filter(d -> message0.get("new-field").equals(d.get("new-field"))) + .count(); } + Assert.assertNotEquals("Elasticsearch is not updated!", cnt, 0); } } //modify the same message and modify the new field 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 abddf774a6..19285a9463 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 @@ -43,10 +43,7 @@ import java.io.File; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; public class ElasticSearchComponent implements InMemoryComponent { @@ -98,9 +95,7 @@ private void cleanDir(File dir) throws IOException { public BulkResponse add(String indexName, String sensorType, String... docs) throws IOException { List d = new ArrayList<>(); - for(String doc : docs) { - d.add(doc); - } + Collections.addAll(d, docs); return add(indexName, sensorType, d); } diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java index 7481aa0d9e..e9a4a9a299 100644 --- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java +++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MultiIndexDao.java @@ -37,9 +37,7 @@ public class MultiIndexDao implements IndexDao { public MultiIndexDao( IndexDao... composedDao) { indices = new ArrayList<>(); - for(IndexDao dao: composedDao) { - indices.add(dao); - } + Collections.addAll(indices, composedDao); } public MultiIndexDao(Iterable composedDao) { From 68589f556cba4fb9362be604016c5cf525ebd24d Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 3 Aug 2017 16:28:31 +0100 Subject: [PATCH 35/38] Cleaned up some unit tests. --- .../ElasticsearchUpdateIntegrationTest.java | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) 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 0af795f94d..9a1d7a77df 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 @@ -205,14 +205,11 @@ public void test() throws Exception { docs = es.getAllIndexedDocs(index, SENSOR_NAME + "_doc"); cnt = docs .stream() - .filter(d -> { - Object newfield = d.get("new-field"); - return newfield != null && newfield.equals(message0.get("new-field")); - }).count(); - } - if (cnt == 0) { - Assert.fail("Elasticsearch is not updated!"); + .filter(d -> message0.get("new-field").equals(d.get("new-field"))) + .count(); } + + Assert.assertNotEquals("Elasticsearch is not updated!", cnt, 0); } } } From dcdda3daac738dc8fb9f37b622f16a426f7b63bc Mon Sep 17 00:00:00 2001 From: cstella Date: Thu, 3 Aug 2017 21:22:34 +0100 Subject: [PATCH 36/38] Removing star imports. --- .../integration/components/ElasticSearchComponent.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 19285a9463..7facff5741 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 @@ -43,7 +43,11 @@ import java.io.File; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; public class ElasticSearchComponent implements InMemoryComponent { From fb0991409899fddf0390de586b3e21ecc22f69a5 Mon Sep 17 00:00:00 2001 From: cstella Date: Mon, 7 Aug 2017 11:25:08 -0400 Subject: [PATCH 37/38] Whoops, bad merge. --- .../org/apache/metron/elasticsearch/dao/ElasticsearchDao.java | 1 + 1 file changed, 1 insertion(+) 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 d52a87fa5c..217da84853 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 @@ -69,6 +69,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.function.Function; import java.util.stream.Collectors; public class ElasticsearchDao implements IndexDao { From cc778c828e0adececacc350d9ef5ae9b21f5ead8 Mon Sep 17 00:00:00 2001 From: cstella Date: Mon, 7 Aug 2017 11:50:35 -0400 Subject: [PATCH 38/38] wrong constructor. --- .../org/apache/metron/rest/service/impl/SearchServiceImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/SearchServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/SearchServiceImpl.java index e38fa02ac8..bdf6037bb6 100644 --- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/SearchServiceImpl.java +++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/SearchServiceImpl.java @@ -42,7 +42,7 @@ public class SearchServiceImpl implements SearchService { private Environment environment; @Autowired - public IndexDaoSearchServiceImpl(IndexDao dao, Environment environment) { + public SearchServiceImpl(IndexDao dao, Environment environment) { this.dao = dao; this.environment = environment; }