You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by le...@apache.org on 2018/05/23 14:34:28 UTC

[1/7] metron git commit: METRON-1421 Create a SolrMetaAlertDao (justinleet) closes apache/metron#970

Repository: metron
Updated Branches:
  refs/heads/feature/METRON-1416-upgrade-solr eb3366656 -> 49f851e0b


http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java
index 1cf8c34..4bc9f8a 100644
--- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/components/SolrComponent.java
@@ -19,9 +19,13 @@ package org.apache.metron.solr.integration.components;
 
 import com.google.common.base.Function;
 import java.util.Collection;
+import java.util.Map.Entry;
 import java.util.stream.Collectors;
+import org.apache.metron.common.Constants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
 import org.apache.metron.integration.InMemoryComponent;
 import org.apache.metron.integration.UnableToStartException;
+import org.apache.metron.solr.dao.SolrUtilities;
 import org.apache.metron.solr.writer.MetronSolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -29,6 +33,7 @@ import org.apache.solr.client.solrj.embedded.JettyConfig;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.cloud.MiniSolrCloudCluster;
 import org.apache.solr.common.SolrDocument;
 
@@ -40,13 +45,15 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.zookeeper.KeeperException;
 
 public class SolrComponent implements InMemoryComponent {
 
   public static class Builder {
+
     private int port = 8983;
     private String solrXmlPath = "../metron-solr/src/test/resources/solr/solr.xml";
-    private Map<String, String> collections = new HashMap<>();
+    private Map<String, String> initialCollections = new HashMap<>();
     private Function<SolrComponent, Void> postStartCallback;
 
     public Builder withPort(int port) {
@@ -59,8 +66,8 @@ public class SolrComponent implements InMemoryComponent {
       return this;
     }
 
-    public Builder addCollection(String name, String configPath) {
-      collections.put(name, configPath);
+    public Builder addInitialCollection(String name, String configPath) {
+      initialCollections.put(name, configPath);
       return this;
     }
 
@@ -69,9 +76,8 @@ public class SolrComponent implements InMemoryComponent {
       return this;
     }
 
-    public SolrComponent build() throws Exception {
-      if (collections.isEmpty()) throw new Exception("Must add at least 1 collection");
-      return new SolrComponent(port, solrXmlPath, collections, postStartCallback);
+    public SolrComponent build() {
+      return new SolrComponent(port, solrXmlPath, initialCollections, postStartCallback);
     }
   }
 
@@ -81,7 +87,8 @@ public class SolrComponent implements InMemoryComponent {
   private MiniSolrCloudCluster miniSolrCloudCluster;
   private Function<SolrComponent, Void> postStartCallback;
 
-  private SolrComponent(int port, String solrXmlPath, Map<String, String> collections, Function<SolrComponent, Void> postStartCallback) throws Exception {
+  private SolrComponent(int port, String solrXmlPath, Map<String, String> collections,
+      Function<SolrComponent, Void> postStartCallback) {
     this.port = port;
     this.solrXmlPath = solrXmlPath;
     this.collections = collections;
@@ -93,14 +100,17 @@ public class SolrComponent implements InMemoryComponent {
     try {
       File baseDir = Files.createTempDirectory("solrcomponent").toFile();
       baseDir.deleteOnExit();
-      miniSolrCloudCluster = new MiniSolrCloudCluster(1, baseDir.toPath(), JettyConfig.builder().setPort(port).build());
+      miniSolrCloudCluster = new MiniSolrCloudCluster(1, baseDir.toPath(),
+          JettyConfig.builder().setPort(port).build());
       for(String name: collections.keySet()) {
         String configPath = collections.get(name);
         miniSolrCloudCluster.uploadConfigSet(new File(configPath).toPath(), name);
         CollectionAdminRequest.createCollection(name, 1, 1).process(miniSolrCloudCluster.getSolrClient());
       }
-      if (postStartCallback != null) postStartCallback.apply(this);
-    } catch(Exception e) {
+      if (postStartCallback != null) {
+        postStartCallback.apply(this);
+      }
+    } catch (Exception e) {
       throw new UnableToStartException(e.getMessage(), e);
     }
   }
@@ -108,21 +118,18 @@ public class SolrComponent implements InMemoryComponent {
   @Override
   public void stop() {
     try {
+      miniSolrCloudCluster.deleteAllCollections();
       miniSolrCloudCluster.shutdown();
     } catch (Exception e) {
+      // Do nothing
     }
   }
 
   @Override
   public void reset() {
     try {
-      MetronSolrClient solr = getSolrClient();
-      for (String collection: collections.keySet()) {
-        solr.setDefaultCollection(collection);
-        solr.deleteByQuery("*:*");
-      }
-      solr.commit();
-    } catch (SolrServerException | IOException e) {
+      miniSolrCloudCluster.deleteAllCollections();
+    } catch (Exception e) {
       // Do nothing
     }
   }
@@ -139,12 +146,19 @@ public class SolrComponent implements InMemoryComponent {
     return miniSolrCloudCluster.getZkServer().getZkAddress();
   }
 
+  public void addCollection(String name, String configPath)
+      throws InterruptedException, IOException, KeeperException, SolrServerException {
+    miniSolrCloudCluster.uploadConfigSet(new File(configPath).toPath(), name);
+    CollectionAdminRequest.createCollection(name, 1, 1)
+        .process(miniSolrCloudCluster.getSolrClient());
+  }
+
   public boolean hasCollection(String collection) {
     MetronSolrClient solr = getSolrClient();
     boolean collectionFound = false;
     try {
       collectionFound = solr.listCollections().contains(collection);
-    } catch(Exception e) {
+    } catch (Exception e) {
       e.printStackTrace();
     }
     return collectionFound;
@@ -155,12 +169,21 @@ public class SolrComponent implements InMemoryComponent {
     CloudSolrClient solr = miniSolrCloudCluster.getSolrClient();
     solr.setDefaultCollection(collection);
     SolrQuery parameters = new SolrQuery();
-    parameters.set("q", "*:*");
+
+    // If it's metaalert, we need to adjust the query. We want child docs with the parent,
+    // not separate.
+    if (collection.equals("metaalert")) {
+      parameters.setQuery("source.type:metaalert")
+          .setFields("*", "[child parentFilter=source.type:metaalert limit=999]");
+    } else {
+      parameters.set("q", "*:*");
+    }
     try {
       solr.commit();
       QueryResponse response = solr.query(parameters);
       for (SolrDocument solrDocument : response.getResults()) {
-        docs.add(solrDocument);
+        // Use the utils to make sure we get child docs.
+        docs.add(SolrUtilities.toDocument(solrDocument).getDocument());
       }
     } catch (SolrServerException | IOException e) {
       e.printStackTrace();
@@ -174,9 +197,36 @@ public class SolrComponent implements InMemoryComponent {
     solr.setDefaultCollection(collection);
     Collection<SolrInputDocument> solrInputDocuments = docs.stream().map(doc -> {
       SolrInputDocument solrInputDocument = new SolrInputDocument();
-      doc.forEach(solrInputDocument::addField);
+      for (Entry<String, Object> entry : doc.entrySet()) {
+        // If the entry itself is a map, add it as a child document. Handle one level of nesting.
+        if (entry.getValue() instanceof List && !entry.getKey().equals(
+            MetaAlertConstants.METAALERT_FIELD)) {
+          for (Object entryItem : (List)entry.getValue()) {
+            if (entryItem instanceof Map) {
+              @SuppressWarnings("unchecked")
+              Map<String, Object> childDoc = (Map<String, Object>) entryItem;
+              SolrInputDocument childInputDoc = new SolrInputDocument();
+              for (Entry<String, Object> childEntry : childDoc.entrySet()) {
+                childInputDoc.addField(childEntry.getKey(), childEntry.getValue());
+              }
+              solrInputDocument.addChildDocument(childInputDoc);
+            }
+          }
+        } else {
+          solrInputDocument.addField(entry.getKey(), entry.getValue());
+        }
+      }
       return solrInputDocument;
     }).collect(Collectors.toList());
-    solr.add(collection, solrInputDocuments);
+
+    checkUpdateResponse(solr.add(collection, solrInputDocuments));
+    // Make sure to commit so things show up
+    checkUpdateResponse(solr.commit(true, true));
+  }
+
+  protected void checkUpdateResponse(UpdateResponse result) throws IOException {
+    if (result.getStatus() != 0) {
+      throw new IOException("Response error received while adding documents: " + result);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/schema/SchemaValidationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/schema/SchemaValidationIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/schema/SchemaValidationIntegrationTest.java
index e655428..73113f6 100644
--- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/schema/SchemaValidationIntegrationTest.java
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/schema/SchemaValidationIntegrationTest.java
@@ -25,9 +25,12 @@ import org.apache.metron.common.writer.BulkWriterResponse;
 import org.apache.metron.solr.integration.components.SolrComponent;
 import org.apache.metron.solr.writer.SolrWriter;
 import org.apache.metron.stellar.common.utils.ConversionUtils;
+import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.storm.tuple.Tuple;
+import org.apache.zookeeper.KeeperException;
 import org.json.simple.JSONObject;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mock;
 import java.io.File;
@@ -38,7 +41,6 @@ import java.util.*;
 import static org.mockito.Mockito.mock;
 
 public class SchemaValidationIntegrationTest {
-
   public static Iterable<String> getData(String sensor) throws IOException {
     return Iterables.filter(
             Files.readLines(new File("src/test/resources/example_data/" + sensor), Charset.defaultCharset()),
@@ -53,9 +55,7 @@ public class SchemaValidationIntegrationTest {
   }
 
   public static SolrComponent createSolrComponent(String sensor) throws Exception {
-    return new SolrComponent.Builder()
-            .addCollection(String.format("%s", sensor), String.format("src/main/config/schema/%s", sensor))
-            .build();
+    return new SolrComponent.Builder().build();
   }
 
   @Test
@@ -92,6 +92,7 @@ public class SchemaValidationIntegrationTest {
     try {
       component = createSolrComponent(sensorType);
       component.start();
+      component.addCollection(String.format("%s", sensorType), String.format("src/main/config/schema/%s", sensorType));
       Map<String, Object> globalConfig = getGlobalConfig(sensorType, component);
 
       List<JSONObject> inputs = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/writer/SolrWriterTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/writer/SolrWriterTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/writer/SolrWriterTest.java
index 685c5fd..7b7d208 100644
--- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/writer/SolrWriterTest.java
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/writer/SolrWriterTest.java
@@ -17,11 +17,19 @@
  */
 package org.apache.metron.solr.writer;
 
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.argThat;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import org.apache.metron.common.Constants;
-import org.apache.metron.common.configuration.EnrichmentConfigurations;
 import org.apache.metron.common.configuration.IndexingConfigurations;
 import org.apache.metron.common.configuration.writer.IndexingWriterConfiguration;
 import org.apache.metron.enrichment.integration.utils.SampleUtil;
@@ -34,17 +42,6 @@ import org.junit.Test;
 import org.mockito.ArgumentMatcher;
 import org.mockito.Mockito;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.argThat;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-
 
 public class SolrWriterTest {
 

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/resources/config/test/conf/managed-schema
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/resources/config/test/conf/managed-schema b/metron-platform/metron-solr/src/test/resources/config/test/conf/managed-schema
index 85d6915..bb2de59 100644
--- a/metron-platform/metron-solr/src/test/resources/config/test/conf/managed-schema
+++ b/metron-platform/metron-solr/src/test/resources/config/test/conf/managed-schema
@@ -16,53 +16,59 @@
  limitations under the License.
 -->
 <schema name="example" version="1.6">
-   <field name="_version_" type="plong" indexed="false" stored="false"/>
+  <field name="_version_" type="plong" indexed="false" stored="false"/>
 
-   <!-- points to the root document of a block of nested documents. Required for nested
-      document support, may be removed otherwise
-   -->
-   <field name="_root_" type="string" indexed="true" stored="false" docValues="false" />
+  <!-- points to the root document of a block of nested documents. Required for nested
+     document support, may be removed otherwise
+  -->
+  <field name="_root_" type="string" indexed="true" stored="false" docValues="false"/>
 
-   <!-- Only remove the "id" field if you have a very good reason to. While not strictly
-     required, it is highly recommended. A <uniqueKey> is present in almost all Solr
-     installations. See the <uniqueKey> declaration below where <uniqueKey> is set to "id".
-     Do NOT change the type and apply index-time analysis to the <uniqueKey> as it will likely
-     make routing in SolrCloud and document replacement in general fail. Limited _query_ time
-     analysis is possible as long as the indexing process is guaranteed to index the term
-     in a compatible way. Any analysis applied to the <uniqueKey> should _not_ produce multiple
-     tokens
-   -->
-   <field name="guid" type="string" indexed="true" stored="true" required="true" multiValued="false" />
+  <!-- Only remove the "id" field if you have a very good reason to. While not strictly
+    required, it is highly recommended. A <uniqueKey> is present in almost all Solr
+    installations. See the <uniqueKey> declaration below where <uniqueKey> is set to "id".
+    Do NOT change the type and apply index-time analysis to the <uniqueKey> as it will likely
+    make routing in SolrCloud and document replacement in general fail. Limited _query_ time
+    analysis is possible as long as the indexing process is guaranteed to index the term
+    in a compatible way. Any analysis applied to the <uniqueKey> should _not_ produce multiple
+    tokens
+  -->
+  <field name="guid" type="string" indexed="true" stored="true" required="true"
+    multiValued="false"/>
 
-   <field name="source:type" type="string" indexed="true" stored="true"/>
-   <field name="name" type="string" indexed="true" stored="true" />
-   <field name="timestamp" type="plong" indexed="true" stored="true" />
-   <field name="new-field" type="string" indexed="true" stored="true" />
+  <field name="source.type" type="string" indexed="true" stored="true"/>
+  <field name="name" type="string" indexed="true" stored="true"/>
+  <field name="timestamp" type="plong" indexed="true" stored="true"/>
+  <field name="new-field" type="string" indexed="true" stored="true"/>
+  <field name="metaalerts" type="string" multiValued="true" indexed="true" stored="true"/>
+  <field name="threat:triage:score" type="pdouble" indexed="true" stored="true"/>
+  <field name="score" type="pdouble" indexed="true" stored="true"/>
 
-   <dynamicField name="*" type="ignored" multiValued="false" docValues="true"/>
 
+  <dynamicField name="*" type="ignored" multiValued="false" docValues="true"/>
 
- <!-- Field to use to determine and enforce document uniqueness. 
-      Unless this field is marked with required="false", it will be a required field
-   -->
-    <uniqueKey>guid</uniqueKey>
 
-
-    <!-- field type definitions. The "name" attribute is
-       just a label to be used by field definitions.  The "class"
-       attribute and any other attributes determine the real
-       behavior of the fieldType.
-         Class names starting with "solr" refer to java classes in a
-       standard package such as org.apache.solr.analysis
+  <!-- Field to use to determine and enforce document uniqueness.
+       Unless this field is marked with required="false", it will be a required field
     -->
+  <uniqueKey>guid</uniqueKey>
+
+
+  <!-- field type definitions. The "name" attribute is
+     just a label to be used by field definitions.  The "class"
+     attribute and any other attributes determine the real
+     behavior of the fieldType.
+       Class names starting with "solr" refer to java classes in a
+     standard package such as org.apache.solr.analysis
+  -->
 
-    <!-- The StrField type is not analyzed, but indexed/stored verbatim. -->
-    <fieldType name="string" class="solr.StrField" sortMissingLast="true" />
-    <fieldType name="boolean" class="solr.BoolField" sortMissingLast="true"/>
-    <fieldType name="pint" class="solr.IntPointField" docValues="true"/>
-    <fieldType name="pfloat" class="solr.FloatPointField" docValues="true"/>
-    <fieldType name="plong" class="solr.LongPointField" docValues="true"/>
-    <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
-    <fieldType name="ignored" stored="false" indexed="false" multiValued="true" class="solr.StrField" />
+  <!-- The StrField type is not analyzed, but indexed/stored verbatim. -->
+  <fieldType name="string" class="solr.StrField" sortMissingLast="true"/>
+  <fieldType name="boolean" class="solr.BoolField" sortMissingLast="true"/>
+  <fieldType name="pint" class="solr.IntPointField" docValues="true"/>
+  <fieldType name="pfloat" class="solr.FloatPointField" docValues="true"/>
+  <fieldType name="plong" class="solr.LongPointField" docValues="true"/>
+  <fieldType name="pdouble" class="solr.DoublePointField" docValues="true"/>
+  <fieldType name="ignored" stored="false" indexed="false" multiValued="true"
+    class="solr.StrField"/>
 
 </schema>


[4/7] metron git commit: METRON-1421 Create a SolrMetaAlertDao (justinleet) closes apache/metron#970

Posted by le...@apache.org.
http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/SearchIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/SearchIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/SearchIntegrationTest.java
index 56406f4..7fca764 100644
--- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/SearchIntegrationTest.java
+++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/SearchIntegrationTest.java
@@ -39,8 +39,6 @@ import org.apache.metron.indexing.dao.update.Document;
 import org.apache.metron.integration.InMemoryComponent;
 import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -472,25 +470,15 @@ public abstract class SearchIntegrationTest {
   @Multiline
   public static String differentTypeFilterQuery;
 
-  protected static IndexDao dao;
   protected static InMemoryComponent indexComponent;
 
-  @Before
-  public synchronized void setup() throws Exception {
-    if(dao == null && indexComponent == null) {
-      indexComponent = startIndex();
-      loadTestData();
-      dao = createDao();
-    }
-  }
-
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   @Test
   public void all_query_returns_all_results() throws Exception {
     SearchRequest request = JSONUtils.INSTANCE.load(allQuery, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertEquals(10, response.getTotal());
     List<SearchResult> results = response.getResults();
     Assert.assertEquals(10, results.size());
@@ -507,7 +495,7 @@ public abstract class SearchIntegrationTest {
   @Test
   public void find_one_guid() throws Exception {
     GetRequest request = JSONUtils.INSTANCE.load(findOneGuidQuery, GetRequest.class);
-    Optional<Map<String, Object>> response = dao.getLatestResult(request);
+    Optional<Map<String, Object>> response = getIndexDao().getLatestResult(request);
     Assert.assertTrue(response.isPresent());
     Map<String, Object> doc = response.get();
     Assert.assertEquals("bro", doc.get(getSourceTypeField()));
@@ -519,7 +507,7 @@ public abstract class SearchIntegrationTest {
     List<GetRequest> request = JSONUtils.INSTANCE.load(getAllLatestQuery, new JSONUtils.ReferenceSupplier<List<GetRequest>>(){});
     Map<String, Document> docs = new HashMap<>();
 
-    for(Document doc : dao.getAllLatest(request)) {
+    for(Document doc : getIndexDao().getAllLatest(request)) {
       docs.put(doc.getGuid(), doc);
     }
     Assert.assertEquals(2, docs.size());
@@ -532,7 +520,7 @@ public abstract class SearchIntegrationTest {
   @Test
   public void filter_query_filters_results() throws Exception {
     SearchRequest request = JSONUtils.INSTANCE.load(filterQuery, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertEquals(3, response.getTotal());
     List<SearchResult> results = response.getResults();
     Assert.assertEquals("snort", results.get(0).getSource().get(getSourceTypeField()));
@@ -546,7 +534,7 @@ public abstract class SearchIntegrationTest {
   @Test
   public void sort_query_sorts_results_ascending() throws Exception {
     SearchRequest request = JSONUtils.INSTANCE.load(sortQuery, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertEquals(10, response.getTotal());
     List<SearchResult> results = response.getResults();
     for (int i = 8001; i < 8011; ++i) {
@@ -557,7 +545,7 @@ public abstract class SearchIntegrationTest {
   @Test
   public void sort_ascending_with_missing_fields() throws Exception {
     SearchRequest request = JSONUtils.INSTANCE.load(sortAscendingWithMissingFields, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertEquals(10, response.getTotal());
     List<SearchResult> results = response.getResults();
     Assert.assertEquals(10, results.size());
@@ -575,7 +563,7 @@ public abstract class SearchIntegrationTest {
   @Test
   public void sort_descending_with_missing_fields() throws Exception {
     SearchRequest request = JSONUtils.INSTANCE.load(sortDescendingWithMissingFields, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertEquals(10, response.getTotal());
     List<SearchResult> results = response.getResults();
     Assert.assertEquals(10, results.size());
@@ -593,7 +581,7 @@ public abstract class SearchIntegrationTest {
   @Test
   public void results_are_paginated() throws Exception {
     SearchRequest request = JSONUtils.INSTANCE.load(paginationQuery, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertEquals(10, response.getTotal());
     List<SearchResult> results = response.getResults();
     Assert.assertEquals(3, results.size());
@@ -608,7 +596,7 @@ public abstract class SearchIntegrationTest {
   @Test
   public void returns_results_only_for_specified_indices() throws Exception {
     SearchRequest request = JSONUtils.INSTANCE.load(indexQuery, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertEquals(5, response.getTotal());
     List<SearchResult> results = response.getResults();
     for (int i = 5, j = 0; i > 0; i--, j++) {
@@ -621,7 +609,7 @@ public abstract class SearchIntegrationTest {
   public void facet_query_yields_field_types() throws Exception {
     String facetQuery = facetQueryRaw.replace("source:type", getSourceTypeField());
     SearchRequest request = JSONUtils.INSTANCE.load(facetQuery, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertEquals(10, response.getTotal());
     Map<String, Map<String, Long>> facetCounts = response.getFacetCounts();
     Assert.assertEquals(8, facetCounts.size());
@@ -696,14 +684,14 @@ public abstract class SearchIntegrationTest {
   @Test
   public void disabled_facet_query_returns_null_count() throws Exception {
     SearchRequest request = JSONUtils.INSTANCE.load(disabledFacetQuery, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertNull(response.getFacetCounts());
   }
 
   @Test
   public void missing_type_facet_query() throws Exception {
     SearchRequest request = JSONUtils.INSTANCE.load(missingTypeFacetQuery, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertEquals(10, response.getTotal());
 
     Map<String, Map<String, Long>> facetCounts = response.getFacetCounts();
@@ -723,7 +711,7 @@ public abstract class SearchIntegrationTest {
   public void different_type_facet_query() throws Exception {
     thrown.expect(Exception.class);
     SearchRequest request = JSONUtils.INSTANCE.load(differentTypeFacetQuery, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertEquals(3, response.getTotal());
   }
 
@@ -732,14 +720,14 @@ public abstract class SearchIntegrationTest {
     thrown.expect(InvalidSearchException.class);
     thrown.expectMessage("Search result size must be less than 100");
     SearchRequest request = JSONUtils.INSTANCE.load(exceededMaxResultsQuery, SearchRequest.class);
-    dao.search(request);
+    getIndexDao().search(request);
   }
 
   @Test
   public void column_metadata_for_missing_index() throws Exception {
     // getColumnMetadata with an index that doesn't exist
     {
-      Map<String, FieldType> fieldTypes = dao.getColumnMetadata(Collections.singletonList("someindex"));
+      Map<String, FieldType> fieldTypes = getIndexDao().getColumnMetadata(Collections.singletonList("someindex"));
       Assert.assertEquals(0, fieldTypes.size());
     }
   }
@@ -747,14 +735,14 @@ public abstract class SearchIntegrationTest {
   @Test
   public void no_results_returned_when_query_does_not_match() throws Exception {
     SearchRequest request = JSONUtils.INSTANCE.load(noResultsFieldsQuery, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertEquals(0, response.getTotal());
   }
 
   @Test
   public void group_by_ip_query() throws Exception {
     GroupRequest request = JSONUtils.INSTANCE.load(groupByIpQuery, GroupRequest.class);
-    GroupResponse response = dao.group(request);
+    GroupResponse response = getIndexDao().group(request);
 
     // expect only 1 group for 'ip_src_addr'
     Assert.assertEquals("ip_src_addr", response.getGroupedBy());
@@ -778,7 +766,7 @@ public abstract class SearchIntegrationTest {
   public void group_by_returns_results_in_groups() throws Exception {
     // Group by test case, default order is count descending
     GroupRequest request = JSONUtils.INSTANCE.load(groupByQuery, GroupRequest.class);
-    GroupResponse response = dao.group(request);
+    GroupResponse response = getIndexDao().group(request);
     Assert.assertEquals("is_alert", response.getGroupedBy());
     List<GroupResult> isAlertGroups = response.getGroupResults();
     Assert.assertEquals(2, isAlertGroups.size());
@@ -830,7 +818,7 @@ public abstract class SearchIntegrationTest {
   public void group_by_returns_results_in_sorted_groups() throws Exception {
     // Group by with sorting test case where is_alert is sorted by count ascending and ip_src_addr is sorted by term descending
     GroupRequest request = JSONUtils.INSTANCE.load(sortedGroupByQuery, GroupRequest.class);
-    GroupResponse response = dao.group(request);
+    GroupResponse response = getIndexDao().group(request);
     Assert.assertEquals("is_alert", response.getGroupedBy());
     List<GroupResult> isAlertGroups = response.getGroupResults();
     Assert.assertEquals(2, isAlertGroups.size());
@@ -909,7 +897,7 @@ public abstract class SearchIntegrationTest {
   @Test
   public void queries_fields() throws Exception {
     SearchRequest request = JSONUtils.INSTANCE.load(fieldsQuery, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertEquals(10, response.getTotal());
     List<SearchResult> results = response.getResults();
     for (int i = 0; i < 5; ++i) {
@@ -927,7 +915,7 @@ public abstract class SearchIntegrationTest {
   @Test
   public void sort_by_guid() throws Exception {
     SearchRequest request = JSONUtils.INSTANCE.load(sortByGuidQuery, SearchRequest.class);
-    SearchResponse response = dao.search(request);
+    SearchResponse response = getIndexDao().search(request);
     Assert.assertEquals(5, response.getTotal());
     List<SearchResult> results = response.getResults();
     for (int i = 0; i < 5; ++i) {
@@ -938,7 +926,7 @@ public abstract class SearchIntegrationTest {
   }
 
   @AfterClass
-  public static void stop() throws Exception {
+  public static void stop() {
     indexComponent.stop();
   }
 
@@ -949,9 +937,7 @@ public abstract class SearchIntegrationTest {
   @Test
   public abstract void different_type_filter_query() throws Exception;
 
+  protected abstract IndexDao getIndexDao();
 
-  protected abstract IndexDao createDao() throws Exception;
-  protected abstract InMemoryComponent startIndex() throws Exception;
-  protected abstract void loadTestData() throws Exception;
   protected abstract String getSourceTypeField();
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java
index 471acf6..eebf0bb 100644
--- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java
+++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/UpdateIntegrationTest.java
@@ -20,86 +20,43 @@ import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.Optional;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.metron.common.Constants;
 import org.apache.metron.common.utils.JSONUtils;
-import org.apache.metron.hbase.mock.MockHBaseTableProvider;
 import org.apache.metron.hbase.mock.MockHTable;
 import org.apache.metron.indexing.dao.update.Document;
 import org.apache.metron.indexing.dao.update.ReplaceRequest;
-import org.apache.metron.integration.InMemoryComponent;
-import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-
 public abstract class UpdateIntegrationTest {
 
   private static final int MAX_RETRIES = 10;
   private static final int SLEEP_MS = 500;
-  protected static final String SENSOR_NAME= "test";
-  private static final String TABLE_NAME = "modifications";
+  protected static final String SENSOR_NAME = "test";
   private static final String CF = "p";
-  private static String index;
-  private static MockHTable table;
-  private static IndexDao hbaseDao;
-  private static AccessConfig accessConfig;
 
   protected static MultiIndexDao dao;
-  protected static InMemoryComponent indexComponent;
-
-  @Before
-  public void setup() throws Exception {
-    if(dao == null && indexComponent == null) {
-      index = getIndexName();
-      indexComponent = startIndex();
-      loadTestData();
-      Configuration config = HBaseConfiguration.create();
-      MockHBaseTableProvider tableProvider = new MockHBaseTableProvider();
-      tableProvider.addToCache(TABLE_NAME, CF);
-      table = (MockHTable)tableProvider.getTable(config, TABLE_NAME);
-
-      hbaseDao = new HBaseDao();
-      accessConfig = new AccessConfig();
-      accessConfig.setTableProvider(tableProvider);
-      Map<String, Object> globalConfig = createGlobalConfig();
-      globalConfig.put(HBaseDao.HBASE_TABLE, TABLE_NAME);
-      globalConfig.put(HBaseDao.HBASE_CF, CF);
-      accessConfig.setGlobalConfigSupplier(() -> globalConfig);
-    }
-  }
-
-  protected AccessConfig getAccessConfig() {
-    return accessConfig;
-  }
 
   @Test
   public void test() throws Exception {
-    dao = new MultiIndexDao(hbaseDao, createDao());
-    dao.init(getAccessConfig());
-
     List<Map<String, Object>> inputData = new ArrayList<>();
     for(int i = 0; i < 10;++i) {
       final String name = "message" + i;
       inputData.add(
           new HashMap<String, Object>() {{
-            put("source:type", SENSOR_NAME);
+            put("source.type", SENSOR_NAME);
             put("name" , name);
             put("timestamp", System.currentTimeMillis());
             put(Constants.GUID, name);
           }}
       );
     }
-    addTestData(index, SENSOR_NAME, inputData);
+    addTestData(getIndexName(), SENSOR_NAME, inputData);
     List<Map<String,Object>> docs = null;
     for(int t = 0;t < MAX_RETRIES;++t, Thread.sleep(SLEEP_MS)) {
-      docs = getIndexedTestData(index, SENSOR_NAME);
+      docs = getIndexedTestData(getIndexName(), SENSOR_NAME);
       if(docs.size() >= 10) {
         break;
       }
@@ -115,16 +72,16 @@ public abstract class UpdateIntegrationTest {
         setReplacement(message0);
         setGuid(guid);
         setSensorType(SENSOR_NAME);
-        setIndex(index);
+        setIndex(getIndexName());
       }}, Optional.empty());
 
-      Assert.assertEquals(1, table.size());
+      Assert.assertEquals(1, getMockHTable().size());
       Document doc = dao.getLatest(guid, SENSOR_NAME);
       Assert.assertEquals(message0, doc.getDocument());
       {
         //ensure hbase is up to date
         Get g = new Get(HBaseDao.Key.toBytes(new HBaseDao.Key(guid, SENSOR_NAME)));
-        Result r = table.get(g);
+        Result r = getMockHTable().get(g);
         NavigableMap<byte[], byte[]> columns = r.getFamilyMap(CF.getBytes());
         Assert.assertEquals(1, columns.size());
         Assert.assertEquals(message0
@@ -136,7 +93,7 @@ public abstract class UpdateIntegrationTest {
         //ensure ES is up-to-date
         long cnt = 0;
         for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t, Thread.sleep(SLEEP_MS)) {
-          docs = getIndexedTestData(index, SENSOR_NAME);
+          docs = getIndexedTestData(getIndexName(), SENSOR_NAME);
           cnt = docs
               .stream()
               .filter(d -> message0.get("new-field").equals(d.get("new-field")))
@@ -155,15 +112,15 @@ public abstract class UpdateIntegrationTest {
         setReplacement(message0);
         setGuid(guid);
         setSensorType(SENSOR_NAME);
-        setIndex(index);
+        setIndex(getIndexName());
       }}, Optional.empty());
-      Assert.assertEquals(1, table.size());
+      Assert.assertEquals(1, getMockHTable().size());
       Document doc = dao.getLatest(guid, SENSOR_NAME);
       Assert.assertEquals(message0, doc.getDocument());
       {
         //ensure hbase is up to date
         Get g = new Get(HBaseDao.Key.toBytes(new HBaseDao.Key(guid, SENSOR_NAME)));
-        Result r = table.get(g);
+        Result r = getMockHTable().get(g);
         NavigableMap<byte[], byte[]> columns = r.getFamilyMap(CF.getBytes());
         Assert.assertEquals(2, columns.size());
         Assert.assertEquals(message0, JSONUtils.INSTANCE.load(new String(columns.lastEntry().getValue())
@@ -177,36 +134,20 @@ public abstract class UpdateIntegrationTest {
         //ensure ES is up-to-date
         long cnt = 0;
         for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t,Thread.sleep(SLEEP_MS)) {
-          docs = getIndexedTestData(index, SENSOR_NAME);
+          docs = getIndexedTestData(getIndexName(), SENSOR_NAME);
           cnt = docs
               .stream()
               .filter(d -> message0.get("new-field").equals(d.get("new-field")))
               .count();
         }
 
-        Assert.assertNotEquals("Elasticsearch is not updated!", cnt, 0);
+        Assert.assertNotEquals("Index is not updated!", cnt, 0);
       }
     }
   }
 
-  @After
-  public void reset() throws Exception {
-    indexComponent.reset();
-  }
-
-  @AfterClass
-  public static void teardown() {
-    if(indexComponent != null) {
-      indexComponent.stop();
-    }
-  }
-
   protected abstract String getIndexName();
-  protected abstract Map<String, Object> createGlobalConfig() throws Exception;
-  protected abstract IndexDao createDao() throws Exception;
-  protected abstract InMemoryComponent startIndex() throws Exception;
-  protected abstract void loadTestData() throws Exception;
+  protected abstract MockHTable getMockHTable();
   protected abstract void addTestData(String indexName, String sensorType, List<Map<String,Object>> docs) throws Exception;
   protected abstract List<Map<String,Object>> getIndexedTestData(String indexName, String sensorType) throws Exception;
-
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java
new file mode 100644
index 0000000..b4f7d38
--- /dev/null
+++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaAlertIntegrationTest.java
@@ -0,0 +1,1012 @@
+/*
+ * 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.metaalert;
+
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.ALERT_FIELD;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_FIELD;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_TYPE;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.STATUS_FIELD;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.THREAT_FIELD_DEFAULT;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.utils.JSONUtils;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.search.Group;
+import org.apache.metron.indexing.dao.search.GroupRequest;
+import org.apache.metron.indexing.dao.search.GroupResponse;
+import org.apache.metron.indexing.dao.search.GroupResult;
+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.search.SearchResult;
+import org.apache.metron.indexing.dao.search.SortField;
+import org.apache.metron.indexing.dao.update.Document;
+import org.apache.metron.indexing.dao.update.OriginalNotFoundException;
+import org.apache.metron.indexing.dao.update.PatchRequest;
+import org.junit.Assert;
+import org.junit.Test;
+
+public abstract class MetaAlertIntegrationTest {
+
+  private static final String META_INDEX_FLAG = "%META_INDEX%";
+  // To change back after testing
+  protected static int MAX_RETRIES = 10;
+  protected static final int SLEEP_MS = 500;
+  protected static final String SENSOR_NAME = "test";
+
+  protected static final String NEW_FIELD = "new-field";
+  protected static final String NAME_FIELD = "name";
+  protected static final String DATE_FORMAT = "yyyy.MM.dd.HH";
+
+  // Separate the raw indices from the query indices. ES for example, modifies the indices to
+  // have a separator
+  protected ArrayList<String> allIndices = new ArrayList<String>() {
+    {
+      add(getTestIndexName());
+      add(getMetaAlertIndex());
+    }
+  };
+
+  protected ArrayList<String> queryIndices = allIndices;
+
+  protected static MetaAlertDao metaDao;
+
+  /**
+   {
+   "guid": "meta_alert",
+   "index": "%META_INDEX%",
+   "patch": [
+   {
+   "op": "add",
+   "path": "/name",
+   "value": "New Meta Alert"
+   }
+   ],
+   "sensorType": "metaalert"
+   }
+   */
+  @Multiline
+  public static String namePatchRequest;
+
+  /**
+   {
+   "guid": "meta_alert",
+   "index": "%META_INDEX%",
+   "patch": [
+   {
+   "op": "add",
+   "path": "/name",
+   "value": "New Meta Alert"
+   },
+   {
+   "op": "add",
+   "path": "/alert",
+   "value": []
+   }
+   ],
+   "sensorType": "metaalert"
+   }
+   */
+  @Multiline
+  public static String alertPatchRequest;
+
+  /**
+   {
+   "guid": "meta_alert",
+   "index": "%META_INDEX%",
+   "patch": [
+   {
+   "op": "add",
+   "path": "/status",
+   "value": "inactive"
+   },
+   {
+   "op": "add",
+   "path": "/name",
+   "value": "New Meta Alert"
+   }
+   ],
+   "sensorType": "metaalert"
+   }
+   */
+  @Multiline
+  public static String statusPatchRequest;
+
+
+  @Test
+  public void shouldGetAllMetaAlertsForAlert() throws Exception {
+    // Load alerts
+    List<Map<String, Object>> alerts = buildAlerts(3);
+    addRecords(alerts, getTestIndexFullName(), SENSOR_NAME);
+
+    // Load metaAlerts
+    List<Map<String, Object>> metaAlerts = buildMetaAlerts(12, MetaAlertStatus.ACTIVE,
+        Optional.of(Collections.singletonList(alerts.get(0))));
+    metaAlerts.add(buildMetaAlert("meta_active_12", MetaAlertStatus.ACTIVE,
+        Optional.of(Arrays.asList(alerts.get(0), alerts.get(2)))));
+    metaAlerts.add(buildMetaAlert("meta_inactive", MetaAlertStatus.INACTIVE,
+        Optional.of(Arrays.asList(alerts.get(0), alerts.get(2)))));
+    // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
+    addRecords(metaAlerts, getMetaAlertIndex(), METAALERT_TYPE);
+
+    // Verify load was successful
+    List<GetRequest> createdDocs = metaAlerts.stream().map(metaAlert ->
+        new GetRequest((String) metaAlert.get(Constants.GUID), METAALERT_TYPE))
+        .collect(Collectors.toList());
+    createdDocs.addAll(alerts.stream().map(alert ->
+        new GetRequest((String) alert.get(Constants.GUID), SENSOR_NAME))
+        .collect(Collectors.toList()));
+    findCreatedDocs(createdDocs);
+
+    {
+      // Verify searches successfully return more than 10 results
+      SearchResponse searchResponse0 = metaDao.getAllMetaAlertsForAlert("message_0");
+      List<SearchResult> searchResults0 = searchResponse0.getResults();
+      Assert.assertEquals(13, searchResults0.size());
+      Set<Map<String, Object>> resultSet = new HashSet<>();
+      Iterables.addAll(resultSet, Iterables.transform(searchResults0, r -> r.getSource()));
+      StringBuffer reason = new StringBuffer("Unable to find " + metaAlerts.get(0) + "\n");
+      reason.append(Joiner.on("\n").join(resultSet));
+      Assert.assertTrue(reason.toString(), resultSet.contains(metaAlerts.get(0)));
+
+      // Verify no meta alerts are returned because message_1 was not added to any
+      SearchResponse searchResponse1 = metaDao.getAllMetaAlertsForAlert("message_1");
+      List<SearchResult> searchResults1 = searchResponse1.getResults();
+      Assert.assertEquals(0, searchResults1.size());
+
+      // Verify only the meta alert message_2 was added to is returned
+      SearchResponse searchResponse2 = metaDao.getAllMetaAlertsForAlert("message_2");
+      List<SearchResult> searchResults2 = searchResponse2.getResults();
+      Assert.assertEquals(1, searchResults2.size());
+      Assert.assertEquals(metaAlerts.get(12), searchResults2.get(0).getSource());
+    }
+  }
+
+  @Test
+  public void getAllMetaAlertsForAlertShouldThrowExceptionForEmptyGuid() throws Exception {
+    try {
+      metaDao.getAllMetaAlertsForAlert("");
+      Assert.fail("An exception should be thrown for empty guid");
+    } catch (InvalidSearchException ise) {
+      Assert.assertEquals("Guid cannot be empty", ise.getMessage());
+    }
+  }
+
+  @Test
+  public void shouldCreateMetaAlert() throws Exception {
+    // Load alerts
+    List<Map<String, Object>> alerts = buildAlerts(3);
+    addRecords(alerts, getTestIndexFullName(), SENSOR_NAME);
+
+    // Verify load was successful
+    findCreatedDocs(Arrays.asList(
+        new GetRequest("message_0", SENSOR_NAME),
+        new GetRequest("message_1", SENSOR_NAME),
+        new GetRequest("message_2", SENSOR_NAME)));
+
+    {
+      MetaAlertCreateRequest metaAlertCreateRequest = new MetaAlertCreateRequest() {{
+        setAlerts(new ArrayList<GetRequest>() {{
+          add(new GetRequest("message_1", SENSOR_NAME));
+          add(new GetRequest("message_2", SENSOR_NAME, getTestIndexFullName()));
+        }});
+        setGroups(Collections.singletonList("group"));
+      }};
+      MetaAlertCreateResponse metaAlertCreateResponse = metaDao
+          .createMetaAlert(metaAlertCreateRequest);
+      {
+        // Verify metaAlert was created
+        findCreatedDoc(metaAlertCreateResponse.getGuid(), METAALERT_TYPE);
+      }
+      {
+        // Verify alert 0 was not updated with metaalert field
+        Document alert = metaDao.getLatest("message_0", SENSOR_NAME);
+        Assert.assertEquals(4, alert.getDocument().size());
+        Assert.assertNull(alert.getDocument().get(METAALERT_FIELD));
+      }
+      {
+        // Verify alert 1 was properly updated with metaalert field
+        Map<String, Object> expectedAlert = new HashMap<>(alerts.get(1));
+        expectedAlert
+            .put(METAALERT_FIELD, Collections.singletonList(metaAlertCreateResponse.getGuid()));
+        findUpdatedDoc(expectedAlert, "message_1", SENSOR_NAME);
+      }
+      {
+        // Verify alert 2 was properly updated with metaalert field
+        Map<String, Object> expectedAlert = new HashMap<>(alerts.get(2));
+        expectedAlert
+            .put(METAALERT_FIELD, Collections.singletonList(metaAlertCreateResponse.getGuid()));
+        findUpdatedDoc(expectedAlert, "message_2", SENSOR_NAME);
+      }
+    }
+  }
+
+  @Test
+  public void shouldAddAlertsToMetaAlert() throws Exception {
+    // Load alerts
+    List<Map<String, Object>> alerts = buildAlerts(4);
+    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
+    addRecords(alerts, getTestIndexFullName(), SENSOR_NAME);
+
+    // Load metaAlert
+    Map<String, Object> metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE,
+        Optional.of(Collections.singletonList(alerts.get(0))));
+    addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), METAALERT_TYPE);
+
+    // Verify load was successful
+    findCreatedDocs(Arrays.asList(
+        new GetRequest("message_0", SENSOR_NAME),
+        new GetRequest("message_1", SENSOR_NAME),
+        new GetRequest("message_2", SENSOR_NAME),
+        new GetRequest("message_3", SENSOR_NAME),
+        new GetRequest("meta_alert", METAALERT_TYPE)
+    ));
+
+    // Build expected metaAlert after alerts are added
+    Map<String, Object> expectedMetaAlert = new HashMap<>(metaAlert);
+
+    // Verify the proper alerts were added
+    @SuppressWarnings("unchecked")
+    List<Map<String, Object>> metaAlertAlerts = new ArrayList<>(
+        (List<Map<String, Object>>) expectedMetaAlert.get(ALERT_FIELD));
+    // Alert 0 is already in the metaalert. Add alerts 1 and 2.
+    Map<String, Object> expectedAlert1 = alerts.get(1);
+    expectedAlert1.put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
+    metaAlertAlerts.add(expectedAlert1);
+    Map<String, Object> expectedAlert2 = alerts.get(2);
+    expectedAlert2.put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
+    metaAlertAlerts.add(expectedAlert2);
+    expectedMetaAlert.put(ALERT_FIELD, metaAlertAlerts);
+
+    // Verify the counts were properly updated
+    expectedMetaAlert.put("average", 1.0d);
+    expectedMetaAlert.put("min", 0.0d);
+    expectedMetaAlert.put("median", 1.0d);
+    expectedMetaAlert.put("max", 2.0d);
+    expectedMetaAlert.put("count", 3);
+    expectedMetaAlert.put("sum", 3.0d);
+    expectedMetaAlert.put(getThreatTriageField(), 3.0d);
+
+    {
+      // Verify alerts were successfully added to the meta alert
+      Assert.assertTrue(metaDao.addAlertsToMetaAlert("meta_alert", Arrays
+          .asList(new GetRequest("message_1", SENSOR_NAME),
+              new GetRequest("message_2", SENSOR_NAME))));
+      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
+    }
+
+    {
+      // Verify False when alerts are already in a meta alert and no new alerts are added
+      Assert.assertFalse(metaDao.addAlertsToMetaAlert("meta_alert", Arrays
+          .asList(new GetRequest("message_0", SENSOR_NAME),
+              new GetRequest("message_1", SENSOR_NAME))));
+      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
+    }
+
+    {
+      // Verify only 1 alert is added when a list of alerts only contains 1 alert that is not in the meta alert
+      metaAlertAlerts = (List<Map<String, Object>>) expectedMetaAlert.get(ALERT_FIELD);
+      Map<String, Object> expectedAlert3 = alerts.get(3);
+      expectedAlert3.put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
+      metaAlertAlerts.add(expectedAlert3);
+      expectedMetaAlert.put(ALERT_FIELD, metaAlertAlerts);
+
+      expectedMetaAlert.put("average", 1.5d);
+      expectedMetaAlert.put("min", 0.0d);
+      expectedMetaAlert.put("median", 1.5d);
+      expectedMetaAlert.put("max", 3.0d);
+      expectedMetaAlert.put("count", 4);
+      expectedMetaAlert.put("sum", 6.0d);
+      expectedMetaAlert.put(getThreatTriageField(), 6.0d);
+
+      Assert.assertTrue(metaDao.addAlertsToMetaAlert("meta_alert", Arrays
+          .asList(new GetRequest("message_2", SENSOR_NAME),
+              new GetRequest("message_3", SENSOR_NAME))));
+      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
+    }
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void shouldRemoveAlertsFromMetaAlert() throws Exception {
+    // Load alerts
+    List<Map<String, Object>> alerts = buildAlerts(4);
+    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
+    alerts.get(1).put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
+    alerts.get(2).put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
+    alerts.get(3).put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
+    addRecords(alerts, getTestIndexFullName(), SENSOR_NAME);
+
+    // Load metaAlert
+    Map<String, Object> metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE,
+        Optional.of(Arrays.asList(alerts.get(0), alerts.get(1), alerts.get(2), alerts.get(3))));
+    addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), METAALERT_TYPE);
+
+    // Verify load was successful
+    findCreatedDocs(Arrays.asList(
+        new GetRequest("message_0", SENSOR_NAME),
+        new GetRequest("message_1", SENSOR_NAME),
+        new GetRequest("message_2", SENSOR_NAME),
+        new GetRequest("message_3", SENSOR_NAME),
+        new GetRequest("meta_alert", METAALERT_TYPE)));
+
+    // Build expected metaAlert after alerts are added
+    Map<String, Object> expectedMetaAlert = new HashMap<>(metaAlert);
+
+    // Verify the proper alerts were added
+    List<Map<String, Object>> metaAlertAlerts = new ArrayList<>(
+        (List<Map<String, Object>>) expectedMetaAlert.get(ALERT_FIELD));
+    metaAlertAlerts.remove(0);
+    metaAlertAlerts.remove(0);
+    expectedMetaAlert.put(ALERT_FIELD, metaAlertAlerts);
+
+    // Verify the counts were properly updated
+    expectedMetaAlert.put("average", 2.5d);
+    expectedMetaAlert.put("min", 2.0d);
+    expectedMetaAlert.put("median", 2.5d);
+    expectedMetaAlert.put("max", 3.0d);
+    expectedMetaAlert.put("count", 2);
+    expectedMetaAlert.put("sum", 5.0d);
+    expectedMetaAlert.put(getThreatTriageField(), 5.0d);
+
+    {
+      // Verify a list of alerts are removed from a meta alert
+      Assert.assertTrue(metaDao.removeAlertsFromMetaAlert("meta_alert", Arrays
+          .asList(new GetRequest("message_0", SENSOR_NAME),
+              new GetRequest("message_1", SENSOR_NAME))));
+      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
+    }
+
+    {
+      // Verify False when alerts are not present in a meta alert and no alerts are removed
+      Assert.assertFalse(metaDao.removeAlertsFromMetaAlert("meta_alert", Arrays
+          .asList(new GetRequest("message_0", SENSOR_NAME),
+              new GetRequest("message_1", SENSOR_NAME))));
+      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
+    }
+
+    {
+      // Verify only 1 alert is removed when a list of alerts only contains 1 alert that is in the meta alert
+      metaAlertAlerts = new ArrayList<>(
+          (List<Map<String, Object>>) expectedMetaAlert.get(ALERT_FIELD));
+      metaAlertAlerts.remove(0);
+      expectedMetaAlert.put(ALERT_FIELD, metaAlertAlerts);
+
+      expectedMetaAlert.put("average", 3.0d);
+      expectedMetaAlert.put("min", 3.0d);
+      expectedMetaAlert.put("median", 3.0d);
+      expectedMetaAlert.put("max", 3.0d);
+      expectedMetaAlert.put("count", 1);
+      expectedMetaAlert.put("sum", 3.0d);
+      expectedMetaAlert.put(getThreatTriageField(), 3.0d);
+
+      Assert.assertTrue(metaDao.removeAlertsFromMetaAlert("meta_alert", Arrays
+          .asList(new GetRequest("message_0", SENSOR_NAME),
+              new GetRequest("message_2", SENSOR_NAME))));
+      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
+    }
+
+    {
+      // Verify all alerts are removed from a metaAlert
+      metaAlertAlerts = new ArrayList<>(
+          (List<Map<String, Object>>) expectedMetaAlert.get(ALERT_FIELD));
+      metaAlertAlerts.remove(0);
+      if (isEmptyMetaAlertList()) {
+        expectedMetaAlert.put(ALERT_FIELD, metaAlertAlerts);
+      } else {
+        expectedMetaAlert.remove(ALERT_FIELD);
+      }
+
+      expectedMetaAlert.put("average", 0.0d);
+      expectedMetaAlert.put("count", 0);
+      expectedMetaAlert.put("sum", 0.0d);
+      expectedMetaAlert.put(getThreatTriageField(), 0.0d);
+
+      // Handle the cases with non-finite Double values on a per store basis
+      if (isFiniteDoubleOnly()) {
+        expectedMetaAlert.put("min", String.valueOf(Double.POSITIVE_INFINITY));
+        expectedMetaAlert.put("median", String.valueOf(Double.NaN));
+        expectedMetaAlert.put("max", String.valueOf(Double.NEGATIVE_INFINITY));
+      } else {
+        expectedMetaAlert.put("min", Double.POSITIVE_INFINITY);
+        expectedMetaAlert.put("median", Double.NaN);
+        expectedMetaAlert.put("max", Double.NEGATIVE_INFINITY);
+      }
+
+      // Verify removing alerts cannot result in an empty meta alert
+      try {
+        metaDao.removeAlertsFromMetaAlert("meta_alert",
+                Collections.singletonList(new GetRequest("message_3", SENSOR_NAME)));
+        Assert.fail("Removing these alerts will result in an empty meta alert.  Empty meta alerts are not allowed.");
+      } catch (IllegalStateException ise) {
+        Assert.assertEquals("Removing these alerts will result in an empty meta alert.  Empty meta alerts are not allowed.",
+                ise.getMessage());
+      }
+    }
+  }
+
+  @Test
+  public void addRemoveAlertsShouldThrowExceptionForInactiveMetaAlert() throws Exception {
+    // Load alerts
+    List<Map<String, Object>> alerts = buildAlerts(2);
+    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
+    addRecords(alerts, getTestIndexFullName(), SENSOR_NAME);
+
+    // Load metaAlert
+    Map<String, Object> metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.INACTIVE,
+        Optional.of(Collections.singletonList(alerts.get(0))));
+    addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), METAALERT_TYPE);
+
+    // Verify load was successful
+    findCreatedDocs(Arrays.asList(
+        new GetRequest("message_0", SENSOR_NAME),
+        new GetRequest("message_1", SENSOR_NAME),
+        new GetRequest("meta_alert", METAALERT_TYPE)));
+
+    {
+      // Verify alerts cannot be added to an INACTIVE meta alert
+      try {
+        metaDao.addAlertsToMetaAlert("meta_alert",
+            Collections.singletonList(new GetRequest("message_1", SENSOR_NAME)));
+        Assert.fail("Adding alerts to an inactive meta alert should throw an exception");
+      } catch (IllegalStateException ise) {
+        Assert.assertEquals("Adding alerts to an INACTIVE meta alert is not allowed",
+            ise.getMessage());
+      }
+    }
+
+    {
+      // Verify alerts cannot be removed from an INACTIVE meta alert
+      try {
+        metaDao.removeAlertsFromMetaAlert("meta_alert",
+            Collections.singletonList(new GetRequest("message_0", SENSOR_NAME)));
+        Assert.fail("Removing alerts from an inactive meta alert should throw an exception");
+      } catch (IllegalStateException ise) {
+        Assert.assertEquals("Removing alerts from an INACTIVE meta alert is not allowed",
+            ise.getMessage());
+      }
+    }
+  }
+
+  @Test
+  public void shouldUpdateMetaAlertStatus() throws Exception {
+    int numChildAlerts = 25;
+    int numUnrelatedAlerts = 25;
+    int totalAlerts = numChildAlerts + numUnrelatedAlerts;
+
+    // Load alerts
+    List<Map<String, Object>> alerts = buildAlerts(totalAlerts);
+    List<Map<String, Object>> childAlerts = alerts.subList(0, numChildAlerts);
+    List<Map<String, Object>> unrelatedAlerts = alerts.subList(numChildAlerts, totalAlerts);
+    for (Map<String, Object> alert : childAlerts) {
+      alert.put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
+    }
+    addRecords(alerts, getTestIndexFullName(), SENSOR_NAME);
+
+    // Load metaAlerts
+    Map<String, Object> metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE,
+        Optional.of(childAlerts));
+    // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
+    addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(),
+        METAALERT_TYPE);
+
+    List<GetRequest> requests = new ArrayList<>();
+    for (int i = 0; i < numChildAlerts; ++i) {
+      requests.add(new GetRequest("message_" + i, SENSOR_NAME));
+    }
+    requests.add(new GetRequest("meta_alert", METAALERT_TYPE));
+
+    // Verify load was successful
+    findCreatedDocs(requests);
+
+    {
+      // Verify status changed to inactive and child alerts are updated
+      Assert.assertTrue(metaDao.updateMetaAlertStatus("meta_alert", MetaAlertStatus.INACTIVE));
+
+      Map<String, Object> expectedMetaAlert = new HashMap<>(metaAlert);
+      expectedMetaAlert.put(STATUS_FIELD, MetaAlertStatus.INACTIVE.getStatusString());
+
+      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
+
+      for (int i = 0; i < numChildAlerts; ++i) {
+        Map<String, Object> expectedAlert = new HashMap<>(childAlerts.get(i));
+        setEmptiedMetaAlertField(expectedAlert);
+        findUpdatedDoc(expectedAlert, "message_" + i, SENSOR_NAME);
+      }
+
+      // Ensure unrelated alerts are unaffected
+      for (int i = 0; i < numUnrelatedAlerts; ++i) {
+        Map<String, Object> expectedAlert = new HashMap<>(unrelatedAlerts.get(i));
+        // Make sure to handle the guid offset from creation
+        findUpdatedDoc(expectedAlert, "message_" + (i + numChildAlerts), SENSOR_NAME);
+      }
+    }
+
+    {
+      // Verify status changed to active and child alerts are updated
+      Assert.assertTrue(metaDao.updateMetaAlertStatus("meta_alert", MetaAlertStatus.ACTIVE));
+
+      Map<String, Object> expectedMetaAlert = new HashMap<>(metaAlert);
+      expectedMetaAlert.put(STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString());
+
+      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
+
+      for (int i = 0; i < numChildAlerts; ++i) {
+        Map<String, Object> expectedAlert = new HashMap<>(alerts.get(i));
+        expectedAlert.put("metaalerts", Collections.singletonList("meta_alert"));
+        findUpdatedDoc(expectedAlert, "message_" + i, SENSOR_NAME);
+      }
+
+      // Ensure unrelated alerts are unaffected
+      for (int i = 0; i < numUnrelatedAlerts; ++i) {
+        Map<String, Object> expectedAlert = new HashMap<>(unrelatedAlerts.get(i));
+        // Make sure to handle the guid offset from creation
+        findUpdatedDoc(expectedAlert, "message_" + (i + numChildAlerts), SENSOR_NAME);
+      }
+
+      {
+        // Verify status changed to current status has no effect
+        Assert.assertFalse(metaDao.updateMetaAlertStatus("meta_alert", MetaAlertStatus.ACTIVE));
+
+        findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
+
+        for (int i = 0; i < numChildAlerts; ++i) {
+          Map<String, Object> expectedAlert = new HashMap<>(alerts.get(i));
+          expectedAlert.put("metaalerts", Collections.singletonList("meta_alert"));
+          findUpdatedDoc(expectedAlert, "message_" + i, SENSOR_NAME);
+        }
+
+        // Ensure unrelated alerts are unaffected
+        for (int i = 0; i < numUnrelatedAlerts; ++i) {
+          Map<String, Object> expectedAlert = new HashMap<>(unrelatedAlerts.get(i));
+          // Make sure to handle the guid offset from creation
+          findUpdatedDoc(expectedAlert, "message_" + (i + numChildAlerts), SENSOR_NAME);
+        }
+      }
+    }
+  }
+
+  @Test
+  public void shouldSearchByStatus() throws Exception {
+    // Load alert
+    List<Map<String, Object>> alerts = buildAlerts(1);
+    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_active"));
+    alerts.get(0).put("ip_src_addr", "192.168.1.1");
+    alerts.get(0).put("ip_src_port", 8010);
+
+    // Load metaAlerts
+    Map<String, Object> activeMetaAlert = buildMetaAlert("meta_active", MetaAlertStatus.ACTIVE,
+        Optional.of(Collections.singletonList(alerts.get(0))));
+    Map<String, Object> inactiveMetaAlert = buildMetaAlert("meta_inactive",
+        MetaAlertStatus.INACTIVE,
+        Optional.empty());
+
+    // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
+    addRecords(Arrays.asList(activeMetaAlert, inactiveMetaAlert), getMetaAlertIndex(),
+        METAALERT_TYPE);
+
+    // Verify load was successful
+    findCreatedDocs(Arrays.asList(
+        new GetRequest("meta_active", METAALERT_TYPE),
+        new GetRequest("meta_inactive", METAALERT_TYPE)));
+
+    SearchResponse searchResponse = metaDao.search(new SearchRequest() {
+      {
+        setQuery("*:*");
+        setIndices(Collections.singletonList(METAALERT_TYPE));
+        setFrom(0);
+        setSize(5);
+        setSort(Collections.singletonList(new SortField() {{
+          setField(Constants.GUID);
+        }}));
+      }
+    });
+
+    // Verify only active meta alerts are returned
+    Assert.assertEquals(1, searchResponse.getTotal());
+    Assert.assertEquals(MetaAlertStatus.ACTIVE.getStatusString(),
+        searchResponse.getResults().get(0).getSource().get(STATUS_FIELD));
+  }
+
+
+  @Test
+  public void shouldHidesAlertsOnGroup() throws Exception {
+    // Load alerts
+    List<Map<String, Object>> alerts = buildAlerts(2);
+    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_active"));
+    alerts.get(0).put("ip_src_addr", "192.168.1.1");
+    alerts.get(0).put("score", 1);
+    alerts.get(1).put("ip_src_addr", "192.168.1.1");
+    alerts.get(1).put("score", 10);
+    addRecords(alerts, getTestIndexFullName(), SENSOR_NAME);
+
+    // Put the nested type into the test index, so that it'll match appropriately
+    setupTypings();
+
+    // Don't need any meta alerts to actually exist, since we've populated the field on the alerts.
+
+    // Verify load was successful
+    findCreatedDocs(Arrays.asList(
+        new GetRequest("message_0", SENSOR_NAME),
+        new GetRequest("message_1", SENSOR_NAME)));
+
+    // Build our group request
+    Group searchGroup = new Group();
+    searchGroup.setField("ip_src_addr");
+    List<Group> groupList = new ArrayList<>();
+    groupList.add(searchGroup);
+    GroupResponse groupResponse = metaDao.group(new GroupRequest() {
+      {
+        setQuery("ip_src_addr:192.168.1.1");
+        setIndices(queryIndices);
+        setScoreField("score");
+        setGroups(groupList);
+      }
+    });
+
+    // Should only return the standalone alert in the group
+    GroupResult result = groupResponse.getGroupResults().get(0);
+    Assert.assertEquals(1, result.getTotal());
+    Assert.assertEquals("192.168.1.1", result.getKey());
+    // No delta, since no ops happen
+    Assert.assertEquals(10.0d, result.getScore(), 0.0d);
+  }
+
+  // This test is important enough that everyone should implement it, but is pretty specific to
+  // implementation
+  @Test
+  public abstract void shouldSearchByNestedAlert() throws Exception;
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void shouldUpdateMetaAlertOnAlertUpdate() throws Exception {
+    // Load alerts
+    List<Map<String, Object>> alerts = buildAlerts(2);
+    alerts.get(0).put(METAALERT_FIELD, Arrays.asList("meta_active", "meta_inactive"));
+    addRecords(alerts, getTestIndexFullName(), SENSOR_NAME);
+
+    // Load metaAlerts
+    Map<String, Object> activeMetaAlert = buildMetaAlert("meta_active", MetaAlertStatus.ACTIVE,
+        Optional.of(Collections.singletonList(alerts.get(0))));
+    Map<String, Object> inactiveMetaAlert = buildMetaAlert("meta_inactive",
+        MetaAlertStatus.INACTIVE,
+        Optional.of(Collections.singletonList(alerts.get(0))));
+    // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
+    addRecords(Arrays.asList(activeMetaAlert, inactiveMetaAlert), getMetaAlertIndex(),
+        METAALERT_TYPE);
+
+    // Verify load was successful
+    findCreatedDocs(Arrays.asList(
+        new GetRequest("message_0", SENSOR_NAME),
+        new GetRequest("message_1", SENSOR_NAME),
+        new GetRequest("meta_active", METAALERT_TYPE),
+        new GetRequest("meta_inactive", METAALERT_TYPE)));
+
+    {
+      // Modify the first message and add a new field
+      Map<String, Object> message0 = new HashMap<String, Object>(alerts.get(0)) {
+        {
+          put(NEW_FIELD, "metron");
+          put(THREAT_FIELD_DEFAULT, 10.0d);
+        }
+      };
+      String guid = "" + message0.get(Constants.GUID);
+      metaDao.update(new Document(message0, guid, SENSOR_NAME, null),
+          Optional.of(getTestIndexFullName()));
+
+      {
+        // Verify alerts are up-to-date
+        findUpdatedDoc(message0, guid, SENSOR_NAME);
+        long cnt = getMatchingAlertCount(NEW_FIELD, message0.get(NEW_FIELD));
+        if (cnt == 0) {
+          Assert.fail("Alert not updated!");
+        }
+      }
+
+      {
+        // Verify meta alerts are up-to-date
+        long cnt = getMatchingMetaAlertCount(NEW_FIELD, "metron");
+        if (cnt == 0) {
+          Assert.fail("Active metaalert was not updated!");
+        }
+        if (cnt != 1) {
+          Assert.fail("Metaalerts not updated correctly!");
+        }
+      }
+    }
+    //modify the same message and modify the new field
+    {
+      Map<String, Object> message0 = new HashMap<String, Object>(alerts.get(0)) {
+        {
+          put(NEW_FIELD, "metron2");
+        }
+      };
+      String guid = "" + message0.get(Constants.GUID);
+      metaDao.update(new Document(message0, guid, SENSOR_NAME, null), Optional.empty());
+
+      {
+        // Verify index is up-to-date
+        findUpdatedDoc(message0, guid, SENSOR_NAME);
+        long cnt = getMatchingAlertCount(NEW_FIELD, message0.get(NEW_FIELD));
+        if (cnt == 0) {
+          Assert.fail("Alert not updated!");
+        }
+      }
+      {
+        // Verify meta alerts are up-to-date
+        long cnt = getMatchingMetaAlertCount(NEW_FIELD, "metron2");
+        if (cnt == 0) {
+          Assert.fail("Active metaalert was not updated!");
+        }
+        if (cnt != 1) {
+          Assert.fail("Metaalerts not updated correctly!");
+        }
+      }
+    }
+  }
+
+  @Test
+  public void shouldThrowExceptionOnMetaAlertUpdate() throws Exception {
+    Document metaAlert = new Document(new HashMap<>(), "meta_alert", METAALERT_TYPE, 0L);
+    try {
+      // Verify a meta alert cannot be updated in the meta alert dao
+      metaDao.update(metaAlert, Optional.empty());
+      Assert.fail("Direct meta alert update should throw an exception");
+    } catch (UnsupportedOperationException uoe) {
+      Assert.assertEquals("Meta alerts cannot be directly updated", uoe.getMessage());
+    }
+  }
+
+  @Test
+  public void shouldPatchAllowedMetaAlerts() throws Exception {
+    // Load alerts
+    List<Map<String, Object>> alerts = buildAlerts(2);
+    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_active"));
+    alerts.get(1).put(METAALERT_FIELD, Collections.singletonList("meta_active"));
+    addRecords(alerts, getTestIndexFullName(), SENSOR_NAME);
+
+    // Put the nested type into the test index, so that it'll match appropriately
+    setupTypings();
+
+    // Load metaAlerts
+    Map<String, Object> metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE,
+        Optional.of(Arrays.asList(alerts.get(0), alerts.get(1))));
+    // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
+    addRecords(Collections.singletonList(metaAlert), getMetaAlertIndex(), METAALERT_TYPE);
+
+    // Verify load was successful
+    findCreatedDocs(Arrays.asList(
+        new GetRequest("message_0", SENSOR_NAME),
+        new GetRequest("message_1", SENSOR_NAME),
+        new GetRequest("meta_alert", METAALERT_TYPE)));
+
+    Map<String, Object> expectedMetaAlert = new HashMap<>(metaAlert);
+    expectedMetaAlert.put(NAME_FIELD, "New Meta Alert");
+    {
+      // Verify a patch to a field other than "status" or "alert" can be patched
+      String namePatch = namePatchRequest.replace(META_INDEX_FLAG, getMetaAlertIndex());
+      PatchRequest patchRequest = JSONUtils.INSTANCE.load(namePatch, PatchRequest.class);
+      metaDao.patch(metaDao, patchRequest, Optional.of(System.currentTimeMillis()));
+
+      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
+    }
+
+    {
+      // Verify a patch to an alert field should throw an exception
+      try {
+        String alertPatch = alertPatchRequest.replace(META_INDEX_FLAG, getMetaAlertIndex());
+        PatchRequest patchRequest = JSONUtils.INSTANCE.load(alertPatch, PatchRequest.class);
+        metaDao.patch(metaDao, patchRequest, Optional.of(System.currentTimeMillis()));
+
+        Assert.fail("A patch on the alert field should throw an exception");
+      } catch (IllegalArgumentException iae) {
+        Assert.assertEquals("Meta alert patches are not allowed for /alert or /status paths.  "
+                + "Please use the add/remove alert or update status functions instead.",
+            iae.getMessage());
+      }
+
+      // Verify the metaAlert was not updated
+      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
+    }
+
+    {
+      // Verify a patch to a status field should throw an exception
+      try {
+        String statusPatch = statusPatchRequest
+            .replace(META_INDEX_FLAG, getMetaAlertIndex());
+        PatchRequest patchRequest = JSONUtils.INSTANCE.load(statusPatch, PatchRequest.class);
+        metaDao.patch(metaDao, patchRequest, Optional.of(System.currentTimeMillis()));
+
+        Assert.fail("A patch on the status field should throw an exception");
+      } catch (IllegalArgumentException iae) {
+        Assert.assertEquals("Meta alert patches are not allowed for /alert or /status paths.  "
+                + "Please use the add/remove alert or update status functions instead.",
+            iae.getMessage());
+      }
+
+      // Verify the metaAlert was not updated
+      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
+    }
+  }
+
+  protected void findUpdatedDoc(Map<String, Object> message0, String guid, String sensorType)
+      throws InterruptedException, IOException, OriginalNotFoundException {
+    commit();
+    for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) {
+      Document doc = metaDao.getLatest(guid, sensorType);
+      // Change the underlying document alerts lists to sets to avoid ordering issues.
+      convertAlertsFieldToSet(doc.getDocument());
+      convertAlertsFieldToSet(message0);
+
+      if (doc.getDocument() != null && message0.equals(doc.getDocument())) {
+        convertAlertsFieldToList(doc.getDocument());
+        convertAlertsFieldToList(message0);
+        return;
+      }
+    }
+
+    throw new OriginalNotFoundException(
+        "Count not find " + guid + " after " + MAX_RETRIES + " tries");
+  }
+
+  protected void convertAlertsFieldToSet(Map<String, Object> document) {
+    if (document.get(ALERT_FIELD) instanceof List) {
+      @SuppressWarnings("unchecked")
+      List<Map<String, Object>> message0AlertField = (List<Map<String, Object>>) document
+          .get(ALERT_FIELD);
+      Set<Map<String, Object>> message0AlertSet = new HashSet<>(message0AlertField);
+      document.put(ALERT_FIELD, message0AlertSet);
+    }
+  }
+
+  protected void convertAlertsFieldToList(Map<String, Object> document) {
+    if (document.get(ALERT_FIELD) instanceof Set) {
+      @SuppressWarnings("unchecked")
+      Set<Map<String, Object>> message0AlertField = (Set<Map<String, Object>>) document
+          .get(ALERT_FIELD);
+      List<Map<String, Object>> message0AlertList = new ArrayList<>(message0AlertField);
+      message0AlertList.sort(Comparator.comparing(o -> ((String) o.get(Constants.GUID))));
+      document.put(ALERT_FIELD, message0AlertList);
+    }
+  }
+
+  protected boolean findCreatedDoc(String guid, String sensorType)
+      throws InterruptedException, IOException, OriginalNotFoundException {
+    for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) {
+      Document doc = metaDao.getLatest(guid, sensorType);
+      if (doc != null) {
+        return true;
+      }
+    }
+    throw new OriginalNotFoundException(
+        "Count not find " + guid + " after " + MAX_RETRIES + "tries");
+  }
+
+  protected boolean findCreatedDocs(List<GetRequest> getRequests)
+      throws InterruptedException, IOException, OriginalNotFoundException {
+    for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) {
+      Iterable<Document> docs = metaDao.getAllLatest(getRequests);
+      if (docs != null) {
+        int docCount = 0;
+        for (Document doc : docs) {
+          docCount++;
+        }
+        if (getRequests.size() == docCount) {
+          return true;
+        }
+      }
+    }
+    throw new OriginalNotFoundException("Count not find guids after " + MAX_RETRIES + "tries");
+  }
+
+  protected List<Map<String, Object>> buildAlerts(int count) {
+    List<Map<String, Object>> inputData = new ArrayList<>();
+    for (int i = 0; i < count; ++i) {
+      final String guid = "message_" + i;
+      Map<String, Object> alerts = new HashMap<>();
+      alerts.put(Constants.GUID, guid);
+      alerts.put(getSourceTypeField(), SENSOR_NAME);
+      alerts.put(THREAT_FIELD_DEFAULT, (double) i);
+      alerts.put("timestamp", System.currentTimeMillis());
+      inputData.add(alerts);
+    }
+    return inputData;
+  }
+
+  protected List<Map<String, Object>> buildMetaAlerts(int count, MetaAlertStatus status,
+      Optional<List<Map<String, Object>>> alerts) {
+    List<Map<String, Object>> inputData = new ArrayList<>();
+    for (int i = 0; i < count; ++i) {
+      final String guid = "meta_" + status.getStatusString() + "_" + i;
+      inputData.add(buildMetaAlert(guid, status, alerts));
+    }
+    return inputData;
+  }
+
+  protected Map<String, Object> buildMetaAlert(String guid, MetaAlertStatus status,
+      Optional<List<Map<String, Object>>> alerts) {
+    Map<String, Object> metaAlert = new HashMap<>();
+    metaAlert.put(Constants.GUID, guid);
+    metaAlert.put(getSourceTypeField(), METAALERT_TYPE);
+    metaAlert.put(STATUS_FIELD, status.getStatusString());
+    if (alerts.isPresent()) {
+      List<Map<String, Object>> alertsList = alerts.get();
+      metaAlert.put(ALERT_FIELD, alertsList);
+    }
+    return metaAlert;
+  }
+
+  protected abstract long getMatchingAlertCount(String fieldName, Object fieldValue)
+      throws IOException, InterruptedException;
+
+  protected abstract void addRecords(List<Map<String, Object>> inputData, String index,
+      String docType) throws IOException;
+
+  protected abstract long getMatchingMetaAlertCount(String fieldName, String fieldValue)
+      throws IOException, InterruptedException;
+
+  protected abstract void setupTypings();
+
+  // Get the base index name without any adjustments (e.g. without ES's "_index")
+  protected abstract String getTestIndexName();
+
+  // Get the full name of the test index.  E.g. Elasticsearch appends "_index"
+  protected String getTestIndexFullName() {
+    return getTestIndexName();
+  }
+
+  protected abstract String getMetaAlertIndex();
+
+  protected abstract String getSourceTypeField();
+
+  protected String getThreatTriageField() {
+    return THREAT_FIELD_DEFAULT;
+  }
+
+  // Allow for impls to do any commit they need to do.
+  protected void commit() throws IOException {
+  }
+
+  // Different stores can have different representations of empty metaalerts field.
+  // E.g. Solr expects the field to not be present, ES expects it to be empty.
+  protected abstract void setEmptiedMetaAlertField(Map<String, Object> docMap);
+
+  // Different stores may choose to store non finite double values as Strings.
+  // E.g. NaN may be a string, not a double value.
+  protected abstract boolean isFiniteDoubleOnly();
+
+  // Different stores may choose to return empty alerts lists differently.
+  // E.g. It may be missing completely, or may be an empty list
+  protected abstract boolean isEmptyMetaAlertList();
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaScoresTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaScoresTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaScoresTest.java
new file mode 100644
index 0000000..1359ba9
--- /dev/null
+++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/MetaScoresTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.metron.indexing.dao.metaalert;
+
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.ALERT_FIELD;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_TYPE;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.THREAT_FIELD_DEFAULT;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.THREAT_SORT_DEFAULT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.metron.indexing.dao.update.Document;
+import org.junit.Test;
+
+public class MetaScoresTest {
+  @Test
+  public void testCalculateMetaScoresList() {
+    final double delta = 0.001;
+    List<Map<String, Object>> alertList = new ArrayList<>();
+
+    // add an alert with a threat score
+    alertList.add(Collections.singletonMap(THREAT_FIELD_DEFAULT, 10.0f));
+
+    // add a second alert with a threat score
+    alertList.add(Collections.singletonMap(THREAT_FIELD_DEFAULT, 20.0f));
+
+    // add a third alert with NO threat score
+    alertList.add(Collections.singletonMap("alert3", "has no threat score"));
+
+    // create the metaalert
+    Map<String, Object> docMap = new HashMap<>();
+    docMap.put(ALERT_FIELD, alertList);
+    Document metaalert = new Document(docMap, "guid", METAALERT_TYPE, 0L);
+
+    // calculate the threat score for the metaalert
+    MetaScores.calculateMetaScores(metaalert, THREAT_FIELD_DEFAULT, THREAT_SORT_DEFAULT);
+
+    // the metaalert must contain a summary of all child threat scores
+    assertEquals(20D, (Double) metaalert.getDocument().get("max"), delta);
+    assertEquals(10D, (Double) metaalert.getDocument().get("min"), delta);
+    assertEquals(15D, (Double) metaalert.getDocument().get("average"), delta);
+    assertEquals(2L, metaalert.getDocument().get("count"));
+    assertEquals(30D, (Double) metaalert.getDocument().get("sum"), delta);
+    assertEquals(15D, (Double) metaalert.getDocument().get("median"), delta);
+
+    // it must contain an overall threat score; a float to match the type of the threat score of
+    // the other sensor indices
+    Object threatScore = metaalert.getDocument().get(THREAT_FIELD_DEFAULT);
+    assertTrue(threatScore instanceof Float);
+
+    // by default, the overall threat score is the sum of all child threat scores
+    assertEquals(30.0F, threatScore);
+  }
+}


[2/7] metron git commit: METRON-1421 Create a SolrMetaAlertDao (justinleet) closes apache/metron#970

Posted by le...@apache.org.
http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrSearchDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrSearchDao.java b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrSearchDao.java
index f1b7102..53ac083 100644
--- a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrSearchDao.java
+++ b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrSearchDao.java
@@ -18,10 +18,17 @@
 package org.apache.metron.solr.dao;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
-import org.apache.metron.common.Constants;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.metron.common.utils.JSONUtils;
 import org.apache.metron.indexing.dao.AccessConfig;
-import org.apache.metron.indexing.dao.search.GetRequest;
 import org.apache.metron.indexing.dao.search.Group;
 import org.apache.metron.indexing.dao.search.GroupOrder;
 import org.apache.metron.indexing.dao.search.GroupOrderType;
@@ -35,7 +42,6 @@ import org.apache.metron.indexing.dao.search.SearchResponse;
 import org.apache.metron.indexing.dao.search.SearchResult;
 import org.apache.metron.indexing.dao.search.SortField;
 import org.apache.metron.indexing.dao.search.SortOrder;
-import org.apache.metron.indexing.dao.update.Document;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrQuery.ORDER;
@@ -45,25 +51,11 @@ import org.apache.solr.client.solrj.response.FacetField;
 import org.apache.solr.client.solrj.response.FacetField.Count;
 import org.apache.solr.client.solrj.response.PivotField;
 import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.stream.Collectors;
-
-import static org.apache.metron.common.Constants.SENSOR_TYPE;
-
 public class SolrSearchDao implements SearchDao {
 
   private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -78,6 +70,13 @@ public class SolrSearchDao implements SearchDao {
 
   @Override
   public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException {
+    return search(searchRequest, null);
+  }
+
+  // Allow for the fieldList to be explicitly specified, letting things like metaalerts expand on them.
+  // If null, use whatever the searchRequest defines.
+  public SearchResponse search(SearchRequest searchRequest, String fieldList)
+      throws InvalidSearchException {
     if (searchRequest.getQuery() == null) {
       throw new InvalidSearchException("Search query is invalid: null");
     }
@@ -89,7 +88,7 @@ public class SolrSearchDao implements SearchDao {
           "Search result size must be less than " + accessConfig.getMaxSearchResults());
     }
     try {
-      SolrQuery query = buildSearchRequest(searchRequest);
+      SolrQuery query = buildSearchRequest(searchRequest, fieldList);
       QueryResponse response = client.query(query);
       return buildSearchResponse(searchRequest, response);
     } catch (SolrException | IOException | SolrServerException e) {
@@ -108,6 +107,7 @@ public class SolrSearchDao implements SearchDao {
           .setStart(0)
           .setRows(0)
           .setQuery(groupRequest.getQuery());
+
       query.set("collection", getCollections(groupRequest.getIndices()));
       Optional<String> scoreField = groupRequest.getScoreField();
       if (scoreField.isPresent()) {
@@ -125,39 +125,10 @@ public class SolrSearchDao implements SearchDao {
     }
   }
 
-  @Override
-  public Document getLatest(String guid, String collection) throws IOException {
-    try {
-      SolrDocument solrDocument = client.getById(collection, guid);
-      return toDocument(solrDocument);
-    } catch (SolrServerException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override
-  public Iterable<Document> getAllLatest(List<GetRequest> getRequests) throws IOException {
-    Map<String, Collection<String>> collectionIdMap = new HashMap<>();
-    for (GetRequest getRequest: getRequests) {
-      Collection<String> ids = collectionIdMap.getOrDefault(getRequest.getSensorType(), new HashSet<>());
-      ids.add(getRequest.getGuid());
-      collectionIdMap.put(getRequest.getSensorType(), ids);
-    }
-    try {
-      List<Document> documents = new ArrayList<>();
-      for (String collection: collectionIdMap.keySet()) {
-        SolrDocumentList solrDocumentList = client.getById(collectionIdMap.get(collection),
-            new SolrQuery().set("collection", collection));
-        documents.addAll(solrDocumentList.stream().map(this::toDocument).collect(Collectors.toList()));
-      }
-      return documents;
-    } catch (SolrServerException e) {
-      throw new IOException(e);
-    }
-  }
-
+  // An explicit, overriding fieldList can be provided.  This is useful for things like metaalerts,
+  // which may need to modify that parameter.
   protected SolrQuery buildSearchRequest(
-      SearchRequest searchRequest) throws IOException, SolrServerException {
+      SearchRequest searchRequest, String fieldList) throws IOException, SolrServerException {
     SolrQuery query = new SolrQuery()
         .setStart(searchRequest.getFrom())
         .setRows(searchRequest.getSize())
@@ -170,9 +141,13 @@ public class SolrSearchDao implements SearchDao {
 
     // handle search fields
     List<String> fields = searchRequest.getFields();
-    if (fields != null) {
-      fields.forEach(query::addField);
+    if (fieldList == null) {
+      fieldList = "*";
+      if (fields != null) {
+        fieldList = StringUtils.join(fields, ",");
+      }
     }
+    query.set("fl", fieldList);
 
     //handle facet fields
     List<String> facetFields = searchRequest.getFacetFields();
@@ -192,8 +167,8 @@ public class SolrSearchDao implements SearchDao {
 
   private SolrQuery.ORDER getSolrSortOrder(
       SortOrder sortOrder) {
-    return sortOrder == SortOrder.DESC ?
-        ORDER.desc : ORDER.asc;
+    return sortOrder == SortOrder.DESC
+        ? ORDER.desc : ORDER.asc;
   }
 
   protected SearchResponse buildSearchResponse(
@@ -206,7 +181,7 @@ public class SolrSearchDao implements SearchDao {
 
     // search hits --> search results
     List<SearchResult> results = solrDocumentList.stream()
-        .map(solrDocument -> getSearchResult(solrDocument, searchRequest.getFields()))
+        .map(solrDocument -> SolrUtilities.getSearchResult(solrDocument, searchRequest.getFields()))
         .collect(Collectors.toList());
     searchResponse.setResults(results);
 
@@ -228,19 +203,6 @@ public class SolrSearchDao implements SearchDao {
     return searchResponse;
   }
 
-  protected SearchResult getSearchResult(SolrDocument solrDocument, List<String> fields) {
-    SearchResult searchResult = new SearchResult();
-    searchResult.setId((String) solrDocument.getFieldValue(Constants.GUID));
-    final Map<String, Object> source = new HashMap<>();
-    if (fields != null) {
-      fields.forEach(field -> source.put(field, solrDocument.getFieldValue(field)));
-    } else {
-      solrDocument.getFieldNames().forEach(field -> source.put(field, solrDocument.getFieldValue(field)));
-    }
-    searchResult.setSource(source);
-    return searchResult;
-  }
-
   protected Map<String, Map<String, Long>> getFacetCounts(List<String> fields,
       QueryResponse solrResponse) {
     Map<String, Map<String, Long>> fieldCounts = new HashMap<>();
@@ -273,15 +235,16 @@ public class SolrSearchDao implements SearchDao {
     return groupResponse;
   }
 
-  protected List<GroupResult> getGroupResults(GroupRequest groupRequest, int index, List<PivotField> pivotFields) {
+  protected List<GroupResult> getGroupResults(GroupRequest groupRequest, int index,
+      List<PivotField> pivotFields) {
     List<Group> groups = groupRequest.getGroups();
     List<GroupResult> searchResultGroups = new ArrayList<>();
     final GroupOrder groupOrder = groups.get(index).getOrder();
     pivotFields.sort((o1, o2) -> {
-      String s1 = groupOrder.getGroupOrderType() == GroupOrderType.TERM ?
-          o1.getValue().toString() : Integer.toString(o1.getCount());
-      String s2 = groupOrder.getGroupOrderType() == GroupOrderType.TERM ?
-          o2.getValue().toString() : Integer.toString(o2.getCount());
+      String s1 = groupOrder.getGroupOrderType() == GroupOrderType.TERM
+          ? o1.getValue().toString() : Integer.toString(o1.getCount());
+      String s2 = groupOrder.getGroupOrderType() == GroupOrderType.TERM
+          ? o2.getValue().toString() : Integer.toString(o2.getCount());
       if (groupOrder.getSortOrder() == SortOrder.ASC) {
         return s1.compareTo(s2);
       } else {
@@ -289,30 +252,22 @@ public class SolrSearchDao implements SearchDao {
       }
     });
 
-    for(PivotField pivotField: pivotFields) {
+    for (PivotField pivotField : pivotFields) {
       GroupResult groupResult = new GroupResult();
       groupResult.setKey(pivotField.getValue().toString());
       groupResult.setTotal(pivotField.getCount());
       Optional<String> scoreField = groupRequest.getScoreField();
       if (scoreField.isPresent()) {
-        groupResult.setScore((Double) pivotField.getFieldStatsInfo().get("score").getSum());
+        groupResult
+            .setScore((Double) pivotField.getFieldStatsInfo().get(scoreField.get()).getSum());
       }
       if (index < groups.size() - 1) {
         groupResult.setGroupedBy(groups.get(index + 1).getField());
-        groupResult.setGroupResults(getGroupResults(groupRequest, index + 1, pivotField.getPivot()));
+        groupResult
+            .setGroupResults(getGroupResults(groupRequest, index + 1, pivotField.getPivot()));
       }
       searchResultGroups.add(groupResult);
     }
     return searchResultGroups;
   }
-
-  protected Document toDocument(SolrDocument solrDocument) {
-    Map<String, Object> document = new HashMap<>();
-    solrDocument.getFieldNames().stream()
-        .filter(name -> !name.equals(SolrDao.VERSION_FIELD))
-        .forEach(name -> document.put(name, solrDocument.getFieldValue(name)));
-    return new Document(document,
-        (String) solrDocument.getFieldValue(Constants.GUID),
-        (String) solrDocument.getFieldValue(SENSOR_TYPE), 0L);
-  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrUpdateDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrUpdateDao.java b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrUpdateDao.java
index f25253d..54b10ad 100644
--- a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrUpdateDao.java
+++ b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrUpdateDao.java
@@ -22,9 +22,12 @@ import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Optional;
+import java.util.Set;
+import org.apache.metron.indexing.dao.AccessConfig;
 import org.apache.metron.indexing.dao.update.Document;
 import org.apache.metron.indexing.dao.update.UpdateDao;
 import org.apache.solr.client.solrj.SolrClient;
@@ -38,19 +41,23 @@ public class SolrUpdateDao implements UpdateDao {
   private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private transient SolrClient client;
+  private AccessConfig config;
 
-  public SolrUpdateDao(SolrClient client) {
+  public SolrUpdateDao(SolrClient client, AccessConfig config) {
     this.client = client;
+    this.config = config;
   }
 
   @Override
   public void update(Document update, Optional<String> index) throws IOException {
     try {
-      SolrInputDocument solrInputDocument = toSolrInputDocument(update);
+      SolrInputDocument solrInputDocument = SolrUtilities.toSolrInputDocument(update);
       if (index.isPresent()) {
         this.client.add(index.get(), solrInputDocument);
+        this.client.commit(index.get());
       } else {
         this.client.add(solrInputDocument);
+        this.client.commit();
       }
     } catch (SolrServerException e) {
       throw new IOException(e);
@@ -61,40 +68,36 @@ public class SolrUpdateDao implements UpdateDao {
   public void batchUpdate(Map<Document, Optional<String>> updates) throws IOException {
     // updates with a collection specified
     Map<String, Collection<SolrInputDocument>> solrCollectionUpdates = new HashMap<>();
+    Set<String> collectionsUpdated = new HashSet<>();
 
-    // updates with no collection specified
-    Collection<SolrInputDocument> solrUpdates = new ArrayList<>();
-
-    for(Entry<Document, Optional<String>> entry: updates.entrySet()) {
-      SolrInputDocument solrInputDocument = toSolrInputDocument(entry.getKey());
+    for (Entry<Document, Optional<String>> entry : updates.entrySet()) {
+      SolrInputDocument solrInputDocument = SolrUtilities.toSolrInputDocument(entry.getKey());
       Optional<String> index = entry.getValue();
       if (index.isPresent()) {
-        Collection<SolrInputDocument> solrInputDocuments = solrCollectionUpdates.get(index.get());
-        if (solrInputDocuments == null) {
-          solrInputDocuments = new ArrayList<>();
-        }
+        Collection<SolrInputDocument> solrInputDocuments = solrCollectionUpdates
+            .getOrDefault(index.get(), new ArrayList<>());
         solrInputDocuments.add(solrInputDocument);
         solrCollectionUpdates.put(index.get(), solrInputDocuments);
+        collectionsUpdated.add(index.get());
       } else {
-        solrUpdates.add(solrInputDocument);
+        String lookupIndex = config.getIndexSupplier().apply(entry.getKey().getSensorType());
+        Collection<SolrInputDocument> solrInputDocuments = solrCollectionUpdates
+            .getOrDefault(lookupIndex, new ArrayList<>());
+        solrInputDocuments.add(solrInputDocument);
+        solrCollectionUpdates.put(lookupIndex, solrInputDocuments);
+        collectionsUpdated.add(lookupIndex);
       }
     }
     try {
-      if (!solrCollectionUpdates.isEmpty()) {
-        for(Entry<String, Collection<SolrInputDocument>> entry: solrCollectionUpdates.entrySet()) {
-          this.client.add(entry.getKey(), entry.getValue());
-        }
-      } else {
-        this.client.add(solrUpdates);
+      for (Entry<String, Collection<SolrInputDocument>> entry : solrCollectionUpdates
+          .entrySet()) {
+        this.client.add(entry.getKey(), entry.getValue());
+      }
+      for (String collection : collectionsUpdated) {
+        this.client.commit(collection);
       }
     } catch (SolrServerException e) {
       throw new IOException(e);
     }
   }
-
-  private SolrInputDocument toSolrInputDocument(Document document) {
-    SolrInputDocument solrInputDocument = new SolrInputDocument();
-    document.getDocument().forEach(solrInputDocument::addField);
-    return solrInputDocument;
-  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrUtilities.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrUtilities.java b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrUtilities.java
new file mode 100644
index 0000000..ce0ae84
--- /dev/null
+++ b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrUtilities.java
@@ -0,0 +1,92 @@
+/*
+ * 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.solr.dao;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.metron.common.Constants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
+import org.apache.metron.indexing.dao.search.SearchResult;
+import org.apache.metron.indexing.dao.update.Document;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrInputDocument;
+
+public class SolrUtilities {
+
+  public static SearchResult getSearchResult(SolrDocument solrDocument, List<String> fields) {
+    SearchResult searchResult = new SearchResult();
+    searchResult.setId((String) solrDocument.getFieldValue(Constants.GUID));
+    Map<String, Object> docSource = toDocument(solrDocument).getDocument();
+    final Map<String, Object> source = new HashMap<>();
+    if (fields != null) {
+      fields.forEach(field -> source.put(field, docSource.get(field)));
+    } else {
+      source.putAll(docSource);
+    }
+    searchResult.setSource(source);
+    return searchResult;
+  }
+
+  public static Document toDocument(SolrDocument solrDocument) {
+    Map<String, Object> document = new HashMap<>();
+    solrDocument.getFieldNames().stream()
+        .filter(name -> !name.equals(SolrDao.VERSION_FIELD))
+        .forEach(name -> document.put(name, solrDocument.getFieldValue(name)));
+    // Make sure to put child alerts in
+    if (solrDocument.hasChildDocuments() && solrDocument
+        .getFieldValue(Constants.SENSOR_TYPE)
+        .equals(MetaAlertConstants.METAALERT_TYPE)) {
+      List<Map<String, Object>> childDocuments = new ArrayList<>();
+      for (SolrDocument childDoc : solrDocument.getChildDocuments()) {
+        Map<String, Object> childDocMap = new HashMap<>();
+        childDoc.getFieldNames().stream()
+            .filter(name -> !name.equals(SolrDao.VERSION_FIELD))
+            .forEach(name -> childDocMap.put(name, childDoc.getFieldValue(name)));
+        childDocuments.add(childDocMap);
+      }
+
+      document.put(MetaAlertConstants.ALERT_FIELD, childDocuments);
+    }
+    return new Document(document,
+        (String) solrDocument.getFieldValue(Constants.GUID),
+        (String) solrDocument.getFieldValue(Constants.SENSOR_TYPE), 0L);
+  }
+
+  public static SolrInputDocument toSolrInputDocument(Document document) {
+    SolrInputDocument solrInputDocument = new SolrInputDocument();
+    for (Map.Entry<String, Object> field : document.getDocument().entrySet()) {
+      if (field.getKey().equals(MetaAlertConstants.ALERT_FIELD)) {
+        // We have a children, that needs to be translated as a child doc, not a field.
+        List<Map<String, Object>> alerts = (List<Map<String, Object>>) field.getValue();
+        for (Map<String, Object> alert : alerts) {
+          SolrInputDocument childDocument = new SolrInputDocument();
+          for (Map.Entry<String, Object> alertField : alert.entrySet()) {
+            childDocument.addField(alertField.getKey(), alertField.getValue());
+          }
+          solrInputDocument.addChildDocument(childDocument);
+        }
+      } else {
+        solrInputDocument.addField(field.getKey(), field.getValue());
+      }
+    }
+    return solrInputDocument;
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrDaoTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrDaoTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrDaoTest.java
index 56e363f..fc328be 100644
--- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrDaoTest.java
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrDaoTest.java
@@ -17,6 +17,18 @@
  */
 package org.apache.metron.solr.dao;
 
+import static org.mockito.Mockito.verify;
+import static org.mockito.internal.verification.VerificationModeFactory.times;
+import static org.powermock.api.mockito.PowerMockito.doNothing;
+import static org.powermock.api.mockito.PowerMockito.doReturn;
+import static org.powermock.api.mockito.PowerMockito.mock;
+import static org.powermock.api.mockito.PowerMockito.spy;
+import static org.powermock.api.mockito.PowerMockito.whenNew;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
 import org.apache.metron.indexing.dao.AccessConfig;
 import org.apache.metron.indexing.dao.search.GetRequest;
 import org.apache.metron.indexing.dao.search.GroupRequest;
@@ -31,19 +43,6 @@ import org.junit.runner.RunWith;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-
-import static org.mockito.Mockito.verify;
-import static org.mockito.internal.verification.VerificationModeFactory.times;
-import static org.powermock.api.mockito.PowerMockito.doNothing;
-import static org.powermock.api.mockito.PowerMockito.doReturn;
-import static org.powermock.api.mockito.PowerMockito.mock;
-import static org.powermock.api.mockito.PowerMockito.spy;
-import static org.powermock.api.mockito.PowerMockito.whenNew;
-
 @RunWith(PowerMockRunner.class)
 @PrepareForTest({SolrDao.class})
 public class SolrDaoTest {
@@ -54,23 +53,31 @@ public class SolrDaoTest {
   private SolrClient client;
   private SolrSearchDao solrSearchDao;
   private SolrUpdateDao solrUpdateDao;
+  private SolrRetrieveLatestDao solrRetrieveLatestDao;
   private SolrColumnMetadataDao solrColumnMetadataDao;
   private SolrDao solrDao;
 
   @SuppressWarnings("unchecked")
   @Before
-  public void setUp() throws Exception {
+  public void setUp() {
     client = mock(SolrClient.class);
     solrSearchDao = mock(SolrSearchDao.class);
     solrUpdateDao = mock(SolrUpdateDao.class);
+    solrRetrieveLatestDao = mock(SolrRetrieveLatestDao.class);
     solrColumnMetadataDao = mock(SolrColumnMetadataDao.class);
   }
 
   @Test
-  public void initShouldEnableKerberos() throws Exception {
+  public void initShouldEnableKerberos() {
     AccessConfig accessConfig = new AccessConfig();
 
-    solrDao = spy(new SolrDao(client, accessConfig, solrSearchDao, solrUpdateDao, solrColumnMetadataDao));
+    solrDao = spy(new SolrDao(
+        client,
+        accessConfig,
+        solrSearchDao,
+        solrUpdateDao,
+        solrRetrieveLatestDao,
+        solrColumnMetadataDao));
     doNothing().when(solrDao).enableKerberos();
 
     solrDao.init(accessConfig);
@@ -86,17 +93,20 @@ public class SolrDaoTest {
   @Test
   public void initShouldCreateDaos() throws Exception {
     AccessConfig accessConfig = new AccessConfig();
-    accessConfig.setGlobalConfigSupplier( () ->
-            new HashMap<String, Object>() {{
-              put("solr.zookeeper", "zookeeper:2181");
-            }}
+    accessConfig.setGlobalConfigSupplier(() ->
+        new HashMap<String, Object>() {{
+          put("solr.zookeeper", "zookeeper:2181");
+        }}
     );
 
     solrDao = spy(new SolrDao());
     doReturn(client).when(solrDao).getSolrClient("zookeeper:2181");
     whenNew(SolrSearchDao.class).withArguments(client, accessConfig).thenReturn(solrSearchDao);
-    whenNew(SolrUpdateDao.class).withArguments(client).thenReturn(solrUpdateDao);
-    whenNew(SolrColumnMetadataDao.class).withArguments("zookeeper:2181").thenReturn(solrColumnMetadataDao);
+    whenNew(SolrUpdateDao.class).withArguments(client, accessConfig).thenReturn(solrUpdateDao);
+    whenNew(SolrRetrieveLatestDao.class).withArguments(client)
+        .thenReturn(solrRetrieveLatestDao);
+    whenNew(SolrColumnMetadataDao.class).withArguments("zookeeper:2181")
+        .thenReturn(solrColumnMetadataDao);
 
     solrDao.init(accessConfig);
 
@@ -109,18 +119,18 @@ public class SolrDaoTest {
     verify(solrSearchDao).group(groupRequest);
 
     solrDao.getLatest("guid", "collection");
-    verify(solrSearchDao).getLatest("guid", "collection");
+    verify(solrRetrieveLatestDao).getLatest("guid", "collection");
 
     GetRequest getRequest1 = mock(GetRequest.class);
     GetRequest getRequest2 = mock(GetRequest.class);
     solrDao.getAllLatest(Arrays.asList(getRequest1, getRequest2));
-    verify(solrSearchDao).getAllLatest(Arrays.asList(getRequest1, getRequest2));
+    verify(solrRetrieveLatestDao).getAllLatest(Arrays.asList(getRequest1, getRequest2));
 
     Document document = mock(Document.class);
     solrDao.update(document, Optional.of("bro"));
     verify(solrUpdateDao).update(document, Optional.of("bro"));
 
-    Map<Document, Optional<String>> updates = new HashMap<Document, Optional<String>>(){{
+    Map<Document, Optional<String>> updates = new HashMap<Document, Optional<String>>() {{
       put(document, Optional.of("bro"));
     }};
     solrDao.batchUpdate(updates);
@@ -129,5 +139,4 @@ public class SolrDaoTest {
     solrDao.getColumnMetadata(Arrays.asList("bro", "snort"));
     verify(solrColumnMetadataDao).getColumnMetadata(Arrays.asList("bro", "snort"));
   }
-
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrMetaAlertDaoTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrMetaAlertDaoTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrMetaAlertDaoTest.java
new file mode 100644
index 0000000..5378467
--- /dev/null
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrMetaAlertDaoTest.java
@@ -0,0 +1,137 @@
+/*
+ * 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.solr.dao;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.metron.indexing.dao.AccessConfig;
+import org.apache.metron.indexing.dao.HBaseDao;
+import org.apache.metron.indexing.dao.IndexDao;
+import org.apache.metron.indexing.dao.MultiIndexDao;
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest;
+import org.apache.metron.indexing.dao.search.FieldType;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.search.GroupRequest;
+import org.apache.metron.indexing.dao.search.GroupResponse;
+import org.apache.metron.indexing.dao.search.InvalidCreateException;
+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.indexing.dao.update.OriginalNotFoundException;
+import org.apache.metron.indexing.dao.update.PatchRequest;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class SolrMetaAlertDaoTest {
+  private static AccessConfig accessConfig = new AccessConfig();
+
+  @BeforeClass
+  public static void setupBefore() {
+    accessConfig.setGlobalConfigSupplier(() ->
+        new HashMap<String, Object>() {{
+          put("solr.zookeeper", "zookeeper:2181");
+        }}
+    );
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidInit() {
+    IndexDao dao = new IndexDao() {
+      @Override
+      public SearchResponse search(SearchRequest searchRequest) {
+        return null;
+      }
+
+      @Override
+      public GroupResponse group(GroupRequest groupRequest) {
+        return null;
+      }
+
+      @Override
+      public void init(AccessConfig config) {
+      }
+
+      @Override
+      public Document getLatest(String guid, String sensorType) {
+        return null;
+      }
+
+      @Override
+      public Iterable<Document> getAllLatest(
+          List<GetRequest> getRequests) {
+        return null;
+      }
+
+      @Override
+      public void update(Document update, Optional<String> index) {
+      }
+
+      @Override
+      public void batchUpdate(Map<Document, Optional<String>> updates) {
+      }
+
+      @Override
+      public void patch(RetrieveLatestDao dao, PatchRequest request, Optional<Long> timestamp) {
+      }
+
+      @Override
+      public Map<String, FieldType> getColumnMetadata(List<String> indices) {
+        return null;
+      }
+    };
+    SolrMetaAlertDao metaAlertDao = new SolrMetaAlertDao();
+    metaAlertDao.init(dao);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInitInvalidDao() {
+    HBaseDao dao = new HBaseDao();
+    SolrMetaAlertDao solrDao = new SolrMetaAlertDao();
+    solrDao.init(dao, Optional.empty());
+  }
+
+  @Test(expected = InvalidCreateException.class)
+  public void testCreateMetaAlertEmptyGuids() throws InvalidCreateException, IOException {
+    SolrDao solrDao = new SolrDao();
+    solrDao.init(accessConfig);
+    SolrMetaAlertDao emaDao = new SolrMetaAlertDao();
+    emaDao.init(solrDao);
+
+    MetaAlertCreateRequest createRequest = new MetaAlertCreateRequest();
+    emaDao.createMetaAlert(createRequest);
+  }
+
+  @Test(expected = InvalidCreateException.class)
+  public void testCreateMetaAlertEmptyGroups() throws InvalidCreateException, IOException {
+    SolrDao solrDao = new SolrDao();
+    solrDao.init(accessConfig);
+    MultiIndexDao miDao = new MultiIndexDao(solrDao);
+    SolrMetaAlertDao emaDao = new SolrMetaAlertDao();
+    emaDao.init(miDao);
+
+    MetaAlertCreateRequest createRequest = new MetaAlertCreateRequest();
+    createRequest.setAlerts(Collections.singletonList(new GetRequest("don't", "care")));
+    emaDao.createMetaAlert(createRequest);
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrSearchDaoTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrSearchDaoTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrSearchDaoTest.java
index 9f2414a..49a40ef 100644
--- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrSearchDaoTest.java
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrSearchDaoTest.java
@@ -17,6 +17,27 @@
  */
 package org.apache.metron.solr.dao;
 
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.IsCollectionContaining.hasItems;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+import static org.powermock.api.mockito.PowerMockito.mockStatic;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
 import org.apache.metron.common.Constants;
 import org.apache.metron.indexing.dao.AccessConfig;
 import org.apache.metron.indexing.dao.search.GetRequest;
@@ -52,28 +73,6 @@ import org.junit.runner.RunWith;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.core.IsCollectionContaining.hasItems;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.argThat;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.when;
-import static org.powermock.api.mockito.PowerMockito.mockStatic;
-
 @RunWith(PowerMockRunner.class)
 @PrepareForTest({CollectionAdminRequest.class})
 public class SolrSearchDaoTest {
@@ -84,6 +83,7 @@ public class SolrSearchDaoTest {
   private SolrClient client;
   private AccessConfig accessConfig;
   private SolrSearchDao solrSearchDao;
+  private SolrRetrieveLatestDao solrRetrieveLatestDao;
 
   @SuppressWarnings("unchecked")
   @Before
@@ -91,6 +91,7 @@ public class SolrSearchDaoTest {
     client = mock(SolrClient.class);
     accessConfig = mock(AccessConfig.class);
     solrSearchDao = new SolrSearchDao(client, accessConfig);
+    solrRetrieveLatestDao = new SolrRetrieveLatestDao(client);
     mockStatic(CollectionAdminRequest.class);
     when(CollectionAdminRequest.listCollections(client)).thenReturn(Arrays.asList("bro", "snort"));
   }
@@ -104,12 +105,12 @@ public class SolrSearchDaoTest {
 
     solrSearchDao = spy(new SolrSearchDao(client, accessConfig));
     when(searchRequest.getQuery()).thenReturn("query");
-    doReturn(solrQuery).when(solrSearchDao).buildSearchRequest(searchRequest);
+    doReturn(solrQuery).when(solrSearchDao).buildSearchRequest(searchRequest, "*");
     when(client.query(solrQuery)).thenReturn(queryResponse);
     doReturn(searchResponse).when(solrSearchDao).buildSearchResponse(searchRequest, queryResponse);
 
-    assertEquals(searchResponse, solrSearchDao.search(searchRequest));
-    verify(solrSearchDao).buildSearchRequest(searchRequest);
+    assertEquals(searchResponse, solrSearchDao.search(searchRequest, "*"));
+    verify(solrSearchDao).buildSearchRequest(searchRequest, "*");
     verify(client).query(solrQuery);
     verify(solrSearchDao).buildSearchResponse(searchRequest, queryResponse);
     verifyNoMoreInteractions(client);
@@ -163,9 +164,9 @@ public class SolrSearchDaoTest {
     when(client.query(any())).thenReturn(queryResponse);
     doReturn(groupResponse).when(solrSearchDao).buildGroupResponse(groupRequest, queryResponse);
     SolrQuery expectedSolrQuery = new SolrQuery()
-            .setStart(0)
-            .setRows(0)
-            .setQuery("query");
+        .setStart(0)
+        .setRows(0)
+        .setQuery("query");
     expectedSolrQuery.set("collection", "bro,snort");
     expectedSolrQuery.set("stats", true);
     expectedSolrQuery.set("stats.field", "{!tag=piv1 sum=true}scoreField");
@@ -182,16 +183,14 @@ public class SolrSearchDaoTest {
   @Test
   public void getLatestShouldProperlyReturnDocument() throws Exception {
     SolrDocument solrDocument = mock(SolrDocument.class);
-    Document document = mock(Document.class);
 
     solrSearchDao = spy(new SolrSearchDao(client, accessConfig));
     when(client.getById("collection", "guid")).thenReturn(solrDocument);
-    doReturn(document).when(solrSearchDao).toDocument(solrDocument);
+    Document document = SolrUtilities.toDocument(solrDocument);
 
-    assertEquals(document, solrSearchDao.getLatest("guid", "collection"));
+    assertEquals(document, solrRetrieveLatestDao.getLatest("guid", "collection"));
 
     verify(client).getById("collection", "guid");
-    verify(solrSearchDao).toDocument(solrDocument);
     verifyNoMoreInteractions(client);
   }
 
@@ -205,16 +204,12 @@ public class SolrSearchDaoTest {
     SolrDocument broSolrDoc2 = mock(SolrDocument.class);
     SolrDocument snortSolrDoc1 = mock(SolrDocument.class);
     SolrDocument snortSolrDoc2 = mock(SolrDocument.class);
-    Document broDoc1 = mock(Document.class);
-    Document broDoc2 = mock(Document.class);
-    Document snortDoc1 = mock(Document.class);
-    Document snortDoc2 = mock(Document.class);
+    Document broDoc1 = SolrUtilities.toDocument(broSolrDoc1);
+    Document broDoc2 = SolrUtilities.toDocument(broSolrDoc2);
+    Document snortDoc1 = SolrUtilities.toDocument(snortSolrDoc1);
+    Document snortDoc2 = SolrUtilities.toDocument(snortSolrDoc2);
 
     solrSearchDao = spy(new SolrSearchDao(client, accessConfig));
-    doReturn(broDoc1).when(solrSearchDao).toDocument(broSolrDoc1);
-    doReturn(broDoc2).when(solrSearchDao).toDocument(broSolrDoc2);
-    doReturn(snortDoc1).when(solrSearchDao).toDocument(snortSolrDoc1);
-    doReturn(snortDoc2).when(solrSearchDao).toDocument(snortSolrDoc2);
     SolrDocumentList broList = new SolrDocumentList();
     broList.add(broSolrDoc1);
     broList.add(broSolrDoc2);
@@ -222,10 +217,14 @@ public class SolrSearchDaoTest {
     snortList.add(snortSolrDoc1);
     snortList.add(snortSolrDoc2);
     when(client.getById((Collection<String>) argThat(hasItems("bro-1", "bro-2")),
-            argThat(new ModifiableSolrParamsMatcher(new ModifiableSolrParams().set("collection", "bro"))))).thenReturn(broList);
+        argThat(
+            new ModifiableSolrParamsMatcher(new ModifiableSolrParams().set("collection", "bro")))))
+        .thenReturn(broList);
     when(client.getById((Collection<String>) argThat(hasItems("snort-1", "snort-2")),
-            argThat(new ModifiableSolrParamsMatcher(new ModifiableSolrParams().set("collection", "snort"))))).thenReturn(snortList);
-    assertEquals(Arrays.asList(broDoc1, broDoc2, snortDoc1, snortDoc2), solrSearchDao.getAllLatest(Arrays.asList(broRequest1, broRequest2, snortRequest1, snortRequest2)));
+        argThat(new ModifiableSolrParamsMatcher(
+            new ModifiableSolrParams().set("collection", "snort"))))).thenReturn(snortList);
+    assertEquals(Arrays.asList(broDoc1, broDoc2, snortDoc1, snortDoc2), solrRetrieveLatestDao
+        .getAllLatest(Arrays.asList(broRequest1, broRequest2, snortRequest1, snortRequest2)));
   }
 
   @Test
@@ -243,26 +242,30 @@ public class SolrSearchDaoTest {
     searchRequest.setFacetFields(Arrays.asList("facetField1", "facetField2"));
 
     SolrQuery exceptedSolrQuery = new SolrQuery()
-            .setStart(10)
-            .setRows(5)
-            .setQuery("query")
-            .addSort("sortField", SolrQuery.ORDER.asc)
-            .addField("field1").addField("field2")
-            .addFacetField("facetField1", "facetField2");
+        .setStart(10)
+        .setRows(5)
+        .setQuery("query")
+        .addSort("sortField", SolrQuery.ORDER.asc)
+        .addField("field1").addField("field2")
+        .addFacetField("facetField1", "facetField2");
     exceptedSolrQuery.set("collection", "bro,snort");
 
-    SolrQuery solrQuery = solrSearchDao.buildSearchRequest(searchRequest);
+    SolrQuery solrQuery = solrSearchDao.buildSearchRequest(searchRequest, "field1,field2");
     assertThat(solrQuery, new SolrQueryMatcher(exceptedSolrQuery));
   }
 
   @Test
-  public void buildSearchResponseShouldReturnSearchResponse() throws Exception {
+  public void buildSearchResponseShouldReturnSearchResponse() {
     SearchRequest searchRequest = new SearchRequest();
     searchRequest.setFields(Collections.singletonList("id"));
     searchRequest.setFacetFields(Collections.singletonList("facetField"));
     QueryResponse queryResponse = mock(QueryResponse.class);
-    SolrDocument solrDocument1 = mock(SolrDocument.class);
-    SolrDocument solrDocument2 = mock(SolrDocument.class);
+    SolrDocument solrDocument1 = new SolrDocument();
+    solrDocument1.setField(Constants.GUID, "id1");
+    solrDocument1.setField("id", "id1");
+    SolrDocument solrDocument2 = new SolrDocument();
+    solrDocument2.setField(Constants.GUID, "id2");
+    solrDocument2.setField("id", "id2");
 
     solrSearchDao = spy(new SolrSearchDao(client, accessConfig));
     SolrDocumentList solrDocumentList = new SolrDocumentList();
@@ -272,33 +275,40 @@ public class SolrSearchDaoTest {
     when(queryResponse.getResults()).thenReturn(solrDocumentList);
     SearchResult searchResult1 = new SearchResult();
     searchResult1.setId("id1");
+    HashMap<String, Object> source1 = new HashMap<>();
+    source1.put("id", "id1");
+    searchResult1.setSource(source1);
     SearchResult searchResult2 = new SearchResult();
     searchResult2.setId("id2");
-    doReturn(searchResult1).when(solrSearchDao).getSearchResult(solrDocument1,
-            Collections.singletonList("id"));
-    doReturn(searchResult2).when(solrSearchDao).getSearchResult(solrDocument2,
-            Collections.singletonList("id"));
+    HashMap<String, Object> source2 = new HashMap<>();
+    source2.put("id", "id2");
+    searchResult2.setSource(source2);
     Map<String, Map<String, Long>> facetCounts = new HashMap<String, Map<String, Long>>() {{
       put("id", new HashMap<String, Long>() {{
         put("id1", 1L);
         put("id2", 1L);
       }});
     }};
-    doReturn(facetCounts).when(solrSearchDao).getFacetCounts(Collections.singletonList("facetField"), queryResponse);
+    doReturn(facetCounts).when(solrSearchDao)
+        .getFacetCounts(Collections.singletonList("facetField"), queryResponse);
     SearchResponse expectedSearchResponse = new SearchResponse();
     SearchResult expectedSearchResult1 = new SearchResult();
     expectedSearchResult1.setId("id1");
+    expectedSearchResult1.setSource(source1);
     SearchResult expectedSearchResult2 = new SearchResult();
     expectedSearchResult2.setId("id2");
+    expectedSearchResult2.setSource(source2);
+
     expectedSearchResponse.setResults(Arrays.asList(expectedSearchResult1, expectedSearchResult2));
     expectedSearchResponse.setTotal(100);
     expectedSearchResponse.setFacetCounts(facetCounts);
 
-    assertEquals(expectedSearchResponse, solrSearchDao.buildSearchResponse(searchRequest, queryResponse));
+    assertEquals(expectedSearchResponse,
+        solrSearchDao.buildSearchResponse(searchRequest, queryResponse));
   }
 
   @Test
-  public void getSearchResultShouldProperlyReturnResults() throws Exception {
+  public void getSearchResultShouldProperlyReturnResults() {
     SolrDocument solrDocument = mock(SolrDocument.class);
 
     when(solrDocument.getFieldValue(Constants.GUID)).thenReturn("guid");
@@ -312,8 +322,8 @@ public class SolrSearchDaoTest {
       put("field1", "value1");
     }});
 
-    assertEquals(expectedSearchResult, solrSearchDao.getSearchResult(solrDocument,
-            Collections.singletonList("field1")));
+    assertEquals(expectedSearchResult, SolrUtilities.getSearchResult(solrDocument,
+        Collections.singletonList("field1")));
 
     SearchResult expectedSearchResultAllFields = new SearchResult();
     expectedSearchResultAllFields.setId("guid");
@@ -322,11 +332,12 @@ public class SolrSearchDaoTest {
       put("field2", "value2");
     }});
 
-    assertEquals(expectedSearchResultAllFields, solrSearchDao.getSearchResult(solrDocument, null));
+    assertEquals(expectedSearchResultAllFields,
+        SolrUtilities.getSearchResult(solrDocument, null));
   }
 
   @Test
-  public void getFacetCountsShouldProperlyReturnFacetCounts() throws Exception {
+  public void getFacetCountsShouldProperlyReturnFacetCounts() {
     QueryResponse queryResponse = mock(QueryResponse.class);
 
     FacetField facetField1 = new FacetField("field1");
@@ -349,11 +360,12 @@ public class SolrSearchDaoTest {
       }});
     }};
 
-    assertEquals(expectedFacetCounts, solrSearchDao.getFacetCounts(Arrays.asList("field1", "field2"), queryResponse));
+    assertEquals(expectedFacetCounts,
+        solrSearchDao.getFacetCounts(Arrays.asList("field1", "field2"), queryResponse));
   }
 
   @Test
-  public void buildGroupResponseShouldProperlyReturnGroupReponse() throws Exception {
+  public void buildGroupResponseShouldProperlyReturnGroupReponse() {
     GroupRequest groupRequest = mock(GroupRequest.class);
     QueryResponse queryResponse = mock(QueryResponse.class);
     NamedList namedList = mock(NamedList.class);
@@ -378,7 +390,7 @@ public class SolrSearchDaoTest {
   }
 
   @Test
-  public void getGroupResultsShouldProperlyReturnGroupResults() throws Exception {
+  public void getGroupResultsShouldProperlyReturnGroupResults() {
     GroupRequest groupRequest = new GroupRequest();
     Group group1 = new Group();
     group1.setField("field1");
@@ -393,7 +405,7 @@ public class SolrSearchDaoTest {
     groupOrder2.setGroupOrderType("COUNT");
     group2.setOrder(groupOrder2);
     groupRequest.setGroups(Arrays.asList(group1, group2));
-    groupRequest.setScoreField("scoreField");
+    groupRequest.setScoreField("score");
 
     PivotField level1Pivot1 = mock(PivotField.class);
     PivotField level1Pivot2 = mock(PivotField.class);
@@ -409,30 +421,31 @@ public class SolrSearchDaoTest {
     when(level1Pivot1.getValue()).thenReturn("field1value1");
     when(level1Pivot1.getCount()).thenReturn(1);
     when(level1Pivot1FieldStatsInfo.getSum()).thenReturn(1.0);
-    when(level1Pivot1.getFieldStatsInfo()).thenReturn(new HashMap<String, FieldStatsInfo>(){{
+    when(level1Pivot1.getFieldStatsInfo()).thenReturn(new HashMap<String, FieldStatsInfo>() {{
       put("score", level1Pivot1FieldStatsInfo);
     }});
     when(level1Pivot2.getValue()).thenReturn("field1value2");
     when(level1Pivot2.getCount()).thenReturn(2);
     when(level1Pivot2FieldStatsInfo.getSum()).thenReturn(2.0);
-    when(level1Pivot2.getFieldStatsInfo()).thenReturn(new HashMap<String, FieldStatsInfo>(){{
+    when(level1Pivot2.getFieldStatsInfo()).thenReturn(new HashMap<String, FieldStatsInfo>() {{
       put("score", level1Pivot2FieldStatsInfo);
     }});
     when(level2Pivot1.getValue()).thenReturn("field2value1");
     when(level2Pivot1.getCount()).thenReturn(3);
     when(level2Pivot1FieldStatsInfo.getSum()).thenReturn(3.0);
-    when(level2Pivot1.getFieldStatsInfo()).thenReturn(new HashMap<String, FieldStatsInfo>(){{
+    when(level2Pivot1.getFieldStatsInfo()).thenReturn(new HashMap<String, FieldStatsInfo>() {{
       put("score", level2Pivot1FieldStatsInfo);
     }});
     when(level2Pivot2.getValue()).thenReturn("field2value2");
     when(level2Pivot2.getCount()).thenReturn(4);
     when(level2Pivot2FieldStatsInfo.getSum()).thenReturn(4.0);
-    when(level2Pivot2.getFieldStatsInfo()).thenReturn(new HashMap<String, FieldStatsInfo>(){{
+    when(level2Pivot2.getFieldStatsInfo()).thenReturn(new HashMap<String, FieldStatsInfo>() {{
       put("score", level2Pivot2FieldStatsInfo);
     }});
     when(level1Pivot1.getPivot()).thenReturn(level2Pivots);
 
-    List<GroupResult> level1GroupResults = solrSearchDao.getGroupResults(groupRequest, 0, level1Pivots);
+    List<GroupResult> level1GroupResults = solrSearchDao
+        .getGroupResults(groupRequest, 0, level1Pivots);
 
     assertEquals("field1value1", level1GroupResults.get(0).getKey());
     assertEquals(1, level1GroupResults.get(0).getTotal());
@@ -457,22 +470,5 @@ public class SolrSearchDaoTest {
     assertNull(level2GroupResults.get(1).getGroupResults());
   }
 
-  @Test
-  public void toDocumentShouldProperlyReturnDocument() throws Exception {
-    SolrDocument solrDocument = new SolrDocument();
-    solrDocument.addField(SolrDao.VERSION_FIELD, 1.0);
-    solrDocument.addField(Constants.GUID, "guid");
-    solrDocument.addField(Constants.SENSOR_TYPE, "bro");
-    solrDocument.addField("field", "value");
-
-    Document expectedDocument = new Document(new HashMap<String, Object>(){{
-      put("field", "value");
-      put(Constants.GUID, "guid");
-      put(Constants.SENSOR_TYPE, "bro");
-    }}, "guid", "bro", 0L);
-
-    Document actualDocument = solrSearchDao.toDocument(solrDocument);
-    assertEquals(expectedDocument, actualDocument);
-  }
 
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrUpdateDaoTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrUpdateDaoTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrUpdateDaoTest.java
index 5315302..b9ed0b5 100644
--- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrUpdateDaoTest.java
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrUpdateDaoTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.metron.solr.dao;
 
+import org.apache.metron.indexing.dao.AccessConfig;
 import org.apache.metron.indexing.dao.update.Document;
 import org.apache.metron.solr.matcher.SolrInputDocumentListMatcher;
 import org.apache.metron.solr.matcher.SolrInputDocumentMatcher;
@@ -24,10 +25,12 @@ import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.common.SolrInputDocument;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
+import org.mockito.Matchers;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
@@ -51,11 +54,23 @@ public class SolrUpdateDaoTest {
   private SolrClient client;
   private SolrUpdateDao solrUpdateDao;
 
+  private static AccessConfig accessConfig = new AccessConfig();
+
+  @BeforeClass
+  public static void setupBefore() {
+    accessConfig.setGlobalConfigSupplier(() ->
+        new HashMap<String, Object>() {{
+          put("solr.zookeeper", "zookeeper:2181");
+        }}
+    );
+    accessConfig.setIndexSupplier(s -> null);
+  }
+
   @SuppressWarnings("unchecked")
   @Before
   public void setUp() throws Exception {
     client = mock(SolrClient.class);
-    solrUpdateDao = new SolrUpdateDao(client);
+    solrUpdateDao = new SolrUpdateDao(client, accessConfig);
   }
 
   @Test
@@ -129,7 +144,7 @@ public class SolrUpdateDaoTest {
 
     solrUpdateDao.batchUpdate(updates);
 
-    verify(client).add(argThat(new SolrInputDocumentListMatcher(Arrays.asList(snortSolrInputDocument1, snortSolrInputDocument2))));
+    verify(client).add((String) Matchers.isNull(), argThat(new SolrInputDocumentListMatcher(Arrays.asList(snortSolrInputDocument1, snortSolrInputDocument2))));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrUtilitiesTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrUtilitiesTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrUtilitiesTest.java
new file mode 100644
index 0000000..f284f25
--- /dev/null
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrUtilitiesTest.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.metron.solr.dao;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.HashMap;
+import org.apache.metron.common.Constants;
+import org.apache.metron.indexing.dao.update.Document;
+import org.apache.solr.common.SolrDocument;
+import org.junit.Test;
+
+public class SolrUtilitiesTest {
+
+  @Test
+  public void toDocumentShouldProperlyReturnDocument() throws Exception {
+    SolrDocument solrDocument = new SolrDocument();
+    solrDocument.addField(SolrDao.VERSION_FIELD, 1.0);
+    solrDocument.addField(Constants.GUID, "guid");
+    solrDocument.addField(Constants.SENSOR_TYPE, "bro");
+    solrDocument.addField("field", "value");
+
+    Document expectedDocument = new Document(new HashMap<String, Object>() {{
+      put("field", "value");
+      put(Constants.GUID, "guid");
+      put(Constants.SENSOR_TYPE, "bro");
+    }}, "guid", "bro", 0L);
+
+    Document actualDocument = SolrUtilities.toDocument(solrDocument);
+    assertEquals(expectedDocument, actualDocument);
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrIndexingIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrIndexingIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrIndexingIntegrationTest.java
index 10239f1..8e0ddd6 100644
--- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrIndexingIntegrationTest.java
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrIndexingIntegrationTest.java
@@ -40,6 +40,7 @@ import org.apache.metron.solr.integration.components.SolrComponent;
 public class SolrIndexingIntegrationTest extends IndexingIntegrationTest {
 
   private String collection = "yaf";
+
   private FieldNameConverter fieldNameConverter = fieldName -> fieldName;
   @Override
   public FieldNameConverter getFieldNameConverter() {
@@ -49,8 +50,8 @@ public class SolrIndexingIntegrationTest extends IndexingIntegrationTest {
   @Override
   public InMemoryComponent getSearchComponent(final Properties topologyProperties) throws Exception {
     SolrComponent solrComponent = new SolrComponent.Builder()
-            .addCollection(collection, "../metron-solr/src/main/config/schema/yaf")
-            .withPostStartCallback(new Function<SolrComponent, Void>() {
+        .addInitialCollection(collection, "../metron-solr/src/main/config/schema/yaf")
+        .withPostStartCallback(new Function<SolrComponent, Void>() {
               @Nullable
               @Override
               public Void apply(@Nullable SolrComponent solrComponent) {

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrMetaAlertIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrMetaAlertIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrMetaAlertIntegrationTest.java
new file mode 100644
index 0000000..aced539
--- /dev/null
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrMetaAlertIntegrationTest.java
@@ -0,0 +1,397 @@
+/*
+ * 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.solr.integration;
+
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.ALERT_FIELD;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_FIELD;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_TYPE;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.THREAT_FIELD_DEFAULT;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.THREAT_SORT_DEFAULT;
+import static org.apache.metron.solr.dao.SolrMetaAlertDao.METAALERTS_COLLECTION;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.metron.common.Constants;
+import org.apache.metron.indexing.dao.AccessConfig;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConfig;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertIntegrationTest;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.search.SearchRequest;
+import org.apache.metron.indexing.dao.search.SearchResponse;
+import org.apache.metron.indexing.dao.search.SortField;
+import org.apache.metron.solr.dao.SolrDao;
+import org.apache.metron.solr.dao.SolrMetaAlertDao;
+import org.apache.metron.solr.dao.SolrMetaAlertRetrieveLatestDao;
+import org.apache.metron.solr.dao.SolrMetaAlertSearchDao;
+import org.apache.metron.solr.dao.SolrMetaAlertUpdateDao;
+import org.apache.metron.solr.integration.components.SolrComponent;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.zookeeper.KeeperException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class SolrMetaAlertIntegrationTest extends MetaAlertIntegrationTest {
+
+  private static final String COLLECTION = "test";
+
+  private static SolrDao solrDao;
+  private static SolrComponent solr;
+
+  @BeforeClass
+  public static void setupBefore() throws Exception {
+    // Solr doesn't need retries, it'll show up after a commit.
+
+    MAX_RETRIES = 1;
+    // setup the client
+    solr = new SolrComponent.Builder().build();
+    solr.start();
+
+    AccessConfig accessConfig = new AccessConfig();
+    Map<String, Object> globalConfig = new HashMap<String, Object>() {
+      {
+        put("solr.clustername", "metron");
+        put("solr.port", "9300");
+        put("solr.ip", "localhost");
+        put("solr.date.format", DATE_FORMAT);
+        put("solr.zookeeper", solr.getZookeeperUrl());
+      }
+    };
+    accessConfig.setMaxSearchResults(1000);
+    accessConfig.setGlobalConfigSupplier(() -> globalConfig);
+    accessConfig.setMaxSearchGroups(100);
+    // Just use sensorType directly as the collection name.
+    accessConfig.setIndexSupplier(s -> s);
+
+    solrDao = new SolrDao();
+    solrDao.init(accessConfig);
+
+    MetaAlertConfig config = new MetaAlertConfig(
+        METAALERTS_COLLECTION,
+        THREAT_FIELD_DEFAULT,
+        THREAT_SORT_DEFAULT,
+        Constants.SENSOR_TYPE
+    );
+
+    SolrMetaAlertSearchDao searchDao = new SolrMetaAlertSearchDao(
+        solrDao.getSolrClient(solrDao.getZkHost()),
+        solrDao.getSolrSearchDao());
+    SolrMetaAlertRetrieveLatestDao retrieveLatestDao = new SolrMetaAlertRetrieveLatestDao(solrDao);
+    SolrMetaAlertUpdateDao updateDao = new SolrMetaAlertUpdateDao(solrDao, searchDao,
+        retrieveLatestDao, config);
+    metaDao = new SolrMetaAlertDao(solrDao, searchDao, updateDao, retrieveLatestDao);
+  }
+
+  @Before
+  public void setup()
+      throws IOException, InterruptedException, SolrServerException, KeeperException {
+    solr.addCollection(METAALERTS_COLLECTION,
+        "../metron-solr/src/main/config/schema//metaalert");
+    solr.addCollection(SENSOR_NAME, "../metron-solr/src/test/resources/config/test/conf");
+  }
+
+  @AfterClass
+  public static void teardown() {
+    if (solr != null) {
+      solr.stop();
+    }
+  }
+
+  @After
+  public void reset() {
+    solr.reset();
+  }
+
+  @Test
+  @Override
+  @SuppressWarnings("unchecked")
+  public void shouldSearchByNestedAlert() throws Exception {
+    // Load alerts
+    List<Map<String, Object>> alerts = buildAlerts(4);
+    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_active"));
+    alerts.get(0).put("ip_src_addr", "192.168.1.1");
+    alerts.get(0).put("ip_src_port", 8010);
+    alerts.get(1).put(METAALERT_FIELD, Collections.singletonList("meta_active"));
+    alerts.get(1).put("ip_src_addr", "192.168.1.2");
+    alerts.get(1).put("ip_src_port", 8009);
+    alerts.get(2).put("ip_src_addr", "192.168.1.3");
+    alerts.get(2).put("ip_src_port", 8008);
+    alerts.get(3).put("ip_src_addr", "192.168.1.4");
+    alerts.get(3).put("ip_src_port", 8007);
+    addRecords(alerts, getTestIndexName(), SENSOR_NAME);
+
+    // Put the nested type into the test index, so that it'll match appropriately
+    setupTypings();
+
+    // Load metaAlerts
+    Map<String, Object> activeMetaAlert = buildMetaAlert("meta_active", MetaAlertStatus.ACTIVE,
+        Optional.of(Arrays.asList(alerts.get(0), alerts.get(1))));
+    Map<String, Object> inactiveMetaAlert = buildMetaAlert("meta_inactive",
+        MetaAlertStatus.INACTIVE,
+        Optional.of(Arrays.asList(alerts.get(2), alerts.get(3))));
+    // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
+    addRecords(Arrays.asList(activeMetaAlert, inactiveMetaAlert), METAALERTS_COLLECTION,
+        METAALERT_TYPE);
+
+    // Verify load was successful
+    findCreatedDocs(Arrays.asList(
+        new GetRequest("message_0", SENSOR_NAME),
+        new GetRequest("message_1", SENSOR_NAME),
+        new GetRequest("message_2", SENSOR_NAME),
+        new GetRequest("message_3", SENSOR_NAME),
+        new GetRequest("meta_active", METAALERT_TYPE),
+        new GetRequest("meta_inactive", METAALERT_TYPE)));
+
+    SearchResponse searchResponse = metaDao.search(new SearchRequest() {
+      {
+        setQuery(
+            "ip_src_addr:192.168.1.1 AND ip_src_port:8010");
+        setIndices(Collections.singletonList(METAALERT_TYPE));
+        setFrom(0);
+        setSize(5);
+        setSort(Collections.singletonList(new SortField() {
+          {
+            setField(Constants.GUID);
+          }
+        }));
+      }
+    });
+    // Should have one result because Solr will return the parent.
+    Assert.assertEquals(1, searchResponse.getTotal());
+    // Ensure we returned the child alerts
+    List<Map<String, Object>> actualAlerts = (List<Map<String, Object>>) searchResponse.getResults()
+        .get(0).getSource()
+        .get(MetaAlertConstants.ALERT_FIELD);
+    Assert.assertEquals(2, actualAlerts.size());
+    Assert.assertEquals("meta_active",
+        searchResponse.getResults().get(0).getSource().get("guid"));
+
+    // Query against all indices. Only the single active meta alert should be returned.
+    // The child alerts should be hidden.
+    searchResponse = metaDao.search(new SearchRequest() {
+      {
+        setQuery(
+            "ip_src_addr:192.168.1.1 AND ip_src_port:8010");
+        setIndices(queryIndices);
+        setFrom(0);
+        setSize(5);
+        setSort(Collections.singletonList(new SortField() {
+          {
+            setField(Constants.GUID);
+          }
+        }));
+      }
+    });
+
+    // Query should match a parent alert
+    Assert.assertEquals(1, searchResponse.getTotal());
+    // Ensure we returned the child alerts
+    actualAlerts = (List<Map<String, Object>>) searchResponse.getResults().get(0).getSource()
+        .get(MetaAlertConstants.ALERT_FIELD);
+    Assert.assertEquals(2, actualAlerts.size());
+    Assert.assertEquals("meta_active",
+        searchResponse.getResults().get(0).getSource().get("guid"));
+
+    // Query against all indices. The child alert has no actual attached meta alerts, and should
+    // be returned on its own.
+    searchResponse = metaDao.search(new SearchRequest() {
+      {
+        setQuery(
+            "ip_src_addr:192.168.1.3 AND ip_src_port:8008");
+        setIndices(queryIndices);
+        setFrom(0);
+        setSize(1);
+        setSort(Collections.singletonList(new SortField() {
+          {
+            setField(Constants.GUID);
+          }
+        }));
+      }
+    });
+
+    // Query should match a plain alert
+    Assert.assertEquals(1, searchResponse.getTotal());
+    // Ensure we have no child alerts
+    actualAlerts = (List<Map<String, Object>>) searchResponse.getResults()
+        .get(0).getSource()
+        .get(MetaAlertConstants.ALERT_FIELD);
+    Assert.assertNull(actualAlerts);
+    Assert.assertEquals("message_2",
+        searchResponse.getResults().get(0).getSource().get("guid"));
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void shouldNotRetrieveFullChildrenWithoutSourceType() throws Exception {
+    // Load alerts
+    List<Map<String, Object>> alerts = buildAlerts(1);
+    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_active"));
+    alerts.get(0).put("ip_src_addr", "192.168.1.1");
+    alerts.get(0).put("ip_src_port", 8010);
+    addRecords(alerts, getTestIndexName(), SENSOR_NAME);
+
+    // Put the nested type into the test index, so that it'll match appropriately
+    setupTypings();
+
+    // Load metaAlerts
+    Map<String, Object> activeMetaAlert = buildMetaAlert("meta_active", MetaAlertStatus.ACTIVE,
+        Optional.of(Arrays.asList(alerts.get(0))));
+    // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
+    addRecords(Collections.singletonList(activeMetaAlert), METAALERTS_COLLECTION, METAALERT_TYPE);
+
+    // Verify load was successful
+    findCreatedDocs(Collections.singletonList(new GetRequest("meta_active", METAALERT_TYPE)));
+
+    SearchResponse searchResponse = metaDao.search(new SearchRequest() {
+      {
+        setQuery(
+            "ip_src_addr:192.168.1.1 AND ip_src_port:8010");
+        setIndices(Collections.singletonList(METAALERT_TYPE));
+        setFrom(0);
+        setSize(5);
+        setFields(Collections.singletonList(Constants.GUID));
+        setSort(Collections.singletonList(new SortField() {
+          {
+            setField(Constants.GUID);
+          }
+        }));
+      }
+    });
+
+    // Should have one result because Solr will return the parent.
+    Assert.assertEquals(1, searchResponse.getTotal());
+    // Ensure we returned didn't return the child alerts
+    List<Map<String, Object>> actualAlerts = (List<Map<String, Object>>) searchResponse.getResults()
+        .get(0).getSource()
+        .get(MetaAlertConstants.ALERT_FIELD);
+    Assert.assertNull(actualAlerts);
+    Assert.assertEquals("meta_active",
+        searchResponse.getResults().get(0).getSource().get("guid"));
+  }
+
+  @Override
+  protected long getMatchingAlertCount(String fieldName, Object fieldValue)
+      throws InterruptedException {
+    long cnt = 0;
+    for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t, Thread.sleep(SLEEP_MS)) {
+      List<Map<String, Object>> docs = solr.getAllIndexedDocs(getTestIndexName());
+      cnt = docs
+          .stream()
+          .filter(d -> {
+            Object newfield = d.get(fieldName);
+            return newfield != null && newfield.equals(fieldValue);
+          }).count();
+    }
+    return cnt;
+  }
+
+  @Override
+  protected long getMatchingMetaAlertCount(String fieldName, String fieldValue)
+      throws InterruptedException {
+    long cnt = 0;
+    for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t, Thread.sleep(SLEEP_MS)) {
+      List<Map<String, Object>> docs = solr.getAllIndexedDocs(METAALERTS_COLLECTION);
+      cnt = docs
+          .stream()
+          .filter(d -> {
+            @SuppressWarnings("unchecked")
+            List<Map<String, Object>> alerts = (List<Map<String, Object>>) d
+                .get(ALERT_FIELD);
+
+            for (Map<String, Object> alert : alerts) {
+              Object newField = alert.get(fieldName);
+              if (newField != null && newField.equals(fieldValue)) {
+                return true;
+              }
+            }
+
+            return false;
+          }).count();
+    }
+    return cnt;
+  }
+
+  @Override
+  protected void addRecords(List<Map<String, Object>> inputData, String index, String docType)
+      throws IOException {
+    // Ignore docType for Solr. It's unused.
+    try {
+      solr.addDocs(index, inputData);
+    } catch (SolrServerException e) {
+      throw new IOException("Unable to load Solr Docs", e);
+    }
+  }
+
+  @Override
+  protected void setupTypings() {
+
+  }
+
+  @Override
+  protected String getTestIndexName() {
+    return COLLECTION;
+  }
+
+  @Override
+  protected String getMetaAlertIndex() {
+    return METAALERTS_COLLECTION;
+  }
+
+  @Override
+  protected String getSourceTypeField() {
+    return Constants.SENSOR_TYPE;
+  }
+
+  @Override
+  protected void commit() throws IOException {
+    try {
+      List<String> collections = solr.getSolrClient().listCollections();
+      for (String collection : collections) {
+        solr.getSolrClient().commit(collection);
+      }
+    } catch (SolrServerException e) {
+      throw new IOException("Unable to commit", e);
+    }
+  }
+
+  @Override
+  protected void setEmptiedMetaAlertField(Map<String, Object> docMap) {
+    docMap.remove(METAALERT_FIELD);
+  }
+
+  @Override
+  protected boolean isFiniteDoubleOnly() {
+    return false;
+  }
+
+  @Override
+  protected boolean isEmptyMetaAlertList() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrSearchIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrSearchIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrSearchIntegrationTest.java
index 92941cd..2aa65ae 100644
--- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrSearchIntegrationTest.java
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrSearchIntegrationTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.metron.solr.integration;
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
@@ -33,22 +34,45 @@ import org.apache.metron.indexing.dao.search.SearchResponse;
 import org.apache.metron.integration.InMemoryComponent;
 import org.apache.metron.solr.dao.SolrDao;
 import org.apache.metron.solr.integration.components.SolrComponent;
-import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.SolrServerException;
 import org.json.simple.JSONArray;
 import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.Ignore;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class SolrSearchIntegrationTest extends SearchIntegrationTest {
+  private static SolrComponent solrComponent;
+  private static IndexDao dao;
+
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    indexComponent = startIndex();
+    dao = createDao();
+    // The data is all static for searches, so we can set it up here, and not do anything between tests.
+    broData = SearchIntegrationTest.broData.replace("source:type", "source.type");
+    snortData = SearchIntegrationTest.snortData.replace("source:type", "source.type");
+    solrComponent.addCollection("bro", "../metron-solr/src/main/config/schema/bro");
+    solrComponent.addCollection("snort", "../metron-solr/src/main/config/schema/snort");
+    loadTestData();
+  }
 
-  private SolrComponent solrComponent;
-
-  private String broData = SearchIntegrationTest.broData.replace("source:type", "source.type");
-  private String snortData = SearchIntegrationTest.snortData.replace("source:type", "source.type");
+  @AfterClass
+  public static void teardown() {
+    if (solrComponent != null) {
+      solrComponent.stop();
+    }
+  }
 
   @Override
-  protected IndexDao createDao() throws Exception {
+  public IndexDao getIndexDao() {
+    return dao;
+  }
+
+  protected static IndexDao createDao() {
     AccessConfig config = new AccessConfig();
     config.setMaxSearchResults(100);
     config.setMaxSearchGroups(100);
@@ -63,29 +87,18 @@ public class SolrSearchIntegrationTest extends SearchIntegrationTest {
     return dao;
   }
 
-  @Override
-  protected InMemoryComponent startIndex() throws Exception {
-
-    solrComponent = new SolrComponent.Builder()
-        .addCollection("bro", "../metron-solr/src/main/config/schema/bro")
-        .addCollection("snort", "../metron-solr/src/main/config/schema/snort")
-        .build();
+  protected static InMemoryComponent startIndex() throws Exception {
+    solrComponent = new SolrComponent.Builder().build();
     solrComponent.start();
     return solrComponent;
   }
 
   @SuppressWarnings("unchecked")
-  @Override
-  protected void loadTestData() throws Exception {
-    CloudSolrClient solrClient = solrComponent.getSolrClient();
+  protected static void loadTestData() throws ParseException, IOException, SolrServerException {
     JSONArray broArray = (JSONArray) new JSONParser().parse(broData);
     solrComponent.addDocs("bro", broArray);
-    solrClient.setDefaultCollection("bro");
-    solrClient.commit();
     JSONArray snortArray = (JSONArray) new JSONParser().parse(snortData);
     solrComponent.addDocs("snort", snortArray);
-    solrClient.setDefaultCollection("snort");
-    solrClient.commit();
   }
 
   @Override
@@ -95,7 +108,7 @@ public class SolrSearchIntegrationTest extends SearchIntegrationTest {
     {
       Map<String, FieldType> fieldTypes = dao.getColumnMetadata(Collections.singletonList("bro"));
       // Don't test all 256, just test a sample of different fields
-      Assert.assertEquals(261, fieldTypes.size());
+      Assert.assertEquals(262, fieldTypes.size());
 
       // Fields present in both with same type
       Assert.assertEquals(FieldType.TEXT, fieldTypes.get("guid"));
@@ -131,7 +144,7 @@ public class SolrSearchIntegrationTest extends SearchIntegrationTest {
     // getColumnMetadata with only snort
     {
       Map<String, FieldType> fieldTypes = dao.getColumnMetadata(Collections.singletonList("snort"));
-      Assert.assertEquals(31, fieldTypes.size());
+      Assert.assertEquals(32, fieldTypes.size());
 
       // Fields present in both with same type
       Assert.assertEquals(FieldType.TEXT, fieldTypes.get("guid"));

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java
index dc08967..8f1ff12 100644
--- a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrUpdateIntegrationTest.java
@@ -25,13 +25,22 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.metron.hbase.mock.MockHBaseTableProvider;
+import org.apache.metron.hbase.mock.MockHTable;
+import org.apache.metron.indexing.dao.AccessConfig;
+import org.apache.metron.indexing.dao.HBaseDao;
 import org.apache.metron.indexing.dao.IndexDao;
 import org.apache.metron.indexing.dao.MultiIndexDao;
 import org.apache.metron.indexing.dao.UpdateIntegrationTest;
 import org.apache.metron.indexing.dao.update.Document;
-import org.apache.metron.integration.InMemoryComponent;
 import org.apache.metron.solr.dao.SolrDao;
 import org.apache.metron.solr.integration.components.SolrComponent;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -40,37 +49,66 @@ public class SolrUpdateIntegrationTest extends UpdateIntegrationTest {
   @Rule
   public final ExpectedException exception = ExpectedException.none();
 
-  protected static SolrComponent solrComponent;
+  private static SolrComponent solrComponent;
 
-  @Override
-  protected String getIndexName() {
-    return SENSOR_NAME;
+  private static final String TABLE_NAME = "modifications";
+  private static final String CF = "p";
+  private static MockHTable table;
+  private static IndexDao hbaseDao;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    solrComponent = new SolrComponent.Builder().build();
+    solrComponent.start();
   }
 
-  @Override
-  protected Map<String, Object> createGlobalConfig() throws Exception {
-    return new HashMap<String, Object>() {{
-      put("solr.zookeeper", solrComponent.getZookeeperUrl());
-    }};
+  @Before
+  public void setup() throws Exception {
+    solrComponent.addCollection(SENSOR_NAME, "../metron-solr/src/test/resources/config/test/conf");
+    solrComponent.addCollection("error", "../metron-solr/src/main/config/schema/error");
+
+    Configuration config = HBaseConfiguration.create();
+    MockHBaseTableProvider tableProvider = new MockHBaseTableProvider();
+    MockHBaseTableProvider.addToCache(TABLE_NAME, CF);
+    table = (MockHTable) tableProvider.getTable(config, TABLE_NAME);
+
+    hbaseDao = new HBaseDao();
+    AccessConfig accessConfig = new AccessConfig();
+    accessConfig.setTableProvider(tableProvider);
+    Map<String, Object> globalConfig = createGlobalConfig();
+    globalConfig.put(HBaseDao.HBASE_TABLE, TABLE_NAME);
+    globalConfig.put(HBaseDao.HBASE_CF, CF);
+    accessConfig.setGlobalConfigSupplier(() -> globalConfig);
+
+    dao = new MultiIndexDao(hbaseDao, new SolrDao());
+    dao.init(accessConfig);
   }
 
-  @Override
-  protected IndexDao createDao() throws Exception {
-    return new SolrDao();
+  @After
+  public void reset() {
+    solrComponent.reset();
+    table.clear();
+  }
+
+  @AfterClass
+  public static void teardown() {
+    solrComponent.stop();
   }
 
   @Override
-  protected InMemoryComponent startIndex() throws Exception {
-    solrComponent = new SolrComponent.Builder().addCollection(SENSOR_NAME, "../metron-solr/src/main/config/schema/bro")
-        .addCollection("error", "../metron-solr/src/main/config/schema/error")
-        .build();
-    solrComponent.start();
-    return solrComponent;
+  protected String getIndexName() {
+    return SENSOR_NAME;
   }
 
   @Override
-  protected void loadTestData() throws Exception {
+  protected MockHTable getMockHTable() {
+    return table;
+  }
 
+  private static Map<String, Object> createGlobalConfig() {
+    return new HashMap<String, Object>() {{
+      put("solr.zookeeper", solrComponent.getZookeeperUrl());
+    }};
   }
 
   @Override
@@ -80,16 +118,12 @@ public class SolrUpdateIntegrationTest extends UpdateIntegrationTest {
   }
 
   @Override
-  protected List<Map<String, Object>> getIndexedTestData(String indexName, String sensorType)
-      throws Exception {
+  protected List<Map<String, Object>> getIndexedTestData(String indexName, String sensorType) {
     return solrComponent.getAllIndexedDocs(indexName);
   }
 
   @Test
   public void suppress_expanded_fields() throws Exception {
-    dao = new MultiIndexDao(createDao());
-    dao.init(getAccessConfig());
-
     Map<String, Object> fields = new HashMap<>();
     fields.put("guid", "bro_1");
     fields.put("source.type", SENSOR_NAME);
@@ -111,9 +145,6 @@ public class SolrUpdateIntegrationTest extends UpdateIntegrationTest {
 
   @Test
   public void testHugeErrorFields() throws Exception {
-    dao = new MultiIndexDao(createDao());
-    dao.init(getAccessConfig());
-
     String hugeString = StringUtils.repeat("test ", 1_000_000);
     String hugeStringTwo = hugeString + "-2";
 


[3/7] metron git commit: METRON-1421 Create a SolrMetaAlertDao (justinleet) closes apache/metron#970

Posted by le...@apache.org.
http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/lucene/AbstractLuceneMetaAlertUpdateDaoTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/lucene/AbstractLuceneMetaAlertUpdateDaoTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/lucene/AbstractLuceneMetaAlertUpdateDaoTest.java
new file mode 100644
index 0000000..2d620d9
--- /dev/null
+++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/metaalert/lucene/AbstractLuceneMetaAlertUpdateDaoTest.java
@@ -0,0 +1,854 @@
+/*
+ * 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.metaalert.lucene;
+
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.ALERT_FIELD;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.GROUPS_FIELD;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_FIELD;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_TYPE;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.STATUS_FIELD;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.THREAT_FIELD_DEFAULT;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.THREAT_SORT_DEFAULT;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertStatus.ACTIVE;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertStatus.INACTIVE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.UUID;
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.commons.math.util.MathUtils;
+import org.apache.metron.common.Constants;
+import org.apache.metron.common.Constants.Fields;
+import org.apache.metron.indexing.dao.IndexDao;
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConfig;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateResponse;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertRetrieveLatestDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
+import org.apache.metron.indexing.dao.metaalert.MetaScores;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.search.InvalidSearchException;
+import org.apache.metron.indexing.dao.update.Document;
+import org.apache.metron.indexing.dao.update.PatchRequest;
+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.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class AbstractLuceneMetaAlertUpdateDaoTest {
+
+  @Mock
+  IndexDao indexDao;
+
+  @Before
+  public void setup() {
+    dao = new TestLuceneMetaAlertUpdateDao();
+  }
+
+  private static final double EPS = 0.00001;
+  private static final String METAALERT_INDEX = "metaalert_index";
+  private static final String METAALERT_GUID = "meta_0";
+  private static final String DEFAULT_PREFIX = "child_";
+  private static final MetaAlertConfig TEST_CONFIG = new MetaAlertConfig(
+      METAALERT_INDEX,
+      THREAT_FIELD_DEFAULT,
+      THREAT_SORT_DEFAULT,
+      Constants.SENSOR_TYPE
+  );
+
+  private static Map<String, Document> documents = new HashMap<>();
+
+  static {
+    Document active = new Document(
+        new HashMap<>(),
+        ACTIVE.getStatusString(),
+        METAALERT_TYPE,
+        0L
+    );
+    documents.put(ACTIVE.getStatusString(), active);
+
+    Document inactive = new Document(
+        new HashMap<>(),
+        INACTIVE.getStatusString(),
+        METAALERT_TYPE,
+        0L
+    );
+    inactive.getDocument().put(
+        STATUS_FIELD,
+        INACTIVE.getStatusString()
+    );
+    documents.put(INACTIVE.getStatusString(), inactive);
+  }
+
+  TestMetaAlertRetrieveLatestDao retrieveLatestDao = new TestMetaAlertRetrieveLatestDao();
+
+  private class TestMetaAlertRetrieveLatestDao implements MetaAlertRetrieveLatestDao {
+
+    @Override
+    public Document getLatest(String guid, String sensorType) {
+      return documents.get(guid);
+    }
+
+    @Override
+    public Iterable<Document> getAllLatest(List<GetRequest> getRequests) {
+      return null;
+    }
+  }
+
+  TestLuceneMetaAlertUpdateDao dao = new TestLuceneMetaAlertUpdateDao();
+
+  private class TestLuceneMetaAlertUpdateDao extends AbstractLuceneMetaAlertUpdateDao {
+
+    TestLuceneMetaAlertUpdateDao() {
+      super(indexDao, retrieveLatestDao, TEST_CONFIG);
+    }
+
+    @Override
+    public void update(Document update, Optional<String> index) {
+    }
+
+    @Override
+    public void patch(RetrieveLatestDao retrieveLatestDao, PatchRequest request,
+        Optional<Long> timestamp) {
+    }
+
+    @Override
+    public MetaAlertCreateResponse createMetaAlert(MetaAlertCreateRequest request) {
+      return null;
+    }
+
+    @Override
+    public boolean addAlertsToMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests) {
+      return false;
+    }
+
+    @Override
+    public boolean updateMetaAlertStatus(String metaAlertGuid, MetaAlertStatus status) {
+      return false;
+    }
+  }
+
+  /**
+   {
+   "guid": "meta_alert",
+   "index": "metaalert_index",
+   "patch": [
+   {
+   "op": "add",
+   "path": "/alert",
+   "value": []
+   }
+   ],
+   "sensorType": "metaalert"
+   }
+   */
+  @Multiline
+  public static String alertPatchRequest;
+
+  /**
+   {
+   "guid": "meta_alert",
+   "index": "metaalert_index",
+   "patch": [
+   {
+   "op": "add",
+   "path": "/status",
+   "value": []
+   }
+   ],
+   "sensorType": "metaalert"
+   }
+   */
+  @Multiline
+  public static String statusPatchRequest;
+
+  /**
+   {
+   "guid": "meta_alert",
+   "index": "metaalert_index",
+   "patch": [
+   {
+   "op": "add",
+   "path": "/name",
+   "value": []
+   }
+   ],
+   "sensorType": "metaalert"
+   }
+   */
+  @Multiline
+  public static String namePatchRequest;
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testBatchUpdateThrowsException() {
+    dao.batchUpdate(null);
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testPatchNotAllowedAlert() throws ParseException {
+    PatchRequest pr = new PatchRequest();
+    Map<String, Object> patch = (JSONObject) new JSONParser().parse(alertPatchRequest);
+    pr.setPatch(Collections.singletonList((JSONObject) ((JSONArray) patch.get("patch")).get(0)));
+    assertFalse(dao.isPatchAllowed(pr));
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testPatchNotAllowedStatus() throws ParseException {
+    PatchRequest pr = new PatchRequest();
+    Map<String, Object> patch = (JSONObject) new JSONParser().parse(statusPatchRequest);
+    pr.setPatch(Collections.singletonList((JSONObject) ((JSONArray) patch.get("patch")).get(0)));
+    assertFalse(dao.isPatchAllowed(pr));
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testPatchAllowedName() throws ParseException {
+    PatchRequest pr = new PatchRequest();
+    Map<String, Object> patch = (JSONObject) new JSONParser().parse(namePatchRequest);
+    pr.setPatch(Collections.singletonList((JSONObject) ((JSONArray) patch.get("patch")).get(0)));
+    assertTrue(dao.isPatchAllowed(pr));
+  }
+
+  @Test
+  public void testUpdateSingle() throws IOException {
+    Map<Document, Optional<String>> updates = new HashMap<>();
+    Document document = new Document(new HashMap<>(), "guid", "sensor", 0L);
+    updates.put(document, Optional.empty());
+    dao.update(updates);
+    verify(indexDao, times(1)).update(document, Optional.empty());
+  }
+
+  @Test
+  public void testUpdateMultiple() throws IOException {
+    Map<Document, Optional<String>> updates = new HashMap<>();
+    Document documentOne = new Document(new HashMap<>(), "guid", "sensor", 0L);
+    updates.put(documentOne, Optional.empty());
+    Document documentTwo = new Document(new HashMap<>(), "guid2", "sensor", 0L);
+    updates.put(documentTwo, Optional.empty());
+    dao.update(updates);
+    verify(indexDao, times(1)).batchUpdate(updates);
+  }
+
+  @Test
+  public void testBuildAddAlertToMetaAlertUpdatesEmpty() {
+    Document metaDoc = new Document(
+        new HashMap<>(),
+        METAALERT_GUID,
+        METAALERT_TYPE,
+        0L
+    );
+    metaDoc.getDocument().put(
+        ALERT_FIELD,
+        getRawMaps(buildChildAlerts(1, METAALERT_GUID, null))
+    );
+    Map<Document, Optional<String>> actual = dao
+        .buildAddAlertToMetaAlertUpdates(metaDoc, new ArrayList<>());
+    assertEquals(0, actual.size());
+  }
+
+  @Test
+  public void testBuildAddAlertToMetaAlertUpdates() {
+    List<Document> alerts = buildChildAlerts(1, METAALERT_GUID, null);
+
+    Document metaDoc = buildMetaAlert(alerts);
+
+    List<Document> newAlerts = buildChildAlerts(2, null, "new_");
+    Map<Document, Optional<String>> actual = dao
+        .buildAddAlertToMetaAlertUpdates(metaDoc, newAlerts);
+    assertEquals(3, actual.size());
+
+    HashMap<String, Object> expectedExistingAlert = new HashMap<>();
+    expectedExistingAlert.put(Constants.GUID, "child_0");
+    expectedExistingAlert.put(METAALERT_FIELD, Collections.singletonList(METAALERT_GUID));
+    expectedExistingAlert.put(THREAT_FIELD_DEFAULT, 0.0f);
+
+    List<Map<String, Object>> expectedAlerts = new ArrayList<>();
+    expectedAlerts.add(expectedExistingAlert);
+    expectedAlerts.addAll(getRawMaps(newAlerts));
+
+    List<Double> scores = new ArrayList<>();
+    scores.add(0.0d);
+    scores.add(0.0d);
+    scores.add(0.0d);
+
+    Map<String, Object> expectedMetaAlertMap = new HashMap<>();
+    expectedMetaAlertMap.put(Constants.GUID, METAALERT_GUID);
+    expectedMetaAlertMap.put(ALERT_FIELD, expectedAlerts);
+    expectedMetaAlertMap.put(THREAT_FIELD_DEFAULT, 0.0f);
+
+    expectedMetaAlertMap.putAll(new MetaScores(scores).getMetaScores());
+    Document expectedMetaAlertDoc = new Document(expectedMetaAlertMap, METAALERT_GUID,
+        METAALERT_TYPE,
+        0L);
+
+    Map<Document, Optional<String>> expected = new HashMap<>();
+    expected.put(expectedMetaAlertDoc, Optional.of(METAALERT_INDEX));
+    expected.put(newAlerts.get(0), Optional.empty());
+    expected.put(newAlerts.get(1), Optional.empty());
+
+    assertTrue(updatesMapEquals(expected, actual));
+  }
+
+  @Test
+  public void testRemoveAlertsFromMetaAlert() throws IOException {
+    List<Document> alerts = buildChildAlerts(3, METAALERT_GUID, null);
+
+    Document metaDoc = buildMetaAlert(alerts);
+
+    List<Document> deletedAlerts = new ArrayList<>();
+    deletedAlerts.add(alerts.get(0));
+    deletedAlerts.add(alerts.get(2));
+
+    Map<Document, Optional<String>> actual = dao
+        .buildRemoveAlertsFromMetaAlert(metaDoc, deletedAlerts);
+    assertEquals(3, actual.size());
+
+    Map<String, Object> expectedDeletedAlert = new HashMap<>();
+    expectedDeletedAlert.put(Constants.GUID, "child_0");
+    expectedDeletedAlert.put(THREAT_FIELD_DEFAULT, 0.0f);
+    expectedDeletedAlert
+        .put(MetaAlertConstants.METAALERT_FIELD, new ArrayList<>());
+    Document expectedDeletedDocument = new Document(expectedDeletedAlert, "child_0", "test", 0L);
+
+    Map<String, Object> expectedDeletedAlert3 = new HashMap<>();
+    expectedDeletedAlert3.put(Constants.GUID, "child_2");
+    expectedDeletedAlert3.put(THREAT_FIELD_DEFAULT, 0.0f);
+    expectedDeletedAlert3
+        .put(MetaAlertConstants.METAALERT_FIELD, new ArrayList<>());
+    Document expectedDeletedDocument2 = new Document(expectedDeletedAlert3, "child_2", "test", 0L);
+
+    List<Map<String, Object>> expectedAlerts = new ArrayList<>();
+    expectedAlerts.add(alerts.get(1).getDocument());
+
+    Map<String, Object> expectedMetaAlertMap = new HashMap<>();
+    expectedMetaAlertMap.put(Constants.GUID, METAALERT_GUID);
+    expectedMetaAlertMap.put(ALERT_FIELD, expectedAlerts);
+    expectedMetaAlertMap.put(THREAT_FIELD_DEFAULT, 0.0f);
+    expectedMetaAlertMap.putAll(new MetaScores(Collections.singletonList(0.0d)).getMetaScores());
+    Document expectedMetaAlertDoc = new Document(expectedMetaAlertMap, METAALERT_GUID,
+        METAALERT_TYPE,
+        0L);
+
+    Map<Document, Optional<String>> expected = new HashMap<>();
+    expected.put(expectedDeletedDocument, Optional.empty());
+    expected.put(expectedDeletedDocument2, Optional.empty());
+    expected.put(expectedMetaAlertDoc, Optional.of(METAALERT_INDEX));
+
+    assertTrue(updatesMapEquals(expected, actual));
+  }
+
+  @Test
+  public void testBuildRemoveAlertsFromMetaAlertThrowsException() throws Exception {
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage("Removing these alerts will result in an empty meta alert.  Empty meta alerts are not allowed.");
+
+    List<Document> alerts = buildChildAlerts(1, METAALERT_GUID, null);
+    Document metaDoc = buildMetaAlert(alerts);
+
+    dao.buildRemoveAlertsFromMetaAlert(metaDoc, alerts);
+  }
+
+  @Test
+  public void testRemoveAlertsFromMetaAlertNoChildAlerts() {
+    Document empty = new Document(new HashMap<>(), "empty", METAALERT_TYPE, 0L);
+    boolean actual = dao.removeAlertsFromMetaAlert(empty, Collections.singletonList("child"));
+    assertFalse(actual);
+  }
+
+  @Test
+  public void testRemoveAlertsFromMetaAlertEmptyRemoveList() {
+    Document metaDoc = new Document(
+        new HashMap<>(),
+        METAALERT_GUID,
+        METAALERT_TYPE,
+        0L
+    );
+    metaDoc.getDocument().put(
+        STATUS_FIELD,
+        ACTIVE.getStatusString()
+    );
+    metaDoc.getDocument().put(
+        ALERT_FIELD,
+        new HashMap<String, Object>() {{
+          put(Constants.GUID, "child_0");
+        }}
+    );
+    boolean actual = dao.removeAlertsFromMetaAlert(metaDoc, new ArrayList<>());
+    assertFalse(actual);
+  }
+
+  @Test
+  public void testRemoveAlertsFromMetaAlertEmptyRemoveSingle() {
+    Document metaDoc = new Document(
+        new HashMap<>(),
+        METAALERT_GUID,
+        METAALERT_TYPE,
+        0L
+    );
+    metaDoc.getDocument().put(
+        STATUS_FIELD,
+        ACTIVE.getStatusString()
+    );
+    List<Map<String, Object>> alerts = new ArrayList<>();
+    alerts.add(new HashMap<String, Object>() {{
+      put(Constants.GUID, "child_0");
+    }});
+    metaDoc.getDocument().put(
+        ALERT_FIELD,
+        alerts
+    );
+    boolean actual = dao.removeAlertsFromMetaAlert(metaDoc, Collections.singletonList("child_0"));
+
+    Document expected = new Document(
+        new HashMap<>(),
+        METAALERT_GUID,
+        METAALERT_TYPE,
+        0L
+    );
+    expected.getDocument().put(
+        STATUS_FIELD,
+        ACTIVE.getStatusString()
+    );
+    expected.getDocument().put(ALERT_FIELD, new ArrayList<>());
+    assertTrue(actual);
+    assertEquals(expected, metaDoc);
+  }
+
+  @Test
+  public void testBuildStatusChangeUpdatesToInactive() {
+    List<Document> alerts = buildChildAlerts(2, METAALERT_GUID, null);
+
+    Map<String, Object> metaAlertMap = new HashMap<>();
+    metaAlertMap.put(ALERT_FIELD, getRawMaps(alerts));
+    metaAlertMap.put(Constants.GUID, METAALERT_GUID);
+    metaAlertMap.put(STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString());
+    Document metaDoc = new Document(
+        metaAlertMap,
+        METAALERT_GUID,
+        METAALERT_TYPE,
+        0L
+    );
+
+    Map<Document, Optional<String>> actual = dao
+        .buildStatusChangeUpdates(metaDoc, alerts, MetaAlertStatus.INACTIVE);
+    assertEquals(3, actual.size());
+
+    List<Document> expectedDeletedAlerts = buildChildAlerts(2, null, null);
+    List<Map<String, Object>> expectedAlerts = new ArrayList<>();
+    expectedAlerts.add(alerts.get(0).getDocument());
+    expectedAlerts.add(alerts.get(1).getDocument());
+
+    Map<String, Object> expectedMetaAlertMap = new HashMap<>();
+    expectedMetaAlertMap.put(Constants.GUID, METAALERT_GUID);
+    expectedMetaAlertMap.put(ALERT_FIELD, expectedAlerts);
+    expectedMetaAlertMap.put(STATUS_FIELD, MetaAlertStatus.INACTIVE.getStatusString());
+    Document expectedMetaAlertDoc = new Document(expectedMetaAlertMap, METAALERT_GUID,
+        METAALERT_TYPE,
+        0L);
+
+    Map<Document, Optional<String>> expected = new HashMap<>();
+    expected.put(expectedMetaAlertDoc, Optional.of(METAALERT_INDEX));
+    expected.put(expectedDeletedAlerts.get(0), Optional.empty());
+    expected.put(expectedDeletedAlerts.get(1), Optional.empty());
+
+    assertTrue(updatesMapEquals(expected, actual));
+  }
+
+  @Test
+  public void testBuildStatusChangeUpdatesToActive() {
+    List<Document> alerts = buildChildAlerts(2, METAALERT_GUID, null);
+
+    Map<String, Object> metaAlertMap = new HashMap<>();
+    metaAlertMap.put(ALERT_FIELD, getRawMaps(alerts));
+    metaAlertMap.put(Constants.GUID, METAALERT_GUID);
+    metaAlertMap.put(STATUS_FIELD, MetaAlertStatus.INACTIVE.getStatusString());
+    Document metaDoc = new Document(
+        metaAlertMap,
+        METAALERT_GUID,
+        METAALERT_TYPE,
+        0L
+    );
+
+    Map<Document, Optional<String>> actual = dao.buildStatusChangeUpdates(
+        metaDoc,
+        alerts,
+        MetaAlertStatus.ACTIVE
+    );
+
+    List<Map<String, Object>> expectedAlerts = new ArrayList<>();
+    expectedAlerts.add(alerts.get(0).getDocument());
+    expectedAlerts.add(alerts.get(1).getDocument());
+
+    Map<String, Object> expectedMetaAlertMap = new HashMap<>();
+    expectedMetaAlertMap.put(ALERT_FIELD, expectedAlerts);
+    expectedMetaAlertMap.put(Constants.GUID, METAALERT_GUID);
+    expectedMetaAlertMap.put(STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString());
+    Document expectedMetaAlertDoc = new Document(
+        expectedMetaAlertMap,
+        METAALERT_GUID,
+        METAALERT_TYPE,
+        0L
+    );
+
+    Map<Document, Optional<String>> expected = new HashMap<>();
+    expected.put(expectedMetaAlertDoc, Optional.of(METAALERT_INDEX));
+
+    assertTrue(updatesMapEquals(expected, actual));
+  }
+
+  @Test
+  public void testRemoveAlertsFromMetaAlertEmptyRemoveMultiple() {
+    Document metDoc = new Document(new HashMap<>(), METAALERT_GUID, METAALERT_TYPE, 0L);
+    metDoc.getDocument().put(STATUS_FIELD, ACTIVE.getStatusString());
+    List<Document> alerts = buildChildAlerts(3, null, null);
+    metDoc.getDocument().put(ALERT_FIELD, getRawMaps(alerts));
+    List<String> removeGuids = new ArrayList<>();
+    removeGuids.add("child_0");
+    removeGuids.add("child_2");
+    removeGuids.add("child_doesn't_exist");
+
+    boolean actual = dao.removeAlertsFromMetaAlert(metDoc, removeGuids);
+
+    // Build the expected metaalert
+    Document expected = new Document(new HashMap<>(), METAALERT_GUID, METAALERT_TYPE, 0L);
+    expected.getDocument().put(STATUS_FIELD, ACTIVE.getStatusString());
+    List<Map<String, Object>> alertsExpected = new ArrayList<>();
+    alertsExpected.add(new HashMap<String, Object>() {{
+                         put(METAALERT_FIELD, new ArrayList<>());
+                         put(Constants.GUID, "child_1");
+                         put(THREAT_FIELD_DEFAULT, 0.0f);
+                       }}
+    );
+
+    expected.getDocument().put(ALERT_FIELD, alertsExpected);
+    assertEquals(expected, metDoc);
+    assertTrue(actual);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testRemoveAlertsFromMetaAlertInactive() throws IOException {
+    dao.removeAlertsFromMetaAlert(INACTIVE.getStatusString(), null);
+  }
+
+  @Test
+  public void testRemoveMetaAlertFromAlertSuccess() {
+    List<String> metaAlertGuids = new ArrayList<>();
+    metaAlertGuids.add("metaalert1");
+    metaAlertGuids.add("metaalert2");
+    Map<String, Object> alertFields = new HashMap<>();
+    alertFields.put(METAALERT_FIELD, metaAlertGuids);
+    Document alert = new Document(alertFields, "alert", "test", 0L);
+
+    Document expected = new Document(new HashMap<>(), "alert", "test", 0L);
+    List<String> expectedMetaAlertGuids = new ArrayList<>();
+    expectedMetaAlertGuids.add("metaalert2");
+    expected.getDocument().put(METAALERT_FIELD, expectedMetaAlertGuids);
+
+    boolean actual = dao.removeMetaAlertFromAlert("metaalert1", alert);
+    assertTrue(actual);
+    assertEquals(expected, alert);
+  }
+
+  @Test
+  public void testRemoveMetaAlertFromAlertMissing() {
+    List<String> metaAlertGuids = new ArrayList<>();
+    metaAlertGuids.add("metaalert1");
+    metaAlertGuids.add("metaalert2");
+    Map<String, Object> alertFields = new HashMap<>();
+    alertFields.put(METAALERT_FIELD, metaAlertGuids);
+    Document alert = new Document(alertFields, "alert", "test", 0L);
+
+    boolean actual = dao.removeMetaAlertFromAlert("metaalert3", alert);
+    assertFalse(actual);
+  }
+
+  @Test
+  public void testAddMetaAlertToAlertEmpty() {
+    Map<String, Object> alertFields = new HashMap<>();
+    alertFields.put(METAALERT_FIELD, new ArrayList<>());
+    Document alert = new Document(alertFields, "alert", "test", 0L);
+
+    Document expected = new Document(new HashMap<>(), "alert", "test", 0L);
+    List<String> expectedMetaAlertGuids = new ArrayList<>();
+    expectedMetaAlertGuids.add("metaalert1");
+    expected.getDocument().put(METAALERT_FIELD, expectedMetaAlertGuids);
+
+    boolean actual = dao.addMetaAlertToAlert("metaalert1", alert);
+    assertTrue(actual);
+    assertEquals(expected, alert);
+  }
+
+  @Test
+  public void testAddMetaAlertToAlertNonEmpty() {
+    List<String> metaAlertGuids = new ArrayList<>();
+    metaAlertGuids.add("metaalert1");
+    Map<String, Object> alertFields = new HashMap<>();
+    alertFields.put(METAALERT_FIELD, metaAlertGuids);
+    Document alert = new Document(alertFields, "alert", "test", 0L);
+
+    Document expected = new Document(new HashMap<>(), "alert", "test", 0L);
+    List<String> expectedMetaAlertGuids = new ArrayList<>();
+    expectedMetaAlertGuids.add("metaalert1");
+    expectedMetaAlertGuids.add("metaalert2");
+    expected.getDocument().put(METAALERT_FIELD, expectedMetaAlertGuids);
+
+    boolean actual = dao.addMetaAlertToAlert("metaalert2", alert);
+    assertTrue(actual);
+    assertEquals(expected, alert);
+  }
+
+  @Test
+  public void testAddMetaAlertToAlertDuplicate() {
+    List<String> metaAlertGuids = new ArrayList<>();
+    metaAlertGuids.add("metaalert1");
+    Map<String, Object> alertFields = new HashMap<>();
+    alertFields.put(METAALERT_FIELD, metaAlertGuids);
+    Document alert = new Document(alertFields, "alert", "test", 0L);
+
+    boolean actual = dao.addMetaAlertToAlert("metaalert1", alert);
+    assertFalse(actual);
+  }
+
+  @Test
+  public void testBuildCreateDocumentSingleAlert() {
+    List<String> groups = new ArrayList<>();
+    groups.add("group_one");
+    groups.add("group_two");
+
+    // Build the first response from the multiget
+    Map<String, Object> alertOne = new HashMap<>();
+    alertOne.put(Constants.GUID, "alert_one");
+    alertOne.put(THREAT_FIELD_DEFAULT, 10.0d);
+    List<Document> alerts = new ArrayList<Document>() {{
+      add(new Document(alertOne, "", "", 0L));
+    }};
+
+    // Actually build the doc
+    Document actual = dao.buildCreateDocument(alerts, groups, ALERT_FIELD);
+
+    ArrayList<Map<String, Object>> alertList = new ArrayList<>();
+    alertList.add(alertOne);
+
+    Map<String, Object> actualDocument = actual.getDocument();
+    assertEquals(
+        MetaAlertStatus.ACTIVE.getStatusString(),
+        actualDocument.get(STATUS_FIELD)
+    );
+    assertEquals(
+        alertList,
+        actualDocument.get(ALERT_FIELD)
+    );
+    assertEquals(
+        groups,
+        actualDocument.get(GROUPS_FIELD)
+    );
+
+    // Don't care about the result, just that it's a UUID. Exception will be thrown if not.
+    UUID.fromString((String) actualDocument.get(Constants.GUID));
+  }
+
+  @Test
+  public void testBuildCreateDocumentMultipleAlerts() {
+    List<String> groups = new ArrayList<>();
+    groups.add("group_one");
+    groups.add("group_two");
+
+    // Build the first response from the multiget
+    Map<String, Object> alertOne = new HashMap<>();
+    alertOne.put(Constants.GUID, "alert_one");
+    alertOne.put(THREAT_FIELD_DEFAULT, 10.0d);
+
+    // Build the second response from the multiget
+    Map<String, Object> alertTwo = new HashMap<>();
+    alertTwo.put(Constants.GUID, "alert_one");
+    alertTwo.put(THREAT_FIELD_DEFAULT, 5.0d);
+    List<Document> alerts = new ArrayList<>();
+    alerts.add(new Document(alertOne, "", "", 0L));
+    alerts.add(new Document(alertTwo, "", "", 0L));
+
+    // Actually build the doc
+    Document actual = dao.buildCreateDocument(alerts, groups, ALERT_FIELD);
+
+    ArrayList<Map<String, Object>> alertList = new ArrayList<>();
+    alertList.add(alertOne);
+    alertList.add(alertTwo);
+
+    Map<String, Object> actualDocument = actual.getDocument();
+    assertNotNull(actualDocument.get(Fields.TIMESTAMP.getName()));
+    assertEquals(
+        alertList,
+        actualDocument.get(ALERT_FIELD)
+    );
+    assertEquals(
+        groups,
+        actualDocument.get(GROUPS_FIELD)
+    );
+
+    // Don't care about the result, just that it's a UUID. Exception will be thrown if not.
+    UUID.fromString((String) actualDocument.get(Constants.GUID));
+  }
+
+  // Utility method to manage comparing update maps
+  protected boolean updatesMapEquals(Map<Document, Optional<String>> expected,
+      Map<Document, Optional<String>> actual) {
+    Entry<Document, Optional<String>> expectedMetaEntry;
+    Entry<Document, Optional<String>> actualMetaEntry;
+
+    expectedMetaEntry = findMetaEntry(expected);
+    actualMetaEntry = findMetaEntry(actual);
+
+    // Compare the metaalerts directly: they can mess with comparison because of float scores.
+    if (!metaAlertDocumentEquals(expectedMetaEntry.getKey(), actualMetaEntry.getKey())) {
+      return false;
+    } else {
+      // Remove the potentially problematic metaalert comparison.
+      return removeMetaEntry(expected).equals(removeMetaEntry(actual));
+    }
+  }
+
+  protected Entry<Document, Optional<String>> findMetaEntry(
+      Map<Document, Optional<String>> expected) {
+    for (Entry<Document, Optional<String>> entry : expected.entrySet()) {
+      if (entry.getKey().getSensorType().equals(METAALERT_TYPE)) {
+        return entry;
+      }
+    }
+    return null;
+  }
+
+  // Unfortunately, the floating point comparison problem prevents direct remove call.
+  protected Map<Document, Optional<String>> removeMetaEntry(
+      Map<Document, Optional<String>> updates) {
+    Map<Document, Optional<String>> filteredUpdates = new HashMap<>();
+    for (Entry<Document, Optional<String>> entry : updates.entrySet()) {
+      if (!(entry.getKey().getSensorType().equals(METAALERT_TYPE))) {
+        filteredUpdates.put(entry.getKey(), entry.getValue());
+      }
+    }
+    return filteredUpdates;
+  }
+
+
+  // Utility method to ensure that the floating point values contained in a metaalert don't get
+  // incorrectly evaluated as not equal.
+  private boolean metaAlertDocumentEquals(Document expected, Document actual) {
+    if (!expected.getGuid().equals(actual.getGuid())) {
+      return false;
+    }
+    if (!expected.getSensorType().equals(actual.getSensorType())) {
+      return false;
+    }
+    if (!expected.getTimestamp().equals(actual.getTimestamp())) {
+      return false;
+    }
+
+    // The underlying documents have to be compared more thoroughly since it has floating point
+    Map<String, Object> expectedDocument = expected.getDocument();
+    Map<String, Object> actualDocument = actual.getDocument();
+
+    if (expectedDocument.size() != actualDocument.size()) {
+      return false;
+    }
+
+    for (Entry<String, Object> entry : expectedDocument.entrySet()) {
+      Object value = entry.getValue();
+      Object actualValue = actual.getDocument().get(entry.getKey());
+      if (value instanceof Float) {
+        if (!MathUtils.equals((Float) value, (Float) actualValue, EPS)) {
+          return false;
+        }
+      } else if (value instanceof Double) {
+        if (!MathUtils.equals((Double) value, (Double) actualValue, EPS)) {
+          return false;
+        }
+      } else {
+        if (!value.equals(actual.getDocument().get(entry.getKey()))) {
+          return false;
+        }
+      }
+    }
+
+    return true;
+  }
+
+  // Generate some child alerts.
+  protected List<Document> buildChildAlerts(int num, String parent, String guidPrefix) {
+    String prefix = guidPrefix != null ? guidPrefix : DEFAULT_PREFIX;
+    List<Document> alerts = new ArrayList<>();
+    for (int i = 0; i < num; i++) {
+      HashMap<String, Object> fields = new HashMap<>();
+      fields.put(Constants.GUID, prefix + i);
+      fields.put(THREAT_FIELD_DEFAULT, 0.0f);
+      if (parent != null) {
+        fields.put(METAALERT_FIELD, Collections.singletonList(parent));
+      } else {
+        fields.put(METAALERT_FIELD, new ArrayList<>());
+      }
+      alerts.add(new Document(fields, prefix + i, "test", 0L));
+    }
+    return alerts;
+  }
+
+  protected List<Map<String, Object>> getRawMaps(List<Document> documents) {
+    List<Map<String, Object>> rawMaps = new ArrayList<>();
+    for (Document document : documents) {
+      rawMaps.add(document.getDocument());
+    }
+    return rawMaps;
+  }
+
+  protected Document buildMetaAlert(List<Document> alerts) {
+    Map<String, Object> metaAlertMap = new HashMap<>();
+    metaAlertMap.put(ALERT_FIELD, getRawMaps(alerts));
+    metaAlertMap.put(Constants.GUID, METAALERT_GUID);
+    return new Document(
+        metaAlertMap,
+        METAALERT_GUID,
+        METAALERT_TYPE,
+        0L
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/integration/IndexingIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/integration/IndexingIntegrationTest.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/integration/IndexingIntegrationTest.java
index 1671ab3..5cb57d7 100644
--- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/integration/IndexingIntegrationTest.java
+++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/integration/IndexingIntegrationTest.java
@@ -52,12 +52,10 @@ public abstract class IndexingIntegrationTest extends BaseIntegrationTest {
   protected final int NUM_RETRIES = 100;
   protected final long TOTAL_TIME_MS = 150000L;
 
-  protected void preTest() {}
-
+  protected void preTest() { }
 
   @Test
   public void test() throws Exception {
-    preTest();
     final List<byte[]> inputMessages = TestUtils.readSampleData(sampleParsedPath);
     final Properties topologyProperties = new Properties() {{
       setProperty("indexing_kafka_start", "UNCOMMITTED_EARLIEST");

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-pcap-backend/.gitignore
----------------------------------------------------------------------
diff --git a/metron-platform/metron-pcap-backend/.gitignore b/metron-platform/metron-pcap-backend/.gitignore
new file mode 100644
index 0000000..df1a13b
--- /dev/null
+++ b/metron-platform/metron-pcap-backend/.gitignore
@@ -0,0 +1 @@
+/logs
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/pom.xml b/metron-platform/metron-solr/pom.xml
index 9b2e806..736fd15 100644
--- a/metron-platform/metron-solr/pom.xml
+++ b/metron-platform/metron-solr/pom.xml
@@ -31,7 +31,7 @@
         <dependency>
             <groupId>com.google.guava</groupId>
             <artifactId>guava</artifactId>
-            <version>${global_hbase_guava_version}</version>
+            <version>${global_guava_version}</version>
         </dependency>
         <dependency>
             <groupId>org.apache.solr</groupId>
@@ -300,7 +300,7 @@
                             <relocations>
                                 <relocation>
                                     <pattern>com.google.common</pattern>
-                                    <shadedPattern>org.apache.metron.guava</shadedPattern>
+                                    <shadedPattern>org.apache.metron.guava.metron-solr</shadedPattern>
                                 </relocation>
                                 <relocation>
                                     <pattern>com.fasterxml.jackson</pattern>

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/config/schema/bro/schema.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/config/schema/bro/schema.xml b/metron-platform/metron-solr/src/main/config/schema/bro/schema.xml
index b463366..ca69304 100644
--- a/metron-platform/metron-solr/src/main/config/schema/bro/schema.xml
+++ b/metron-platform/metron-solr/src/main/config/schema/bro/schema.xml
@@ -677,6 +677,9 @@
   <dynamicField name="*.reason" type="string" multiValued="false" docValues="true"/>
   <dynamicField name="*.name" type="string" multiValued="false" docValues="true"/>
 
+  <!-- Metaalerts Field -->
+  <field name="metaalerts" type="string" multiValued="true" indexed="true" stored="true"/>
+
   <!-- Catch all, if we don't know about it, it gets dropped. -->
   <dynamicField name="*" type="ignored" multiValued="false" docValues="true"/>
 

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/config/schema/metaalert/schema.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/config/schema/metaalert/schema.xml b/metron-platform/metron-solr/src/main/config/schema/metaalert/schema.xml
index e36c71e..63e729b 100644
--- a/metron-platform/metron-solr/src/main/config/schema/metaalert/schema.xml
+++ b/metron-platform/metron-solr/src/main/config/schema/metaalert/schema.xml
@@ -15,27 +15,44 @@
  See the License for the specific language governing permissions and
  limitations under the License.
 -->
+
 <schema name="metaalert_doc" version="1.6">
   <field name="_version_" type="plong" indexed="true" stored="true"/>
-  <field name="_root_" type="string" indexed="true" stored="false" docValues="false" />
-  <field name="guid" type="string" indexed="true" stored="true" required="true" multiValued="false" />
-  <field name="score" type="string" indexed="true" stored="true" />
-  <field name="status" type="string" indexed="true" stored="true" />
-  <field name="timestamp" type="timestamp" indexed="true" stored="true" />
-  <field name="source.type" type="string" indexed="true" stored="true" />
-  <dynamicField name="alert.*" type="string" multiValued="false" docValues="true"/>
-  <dynamicField name="*score" type="pfloat" multiValued="false" docValues="true"/>
-  <dynamicField name="*" type="ignored" multiValued="false" docValues="true"/>
+  <field name="_root_" type="string" indexed="true" stored="false" docValues="false"/>
+  <field name="_childDocuments_" type="ignored" stored="true" docValues="true"/>
+
+  <field name="guid" type="string" indexed="true" stored="true" required="true"
+    multiValued="false"/>
+
+  <field name="source.type" type="string" indexed="true" stored="true"/>
+  <field name="timestamp" type="plong" indexed="true" stored="true"/>
+  <field name="score" type="pdouble" indexed="true" stored="true"/>
+  <field name="status" type="string" indexed="true" stored="true"/>
+  <field name="threat:triage:score" type="pdouble" indexed="true" stored="true"/>
+  <field name="average" type="pdouble" indexed="true" stored="true"/>
+  <field name="min" type="pdouble" indexed="true" stored="true"/>
+  <field name="median" type="pdouble" indexed="true" stored="true"/>
+  <field name="max" type="pdouble" indexed="true" stored="true"/>
+  <field name="sum" type="pdouble" indexed="true" stored="true"/>
+  <field name="count" type="pint" indexed="true" stored="true"/>
+  <field name="groups" type="string" indexed="true" stored="true" multiValued="true"/>
+
+  <!-- Ensure that metaalerts child field is multivalued -->
+  <field name="metaalerts" type="string" multiValued="true" indexed="true" stored="true"/>
+
+  <dynamicField name="*" type="ignored" indexed="true" stored="true" multiValued="false" docValues="true"/>
+
   <uniqueKey>guid</uniqueKey>
+
+  <!-- Type Definitions -->
   <fieldType name="string" stored="true" indexed="true" multiValued="false" class="solr.StrField" sortMissingLast="true" docValues="false"/>
   <fieldType name="boolean" stored="true" indexed="true" multiValued="false" class="solr.BoolField" sortMissingLast="true" docValues="false"/>
   <fieldType name="pint" stored="true" indexed="true" multiValued="false" class="solr.TrieIntField" sortMissingLast="false" docValues="true"/>
   <fieldType name="pfloat" stored="true" indexed="true" multiValued="false" class="solr.TrieFloatField" sortMissingLast="false" docValues="true"/>
   <fieldType name="plong" stored="true" indexed="true" multiValued="false" class="solr.TrieLongField" sortMissingLast="false" docValues="true"/>
   <fieldType name="pdouble" stored="true" indexed="true" multiValued="false" class="solr.TrieDoubleField" sortMissingLast="false" docValues="true"/>
-  <fieldType name="bytes" stored="true" indexed="true" multiValued="false" class="solr.BinaryField" sortMissingLast="false" docValues="true"/>
   <fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
   <fieldType name="ip" stored="true" indexed="true" multiValued="false" class="solr.StrField" sortMissingLast="true" docValues="false"/>
   <fieldType name="timestamp" stored="true" indexed="true" multiValued="false" class="solr.TrieLongField" sortMissingLast="false" docValues="true"/>
   <fieldType name="ignored" stored="true" indexed="true" multiValued="true" class="solr.StrField" sortMissingLast="false" docValues="false"/>
-</schema>
+</schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/config/schema/snort/schema.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/config/schema/snort/schema.xml b/metron-platform/metron-solr/src/main/config/schema/snort/schema.xml
index 129c0f0..82d0320 100644
--- a/metron-platform/metron-solr/src/main/config/schema/snort/schema.xml
+++ b/metron-platform/metron-solr/src/main/config/schema/snort/schema.xml
@@ -70,6 +70,9 @@
   <dynamicField name="*.reason" type="string" multiValued="false" docValues="true"/>
   <dynamicField name="*.name" type="string" multiValued="false" docValues="true"/>
 
+  <!-- Metaalerts Field -->
+  <field name="metaalerts" type="string" multiValued="true" indexed="true" stored="true"/>
+
   <!-- Catch all, if we don't know about it, it gets dropped. -->
   <dynamicField name="*" type="ignored" multiValued="false" docValues="true"/>
 

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/config/schema/yaf/schema.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/config/schema/yaf/schema.xml b/metron-platform/metron-solr/src/main/config/schema/yaf/schema.xml
index f3abb14..fc8e641 100644
--- a/metron-platform/metron-solr/src/main/config/schema/yaf/schema.xml
+++ b/metron-platform/metron-solr/src/main/config/schema/yaf/schema.xml
@@ -76,6 +76,9 @@
   <dynamicField name="*.reason" type="string" multiValued="false" docValues="true"/>
   <dynamicField name="*.name" type="string" multiValued="false" docValues="true"/>
 
+  <!-- Metaalerts Field -->
+  <field name="metaalerts" type="string" multiValued="true" indexed="true" stored="true"/>
+
   <!-- Catch all, if we don't know about it, it gets dropped. -->
   <dynamicField name="*" type="ignored" multiValued="false" docValues="true"/>
 

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrDao.java b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrDao.java
index b53ae20..ee541eb 100644
--- a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrDao.java
+++ b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrDao.java
@@ -25,6 +25,7 @@ import java.util.Optional;
 import org.apache.metron.indexing.dao.AccessConfig;
 import org.apache.metron.indexing.dao.ColumnMetadataDao;
 import org.apache.metron.indexing.dao.IndexDao;
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
 import org.apache.metron.indexing.dao.search.FieldType;
 import org.apache.metron.indexing.dao.search.GetRequest;
 import org.apache.metron.indexing.dao.search.GroupRequest;
@@ -33,6 +34,8 @@ 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.indexing.dao.update.OriginalNotFoundException;
+import org.apache.metron.indexing.dao.update.PatchRequest;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
@@ -49,6 +52,7 @@ public class SolrDao implements IndexDao {
   private transient SolrClient client;
   private SolrSearchDao solrSearchDao;
   private SolrUpdateDao solrUpdateDao;
+  private SolrRetrieveLatestDao solrRetrieveLatestDao;
   private ColumnMetadataDao solrColumnMetadataDao;
 
   private AccessConfig accessConfig;
@@ -57,11 +61,13 @@ public class SolrDao implements IndexDao {
       AccessConfig config,
       SolrSearchDao solrSearchDao,
       SolrUpdateDao solrUpdateDao,
+      SolrRetrieveLatestDao retrieveLatestDao,
       SolrColumnMetadataDao solrColumnMetadataDao) {
     this.client = client;
     this.accessConfig = config;
     this.solrSearchDao = solrSearchDao;
     this.solrUpdateDao = solrUpdateDao;
+    this.solrRetrieveLatestDao = retrieveLatestDao;
     this.solrColumnMetadataDao = solrColumnMetadataDao;
   }
 
@@ -80,7 +86,8 @@ public class SolrDao implements IndexDao {
       this.client = getSolrClient(zkHost);
       this.accessConfig = config;
       this.solrSearchDao = new SolrSearchDao(this.client, this.accessConfig);
-      this.solrUpdateDao = new SolrUpdateDao(this.client);
+      this.solrUpdateDao = new SolrUpdateDao(this.client, this.accessConfig);
+      this.solrRetrieveLatestDao = new SolrRetrieveLatestDao(this.client);
       this.solrColumnMetadataDao = new SolrColumnMetadataDao(zkHost);
     }
   }
@@ -97,12 +104,12 @@ public class SolrDao implements IndexDao {
 
   @Override
   public Document getLatest(String guid, String collection) throws IOException {
-    return this.solrSearchDao.getLatest(guid, collection);
+    return this.solrRetrieveLatestDao.getLatest(guid, collection);
   }
 
   @Override
   public Iterable<Document> getAllLatest(List<GetRequest> getRequests) throws IOException {
-    return this.solrSearchDao.getAllLatest(getRequests);
+    return this.solrRetrieveLatestDao.getAllLatest(getRequests);
   }
 
   @Override
@@ -116,15 +123,35 @@ public class SolrDao implements IndexDao {
   }
 
   @Override
+  public void patch(RetrieveLatestDao retrieveLatestDao, PatchRequest request,
+      Optional<Long> timestamp)
+      throws OriginalNotFoundException, IOException {
+    solrUpdateDao.patch(retrieveLatestDao, request, timestamp);
+  }
+
+  @Override
   public Map<String, FieldType> getColumnMetadata(List<String> indices) throws IOException {
     return this.solrColumnMetadataDao.getColumnMetadata(indices);
   }
 
-  protected SolrClient getSolrClient(String zkHost) {
+  public SolrClient getSolrClient(String zkHost) {
     return new CloudSolrClient.Builder().withZkHost(zkHost).build();
   }
 
-  protected void enableKerberos() {
+  public String getZkHost() {
+    Map<String, Object> globalConfig = accessConfig.getGlobalConfigSupplier().get();
+    return (String) globalConfig.get("solr.zookeeper");
+  }
+
+  void enableKerberos() {
     HttpClientUtil.addConfigurer(new Krb5HttpClientConfigurer());
   }
+
+  public SolrSearchDao getSolrSearchDao() {
+    return solrSearchDao;
+  }
+
+  public SolrSearchDao getSolrUpdateDao() {
+    return solrSearchDao;
+  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertDao.java b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertDao.java
index 389cb4e..ca4a172 100644
--- a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertDao.java
+++ b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertDao.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -15,112 +15,211 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.metron.solr.dao;
 
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.metron.common.Constants;
 import org.apache.metron.indexing.dao.AccessConfig;
 import org.apache.metron.indexing.dao.IndexDao;
-import org.apache.metron.indexing.dao.MetaAlertDao;
 import org.apache.metron.indexing.dao.MultiIndexDao;
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConfig;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateResponse;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertDao;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
-import org.apache.metron.indexing.dao.search.*;
+import org.apache.metron.indexing.dao.search.FieldType;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.search.GroupRequest;
+import org.apache.metron.indexing.dao.search.GroupResponse;
+import org.apache.metron.indexing.dao.search.InvalidCreateException;
+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.Optional;
+import org.apache.metron.indexing.dao.update.OriginalNotFoundException;
+import org.apache.metron.indexing.dao.update.PatchRequest;
+import org.apache.solr.client.solrj.SolrClient;
 
 public class SolrMetaAlertDao implements MetaAlertDao {
 
-    private SolrDao solrDao;
-
-    @Override
-    public SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException {
-        return null;
-    }
-
-    @Override
-    public MetaAlertCreateResponse createMetaAlert(MetaAlertCreateRequest request) throws InvalidCreateException, IOException {
-        return null;
-    }
-
-    @Override
-    public boolean addAlertsToMetaAlert(String metaAlertGuid, List<GetRequest> getRequests) throws IOException {
-        return false;
-    }
-
-    @Override
-    public boolean removeAlertsFromMetaAlert(String metaAlertGuid, List<GetRequest> getRequests) throws IOException {
-        return false;
-    }
-
-    @Override
-    public boolean updateMetaAlertStatus(String metaAlertGuid, MetaAlertStatus status) throws IOException {
-        return false;
-    }
-
-    @Override
-    public void init(IndexDao indexDao) {
-
-    }
-
-    @Override
-    public void init(IndexDao indexDao, Optional<String> threatSort) {
-        if (indexDao instanceof MultiIndexDao) {
-            MultiIndexDao multiIndexDao = (MultiIndexDao) indexDao;
-            for (IndexDao childDao : multiIndexDao.getIndices()) {
-                if (childDao instanceof SolrDao) {
-                    this.solrDao = (SolrDao) childDao;
-                }
-            }
-        } else if (indexDao instanceof SolrDao) {
-            this.solrDao = (SolrDao) indexDao;
-        } else {
-            throw new IllegalArgumentException(
-                    "Need an SolrDao when using SolrMetaAlertDao"
-            );
+  public static final String METAALERTS_COLLECTION = "metaalert";
+
+  private IndexDao indexDao;
+  private SolrDao solrDao;
+  private SolrMetaAlertSearchDao metaAlertSearchDao;
+  private SolrMetaAlertUpdateDao metaAlertUpdateDao;
+  private SolrMetaAlertRetrieveLatestDao metaAlertRetrieveLatestDao;
+  protected String metaAlertsCollection = METAALERTS_COLLECTION;
+  protected String threatTriageField = MetaAlertConstants.THREAT_FIELD_DEFAULT;
+  protected String threatSort = MetaAlertConstants.THREAT_SORT_DEFAULT;
+
+  /**
+   * Wraps an {@link org.apache.metron.indexing.dao.IndexDao} to handle meta alerts.
+   * @param indexDao The Dao to wrap
+   */
+  public SolrMetaAlertDao(IndexDao indexDao, SolrMetaAlertSearchDao metaAlertSearchDao,
+      SolrMetaAlertUpdateDao metaAlertUpdateDao,
+      SolrMetaAlertRetrieveLatestDao metaAlertRetrieveLatestDao) {
+    this(indexDao, metaAlertSearchDao, metaAlertUpdateDao, metaAlertRetrieveLatestDao,
+        METAALERTS_COLLECTION,
+        MetaAlertConstants.THREAT_FIELD_DEFAULT,
+        MetaAlertConstants.THREAT_SORT_DEFAULT);
+  }
+
+  /**
+   * Wraps an {@link org.apache.metron.indexing.dao.IndexDao} to handle meta alerts.
+   * @param indexDao The Dao to wrap
+   * @param triageLevelField The field name to use as the threat scoring field
+   * @param threatSort The summary aggregation of all child threat triage scores used
+   *                   as the overall threat triage score for the metaalert. This
+   *                   can be either max, min, average, count, median, or sum.
+   */
+  public SolrMetaAlertDao(IndexDao indexDao, SolrMetaAlertSearchDao metaAlertSearchDao,
+      SolrMetaAlertUpdateDao metaAlertUpdateDao,
+      SolrMetaAlertRetrieveLatestDao metaAlertRetrieveLatestDao,
+      String metaAlertsCollection,
+      String triageLevelField,
+      String threatSort) {
+    init(indexDao, Optional.of(threatSort));
+    this.metaAlertSearchDao = metaAlertSearchDao;
+    this.metaAlertUpdateDao = metaAlertUpdateDao;
+    this.metaAlertRetrieveLatestDao = metaAlertRetrieveLatestDao;
+    this.metaAlertsCollection = metaAlertsCollection;
+    this.threatTriageField = triageLevelField;
+    this.threatSort = threatSort;
+  }
+
+  public SolrMetaAlertDao() {
+    //uninitialized.
+  }
+
+  /**
+   * Initializes this implementation by setting the supplied IndexDao and also setting a separate SolrDao.
+   * This is needed for some specific Solr functions (looking up an index from a GUID for example).
+   * @param indexDao The DAO to wrap for our queries
+   * @param threatSort The summary aggregation of the child threat triage scores used
+   *                   as the overall threat triage score for the metaalert. This
+   *                   can be either max, min, average, count, median, or sum.
+   */
+  @Override
+  public void init(IndexDao indexDao, Optional<String> threatSort) {
+    if (indexDao instanceof MultiIndexDao) {
+      this.indexDao = indexDao;
+      MultiIndexDao multiIndexDao = (MultiIndexDao) indexDao;
+      for (IndexDao childDao : multiIndexDao.getIndices()) {
+        if (childDao instanceof SolrDao) {
+          this.solrDao = (SolrDao) childDao;
         }
+      }
+    } else if (indexDao instanceof SolrDao) {
+      this.indexDao = indexDao;
+      this.solrDao = (SolrDao) indexDao;
+    } else {
+      throw new IllegalArgumentException(
+          "Need a SolrDao when using SolrMetaAlertDao"
+      );
     }
 
-    @Override
-    public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException {
-        return solrDao.search(searchRequest);
-    }
-
-    @Override
-    public GroupResponse group(GroupRequest groupRequest) throws InvalidSearchException {
-        return solrDao.group(groupRequest);
-    }
-
-    @Override
-    public void init(AccessConfig config) {
-
-    }
-
-    @Override
-    public Document getLatest(String guid, String sensorType) throws IOException {
-        return solrDao.getLatest(guid, sensorType);
-    }
-
-    @Override
-    public Iterable<Document> getAllLatest(List<GetRequest> getRequests) throws IOException {
-        return solrDao.getAllLatest(getRequests);
-    }
-
-    @Override
-    public void update(Document update, Optional<String> index) throws IOException {
-        solrDao.update(update, index);
-    }
-
-    @Override
-    public void batchUpdate(Map<Document, Optional<String>> updates) throws IOException {
-        solrDao.batchUpdate(updates);
-    }
-
-    @Override
-    public Map<String, FieldType> getColumnMetadata(List<String> indices) throws IOException {
-        return solrDao.getColumnMetadata(indices);
+    MetaAlertConfig config = new MetaAlertConfig(
+        metaAlertsCollection,
+        threatTriageField,
+        this.threatSort,
+        Constants.SENSOR_TYPE
+    );
+
+    SolrClient solrClient = solrDao.getSolrClient(solrDao.getZkHost());
+    this.metaAlertSearchDao = new SolrMetaAlertSearchDao(solrClient, solrDao.getSolrSearchDao());
+    this.metaAlertRetrieveLatestDao = new SolrMetaAlertRetrieveLatestDao(solrDao);
+    this.metaAlertUpdateDao = new SolrMetaAlertUpdateDao(
+        solrDao,
+        metaAlertSearchDao,
+        metaAlertRetrieveLatestDao,
+        config);
+
+    if (threatSort.isPresent()) {
+      this.threatSort = threatSort.get();
     }
+  }
+
+  @Override
+  public void init(AccessConfig config) {
+    // Do nothing. We're just wrapping a child dao
+  }
+
+  @Override
+  public Map<String, FieldType> getColumnMetadata(List<String> indices) throws IOException {
+    return indexDao.getColumnMetadata(indices);
+  }
+
+  @Override
+  public Document getLatest(String guid, String sensorType) throws IOException {
+    return metaAlertRetrieveLatestDao.getLatest(guid, sensorType);
+  }
+
+  @Override
+  public Iterable<Document> getAllLatest(List<GetRequest> getRequests) throws IOException {
+    return metaAlertRetrieveLatestDao.getAllLatest(getRequests);
+  }
+
+  @Override
+  public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException {
+    return metaAlertSearchDao.search(searchRequest);
+  }
+
+  @Override
+  public GroupResponse group(GroupRequest groupRequest) throws InvalidSearchException {
+    return metaAlertSearchDao.group(groupRequest);
+  }
+
+  @Override
+  public void update(Document update, Optional<String> index) throws IOException {
+    metaAlertUpdateDao.update(update, index);
+  }
+
+  @Override
+  public void batchUpdate(Map<Document, Optional<String>> updates) {
+    metaAlertUpdateDao.batchUpdate(updates);
+  }
+
+  @Override
+  public void patch(RetrieveLatestDao retrieveLatestDao, PatchRequest request,
+      Optional<Long> timestamp)
+      throws OriginalNotFoundException, IOException {
+    metaAlertUpdateDao.patch(retrieveLatestDao, request, timestamp);
+  }
+
+  @Override
+  public SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException {
+    return metaAlertSearchDao.getAllMetaAlertsForAlert(guid);
+  }
+
+  @Override
+  public MetaAlertCreateResponse createMetaAlert(MetaAlertCreateRequest request)
+      throws InvalidCreateException, IOException {
+    return metaAlertUpdateDao.createMetaAlert(request);
+  }
+
+  @Override
+  public boolean addAlertsToMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests)
+      throws IOException {
+    return metaAlertUpdateDao.addAlertsToMetaAlert(metaAlertGuid, alertRequests);
+  }
+
+  @Override
+  public boolean removeAlertsFromMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests)
+      throws IOException {
+    return metaAlertUpdateDao.removeAlertsFromMetaAlert(metaAlertGuid, alertRequests);
+  }
+
+  @Override
+  public boolean updateMetaAlertStatus(String metaAlertGuid, MetaAlertStatus status)
+      throws IOException {
+    return metaAlertUpdateDao.updateMetaAlertStatus(metaAlertGuid, status);
+  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertRetrieveLatestDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertRetrieveLatestDao.java b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertRetrieveLatestDao.java
new file mode 100644
index 0000000..7afe113
--- /dev/null
+++ b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertRetrieveLatestDao.java
@@ -0,0 +1,77 @@
+/*
+ * 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.solr.dao;
+
+import static org.apache.metron.solr.dao.SolrMetaAlertDao.METAALERTS_COLLECTION;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.metron.common.Constants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertRetrieveLatestDao;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.update.Document;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrDocument;
+
+public class SolrMetaAlertRetrieveLatestDao implements
+    MetaAlertRetrieveLatestDao {
+
+  private SolrDao solrDao;
+
+  public SolrMetaAlertRetrieveLatestDao(SolrDao solrDao) {
+    this.solrDao = solrDao;
+  }
+
+  @Override
+  public Document getLatest(String guid, String sensorType) throws IOException {
+    if (MetaAlertConstants.METAALERT_TYPE.equals(sensorType)) {
+      // Unfortunately, we can't just defer to the indexDao for this. Child alerts in Solr end up
+      // having to be dug out.
+      String guidClause = Constants.GUID + ":" + guid;
+      SolrQuery query = new SolrQuery();
+      query.setQuery(guidClause)
+          .setFields("*", "[child parentFilter=" + guidClause + " limit=999]");
+
+      try {
+        QueryResponse response = solrDao.getSolrClient(solrDao.getZkHost())
+            .query(METAALERTS_COLLECTION, query);
+        // GUID is unique, so it's definitely the first result
+        if (response.getResults().size() == 1) {
+          SolrDocument result = response.getResults().get(0);
+
+          return SolrUtilities.toDocument(result);
+        } else {
+          return null;
+        }
+      } catch (SolrServerException e) {
+        throw new IOException("Unable to retrieve metaalert", e);
+      }
+    } else {
+      return solrDao.getLatest(guid, sensorType);
+    }
+  }
+
+  @Override
+  public Iterable<Document> getAllLatest(List<GetRequest> getRequests) throws IOException {
+    return solrDao.getAllLatest(getRequests);
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertSearchDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertSearchDao.java b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertSearchDao.java
new file mode 100644
index 0000000..6b5b3a8
--- /dev/null
+++ b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertSearchDao.java
@@ -0,0 +1,211 @@
+/*
+ * 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.solr.dao;
+
+import static org.apache.metron.solr.dao.SolrMetaAlertDao.METAALERTS_COLLECTION;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.metron.common.Constants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertSearchDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
+import org.apache.metron.indexing.dao.search.GroupRequest;
+import org.apache.metron.indexing.dao.search.GroupResponse;
+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.search.SearchResult;
+import org.apache.metron.indexing.dao.update.Document;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.util.ClientUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.params.CursorMarkParams;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SolrMetaAlertSearchDao implements MetaAlertSearchDao {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  transient SolrSearchDao solrSearchDao;
+  transient SolrClient solrClient;
+
+  public SolrMetaAlertSearchDao(SolrClient solrClient, SolrSearchDao solrSearchDao) {
+    this.solrClient = solrClient;
+    this.solrSearchDao = solrSearchDao;
+  }
+
+  @Override
+  public SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException {
+    if (guid == null || guid.trim().isEmpty()) {
+      throw new InvalidSearchException("Guid cannot be empty");
+    }
+
+    // Searches for all alerts containing the meta alert guid in it's "metalerts" array
+    // The query has to match the parentFilter to avoid errors.  Guid must also be explicitly
+    // included.
+    String activeClause =
+        MetaAlertConstants.STATUS_FIELD + ":" + MetaAlertStatus.ACTIVE.getStatusString();
+    String guidClause = Constants.GUID + ":" + guid;
+    String fullClause = "{!parent which=" + activeClause + "}" + guidClause;
+    String metaalertTypeClause = Constants.SENSOR_TYPE + ":" + MetaAlertConstants.METAALERT_TYPE;
+    SolrQuery solrQuery = new SolrQuery()
+        .setQuery(fullClause)
+        .setFields("*", "[child parentFilter=" + metaalertTypeClause + " limit=999]")
+        .addSort(Constants.GUID,
+            SolrQuery.ORDER.asc); // Just do basic sorting to track where we are
+
+    // Use Solr's Cursors to handle the paging, rather than doing it manually.
+    List<SearchResult> allResults = new ArrayList<>();
+    try {
+      String cursorMark = CursorMarkParams.CURSOR_MARK_START;
+      boolean done = false;
+      while (!done) {
+        solrQuery.set(CursorMarkParams.CURSOR_MARK_PARAM, cursorMark);
+        QueryResponse rsp = solrClient.query(METAALERTS_COLLECTION, solrQuery);
+        String nextCursorMark = rsp.getNextCursorMark();
+        rsp.getResults().stream()
+            .map(solrDocument -> SolrUtilities.getSearchResult(solrDocument, null))
+            .forEachOrdered(allResults::add);
+        if (cursorMark.equals(nextCursorMark)) {
+          done = true;
+        }
+        cursorMark = nextCursorMark;
+      }
+    } catch (IOException | SolrServerException e) {
+      throw new InvalidSearchException("Unable to complete search", e);
+    }
+
+    SearchResponse searchResponse = new SearchResponse();
+    searchResponse.setResults(allResults);
+    searchResponse.setTotal(allResults.size());
+    return searchResponse;
+  }
+
+  @Override
+  public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException {
+    // Need to wrap such that two things are true
+    // 1. The provided query is true OR nested query on the alert field is true
+    // 2. Metaalert is active OR it's not a metaalert
+
+    String activeStatusClause =
+        MetaAlertConstants.STATUS_FIELD + ":" + MetaAlertStatus.ACTIVE.getStatusString();
+
+    String metaalertTypeClause = Constants.SENSOR_TYPE + ":" + MetaAlertConstants.METAALERT_TYPE;
+    // Use the 'v=' form in order to ensure complex clauses are properly handled.
+    // Per the docs, the 'which=' clause should be used to identify all metaalert parents, not to
+    //   filter
+    // Status is a filter on parents and must be done outside the '!parent' construct
+    String parentChildQuery =
+        "(+" + activeStatusClause + " +" + "{!parent which=" + metaalertTypeClause + " v='"
+            + searchRequest.getQuery() + "'})";
+
+    // Put everything together to get our full query
+    // The '-metaalert:[* TO *]' construct is to ensure the field doesn't exist on or is empty for
+    //   plain alerts.
+    // Also make sure that it's not a metaalert
+    String fullQuery =
+        "(" + searchRequest.getQuery() + " AND -" + MetaAlertConstants.METAALERT_FIELD + ":[* TO *]"
+            + " AND " + "-" + metaalertTypeClause + ")" + " OR " + parentChildQuery;
+
+    LOG.debug("MetaAlert search query {}", fullQuery);
+
+    searchRequest.setQuery(fullQuery);
+
+    // Build the custom field list
+    List<String> fields = searchRequest.getFields();
+    String fieldList = "*";
+    if (fields != null) {
+      fieldList = StringUtils.join(fields, ",");
+    }
+
+    LOG.debug("MetaAlert Search Field list {}", fullQuery);
+
+    SearchResponse results = solrSearchDao.search(searchRequest, fieldList);
+    LOG.debug("MetaAlert Search Number of results {}", results.getResults().size());
+
+    // Unfortunately, we can't get the full metaalert results at the same time
+    // Get them in a second query.
+    // However, we can only retrieve them if we have the source type field (either explicit or
+    // wildcard).
+    if (fieldList.contains("*") || fieldList.contains(Constants.SENSOR_TYPE)) {
+      List<String> metaalertGuids = new ArrayList<>();
+      for (SearchResult result : results.getResults()) {
+        if (result.getSource().get(Constants.SENSOR_TYPE)
+            .equals(MetaAlertConstants.METAALERT_TYPE)) {
+          // Then we need to add it to the list to retrieve child alerts in a second query.
+          metaalertGuids.add(result.getId());
+        }
+      }
+      LOG.debug("MetaAlert Search guids requiring retrieval: {}", metaalertGuids);
+
+      // If we have any metaalerts in our result, attach the full data.
+      if (metaalertGuids.size() > 0) {
+        Map<String, String> params = new HashMap<>();
+        params.put("fl", fieldList + ",[child parentFilter=" + metaalertTypeClause + " limit=999]");
+        SolrParams solrParams = new MapSolrParams(params);
+        try {
+          SolrDocumentList solrDocumentList = solrClient
+              .getById(METAALERTS_COLLECTION, metaalertGuids, solrParams);
+          Map<String, Document> guidToDocuments = new HashMap<>();
+          for (SolrDocument doc : solrDocumentList) {
+            Document document = SolrUtilities.toDocument(doc);
+            guidToDocuments.put(document.getGuid(), document);
+          }
+
+          // Run through our results and update them with the full metaalert
+          for (SearchResult result : results.getResults()) {
+            Document fullDoc = guidToDocuments.get(result.getId());
+            if (fullDoc != null) {
+              result.setSource(fullDoc.getDocument());
+            }
+          }
+        } catch (SolrServerException | IOException e) {
+          throw new InvalidSearchException("Error when retrieving child alerts for metaalerts", e);
+        }
+
+      }
+    }
+    return results;
+  }
+
+  @Override
+  public GroupResponse group(GroupRequest groupRequest) throws InvalidSearchException {
+    // Make sure to escape any problematic characters here
+    String sourceType = ClientUtils.escapeQueryChars(Constants.SENSOR_TYPE);
+    String baseQuery = groupRequest.getQuery();
+    String adjustedQuery = baseQuery + " -" + MetaAlertConstants.METAALERT_FIELD + ":[* TO *]"
+        + " -" + sourceType + ":" + MetaAlertConstants.METAALERT_TYPE;
+    LOG.debug("MetaAlert group adjusted query: {}", adjustedQuery);
+    groupRequest.setQuery(adjustedQuery);
+    return solrSearchDao.group(groupRequest);
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertUpdateDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertUpdateDao.java b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertUpdateDao.java
new file mode 100644
index 0000000..b00954a
--- /dev/null
+++ b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrMetaAlertUpdateDao.java
@@ -0,0 +1,216 @@
+/*
+ * 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.solr.dao;
+
+import static org.apache.metron.solr.dao.SolrMetaAlertDao.METAALERTS_COLLECTION;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.metron.common.Constants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConfig;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateResponse;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertUpdateDao;
+import org.apache.metron.indexing.dao.metaalert.MetaScores;
+import org.apache.metron.indexing.dao.metaalert.lucene.AbstractLuceneMetaAlertUpdateDao;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.search.InvalidCreateException;
+import org.apache.metron.indexing.dao.search.InvalidSearchException;
+import org.apache.metron.indexing.dao.search.SearchResponse;
+import org.apache.metron.indexing.dao.search.SearchResult;
+import org.apache.metron.indexing.dao.update.Document;
+import org.apache.metron.indexing.dao.update.UpdateDao;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrServerException;
+
+public class SolrMetaAlertUpdateDao extends AbstractLuceneMetaAlertUpdateDao implements
+    MetaAlertUpdateDao, UpdateDao {
+
+  private SolrClient solrClient;
+  private SolrMetaAlertSearchDao metaAlertSearchDao;
+
+  /**
+   * Constructor a SolrMetaAlertUpdateDao
+   * @param solrDao An SolrDao to defer queries to.
+   * @param metaAlertSearchDao A MetaAlert aware search DAO used in retrieving items being mutated.
+   * @param retrieveLatestDao A RetrieveLatestDao for getting the current state of items being
+   *     mutated.
+   */
+  public SolrMetaAlertUpdateDao(SolrDao solrDao,
+      SolrMetaAlertSearchDao metaAlertSearchDao,
+      SolrMetaAlertRetrieveLatestDao retrieveLatestDao,
+      MetaAlertConfig config) {
+    super(solrDao, retrieveLatestDao, config);
+    this.solrClient = solrDao.getSolrClient(solrDao.getZkHost());
+    this.metaAlertSearchDao = metaAlertSearchDao;
+  }
+
+  @Override
+  public MetaAlertCreateResponse createMetaAlert(MetaAlertCreateRequest request)
+      throws InvalidCreateException, IOException {
+    List<GetRequest> alertRequests = request.getAlerts();
+    if (request.getAlerts().isEmpty()) {
+      throw new InvalidCreateException("MetaAlertCreateRequest must contain alerts");
+    }
+    if (request.getGroups().isEmpty()) {
+      throw new InvalidCreateException("MetaAlertCreateRequest must contain UI groups");
+    }
+
+    // Retrieve the documents going into the meta alert and build it
+    Iterable<Document> alerts = getRetrieveLatestDao().getAllLatest(alertRequests);
+
+    Document metaAlert = buildCreateDocument(alerts, request.getGroups(),
+        MetaAlertConstants.ALERT_FIELD);
+    MetaScores.calculateMetaScores(metaAlert, getConfig().getThreatTriageField(),
+        getConfig().getThreatSort());
+
+    // Add source type to be consistent with other sources and allow filtering
+    metaAlert.getDocument().put(Constants.SENSOR_TYPE, MetaAlertConstants.METAALERT_TYPE);
+
+    // Start a list of updates / inserts we need to run
+    Map<Document, Optional<String>> updates = new HashMap<>();
+    updates.put(metaAlert, Optional.of(METAALERTS_COLLECTION));
+
+    try {
+      // We need to update the associated alerts with the new meta alerts, making sure existing
+      // links are maintained.
+      Map<String, Optional<String>> guidToIndices = alertRequests.stream().collect(Collectors.toMap(
+          GetRequest::getGuid, GetRequest::getIndex));
+      Map<String, String> guidToSensorTypes = alertRequests.stream().collect(Collectors.toMap(
+          GetRequest::getGuid, GetRequest::getSensorType));
+      for (Document alert : alerts) {
+        if (addMetaAlertToAlert(metaAlert.getGuid(), alert)) {
+          // Use the index in the request if it exists
+          Optional<String> index = guidToIndices.get(alert.getGuid());
+          if (!index.isPresent()) {
+            index = Optional.ofNullable(guidToSensorTypes.get(alert.getGuid()));
+            if (!index.isPresent()) {
+              throw new IllegalArgumentException("Could not find index for " + alert.getGuid());
+            }
+          }
+          updates.put(alert, index);
+        }
+      }
+
+      // Kick off any updates.
+      update(updates);
+
+      MetaAlertCreateResponse createResponse = new MetaAlertCreateResponse();
+      createResponse.setCreated(true);
+      createResponse.setGuid(metaAlert.getGuid());
+      solrClient.commit(METAALERTS_COLLECTION);
+      return createResponse;
+    } catch (IOException | SolrServerException e) {
+      throw new InvalidCreateException("Unable to create meta alert", e);
+    }
+  }
+
+
+  /**
+   * Updates a document in Solr for a given collection.  Collection is not optional for Solr.
+   * @param update The update to be run
+   * @param collection The index to be updated. Mandatory for Solr
+   * @throws IOException Thrown when an error occurs during the write.
+   */
+  @Override
+  public void update(Document update, Optional<String> collection) throws IOException {
+    if (MetaAlertConstants.METAALERT_TYPE.equals(update.getSensorType())) {
+      // We've been passed an update to the meta alert.
+      throw new UnsupportedOperationException("Meta alerts cannot be directly updated");
+    }
+    // Index can't be optional, or it won't be committed
+
+    Map<Document, Optional<String>> updates = new HashMap<>();
+    updates.put(update, collection);
+
+    // We need to update an alert itself. It cannot be delegated in Solr; we need to retrieve all
+    // metaalerts and update the entire document for each.
+    SearchResponse searchResponse;
+    try {
+      searchResponse = metaAlertSearchDao.getAllMetaAlertsForAlert(update.getGuid());
+    } catch (InvalidSearchException e) {
+      throw new IOException("Unable to retrieve metaalerts for alert", e);
+    }
+
+    ArrayList<Document> metaAlerts = new ArrayList<>();
+    for (SearchResult searchResult : searchResponse.getResults()) {
+      Document doc = new Document(searchResult.getSource(), searchResult.getId(),
+          MetaAlertConstants.METAALERT_TYPE, 0L);
+      metaAlerts.add(doc);
+    }
+
+    for (Document metaAlert : metaAlerts) {
+      if (replaceAlertInMetaAlert(metaAlert, update)) {
+        updates.put(metaAlert, Optional.of(METAALERTS_COLLECTION));
+      }
+    }
+
+    // Run the alert's update
+    getUpdateDao().batchUpdate(updates);
+
+    try {
+      solrClient.commit(METAALERTS_COLLECTION);
+      if (collection.isPresent()) {
+        solrClient.commit(collection.get());
+      }
+    } catch (SolrServerException e) {
+      throw new IOException("Unable to update document", e);
+    }
+  }
+
+  protected boolean replaceAlertInMetaAlert(Document metaAlert, Document alert) {
+    boolean metaAlertUpdated = removeAlertsFromMetaAlert(metaAlert,
+        Collections.singleton(alert.getGuid()));
+    if (metaAlertUpdated) {
+      addAlertsToMetaAlert(metaAlert, Collections.singleton(alert));
+    }
+    return metaAlertUpdated;
+  }
+
+  @Override
+  public boolean addAlertsToMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests)
+      throws IOException {
+    boolean success;
+    Document metaAlert = getRetrieveLatestDao()
+        .getLatest(metaAlertGuid, MetaAlertConstants.METAALERT_TYPE);
+    if (MetaAlertStatus.ACTIVE.getStatusString()
+        .equals(metaAlert.getDocument().get(MetaAlertConstants.STATUS_FIELD))) {
+      Iterable<Document> alerts = getRetrieveLatestDao().getAllLatest(alertRequests);
+      Map<Document, Optional<String>> updates = buildAddAlertToMetaAlertUpdates(metaAlert, alerts);
+      update(updates);
+      success = updates.size() != 0;
+    } else {
+      throw new IllegalStateException("Adding alerts to an INACTIVE meta alert is not allowed");
+    }
+    try {
+      solrClient.commit(METAALERTS_COLLECTION);
+    } catch (SolrServerException e) {
+      throw new IOException("Unable to commit alerts to metaalert: " + metaAlertGuid, e);
+    }
+    return success;
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrRetrieveLatestDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrRetrieveLatestDao.java b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrRetrieveLatestDao.java
new file mode 100644
index 0000000..8578bfb
--- /dev/null
+++ b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrRetrieveLatestDao.java
@@ -0,0 +1,81 @@
+/*
+ * 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.solr.dao;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.update.Document;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+
+public class SolrRetrieveLatestDao implements RetrieveLatestDao {
+
+  private transient SolrClient client;
+
+  public SolrRetrieveLatestDao(SolrClient client) {
+    this.client = client;
+  }
+
+  @Override
+  public Document getLatest(String guid, String collection) throws IOException {
+    try {
+      SolrDocument solrDocument = client.getById(collection, guid);
+      if (solrDocument == null) {
+        return null;
+      }
+      return SolrUtilities.toDocument(solrDocument);
+    } catch (SolrServerException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public Iterable<Document> getAllLatest(List<GetRequest> getRequests) throws IOException {
+    Map<String, Collection<String>> collectionIdMap = new HashMap<>();
+    for (GetRequest getRequest : getRequests) {
+      Collection<String> ids = collectionIdMap
+          .getOrDefault(getRequest.getSensorType(), new HashSet<>());
+      ids.add(getRequest.getGuid());
+      collectionIdMap.put(getRequest.getSensorType(), ids);
+    }
+    try {
+      List<Document> documents = new ArrayList<>();
+      for (String collection : collectionIdMap.keySet()) {
+        SolrDocumentList solrDocumentList = client.getById(collectionIdMap.get(collection),
+            new SolrQuery().set("collection", collection));
+        documents.addAll(
+            solrDocumentList.stream().map(SolrUtilities::toDocument).collect(Collectors.toList()));
+      }
+      return documents;
+    } catch (SolrServerException e) {
+      throw new IOException(e);
+    }
+  }
+}


[7/7] metron git commit: METRON-1421 Create a SolrMetaAlertDao (justinleet) closes apache/metron#970

Posted by le...@apache.org.
METRON-1421 Create a SolrMetaAlertDao (justinleet) closes apache/metron#970


Project: http://git-wip-us.apache.org/repos/asf/metron/repo
Commit: http://git-wip-us.apache.org/repos/asf/metron/commit/49f851e0
Tree: http://git-wip-us.apache.org/repos/asf/metron/tree/49f851e0
Diff: http://git-wip-us.apache.org/repos/asf/metron/diff/49f851e0

Branch: refs/heads/feature/METRON-1416-upgrade-solr
Commit: 49f851e0b8c2ffa1cdd7c8f169bed3dfa07cf35c
Parents: eb33666
Author: justinleet <ju...@gmail.com>
Authored: Wed May 23 10:32:34 2018 -0400
Committer: leet <le...@apache.org>
Committed: Wed May 23 10:32:34 2018 -0400

----------------------------------------------------------------------
 metron-analytics/metron-profiler/.gitignore     |    1 +
 .../apache/metron/rest/config/IndexConfig.java  |   12 +-
 .../rest/service/impl/MetaAlertServiceImpl.java |    5 +-
 .../rest/service/impl/SearchServiceImpl.java    |    2 +-
 .../rest/service/impl/UpdateServiceImpl.java    |    2 +-
 .../MetaAlertControllerIntegrationTest.java     |    6 +-
 .../UpdateControllerIntegrationTest.java        |    4 +-
 .../elasticsearch/dao/ElasticsearchDao.java     |   65 +-
 .../dao/ElasticsearchMetaAlertDao.java          |  641 ++---------
 ...ElasticsearchMetaAlertRetrieveLatestDao.java |   44 +
 .../dao/ElasticsearchMetaAlertSearchDao.java    |  110 ++
 .../dao/ElasticsearchMetaAlertUpdateDao.java    |  219 ++++
 .../dao/ElasticsearchRetrieveLatestDao.java     |  151 +++
 .../dao/ElasticsearchSearchDao.java             |  102 --
 .../dao/ElasticsearchUpdateDao.java             |   10 +-
 .../elasticsearch/utils/ElasticsearchUtils.java |   64 ++
 .../elasticsearch/dao/ElasticsearchDaoTest.java |   70 +-
 .../dao/ElasticsearchMetaAlertDaoTest.java      |  164 +--
 .../ElasticsearchMetaAlertIntegrationTest.java  |  986 ++---------------
 .../ElasticsearchSearchIntegrationTest.java     |   64 +-
 .../ElasticsearchUpdateIntegrationTest.java     |   84 +-
 .../components/ElasticSearchComponent.java      |   26 +-
 metron-platform/metron-indexing/README.md       |    2 +-
 metron-platform/metron-indexing/pom.xml         |    8 +-
 .../metron/indexing/dao/AccessConfig.java       |   10 +
 .../apache/metron/indexing/dao/IndexDao.java    |  141 +--
 .../metron/indexing/dao/MetaAlertDao.java       |  154 ---
 .../metron/indexing/dao/RetrieveLatestDao.java  |   67 ++
 .../metaalert/DeferredMetaAlertIndexDao.java    |   42 +
 .../metaalert/MetaAlertAddRemoveRequest.java    |    1 -
 .../indexing/dao/metaalert/MetaAlertConfig.java |   74 ++
 .../dao/metaalert/MetaAlertConstants.java       |   30 +
 .../indexing/dao/metaalert/MetaAlertDao.java    |   77 ++
 .../metaalert/MetaAlertRetrieveLatestDao.java   |   25 +
 .../dao/metaalert/MetaAlertSearchDao.java       |   35 +
 .../dao/metaalert/MetaAlertUpdateDao.java       |  146 +++
 .../indexing/dao/metaalert/MetaScores.java      |   52 +-
 .../AbstractLuceneMetaAlertUpdateDao.java       |  334 ++++++
 .../metron/indexing/dao/search/SearchDao.java   |   22 +-
 .../indexing/dao/search/SearchResponse.java     |   10 +-
 .../metron/indexing/dao/update/PatchUtil.java   |   50 +
 .../metron/indexing/dao/update/UpdateDao.java   |   47 +
 .../metron/indexing/util/IndexingCacheUtil.java |   35 +
 .../indexing/dao/InMemoryMetaAlertDao.java      |   69 +-
 .../indexing/dao/SearchIntegrationTest.java     |   60 +-
 .../indexing/dao/UpdateIntegrationTest.java     |   87 +-
 .../dao/metaalert/MetaAlertIntegrationTest.java | 1012 ++++++++++++++++++
 .../indexing/dao/metaalert/MetaScoresTest.java  |   75 ++
 .../AbstractLuceneMetaAlertUpdateDaoTest.java   |  854 +++++++++++++++
 .../integration/IndexingIntegrationTest.java    |    4 +-
 metron-platform/metron-pcap-backend/.gitignore  |    1 +
 metron-platform/metron-solr/pom.xml             |    4 +-
 .../src/main/config/schema/bro/schema.xml       |    3 +
 .../src/main/config/schema/metaalert/schema.xml |   39 +-
 .../src/main/config/schema/snort/schema.xml     |    3 +
 .../src/main/config/schema/yaf/schema.xml       |    3 +
 .../org/apache/metron/solr/dao/SolrDao.java     |   37 +-
 .../metron/solr/dao/SolrMetaAlertDao.java       |  285 +++--
 .../dao/SolrMetaAlertRetrieveLatestDao.java     |   77 ++
 .../metron/solr/dao/SolrMetaAlertSearchDao.java |  211 ++++
 .../metron/solr/dao/SolrMetaAlertUpdateDao.java |  216 ++++
 .../metron/solr/dao/SolrRetrieveLatestDao.java  |   81 ++
 .../apache/metron/solr/dao/SolrSearchDao.java   |  127 +--
 .../apache/metron/solr/dao/SolrUpdateDao.java   |   51 +-
 .../apache/metron/solr/dao/SolrUtilities.java   |   92 ++
 .../org/apache/metron/solr/dao/SolrDaoTest.java |   61 +-
 .../metron/solr/dao/SolrMetaAlertDaoTest.java   |  137 +++
 .../metron/solr/dao/SolrSearchDaoTest.java      |  176 ++-
 .../metron/solr/dao/SolrUpdateDaoTest.java      |   19 +-
 .../metron/solr/dao/SolrUtilitiesTest.java      |   48 +
 .../SolrIndexingIntegrationTest.java            |    5 +-
 .../SolrMetaAlertIntegrationTest.java           |  397 +++++++
 .../integration/SolrSearchIntegrationTest.java  |   59 +-
 .../integration/SolrUpdateIntegrationTest.java  |   87 +-
 .../integration/components/SolrComponent.java   |   94 +-
 .../schema/SchemaValidationIntegrationTest.java |    9 +-
 .../metron/solr/writer/SolrWriterTest.java      |   23 +-
 .../resources/config/test/conf/managed-schema   |   84 +-
 78 files changed, 6051 insertions(+), 2733 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-analytics/metron-profiler/.gitignore
----------------------------------------------------------------------
diff --git a/metron-analytics/metron-profiler/.gitignore b/metron-analytics/metron-profiler/.gitignore
new file mode 100644
index 0000000..df1a13b
--- /dev/null
+++ b/metron-analytics/metron-profiler/.gitignore
@@ -0,0 +1 @@
+/logs
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/config/IndexConfig.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 635d1de..c432c6c 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
@@ -20,12 +20,14 @@ package org.apache.metron.rest.config;
 import static org.apache.metron.rest.MetronRestConstants.INDEX_DAO_IMPL;
 
 import java.util.Optional;
+import org.apache.metron.common.zookeeper.ConfigurationsCache;
 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;
-import org.apache.metron.indexing.dao.MetaAlertDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertDao;
+import org.apache.metron.indexing.util.IndexingCacheUtil;
 import org.apache.metron.rest.MetronRestConstants;
 import org.apache.metron.rest.RestException;
 import org.apache.metron.rest.service.GlobalConfigService;
@@ -34,10 +36,6 @@ import org.springframework.context.annotation.Bean;
 import org.springframework.context.annotation.Configuration;
 import org.springframework.core.env.Environment;
 
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Set;
-
 @Configuration
 public class IndexConfig {
 
@@ -45,6 +43,9 @@ public class IndexConfig {
   private GlobalConfigService globalConfigService;
 
   @Autowired
+  private ConfigurationsCache cache;
+
+  @Autowired
   private Environment environment;
 
   @Autowired
@@ -72,6 +73,7 @@ public class IndexConfig {
           throw new IllegalStateException("Unable to retrieve the global config.", e);
         }
       });
+      config.setIndexSupplier(IndexingCacheUtil.getIndexLookupFunction(cache));
       config.setTableProvider(TableProvider.create(hbaseProviderImpl, () -> new HTableProvider()));
       config.setKerberosEnabled(environment.getProperty(MetronRestConstants.KERBEROS_ENABLED_SPRING_PROPERTY, Boolean.class, false));
       if (indexDaoImpl == null) {

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/MetaAlertServiceImpl.java
----------------------------------------------------------------------
diff --git a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/MetaAlertServiceImpl.java b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/MetaAlertServiceImpl.java
index aafab24..3f9b3e4 100644
--- a/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/MetaAlertServiceImpl.java
+++ b/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/MetaAlertServiceImpl.java
@@ -19,16 +19,14 @@
 package org.apache.metron.rest.service.impl;
 
 import java.io.IOException;
-import java.util.Collection;
 import org.apache.metron.indexing.dao.IndexDao;
-import org.apache.metron.indexing.dao.MetaAlertDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertDao;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertAddRemoveRequest;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateResponse;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
 import org.apache.metron.indexing.dao.search.InvalidCreateException;
 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.rest.RestException;
 import org.apache.metron.rest.service.MetaAlertService;
@@ -48,7 +46,6 @@ public class MetaAlertServiceImpl implements MetaAlertService {
     this.environment = environment;
   }
 
-
   @Override
   public MetaAlertCreateResponse create(MetaAlertCreateRequest createRequest) throws RestException {
     try {

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/SearchServiceImpl.java
----------------------------------------------------------------------
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 21d158f..82b9c11 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
@@ -18,7 +18,7 @@
 package org.apache.metron.rest.service.impl;
 
 import static org.apache.metron.common.Constants.ERROR_TYPE;
-import static org.apache.metron.indexing.dao.MetaAlertDao.METAALERT_TYPE;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_TYPE;
 import static org.apache.metron.rest.MetronRestConstants.INDEX_WRITER_NAME;
 import static org.apache.metron.rest.MetronRestConstants.SEARCH_FACET_FIELDS_SPRING_PROPERTY;
 

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-interface/metron-rest/src/main/java/org/apache/metron/rest/service/impl/UpdateServiceImpl.java
----------------------------------------------------------------------
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 76ac75d..6a42248 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
@@ -44,7 +44,7 @@ public class UpdateServiceImpl implements UpdateService {
   @Override
   public void patch(PatchRequest request) throws RestException, OriginalNotFoundException {
     try {
-      dao.patch(request, Optional.of(System.currentTimeMillis()));
+      dao.patch(dao, request, Optional.of(System.currentTimeMillis()));
     } catch (Exception e) {
 
       throw new RestException(e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/MetaAlertControllerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/MetaAlertControllerIntegrationTest.java b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/MetaAlertControllerIntegrationTest.java
index 3e69e37..9200fd1 100644
--- a/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/MetaAlertControllerIntegrationTest.java
+++ b/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/MetaAlertControllerIntegrationTest.java
@@ -30,13 +30,10 @@ import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.
 import com.google.common.collect.ImmutableMap;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
 import org.adrianwalker.multilinestring.Multiline;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.metron.common.utils.JSONUtils;
 import org.apache.metron.indexing.dao.InMemoryMetaAlertDao;
-import org.apache.metron.indexing.dao.MetaAlertDao;
 import org.apache.metron.indexing.dao.SearchIntegrationTest;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertAddRemoveRequest;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest;
@@ -75,6 +72,7 @@ public class MetaAlertControllerIntegrationTest extends DaoControllerTest {
   private String metaalertUrl = "/api/v1/metaalert";
   private String user = "user";
   private String password = "password";
+  private String metaAlertIndex = "metaalert_index";
 
   /**
    {
@@ -111,7 +109,7 @@ public class MetaAlertControllerIntegrationTest extends DaoControllerTest {
     ImmutableMap<String, String> testData = ImmutableMap.of(
         "bro_index_2017.01.01.01", SearchIntegrationTest.broData,
         "snort_index_2017.01.01.01", SearchIntegrationTest.snortData,
-        MetaAlertDao.METAALERTS_INDEX, metaAlertData
+        metaAlertIndex, metaAlertData
     );
     loadTestData(testData);
   }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-interface/metron-rest/src/test/java/org/apache/metron/rest/controller/UpdateControllerIntegrationTest.java
----------------------------------------------------------------------
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 e8d00d3..e437325 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
@@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.metron.hbase.mock.MockHTable;
 import org.apache.metron.hbase.mock.MockHBaseTableProvider;
 import org.apache.metron.indexing.dao.HBaseDao;
-import org.apache.metron.indexing.dao.MetaAlertDao;
 import org.apache.metron.indexing.dao.SearchIntegrationTest;
 import org.apache.metron.rest.service.UpdateService;
 import org.junit.Assert;
@@ -72,6 +71,7 @@ public class UpdateControllerIntegrationTest extends DaoControllerTest {
   private String searchUrl = "/api/v1/search";
   private String user = "user";
   private String password = "password";
+  private String metaAlertIndex = "metaalert_index";
 
   /**
    {
@@ -121,7 +121,7 @@ public class UpdateControllerIntegrationTest extends DaoControllerTest {
     ImmutableMap<String, String> testData = ImmutableMap.of(
         "bro_index_2017.01.01.01", SearchIntegrationTest.broData,
         "snort_index_2017.01.01.01", SearchIntegrationTest.snortData,
-        MetaAlertDao.METAALERTS_INDEX, MetaAlertControllerIntegrationTest.metaAlertData
+        metaAlertIndex, MetaAlertControllerIntegrationTest.metaAlertData
     );
     loadTestData(testData);
   }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchDao.java
----------------------------------------------------------------------
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 a09086a..246de6a 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
@@ -24,8 +24,8 @@ import java.util.Map;
 import java.util.Optional;
 import org.apache.metron.elasticsearch.utils.ElasticsearchUtils;
 import org.apache.metron.indexing.dao.AccessConfig;
-import org.apache.metron.indexing.dao.ColumnMetadataDao;
 import org.apache.metron.indexing.dao.IndexDao;
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
 import org.apache.metron.indexing.dao.search.FieldType;
 import org.apache.metron.indexing.dao.search.GetRequest;
 import org.apache.metron.indexing.dao.search.GroupRequest;
@@ -34,6 +34,9 @@ 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.indexing.dao.update.OriginalNotFoundException;
+import org.apache.metron.indexing.dao.update.PatchRequest;
+import org.apache.metron.indexing.dao.update.ReplaceRequest;
 import org.elasticsearch.client.transport.TransportClient;
 import org.elasticsearch.index.query.QueryBuilder;
 import org.slf4j.Logger;
@@ -46,6 +49,7 @@ public class ElasticsearchDao implements IndexDao {
   private transient TransportClient client;
   private ElasticsearchSearchDao searchDao;
   private ElasticsearchUpdateDao updateDao;
+  private ElasticsearchRetrieveLatestDao retrieveLatestDao;
 
   /**
    * Retrieves column metadata about search indices.
@@ -63,12 +67,14 @@ public class ElasticsearchDao implements IndexDao {
       AccessConfig config,
       ElasticsearchSearchDao searchDao,
       ElasticsearchUpdateDao updateDao,
+      ElasticsearchRetrieveLatestDao retrieveLatestDao,
       ElasticsearchColumnMetadataDao columnMetadataDao,
       ElasticsearchRequestSubmitter requestSubmitter
-                             ) {
+  ) {
     this.client = client;
     this.searchDao = searchDao;
     this.updateDao = updateDao;
+    this.retrieveLatestDao = retrieveLatestDao;
     this.columnMetadataDao = columnMetadataDao;
     this.requestSubmitter = requestSubmitter;
     this.accessConfig = config;
@@ -78,32 +84,25 @@ public class ElasticsearchDao implements IndexDao {
     //uninitialized.
   }
 
-  public ElasticsearchDao columnMetadataDao(ElasticsearchColumnMetadataDao columnMetadataDao) {
-    this.columnMetadataDao = columnMetadataDao;
-    return this;
-  }
-
-  public ElasticsearchDao accessConfig(AccessConfig accessConfig) {
-    this.accessConfig = accessConfig;
-    return this;
-  }
-
   @Override
   public synchronized void init(AccessConfig config) {
-    if(this.client == null) {
-      this.client = ElasticsearchUtils.getClient(config.getGlobalConfigSupplier().get());
+    if (this.client == null) {
+      this.client = ElasticsearchUtils
+          .getClient(config.getGlobalConfigSupplier().get());
       this.accessConfig = config;
       this.columnMetadataDao = new ElasticsearchColumnMetadataDao(this.client.admin());
       this.requestSubmitter = new ElasticsearchRequestSubmitter(this.client);
-      this.searchDao = new ElasticsearchSearchDao(client, accessConfig, columnMetadataDao, requestSubmitter);
-      this.updateDao = new ElasticsearchUpdateDao(client, accessConfig, searchDao);
+      this.searchDao = new ElasticsearchSearchDao(client, accessConfig, columnMetadataDao,
+          requestSubmitter);
+      this.retrieveLatestDao = new ElasticsearchRetrieveLatestDao(client);
+      this.updateDao = new ElasticsearchUpdateDao(client, accessConfig, retrieveLatestDao);
     }
 
-    if(columnMetadataDao == null) {
+    if (columnMetadataDao == null) {
       throw new IllegalArgumentException("No ColumnMetadataDao available");
     }
 
-    if(requestSubmitter == null) {
+    if (requestSubmitter == null) {
       throw new IllegalArgumentException("No ElasticsearchRequestSubmitter available");
     }
   }
@@ -119,14 +118,14 @@ public class ElasticsearchDao implements IndexDao {
   }
 
   @Override
-  public Document getLatest(final String guid, final String sensorType) throws IOException {
-    return searchDao.getLatest(guid, sensorType);
+  public Document getLatest(final String guid, final String sensorType) {
+    return retrieveLatestDao.getLatest(guid, sensorType);
   }
 
   @Override
   public Iterable<Document> getAllLatest(
-      final List<GetRequest> getRequests) throws IOException {
-    return searchDao.getAllLatest(getRequests);
+      final List<GetRequest> getRequests) {
+    return retrieveLatestDao.getAllLatest(getRequests);
   }
 
   @Override
@@ -140,19 +139,37 @@ public class ElasticsearchDao implements IndexDao {
   }
 
   @Override
+  public void patch(RetrieveLatestDao retrieveLatestDao, PatchRequest request, Optional<Long> timestamp)
+      throws OriginalNotFoundException, IOException {
+    updateDao.patch(retrieveLatestDao, request, timestamp);
+  }
+
+  @Override
+  public void replace(ReplaceRequest request, Optional<Long> timestamp) throws IOException {
+    updateDao.replace(request, timestamp);
+  }
+
+  @Override
   public Map<String, FieldType> getColumnMetadata(List<String> indices) throws IOException {
     return this.columnMetadataDao.getColumnMetadata(indices);
   }
 
+  @Override
+  public Optional<Map<String, Object>> getLatestResult(GetRequest request) throws IOException {
+    return retrieveLatestDao.getLatestResult(request);
+  }
+
   protected Optional<String> getIndexName(String guid, String sensorType) {
     return updateDao.getIndexName(guid, sensorType);
   }
 
-  protected SearchResponse search(SearchRequest request, QueryBuilder queryBuilder) throws InvalidSearchException {
+  protected SearchResponse search(SearchRequest request, QueryBuilder queryBuilder)
+      throws InvalidSearchException {
     return searchDao.search(request, queryBuilder);
   }
 
-  protected GroupResponse group(GroupRequest groupRequest, QueryBuilder queryBuilder) throws InvalidSearchException {
+  protected GroupResponse group(GroupRequest groupRequest, QueryBuilder queryBuilder)
+      throws InvalidSearchException {
     return searchDao.group(groupRequest, queryBuilder);
   }
 

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDao.java
index 2311a2b..faec939 100644
--- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDao.java
+++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDao.java
@@ -18,29 +18,21 @@
 
 package org.apache.metron.elasticsearch.dao;
 
-import static org.apache.metron.common.Constants.GUID;
-import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
-import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery;
-import static org.elasticsearch.index.query.QueryBuilders.existsQuery;
-import static org.elasticsearch.index.query.QueryBuilders.nestedQuery;
-import static org.elasticsearch.index.query.QueryBuilders.termQuery;
-
-import com.fasterxml.jackson.databind.JsonNode;
 import java.io.IOException;
-import java.util.*;
-import java.util.Map.Entry;
-import java.util.stream.Collectors;
-import org.apache.commons.collections4.SetUtils;
-import org.apache.lucene.search.join.ScoreMode;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
 import org.apache.metron.common.Constants;
 import org.apache.metron.indexing.dao.AccessConfig;
 import org.apache.metron.indexing.dao.IndexDao;
-import org.apache.metron.indexing.dao.MetaAlertDao;
 import org.apache.metron.indexing.dao.MultiIndexDao;
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConfig;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateResponse;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertDao;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
-import org.apache.metron.indexing.dao.metaalert.MetaScores;
 import org.apache.metron.indexing.dao.search.FieldType;
 import org.apache.metron.indexing.dao.search.GetRequest;
 import org.apache.metron.indexing.dao.search.GroupRequest;
@@ -49,59 +41,36 @@ import org.apache.metron.indexing.dao.search.InvalidCreateException;
 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.search.SearchResult;
 import org.apache.metron.indexing.dao.update.Document;
-import org.elasticsearch.action.get.GetResponse;
-import org.elasticsearch.action.get.MultiGetItemResponse;
-import org.elasticsearch.action.get.MultiGetRequest.Item;
-import org.elasticsearch.action.get.MultiGetRequestBuilder;
-import org.elasticsearch.action.get.MultiGetResponse;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.action.search.SearchRequestBuilder;
-import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo;
-import org.elasticsearch.action.update.UpdateRequest;
-import org.elasticsearch.action.update.UpdateResponse;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.index.query.InnerHitBuilder;
-import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.index.query.QueryStringQueryBuilder;
-import org.elasticsearch.search.SearchHit;
 import org.apache.metron.indexing.dao.update.OriginalNotFoundException;
 import org.apache.metron.indexing.dao.update.PatchRequest;
-import org.apache.metron.stellar.common.utils.ConversionUtils;
-import org.elasticsearch.action.search.SearchRequestBuilder;
-import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.index.query.QueryStringQueryBuilder;
 
 public class ElasticsearchMetaAlertDao implements MetaAlertDao {
 
-  public static final String SOURCE_TYPE = Constants.SENSOR_TYPE.replace('.', ':');
-  private static final String STATUS_PATH = "/status";
-  private static final String ALERT_PATH = "/alert";
+  public static final String THREAT_TRIAGE_FIELD = MetaAlertConstants.THREAT_FIELD_DEFAULT
+      .replace('.', ':');
+  public static final String METAALERTS_INDEX = "metaalert_index";
+
+  public static final String SOURCE_TYPE_FIELD = Constants.SENSOR_TYPE.replace('.', ':');
+  protected String metaAlertsIndex = METAALERTS_INDEX;
+  protected String threatTriageField = THREAT_TRIAGE_FIELD;
+  protected String threatSort = MetaAlertConstants.THREAT_SORT_DEFAULT;
 
-  private IndexDao indexDao;
   private ElasticsearchDao elasticsearchDao;
-  private String index = METAALERTS_INDEX;
-  private String threatTriageField = THREAT_FIELD_DEFAULT;
+  private IndexDao indexDao;
+  private ElasticsearchMetaAlertSearchDao metaAlertSearchDao;
+  private ElasticsearchMetaAlertUpdateDao metaAlertUpdateDao;
+  private ElasticsearchMetaAlertRetrieveLatestDao metaAlertRetrieveLatestDao;
 
-  /**
-   * Defines which summary aggregation is used to represent the overall threat triage score for
-   * the metaalert. The summary aggregation is applied to the threat triage score of all child alerts.
-   *
-   * This overall score is primarily used for sorting; hence it is called the 'threatSort'.  This
-   * can be either max, min, average, count, median, or sum.
-   */
-  private String threatSort = THREAT_SORT_DEFAULT;
-  private int pageSize = 500;
+  protected int pageSize = 500;
 
   /**
    * Wraps an {@link org.apache.metron.indexing.dao.IndexDao} to handle meta alerts.
    * @param indexDao The Dao to wrap
    */
   public ElasticsearchMetaAlertDao(IndexDao indexDao) {
-    this(indexDao, METAALERTS_INDEX, THREAT_FIELD_DEFAULT, THREAT_SORT_DEFAULT);
+    this(indexDao, METAALERTS_INDEX, MetaAlertConstants.THREAT_FIELD_DEFAULT,
+        MetaAlertConstants.THREAT_SORT_DEFAULT);
   }
 
   /**
@@ -112,10 +81,13 @@ public class ElasticsearchMetaAlertDao implements MetaAlertDao {
    *                   as the overall threat triage score for the metaalert. This
    *                   can be either max, min, average, count, median, or sum.
    */
-  public ElasticsearchMetaAlertDao(IndexDao indexDao, String index, String triageLevelField, String threatSort) {
+  public ElasticsearchMetaAlertDao(IndexDao indexDao, String metaAlertsIndex,
+      String triageLevelField,
+      String threatSort) {
     init(indexDao, Optional.of(threatSort));
-    this.index = index;
     this.threatTriageField = triageLevelField;
+    this.threatSort = threatSort;
+    this.metaAlertsIndex = metaAlertsIndex;
   }
 
   public ElasticsearchMetaAlertDao() {
@@ -123,8 +95,10 @@ public class ElasticsearchMetaAlertDao implements MetaAlertDao {
   }
 
   /**
-   * Initializes this implementation by setting the supplied IndexDao and also setting a separate ElasticsearchDao.
-   * This is needed for some specific Elasticsearch functions (looking up an index from a GUID for example).
+   * Initializes this implementation by setting the supplied IndexDao and also setting a separate
+   *     ElasticsearchDao.
+   * This is needed for some specific Elasticsearch functions (looking up an index from a GUID for
+   *     example).
    * @param indexDao The DAO to wrap for our queries
    * @param threatSort The summary aggregation of the child threat triage scores used
    *                   as the overall threat triage score for the metaalert. This
@@ -152,6 +126,24 @@ public class ElasticsearchMetaAlertDao implements MetaAlertDao {
     if (threatSort.isPresent()) {
       this.threatSort = threatSort.get();
     }
+
+    MetaAlertConfig config = new MetaAlertConfig(
+        metaAlertsIndex,
+        threatTriageField,
+        this.threatSort,
+        ElasticsearchMetaAlertDao.SOURCE_TYPE_FIELD
+    );
+
+    this.metaAlertSearchDao = new ElasticsearchMetaAlertSearchDao(
+        elasticsearchDao,
+        config,
+        pageSize);
+    this.metaAlertRetrieveLatestDao = new ElasticsearchMetaAlertRetrieveLatestDao(indexDao);
+    this.metaAlertUpdateDao = new ElasticsearchMetaAlertUpdateDao(
+        elasticsearchDao,
+        metaAlertRetrieveLatestDao,
+        config,
+        pageSize);
   }
 
   @Override
@@ -160,551 +152,74 @@ public class ElasticsearchMetaAlertDao implements MetaAlertDao {
   }
 
   @Override
-  public SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException {
-    if (guid == null || guid.trim().isEmpty()) {
-      throw new InvalidSearchException("Guid cannot be empty");
-    }
-    // Searches for all alerts containing the meta alert guid in it's "metalerts" array
-    QueryBuilder qb = boolQuery()
-        .must(
-            nestedQuery(
-                ALERT_FIELD,
-                boolQuery()
-                    .must(termQuery(ALERT_FIELD + "." + GUID, guid)),
-                    ScoreMode.None
-            ).innerHit(new InnerHitBuilder())
-        )
-        .must(termQuery(STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString()));
-    return queryAllResults(qb);
+  public Map<String, FieldType> getColumnMetadata(List<String> indices) throws IOException {
+    return indexDao.getColumnMetadata(indices);
   }
 
   @Override
-  @SuppressWarnings("unchecked")
-  public MetaAlertCreateResponse createMetaAlert(MetaAlertCreateRequest request)
-      throws InvalidCreateException, IOException {
-    List<GetRequest> alertRequests = request.getAlerts();
-    if (request.getAlerts().isEmpty()) {
-      throw new InvalidCreateException("MetaAlertCreateRequest must contain alerts");
-    }
-    if (request.getGroups().isEmpty()) {
-      throw new InvalidCreateException("MetaAlertCreateRequest must contain UI groups");
-    }
-
-    // Retrieve the documents going into the meta alert and build it
-    Iterable<Document> alerts = indexDao.getAllLatest(alertRequests);
-
-    Document metaAlert = buildCreateDocument(alerts, request.getGroups());
-    calculateMetaScores(metaAlert);
-    // Add source type to be consistent with other sources and allow filtering
-    metaAlert.getDocument().put(SOURCE_TYPE, MetaAlertDao.METAALERT_TYPE);
-
-    // Start a list of updates / inserts we need to run
-    Map<Document, Optional<String>> updates = new HashMap<>();
-    updates.put(metaAlert, Optional.of(MetaAlertDao.METAALERTS_INDEX));
-
-    try {
-      // We need to update the associated alerts with the new meta alerts, making sure existing
-      // links are maintained.
-      Map<String, Optional<String>> guidToIndices = alertRequests.stream().collect(Collectors.toMap(
-          GetRequest::getGuid, GetRequest::getIndex));
-      Map<String, String> guidToSensorTypes = alertRequests.stream().collect(Collectors.toMap(
-          GetRequest::getGuid, GetRequest::getSensorType));
-      for (Document alert: alerts) {
-        if (addMetaAlertToAlert(metaAlert.getGuid(), alert)) {
-          // Use the index in the request if it exists
-          Optional<String> index = guidToIndices.get(alert.getGuid());
-          if (!index.isPresent()) {
-            // Look up the index from Elasticsearch if one is not supplied in the request
-            index = elasticsearchDao.getIndexName(alert.getGuid(), guidToSensorTypes.get(alert.getGuid()));
-            if (!index.isPresent()) {
-              throw new IllegalArgumentException("Could not find index for " + alert.getGuid());
-            }
-          }
-          updates.put(alert, index);
-        }
-      }
-
-      // Kick off any updates.
-      indexDaoUpdate(updates);
-
-      MetaAlertCreateResponse createResponse = new MetaAlertCreateResponse();
-      createResponse.setCreated(true);
-      createResponse.setGuid(metaAlert.getGuid());
-      return createResponse;
-    } catch (IOException ioe) {
-      throw new InvalidCreateException("Unable to create meta alert", ioe);
-    }
+  public Document getLatest(String guid, String sensorType) throws IOException {
+    return indexDao.getLatest(guid, sensorType);
   }
 
   @Override
-  public boolean addAlertsToMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests)
-      throws IOException {
-    Map<Document, Optional<String>> updates = new HashMap<>();
-    Document metaAlert = indexDao.getLatest(metaAlertGuid, METAALERT_TYPE);
-    if (MetaAlertStatus.ACTIVE.getStatusString().equals(metaAlert.getDocument().get(STATUS_FIELD))) {
-      Iterable<Document> alerts = indexDao.getAllLatest(alertRequests);
-      boolean metaAlertUpdated = addAlertsToMetaAlert(metaAlert, alerts);
-      if (metaAlertUpdated) {
-        calculateMetaScores(metaAlert);
-        updates.put(metaAlert, Optional.of(index));
-        for(Document alert: alerts) {
-          if (addMetaAlertToAlert(metaAlert.getGuid(), alert)) {
-            updates.put(alert, Optional.empty());
-          }
-        }
-        indexDaoUpdate(updates);
-      }
-      return metaAlertUpdated;
-    } else {
-      throw new IllegalStateException("Adding alerts to an INACTIVE meta alert is not allowed");
-    }
+  public Iterable<Document> getAllLatest(List<GetRequest> getRequests) throws IOException {
+    return indexDao.getAllLatest(getRequests);
   }
 
-  protected boolean addAlertsToMetaAlert(Document metaAlert, Iterable<Document> alerts) {
-    boolean alertAdded = false;
-    List<Map<String,Object>> currentAlerts = (List<Map<String, Object>>) metaAlert.getDocument().get(ALERT_FIELD);
-    Set<String> currentAlertGuids = currentAlerts.stream().map(currentAlert ->
-        (String) currentAlert.get(GUID)).collect(Collectors.toSet());
-    for (Document alert: alerts) {
-      String alertGuid = alert.getGuid();
-      // Only add an alert if it isn't already in the meta alert
-      if (!currentAlertGuids.contains(alertGuid)) {
-        currentAlerts.add(alert.getDocument());
-        alertAdded = true;
-      }
-    }
-    return alertAdded;
+  @Override
+  public SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException {
+    return metaAlertSearchDao.getAllMetaAlertsForAlert(guid);
   }
 
-  protected boolean addMetaAlertToAlert(String metaAlertGuid, Document alert) {
-    List<String> metaAlertField = new ArrayList<>();
-    List<String> alertField = (List<String>) alert.getDocument()
-        .get(MetaAlertDao.METAALERT_FIELD);
-    if (alertField != null) {
-      metaAlertField.addAll(alertField);
-    }
-    boolean metaAlertAdded = !metaAlertField.contains(metaAlertGuid);
-    if (metaAlertAdded) {
-      metaAlertField.add(metaAlertGuid);
-      alert.getDocument().put(MetaAlertDao.METAALERT_FIELD, metaAlertField);
-    }
-    return metaAlertAdded;
+  @Override
+  public MetaAlertCreateResponse createMetaAlert(MetaAlertCreateRequest request)
+      throws InvalidCreateException, IOException {
+    return metaAlertUpdateDao.createMetaAlert(request);
   }
 
   @Override
-  public boolean removeAlertsFromMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests)
+  public boolean addAlertsToMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests)
       throws IOException {
-    Map<Document, Optional<String>> updates = new HashMap<>();
-    Document metaAlert = indexDao.getLatest(metaAlertGuid, METAALERT_TYPE);
-    if (MetaAlertStatus.ACTIVE.getStatusString().equals(metaAlert.getDocument().get(STATUS_FIELD))) {
-      Iterable<Document> alerts = indexDao.getAllLatest(alertRequests);
-      Collection<String> alertGuids = alertRequests.stream().map(GetRequest::getGuid).collect(
-          Collectors.toList());
-      boolean metaAlertUpdated = removeAlertsFromMetaAlert(metaAlert, alertGuids);
-      if (metaAlertUpdated) {
-        calculateMetaScores(metaAlert);
-        updates.put(metaAlert, Optional.of(index));
-        for(Document alert: alerts) {
-          if (removeMetaAlertFromAlert(metaAlert.getGuid(), alert)) {
-            updates.put(alert, Optional.empty());
-          }
-        }
-        indexDaoUpdate(updates);
-      }
-      return metaAlertUpdated;
-    } else {
-      throw new IllegalStateException("Removing alerts from an INACTIVE meta alert is not allowed");
-    }
-
+    return metaAlertUpdateDao.addAlertsToMetaAlert(metaAlertGuid, alertRequests);
   }
 
-  protected boolean removeAlertsFromMetaAlert(Document metaAlert, Collection<String> alertGuids) {
-    List<Map<String,Object>> currentAlerts = (List<Map<String, Object>>) metaAlert.getDocument().get(ALERT_FIELD);
-    int previousSize = currentAlerts.size();
-    // Only remove an alert if it is in the meta alert
-    currentAlerts.removeIf(currentAlert -> alertGuids.contains((String) currentAlert.get(GUID)));
-    return currentAlerts.size() != previousSize;
-  }
-
-  protected boolean removeMetaAlertFromAlert(String metaAlertGuid, Document alert) {
-    List<String> metaAlertField = new ArrayList<>();
-    List<String> alertField = (List<String>) alert.getDocument()
-        .get(MetaAlertDao.METAALERT_FIELD);
-    if (alertField != null) {
-      metaAlertField.addAll(alertField);
-    }
-    boolean metaAlertRemoved = metaAlertField.remove(metaAlertGuid);
-    if (metaAlertRemoved) {
-      alert.getDocument().put(MetaAlertDao.METAALERT_FIELD, metaAlertField);
-    }
-    return metaAlertRemoved;
+  @Override
+  public boolean removeAlertsFromMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests)
+      throws IOException {
+    return metaAlertUpdateDao.removeAlertsFromMetaAlert(metaAlertGuid, alertRequests);
   }
 
   @Override
   public boolean updateMetaAlertStatus(String metaAlertGuid, MetaAlertStatus status)
       throws IOException {
-    Map<Document, Optional<String>> updates = new HashMap<>();
-    Document metaAlert = indexDao.getLatest(metaAlertGuid, METAALERT_TYPE);
-    String currentStatus = (String) metaAlert.getDocument().get(MetaAlertDao.STATUS_FIELD);
-    boolean metaAlertUpdated = !status.getStatusString().equals(currentStatus);
-    if (metaAlertUpdated) {
-      metaAlert.getDocument().put(MetaAlertDao.STATUS_FIELD, status.getStatusString());
-      updates.put(metaAlert, Optional.of(index));
-      List<GetRequest> getRequests = new ArrayList<>();
-      List<Map<String, Object>> currentAlerts = (List<Map<String, Object>>) metaAlert.getDocument()
-          .get(MetaAlertDao.ALERT_FIELD);
-      currentAlerts.stream().forEach(currentAlert -> {
-        getRequests.add(new GetRequest((String) currentAlert.get(GUID), (String) currentAlert.get(SOURCE_TYPE)));
-      });
-      Iterable<Document> alerts = indexDao.getAllLatest(getRequests);
-      for (Document alert : alerts) {
-        boolean metaAlertAdded = false;
-        boolean metaAlertRemoved = false;
-        // If we're making it active add add the meta alert guid for every alert.
-        if (MetaAlertStatus.ACTIVE.equals(status)) {
-          metaAlertAdded = addMetaAlertToAlert(metaAlert.getGuid(), alert);
-        }
-        // If we're making it inactive, remove the meta alert guid from every alert.
-        if (MetaAlertStatus.INACTIVE.equals(status)) {
-          metaAlertRemoved = removeMetaAlertFromAlert(metaAlert.getGuid(), alert);
-        }
-        if (metaAlertAdded || metaAlertRemoved) {
-          updates.put(alert, Optional.empty());
-        }
-      }
-    }
-    if (metaAlertUpdated) {
-      indexDaoUpdate(updates);
-    }
-    return metaAlertUpdated;
+    return metaAlertUpdateDao.updateMetaAlertStatus(metaAlertGuid, status);
   }
 
   @Override
   public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException {
-    // Wrap the query to also get any meta-alerts.
-    QueryBuilder qb = constantScoreQuery(boolQuery()
-        .must(boolQuery()
-            .should(new QueryStringQueryBuilder(searchRequest.getQuery()))
-            .should(nestedQuery(
-                ALERT_FIELD,
-                new QueryStringQueryBuilder(searchRequest.getQuery()),
-                ScoreMode.None
-                )
-            )
-        )
-        // Ensures that it's a meta alert with active status or that it's an alert (signified by
-        // having no status field)
-        .must(boolQuery()
-            .should(termQuery(MetaAlertDao.STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString()))
-            .should(boolQuery().mustNot(existsQuery(MetaAlertDao.STATUS_FIELD)))
-        )
-        .mustNot(existsQuery(MetaAlertDao.METAALERT_FIELD))
-    );
-    return elasticsearchDao.search(searchRequest, qb);
+    return metaAlertSearchDao.search(searchRequest);
   }
 
   @Override
-  public Document getLatest(String guid, String sensorType) throws IOException {
-    return indexDao.getLatest(guid, sensorType);
-  }
-
-  @Override
-  public Iterable<Document> getAllLatest(
-      List<GetRequest> getRequests) throws IOException {
-    return indexDao.getAllLatest(getRequests);
+  public GroupResponse group(GroupRequest groupRequest) throws InvalidSearchException {
+    return metaAlertSearchDao.group(groupRequest);
   }
 
   @Override
   public void update(Document update, Optional<String> index) throws IOException {
-    if (METAALERT_TYPE.equals(update.getSensorType())) {
-      // We've been passed an update to the meta alert.
-      throw new UnsupportedOperationException("Meta alerts cannot be directly updated");
-    } else {
-      Map<Document, Optional<String>> updates = new HashMap<>();
-      updates.put(update, index);
-      // We need to update an alert itself.  Only that portion of the update can be delegated.
-      // We still need to get meta alerts potentially associated with it and update.
-      Collection<Document> metaAlerts = getMetaAlertsForAlert(update.getGuid()).getResults().stream()
-          .map(searchResult -> new Document(searchResult.getSource(), searchResult.getId(), METAALERT_TYPE, 0L))
-          .collect(Collectors.toList());
-      // Each meta alert needs to be updated with the new alert
-      for (Document metaAlert : metaAlerts) {
-        replaceAlertInMetaAlert(metaAlert, update);
-        updates.put(metaAlert, Optional.of(METAALERTS_INDEX));
-      }
-
-      // Run the alert's update
-      indexDao.batchUpdate(updates);
-    }
-  }
-
-  protected boolean replaceAlertInMetaAlert(Document metaAlert, Document alert) {
-    boolean metaAlertUpdated = removeAlertsFromMetaAlert(metaAlert, Collections.singleton(alert.getGuid()));
-    if (metaAlertUpdated) {
-      addAlertsToMetaAlert(metaAlert, Collections.singleton(alert));
-    }
-    return metaAlertUpdated;
+    metaAlertUpdateDao.update(update, index);
   }
 
   @Override
-  public void batchUpdate(Map<Document, Optional<String>> updates) throws IOException {
-    throw new UnsupportedOperationException("Meta alerts do not allow for bulk updates");
+  public void batchUpdate(Map<Document, Optional<String>> updates) {
+    metaAlertUpdateDao.batchUpdate(updates);
   }
 
-  /**
-   * Does not allow patches on the "alerts" or "status" fields.  These fields must be updated with their
-   * dedicated methods.
-   *
-   * @param request The patch request
-   * @param timestamp Optionally a timestamp to set. If not specified then current time is used.
-   * @throws OriginalNotFoundException
-   * @throws IOException
-   */
   @Override
-  public void patch(PatchRequest request, Optional<Long> timestamp)
+  public void patch(RetrieveLatestDao retrieveLatestDao, PatchRequest request,
+      Optional<Long> timestamp)
       throws OriginalNotFoundException, IOException {
-    if (isPatchAllowed(request)) {
-      Document d = getPatchedDocument(request, timestamp);
-      indexDao.update(d, Optional.ofNullable(request.getIndex()));
-    } else {
-      throw new IllegalArgumentException("Meta alert patches are not allowed for /alert or /status paths.  "
-          + "Please use the add/remove alert or update status functions instead.");
-    }
-  }
-
-  protected boolean isPatchAllowed(PatchRequest request) {
-    if(request.getPatch() != null && !request.getPatch().isEmpty()) {
-      for(Map<String, Object> patch : request.getPatch()) {
-        Object pathObj = patch.get("path");
-        if(pathObj != null && pathObj instanceof String) {
-          String path = (String)pathObj;
-          if (STATUS_PATH.equals(path) || ALERT_PATH.equals(path)) {
-            return false;
-          }
-        }
-      }
-    }
-    return true;
-  }
-
-  /**
-   * Given an alert GUID, retrieve all associated meta alerts.
-   * @param alertGuid The GUID of the child alert
-   * @return The Elasticsearch response containing the meta alerts
-   */
-  protected SearchResponse getMetaAlertsForAlert(String alertGuid) {
-    QueryBuilder qb = boolQuery()
-        .must(
-            nestedQuery(
-                ALERT_FIELD,
-                boolQuery()
-                    .must(termQuery(ALERT_FIELD + "." + Constants.GUID, alertGuid)),
-                ScoreMode.None
-            ).innerHit(new InnerHitBuilder())
-        )
-        .must(termQuery(STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString()));
-    return queryAllResults(qb);
-  }
-
-  /**
-   * Elasticsearch queries default to 10 records returned.  Some internal queries require that all
-   * results are returned.  Rather than setting an arbitrarily high size, this method pages through results
-   * and returns them all in a single SearchResponse.
-   * @param qb
-   * @return
-   */
-  protected SearchResponse queryAllResults(QueryBuilder qb) {
-    SearchRequestBuilder searchRequestBuilder = elasticsearchDao
-        .getClient()
-        .prepareSearch(index)
-        .addStoredField("*")
-        .setFetchSource(true)
-        .setQuery(qb)
-        .setSize(pageSize);
-    org.elasticsearch.action.search.SearchResponse esResponse = searchRequestBuilder
-        .execute()
-        .actionGet();
-    List<SearchResult> allResults = getSearchResults(esResponse);
-    long total = esResponse.getHits().getTotalHits();
-    if (total > pageSize) {
-      int pages = (int) (total / pageSize) + 1;
-      for (int i = 1; i < pages; i++) {
-        int from = i * pageSize;
-        searchRequestBuilder.setFrom(from);
-        esResponse = searchRequestBuilder
-            .execute()
-            .actionGet();
-        allResults.addAll(getSearchResults(esResponse));
-      }
-    }
-    SearchResponse searchResponse = new SearchResponse();
-    searchResponse.setTotal(total);
-    searchResponse.setResults(allResults);
-    return searchResponse;
-  }
-
-  /**
-   * Transforms a list of Elasticsearch SearchHits to a list of SearchResults
-   * @param searchResponse
-   * @return
-   */
-  protected List<SearchResult> getSearchResults(org.elasticsearch.action.search.SearchResponse searchResponse) {
-    return Arrays.stream(searchResponse.getHits().getHits()).map(searchHit -> {
-          SearchResult searchResult = new SearchResult();
-          searchResult.setId(searchHit.getId());
-          searchResult.setSource(searchHit.getSource());
-          searchResult.setScore(searchHit.getScore());
-          searchResult.setIndex(searchHit.getIndex());
-          return searchResult;
-        }
-    ).collect(Collectors.toList());
-  }
-
-  /**
-   * Build the Document representing a meta alert to be created.
-   * @param alerts The Elasticsearch results for the meta alerts child documents
-   * @param groups The groups used to create this meta alert
-   * @return A Document representing the new meta alert
-   */
-  protected Document buildCreateDocument(Iterable<Document> alerts, List<String> groups) {
-    // Need to create a Document from the multiget. Scores will be calculated later
-    Map<String, Object> metaSource = new HashMap<>();
-    List<Map<String, Object>> alertList = new ArrayList<>();
-    for (Document alert: alerts) {
-      alertList.add(alert.getDocument());
-    }
-    metaSource.put(ALERT_FIELD, alertList);
-
-    // Add any meta fields
-    String guid = UUID.randomUUID().toString();
-    metaSource.put(GUID, guid);
-    metaSource.put(Constants.Fields.TIMESTAMP.getName(), System.currentTimeMillis());
-    metaSource.put(GROUPS_FIELD, groups);
-    metaSource.put(STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString());
-
-    return new Document(metaSource, guid, METAALERT_TYPE, System.currentTimeMillis());
-  }
-
-  /**
-   * Calls the single update variant if there's only one update, otherwise calls batch.
-   * @param updates The list of updates to run
-   * @throws IOException If there's an update error
-   */
-  protected void indexDaoUpdate(Map<Document, Optional<String>> updates) throws IOException {
-    if (updates.size() == 1) {
-      Entry<Document, Optional<String>> singleUpdate = updates.entrySet().iterator().next();
-      indexDao.update(singleUpdate.getKey(), singleUpdate.getValue());
-    } else if (updates.size() > 1) {
-      indexDao.batchUpdate(updates);
-    } // else we have no updates, so don't do anything
-  }
-
-
-
-  @SuppressWarnings("unchecked")
-  protected List<Map<String, Object>> getAllAlertsForMetaAlert(Document update) throws IOException {
-    Document latest = indexDao.getLatest(update.getGuid(), MetaAlertDao.METAALERT_TYPE);
-    if (latest == null) {
-      return new ArrayList<>();
-    }
-    List<String> guids = new ArrayList<>();
-    List<Map<String, Object>> latestAlerts = (List<Map<String, Object>>) latest.getDocument()
-        .get(MetaAlertDao.ALERT_FIELD);
-    for (Map<String, Object> alert : latestAlerts) {
-      guids.add((String) alert.get(Constants.GUID));
-    }
-
-    List<Map<String, Object>> alerts = new ArrayList<>();
-    QueryBuilder query = QueryBuilders.idsQuery().addIds(guids.toArray(new String[0]));
-    SearchRequestBuilder request = elasticsearchDao.getClient().prepareSearch()
-        .setQuery(query);
-    org.elasticsearch.action.search.SearchResponse response = request.get();
-    for (SearchHit hit : response.getHits().getHits()) {
-      alerts.add(hit.sourceAsMap());
-    }
-    return alerts;
-  }
-
-  /**
-   * Builds an update Document for updating the meta alerts list.
-   * @param alertGuid The GUID of the alert to update
-   * @param sensorType The sensor type to update
-   * @param metaAlertField The new metaAlertList to use
-   * @return The update Document
-   */
-  protected Document buildAlertUpdate(String alertGuid, String sensorType,
-      List<String> metaAlertField, Long timestamp) {
-    Document alertUpdate;
-    Map<String, Object> document = new HashMap<>();
-    document.put(MetaAlertDao.METAALERT_FIELD, metaAlertField);
-    alertUpdate = new Document(
-        document,
-        alertGuid,
-        sensorType,
-        timestamp
-    );
-    return alertUpdate;
-  }
-
-
-  @Override
-  public Map<String, FieldType> getColumnMetadata(List<String> indices)
-      throws IOException {
-    return indexDao.getColumnMetadata(indices);
-  }
-
-  @Override
-  public GroupResponse group(GroupRequest groupRequest) throws InvalidSearchException {
-    // Wrap the query to hide any alerts already contained in meta alerts
-    QueryBuilder qb = QueryBuilders.boolQuery()
-        .must(new QueryStringQueryBuilder(groupRequest.getQuery()))
-        .mustNot(existsQuery(MetaAlertDao.METAALERT_FIELD));
-    return elasticsearchDao.group(groupRequest, qb);
-  }
-
-  /**
-   * Calculate the meta alert scores for a Document.
-   * @param metaAlert The Document containing scores
-   * @return Set of score statistics
-   */
-  @SuppressWarnings("unchecked")
-  protected void calculateMetaScores(Document metaAlert) {
-    MetaScores metaScores = new MetaScores(new ArrayList<>());
-    List<Object> alertsRaw = ((List<Object>) metaAlert.getDocument().get(ALERT_FIELD));
-    if (alertsRaw != null && !alertsRaw.isEmpty()) {
-      ArrayList<Double> scores = new ArrayList<>();
-      for (Object alertRaw : alertsRaw) {
-        Map<String, Object> alert = (Map<String, Object>) alertRaw;
-        Double scoreNum = parseThreatField(alert.get(threatTriageField));
-        if (scoreNum != null) {
-          scores.add(scoreNum);
-        }
-      }
-      metaScores = new MetaScores(scores);
-    }
-
-    // add a summary (max, min, avg, ...) of all the threat scores from the child alerts
-    metaAlert.getDocument().putAll(metaScores.getMetaScores());
-
-    // add the overall threat score for the metaalert; one of the summary aggregations as defined by `threatSort`
-    Object threatScore = metaScores.getMetaScores().get(threatSort);
-
-    // add the threat score as a float; type needs to match the threat score field from each of the sensor indices
-    metaAlert.getDocument().put(threatTriageField, ConversionUtils.convert(threatScore, Float.class));
-  }
-
-  private Double parseThreatField(Object threatRaw) {
-    Double threat = null;
-    if (threatRaw instanceof Number) {
-      threat = ((Number) threatRaw).doubleValue();
-    } else if (threatRaw instanceof String) {
-      threat = Double.parseDouble((String) threatRaw);
-    }
-    return threat;
-  }
-
-  public int getPageSize() {
-    return pageSize;
+    metaAlertUpdateDao.patch(retrieveLatestDao, request, timestamp);
   }
 
   public void setPageSize(int pageSize) {

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertRetrieveLatestDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertRetrieveLatestDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertRetrieveLatestDao.java
new file mode 100644
index 0000000..8aa55d6
--- /dev/null
+++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertRetrieveLatestDao.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.metron.elasticsearch.dao;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertRetrieveLatestDao;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.update.Document;
+
+public class ElasticsearchMetaAlertRetrieveLatestDao implements MetaAlertRetrieveLatestDao {
+  private RetrieveLatestDao retrieveLatestDao;
+
+  public ElasticsearchMetaAlertRetrieveLatestDao(RetrieveLatestDao retrieveLatestDao) {
+    this.retrieveLatestDao = retrieveLatestDao;
+  }
+
+  @Override
+  public Document getLatest(String guid, String sensorType) throws IOException {
+    return retrieveLatestDao.getLatest(guid, sensorType);
+  }
+
+  @Override
+  public Iterable<Document> getAllLatest(List<GetRequest> getRequests) throws IOException {
+    return retrieveLatestDao.getAllLatest(getRequests);
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertSearchDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertSearchDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertSearchDao.java
new file mode 100644
index 0000000..00fc9d0
--- /dev/null
+++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertSearchDao.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.metron.elasticsearch.dao;
+
+import static org.apache.metron.common.Constants.GUID;
+import static org.apache.metron.elasticsearch.utils.ElasticsearchUtils.queryAllResults;
+import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
+import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery;
+import static org.elasticsearch.index.query.QueryBuilders.existsQuery;
+import static org.elasticsearch.index.query.QueryBuilders.nestedQuery;
+import static org.elasticsearch.index.query.QueryBuilders.termQuery;
+
+import org.apache.lucene.search.join.ScoreMode;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConfig;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertSearchDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
+import org.apache.metron.indexing.dao.search.GroupRequest;
+import org.apache.metron.indexing.dao.search.GroupResponse;
+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.elasticsearch.index.query.InnerHitBuilder;
+import org.elasticsearch.index.query.QueryBuilder;
+import org.elasticsearch.index.query.QueryBuilders;
+import org.elasticsearch.index.query.QueryStringQueryBuilder;
+
+public class ElasticsearchMetaAlertSearchDao implements MetaAlertSearchDao {
+
+  protected ElasticsearchDao elasticsearchDao;
+  private MetaAlertConfig config;
+  private int pageSize;
+
+  public ElasticsearchMetaAlertSearchDao(ElasticsearchDao elasticsearchDao,
+      MetaAlertConfig config, int pageSize) {
+    this.elasticsearchDao = elasticsearchDao;
+    this.config = config;
+    this.pageSize = pageSize;
+  }
+
+  @Override
+  public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException {
+    // Wrap the query to also get any meta-alerts.
+    QueryBuilder qb = constantScoreQuery(boolQuery()
+        .must(boolQuery()
+            .should(new QueryStringQueryBuilder(searchRequest.getQuery()))
+            .should(nestedQuery(
+                MetaAlertConstants.ALERT_FIELD,
+                new QueryStringQueryBuilder(searchRequest.getQuery()),
+                ScoreMode.None
+                )
+            )
+        )
+        // Ensures that it's a meta alert with active status or that it's an alert (signified by
+        // having no status field)
+        .must(boolQuery()
+            .should(termQuery(MetaAlertConstants.STATUS_FIELD,
+                MetaAlertStatus.ACTIVE.getStatusString()))
+            .should(boolQuery().mustNot(existsQuery(MetaAlertConstants.STATUS_FIELD)))
+        )
+        .mustNot(existsQuery(MetaAlertConstants.METAALERT_FIELD))
+    );
+    return elasticsearchDao.search(searchRequest, qb);
+  }
+
+  @Override
+  public GroupResponse group(GroupRequest groupRequest) throws InvalidSearchException {
+    // Wrap the query to hide any alerts already contained in meta alerts
+    QueryBuilder qb = QueryBuilders.boolQuery()
+        .must(new QueryStringQueryBuilder(groupRequest.getQuery()))
+        .mustNot(existsQuery(MetaAlertConstants.METAALERT_FIELD));
+    return elasticsearchDao.group(groupRequest, qb);
+  }
+
+  @Override
+  public SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException {
+    if (guid == null || guid.trim().isEmpty()) {
+      throw new InvalidSearchException("Guid cannot be empty");
+    }
+    // Searches for all alerts containing the meta alert guid in it's "metalerts" array
+    QueryBuilder qb = boolQuery()
+        .must(
+            nestedQuery(
+                MetaAlertConstants.ALERT_FIELD,
+                boolQuery()
+                    .must(termQuery(MetaAlertConstants.ALERT_FIELD + "." + GUID, guid)),
+                ScoreMode.None
+            ).innerHit(new InnerHitBuilder())
+        )
+        .must(termQuery(MetaAlertConstants.STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString()));
+    return queryAllResults(elasticsearchDao.getClient(), qb, config.getMetaAlertIndex(),
+        pageSize);
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertUpdateDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertUpdateDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertUpdateDao.java
new file mode 100644
index 0000000..6c709a6
--- /dev/null
+++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertUpdateDao.java
@@ -0,0 +1,219 @@
+/*
+ * 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.dao;
+
+import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
+import static org.elasticsearch.index.query.QueryBuilders.nestedQuery;
+import static org.elasticsearch.index.query.QueryBuilders.termQuery;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.lucene.search.join.ScoreMode;
+import org.apache.metron.common.Constants;
+import org.apache.metron.elasticsearch.utils.ElasticsearchUtils;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConfig;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateResponse;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertRetrieveLatestDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
+import org.apache.metron.indexing.dao.metaalert.MetaScores;
+import org.apache.metron.indexing.dao.metaalert.lucene.AbstractLuceneMetaAlertUpdateDao;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.search.InvalidCreateException;
+import org.apache.metron.indexing.dao.search.SearchResponse;
+import org.apache.metron.indexing.dao.update.Document;
+import org.elasticsearch.index.query.InnerHitBuilder;
+import org.elasticsearch.index.query.QueryBuilder;
+
+public class ElasticsearchMetaAlertUpdateDao extends AbstractLuceneMetaAlertUpdateDao {
+
+  private ElasticsearchDao elasticsearchDao;
+  private MetaAlertRetrieveLatestDao retrieveLatestDao;
+  private int pageSize;
+
+  /**
+   * Constructor an ElasticsearchMetaAlertUpdateDao
+   * @param elasticsearchDao An UpdateDao to defer queries to.
+   * @param retrieveLatestDao A RetrieveLatestDao for getting the current state of items being
+   *     mutated.
+   * @param config The meta alert config to use.
+   */
+  public ElasticsearchMetaAlertUpdateDao(
+      ElasticsearchDao elasticsearchDao,
+      MetaAlertRetrieveLatestDao retrieveLatestDao,
+      MetaAlertConfig config,
+      int pageSize
+  ) {
+    super(elasticsearchDao, retrieveLatestDao, config);
+    this.elasticsearchDao = elasticsearchDao;
+    this.retrieveLatestDao = retrieveLatestDao;
+    this.pageSize = pageSize;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public MetaAlertCreateResponse createMetaAlert(MetaAlertCreateRequest request)
+      throws InvalidCreateException, IOException {
+    List<GetRequest> alertRequests = request.getAlerts();
+    if (request.getAlerts().isEmpty()) {
+      throw new InvalidCreateException("MetaAlertCreateRequest must contain alerts");
+    }
+    if (request.getGroups().isEmpty()) {
+      throw new InvalidCreateException("MetaAlertCreateRequest must contain UI groups");
+    }
+
+    // Retrieve the documents going into the meta alert and build it
+    Iterable<Document> alerts = retrieveLatestDao.getAllLatest(alertRequests);
+
+    Document metaAlert = buildCreateDocument(alerts, request.getGroups(),
+        MetaAlertConstants.ALERT_FIELD);
+    MetaScores
+        .calculateMetaScores(metaAlert, getConfig().getThreatTriageField(),
+            getConfig().getThreatSort());
+    // Add source type to be consistent with other sources and allow filtering
+    metaAlert.getDocument()
+        .put(ElasticsearchMetaAlertDao.SOURCE_TYPE_FIELD, MetaAlertConstants.METAALERT_TYPE);
+
+    // Start a list of updates / inserts we need to run
+    Map<Document, Optional<String>> updates = new HashMap<>();
+    updates.put(metaAlert, Optional.of(getConfig().getMetaAlertIndex()));
+
+    try {
+      // We need to update the associated alerts with the new meta alerts, making sure existing
+      // links are maintained.
+      Map<String, Optional<String>> guidToIndices = alertRequests.stream().collect(Collectors.toMap(
+          GetRequest::getGuid, GetRequest::getIndex));
+      Map<String, String> guidToSensorTypes = alertRequests.stream().collect(Collectors.toMap(
+          GetRequest::getGuid, GetRequest::getSensorType));
+      for (Document alert : alerts) {
+        if (addMetaAlertToAlert(metaAlert.getGuid(), alert)) {
+          // Use the index in the request if it exists
+          Optional<String> index = guidToIndices.get(alert.getGuid());
+          if (!index.isPresent()) {
+            // Look up the index from Elasticsearch if one is not supplied in the request
+            index = elasticsearchDao
+                .getIndexName(alert.getGuid(), guidToSensorTypes.get(alert.getGuid()));
+            if (!index.isPresent()) {
+              throw new IllegalArgumentException("Could not find index for " + alert.getGuid());
+            }
+          }
+          updates.put(alert, index);
+        }
+      }
+
+      // Kick off any updates.
+      update(updates);
+
+      MetaAlertCreateResponse createResponse = new MetaAlertCreateResponse();
+      createResponse.setCreated(true);
+      createResponse.setGuid(metaAlert.getGuid());
+      return createResponse;
+    } catch (IOException ioe) {
+      throw new InvalidCreateException("Unable to create meta alert", ioe);
+    }
+  }
+
+  /**
+   * Adds alerts to a metaalert, based on a list of GetRequests provided for retrieval.
+   * @param metaAlertGuid The GUID of the metaalert to be given new children.
+   * @param alertRequests GetRequests for the appropriate alerts to add.
+   * @return True if metaalert is modified, false otherwise.
+   */
+  public boolean addAlertsToMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests)
+      throws IOException {
+
+    Document metaAlert = retrieveLatestDao
+        .getLatest(metaAlertGuid, MetaAlertConstants.METAALERT_TYPE);
+    if (MetaAlertStatus.ACTIVE.getStatusString()
+        .equals(metaAlert.getDocument().get(MetaAlertConstants.STATUS_FIELD))) {
+      Iterable<Document> alerts = retrieveLatestDao.getAllLatest(alertRequests);
+      Map<Document, Optional<String>> updates = buildAddAlertToMetaAlertUpdates(metaAlert, alerts);
+      update(updates);
+      return updates.size() != 0;
+    } else {
+      throw new IllegalStateException("Adding alerts to an INACTIVE meta alert is not allowed");
+    }
+  }
+
+  @Override
+  public void update(Document update, Optional<String> index) throws IOException {
+    if (MetaAlertConstants.METAALERT_TYPE.equals(update.getSensorType())) {
+      // We've been passed an update to the meta alert.
+      throw new UnsupportedOperationException("Meta alerts cannot be directly updated");
+    } else {
+      Map<Document, Optional<String>> updates = new HashMap<>();
+      updates.put(update, index);
+      // We need to update an alert itself.  Only that portion of the update can be delegated.
+      // We still need to get meta alerts potentially associated with it and update.
+      Collection<Document> metaAlerts = getMetaAlertsForAlert(update.getGuid()).getResults()
+          .stream()
+          .map(searchResult -> new Document(searchResult.getSource(), searchResult.getId(),
+              MetaAlertConstants.METAALERT_TYPE, 0L))
+          .collect(Collectors.toList());
+      // Each meta alert needs to be updated with the new alert
+      for (Document metaAlert : metaAlerts) {
+        if (replaceAlertInMetaAlert(metaAlert, update)) {
+          updates.put(metaAlert, Optional.of(getConfig().getMetaAlertIndex()));
+        }
+      }
+
+      // Run the alert's update
+      elasticsearchDao.batchUpdate(updates);
+    }
+  }
+
+  /**
+   * Given an alert GUID, retrieve all associated meta alerts.
+   * @param alertGuid The GUID of the child alert
+   * @return The Elasticsearch response containing the meta alerts
+   */
+  protected SearchResponse getMetaAlertsForAlert(String alertGuid) {
+    QueryBuilder qb = boolQuery()
+        .must(
+            nestedQuery(
+                MetaAlertConstants.ALERT_FIELD,
+                boolQuery()
+                    .must(termQuery(MetaAlertConstants.ALERT_FIELD + "." + Constants.GUID,
+                        alertGuid)),
+                ScoreMode.None
+            ).innerHit(new InnerHitBuilder())
+        )
+        .must(termQuery(MetaAlertConstants.STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString()));
+    return ElasticsearchUtils
+        .queryAllResults(elasticsearchDao.getClient(), qb, getConfig().getMetaAlertIndex(),
+            pageSize);
+  }
+
+
+  protected boolean replaceAlertInMetaAlert(Document metaAlert, Document alert) {
+    boolean metaAlertUpdated = removeAlertsFromMetaAlert(metaAlert,
+        Collections.singleton(alert.getGuid()));
+    if (metaAlertUpdated) {
+      addAlertsToMetaAlert(metaAlert, Collections.singleton(alert));
+    }
+    return metaAlertUpdated;
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java
new file mode 100644
index 0000000..f6bfeda
--- /dev/null
+++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchRetrieveLatestDao.java
@@ -0,0 +1,151 @@
+/*
+ * 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.dao;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.update.Document;
+import org.elasticsearch.action.search.SearchRequestBuilder;
+import org.elasticsearch.client.transport.TransportClient;
+import org.elasticsearch.index.query.IdsQueryBuilder;
+import org.elasticsearch.index.query.QueryBuilder;
+import org.elasticsearch.index.query.QueryBuilders;
+import org.elasticsearch.search.SearchHit;
+import org.elasticsearch.search.SearchHits;
+
+public class ElasticsearchRetrieveLatestDao implements RetrieveLatestDao {
+
+  private TransportClient transportClient;
+
+  public ElasticsearchRetrieveLatestDao(TransportClient transportClient) {
+    this.transportClient = transportClient;
+  }
+
+  @Override
+  public Document getLatest(String guid, String sensorType) {
+    Optional<Document> doc = searchByGuid(guid, sensorType, hit -> toDocument(guid, hit));
+    return doc.orElse(null);
+  }
+
+  @Override
+  public Iterable<Document> getAllLatest(List<GetRequest> getRequests) {
+    Collection<String> guids = new HashSet<>();
+    Collection<String> sensorTypes = new HashSet<>();
+    for (GetRequest getRequest : getRequests) {
+      guids.add(getRequest.getGuid());
+      sensorTypes.add(getRequest.getSensorType());
+    }
+    List<Document> documents = searchByGuids(
+        guids,
+        sensorTypes,
+        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, hit.getId(), sourceType, ts));
+          } catch (IOException e) {
+            throw new IllegalStateException("Unable to retrieve latest: " + e.getMessage(), e);
+          }
+        }
+
+    );
+    return documents;
+  }
+
+  <T> Optional<T> searchByGuid(String guid, String sensorType,
+      Function<SearchHit, Optional<T>> callback) {
+    Collection<String> sensorTypes = sensorType != null ? Collections.singleton(sensorType) : null;
+    List<T> results = searchByGuids(Collections.singleton(guid), sensorTypes, callback);
+    if (results.size() > 0) {
+      return Optional.of(results.get(0));
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  /**
+   * 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.
+   */
+  <T> List<T> searchByGuids(Collection<String> guids, Collection<String> sensorTypes,
+      Function<SearchHit, Optional<T>> callback) {
+    if (guids == null || guids.isEmpty()) {
+      return Collections.emptyList();
+    }
+    QueryBuilder query = null;
+    IdsQueryBuilder idsQuery;
+    if (sensorTypes != null) {
+      String[] types = sensorTypes.stream().map(sensorType -> sensorType + "_doc")
+          .toArray(String[]::new);
+      idsQuery = QueryBuilders.idsQuery(types);
+    } else {
+      idsQuery = QueryBuilders.idsQuery();
+    }
+
+    for (String guid : guids) {
+      query = idsQuery.addIds(guid);
+    }
+
+    SearchRequestBuilder request = transportClient.prepareSearch()
+        .setQuery(query)
+        .setSize(guids.size());
+    org.elasticsearch.action.search.SearchResponse response = request.get();
+    SearchHits hits = response.getHits();
+    List<T> results = new ArrayList<>();
+    for (SearchHit hit : hits) {
+      Optional<T> result = callback.apply(hit);
+      if (result.isPresent()) {
+        results.add(result.get());
+      }
+    }
+    return results;
+  }
+
+  private Optional<Document> toDocument(final String guid, SearchHit hit) {
+    Long ts = 0L;
+    String doc = hit.getSourceAsString();
+    String sourceType = toSourceType(hit.getType());
+    try {
+      return Optional.of(new Document(doc, guid, sourceType, ts));
+    } catch (IOException e) {
+      throw new IllegalStateException("Unable to retrieve latest: " + e.getMessage(), e);
+    }
+  }
+
+  /**
+   * Returns the source type based on a given doc type.
+   * @param docType The document type.
+   * @return The source type.
+   */
+  private String toSourceType(String docType) {
+    return Iterables.getFirst(Splitter.on("_doc").split(docType), null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchDao.java
index 3971237..5725534 100644
--- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchDao.java
+++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchSearchDao.java
@@ -116,49 +116,6 @@ public class ElasticsearchSearchDao implements SearchDao {
     return group(groupRequest, new QueryStringQueryBuilder(groupRequest.getQuery()));
   }
 
-  @Override
-  public Document getLatest(String guid, String sensorType) throws IOException {
-    Optional<Document> doc = searchByGuid(guid, sensorType, hit -> toDocument(guid, hit));
-    return doc.orElse(null);
-  }
-
-  <T> Optional<T> searchByGuid(String guid, String sensorType,
-      Function<SearchHit, Optional<T>> callback) {
-    Collection<String> sensorTypes = sensorType != null ? Collections.singleton(sensorType) : null;
-    List<T> results = searchByGuids(Collections.singleton(guid), sensorTypes, callback);
-    if (results.size() > 0) {
-      return Optional.of(results.get(0));
-    } else {
-      return Optional.empty();
-    }
-  }
-
-  @Override
-  public Iterable<Document> getAllLatest(List<GetRequest> getRequests) throws IOException {
-    Collection<String> guids = new HashSet<>();
-    Collection<String> sensorTypes = new HashSet<>();
-    for (GetRequest getRequest: getRequests) {
-      guids.add(getRequest.getGuid());
-      sensorTypes.add(getRequest.getSensorType());
-    }
-    List<Document> documents = searchByGuids(
-        guids
-        , sensorTypes
-        , 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, hit.getId(), sourceType, ts));
-          } catch (IOException e) {
-            throw new IllegalStateException("Unable to retrieve latest: " + e.getMessage(), e);
-          }
-        }
-
-    );
-    return documents;
-  }
-
   /**
    * Defers to a provided {@link org.elasticsearch.index.query.QueryBuilder} for the query.
    * @param request The request defining the parameters of the search
@@ -505,63 +462,4 @@ public class ElasticsearchSearchDao implements SearchDao {
     }
     return searchResultGroups;
   }
-
-  /**
-   * 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.
-   */
-  <T> List<T> searchByGuids(Collection<String> guids, Collection<String> sensorTypes,
-      Function<SearchHit, Optional<T>> callback) {
-    if(guids == null || guids.isEmpty()) {
-      return Collections.EMPTY_LIST;
-    }
-    QueryBuilder query = null;
-    IdsQueryBuilder idsQuery = null;
-    if (sensorTypes != null) {
-      String[] types = sensorTypes.stream().map(sensorType -> sensorType + "_doc").toArray(String[]::new);
-      idsQuery = QueryBuilders.idsQuery(types);
-    } else {
-      idsQuery = QueryBuilders.idsQuery();
-    }
-
-    for(String guid : guids) {
-      query = idsQuery.addIds(guid);
-    }
-
-    SearchRequestBuilder request = client.prepareSearch()
-        .setQuery(query)
-        .setSize(guids.size())
-        ;
-    org.elasticsearch.action.search.SearchResponse response = request.get();
-    SearchHits hits = response.getHits();
-    List<T> results = new ArrayList<>();
-    for (SearchHit hit : hits) {
-      Optional<T> result = callback.apply(hit);
-      if (result.isPresent()) {
-        results.add(result.get());
-      }
-    }
-    return results;
-  }
-
-  private Optional<Document> toDocument(final String guid, SearchHit hit) {
-    Long ts = 0L;
-    String doc = hit.getSourceAsString();
-    String sourceType = toSourceType(hit.getType());
-    try {
-      return Optional.of(new Document(doc, guid, sourceType, ts));
-    } catch (IOException e) {
-      throw new IllegalStateException("Unable to retrieve latest: " + e.getMessage(), e);
-    }
-  }
-
-  /**
-   * Returns the source type based on a given doc type.
-   * @param docType The document type.
-   * @return The source type.
-   */
-  private String toSourceType(String docType) {
-    return Iterables.getFirst(Splitter.on("_doc").split(docType), null);
-  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java
index a7c3a71..c4d7412 100644
--- a/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java
+++ b/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/dao/ElasticsearchUpdateDao.java
@@ -42,14 +42,14 @@ public class ElasticsearchUpdateDao implements UpdateDao {
 
   private transient TransportClient client;
   private AccessConfig accessConfig;
-  private ElasticsearchSearchDao searchDao;
+  private ElasticsearchRetrieveLatestDao retrieveLatestDao;
 
   public ElasticsearchUpdateDao(TransportClient client,
       AccessConfig accessConfig,
-      ElasticsearchSearchDao searchDao) {
+      ElasticsearchRetrieveLatestDao searchDao) {
     this.client = client;
     this.accessConfig = accessConfig;
-    this.searchDao = searchDao;
+    this.retrieveLatestDao = searchDao;
   }
 
   @Override
@@ -110,7 +110,7 @@ public class ElasticsearchUpdateDao implements UpdateDao {
   }
 
   protected Optional<String> getIndexName(String guid, String sensorType) {
-    return searchDao.searchByGuid(guid,
+    return retrieveLatestDao.searchByGuid(guid,
         sensorType,
         hit -> Optional.ofNullable(hit.getIndex())
     );
@@ -121,7 +121,7 @@ public class ElasticsearchUpdateDao implements UpdateDao {
     Object ts = update.getTimestamp();
     IndexRequest indexRequest = new IndexRequest(indexName, type, update.getGuid())
         .source(update.getDocument());
-    if(ts != null) {
+    if (ts != null) {
       indexRequest = indexRequest.timestamp(ts.toString());
     }
 


[6/7] metron git commit: METRON-1421 Create a SolrMetaAlertDao (justinleet) closes apache/metron#970

Posted by le...@apache.org.
http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/main/java/org/apache/metron/elasticsearch/utils/ElasticsearchUtils.java
----------------------------------------------------------------------
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 24f7a27..98dc66d 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
@@ -36,18 +36,24 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
+import java.util.stream.Collectors;
 import org.apache.commons.lang.StringUtils;
 import org.apache.metron.common.configuration.writer.WriterConfiguration;
 import org.apache.metron.common.utils.HDFSUtils;
 import org.apache.metron.common.utils.ReflectionUtils;
+import org.apache.metron.indexing.dao.search.SearchResponse;
+import org.apache.metron.indexing.dao.search.SearchResult;
 import org.apache.metron.netty.utils.NettyRuntimeWrapper;
 import org.apache.metron.stellar.common.utils.ConversionUtils;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.elasticsearch.action.search.SearchRequestBuilder;
 import org.elasticsearch.client.transport.TransportClient;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.transport.InetSocketTransportAddress;
 import org.elasticsearch.common.xcontent.XContentHelper;
+import org.elasticsearch.index.query.QueryBuilder;
+import org.elasticsearch.transport.client.PreBuiltTransportClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -321,4 +327,62 @@ public class ElasticsearchUtils {
 
     return json;
   }
+
+  /**
+   * Elasticsearch queries default to 10 records returned.  Some internal queries require that all
+   * results are returned.  Rather than setting an arbitrarily high size, this method pages through results
+   * and returns them all in a single SearchResponse.
+   * @param qb A QueryBuilder that provides the query to be run.
+   * @return A SearchResponse containing the appropriate results.
+   */
+  public static  SearchResponse queryAllResults(TransportClient transportClient,
+      QueryBuilder qb,
+      String index,
+      int pageSize
+  ) {
+    SearchRequestBuilder searchRequestBuilder = transportClient
+        .prepareSearch(index)
+        .addStoredField("*")
+        .setFetchSource(true)
+        .setQuery(qb)
+        .setSize(pageSize);
+    org.elasticsearch.action.search.SearchResponse esResponse = searchRequestBuilder
+        .execute()
+        .actionGet();
+    List<SearchResult> allResults = getSearchResults(esResponse);
+    long total = esResponse.getHits().getTotalHits();
+    if (total > pageSize) {
+      int pages = (int) (total / pageSize) + 1;
+      for (int i = 1; i < pages; i++) {
+        int from = i * pageSize;
+        searchRequestBuilder.setFrom(from);
+        esResponse = searchRequestBuilder
+            .execute()
+            .actionGet();
+        allResults.addAll(getSearchResults(esResponse));
+      }
+    }
+    SearchResponse searchResponse = new SearchResponse();
+    searchResponse.setTotal(total);
+    searchResponse.setResults(allResults);
+    return searchResponse;
+  }
+
+  /**
+   * Transforms a list of Elasticsearch SearchHits to a list of SearchResults
+   * @param searchResponse An Elasticsearch SearchHit to be converted.
+   * @return The list of SearchResults for the SearchHit
+   */
+  protected static List<SearchResult> getSearchResults(
+      org.elasticsearch.action.search.SearchResponse searchResponse) {
+    return Arrays.stream(searchResponse.getHits().getHits()).map(searchHit -> {
+          SearchResult searchResult = new SearchResult();
+          searchResult.setId(searchHit.getId());
+          searchResult.setSource(searchHit.getSource());
+          searchResult.setScore(searchHit.getScore());
+          searchResult.setIndex(searchHit.getIndex());
+          return searchResult;
+        }
+    ).collect(Collectors.toList());
+  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchDaoTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchDaoTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchDaoTest.java
index ca1b860..6c3c327 100644
--- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchDaoTest.java
+++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchDaoTest.java
@@ -17,7 +17,9 @@
  */
 package org.apache.metron.elasticsearch.dao;
 
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
@@ -26,15 +28,15 @@ import static org.mockito.Mockito.when;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import org.apache.metron.elasticsearch.utils.ElasticsearchUtils;
 import org.apache.metron.indexing.dao.AccessConfig;
-import org.apache.metron.indexing.dao.ColumnMetadataDao;
+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.search.SortField;
 import org.apache.metron.indexing.dao.search.SortOrder;
-import org.apache.metron.elasticsearch.utils.ElasticsearchUtils;
-import org.apache.metron.indexing.dao.search.FieldType;
 import org.elasticsearch.client.transport.TransportClient;
 import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.search.SearchHit;
@@ -45,37 +47,38 @@ import org.json.simple.parser.JSONParser;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
-import java.util.Map;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertNotNull;
-
 public class ElasticsearchDaoTest {
 
   private ElasticsearchDao dao;
   private ElasticsearchRequestSubmitter requestSubmitter;
 
-  private void setup(RestStatus status, int maxSearchResults, Map<String, FieldType> metadata) throws Exception {
+  private void setup(RestStatus status, int maxSearchResults, Map<String, FieldType> metadata)
+      throws Exception {
 
     // setup the mock search hits
     SearchHit hit1 = mock(SearchHit.class);
     when(hit1.getId()).thenReturn("id1");
-    when(hit1.getSource()).thenReturn(new HashMap<String, Object>(){{ put("field", "value1"); }});
+    when(hit1.getSource()).thenReturn(new HashMap<String, Object>() {{
+      put("field", "value1");
+    }});
     when(hit1.getScore()).thenReturn(0.1f);
 
     SearchHit hit2 = mock(SearchHit.class);
     when(hit2.getId()).thenReturn("id2");
-    when(hit2.getSource()).thenReturn(new HashMap<String, Object>(){{ put("field", "value2"); }});
+    when(hit2.getSource()).thenReturn(new HashMap<String, Object>() {{
+      put("field", "value2");
+    }});
     when(hit2.getScore()).thenReturn(0.2f);
 
     // search hits
-    SearchHit[] hits = { hit1, hit2 };
+    SearchHit[] hits = {hit1, hit2};
     SearchHits searchHits = mock(SearchHits.class);
     when(searchHits.getHits()).thenReturn(hits);
     when(searchHits.getTotalHits()).thenReturn(Integer.toUnsignedLong(hits.length));
 
     // search response which returns the search hits
-    org.elasticsearch.action.search.SearchResponse response = mock(org.elasticsearch.action.search.SearchResponse.class);
+    org.elasticsearch.action.search.SearchResponse response = mock(
+        org.elasticsearch.action.search.SearchResponse.class);
     when(response.status()).thenReturn(status);
     when(response.getHits()).thenReturn(searchHits);
 
@@ -93,10 +96,21 @@ public class ElasticsearchDaoTest {
     AccessConfig config = mock(AccessConfig.class);
     when(config.getMaxSearchResults()).thenReturn(maxSearchResults);
 
-    ElasticsearchSearchDao elasticsearchSearchDao = new ElasticsearchSearchDao(client, config, columnMetadataDao, requestSubmitter);
-    ElasticsearchUpdateDao elasticsearchUpdateDao = new ElasticsearchUpdateDao(client, config, elasticsearchSearchDao);
-
-    dao = new ElasticsearchDao(client, config, elasticsearchSearchDao, elasticsearchUpdateDao, columnMetadataDao, requestSubmitter);
+    ElasticsearchSearchDao elasticsearchSearchDao = new ElasticsearchSearchDao(client, config,
+        columnMetadataDao, requestSubmitter);
+    ElasticsearchRetrieveLatestDao elasticsearchRetrieveLatestDao = new ElasticsearchRetrieveLatestDao(
+        client);
+    ElasticsearchUpdateDao elasticsearchUpdateDao = new ElasticsearchUpdateDao(client, config,
+        elasticsearchRetrieveLatestDao);
+
+    dao = new ElasticsearchDao(
+        client,
+        config,
+        elasticsearchSearchDao,
+        elasticsearchUpdateDao,
+        elasticsearchRetrieveLatestDao,
+        columnMetadataDao,
+        requestSubmitter);
   }
 
   private void setup(RestStatus status, int maxSearchResults) throws Exception {
@@ -116,9 +130,9 @@ public class ElasticsearchDaoTest {
 
     // "sort by" fields for the search request
     SortField[] expectedSortFields = {
-            sortBy("sortByStringDesc", SortOrder.DESC),
-            sortBy("sortByIntAsc", SortOrder.ASC),
-            sortBy("sortByUndefinedDesc", SortOrder.DESC)
+        sortBy("sortByStringDesc", SortOrder.DESC),
+        sortBy("sortByIntAsc", SortOrder.ASC),
+        sortBy("sortByUndefinedDesc", SortOrder.DESC)
     };
 
     // create a metron search request
@@ -135,7 +149,8 @@ public class ElasticsearchDaoTest {
     assertNotNull(searchResponse);
 
     // capture the elasticsearch search request that was created
-    ArgumentCaptor<org.elasticsearch.action.search.SearchRequest> argument = ArgumentCaptor.forClass(org.elasticsearch.action.search.SearchRequest.class);
+    ArgumentCaptor<org.elasticsearch.action.search.SearchRequest> argument = ArgumentCaptor
+        .forClass(org.elasticsearch.action.search.SearchRequest.class);
     verify(requestSubmitter).submitSearch(argument.capture());
     org.elasticsearch.action.search.SearchRequest request = argument.getValue();
 
@@ -181,9 +196,9 @@ public class ElasticsearchDaoTest {
 
     // "sort by" fields for the search request
     SortField[] expectedSortFields = {
-            sortBy("sortByStringDesc", SortOrder.DESC),
-            sortBy("sortByIntAsc", SortOrder.ASC),
-            sortBy("sortByUndefinedDesc", SortOrder.DESC)
+        sortBy("sortByStringDesc", SortOrder.DESC),
+        sortBy("sortByIntAsc", SortOrder.ASC),
+        sortBy("sortByUndefinedDesc", SortOrder.DESC)
     };
 
     // create a metron search request
@@ -200,7 +215,8 @@ public class ElasticsearchDaoTest {
     assertNotNull(searchResponse);
 
     // capture the elasticsearch search request that was created
-    ArgumentCaptor<org.elasticsearch.action.search.SearchRequest> argument = ArgumentCaptor.forClass(org.elasticsearch.action.search.SearchRequest.class);
+    ArgumentCaptor<org.elasticsearch.action.search.SearchRequest> argument = ArgumentCaptor
+        .forClass(org.elasticsearch.action.search.SearchRequest.class);
     verify(requestSubmitter).submitSearch(argument.capture());
     org.elasticsearch.action.search.SearchRequest request = argument.getValue();
 
@@ -209,7 +225,7 @@ public class ElasticsearchDaoTest {
     JSONObject json = (JSONObject) parser.parse(ElasticsearchUtils.toJSON(request).orElse("???"));
 
     // ensure that the index names are 'wildcard-ed'
-    String[] expected = { "bro_index*", "snort_index*" };
+    String[] expected = {"bro_index*", "snort_index*"};
     assertArrayEquals(expected, request.indices());
   }
 
@@ -221,7 +237,7 @@ public class ElasticsearchDaoTest {
     setup(RestStatus.OK, maxSearchResults);
 
     SearchRequest searchRequest = new SearchRequest();
-    searchRequest.setSize(maxSearchResults+1);
+    searchRequest.setSize(maxSearchResults + 1);
     searchRequest.setQuery("");
     dao.search(searchRequest);
     // exception expected - size > max

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDaoTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDaoTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDaoTest.java
index 1bfa9d6..25799ad 100644
--- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDaoTest.java
+++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/dao/ElasticsearchMetaAlertDaoTest.java
@@ -18,32 +18,21 @@
 
 package org.apache.metron.elasticsearch.dao;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.UUID;
-import org.apache.metron.common.Constants;
-import org.apache.metron.common.Constants.Fields;
 import org.apache.metron.indexing.dao.AccessConfig;
+import org.apache.metron.indexing.dao.HBaseDao;
 import org.apache.metron.indexing.dao.IndexDao;
-import org.apache.metron.indexing.dao.MetaAlertDao;
+import org.apache.metron.indexing.dao.MultiIndexDao;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest;
-import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
 import org.apache.metron.indexing.dao.search.FieldType;
 import org.apache.metron.indexing.dao.search.GetRequest;
 import org.apache.metron.indexing.dao.search.GroupRequest;
 import org.apache.metron.indexing.dao.search.GroupResponse;
 import org.apache.metron.indexing.dao.search.InvalidCreateException;
-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;
@@ -52,17 +41,16 @@ import org.junit.Test;
 public class ElasticsearchMetaAlertDaoTest {
 
 
-
   @Test(expected = IllegalArgumentException.class)
   public void testInvalidInit() {
     IndexDao dao = new IndexDao() {
       @Override
-      public SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException {
+      public SearchResponse search(SearchRequest searchRequest) {
         return null;
       }
 
       @Override
-      public GroupResponse group(GroupRequest groupRequest) throws InvalidSearchException {
+      public GroupResponse group(GroupRequest groupRequest) {
         return null;
       }
 
@@ -71,27 +59,26 @@ public class ElasticsearchMetaAlertDaoTest {
       }
 
       @Override
-      public Document getLatest(String guid, String sensorType) throws IOException {
+      public Document getLatest(String guid, String sensorType) {
         return null;
       }
 
       @Override
       public Iterable<Document> getAllLatest(
-          List<GetRequest> getRequests) throws IOException {
+          List<GetRequest> getRequests) {
         return null;
       }
 
       @Override
-      public void update(Document update, Optional<String> index) throws IOException {
+      public void update(Document update, Optional<String> index) {
       }
 
       @Override
-      public void batchUpdate(Map<Document, Optional<String>> updates) throws IOException {
+      public void batchUpdate(Map<Document, Optional<String>> updates) {
       }
 
       @Override
-      public Map<String, FieldType> getColumnMetadata(List<String> indices)
-          throws IOException {
+      public Map<String, FieldType> getColumnMetadata(List<String> indices) {
         return null;
       }
     };
@@ -99,92 +86,11 @@ public class ElasticsearchMetaAlertDaoTest {
     metaAlertDao.init(dao);
   }
 
-  @Test
-  public void testBuildCreateDocumentSingleAlert() throws InvalidCreateException, IOException {
-    ElasticsearchDao esDao = new ElasticsearchDao();
-    ElasticsearchMetaAlertDao emaDao = new ElasticsearchMetaAlertDao();
-    emaDao.init(esDao);
-
-    List<String> groups = new ArrayList<>();
-    groups.add("group_one");
-    groups.add("group_two");
-
-    // Build the first response from the multiget
-    Map<String, Object> alertOne = new HashMap<>();
-    alertOne.put(Constants.GUID, "alert_one");
-    alertOne.put(MetaAlertDao.THREAT_FIELD_DEFAULT, 10.0d);
-    List<Document> alerts = new ArrayList<Document>() {{
-      add(new Document(alertOne, "", "", 0L));
-    }};
-
-    // Actually build the doc
-    Document actual = emaDao.buildCreateDocument(alerts, groups);
-
-    ArrayList<Map<String, Object>> alertList = new ArrayList<>();
-    alertList.add(alertOne);
-
-    Map<String, Object> actualDocument = actual.getDocument();
-    assertEquals(
-        MetaAlertStatus.ACTIVE.getStatusString(),
-        actualDocument.get(MetaAlertDao.STATUS_FIELD)
-    );
-    assertEquals(
-        alertList,
-        actualDocument.get(MetaAlertDao.ALERT_FIELD)
-    );
-    assertEquals(
-        groups,
-        actualDocument.get(MetaAlertDao.GROUPS_FIELD)
-    );
-
-    // Don't care about the result, just that it's a UUID. Exception will be thrown if not.
-    UUID.fromString((String) actualDocument.get(Constants.GUID));
-  }
-
-  @Test
-  public void testBuildCreateDocumentMultipleAlerts() throws InvalidCreateException, IOException {
-    ElasticsearchDao esDao = new ElasticsearchDao();
-    ElasticsearchMetaAlertDao emaDao = new ElasticsearchMetaAlertDao();
-    emaDao.init(esDao);
-
-    List<String> groups = new ArrayList<>();
-    groups.add("group_one");
-    groups.add("group_two");
-
-    // Build the first response from the multiget
-    Map<String, Object> alertOne = new HashMap<>();
-    alertOne.put(Constants.GUID, "alert_one");
-    alertOne.put(MetaAlertDao.THREAT_FIELD_DEFAULT, 10.0d);
-
-    // Build the second response from the multiget
-    Map<String, Object> alertTwo = new HashMap<>();
-    alertTwo.put(Constants.GUID, "alert_one");
-    alertTwo.put(MetaAlertDao.THREAT_FIELD_DEFAULT, 5.0d);
-    List<Document> alerts = new ArrayList<Document>() {{
-      add(new Document(alertOne, "", "", 0L));
-      add(new Document(alertTwo, "", "", 0L));
-    }};
-
-    // Actually build the doc
-    Document actual = emaDao.buildCreateDocument(alerts, groups);
-
-    ArrayList<Map<String, Object>> alertList = new ArrayList<>();
-    alertList.add(alertOne);
-    alertList.add(alertTwo);
-
-    Map<String, Object> actualDocument = actual.getDocument();
-    assertNotNull(actualDocument.get(Fields.TIMESTAMP.getName()));
-    assertEquals(
-        alertList,
-        actualDocument.get(MetaAlertDao.ALERT_FIELD)
-    );
-    assertEquals(
-        groups,
-        actualDocument.get(MetaAlertDao.GROUPS_FIELD)
-    );
-
-    // Don't care about the result, just that it's a UUID. Exception will be thrown if not.
-    UUID.fromString((String) actualDocument.get(Constants.GUID));
+  @Test(expected = IllegalArgumentException.class)
+  public void testInitInvalidDao() {
+    HBaseDao dao = new HBaseDao();
+    ElasticsearchMetaAlertDao esDao = new ElasticsearchMetaAlertDao();
+    esDao.init(dao, Optional.empty());
   }
 
   @Test(expected = InvalidCreateException.class)
@@ -200,50 +106,12 @@ public class ElasticsearchMetaAlertDaoTest {
   @Test(expected = InvalidCreateException.class)
   public void testCreateMetaAlertEmptyGroups() throws InvalidCreateException, IOException {
     ElasticsearchDao esDao = new ElasticsearchDao();
+    MultiIndexDao miDao = new MultiIndexDao(esDao);
     ElasticsearchMetaAlertDao emaDao = new ElasticsearchMetaAlertDao();
-    emaDao.init(esDao);
+    emaDao.init(miDao);
 
     MetaAlertCreateRequest createRequest = new MetaAlertCreateRequest();
     createRequest.setAlerts(Collections.singletonList(new GetRequest("don't", "care")));
     emaDao.createMetaAlert(createRequest);
   }
-
-  @Test
-  public void testCalculateMetaScoresList() {
-    final double delta = 0.001;
-    List<Map<String, Object>> alertList = new ArrayList<>();
-
-    // add an alert with a threat score
-    alertList.add( Collections.singletonMap(MetaAlertDao.THREAT_FIELD_DEFAULT, 10.0f));
-
-    // add a second alert with a threat score
-    alertList.add( Collections.singletonMap(MetaAlertDao.THREAT_FIELD_DEFAULT, 20.0f));
-
-    // add a third alert with NO threat score
-    alertList.add( Collections.singletonMap("alert3", "has no threat score"));
-
-    // create the metaalert
-    Map<String, Object> docMap = new HashMap<>();
-    docMap.put(MetaAlertDao.ALERT_FIELD, alertList);
-    Document metaalert = new Document(docMap, "guid", MetaAlertDao.METAALERT_TYPE, 0L);
-
-    // calculate the threat score for the metaalert
-    ElasticsearchMetaAlertDao metaAlertDao = new ElasticsearchMetaAlertDao();
-    metaAlertDao.calculateMetaScores(metaalert);
-    Object threatScore = metaalert.getDocument().get(ElasticsearchMetaAlertDao.THREAT_FIELD_DEFAULT);
-
-    // the metaalert must contain a summary of all child threat scores
-    assertEquals(20D, (Double) metaalert.getDocument().get("max"), delta);
-    assertEquals(10D, (Double) metaalert.getDocument().get("min"), delta);
-    assertEquals(15D, (Double) metaalert.getDocument().get("average"), delta);
-    assertEquals(2L, metaalert.getDocument().get("count"));
-    assertEquals(30D, (Double) metaalert.getDocument().get("sum"), delta);
-    assertEquals(15D, (Double) metaalert.getDocument().get("median"), delta);
-
-    // it must contain an overall threat score; a float to match the type of the threat score of the other sensor indices
-    assertTrue(threatScore instanceof Float);
-
-    // by default, the overall threat score is the sum of all child threat scores
-    assertEquals(30.0F, threatScore);
-  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMetaAlertIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMetaAlertIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMetaAlertIntegrationTest.java
index 9e74fb6..6fa6956 100644
--- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMetaAlertIntegrationTest.java
+++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchMetaAlertIntegrationTest.java
@@ -18,15 +18,14 @@
 
 package org.apache.metron.elasticsearch.integration;
 
-import static org.apache.metron.indexing.dao.MetaAlertDao.ALERT_FIELD;
-import static org.apache.metron.indexing.dao.MetaAlertDao.METAALERTS_INDEX;
-import static org.apache.metron.indexing.dao.MetaAlertDao.METAALERT_FIELD;
-import static org.apache.metron.indexing.dao.MetaAlertDao.METAALERT_TYPE;
-import static org.apache.metron.indexing.dao.MetaAlertDao.STATUS_FIELD;
+import static org.apache.metron.elasticsearch.dao.ElasticsearchMetaAlertDao.METAALERTS_INDEX;
+import static org.apache.metron.elasticsearch.dao.ElasticsearchMetaAlertDao.THREAT_TRIAGE_FIELD;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.ALERT_FIELD;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_DOC;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_FIELD;
+import static org.apache.metron.indexing.dao.metaalert.MetaAlertConstants.METAALERT_TYPE;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
-import com.google.common.base.Joiner;
-import com.google.common.collect.Iterables;
 import java.io.File;
 import java.io.IOException;
 import java.text.SimpleDateFormat;
@@ -35,11 +34,9 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.Set;
 import java.util.stream.Collectors;
 import org.adrianwalker.multilinestring.Multiline;
 import org.apache.metron.common.Constants;
@@ -49,23 +46,12 @@ import org.apache.metron.elasticsearch.dao.ElasticsearchMetaAlertDao;
 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.MetaAlertDao;
-import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest;
-import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateResponse;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertIntegrationTest;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
 import org.apache.metron.indexing.dao.search.GetRequest;
-import org.apache.metron.indexing.dao.search.Group;
-import org.apache.metron.indexing.dao.search.GroupRequest;
-import org.apache.metron.indexing.dao.search.GroupResponse;
-import org.apache.metron.indexing.dao.search.GroupResult;
-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.search.SearchResult;
 import org.apache.metron.indexing.dao.search.SortField;
-import org.apache.metron.indexing.dao.update.Document;
-import org.apache.metron.indexing.dao.update.OriginalNotFoundException;
-import org.apache.metron.indexing.dao.update.PatchRequest;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -73,113 +59,50 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class ElasticsearchMetaAlertIntegrationTest {
-
-  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 INDEX_DIR = "target/elasticsearch_meta";
-  private static final String DATE_FORMAT = "yyyy.MM.dd.HH";
-  private static final String INDEX =
-      SENSOR_NAME + "_index_" + new SimpleDateFormat(DATE_FORMAT).format(new Date());
-  private static final String NEW_FIELD = "new-field";
-  private static final String NAME_FIELD = "name";
+public class ElasticsearchMetaAlertIntegrationTest extends MetaAlertIntegrationTest {
 
   private static IndexDao esDao;
-  private static MetaAlertDao metaDao;
   private static ElasticSearchComponent es;
 
-  /**
-   {
-     "properties": {
-       "alert": {
-         "type": "nested"
-       }
-     }
-   }
-   */
-  @Multiline
-  public static String nestedAlertMapping;
+  protected static final String INDEX_DIR = "target/elasticsearch_meta";
 
-  /**
-   {
-     "guid": "meta_alert",
-     "index": "metaalert_index",
-     "patch": [
-       {
-         "op": "add",
-         "path": "/name",
-         "value": "New Meta Alert"
-       }
-     ],
-     "sensorType": "metaalert"
-   }
-   */
-  @Multiline
-  public static String namePatchRequest;
+  protected static final String INDEX =
+      SENSOR_NAME + "_" + new SimpleDateFormat(DATE_FORMAT).format(new Date());
+  protected static final String INDEX_WITH_SEPARATOR = INDEX + "_index";
 
-  /**
-   {
-     "guid": "meta_alert",
-     "index": "metaalert_index",
-     "patch": [
-       {
-         "op": "add",
-         "path": "/name",
-         "value": "New Meta Alert"
-       },
-       {
-         "op": "add",
-         "path": "/alert",
-         "value": []
-       }
-     ],
-     "sensorType": "metaalert"
-   }
-   */
-  @Multiline
-  public static String alertPatchRequest;
+  protected ArrayList<String> queryIndices = allIndices.stream().map(x -> x.replace("_index", ""))
+      .collect(Collectors.toCollection(ArrayList::new));
 
   /**
    {
-     "guid": "meta_alert",
-     "index": "metaalert_index",
-     "patch": [
-       {
-         "op": "add",
-         "path": "/status",
-         "value": "inactive"
-       },
-       {
-         "op": "add",
-         "path": "/name",
-         "value": "New Meta Alert"
-       }
-     ],
-     "sensorType": "metaalert"
+   "properties": {
+   "alert": {
+   "type": "nested"
+   }
+   }
    }
    */
   @Multiline
-  public static String statusPatchRequest;
+  public static String nestedAlertMapping;
 
   /**
    * {
-       "%MAPPING_NAME%_doc" : {
-         "properties" : {
-           "guid" : {
-             "type" : "keyword"
-           },
-           "ip_src_addr" : {
-             "type" : "keyword"
-           },
-           "score" : {
-             "type" : "integer"
-           },
-           "alert" : {
-             "type" : "nested"
-           }
-         }
-       }
+   "%MAPPING_NAME%_doc" : {
+   "properties" : {
+   "guid" : {
+   "type" : "keyword"
+   },
+   "ip_src_addr" : {
+   "type" : "keyword"
+   },
+   "score" : {
+   "type" : "integer"
+   },
+   "alert" : {
+   "type" : "nested"
+   }
+   }
+   }
    }
    */
   @Multiline
@@ -187,6 +110,9 @@ public class ElasticsearchMetaAlertIntegrationTest {
 
   @BeforeClass
   public static void setupBefore() throws Exception {
+    // Ensure ES can retry as needed.
+    MAX_RETRIES = 10;
+
     // setup the client
     es = new ElasticSearchComponent.Builder()
         .withHttpPort(9211)
@@ -209,13 +135,17 @@ public class ElasticsearchMetaAlertIntegrationTest {
 
     esDao = new ElasticsearchDao();
     esDao.init(accessConfig);
-    metaDao = new ElasticsearchMetaAlertDao(esDao);
+    ElasticsearchMetaAlertDao elasticsearchMetaDao = new ElasticsearchMetaAlertDao(esDao);
+    elasticsearchMetaDao.setPageSize(5);
+    metaDao = elasticsearchMetaDao;
   }
 
   @Before
   public void setup() throws IOException {
-    es.createIndexWithMapping(METAALERTS_INDEX, MetaAlertDao.METAALERT_DOC, template.replace("%MAPPING_NAME%", "metaalert"));
-    es.createIndexWithMapping(INDEX, "index_doc", template.replace("%MAPPING_NAME%", "index"));
+    es.createIndexWithMapping(METAALERTS_INDEX, METAALERT_DOC,
+        template.replace("%MAPPING_NAME%", "metaalert"));
+    es.createIndexWithMapping(
+        INDEX_WITH_SEPARATOR, "index_doc", template.replace("%MAPPING_NAME%", "index"));
   }
 
   @AfterClass
@@ -230,464 +160,8 @@ public class ElasticsearchMetaAlertIntegrationTest {
     es.reset();
   }
 
-
-  @Test
-  public void shouldGetAllMetaAlertsForAlert() throws Exception {
-    // Load alerts
-    List<Map<String, Object>> alerts = buildAlerts(3);
-    elasticsearchAdd(alerts, INDEX, SENSOR_NAME);
-
-    // Load metaAlerts
-    List<Map<String, Object>> metaAlerts = buildMetaAlerts(12, MetaAlertStatus.ACTIVE,
-        Optional.of(Collections.singletonList(alerts.get(0))));
-    metaAlerts.add(buildMetaAlert("meta_active_12", MetaAlertStatus.ACTIVE,
-        Optional.of(Arrays.asList(alerts.get(0), alerts.get(2)))));
-    metaAlerts.add(buildMetaAlert("meta_inactive", MetaAlertStatus.INACTIVE,
-        Optional.of(Arrays.asList(alerts.get(0), alerts.get(2)))));
-    // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
-    elasticsearchAdd(metaAlerts, METAALERTS_INDEX, MetaAlertDao.METAALERT_TYPE);
-
-    // Verify load was successful
-    List<GetRequest> createdDocs = metaAlerts.stream().map(metaAlert ->
-        new GetRequest((String) metaAlert.get(Constants.GUID), METAALERT_TYPE))
-        .collect(Collectors.toList());
-    createdDocs.addAll(alerts.stream().map(alert ->
-        new GetRequest((String) alert.get(Constants.GUID), SENSOR_NAME))
-        .collect(Collectors.toList()));
-    findCreatedDocs(createdDocs);
-
-    int previousPageSize = ((ElasticsearchMetaAlertDao) metaDao).getPageSize();
-    ((ElasticsearchMetaAlertDao) metaDao).setPageSize(5);
-
-    {
-      // Verify searches successfully return more than 10 results
-      SearchResponse searchResponse0 = metaDao.getAllMetaAlertsForAlert("message_0");
-      List<SearchResult> searchResults0 = searchResponse0.getResults();
-      Assert.assertEquals(13, searchResults0.size());
-      Set<Map<String, Object>> resultSet = new HashSet<>();
-      Iterables.addAll(resultSet, Iterables.transform(searchResults0, r -> r.getSource()));
-      StringBuffer reason = new StringBuffer("Unable to find " + metaAlerts.get(0) + "\n");
-      reason.append(Joiner.on("\n").join(resultSet));
-      Assert.assertTrue(reason.toString(), resultSet.contains(metaAlerts.get(0)));
-
-      // Verify no meta alerts are returned because message_1 was not added to any
-      SearchResponse searchResponse1 = metaDao.getAllMetaAlertsForAlert("message_1");
-      List<SearchResult> searchResults1 = searchResponse1.getResults();
-      Assert.assertEquals(0, searchResults1.size());
-
-      // Verify only the meta alert message_2 was added to is returned
-      SearchResponse searchResponse2 = metaDao.getAllMetaAlertsForAlert("message_2");
-      List<SearchResult> searchResults2 = searchResponse2.getResults();
-      Assert.assertEquals(1, searchResults2.size());
-      Assert.assertEquals(metaAlerts.get(12), searchResults2.get(0).getSource());
-    }
-    ((ElasticsearchMetaAlertDao) metaDao).setPageSize(previousPageSize);
-  }
-
-  @Test
-  public void getAllMetaAlertsForAlertShouldThrowExceptionForEmtpyGuid() throws Exception {
-    try {
-      metaDao.getAllMetaAlertsForAlert("");
-      Assert.fail("An exception should be thrown for empty guid");
-    } catch (InvalidSearchException ise) {
-      Assert.assertEquals("Guid cannot be empty", ise.getMessage());
-    }
-  }
-
-  @Test
-  public void shouldCreateMetaAlert() throws Exception {
-    // Load alerts
-    List<Map<String, Object>> alerts = buildAlerts(3);
-    elasticsearchAdd(alerts, INDEX, SENSOR_NAME);
-
-    // Verify load was successful
-    findCreatedDocs(Arrays.asList(
-        new GetRequest("message_0", SENSOR_NAME),
-        new GetRequest("message_1", SENSOR_NAME),
-        new GetRequest("message_2", SENSOR_NAME)));
-
-    {
-      MetaAlertCreateRequest metaAlertCreateRequest = new MetaAlertCreateRequest() {{
-        setAlerts(new ArrayList<GetRequest>() {{
-          add(new GetRequest("message_1", SENSOR_NAME));
-          add(new GetRequest("message_2", SENSOR_NAME, INDEX));
-        }});
-        setGroups(Collections.singletonList("group"));
-      }};
-      MetaAlertCreateResponse metaAlertCreateResponse = metaDao.createMetaAlert(metaAlertCreateRequest);
-      {
-        // Verify metaAlert was created
-        findCreatedDoc(metaAlertCreateResponse.getGuid(), MetaAlertDao.METAALERT_TYPE);
-      }
-      {
-        // Verify alert 0 was not updated with metaalert field
-        Document alert = metaDao.getLatest("message_0", SENSOR_NAME);
-        Assert.assertEquals(4, alert.getDocument().size());
-        Assert.assertNull(alert.getDocument().get(METAALERT_FIELD));
-      }
-      {
-        // Verify alert 1 was properly updated with metaalert field
-        Document alert = metaDao.getLatest("message_1", SENSOR_NAME);
-        Assert.assertEquals(5, alert.getDocument().size());
-        Assert.assertEquals(1, ((List) alert.getDocument().get(METAALERT_FIELD)).size());
-        Assert.assertEquals(metaAlertCreateResponse.getGuid(), ((List) alert.getDocument().get(METAALERT_FIELD)).get(0));
-      }
-      {
-        // Verify alert 2 was properly updated with metaalert field
-        Document alert = metaDao.getLatest("message_2", SENSOR_NAME);
-        Assert.assertEquals(5, alert.getDocument().size());
-        Assert.assertEquals(1, ((List) alert.getDocument().get(METAALERT_FIELD)).size());
-        Assert.assertEquals(metaAlertCreateResponse.getGuid(), ((List) alert.getDocument().get(METAALERT_FIELD)).get(0));
-      }
-    }
-  }
-
-  @Test
-  public void shouldAddAlertsToMetaAlert() throws Exception {
-    // Load alerts
-    List<Map<String, Object>> alerts = buildAlerts(4);
-    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
-    elasticsearchAdd(alerts, INDEX, SENSOR_NAME);
-
-    // Load metaAlert
-    Map<String, Object> metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE,
-        Optional.of(Collections.singletonList(alerts.get(0))));
-    elasticsearchAdd(Collections.singletonList(metaAlert), METAALERTS_INDEX, METAALERT_TYPE);
-
-    // Verify load was successful
-    findCreatedDocs(Arrays.asList(
-        new GetRequest("message_0", SENSOR_NAME),
-        new GetRequest("message_1", SENSOR_NAME),
-        new GetRequest("message_2", SENSOR_NAME),
-        new GetRequest("message_3", SENSOR_NAME),
-        new GetRequest("meta_alert", METAALERT_TYPE)));
-
-    // Build expected metaAlert after alerts are added
-    Map<String, Object> expectedMetaAlert = new HashMap<>(metaAlert);
-
-    // Verify the proper alerts were added
-    List<Map<String, Object>> metaAlertAlerts = new ArrayList<>((List<Map<String, Object>>) expectedMetaAlert.get(ALERT_FIELD));
-    Map<String, Object> expectedAlert0 = alerts.get(0);
-    Map<String, Object> expectedAlert1 = alerts.get(1);
-    expectedAlert1.put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
-    metaAlertAlerts.add(expectedAlert1);
-    Map<String, Object> expectedAlert2 = alerts.get(2);
-    expectedAlert2.put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
-    metaAlertAlerts.add(expectedAlert2);
-    expectedMetaAlert.put(ALERT_FIELD, metaAlertAlerts);
-
-    // Verify the counts were properly updated
-    expectedMetaAlert.put("average", 1.0d);
-    expectedMetaAlert.put("min", 0.0d);
-    expectedMetaAlert.put("median", 1.0d);
-    expectedMetaAlert.put("max", 2.0d);
-    expectedMetaAlert.put("count", 3);
-    expectedMetaAlert.put("sum", 3.0d);
-    expectedMetaAlert.put("threat:triage:score", 3.0d);
-
-    {
-      // Verify alerts were successfully added to the meta alert
-      Assert.assertTrue(metaDao.addAlertsToMetaAlert("meta_alert", Arrays.asList(new GetRequest("message_1", SENSOR_NAME), new GetRequest("message_2", SENSOR_NAME))));
-      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
-    }
-
-    {
-      // Verify False when alerts are already in a meta alert and no new alerts are added
-      Assert.assertFalse(metaDao.addAlertsToMetaAlert("meta_alert", Arrays.asList(new GetRequest("message_0", SENSOR_NAME), new GetRequest("message_1", SENSOR_NAME))));
-      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
-    }
-
-    {
-      // Verify only 1 alert is added when a list of alerts only contains 1 alert that is not in the meta alert
-      metaAlertAlerts = new ArrayList<>((List<Map<String, Object>>) expectedMetaAlert.get(ALERT_FIELD));
-      Map<String, Object> expectedAlert3 = alerts.get(3);
-      expectedAlert3.put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
-      metaAlertAlerts.add(expectedAlert3);
-      expectedMetaAlert.put(ALERT_FIELD, metaAlertAlerts);
-
-      expectedMetaAlert.put("average", 1.5d);
-      expectedMetaAlert.put("min", 0.0d);
-      expectedMetaAlert.put("median", 1.5d);
-      expectedMetaAlert.put("max", 3.0d);
-      expectedMetaAlert.put("count", 4);
-      expectedMetaAlert.put("sum", 6.0d);
-      expectedMetaAlert.put("threat:triage:score", 6.0d);
-
-      Assert.assertTrue(metaDao.addAlertsToMetaAlert("meta_alert", Arrays.asList(new GetRequest("message_2", SENSOR_NAME), new GetRequest("message_3", SENSOR_NAME))));
-      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
-    }
-
-  }
-
-  @Test
-  public void shouldRemoveAlertsFromMetaAlert() throws Exception {
-    // Load alerts
-    List<Map<String, Object>> alerts = buildAlerts(4);
-    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
-    alerts.get(1).put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
-    alerts.get(2).put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
-    alerts.get(3).put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
-    elasticsearchAdd(alerts, INDEX, SENSOR_NAME);
-
-    // Load metaAlert
-    Map<String, Object> metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE,
-        Optional.of(Arrays.asList(alerts.get(0), alerts.get(1), alerts.get(2), alerts.get(3))));
-    elasticsearchAdd(Collections.singletonList(metaAlert), METAALERTS_INDEX, METAALERT_TYPE);
-
-    // Verify load was successful
-    findCreatedDocs(Arrays.asList(
-        new GetRequest("message_0", SENSOR_NAME),
-        new GetRequest("message_1", SENSOR_NAME),
-        new GetRequest("message_2", SENSOR_NAME),
-        new GetRequest("message_3", SENSOR_NAME),
-        new GetRequest("meta_alert", METAALERT_TYPE)));
-
-    // Build expected metaAlert after alerts are added
-    Map<String, Object> expectedMetaAlert = new HashMap<>(metaAlert);
-
-    // Verify the proper alerts were added
-    List<Map<String, Object>> metaAlertAlerts = new ArrayList<>((List<Map<String, Object>>) expectedMetaAlert.get(ALERT_FIELD));
-    metaAlertAlerts.remove(0);
-    metaAlertAlerts.remove(0);
-    expectedMetaAlert.put(ALERT_FIELD, metaAlertAlerts);
-
-    // Verify the counts were properly updated
-    expectedMetaAlert.put("average", 2.5d);
-    expectedMetaAlert.put("min", 2.0d);
-    expectedMetaAlert.put("median", 2.5d);
-    expectedMetaAlert.put("max", 3.0d);
-    expectedMetaAlert.put("count", 2);
-    expectedMetaAlert.put("sum", 5.0d);
-    expectedMetaAlert.put("threat:triage:score", 5.0d);
-
-
-    {
-      // Verify a list of alerts are removed from a meta alert
-      Assert.assertTrue(metaDao.removeAlertsFromMetaAlert("meta_alert", Arrays.asList(new GetRequest("message_0", SENSOR_NAME), new GetRequest("message_1", SENSOR_NAME))));
-      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
-    }
-
-    {
-      // Verify False when alerts are not present in a meta alert and no alerts are removed
-      Assert.assertFalse(metaDao.removeAlertsFromMetaAlert("meta_alert", Arrays.asList(new GetRequest("message_0", SENSOR_NAME), new GetRequest("message_1", SENSOR_NAME))));
-      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
-    }
-
-    {
-      // Verify only 1 alert is removed when a list of alerts only contains 1 alert that is in the meta alert
-      metaAlertAlerts = new ArrayList<>((List<Map<String, Object>>) expectedMetaAlert.get(ALERT_FIELD));
-      metaAlertAlerts.remove(0);
-      expectedMetaAlert.put(ALERT_FIELD, metaAlertAlerts);
-
-      expectedMetaAlert.put("average", 3.0d);
-      expectedMetaAlert.put("min", 3.0d);
-      expectedMetaAlert.put("median", 3.0d);
-      expectedMetaAlert.put("max", 3.0d);
-      expectedMetaAlert.put("count", 1);
-      expectedMetaAlert.put("sum", 3.0d);
-      expectedMetaAlert.put("threat:triage:score", 3.0d);
-
-      Assert.assertTrue(metaDao.removeAlertsFromMetaAlert("meta_alert", Arrays.asList(new GetRequest("message_0", SENSOR_NAME), new GetRequest("message_2", SENSOR_NAME))));
-      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
-    }
-
-    {
-      // Verify all alerts are removed from a metaAlert
-      metaAlertAlerts = new ArrayList<>((List<Map<String, Object>>) expectedMetaAlert.get(ALERT_FIELD));
-      metaAlertAlerts.remove(0);
-      expectedMetaAlert.put(ALERT_FIELD, metaAlertAlerts);
-
-      expectedMetaAlert.put("average", 0.0d);
-      expectedMetaAlert.put("min", "Infinity");
-      expectedMetaAlert.put("median", "NaN");
-      expectedMetaAlert.put("max", "-Infinity");
-      expectedMetaAlert.put("count", 0);
-      expectedMetaAlert.put("sum", 0.0d);
-      expectedMetaAlert.put("threat:triage:score", 0.0d);
-
-      Assert.assertTrue(metaDao.removeAlertsFromMetaAlert("meta_alert",
-          Collections.singletonList(new GetRequest("message_3", SENSOR_NAME))));
-      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
-    }
-
-  }
-
-  @Test
-  public void addRemoveAlertsShouldThrowExceptionForInactiveMetaAlert() throws Exception {
-    // Load alerts
-    List<Map<String, Object>> alerts = buildAlerts(2);
-    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
-    elasticsearchAdd(alerts, INDEX, SENSOR_NAME);
-
-    // Load metaAlert
-    Map<String, Object> metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.INACTIVE,
-        Optional.of(Collections.singletonList(alerts.get(0))));
-    elasticsearchAdd(Collections.singletonList(metaAlert), METAALERTS_INDEX, METAALERT_TYPE);
-
-    // Verify load was successful
-    findCreatedDocs(Arrays.asList(
-        new GetRequest("message_0", SENSOR_NAME),
-        new GetRequest("message_1", SENSOR_NAME),
-        new GetRequest("meta_alert", METAALERT_TYPE)));
-
-    {
-      // Verify alerts cannot be added to an INACTIVE meta alert
-      try {
-        metaDao.addAlertsToMetaAlert("meta_alert",
-            Collections.singletonList(new GetRequest("message_1", SENSOR_NAME)));
-        Assert.fail("Adding alerts to an inactive meta alert should throw an exception");
-      } catch (IllegalStateException ise) {
-        Assert.assertEquals("Adding alerts to an INACTIVE meta alert is not allowed", ise.getMessage());
-      }
-    }
-
-    {
-      // Verify alerts cannot be removed from an INACTIVE meta alert
-      try {
-        metaDao.removeAlertsFromMetaAlert("meta_alert",
-            Collections.singletonList(new GetRequest("message_0", SENSOR_NAME)));
-        Assert.fail("Removing alerts from an inactive meta alert should throw an exception");
-      } catch (IllegalStateException ise) {
-        Assert.assertEquals("Removing alerts from an INACTIVE meta alert is not allowed", ise.getMessage());
-      }
-    }
-  }
-
-  @Test
-  public void shouldUpdateMetaAlertStatus() throws Exception {
-    int numChildAlerts = 25;
-    int numUnrelatedAlerts = 25;
-    int totalAlerts = numChildAlerts + numUnrelatedAlerts;
-
-    // Load alerts
-    List<Map<String, Object>> alerts = buildAlerts(totalAlerts);
-    List<Map<String, Object>> childAlerts = alerts.subList(0, numChildAlerts);
-    List<Map<String, Object>> unrelatedAlerts = alerts.subList(numChildAlerts, totalAlerts);
-    for (Map<String, Object> alert : childAlerts) {
-      alert.put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
-    }
-    elasticsearchAdd(alerts, INDEX, SENSOR_NAME);
-
-    // Load metaAlerts
-    Map<String, Object> metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE,
-        Optional.of(childAlerts));
-    // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
-    elasticsearchAdd(Collections.singletonList(metaAlert), METAALERTS_INDEX,
-        MetaAlertDao.METAALERT_TYPE);
-
-    List<GetRequest> requests = new ArrayList<>();
-    for (int i = 0; i < numChildAlerts; ++i) {
-      requests.add(new GetRequest("message_" + i, SENSOR_NAME));
-    }
-    requests.add(new GetRequest("meta_alert", METAALERT_TYPE));
-
-    // Verify load was successful
-    findCreatedDocs(requests);
-
-    {
-      // Verify status changed to inactive and child alerts are updated
-      Assert.assertTrue(metaDao.updateMetaAlertStatus("meta_alert", MetaAlertStatus.INACTIVE));
-
-      Map<String, Object> expectedMetaAlert = new HashMap<>(metaAlert);
-      expectedMetaAlert.put(STATUS_FIELD, MetaAlertStatus.INACTIVE.getStatusString());
-
-      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
-
-      for (int i = 0; i < numChildAlerts; ++i) {
-        Map<String, Object> expectedAlert = new HashMap<>(childAlerts.get(i));
-        expectedAlert.put("metaalerts", new ArrayList());
-        findUpdatedDoc(expectedAlert, "message_" + i, SENSOR_NAME);
-      }
-
-      // Ensure unrelated alerts are unaffected
-      for (int i = 0; i < numUnrelatedAlerts; ++i) {
-        Map<String, Object> expectedAlert = new HashMap<>(unrelatedAlerts.get(i));
-        // Make sure to handle the guid offset from creation
-        findUpdatedDoc(expectedAlert, "message_" + (i + numChildAlerts), SENSOR_NAME);
-      }
-    }
-
-    {
-      // Verify status changed to active and child alerts are updated
-      Assert.assertTrue(metaDao.updateMetaAlertStatus("meta_alert", MetaAlertStatus.ACTIVE));
-
-      Map<String, Object> expectedMetaAlert = new HashMap<>(metaAlert);
-      expectedMetaAlert.put(STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString());
-
-      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
-
-      for (int i = 0; i < numChildAlerts; ++i) {
-        Map<String, Object> expectedAlert = new HashMap<>(alerts.get(i));
-        expectedAlert.put("metaalerts", Collections.singletonList("meta_alert"));
-        findUpdatedDoc(expectedAlert, "message_" + i, SENSOR_NAME);
-      }
-
-      // Ensure unrelated alerts are unaffected
-      for (int i = 0; i < numUnrelatedAlerts; ++i) {
-        Map<String, Object> expectedAlert = new HashMap<>(unrelatedAlerts.get(i));
-        // Make sure to handle the guid offset from creation
-        findUpdatedDoc(expectedAlert, "message_" + (i + numChildAlerts), SENSOR_NAME);
-      }
-
-      {
-        // Verify status changed to current status has no effect
-        Assert.assertFalse(metaDao.updateMetaAlertStatus("meta_alert", MetaAlertStatus.ACTIVE));
-
-        findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
-
-        for (int i = 0; i < numChildAlerts; ++i) {
-          Map<String, Object> expectedAlert = new HashMap<>(alerts.get(i));
-          expectedAlert.put("metaalerts", Collections.singletonList("meta_alert"));
-          findUpdatedDoc(expectedAlert, "message_" + i, SENSOR_NAME);
-        }
-
-        // Ensure unrelated alerts are unaffected
-        for (int i = 0; i < numUnrelatedAlerts; ++i) {
-          Map<String, Object> expectedAlert = new HashMap<>(unrelatedAlerts.get(i));
-          // Make sure to handle the guid offset from creation
-          findUpdatedDoc(expectedAlert, "message_" + (i + numChildAlerts), SENSOR_NAME);
-        }
-      }
-    }
-  }
-
-  @Test
-  public void shouldSearchByStatus() throws Exception {
-    // Load metaAlerts
-    Map<String, Object> activeMetaAlert = buildMetaAlert("meta_active", MetaAlertStatus.ACTIVE,
-        Optional.empty());
-    Map<String, Object> inactiveMetaAlert = buildMetaAlert("meta_inactive", MetaAlertStatus.INACTIVE,
-        Optional.empty());
-
-
-    // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
-    elasticsearchAdd(Arrays.asList(activeMetaAlert, inactiveMetaAlert), METAALERTS_INDEX, MetaAlertDao.METAALERT_TYPE);
-
-    // Verify load was successful
-    findCreatedDocs(Arrays.asList(
-        new GetRequest("meta_active", METAALERT_TYPE),
-        new GetRequest("meta_inactive", METAALERT_TYPE)));
-
-    SearchResponse searchResponse = metaDao.search(new SearchRequest() {
-      {
-        setQuery("*");
-        setIndices(Collections.singletonList(MetaAlertDao.METAALERT_TYPE));
-        setFrom(0);
-        setSize(5);
-        setSort(Collections.singletonList(new SortField() {{
-          setField(Constants.GUID);
-        }}));
-      }
-    });
-
-    // Verify only active meta alerts are returned
-    Assert.assertEquals(1, searchResponse.getTotal());
-    Assert.assertEquals(MetaAlertStatus.ACTIVE.getStatusString(),
-        searchResponse.getResults().get(0).getSource().get(MetaAlertDao.STATUS_FIELD));
-  }
-
-
   @Test
+  @Override
   public void shouldSearchByNestedAlert() throws Exception {
     // Load alerts
     List<Map<String, Object>> alerts = buildAlerts(4);
@@ -701,21 +175,20 @@ public class ElasticsearchMetaAlertIntegrationTest {
     alerts.get(2).put("ip_src_port", 8008);
     alerts.get(3).put("ip_src_addr", "192.168.1.4");
     alerts.get(3).put("ip_src_port", 8007);
-    elasticsearchAdd(alerts, INDEX, SENSOR_NAME);
+    addRecords(alerts, INDEX_WITH_SEPARATOR, SENSOR_NAME);
 
     // Put the nested type into the test index, so that it'll match appropriately
-    ((ElasticsearchDao) esDao).getClient().admin().indices().preparePutMapping(INDEX)
-        .setType("test_doc")
-        .setSource(nestedAlertMapping)
-        .get();
+    setupTypings();
 
     // Load metaAlerts
     Map<String, Object> activeMetaAlert = buildMetaAlert("meta_active", MetaAlertStatus.ACTIVE,
         Optional.of(Arrays.asList(alerts.get(0), alerts.get(1))));
-    Map<String, Object> inactiveMetaAlert = buildMetaAlert("meta_inactive", MetaAlertStatus.INACTIVE,
+    Map<String, Object> inactiveMetaAlert = buildMetaAlert("meta_inactive",
+        MetaAlertStatus.INACTIVE,
         Optional.of(Arrays.asList(alerts.get(2), alerts.get(3))));
     // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
-    elasticsearchAdd(Arrays.asList(activeMetaAlert, inactiveMetaAlert), METAALERTS_INDEX, MetaAlertDao.METAALERT_TYPE);
+    addRecords(Arrays.asList(activeMetaAlert, inactiveMetaAlert), METAALERTS_INDEX,
+        METAALERT_TYPE);
 
     // Verify load was successful
     findCreatedDocs(Arrays.asList(
@@ -726,12 +199,11 @@ public class ElasticsearchMetaAlertIntegrationTest {
         new GetRequest("meta_active", METAALERT_TYPE),
         new GetRequest("meta_inactive", METAALERT_TYPE)));
 
-
     SearchResponse searchResponse = metaDao.search(new SearchRequest() {
       {
         setQuery(
             "(ip_src_addr:192.168.1.1 AND ip_src_port:8009) OR (alert.ip_src_addr:192.168.1.1 AND alert.ip_src_port:8009)");
-        setIndices(Collections.singletonList(MetaAlertDao.METAALERT_TYPE));
+        setIndices(Collections.singletonList(METAALERT_TYPE));
         setFrom(0);
         setSize(5);
         setSort(Collections.singletonList(new SortField() {
@@ -751,7 +223,7 @@ public class ElasticsearchMetaAlertIntegrationTest {
         setQuery(
             "(ip_src_addr:192.168.1.1 AND ip_src_port:8010)"
                 + " OR (alert.ip_src_addr:192.168.1.1 AND alert.ip_src_port:8010)");
-        setIndices(Collections.singletonList("*"));
+        setIndices(queryIndices);
         setFrom(0);
         setSize(5);
         setSort(Collections.singletonList(new SortField() {
@@ -769,12 +241,12 @@ public class ElasticsearchMetaAlertIntegrationTest {
 
     // Query against all indices. The child alert has no actual attached meta alerts, and should
     // be returned on its own.
-   searchResponse = metaDao.search(new SearchRequest() {
+    searchResponse = metaDao.search(new SearchRequest() {
       {
         setQuery(
             "(ip_src_addr:192.168.1.3 AND ip_src_port:8008)"
                 + " OR (alert.ip_src_addr:192.168.1.3 AND alert.ip_src_port:8008)");
-        setIndices(Collections.singletonList("*"));
+        setIndices(queryIndices);
         setFrom(0);
         setSize(1);
         setSort(Collections.singletonList(new SortField() {
@@ -791,221 +263,13 @@ public class ElasticsearchMetaAlertIntegrationTest {
         searchResponse.getResults().get(0).getSource().get("guid"));
   }
 
-  @Test
-  public void shouldHidesAlertsOnGroup() throws Exception {
-    // Load alerts
-    List<Map<String, Object>> alerts = buildAlerts(2);
-    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_active"));
-    alerts.get(0).put("ip_src_addr", "192.168.1.1");
-    alerts.get(0).put("score_field", 1);
-    alerts.get(1).put("ip_src_addr", "192.168.1.1");
-    alerts.get(1).put("score_field", 10);
-    elasticsearchAdd(alerts, INDEX, SENSOR_NAME);
-
-
-    // Put the nested type into the test index, so that it'll match appropriately
-    ((ElasticsearchDao) esDao).getClient().admin().indices().preparePutMapping(INDEX)
-        .setType("test_doc")
-        .setSource(nestedAlertMapping)
-        .get();
-
-    // Don't need any meta alerts to actually exist, since we've populated the field on the alerts.
-
-    // Verify load was successful
-    findCreatedDocs(Arrays.asList(
-        new GetRequest("message_0", SENSOR_NAME),
-        new GetRequest("message_1", SENSOR_NAME)));
-
-    // Build our group request
-    Group searchGroup = new Group();
-    searchGroup.setField("ip_src_addr");
-    List<Group> groupList = new ArrayList<>();
-    groupList.add(searchGroup);
-    GroupResponse groupResponse = metaDao.group(new GroupRequest() {
-      {
-        setQuery("ip_src_addr:192.168.1.1");
-        setIndices(Collections.singletonList("*"));
-        setScoreField("score_field");
-        setGroups(groupList);
-    }});
-
-    // Should only return the standalone alert in the group
-    GroupResult result = groupResponse.getGroupResults().get(0);
-    Assert.assertEquals(1, result.getTotal());
-    Assert.assertEquals("192.168.1.1", result.getKey());
-    // No delta, since no ops happen
-    Assert.assertEquals(10.0d, result.getScore(), 0.0d);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void shouldUpdateMetaAlertOnAlertUpdate() throws Exception {
-    // Load alerts
-    List<Map<String, Object>> alerts = buildAlerts(2);
-    alerts.get(0).put(METAALERT_FIELD, Arrays.asList("meta_active", "meta_inactive"));
-    elasticsearchAdd(alerts, INDEX, SENSOR_NAME);
-
-    // Load metaAlerts
-    Map<String, Object> activeMetaAlert = buildMetaAlert("meta_active", MetaAlertStatus.ACTIVE,
-        Optional.of(Collections.singletonList(alerts.get(0))));
-    Map<String, Object> inactiveMetaAlert = buildMetaAlert("meta_inactive", MetaAlertStatus.INACTIVE,
-        Optional.of(Collections.singletonList(alerts.get(0))));
-    // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
-    elasticsearchAdd(Arrays.asList(activeMetaAlert, inactiveMetaAlert), METAALERTS_INDEX, METAALERT_TYPE);
-
-    // Verify load was successful
-    findCreatedDocs(Arrays.asList(
-        new GetRequest("message_0", SENSOR_NAME),
-        new GetRequest("message_1", SENSOR_NAME),
-        new GetRequest("meta_active", METAALERT_TYPE),
-        new GetRequest("meta_inactive", METAALERT_TYPE)));
-
-    {
-      // Modify the first message and add a new field
-      Map<String, Object> message0 = new HashMap<String, Object>(alerts.get(0)) {
-        {
-          put(NEW_FIELD, "metron");
-          put(MetaAlertDao.THREAT_FIELD_DEFAULT, "10");
-        }
-      };
-      String guid = "" + message0.get(Constants.GUID);
-      metaDao.update(new Document(message0, guid, SENSOR_NAME, null), Optional.empty());
-
-      {
-        // Verify alerts in ES are up-to-date
-        findUpdatedDoc(message0, guid, SENSOR_NAME);
-        long cnt = getMatchingAlertCount(NEW_FIELD, message0.get(NEW_FIELD));
-        if (cnt == 0) {
-          Assert.fail("Elasticsearch alert not updated!");
-        }
-      }
-
-      {
-        // Verify meta alerts in ES are up-to-date
-        long cnt = getMatchingMetaAlertCount(NEW_FIELD, "metron");
-        if (cnt == 0) {
-          Assert.fail("Active metaalert was not updated!");
-        }
-        if (cnt != 1) {
-          Assert.fail("Elasticsearch metaalerts not updated correctly!");
-        }
-      }
-    }
-    //modify the same message and modify the new field
-    {
-      Map<String, Object> message0 = new HashMap<String, Object>(alerts.get(0)) {
-        {
-          put(NEW_FIELD, "metron2");
-        }
-      };
-      String guid = "" + message0.get(Constants.GUID);
-      metaDao.update(new Document(message0, guid, SENSOR_NAME, null), Optional.empty());
-
-      {
-        // Verify ES is up-to-date
-        findUpdatedDoc(message0, guid, SENSOR_NAME);
-        long cnt = getMatchingAlertCount(NEW_FIELD, message0.get(NEW_FIELD));
-        if (cnt == 0) {
-          Assert.fail("Elasticsearch alert not updated!");
-        }
-      }
-      {
-        // Verify meta alerts in ES are up-to-date
-        long cnt = getMatchingMetaAlertCount(NEW_FIELD, "metron2");
-        if (cnt == 0) {
-          Assert.fail("Active metaalert was not updated!");
-        }
-        if (cnt != 1) {
-          Assert.fail("Elasticsearch metaalerts not updated correctly!");
-        }
-      }
-    }
-  }
-
-  @Test
-  public void shouldThrowExceptionOnMetaAlertUpdate() throws Exception {
-    Document metaAlert = new Document(new HashMap<>(), "meta_alert", METAALERT_TYPE, 0L);
-    try {
-      // Verify a meta alert cannot be updated in the meta alert dao
-      metaDao.update(metaAlert, Optional.empty());
-      Assert.fail("Direct meta alert update should throw an exception");
-    } catch (UnsupportedOperationException uoe) {
-      Assert.assertEquals("Meta alerts cannot be directly updated", uoe.getMessage());
-    }
-  }
-  @Test
-  public void shouldPatchAllowedMetaAlerts() throws Exception {
-    // Load alerts
-    List<Map<String, Object>> alerts = buildAlerts(2);
-    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_active"));
-    alerts.get(1).put(METAALERT_FIELD, Collections.singletonList("meta_active"));
-    elasticsearchAdd(alerts, INDEX, SENSOR_NAME);
-
-    // Put the nested type into the test index, so that it'll match appropriately
-    ((ElasticsearchDao) esDao).getClient().admin().indices().preparePutMapping(INDEX)
-        .setType("test_doc")
-        .setSource(nestedAlertMapping)
-        .get();
-
-    // Load metaAlerts
-    Map<String, Object> metaAlert = buildMetaAlert("meta_alert", MetaAlertStatus.ACTIVE,
-        Optional.of(Arrays.asList(alerts.get(0), alerts.get(1))));
-    // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
-    elasticsearchAdd(Collections.singletonList(metaAlert), METAALERTS_INDEX, MetaAlertDao.METAALERT_TYPE);
-
-    // Verify load was successful
-    findCreatedDocs(Arrays.asList(
-        new GetRequest("message_0", SENSOR_NAME),
-        new GetRequest("message_1", SENSOR_NAME),
-        new GetRequest("meta_alert", METAALERT_TYPE)));
-
-    Map<String, Object> expectedMetaAlert = new HashMap<>(metaAlert);
-    expectedMetaAlert.put(NAME_FIELD, "New Meta Alert");
-    {
-      // Verify a patch to a field other than "status" or "alert" can be patched
-      PatchRequest patchRequest = JSONUtils.INSTANCE.load(namePatchRequest, PatchRequest.class);
-      metaDao.patch(patchRequest, Optional.of(System.currentTimeMillis()));
-
-      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
-    }
-
-    {
-      // Verify a patch to an alert field should throw an exception
-      try {
-        PatchRequest patchRequest = JSONUtils.INSTANCE.load(alertPatchRequest, PatchRequest.class);
-        metaDao.patch(patchRequest, Optional.of(System.currentTimeMillis()));
-
-        Assert.fail("A patch on the alert field should throw an exception");
-      } catch (IllegalArgumentException iae) {
-        Assert.assertEquals("Meta alert patches are not allowed for /alert or /status paths.  "
-            + "Please use the add/remove alert or update status functions instead.", iae.getMessage());
-      }
-
-      // Verify the metaAlert was not updated
-      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
-    }
-
-    {
-      // Verify a patch to a status field should throw an exception
-      try {
-        PatchRequest patchRequest = JSONUtils.INSTANCE.load(statusPatchRequest, PatchRequest.class);
-        metaDao.patch(patchRequest, Optional.of(System.currentTimeMillis()));
-
-        Assert.fail("A patch on the status field should throw an exception");
-      } catch (IllegalArgumentException iae) {
-        Assert.assertEquals("Meta alert patches are not allowed for /alert or /status paths.  "
-            + "Please use the add/remove alert or update status functions instead.", iae.getMessage());
-      }
-
-      // Verify the metaAlert was not updated
-      findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
-    }
-  }
-
-  protected long getMatchingAlertCount(String fieldName, Object fieldValue) throws IOException, InterruptedException {
+  @Override
+  protected long getMatchingAlertCount(String fieldName, Object fieldValue)
+      throws IOException, InterruptedException {
     long cnt = 0;
     for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t, Thread.sleep(SLEEP_MS)) {
-      List<Map<String, Object>> docs = es.getAllIndexedDocs(INDEX, SENSOR_NAME + "_doc");
+      List<Map<String, Object>> docs = es
+          .getAllIndexedDocs(INDEX_WITH_SEPARATOR, SENSOR_NAME + "_doc");
       cnt = docs
           .stream()
           .filter(d -> {
@@ -1016,15 +280,19 @@ public class ElasticsearchMetaAlertIntegrationTest {
     return cnt;
   }
 
-  protected long getMatchingMetaAlertCount(String fieldName, String fieldValue) throws IOException, InterruptedException {
+  @Override
+  protected long getMatchingMetaAlertCount(String fieldName, String fieldValue)
+      throws IOException, InterruptedException {
     long cnt = 0;
     for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t, Thread.sleep(SLEEP_MS)) {
-      List<Map<String, Object>> docs = es.getAllIndexedDocs(METAALERTS_INDEX, MetaAlertDao.METAALERT_DOC);
+      List<Map<String, Object>> docs = es
+          .getAllIndexedDocs(METAALERTS_INDEX, METAALERT_DOC);
       cnt = docs
           .stream()
           .filter(d -> {
+            @SuppressWarnings("unchecked")
             List<Map<String, Object>> alerts = (List<Map<String, Object>>) d
-                .get(MetaAlertDao.ALERT_FIELD);
+                .get(ALERT_FIELD);
 
             for (Map<String, Object> alert : alerts) {
               Object newField = alert.get(fieldName);
@@ -1039,90 +307,60 @@ public class ElasticsearchMetaAlertIntegrationTest {
     return cnt;
   }
 
-  protected void findUpdatedDoc(Map<String, Object> message0, String guid, String sensorType)
-      throws InterruptedException, IOException, OriginalNotFoundException {
-    for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) {
-      Document doc = metaDao.getLatest(guid, sensorType);
-      if (doc != null && message0.equals(doc.getDocument())) {
-        return;
-      }
-    }
-    throw new OriginalNotFoundException("Count not find " + guid + " after " + MAX_RETRIES + " tries");
+  @Override
+  protected void addRecords(List<Map<String, Object>> inputData, String index, String docType)
+      throws IOException {
+    es.add(index, docType, inputData.stream().map(m -> {
+          try {
+            return JSONUtils.INSTANCE.toJSON(m, true);
+          } catch (JsonProcessingException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+          }
+        }
+        ).collect(Collectors.toList())
+    );
   }
 
-  protected boolean findCreatedDoc(String guid, String sensorType)
-      throws InterruptedException, IOException, OriginalNotFoundException {
-    for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) {
-      Document doc = metaDao.getLatest(guid, sensorType);
-      if (doc != null) {
-        return true;
-      }
-    }
-    throw new OriginalNotFoundException("Count not find " + guid + " after " + MAX_RETRIES + "tries");
+  @Override
+  protected void setupTypings() {
+    ((ElasticsearchDao) esDao).getClient().admin().indices().preparePutMapping(INDEX_WITH_SEPARATOR)
+        .setType("test_doc")
+        .setSource(nestedAlertMapping)
+        .get();
   }
 
-  protected boolean findCreatedDocs(List<GetRequest> getRequests)
-      throws InterruptedException, IOException, OriginalNotFoundException {
-    for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) {
-      Iterable<Document> docs = metaDao.getAllLatest(getRequests);
-      if (docs != null) {
-        int docCount = 0;
-        for (Document doc: docs) {
-          docCount++;
-        }
-        if (getRequests.size() == docCount) {
-          return true;
-        }
-      }
-    }
-    throw new OriginalNotFoundException("Count not find guids after " + MAX_RETRIES + "tries");
+  @Override
+  protected String getTestIndexName() {
+    return INDEX;
   }
 
-  protected List<Map<String, Object>> buildAlerts(int count) {
-    List<Map<String, Object>> inputData = new ArrayList<>();
-    for (int i = 0; i < count; ++i) {
-      final String guid = "message_" + i;
-      Map<String, Object> alerts = new HashMap<>();
-      alerts.put(Constants.GUID, guid);
-      alerts.put("source:type", SENSOR_NAME);
-      alerts.put(MetaAlertDao.THREAT_FIELD_DEFAULT, i);
-      alerts.put("timestamp", System.currentTimeMillis());
-      inputData.add(alerts);
-    }
-    return inputData;
+  @Override
+  protected String getTestIndexFullName() {
+    return INDEX_WITH_SEPARATOR;
   }
 
-  protected List<Map<String, Object>> buildMetaAlerts(int count, MetaAlertStatus status, Optional<List<Map<String, Object>>> alerts) {
-    List<Map<String, Object>> inputData = new ArrayList<>();
-    for (int i = 0; i < count; ++i) {
-      final String guid = "meta_" + status.getStatusString() + "_" + i;
-      inputData.add(buildMetaAlert(guid, status, alerts));
-    }
-    return inputData;
+  @Override
+  protected String getMetaAlertIndex() {
+    return METAALERTS_INDEX;
   }
 
-  protected Map<String, Object> buildMetaAlert(String guid, MetaAlertStatus status, Optional<List<Map<String, Object>>> alerts) {
-    Map<String, Object> metaAlert = new HashMap<>();
-    metaAlert.put(Constants.GUID, guid);
-    metaAlert.put("source:type", METAALERT_TYPE);
-    metaAlert.put(MetaAlertDao.STATUS_FIELD, status.getStatusString());
-    if (alerts.isPresent()) {
-      List<Map<String, Object>> alertsList = alerts.get();
-      metaAlert.put(ALERT_FIELD, alertsList);
-    }
-    return metaAlert;
+  @Override
+  protected String getSourceTypeField() {
+    return ElasticsearchMetaAlertDao.SOURCE_TYPE_FIELD;
   }
 
-  protected void elasticsearchAdd(List<Map<String, Object>> inputData, String index, String docType)
-      throws IOException {
-    es.add(index, docType, inputData.stream().map(m -> {
-          try {
-            return JSONUtils.INSTANCE.toJSON(m, true);
-          } catch (JsonProcessingException e) {
-            throw new IllegalStateException(e.getMessage(), e);
-          }
-        }
-        ).collect(Collectors.toList())
-    );
+  @Override
+  protected void setEmptiedMetaAlertField(Map<String, Object> docMap) {
+    docMap.put(METAALERT_FIELD, new ArrayList<>());
+  }
+
+  @Override
+  protected boolean isFiniteDoubleOnly() {
+    return true;
+  }
+
+  @Override
+  protected boolean isEmptyMetaAlertList() {
+    return true;
   }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java
index bb28abb..6f76093 100644
--- a/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java
+++ b/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchSearchIntegrationTest.java
@@ -22,12 +22,8 @@ import java.io.File;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
-import java.io.IOException;
 import java.util.List;
 import java.util.Map;
-import java.util.HashMap;
-import java.util.List;
-import java.util.concurrent.ExecutionException;
 import org.adrianwalker.multilinestring.Multiline;
 import org.apache.metron.common.Constants;
 import org.apache.metron.common.utils.JSONUtils;
@@ -36,11 +32,8 @@ import org.apache.metron.elasticsearch.integration.components.ElasticSearchCompo
 import org.apache.metron.indexing.dao.AccessConfig;
 import org.apache.metron.indexing.dao.IndexDao;
 import org.apache.metron.indexing.dao.SearchIntegrationTest;
-import org.apache.metron.indexing.dao.search.GetRequest;
 import org.apache.metron.indexing.dao.search.FieldType;
 import org.apache.metron.indexing.dao.search.GroupRequest;
-import org.apache.metron.indexing.dao.search.GroupResponse;
-import org.apache.metron.indexing.dao.search.GroupResult;
 import org.apache.metron.indexing.dao.search.InvalidSearchException;
 import org.apache.metron.indexing.dao.search.SearchRequest;
 import org.apache.metron.indexing.dao.search.SearchResponse;
@@ -54,13 +47,8 @@ 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.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.concurrent.ExecutionException;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class ElasticsearchSearchIntegrationTest extends SearchIntegrationTest {
@@ -69,6 +57,7 @@ public class ElasticsearchSearchIntegrationTest extends SearchIntegrationTest {
   private static String dateFormat = "yyyy.MM.dd.HH";
   private static final int MAX_RETRIES = 10;
   private static final int SLEEP_MS = 500;
+  protected static IndexDao dao;
 
   /**
    * {
@@ -196,8 +185,15 @@ public class ElasticsearchSearchIntegrationTest extends SearchIntegrationTest {
   @Multiline
   private static String broDefaultStringMappings;
 
-  @Override
-  protected IndexDao createDao() throws Exception {
+  @BeforeClass
+  public static void setup() throws Exception {
+    indexComponent = startIndex();
+    dao = createDao();
+    // The data is all static for searches, so we can set it up beforehand, and it's faster
+    loadTestData();
+  }
+
+  protected static IndexDao createDao() {
     AccessConfig config = new AccessConfig();
     config.setMaxSearchResults(100);
     config.setMaxSearchGroups(100);
@@ -215,8 +211,7 @@ public class ElasticsearchSearchIntegrationTest extends SearchIntegrationTest {
     return dao;
   }
 
-  @Override
-  protected InMemoryComponent startIndex() throws Exception {
+  protected static InMemoryComponent startIndex() throws Exception {
     InMemoryComponent es = new ElasticSearchComponent.Builder()
             .withHttpPort(9211)
             .withIndexDir(new File(indexDir))
@@ -225,32 +220,36 @@ public class ElasticsearchSearchIntegrationTest extends SearchIntegrationTest {
     return es;
   }
 
-  @Override
-  protected void loadTestData()
-      throws ParseException, IOException, ExecutionException, InterruptedException {
-    ElasticSearchComponent es = (ElasticSearchComponent)indexComponent;
+  protected static void loadTestData() throws ParseException {
+    ElasticSearchComponent es = (ElasticSearchComponent) indexComponent;
     es.getClient().admin().indices().prepareCreate("bro_index_2017.01.01.01")
-            .addMapping("bro_doc", broTypeMappings).addMapping("bro_doc_default", broDefaultStringMappings).get();
+        .addMapping("bro_doc", broTypeMappings)
+        .addMapping("bro_doc_default", broDefaultStringMappings).get();
     es.getClient().admin().indices().prepareCreate("snort_index_2017.01.01.02")
-            .addMapping("snort_doc", snortTypeMappings).get();
+        .addMapping("snort_doc", snortTypeMappings).get();
 
-    BulkRequestBuilder bulkRequest = es.getClient().prepareBulk().setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL);
+    BulkRequestBuilder bulkRequest = es.getClient().prepareBulk()
+        .setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL);
     JSONArray broArray = (JSONArray) new JSONParser().parse(broData);
-    for(Object o: broArray) {
+    for (Object o : broArray) {
       JSONObject jsonObject = (JSONObject) o;
-      IndexRequestBuilder indexRequestBuilder = es.getClient().prepareIndex("bro_index_2017.01.01.01", "bro_doc");
+      IndexRequestBuilder indexRequestBuilder = es.getClient()
+          .prepareIndex("bro_index_2017.01.01.01", "bro_doc");
       indexRequestBuilder = indexRequestBuilder.setId((String) jsonObject.get("guid"));
       indexRequestBuilder = indexRequestBuilder.setSource(jsonObject.toJSONString());
-      indexRequestBuilder = indexRequestBuilder.setTimestamp(jsonObject.get("timestamp").toString());
+      indexRequestBuilder = indexRequestBuilder
+          .setTimestamp(jsonObject.get("timestamp").toString());
       bulkRequest.add(indexRequestBuilder);
     }
     JSONArray snortArray = (JSONArray) new JSONParser().parse(snortData);
-    for(Object o: snortArray) {
+    for (Object o : snortArray) {
       JSONObject jsonObject = (JSONObject) o;
-      IndexRequestBuilder indexRequestBuilder = es.getClient().prepareIndex("snort_index_2017.01.01.02", "snort_doc");
+      IndexRequestBuilder indexRequestBuilder = es.getClient()
+          .prepareIndex("snort_index_2017.01.01.02", "snort_doc");
       indexRequestBuilder = indexRequestBuilder.setId((String) jsonObject.get("guid"));
       indexRequestBuilder = indexRequestBuilder.setSource(jsonObject.toJSONString());
-      indexRequestBuilder = indexRequestBuilder.setTimestamp(jsonObject.get("timestamp").toString());
+      indexRequestBuilder = indexRequestBuilder
+          .setTimestamp(jsonObject.get("timestamp").toString());
       bulkRequest.add(indexRequestBuilder);
     }
     BulkResponse bulkResponse = bulkRequest.execute().actionGet();
@@ -357,4 +356,9 @@ public class ElasticsearchSearchIntegrationTest extends SearchIntegrationTest {
   protected String getSourceTypeField() {
     return Constants.SENSOR_TYPE.replace('.', ':');
   }
+
+  @Override
+  protected IndexDao getIndexDao() {
+    return dao;
+  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/ElasticsearchUpdateIntegrationTest.java
----------------------------------------------------------------------
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 0080d75..97993ff 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
@@ -20,17 +20,29 @@ package org.apache.metron.elasticsearch.integration;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.google.common.collect.Iterables;
 import java.io.File;
+import java.io.IOException;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 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.mock.MockHBaseTableProvider;
+import org.apache.metron.hbase.mock.MockHTable;
+import org.apache.metron.indexing.dao.AccessConfig;
+import org.apache.metron.indexing.dao.HBaseDao;
 import org.apache.metron.indexing.dao.IndexDao;
+import org.apache.metron.indexing.dao.MultiIndexDao;
 import org.apache.metron.indexing.dao.UpdateIntegrationTest;
-import org.apache.metron.integration.InMemoryComponent;
+import org.apache.metron.integration.UnableToStartException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
 
 public class ElasticsearchUpdateIntegrationTest extends UpdateIntegrationTest {
   private static final String SENSOR_NAME= "test";
@@ -39,13 +51,56 @@ public class ElasticsearchUpdateIntegrationTest extends UpdateIntegrationTest {
   private static String index = SENSOR_NAME + "_index_" + new SimpleDateFormat(dateFormat).format(new Date());
   private static ElasticSearchComponent es;
 
+  private static final String TABLE_NAME = "modifications";
+  private static final String CF = "p";
+  private static MockHTable table;
+  private static IndexDao hbaseDao;
+
   @Override
   protected String getIndexName() {
     return SENSOR_NAME + "_index_" + new SimpleDateFormat(dateFormat).format(new Date());
   }
 
-  @Override
-  protected Map<String, Object> createGlobalConfig() throws Exception {
+  @BeforeClass
+  public static void setupBeforeClass() throws UnableToStartException {
+    es = new ElasticSearchComponent.Builder()
+        .withHttpPort(9211)
+        .withIndexDir(new File(indexDir))
+        .build();
+    es.start();
+  }
+
+  @Before
+  public void setup() throws IOException {
+    Configuration config = HBaseConfiguration.create();
+    MockHBaseTableProvider tableProvider = new MockHBaseTableProvider();
+    MockHBaseTableProvider.addToCache(TABLE_NAME, CF);
+    table = (MockHTable) tableProvider.getTable(config, TABLE_NAME);
+
+    hbaseDao = new HBaseDao();
+    AccessConfig accessConfig = new AccessConfig();
+    accessConfig.setTableProvider(tableProvider);
+    Map<String, Object> globalConfig = createGlobalConfig();
+    globalConfig.put(HBaseDao.HBASE_TABLE, TABLE_NAME);
+    globalConfig.put(HBaseDao.HBASE_CF, CF);
+    accessConfig.setGlobalConfigSupplier(() -> globalConfig);
+
+    dao = new MultiIndexDao(hbaseDao, createDao());
+    dao.init(accessConfig);
+  }
+
+  @After
+  public void reset() {
+    es.reset();
+    table.clear();
+  }
+
+  @AfterClass
+  public static void teardown() {
+    es.stop();
+  }
+
+  protected static Map<String, Object> createGlobalConfig() {
     return new HashMap<String, Object>() {{
       put("es.clustername", "metron");
       put("es.port", "9300");
@@ -54,27 +109,11 @@ public class ElasticsearchUpdateIntegrationTest extends UpdateIntegrationTest {
     }};
   }
 
-  @Override
-  protected IndexDao createDao() throws Exception {
+  protected static IndexDao createDao() {
     return new ElasticsearchDao();
   }
 
   @Override
-  protected InMemoryComponent startIndex() throws Exception {
-    es = new ElasticSearchComponent.Builder()
-        .withHttpPort(9211)
-        .withIndexDir(new File(indexDir))
-        .build();
-    es.start();
-    return es;
-  }
-
-  @Override
-  protected void loadTestData() throws Exception {
-
-  }
-
-  @Override
   protected void addTestData(String indexName, String sensorType,
       List<Map<String, Object>> docs) throws Exception {
     es.add(index, SENSOR_NAME
@@ -94,4 +133,9 @@ public class ElasticsearchUpdateIntegrationTest extends UpdateIntegrationTest {
   protected List<Map<String, Object>> getIndexedTestData(String indexName, String sensorType) throws Exception {
     return es.getAllIndexedDocs(index, SENSOR_NAME + "_doc");
   }
+
+  @Override
+  protected MockHTable getMockHTable() {
+    return table;
+  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-elasticsearch/src/test/java/org/apache/metron/elasticsearch/integration/components/ElasticSearchComponent.java
----------------------------------------------------------------------
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 e716ce1..45b4d60 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
@@ -274,19 +274,19 @@ public class ElasticSearchComponent implements InMemoryComponent {
 
   }
 
-    @Override
-    public void stop() {
-      try {
-        node.close();
-      } catch (IOException e) {
-        throw new RuntimeException("Unable to stop node." , e);
-      }
-      node = null;
-      client = null;
+  @Override
+  public void stop() {
+    try {
+      node.close();
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to stop node." , e);
     }
+    node = null;
+    client = null;
+  }
 
-    @Override
-    public void reset() {
-        client.admin().indices().delete(new DeleteIndexRequest("*")).actionGet();
-    }
+  @Override
+  public void reset() {
+      client.admin().indices().delete(new DeleteIndexRequest("*")).actionGet();
+  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/README.md b/metron-platform/metron-indexing/README.md
index f4a4501..7a2ec29 100644
--- a/metron-platform/metron-indexing/README.md
+++ b/metron-platform/metron-indexing/README.md
@@ -194,7 +194,7 @@ The HBase column family to use for message updates.
 ### The `MetaAlertDao`
 
 The goal of meta alerts is to be able to group together a set of alerts while being able to transparently perform actions
-like searches, as if meta alerts were normal alerts.  `org.apache.metron.indexing.dao.MetaAlertDao` extends `IndexDao` and
+like searches, as if meta alerts were normal alerts.  `org.apache.metron.indexing.dao.metaalert.MetaAlertDao` extends `IndexDao` and
 enables several features: 
 * the ability to get all meta alerts associated with an alert
 * creation of a meta alert

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/pom.xml b/metron-platform/metron-indexing/pom.xml
index e7164e7..8561368 100644
--- a/metron-platform/metron-indexing/pom.xml
+++ b/metron-platform/metron-indexing/pom.xml
@@ -143,7 +143,7 @@
         </dependency>
         <dependency>
             <groupId>org.mockito</groupId>
-            <artifactId>mockito-all</artifactId>
+            <artifactId>mockito-core</artifactId>
             <version>${global_mockito_version}</version>
             <scope>test</scope>
         </dependency>
@@ -197,6 +197,12 @@
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.hamcrest</groupId>
+            <artifactId>hamcrest-core</artifactId>
+            <version>1.3</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
     <build>
         <plugins>

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/AccessConfig.java
----------------------------------------------------------------------
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 c301050..b1df46a 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,6 +17,7 @@
  */
 package org.apache.metron.indexing.dao;
 
+import java.util.function.Function;
 import org.apache.metron.hbase.TableProvider;
 
 import java.util.HashMap;
@@ -27,6 +28,7 @@ public class AccessConfig {
   private Integer maxSearchResults;
   private Integer maxSearchGroups;
   private Supplier<Map<String, Object>> globalConfigSupplier;
+  private Function<String, String> indexSupplier;
   private Map<String, String> optionalSettings = new HashMap<>();
   private TableProvider tableProvider = null;
   private Boolean isKerberosEnabled = false;
@@ -42,6 +44,14 @@ public class AccessConfig {
     this.globalConfigSupplier = globalConfigSupplier;
   }
 
+  public Function<String, String> getIndexSupplier() {
+    return indexSupplier;
+  }
+
+  public void setIndexSupplier(Function<String, String> indexSupplier) {
+    this.indexSupplier = indexSupplier;
+  }
+
   /**
    * @return The maximum number of search results.
    */


[5/7] metron git commit: METRON-1421 Create a SolrMetaAlertDao (justinleet) closes apache/metron#970

Posted by le...@apache.org.
http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/IndexDao.java
----------------------------------------------------------------------
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 fe546bd..4187428 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,151 +17,18 @@
  */
 package org.apache.metron.indexing.dao;
 
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
-import org.apache.metron.common.utils.JSONUtils;
-import org.apache.metron.indexing.dao.search.FieldType;
-import org.apache.metron.indexing.dao.search.GetRequest;
-import org.apache.metron.indexing.dao.search.GroupRequest;
-import org.apache.metron.indexing.dao.search.GroupResponse;
-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.indexing.dao.update.OriginalNotFoundException;
-import org.apache.metron.indexing.dao.update.PatchRequest;
-import org.apache.metron.indexing.dao.update.ReplaceRequest;
+import org.apache.metron.indexing.dao.search.SearchDao;
+import org.apache.metron.indexing.dao.update.UpdateDao;
 
 /**
  * The IndexDao provides a common interface for retrieving and storing data in a variety of persistent stores.
  * Document reads and writes require a GUID and sensor type with an index being optional.
  */
-public interface IndexDao {
-
-  /**
-   * Return search response based on the search request
-   *
-   * @param searchRequest
-   * @return
-   * @throws InvalidSearchException
-   */
-  SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException;
-
-  GroupResponse group(GroupRequest groupRequest) throws InvalidSearchException;
+public interface IndexDao extends UpdateDao, SearchDao, RetrieveLatestDao, ColumnMetadataDao {
 
   /**
    * Initialize the DAO with the AccessConfig object.
-   * @param config
+   * @param config The config to use for initialization
    */
   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 a list of the latest versions of documents given a list of GUIDs and sensor types.
-   *
-   * @param getRequests A list of get requests for documents
-   * @return A list of documents matching or an empty list in not available.
-   * @throws IOException
-   */
-  Iterable<Document> getAllLatest(List<GetRequest> getRequests) 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<Map<String, Object>> getLatestResult(GetRequest request) throws IOException {
-    Document ret = getLatest(request.getGuid(), request.getSensorType());
-    if(ret == null) {
-      return Optional.empty();
-    }
-    else {
-      return Optional.ofNullable(ret.getDocument());
-    }
-  }
-
-  /**
-   * Update a given Document and optionally the index where the document exists.  This is a full update,
-   * meaning the current document will be replaced if it exists or a new document will be created if it does
-   * not exist.  Partial updates are not supported in this method.
-   *
-   * @param update The document to replace from the index.
-   * @param index The index where the document lives.
-   * @throws IOException
-   */
-  void update(Document update, Optional<String> index) throws IOException;
-
-  /**
-   * Similar to the update method but accepts multiple documents and performs updates in batch.
-   *
-   * @param updates A map of the documents to update to the index where they live.
-   * @throws IOException
-   */
-  void batchUpdate(Map<Document, Optional<String>> updates) 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<Long> timestamp
-                    ) throws OriginalNotFoundException, IOException {
-    Document d = getPatchedDocument(request, timestamp);
-    update(d, Optional.ofNullable(request.getIndex()));
-  }
-
-  default Document getPatchedDocument(PatchRequest request
-      , Optional<Long> timestamp
-  ) throws OriginalNotFoundException, IOException {
-    Map<String, Object> latest = request.getSource();
-    if(latest == null) {
-      Document latestDoc = getLatest(request.getGuid(), request.getSensorType());
-      if(latestDoc != null && latestDoc.getDocument() != null) {
-        latest = latestDoc.getDocument();
-      }
-      else {
-        throw new OriginalNotFoundException("Unable to patch an document that doesn't exist and isn't specified.");
-      }
-    }
-    Map<String, Object> updated = JSONUtils.INSTANCE.applyPatch(request.getPatch(), latest);
-    return new Document(updated
-            , request.getGuid()
-            , request.getSensorType()
-            , timestamp.orElse(System.currentTimeMillis()));
-  }
-
-  /**
-   * 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<Long> timestamp
-                      ) throws IOException {
-    Document d = new Document(request.getReplacement()
-                             , request.getGuid()
-                             , request.getSensorType()
-                             , timestamp.orElse(System.currentTimeMillis())
-                             );
-    update(d, Optional.ofNullable(request.getIndex()));
-  }
-
-  Map<String, FieldType> getColumnMetadata(List<String> indices) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MetaAlertDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MetaAlertDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MetaAlertDao.java
deleted file mode 100644
index 4530d2a..0000000
--- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/MetaAlertDao.java
+++ /dev/null
@@ -1,154 +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.dao;
-
-import java.util.List;
-import java.util.Optional;
-import java.io.IOException;
-import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest;
-import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateResponse;
-import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
-import org.apache.metron.indexing.dao.search.GetRequest;
-import org.apache.metron.indexing.dao.search.InvalidCreateException;
-import org.apache.metron.indexing.dao.search.InvalidSearchException;
-import org.apache.metron.indexing.dao.search.SearchResponse;
-
-/**
- * The MetaAlertDao exposes methods for interacting with meta alerts.  Meta alerts are objects that contain
- * alerts and summary statistics based on the scores of these alerts.  Meta alerts are returned in searches
- * just as alerts are and match based on the field values of child alerts.  If a child alert matches a search
- * the meta alert will be returned while the original child alert will not.  A meta alert also contains a
- * status field that controls it's inclusion in search results and a groups field that can be used to track
- * the groups a meta alert was created from.
- *
- * The structure of a meta alert is as follows:
- * {
- *   "guid": "meta alert guid",
- *   "timestamp": timestamp,
- *   "source:type": "metaalert",
- *   "alerts": [ array of child alerts ],
- *   "status": "active or inactive",
- *   "groups": [ array of group names ],
- *   "average": 10,
- *   "max": 10,
- *   "threat:triage:score": 30,
- *   "count": 3,
- *   "sum": 30,
- *   "min": 10,
- *   "median": 10
- * }
- *
- * A child alert that has been added to a meta alert will store the meta alert GUID in a "metaalerts" field.
- * This field is an array of meta alert GUIDs, meaning a child alert can be contained in multiple meta alerts.
- * Any update to a child alert will trigger an update to the meta alert so that the alert inside a meta alert
- * and the original alert will be kept in sync.
- *
- * Other fields can be added to a meta alert through the patch method on the IndexDao interface.  However, attempts
- * to directly change the "alerts" or "status" field will result in an exception.
- */
-public interface MetaAlertDao extends IndexDao {
-
-  String METAALERTS_INDEX = "metaalert_index";
-  String METAALERT_TYPE = "metaalert";
-  String METAALERT_FIELD = "metaalerts";
-  String METAALERT_DOC = METAALERT_TYPE + "_doc";
-  String THREAT_FIELD_DEFAULT = "threat:triage:score";
-  String THREAT_SORT_DEFAULT = "sum";
-  String ALERT_FIELD = "alert";
-  String STATUS_FIELD = "status";
-  String GROUPS_FIELD = "groups";
-
-  /**
-   * Given an alert GUID, retrieve all associated meta alerts.
-   * @param guid The alert GUID to be searched for
-   * @return All meta alerts with a child alert having the GUID
-   * @throws InvalidSearchException If a problem occurs with the search
-   */
-  SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException;
-
-  /**
-   * Creates a meta alert from a list of child alerts.  The most recent version of each child alert is
-   * retrieved using the DAO abstractions.
-   *
-   * @param request A request object containing get requests for alerts to be added and a list of groups
-   * @return A response indicating success or failure along with the GUID of the new meta alert
-   * @throws InvalidCreateException If a malformed create request is provided
-   * @throws IOException If a problem occurs during communication
-   */
-  MetaAlertCreateResponse createMetaAlert(MetaAlertCreateRequest request)
-      throws InvalidCreateException, IOException;
-
-
-  /**
-   * Adds a list of alerts to an existing meta alert.  This will add each alert object to the "alerts" array in the meta alert
-   * and also add the meta alert GUID to each child alert's "metaalerts" array.  After alerts have been added the
-   * meta alert scores are recalculated.  Any alerts already in the meta alert are skipped and no updates are
-   * performed if all of the alerts are already in the meta alert.  The most recent version of each child alert is
-   * retrieved using the DAO abstractions.  Alerts cannot be added to an 'inactive' meta alert.
-   *
-   * @param metaAlertGuid The meta alert GUID
-   * @param getRequests Get requests for alerts to be added
-   * @return True or false depending on if any alerts were added
-   * @throws IOException
-   */
-  boolean addAlertsToMetaAlert(String metaAlertGuid, List<GetRequest> getRequests) throws IOException;
-
-  /**
-   * Removes a list of alerts from an existing meta alert.  This will remove each alert object from the "alerts" array in the meta alert
-   * and also remove the meta alert GUID from each child alert's "metaalerts" array.  After alerts have been removed the
-   * meta alert scores are recalculated.  Any alerts not contained in the meta alert are skipped and no updates are
-   * performed if no alerts can be found in the meta alert.  Alerts cannot be removed from an 'inactive' meta alert.
-   *
-   * @param metaAlertGuid The meta alert GUID
-   * @param getRequests Get requests for alerts to be removed
-   * @return True or false depending on if any alerts were removed
-   * @throws IOException
-   */
-  boolean removeAlertsFromMetaAlert(String metaAlertGuid, List<GetRequest> getRequests) throws IOException;
-
-  /**
-   * The meta alert status field can be set to either 'active' or 'inactive' and will control whether or not meta alerts
-   * (and child alerts) appear in search results.  An 'active' status will cause meta alerts to appear in search
-   * results instead of it's child alerts and an 'inactive' status will suppress the meta alert from search results
-   * with child alerts appearing in search results as normal.  A change to 'inactive' will cause the meta alert GUID to
-   * be removed from all it's child alert's "metaalerts" field.  A change back to 'active' will have the opposite effect.
-   *
-   * @param metaAlertGuid The GUID of the meta alert
-   * @param status A status value of 'active' or 'inactive'
-   * @return True or false depending on if the status was changed
-   * @throws IOException
-   */
-  boolean updateMetaAlertStatus(String metaAlertGuid, MetaAlertStatus status) throws IOException;
-
-  /**
-   * Initializes a Meta Alert DAO with default "sum" meta alert threat sorting.
-   * @param indexDao The DAO to wrap for our queries.
-   */
-  default void init(IndexDao indexDao) {
-    init(indexDao, Optional.empty());
-  }
-
-  /**
-   * Initializes a Meta Alert DAO.
-   * @param indexDao The DAO to wrap for our queries
-   * @param threatSort The aggregation to use as the threat field. E.g. "sum", "median", etc.
-   *     null is "sum"
-   */
-  void init(IndexDao indexDao, Optional<String> threatSort);
-}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/RetrieveLatestDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/RetrieveLatestDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/RetrieveLatestDao.java
new file mode 100644
index 0000000..caf754c
--- /dev/null
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/RetrieveLatestDao.java
@@ -0,0 +1,67 @@
+/*
+ * 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;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.update.Document;
+
+/**
+ * An base interface for other DAOs to extend.  All DAOs are expected to be able to retrieve
+ * Documents they've stored.
+ */
+public interface RetrieveLatestDao {
+
+  /**
+   * 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 If an error occurs retrieving the latest document.
+   */
+  Document getLatest(String guid, String sensorType) throws IOException;
+
+  /**
+   * Return a list of the latest versions of documents given a list of GUIDs and sensor types.
+   *
+   * @param getRequests A list of get requests for documents
+   * @return A list of documents matching or an empty list in not available.
+   * @throws IOException If an error occurs retrieving the latest documents.
+   */
+  Iterable<Document> getAllLatest(List<GetRequest> getRequests) 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 If an error occurs while retrieving the document.
+   */
+  default Optional<Map<String, Object>> getLatestResult(GetRequest request) throws IOException {
+    Document ret = getLatest(request.getGuid(), request.getSensorType());
+    if (ret == null) {
+      return Optional.empty();
+    } else {
+      return Optional.ofNullable(ret.getDocument());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/DeferredMetaAlertIndexDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/DeferredMetaAlertIndexDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/DeferredMetaAlertIndexDao.java
new file mode 100644
index 0000000..1e5e723
--- /dev/null
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/DeferredMetaAlertIndexDao.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.dao.metaalert;
+
+import org.apache.metron.indexing.dao.IndexDao;
+
+/**
+ * Interface for a DAO that is allowed to defer to a child Index DAO in order to perform tasks.
+ * An example is metaalerts deferring to a base DAO.
+ */
+public interface DeferredMetaAlertIndexDao {
+
+  IndexDao getIndexDao();
+
+  String getMetAlertSensorName();
+
+  String getMetaAlertIndex();
+
+  default String getThreatTriageField() {
+    return MetaAlertConstants.THREAT_FIELD_DEFAULT;
+  }
+
+  default String getThreatSort() {
+    return MetaAlertConstants.THREAT_SORT_DEFAULT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertAddRemoveRequest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertAddRemoveRequest.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertAddRemoveRequest.java
index 6183d37..a14749b 100644
--- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertAddRemoveRequest.java
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertAddRemoveRequest.java
@@ -14,7 +14,6 @@
  */
 package org.apache.metron.indexing.dao.metaalert;
 
-import java.util.Collection;
 import java.util.List;
 import org.apache.metron.indexing.dao.search.GetRequest;
 

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertConfig.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertConfig.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertConfig.java
new file mode 100644
index 0000000..9254425
--- /dev/null
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertConfig.java
@@ -0,0 +1,74 @@
+/*
+ * 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.metaalert;
+
+public class MetaAlertConfig {
+  private String metaAlertIndex;
+  private String threatTriageField;
+  private String threatSort;
+  private String sourceTypeField;
+
+  /**
+   * Simple object for storing and retrieving configs, primarily to make passing all the info to
+   * the sub DAOs easier.
+   * @param metaAlertIndex The metaalert index or collection we're using
+   * @param threatTriageField The threat triage field's name
+   * @param threatSort The sorting operation on the threat triage field
+   * @param sourceTypeField The source type field
+   */
+  public MetaAlertConfig(String metaAlertIndex, String threatTriageField,
+      String threatSort, String sourceTypeField) {
+    this.metaAlertIndex = metaAlertIndex;
+    this.threatTriageField = threatTriageField;
+    this.threatSort = threatSort;
+    this.sourceTypeField = sourceTypeField;
+  }
+
+  public String getMetaAlertIndex() {
+    return metaAlertIndex;
+  }
+
+  public void setMetaAlertIndex(String metaAlertIndex) {
+    this.metaAlertIndex = metaAlertIndex;
+  }
+
+  public String getThreatTriageField() {
+    return threatTriageField;
+  }
+
+  public void setThreatTriageField(String threatTriageField) {
+    this.threatTriageField = threatTriageField;
+  }
+
+  public String getThreatSort() {
+    return threatSort;
+  }
+
+  public void setThreatSort(String threatSort) {
+    this.threatSort = threatSort;
+  }
+
+  public String getSourceTypeField() {
+    return sourceTypeField;
+  }
+
+  public void setSourceTypeField(String sourceTypeField) {
+    this.sourceTypeField = sourceTypeField;
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertConstants.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertConstants.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertConstants.java
new file mode 100644
index 0000000..a055db5
--- /dev/null
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertConstants.java
@@ -0,0 +1,30 @@
+/*
+ * 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.metaalert;
+
+public class MetaAlertConstants {
+  public static String METAALERT_TYPE = "metaalert";
+  public static String METAALERT_FIELD = "metaalerts";
+  public static String METAALERT_DOC = METAALERT_TYPE + "_doc";
+  public static String THREAT_FIELD_DEFAULT = "threat:triage:score";
+  public static String THREAT_SORT_DEFAULT = "sum";
+  public static String ALERT_FIELD = "alert";
+  public static String STATUS_FIELD = "status";
+  public static String GROUPS_FIELD = "groups";
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertDao.java
new file mode 100644
index 0000000..c9e6711
--- /dev/null
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertDao.java
@@ -0,0 +1,77 @@
+/*
+ * 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.metaalert;
+
+import java.util.Optional;
+import org.apache.metron.indexing.dao.IndexDao;
+
+/**
+ * The MetaAlertDao exposes methods for interacting with meta alerts.  Meta alerts are objects that contain
+ * alerts and summary statistics based on the scores of these alerts.  Meta alerts are returned in searches
+ * just as alerts are and match based on the field values of child alerts.  If a child alert matches a search
+ * the meta alert will be returned while the original child alert will not.  A meta alert also contains a
+ * status field that controls it's inclusion in search results and a groups field that can be used to track
+ * the groups a meta alert was created from.
+ *
+ * </p>
+ * The structure of a meta alert is as follows:
+ * {
+ *   "guid": "meta alert guid",
+ *   "timestamp": timestamp,
+ *   "source:type": "metaalert",
+ *   "alerts": [ array of child alerts ],
+ *   "status": "active or inactive",
+ *   "groups": [ array of group names ],
+ *   "average": 10,
+ *   "max": 10,
+ *   "threat:triage:score": 30,
+ *   "count": 3,
+ *   "sum": 30,
+ *   "min": 10,
+ *   "median": 10
+ * }
+ *
+ * </p>
+ * A child alert that has been added to a meta alert will store the meta alert GUID in a "metaalerts" field.
+ * This field is an array of meta alert GUIDs, meaning a child alert can be contained in multiple meta alerts.
+ * Any update to a child alert will trigger an update to the meta alert so that the alert inside a meta alert
+ * and the original alert will be kept in sync.
+ *
+ * </p>
+ * Other fields can be added to a meta alert through the patch method on the IndexDao interface.  However, attempts
+ * to directly change the "alerts" or "status" field will result in an exception.
+ */
+public interface MetaAlertDao extends MetaAlertSearchDao, MetaAlertUpdateDao, IndexDao {
+
+  /**
+   * Initializes a Meta Alert DAO with default "sum" meta alert threat sorting.
+   * @param indexDao The DAO to wrap for our queries.
+   */
+  default void init(IndexDao indexDao) {
+    init(indexDao, Optional.empty());
+  }
+
+  /**
+   * Initializes a Meta Alert DAO.
+   * @param indexDao The DAO to wrap for our queries
+   * @param threatSort The aggregation to use as the threat field. E.g. "sum", "median", etc.
+   *     null is "sum"
+   */
+  void init(IndexDao indexDao, Optional<String> threatSort);
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertRetrieveLatestDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertRetrieveLatestDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertRetrieveLatestDao.java
new file mode 100644
index 0000000..1a0d2a0
--- /dev/null
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertRetrieveLatestDao.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.metaalert;
+
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
+
+public interface MetaAlertRetrieveLatestDao extends RetrieveLatestDao {
+
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java
new file mode 100644
index 0000000..e8b9f26
--- /dev/null
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertSearchDao.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.metron.indexing.dao.metaalert;
+
+import org.apache.metron.indexing.dao.search.InvalidSearchException;
+import org.apache.metron.indexing.dao.search.SearchDao;
+import org.apache.metron.indexing.dao.search.SearchResponse;
+
+public interface MetaAlertSearchDao extends SearchDao {
+
+  /**
+   * Given an alert GUID, retrieve all associated meta alerts.
+   * @param guid The alert GUID to be searched for
+   * @return All meta alerts with a child alert having the GUID
+   * @throws InvalidSearchException If a problem occurs with the search
+   */
+  SearchResponse getAllMetaAlertsForAlert(String guid) throws InvalidSearchException;
+
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertUpdateDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertUpdateDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertUpdateDao.java
new file mode 100644
index 0000000..f4374b4
--- /dev/null
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaAlertUpdateDao.java
@@ -0,0 +1,146 @@
+/*
+ * 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.metaalert;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.search.InvalidCreateException;
+import org.apache.metron.indexing.dao.update.Document;
+import org.apache.metron.indexing.dao.update.PatchRequest;
+import org.apache.metron.indexing.dao.update.UpdateDao;
+
+public interface MetaAlertUpdateDao extends UpdateDao {
+
+  String STATUS_PATH = "/" + MetaAlertConstants.STATUS_FIELD;
+  String ALERT_PATH = "/" + MetaAlertConstants.ALERT_FIELD;
+
+  /**
+   * Determines if a given patch request is allowed or not. By default patching the 'alert' or
+   * 'status' fields are not allowed, because they should be updated via the specific methods.
+   * @param request The patch request to examine
+   * @return True if patch can be performed, false otherwise
+   */
+  default boolean isPatchAllowed(PatchRequest request) {
+    if (request.getPatch() != null && !request.getPatch().isEmpty()) {
+      for (Map<String, Object> patch : request.getPatch()) {
+        Object pathObj = patch.get("path");
+        if (pathObj != null && pathObj instanceof String) {
+          String path = (String) pathObj;
+          if (STATUS_PATH.equals(path) || ALERT_PATH.equals(path)) {
+            return false;
+          }
+        }
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Creates a meta alert from a list of child alerts.  The most recent version of each child alert is
+   * retrieved using the DAO abstractions.
+   *
+   * @param request A request object containing get requests for alerts to be added and a list of groups
+   * @return A response indicating success or failure along with the GUID of the new meta alert
+   * @throws InvalidCreateException If a malformed create request is provided
+   * @throws IOException If a problem occurs during communication
+   */
+  MetaAlertCreateResponse createMetaAlert(MetaAlertCreateRequest request)
+      throws InvalidCreateException, IOException;
+
+  /**
+   * Adds alerts to a metaalert, based on a list of GetRequests provided for retrieval.
+   * @param metaAlertGuid The GUID of the metaalert to be given new children.
+   * @param alertRequests GetRequests for the appropriate alerts to add.
+   * @return True if metaalert is modified, false otherwise.
+   */
+  boolean addAlertsToMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests)
+      throws IOException;
+
+  /**
+   * Removes alerts from a metaalert
+   * @param metaAlertGuid The metaalert guid to be affected.
+   * @param alertRequests A list of GetReqests that will provide the alerts to remove
+   * @return True if there are updates, false otherwise
+   * @throws IOException If an error is thrown during retrieal.
+   */
+  boolean removeAlertsFromMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests)
+      throws IOException;
+
+  /**
+   * Removes a metaalert link from a given alert. An nonexistent link performs no change.
+   * @param metaAlertGuid The metaalert GUID to link.
+   * @param alert The alert to be linked to.
+   * @return True if the alert changed, false otherwise.
+   */
+  default boolean removeMetaAlertFromAlert(String metaAlertGuid, Document alert) {
+    List<String> metaAlertField = new ArrayList<>();
+    @SuppressWarnings("unchecked")
+    List<String> alertField = (List<String>) alert.getDocument()
+        .get(MetaAlertConstants.METAALERT_FIELD);
+    if (alertField != null) {
+      metaAlertField.addAll(alertField);
+    }
+    boolean metaAlertRemoved = metaAlertField.remove(metaAlertGuid);
+    if (metaAlertRemoved) {
+      alert.getDocument().put(MetaAlertConstants.METAALERT_FIELD, metaAlertField);
+    }
+    return metaAlertRemoved;
+  }
+
+  /**
+   * The meta alert status field can be set to either 'active' or 'inactive' and will control whether or not meta alerts
+   * (and child alerts) appear in search results.  An 'active' status will cause meta alerts to appear in search
+   * results instead of it's child alerts and an 'inactive' status will suppress the meta alert from search results
+   * with child alerts appearing in search results as normal.  A change to 'inactive' will cause the meta alert GUID to
+   * be removed from all it's child alert's "metaalerts" field.  A change back to 'active' will have the opposite effect.
+   *
+   * @param metaAlertGuid The GUID of the meta alert
+   * @param status A status value of 'active' or 'inactive'
+   * @return True or false depending on if the status was changed
+   * @throws IOException if an error occurs during the update.
+   */
+  boolean updateMetaAlertStatus(String metaAlertGuid, MetaAlertStatus status)
+      throws IOException;
+
+  /**
+   * Adds a metaalert link to a provided alert Document.  Adding an existing link does no change.
+   * @param metaAlertGuid The GUID to be added.
+   * @param alert The alert we're adding the link to.
+   * @return True if the alert is modified, false if not.
+   */
+  default boolean addMetaAlertToAlert(String metaAlertGuid, Document alert) {
+    List<String> metaAlertField = new ArrayList<>();
+    @SuppressWarnings("unchecked")
+    List<String> alertField = (List<String>) alert.getDocument()
+        .get(MetaAlertConstants.METAALERT_FIELD);
+    if (alertField != null) {
+      metaAlertField.addAll(alertField);
+    }
+
+    boolean metaAlertAdded = !metaAlertField.contains(metaAlertGuid);
+    if (metaAlertAdded) {
+      metaAlertField.add(metaAlertGuid);
+      alert.getDocument().put(MetaAlertConstants.METAALERT_FIELD, metaAlertField);
+    }
+    return metaAlertAdded;
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaScores.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaScores.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaScores.java
index 07285d6..55b1aa0 100644
--- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaScores.java
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/MetaScores.java
@@ -18,12 +18,14 @@
 
 package org.apache.metron.indexing.dao.metaalert;
 
-import org.apache.commons.math3.stat.descriptive.rank.Median;
-
+import java.util.ArrayList;
 import java.util.DoubleSummaryStatistics;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import org.apache.commons.math3.stat.descriptive.rank.Median;
+import org.apache.metron.indexing.dao.update.Document;
+import org.apache.metron.stellar.common.utils.ConversionUtils;
 
 public class MetaScores {
 
@@ -52,4 +54,50 @@ public class MetaScores {
   public Map<String, Object> getMetaScores() {
     return metaScores;
   }
+
+  /**
+   * Calculate the meta alert scores for a Document. The scores are placed directly in the provided
+   * document.
+   * @param metaAlert The Document containing scores
+   */
+  @SuppressWarnings("unchecked")
+  public static void calculateMetaScores(Document metaAlert, String threatTriageField,
+      String threatSort) {
+    MetaScores metaScores = new MetaScores(new ArrayList<>());
+    List<Object> alertsRaw = ((List<Object>) metaAlert.getDocument()
+        .get(MetaAlertConstants.ALERT_FIELD));
+    if (alertsRaw != null && !alertsRaw.isEmpty()) {
+      ArrayList<Double> scores = new ArrayList<>();
+      for (Object alertRaw : alertsRaw) {
+        Map<String, Object> alert = (Map<String, Object>) alertRaw;
+        Double scoreNum = parseThreatField(alert.get(threatTriageField));
+        if (scoreNum != null) {
+          scores.add(scoreNum);
+        }
+      }
+      metaScores = new MetaScores(scores);
+    }
+
+    // add a summary (max, min, avg, ...) of all the threat scores from the child alerts
+    metaAlert.getDocument().putAll(metaScores.getMetaScores());
+
+    // add the overall threat score for the metaalert; one of the summary aggregations as defined
+    // by `threatSort`
+    Object threatScore = metaScores.getMetaScores().get(threatSort);
+
+    // add the threat score as a float; type needs to match the threat score field from each of
+    // the sensor indices
+    metaAlert.getDocument()
+        .put(threatTriageField, ConversionUtils.convert(threatScore, Float.class));
+  }
+
+  protected static Double parseThreatField(Object threatRaw) {
+    Double threat = null;
+    if (threatRaw instanceof Number) {
+      threat = ((Number) threatRaw).doubleValue();
+    } else if (threatRaw instanceof String) {
+      threat = Double.parseDouble((String) threatRaw);
+    }
+    return threat;
+  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/lucene/AbstractLuceneMetaAlertUpdateDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/lucene/AbstractLuceneMetaAlertUpdateDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/lucene/AbstractLuceneMetaAlertUpdateDao.java
new file mode 100644
index 0000000..b47d648
--- /dev/null
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/metaalert/lucene/AbstractLuceneMetaAlertUpdateDao.java
@@ -0,0 +1,334 @@
+/*
+ * 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.metaalert.lucene;
+
+import static org.apache.metron.common.Constants.GUID;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import org.apache.metron.common.Constants;
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConfig;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertRetrieveLatestDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertUpdateDao;
+import org.apache.metron.indexing.dao.metaalert.MetaScores;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.update.Document;
+import org.apache.metron.indexing.dao.update.OriginalNotFoundException;
+import org.apache.metron.indexing.dao.update.PatchRequest;
+import org.apache.metron.indexing.dao.update.UpdateDao;
+
+public abstract class AbstractLuceneMetaAlertUpdateDao implements MetaAlertUpdateDao {
+
+  private UpdateDao updateDao;
+  private MetaAlertRetrieveLatestDao retrieveLatestDao;
+  private MetaAlertConfig config;
+
+  protected AbstractLuceneMetaAlertUpdateDao(
+      UpdateDao updateDao,
+      MetaAlertRetrieveLatestDao retrieveLatestDao,
+      MetaAlertConfig config) {
+    this.updateDao = updateDao;
+    this.retrieveLatestDao = retrieveLatestDao;
+    this.config = config;
+  }
+
+  public UpdateDao getUpdateDao() {
+    return updateDao;
+  }
+
+  public MetaAlertRetrieveLatestDao getRetrieveLatestDao() {
+    return retrieveLatestDao;
+  }
+
+  public MetaAlertConfig getConfig() {
+    return config;
+  }
+
+  /**
+   * Performs a patch operation on a document based on the result of @{link #isPatchAllowed(PatchRequest)}
+   *
+   * @param retrieveLatestDao DAO to retrieve the item to be patched
+   * @param request The patch request.
+   * @param timestamp Optionally a timestamp to set. If not specified then current time is used.
+   * @throws OriginalNotFoundException If no original document is found to patch.
+   * @throws IOException If an error occurs performing the patch.
+   */
+  @Override
+  public void patch(RetrieveLatestDao retrieveLatestDao, PatchRequest request,
+      Optional<Long> timestamp)
+      throws OriginalNotFoundException, IOException {
+    if (isPatchAllowed(request)) {
+      updateDao.patch(retrieveLatestDao, request, timestamp);
+    } else {
+      throw new IllegalArgumentException(
+          "Meta alert patches are not allowed for /alert or /status paths.  "
+              + "Please use the add/remove alert or update status functions instead.");
+    }
+  }
+
+  @Override
+  public void batchUpdate(Map<Document, Optional<String>> updates) {
+    throw new UnsupportedOperationException("Meta alerts do not allow for bulk updates");
+  }
+
+  /**
+   * Build the Document representing a meta alert to be created.
+   * @param alerts The Elasticsearch results for the meta alerts child documents
+   * @param groups The groups used to create this meta alert
+   * @return A Document representing the new meta alert
+   */
+  protected Document buildCreateDocument(Iterable<Document> alerts, List<String> groups,
+      String alertField) {
+    // Need to create a Document from the multiget. Scores will be calculated later
+    Map<String, Object> metaSource = new HashMap<>();
+    List<Map<String, Object>> alertList = new ArrayList<>();
+    for (Document alert : alerts) {
+      alertList.add(alert.getDocument());
+    }
+    metaSource.put(alertField, alertList);
+
+    // Add any meta fields
+    String guid = UUID.randomUUID().toString();
+    metaSource.put(GUID, guid);
+    metaSource.put(Constants.Fields.TIMESTAMP.getName(), System.currentTimeMillis());
+    metaSource.put(MetaAlertConstants.GROUPS_FIELD, groups);
+    metaSource.put(MetaAlertConstants.STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString());
+
+    return new Document(metaSource, guid, MetaAlertConstants.METAALERT_TYPE,
+        System.currentTimeMillis());
+  }
+
+  /**
+   * Builds the set of updates when alerts are removed from a meta alert
+   * @param metaAlert The meta alert to remove alerts from
+   * @param alerts The alert Documents to be removed
+   * @return The updates to be run
+   * @throws IOException If an error is thrown.
+   */
+  @SuppressWarnings("unchecked")
+  protected Map<Document, Optional<String>> buildRemoveAlertsFromMetaAlert(Document metaAlert,
+      Iterable<Document> alerts)
+      throws IOException {
+    Map<Document, Optional<String>> updates = new HashMap<>();
+
+    List<String> alertGuids = new ArrayList<>();
+    for (Document alert : alerts) {
+      alertGuids.add(alert.getGuid());
+    }
+    List<Map<String, Object>> alertsBefore = new ArrayList<>();
+    Map<String, Object> documentBefore = metaAlert.getDocument();
+    if (documentBefore.containsKey(MetaAlertConstants.ALERT_FIELD)) {
+      alertsBefore
+          .addAll((List<Map<String, Object>>) documentBefore.get(MetaAlertConstants.ALERT_FIELD));
+    }
+    boolean metaAlertUpdated = removeAlertsFromMetaAlert(metaAlert, alertGuids);
+    if (metaAlertUpdated) {
+      List<Map<String, Object>> alertsAfter = (List<Map<String, Object>>) metaAlert.getDocument()
+          .get(MetaAlertConstants.ALERT_FIELD);
+      if (alertsAfter.size() < alertsBefore.size() && alertsAfter.size() == 0) {
+        throw new IllegalStateException("Removing these alerts will result in an empty meta alert.  Empty meta alerts are not allowed.");
+      }
+      MetaScores
+          .calculateMetaScores(metaAlert, config.getThreatTriageField(), config.getThreatSort());
+      updates.put(metaAlert, Optional.of(config.getMetaAlertIndex()));
+      for (Document alert : alerts) {
+        if (removeMetaAlertFromAlert(metaAlert.getGuid(), alert)) {
+          updates.put(alert, Optional.empty());
+        }
+      }
+    }
+    return updates;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public boolean removeAlertsFromMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests)
+      throws IOException {
+    Document metaAlert = retrieveLatestDao
+        .getLatest(metaAlertGuid, MetaAlertConstants.METAALERT_TYPE);
+    if (metaAlert == null) {
+      return false;
+    }
+    if (MetaAlertStatus.ACTIVE.getStatusString()
+        .equals(metaAlert.getDocument().get(MetaAlertConstants.STATUS_FIELD))) {
+      Iterable<Document> alerts = retrieveLatestDao.getAllLatest(alertRequests);
+      Map<Document, Optional<String>> updates = buildRemoveAlertsFromMetaAlert(metaAlert, alerts);
+      update(updates);
+      return updates.size() != 0;
+    } else {
+      throw new IllegalStateException("Removing alerts from an INACTIVE meta alert is not allowed");
+    }
+  }
+
+  /**
+   * Removes a given set of alerts from a given alert. AlertGuids that are not found are ignored.
+   * @param metaAlert The metaalert to be mutated.
+   * @param alertGuids The alerts to remove from the metaaelrt.
+   * @return True if the metaAlert changed, false otherwise.
+   */
+  protected boolean removeAlertsFromMetaAlert(Document metaAlert, Collection<String> alertGuids) {
+    // If we don't have child alerts or nothing is being removed, immediately return false.
+    if (!metaAlert.getDocument().containsKey(MetaAlertConstants.ALERT_FIELD)
+        || alertGuids.size() == 0) {
+      return false;
+    }
+
+    @SuppressWarnings("unchecked")
+    List<Map<String, Object>> currentAlerts = (List<Map<String, Object>>) metaAlert.getDocument()
+        .get(MetaAlertConstants.ALERT_FIELD);
+    int previousSize = currentAlerts.size();
+    // Only remove an alert if it is in the meta alert
+    currentAlerts.removeIf(currentAlert -> alertGuids.contains(currentAlert.get(GUID)));
+    return currentAlerts.size() != previousSize;
+  }
+
+  @Override
+  public boolean updateMetaAlertStatus(String metaAlertGuid, MetaAlertStatus status)
+      throws IOException {
+    Document metaAlert = retrieveLatestDao
+        .getLatest(metaAlertGuid, MetaAlertConstants.METAALERT_TYPE);
+    String currentStatus = (String) metaAlert.getDocument().get(MetaAlertConstants.STATUS_FIELD);
+    boolean metaAlertUpdated = !status.getStatusString().equals(currentStatus);
+    if (metaAlertUpdated) {
+      List<GetRequest> getRequests = new ArrayList<>();
+      @SuppressWarnings("unchecked")
+      List<Map<String, Object>> currentAlerts = (List<Map<String, Object>>) metaAlert.getDocument()
+          .get(MetaAlertConstants.ALERT_FIELD);
+      currentAlerts.stream()
+          .forEach(currentAlert -> getRequests.add(new GetRequest((String) currentAlert.get(GUID),
+              (String) currentAlert.get(config.getSourceTypeField()))));
+      Iterable<Document> alerts = retrieveLatestDao.getAllLatest(getRequests);
+      Map<Document, Optional<String>> updates = buildStatusChangeUpdates(metaAlert, alerts, status);
+      update(updates);
+    }
+    return metaAlertUpdated;
+  }
+
+  /**
+   * Given a Metaalert and a status change, builds the set of updates to be run.
+   * @param metaAlert The metaalert to have status changed
+   * @param alerts The alerts to change status for
+   * @param status The status to change to
+   * @return The updates to be run
+   */
+  protected Map<Document, Optional<String>> buildStatusChangeUpdates(Document metaAlert,
+      Iterable<Document> alerts,
+      MetaAlertStatus status) {
+    metaAlert.getDocument().put(MetaAlertConstants.STATUS_FIELD, status.getStatusString());
+
+    Map<Document, Optional<String>> updates = new HashMap<>();
+    updates.put(metaAlert, Optional.of(config.getMetaAlertIndex()));
+
+    for (Document alert : alerts) {
+      boolean metaAlertAdded = false;
+      boolean metaAlertRemoved = false;
+      // If we're making it active add add the meta alert guid for every alert.
+      if (MetaAlertStatus.ACTIVE.equals(status)) {
+        metaAlertAdded = addMetaAlertToAlert(metaAlert.getGuid(), alert);
+      }
+      // If we're making it inactive, remove the meta alert guid from every alert.
+      if (MetaAlertStatus.INACTIVE.equals(status)) {
+        metaAlertRemoved = removeMetaAlertFromAlert(metaAlert.getGuid(), alert);
+      }
+      if (metaAlertAdded || metaAlertRemoved) {
+        updates.put(alert, Optional.empty());
+      }
+    }
+    return updates;
+  }
+
+  /**
+   * Builds the updates to be run based on a given metaalert and a set of new alerts for the it.
+   * @param metaAlert The base metaalert we're building updates for
+   * @param alerts The alerts being added
+   * @return The set of resulting updates.
+   */
+  protected Map<Document, Optional<String>> buildAddAlertToMetaAlertUpdates(Document metaAlert,
+      Iterable<Document> alerts) {
+    Map<Document, Optional<String>> updates = new HashMap<>();
+    boolean metaAlertUpdated = addAlertsToMetaAlert(metaAlert, alerts);
+    if (metaAlertUpdated) {
+      MetaScores
+          .calculateMetaScores(metaAlert, config.getThreatTriageField(), config.getThreatSort());
+      updates.put(metaAlert, Optional.of(config.getMetaAlertIndex()));
+      for (Document alert : alerts) {
+        if (addMetaAlertToAlert(metaAlert.getGuid(), alert)) {
+          updates.put(alert, Optional.empty());
+        }
+      }
+    }
+    return updates;
+  }
+
+  /**
+   * Adds the provided alerts to a given metaalert.
+   * @param metaAlert The metaalert to be given new children.
+   * @param alerts The alerts to be added as children
+   * @return True if metaalert is modified, false otherwise.
+   */
+  protected boolean addAlertsToMetaAlert(Document metaAlert, Iterable<Document> alerts) {
+    boolean alertAdded = false;
+    @SuppressWarnings("unchecked")
+    List<Map<String, Object>> currentAlerts = (List<Map<String, Object>>) metaAlert.getDocument()
+        .get(MetaAlertConstants.ALERT_FIELD);
+    if (currentAlerts == null) {
+      currentAlerts = new ArrayList<>();
+      metaAlert.getDocument().put(MetaAlertConstants.ALERT_FIELD, currentAlerts);
+    }
+    Set<String> currentAlertGuids = currentAlerts.stream().map(currentAlert ->
+        (String) currentAlert.get(GUID)).collect(Collectors.toSet());
+    for (Document alert : alerts) {
+      String alertGuid = alert.getGuid();
+      // Only add an alert if it isn't already in the meta alert
+      if (!currentAlertGuids.contains(alertGuid)) {
+        currentAlerts.add(alert.getDocument());
+        alertAdded = true;
+      }
+    }
+    return alertAdded;
+  }
+
+  /**
+   * Calls the single update variant if there's only one update, otherwise calls batch.
+   * MetaAlerts may defer to an implementation specific IndexDao.
+   * @param updates The list of updates to run
+   * @throws IOException If there's an update error
+   */
+  protected void update(Map<Document, Optional<String>> updates)
+      throws IOException {
+    if (updates.size() == 1) {
+      Entry<Document, Optional<String>> singleUpdate = updates.entrySet().iterator().next();
+      updateDao.update(singleUpdate.getKey(), singleUpdate.getValue());
+    } else if (updates.size() > 1) {
+      updateDao.batchUpdate(updates);
+    } // else we have no updates, so don't do anything
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchDao.java
index eee91ae..582f1ef 100644
--- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchDao.java
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchDao.java
@@ -17,18 +17,22 @@
  */
 package org.apache.metron.indexing.dao.search;
 
-import java.io.IOException;
-import java.util.List;
-import org.apache.metron.indexing.dao.update.Document;
-
 public interface SearchDao {
 
+  /**
+   * Return search response based on the search request
+   *
+   * @param searchRequest The request defining the search parameters.
+   * @return A response containing the results of the search.
+   * @throws InvalidSearchException If the search request is malformed.
+   */
   SearchResponse search(SearchRequest searchRequest) throws InvalidSearchException;
 
+  /**
+   * Return group response based on the group request
+   * @param groupRequest The request defining the grouping parameters.
+   * @return A response containing the results of the grouping operation.
+   * @throws InvalidSearchException If the grouping request is malformed.
+   */
   GroupResponse group(GroupRequest groupRequest) throws InvalidSearchException;
-
-  Document getLatest(String guid, String sensorType) throws IOException;
-
-  Iterable<Document> getAllLatest(List<GetRequest> getRequests) throws IOException;
-
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/search/SearchResponse.java
----------------------------------------------------------------------
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 5b0b006..b4dfab7 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
@@ -18,7 +18,6 @@
 package org.apache.metron.indexing.dao.search;
 
 import com.fasterxml.jackson.annotation.JsonInclude;
-
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -84,4 +83,13 @@ public class SearchResponse {
     result = 31 * result + (getFacetCounts() != null ? getFacetCounts().hashCode() : 0);
     return result;
   }
+
+  @Override
+  public String toString() {
+    return "SearchResponse{" +
+        "total=" + total +
+        ", results=" + results +
+        ", facetCounts=" + facetCounts +
+        '}';
+  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchUtil.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchUtil.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchUtil.java
new file mode 100644
index 0000000..5a4ef27
--- /dev/null
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/PatchUtil.java
@@ -0,0 +1,50 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.metron.common.utils.JSONUtils;
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
+
+public class PatchUtil {
+
+  public static Document getPatchedDocument(
+      RetrieveLatestDao retrieveLatestDao,
+      PatchRequest request
+      , Optional<Long> timestamp
+  ) throws OriginalNotFoundException, IOException {
+    Map<String, Object> latest = request.getSource();
+    if (latest == null) {
+      Document latestDoc = retrieveLatestDao.getLatest(request.getGuid(), request.getSensorType());
+      if (latestDoc != null && latestDoc.getDocument() != null) {
+        latest = latestDoc.getDocument();
+      } else {
+        throw new OriginalNotFoundException(
+            "Unable to patch an document that doesn't exist and isn't specified.");
+      }
+    }
+    Map<String, Object> updated = JSONUtils.INSTANCE.applyPatch(request.getPatch(), latest);
+    return new Document(updated
+        , request.getGuid()
+        , request.getSensorType()
+        , timestamp.orElse(System.currentTimeMillis()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/UpdateDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/UpdateDao.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/UpdateDao.java
index ca21b62..6f136ea 100644
--- a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/UpdateDao.java
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/dao/update/UpdateDao.java
@@ -20,11 +20,58 @@ package org.apache.metron.indexing.dao.update;
 import java.io.IOException;
 import java.util.Map;
 import java.util.Optional;
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
 
 public interface UpdateDao {
 
+  /**
+   * Update a given Document and optionally the index where the document exists.  This is a full
+   * update, meaning the current document will be replaced if it exists or a new document will be
+   * created if it does not exist.  Partial updates are not supported in this method.
+   *
+   * @param update The document to replace from the index.
+   * @param index The index where the document lives.
+   * @throws IOException If an error occurs during the update.
+   */
   void update(Document update, Optional<String> index) throws IOException;
 
+  /**
+   * Similar to the update method but accepts multiple documents and performs updates in batch.
+   *
+   * @param updates A map of the documents to update to the index where they live.
+   * @throws IOException If an error occurs during the updates.
+   */
   void batchUpdate(Map<Document, Optional<String>> updates) 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 If an error occurs while patching.
+   */
+  default void patch(RetrieveLatestDao retrieveLatestDao, PatchRequest request
+      , Optional<Long> timestamp
+  ) throws OriginalNotFoundException, IOException {
+    Document d = PatchUtil.getPatchedDocument(retrieveLatestDao, request, timestamp);
+    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 If an error occurs during replacement.
+   */
+  default void replace(ReplaceRequest request, Optional<Long> timestamp)
+      throws IOException {
+    Document d = new Document(request.getReplacement(),
+        request.getGuid(),
+        request.getSensorType(),
+        timestamp.orElse(System.currentTimeMillis())
+    );
+    update(d, Optional.ofNullable(request.getIndex()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/util/IndexingCacheUtil.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/util/IndexingCacheUtil.java b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/util/IndexingCacheUtil.java
new file mode 100644
index 0000000..3ff3a20
--- /dev/null
+++ b/metron-platform/metron-indexing/src/main/java/org/apache/metron/indexing/util/IndexingCacheUtil.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.metron.indexing.util;
+
+import java.util.Map;
+import java.util.function.Function;
+import org.apache.metron.common.configuration.IndexingConfigurations;
+import org.apache.metron.common.zookeeper.ConfigurationsCache;
+
+public class IndexingCacheUtil {
+  public static Function<String, String> getIndexLookupFunction(ConfigurationsCache cache) {
+    return sensorType -> {
+      IndexingConfigurations indexingConfigs = cache.get( IndexingConfigurations.class);
+      Map<String, Object> indexingSensorConfigs = indexingConfigs.getSensorIndexingConfig(sensorType);
+      String indexingTopic = (String) indexingSensorConfigs.get(IndexingConfigurations.INDEX_CONF);
+      return indexingTopic != null ? indexingTopic : sensorType;
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryMetaAlertDao.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryMetaAlertDao.java b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryMetaAlertDao.java
index baa5416..803d320 100644
--- a/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryMetaAlertDao.java
+++ b/metron-platform/metron-indexing/src/test/java/org/apache/metron/indexing/dao/InMemoryMetaAlertDao.java
@@ -32,8 +32,10 @@ import java.util.Optional;
 import java.util.stream.Collectors;
 import org.adrianwalker.multilinestring.Multiline;
 import org.apache.metron.common.utils.JSONUtils;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateResponse;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertDao;
 import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
 import org.apache.metron.indexing.dao.metaalert.MetaScores;
 import org.apache.metron.indexing.dao.search.FieldType;
@@ -57,6 +59,7 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
   public static Map<String, Collection<String>> METAALERT_STORE = new HashMap<>();
 
   private IndexDao indexDao;
+  private int pageSize = 10;
 
   /**
    * {
@@ -96,6 +99,7 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
     // Ignore threatSort for test.
   }
 
+
   @Override
   public Document getLatest(String guid, String sensorType) throws IOException {
     return indexDao.getLatest(guid, sensorType);
@@ -112,7 +116,7 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
   }
 
   @Override
-  public void batchUpdate(Map<Document, Optional<String>> updates) throws IOException {
+  public void batchUpdate(Map<Document, Optional<String>> updates) {
     throw new UnsupportedOperationException("InMemoryMetaAlertDao can't do bulk updates");
   }
 
@@ -128,9 +132,10 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
   }
 
   @Override
-  public void patch(PatchRequest request, Optional<Long> timestamp)
+  public void patch(RetrieveLatestDao retrieveLatestDao, PatchRequest request,
+      Optional<Long> timestamp)
       throws OriginalNotFoundException, IOException {
-    indexDao.patch(request, timestamp);
+    indexDao.patch(retrieveLatestDao, request, timestamp);
   }
 
   @Override
@@ -153,7 +158,7 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
   @SuppressWarnings("unchecked")
   @Override
   public MetaAlertCreateResponse createMetaAlert(MetaAlertCreateRequest request)
-      throws InvalidCreateException, IOException {
+      throws InvalidCreateException {
     List<GetRequest> alertRequests = request.getAlerts();
     if (alertRequests.isEmpty()) {
       MetaAlertCreateResponse response = new MetaAlertCreateResponse();
@@ -162,12 +167,13 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
     }
     // Build meta alert json.  Give it a reasonable GUID
     JSONObject metaAlert = new JSONObject();
-    String metaAlertGuid = "meta_" + (InMemoryDao.BACKING_STORE.get(MetaAlertDao.METAALERTS_INDEX).size() + 1);
+    String metaAlertGuid =
+        "meta_" + (InMemoryDao.BACKING_STORE.get(getMetaAlertIndex()).size() + 1);
     metaAlert.put(GUID, metaAlertGuid);
 
     JSONArray groupsArray = new JSONArray();
     groupsArray.addAll(request.getGroups());
-    metaAlert.put(MetaAlertDao.GROUPS_FIELD, groupsArray);
+    metaAlert.put(MetaAlertConstants.GROUPS_FIELD, groupsArray);
 
     // Retrieve the alert for each guid
     // For the purpose of testing, we're just using guids for the alerts field and grabbing the scores.
@@ -183,7 +189,8 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
         List<SearchResult> searchResults = searchResponse.getResults();
         if (searchResults.size() > 1) {
           throw new InvalidCreateException(
-              "Found more than one result for: " + alertRequest.getGuid() + ". Values: " + searchResults
+              "Found more than one result for: " + alertRequest.getGuid() + ". Values: "
+                  + searchResults
           );
         }
 
@@ -191,7 +198,9 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
           SearchResult result = searchResults.get(0);
           alertArray.add(result.getSource());
           Double threatScore = Double
-              .parseDouble(result.getSource().getOrDefault(THREAT_FIELD_DEFAULT, "0").toString());
+              .parseDouble(
+                  result.getSource().getOrDefault(MetaAlertConstants.THREAT_FIELD_DEFAULT, "0")
+                      .toString());
 
           threatScores.add(threatScore);
         }
@@ -201,12 +210,12 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
       alertGuids.add(alertRequest.getGuid());
     }
 
-    metaAlert.put(MetaAlertDao.ALERT_FIELD, alertArray);
+    metaAlert.put(MetaAlertConstants.ALERT_FIELD, alertArray);
     metaAlert.putAll(new MetaScores(threatScores).getMetaScores());
-    metaAlert.put(STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString());
+    metaAlert.put(MetaAlertConstants.STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString());
 
     // Add the alert to the store, but make sure not to overwrite existing results
-    InMemoryDao.BACKING_STORE.get(MetaAlertDao.METAALERTS_INDEX).add(metaAlert.toJSONString());
+    InMemoryDao.BACKING_STORE.get(getMetaAlertIndex()).add(metaAlert.toJSONString());
 
     METAALERT_STORE.put(metaAlertGuid, new HashSet<>(alertGuids));
 
@@ -217,12 +226,13 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
   }
 
   @Override
-  public boolean addAlertsToMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests) throws IOException {
+  public boolean addAlertsToMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests) {
     Collection<String> currentAlertGuids = METAALERT_STORE.get(metaAlertGuid);
     if (currentAlertGuids == null) {
       return false;
     }
-    Collection<String> alertGuids = alertRequests.stream().map(GetRequest::getGuid).collect(Collectors.toSet());
+    Collection<String> alertGuids = alertRequests.stream().map(GetRequest::getGuid)
+        .collect(Collectors.toSet());
     boolean added = currentAlertGuids.addAll(alertGuids);
     if (added) {
       METAALERT_STORE.put(metaAlertGuid, currentAlertGuids);
@@ -231,12 +241,13 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
   }
 
   @Override
-  public boolean removeAlertsFromMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests) throws IOException {
+  public boolean removeAlertsFromMetaAlert(String metaAlertGuid, List<GetRequest> alertRequests) {
     Collection<String> currentAlertGuids = METAALERT_STORE.get(metaAlertGuid);
     if (currentAlertGuids == null) {
       return false;
     }
-    Collection<String> alertGuids = alertRequests.stream().map(GetRequest::getGuid).collect(Collectors.toSet());
+    Collection<String> alertGuids = alertRequests.stream().map(GetRequest::getGuid)
+        .collect(Collectors.toSet());
     boolean removed = currentAlertGuids.removeAll(alertGuids);
     if (removed) {
       METAALERT_STORE.put(metaAlertGuid, currentAlertGuids);
@@ -249,16 +260,17 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
   public boolean updateMetaAlertStatus(String metaAlertGuid, MetaAlertStatus status)
       throws IOException {
     boolean statusChanged = false;
-    List<String> metaAlerts = InMemoryDao.BACKING_STORE.get(MetaAlertDao.METAALERTS_INDEX);
-    for (String metaAlert: metaAlerts) {
+    List<String> metaAlerts = InMemoryDao.BACKING_STORE.get(getMetaAlertIndex());
+    for (String metaAlert : metaAlerts) {
       JSONObject metaAlertJSON = JSONUtils.INSTANCE.load(metaAlert, JSONObject.class);
       if (metaAlertGuid.equals(metaAlertJSON.get(GUID))) {
-        statusChanged = !status.getStatusString().equals(metaAlertJSON.get(STATUS_FIELD));
+        statusChanged = !status.getStatusString()
+            .equals(metaAlertJSON.get(MetaAlertConstants.STATUS_FIELD));
         if (statusChanged) {
-          metaAlertJSON.put(STATUS_FIELD, status.getStatusString());
+          metaAlertJSON.put(MetaAlertConstants.STATUS_FIELD, status.getStatusString());
           metaAlerts.remove(metaAlert);
           metaAlerts.add(metaAlertJSON.toJSONString());
-          InMemoryDao.BACKING_STORE.put(MetaAlertDao.METAALERTS_INDEX, metaAlerts);
+          InMemoryDao.BACKING_STORE.put(getMetaAlertIndex(), metaAlerts);
         }
         break;
       }
@@ -266,9 +278,24 @@ public class InMemoryMetaAlertDao implements MetaAlertDao {
     return statusChanged;
   }
 
+  public int getPageSize() {
+    return pageSize;
+  }
+
+  public void setPageSize(int pageSize) {
+    this.pageSize = pageSize;
+  }
+
+  public String getMetAlertSensorName() {
+    return MetaAlertConstants.METAALERT_TYPE;
+  }
+
+  public String getMetaAlertIndex() {
+    return "metaalert_index";
+  }
+
   public static void clear() {
     InMemoryDao.clear();
     METAALERT_STORE.clear();
   }
-
 }