You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by rm...@apache.org on 2017/11/16 01:35:47 UTC

[2/3] metron git commit: METRON-1289 Alert fields are lost when a MetaAlert is created (merrimanr) closes apache/metron#824

http://git-wip-us.apache.org/repos/asf/metron/blob/fd896fbe/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 9a02854..a1027f7 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
@@ -20,17 +20,12 @@ package org.apache.metron.elasticsearch.dao;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -40,10 +35,10 @@ import org.apache.metron.common.Constants.Fields;
 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.metaalert.MetaAlertCreateRequest;
-import org.apache.metron.indexing.dao.metaalert.MetaScores;
+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;
@@ -51,147 +46,10 @@ 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.elasticsearch.action.get.GetResponse;
-import org.elasticsearch.action.get.MultiGetItemResponse;
-import org.elasticsearch.action.get.MultiGetResponse;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.search.SearchHit;
-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;
 
 public class ElasticsearchMetaAlertDaoTest {
 
-  @Test
-  @SuppressWarnings("unchecked")
-  public void testBuildUpdatedMetaAlertSingleAlert() throws IOException, ParseException {
-    // Construct the expected result
-    JSONObject expected = new JSONObject();
-    expected.put("average", 5.0);
-    expected.put("min", 5.0);
-    expected.put("median", 5.0);
-    expected.put("max", 5.0);
-    expected.put("count", 1L);
-    expected.put(Constants.GUID, "m1");
-    expected.put("sum", 5.0);
-    expected.put(MetaAlertDao.STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString());
-    JSONArray expectedAlerts = new JSONArray();
-    JSONObject expectedAlert = new JSONObject();
-    expectedAlert.put(MetaAlertDao.THREAT_FIELD_DEFAULT, 5L);
-    expectedAlert.put("fakekey", "fakevalue");
-    expectedAlerts.add(expectedAlert);
-    expected.put(MetaAlertDao.ALERT_FIELD, expectedAlerts);
-
-    // Construct the meta alert object
-    Map<String, Object> metaSource = new HashMap<>();
-    metaSource.put(Constants.GUID, "m1");
-    metaSource.put(MetaAlertDao.STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString());
-    List<Double> alertScores = new ArrayList<>();
-    alertScores.add(10d);
-    metaSource.putAll(new MetaScores(alertScores).getMetaScores());
-    SearchHit metaHit = mock(SearchHit.class);
-    when(metaHit.getSource()).thenReturn(metaSource);
-
-    // Construct the inner alert
-    HashMap<String, Object> innerAlertSource = new HashMap<>();
-    innerAlertSource.put(Constants.GUID, "a1");
-    innerAlertSource.put(MetaAlertDao.THREAT_FIELD_DEFAULT, 10d);
-
-    Map<String, Object> innerHits = new HashMap<>();
-    innerHits.put(MetaAlertDao.ALERT_FIELD, Collections.singletonList(innerAlertSource));
-    when(metaHit.sourceAsMap()).thenReturn(innerHits);
-
-    // Construct  the updated Document
-    Map<String, Object> updateMap = new HashMap<>();
-    updateMap.put(MetaAlertDao.THREAT_FIELD_DEFAULT, 5);
-    updateMap.put("fakekey", "fakevalue");
-    Document update = new Document(updateMap, "a1", "bro_doc", 0L);
-
-    ElasticsearchDao esDao = new ElasticsearchDao();
-    ElasticsearchMetaAlertDao emaDao = new ElasticsearchMetaAlertDao();
-    emaDao.init(esDao);
-    XContentBuilder builder = emaDao.buildUpdatedMetaAlert(update, metaHit);
-    JSONParser parser = new JSONParser();
-    Object obj = parser.parse(builder.string());
-    JSONObject actual = (JSONObject) obj;
-
-    assertEquals(expected, actual);
-  }
-
-  @Test
-  @SuppressWarnings("unchecked")
-  public void testBuildUpdatedMetaAlertMultipleAlerts() throws IOException, ParseException {
-    // Construct the expected result
-    JSONObject expected = new JSONObject();
-    expected.put("average", 7.5);
-    expected.put("min", 5.0);
-    expected.put("median", 7.5);
-    expected.put("max", 10.0);
-    expected.put("count", 2L);
-    expected.put(Constants.GUID, "m1");
-    expected.put("sum", 15.0);
-    expected.put(MetaAlertDao.STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString());
-    JSONArray expectedAlerts = new JSONArray();
-    JSONObject expectedAlertOne = new JSONObject();
-    expectedAlertOne.put(MetaAlertDao.THREAT_FIELD_DEFAULT, 5d);
-    expectedAlertOne.put("fakekey", "fakevalue");
-    expectedAlerts.add(expectedAlertOne);
-    JSONObject expectedAlertTwo = new JSONObject();
-    expectedAlertTwo.put(MetaAlertDao.THREAT_FIELD_DEFAULT, 10d);
-    String guidTwo = "a2";
-    expectedAlertTwo.put(Constants.GUID, guidTwo);
-    expectedAlerts.add(expectedAlertTwo);
-    expected.put(MetaAlertDao.ALERT_FIELD, expectedAlerts);
-
-    // Construct the meta alert object
-    Map<String, Object> metaSource = new HashMap<>();
-    metaSource.put(Constants.GUID, "m1");
-    metaSource.put(MetaAlertDao.STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString());
-    double threatValueOne = 5d;
-    double threatValueTwo = 10d;
-    List<Double> alertScores = new ArrayList<>();
-    alertScores.add(threatValueOne);
-    alertScores.add(threatValueTwo);
-    metaSource.putAll(new MetaScores(alertScores).getMetaScores());
-    SearchHit metaHit = mock(SearchHit.class);
-    when(metaHit.getSource()).thenReturn(metaSource);
-
-    // Construct the inner alerts
-    HashMap<String, Object> innerAlertSourceOne = new HashMap<>();
-    String guidOne = "a1";
-    innerAlertSourceOne.put(Constants.GUID, guidOne);
-    innerAlertSourceOne.put(MetaAlertDao.THREAT_FIELD_DEFAULT, threatValueTwo);
-
-    HashMap<String, Object> innerAlertSourceTwo = new HashMap<>();
-    innerAlertSourceTwo.put(Constants.GUID, guidTwo);
-    innerAlertSourceTwo.put(MetaAlertDao.THREAT_FIELD_DEFAULT, threatValueTwo);
-
-    Map<String, Object> innerHits = new HashMap<>();
-    innerHits
-        .put(MetaAlertDao.ALERT_FIELD, Arrays.asList(innerAlertSourceOne, innerAlertSourceTwo));
-    when(metaHit.sourceAsMap()).thenReturn(innerHits);
-
-    // Construct  the updated Document
-    Map<String, Object> updateMap = new HashMap<>();
-    updateMap.put(MetaAlertDao.THREAT_FIELD_DEFAULT, threatValueOne);
-    updateMap.put("fakekey", "fakevalue");
-    Document update = new Document(updateMap, guidOne, "bro_doc", 0L);
-
-    ElasticsearchDao esDao = new ElasticsearchDao();
-    ElasticsearchMetaAlertDao emaDao = new ElasticsearchMetaAlertDao();
-    MultiIndexDao multiIndexDao = new MultiIndexDao(esDao);
-    emaDao.init(multiIndexDao);
-    XContentBuilder builder = emaDao.buildUpdatedMetaAlert(update, metaHit);
-
-    JSONParser parser = new JSONParser();
-    Object obj = parser.parse(builder.string());
-    JSONObject actual = (JSONObject) obj;
-
-    assertEquals(expected, actual);
-  }
-
   @Test(expected = IllegalArgumentException.class)
   public void testInvalidInit() {
     IndexDao dao = new IndexDao() {
@@ -215,6 +73,12 @@ public class ElasticsearchMetaAlertDaoTest {
       }
 
       @Override
+      public Iterable<Document> getAllLatest(
+          List<GetRequest> getRequests) throws IOException {
+        return null;
+      }
+
+      @Override
       public void update(Document update, Optional<String> index) throws IOException {
       }
 
@@ -252,24 +116,12 @@ public class ElasticsearchMetaAlertDaoTest {
     Map<String, Object> alertOne = new HashMap<>();
     alertOne.put(Constants.GUID, "alert_one");
     alertOne.put(MetaAlertDao.THREAT_FIELD_DEFAULT, 10.0d);
-    GetResponse getResponseOne = mock(GetResponse.class);
-    when(getResponseOne.isExists()).thenReturn(true);
-    when(getResponseOne.getSource()).thenReturn(alertOne);
-    MultiGetItemResponse multiGetItemResponseOne = mock(MultiGetItemResponse.class);
-    when(multiGetItemResponseOne.getResponse()).thenReturn(getResponseOne);
-
-    // Add it to the iterator
-    @SuppressWarnings("unchecked")
-    Iterator<MultiGetItemResponse> mockIterator = mock(Iterator.class);
-    when(mockIterator.hasNext()).thenReturn(true, false);
-    when(mockIterator.next()).thenReturn(multiGetItemResponseOne);
-
-    // Add it to the response
-    MultiGetResponse mockResponse = mock(MultiGetResponse.class);
-    when(mockResponse.iterator()).thenReturn(mockIterator);
+    List<Document> alerts = new ArrayList<Document>() {{
+      add(new Document(alertOne, "", "", 0L));
+    }};
 
     // Actually build the doc
-    Document actual = emaDao.buildCreateDocument(mockResponse, groups);
+    Document actual = emaDao.buildCreateDocument(alerts, groups);
 
     ArrayList<Map<String, Object>> alertList = new ArrayList<>();
     alertList.add(alertOne);
@@ -306,34 +158,18 @@ public class ElasticsearchMetaAlertDaoTest {
     Map<String, Object> alertOne = new HashMap<>();
     alertOne.put(Constants.GUID, "alert_one");
     alertOne.put(MetaAlertDao.THREAT_FIELD_DEFAULT, 10.0d);
-    GetResponse getResponseOne = mock(GetResponse.class);
-    when(getResponseOne.isExists()).thenReturn(true);
-    when(getResponseOne.getSource()).thenReturn(alertOne);
-    MultiGetItemResponse multiGetItemResponseOne = mock(MultiGetItemResponse.class);
-    when(multiGetItemResponseOne.getResponse()).thenReturn(getResponseOne);
 
     // 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);
-    GetResponse getResponseTwo = mock(GetResponse.class);
-    when(getResponseTwo.isExists()).thenReturn(true);
-    when(getResponseTwo.getSource()).thenReturn(alertTwo);
-    MultiGetItemResponse multiGetItemResponseTwo = mock(MultiGetItemResponse.class);
-    when(multiGetItemResponseTwo.getResponse()).thenReturn(getResponseTwo);
-
-    // Add it to the iterator
-    @SuppressWarnings("unchecked")
-    Iterator<MultiGetItemResponse> mockIterator = mock(Iterator.class);
-    when(mockIterator.hasNext()).thenReturn(true, true, false);
-    when(mockIterator.next()).thenReturn(multiGetItemResponseOne, multiGetItemResponseTwo);
-
-    // Add them to the response
-    MultiGetResponse mockResponse = mock(MultiGetResponse.class);
-    when(mockResponse.iterator()).thenReturn(mockIterator);
+    List<Document> alerts = new ArrayList<Document>() {{
+      add(new Document(alertOne, "", "", 0L));
+      add(new Document(alertTwo, "", "", 0L));
+    }};
 
     // Actually build the doc
-    Document actual = emaDao.buildCreateDocument(mockResponse, groups);
+    Document actual = emaDao.buildCreateDocument(alerts, groups);
 
     ArrayList<Map<String, Object>> alertList = new ArrayList<>();
     alertList.add(alertOne);
@@ -371,9 +207,7 @@ public class ElasticsearchMetaAlertDaoTest {
     emaDao.init(esDao);
 
     MetaAlertCreateRequest createRequest = new MetaAlertCreateRequest();
-    HashMap<String, String> guidsToGroups = new HashMap<>();
-    guidsToGroups.put("don't", "care");
-    createRequest.setGuidToIndices(guidsToGroups);
+    createRequest.setAlerts(Collections.singletonList(new GetRequest("don't", "care")));
     emaDao.createMetaAlert(createRequest);
   }
 
@@ -388,105 +222,11 @@ public class ElasticsearchMetaAlertDaoTest {
 
     Document doc = new Document(docMap, "guid", MetaAlertDao.METAALERT_TYPE, 0L);
 
-    List<Double> scores = new ArrayList<>();
-    scores.add(10.0d);
-    MetaScores expected = new MetaScores(scores);
-
     ElasticsearchMetaAlertDao metaAlertDao = new ElasticsearchMetaAlertDao();
-    MetaScores actual = metaAlertDao.calculateMetaScores(doc);
-    assertEquals(expected.getMetaScores(), actual.getMetaScores());
-  }
-
-  @Test
-  public void testHandleMetaUpdateNonAlertNonStatus() throws IOException {
-    ElasticsearchDao mockEsDao = mock(ElasticsearchDao.class);
-
-    Map<String, Object> docMap = new HashMap<>();
-    docMap.put("test", "value");
-    Document update = new Document(docMap, "guid", MetaAlertDao.METAALERT_TYPE, 0L);
-
-    ElasticsearchMetaAlertDao metaAlertDao = new ElasticsearchMetaAlertDao(mockEsDao);
-    metaAlertDao.handleMetaUpdate(update);
-    verify(mockEsDao, times(1))
-        .update(update, Optional.of(MetaAlertDao.METAALERTS_INDEX));
-  }
-
-  @Test
-  public void testHandleMetaUpdateAlert() throws IOException {
-    // The child alert of the meta alert
-    Map<String, Object> alertMapBefore = new HashMap<>();
-    alertMapBefore.put(MetaAlertDao.THREAT_FIELD_DEFAULT, 10.0d);
-    String guidAlert = "guid_alert";
-    alertMapBefore.put(Constants.GUID, guidAlert);
-    List<Map<String, Object>> alertList = new ArrayList<>();
-    alertList.add(alertMapBefore);
-    String alertSensorType = "alert_sensor";
-    Document alertBefore = new Document(
-        alertMapBefore,
-        guidAlert,
-        alertSensorType,
-        0L
-    );
-
-    // The original meta alert. It contains the alert we previously constructed.
-    Map<String, Object> metaMapBefore = new HashMap<>();
-    String metaGuid = "guid_meta";
-    metaMapBefore.putAll(alertBefore.getDocument());
-    metaMapBefore.put(MetaAlertDao.ALERT_FIELD, alertList);
-    metaMapBefore.put(Constants.GUID, metaGuid);
-    Document metaBefore = new Document(
-        metaMapBefore,
-        metaGuid,
-        MetaAlertDao.METAALERT_TYPE,
-        0L
-    );
-
-    // Build the Documents we expect to see from updates
-    // Build the after alert.  Don't add the original fields: This is only an update.
-    // The new field is the link to the meta alert.
-    Map<String, Object> alertMapAfter = new HashMap<>();
-    List<String> metaAlertField = new ArrayList<>();
-    metaAlertField.add(metaGuid);
-    alertMapAfter.put(MetaAlertDao.METAALERT_FIELD, metaAlertField);
-    Document alertAfter = new Document(
-        alertMapAfter,
-        guidAlert,
-        alertSensorType,
-        0L
+    metaAlertDao.calculateMetaScores(doc);
+    assertEquals(1L, doc.getDocument().get("count"));
+    assertEquals(10.0d,
+        doc.getDocument().get(ElasticsearchMetaAlertDao.THREAT_FIELD_DEFAULT)
     );
-
-    // Build the meta alert after. This'll be a replace, so add the original fields plus the
-    // threat fields
-    Map<String, Object> metaMapAfter = new HashMap<>();
-    metaMapAfter.putAll(metaMapBefore);
-    metaMapAfter.put("average", 10.0d);
-    metaMapAfter.put("min", 10.0d);
-    metaMapAfter.put("median", 10.0d);
-    metaMapAfter.put("max", 10.0d);
-    metaMapAfter.put("count", 1L);
-    metaMapAfter.put("sum", 10.0d);
-    metaMapAfter.put(MetaAlertDao.THREAT_FIELD_DEFAULT, 10.0d);
-
-    Document metaAfter = new Document(
-        metaMapAfter,
-        metaGuid,
-        MetaAlertDao.METAALERT_TYPE,
-        0L
-    );
-
-    // Build the method calls we'd expect to see.
-    Map<Document, Optional<String>> updates = new HashMap<>();
-    updates.put(metaAfter, Optional.of(MetaAlertDao.METAALERTS_INDEX));
-    updates.put(alertAfter, Optional.empty());
-
-    // Build a mock ElasticsearchDao to track interactions.  Actual runs are in integration tests
-    ElasticsearchDao mockEsDao = mock(ElasticsearchDao.class);
-    ElasticsearchMetaAlertDao metaAlertDao = new ElasticsearchMetaAlertDao(mockEsDao);
-    when(mockEsDao.getLatest(guidAlert, null)).thenReturn(alertBefore);
-    metaAlertDao.handleMetaUpdate(metaBefore);
-
-    // Validate we're calling what we need to with what we expect.
-    verify(mockEsDao, times(1)).getLatest(guidAlert, null);
-    verify(mockEsDao, times(1)).batchUpdate(updates);
   }
 }

http://git-wip-us.apache.org/repos/asf/metron/blob/fd896fbe/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 27e5566..c28094b 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,19 +18,23 @@
 
 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.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.core.type.TypeReference;
 import java.io.File;
 import java.io.IOException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -40,24 +44,26 @@ import org.apache.metron.common.Constants;
 import org.apache.metron.common.utils.JSONUtils;
 import org.apache.metron.elasticsearch.dao.ElasticsearchDao;
 import org.apache.metron.elasticsearch.dao.ElasticsearchMetaAlertDao;
-import org.apache.metron.elasticsearch.dao.MetaAlertStatus;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertStatus;
 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.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.apache.metron.indexing.dao.update.ReplaceRequest;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -75,6 +81,7 @@ public class ElasticsearchMetaAlertIntegrationTest {
   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";
 
   private static IndexDao esDao;
   private static MetaAlertDao metaDao;
@@ -82,161 +89,76 @@ public class ElasticsearchMetaAlertIntegrationTest {
 
   /**
    {
-   "guid": "update_metaalert_alert_0",
-   "source:type": "test",
-   "field": "value 0"
-   }
-   */
-  @Multiline
-  public static String updateMetaAlertAlert0;
-
-  /**
-   {
-   "guid": "update_metaalert_alert_1",
-   "source:type": "test",
-   "field":"value 1"
-   }
-   */
-  @Multiline
-  public static String updateMetaAlertAlert1;
-
-  /**
-   {
-   "guid": "update_metaalert_alert_0",
-   "patch": [
-   {
-   "op": "add",
-   "path": "/field",
-   "value": "patched value 0"
-   }
-   ],
-   "sensorType": "test"
-   }
-   */
-  @Multiline
-  public static String updateMetaAlertPatchRequest;
-
-  /**
-   {
-   "guid": "update_metaalert_alert_0",
-   "replacement": {
-   "guid": "update_metaalert_alert_0",
-   "source:type": "test",
-   "field": "replaced value 0"
-   },
-   "sensorType": "test"
-   }
-   */
-  @Multiline
-  public static String updateMetaAlertReplaceRequest;
-
-  /**
-   {
-   "guid": "active_metaalert",
-   "source:type": "metaalert",
-   "alert": [],
-   "status": "active",
-   "timestamp": 0
-   }
-   */
-  @Multiline
-  public static String activeMetaAlert;
-
-  /**
-   {
-   "guid": "inactive_metaalert",
-   "source:type": "metaalert",
-   "alert": [],
-   "status": "inactive"
-   }
-   */
-  @Multiline
-  public static String inactiveMetaAlert;
-
-  /**
-   {
-   "guid": "search_by_nested_alert_active_0",
-   "source:type": "test",
-   "ip_src_addr": "192.168.1.1",
-   "ip_src_port": 8010,
-   "metaalerts": ["active_metaalert"],
-   "timestamp": 0
-   }
-   */
-  @Multiline
-  public static String searchByNestedAlertActive0;
-
-  /**
-   {
-   "guid": "search_by_nested_alert_active_1",
-   "source:type": "test",
-   "ip_src_addr": "192.168.1.2",
-   "ip_src_port": 8009,
-   "metaalerts": ["active_metaalert"],
-   "timestamp": 0
-   }
-   */
-  @Multiline
-  public static String searchByNestedAlertActive1;
-
-  /**
-   {
-   "guid": "search_by_nested_alert_inactive_0",
-   "source:type": "test",
-   "ip_src_addr": "192.168.1.3",
-   "ip_src_port": 8008
-   }
-   */
-  @Multiline
-  public static String searchByNestedAlertInactive0;
-
-  /**
-   {
-   "guid": "search_by_nested_alert_inactive_1",
-   "source:type": "test",
-   "ip_src_addr": "192.168.1.4",
-   "ip_src_port": 8007
+     "properties": {
+       "alert": {
+         "type": "nested"
+       }
+     }
    }
    */
   @Multiline
-  public static String searchByNestedAlertInactive1;
+  public static String nestedAlertMapping;
 
   /**
    {
-     "properties": {
-       "alert": {
-         "type": "nested"
+     "guid": "meta_alert",
+     "index": "metaalert_index",
+     "patch": [
+       {
+         "op": "add",
+         "path": "/name",
+         "value": "New Meta Alert"
        }
-     }
+     ],
+     "sensorType": "metaalert"
    }
    */
   @Multiline
-  public static String nestedAlertMapping;
+  public static String namePatchRequest;
 
   /**
    {
-   "guid": "group_by_child_alert",
-   "source:type": "test",
-   "ip_src_addr": "192.168.1.1",
-   "ip_src_port": 8010,
-   "score_field": 1,
-   "metaalerts": ["active_metaalert"]
+     "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 groupByChildAlert;
+  public static String alertPatchRequest;
 
   /**
    {
-   "guid": "group_by_standalone_alert",
-   "source:type": "test",
-   "ip_src_addr": "192.168.1.1",
-   "ip_src_port": 8010,
-   "score_field": 10
+     "guid": "meta_alert",
+     "index": "metaalert_index",
+     "patch": [
+       {
+         "op": "add",
+         "path": "/status",
+         "value": "inactive"
+       },
+       {
+         "op": "add",
+         "path": "/name",
+         "value": "New Meta Alert"
+       }
+     ],
+     "sensorType": "metaalert"
    }
    */
   @Multiline
-  public static String groupByStandaloneAlert;
+  public static String statusPatchRequest;
 
   @BeforeClass
   public static void setupBefore() throws Exception {
@@ -267,7 +189,7 @@ public class ElasticsearchMetaAlertIntegrationTest {
 
   @Before
   public void setup() throws IOException {
-    es.createIndexWithMapping(MetaAlertDao.METAALERTS_INDEX, MetaAlertDao.METAALERT_DOC,
+    es.createIndexWithMapping(METAALERTS_INDEX, MetaAlertDao.METAALERT_DOC,
         buildMetaMappingSource());
   }
 
@@ -305,206 +227,414 @@ public class ElasticsearchMetaAlertIntegrationTest {
         .string();
   }
 
-
-  @SuppressWarnings("unchecked")
   @Test
-  public void test() throws Exception {
-    List<Map<String, Object>> inputData = new ArrayList<>();
-    for (int i = 0; i < 2; ++i) {
-      final String name = "message" + i;
-      int finalI = i;
-      inputData.add(
-          new HashMap<String, Object>() {
-            {
-              put("source:type", SENSOR_NAME);
-              put("name", name);
-              put(MetaAlertDao.THREAT_FIELD_DEFAULT, finalI);
-              put("timestamp", System.currentTimeMillis());
-              put(Constants.GUID, name);
-            }
-          }
-      );
+  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());
+      Assert.assertEquals(metaAlerts.get(0), searchResults0.get(0).getSource());
+
+      // 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);
+  }
 
-    elasticsearchAdd(inputData, INDEX, SENSOR_NAME);
+  @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());
+    }
+  }
 
-    List<Map<String, Object>> metaInputData = new ArrayList<>();
-    final String name = "meta_message";
-    Map<String, Object>[] alertArray = new Map[1];
-    alertArray[0] = inputData.get(0);
-    metaInputData.add(
-        new HashMap<String, Object>() {
-          {
-            put("source:type", SENSOR_NAME);
-            put("alert", alertArray);
-            put(Constants.GUID, name + "_active");
-            put(MetaAlertDao.STATUS_FIELD, MetaAlertStatus.ACTIVE.getStatusString());
-          }
-        }
-    );
-    // Add an inactive message
-    metaInputData.add(
-        new HashMap<String, Object>() {
-          {
-            put("source:type", SENSOR_NAME);
-            put("alert", alertArray);
-            put(Constants.GUID, name + "_inactive");
-            put(MetaAlertDao.STATUS_FIELD, MetaAlertStatus.INACTIVE.getStatusString());
-          }
-        }
-    );
+  @Test
+  public void shouldCreateMetaAlert() throws Exception {
+    // Load alerts
+    List<Map<String, Object>> alerts = buildAlerts(3);
+    elasticsearchAdd(alerts, INDEX, SENSOR_NAME);
 
-    // We pass MetaAlertDao.METAALERT_TYPE, because the "_doc" gets appended automatically.
-    elasticsearchAdd(metaInputData, MetaAlertDao.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("message_2", SENSOR_NAME)));
 
-    List<Map<String, Object>> docs = null;
-    for (int t = 0; t < MAX_RETRIES; ++t, Thread.sleep(SLEEP_MS)) {
-      docs = es.getAllIndexedDocs(INDEX, SENSOR_NAME + "_doc");
-      if (docs.size() >= 10) {
-        break;
+    {
+      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));
       }
     }
-    Assert.assertEquals(2, docs.size());
+  }
+
+  @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);
+
     {
-      //modify the first message and add a new field
-      Map<String, Object> message0 = new HashMap<String, Object>(inputData.get(0)) {
-        {
-          put(NEW_FIELD, "metron");
-          put(MetaAlertDao.THREAT_FIELD_DEFAULT, "10");
-        }
-      };
-      String guid = "" + message0.get(Constants.GUID);
-      metaDao.replace(new ReplaceRequest() {
-        {
-          setReplacement(message0);
-          setGuid(guid);
-          setSensorType(SENSOR_NAME);
-        }
-      }, Optional.empty());
+      // 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);
+    }
 
-      {
-        //ensure alerts in ES are up-to-date
-        boolean found = findUpdatedDoc(message0, guid, SENSOR_NAME);
-        Assert.assertTrue("Unable to find updated document", found);
-        long cnt = 0;
-        for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t, Thread.sleep(SLEEP_MS)) {
-          docs = es.getAllIndexedDocs(INDEX, SENSOR_NAME + "_doc");
-          cnt = docs
-              .stream()
-              .filter(d -> {
-                Object newfield = d.get(NEW_FIELD);
-                return newfield != null && newfield.equals(message0.get(NEW_FIELD));
-              }).count();
-        }
-        if (cnt == 0) {
-          Assert.fail("Elasticsearch is not updated!");
-        }
+    {
+      // 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());
       }
+    }
 
-      {
-        //ensure meta alerts in ES are up-to-date
-        long cnt = 0;
-        for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t, Thread.sleep(SLEEP_MS)) {
-          docs = es.getAllIndexedDocs(MetaAlertDao.METAALERTS_INDEX, MetaAlertDao.METAALERT_DOC);
-          cnt = docs
-              .stream()
-              .filter(d -> {
-                List<Map<String, Object>> alerts = (List<Map<String, Object>>) d
-                    .get(MetaAlertDao.ALERT_FIELD);
-
-                for (Map<String, Object> alert : alerts) {
-                  Object newField = alert.get(NEW_FIELD);
-                  if (newField != null && newField.equals(message0.get(NEW_FIELD))) {
-                    return true;
-                  }
-                }
-
-                return false;
-              }).count();
-        }
-        if (cnt == 0) {
-          Assert.fail("Elasticsearch metaalerts not updated!");
-        }
+    {
+      // 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());
       }
     }
-    //modify the same message and modify the new field
+  }
+
+  @Test
+  public void shouldUpdateMetaAlertStatus() throws Exception {
+    // Load alerts
+    List<Map<String, Object>> alerts = buildAlerts(3);
+    alerts.get(0).put(METAALERT_FIELD, Collections.singletonList("meta_alert"));
+    alerts.get(1).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(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("message_2", SENSOR_NAME),
+        new GetRequest("meta_alert", METAALERT_TYPE)));
+
     {
-      Map<String, Object> message0 = new HashMap<String, Object>(inputData.get(0)) {
-        {
-          put(NEW_FIELD, "metron2");
-        }
-      };
-      String guid = "" + message0.get(Constants.GUID);
-      metaDao.replace(new ReplaceRequest() {
-        {
-          setReplacement(message0);
-          setGuid(guid);
-          setSensorType(SENSOR_NAME);
-        }
-      }, Optional.empty());
+      // 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);
+
+      Map<String, Object> expectedAlert0 = new HashMap<>(alerts.get(0));
+      expectedAlert0.put("metaalerts", new ArrayList());
+      findUpdatedDoc(expectedAlert0, "message_0", SENSOR_NAME);
+
+      Map<String, Object> expectedAlert1 = new HashMap<>(alerts.get(1));
+      expectedAlert1.put("metaalerts", new ArrayList());
+      findUpdatedDoc(expectedAlert1, "message_1", SENSOR_NAME);
+
+      Map<String, Object> expectedAlert2 = new HashMap<>(alerts.get(2));
+      findUpdatedDoc(expectedAlert2, "message_2", 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);
+
+      Map<String, Object> expectedAlert0 = new HashMap<>(alerts.get(0));
+      expectedAlert0.put("metaalerts", Collections.singletonList("meta_alert"));
+      findUpdatedDoc(expectedAlert0, "message_0", SENSOR_NAME);
+
+      Map<String, Object> expectedAlert1 = new HashMap<>(alerts.get(1));
+      expectedAlert1.put("metaalerts", Collections.singletonList("meta_alert"));
+      findUpdatedDoc(expectedAlert1, "message_1", SENSOR_NAME);
+
+      Map<String, Object> expectedAlert2 = new HashMap<>(alerts.get(2));
+      findUpdatedDoc(expectedAlert2, "message_2", SENSOR_NAME);
 
-      boolean found = findUpdatedDoc(message0, guid, SENSOR_NAME);
-      Assert.assertTrue("Unable to find updated document", found);
-      {
-        //ensure ES is up-to-date
-        long cnt = 0;
-        for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t, Thread.sleep(SLEEP_MS)) {
-          docs = es.getAllIndexedDocs(INDEX, SENSOR_NAME + "_doc");
-          cnt = docs
-              .stream()
-              .filter(d -> message0.get(NEW_FIELD).equals(d.get(NEW_FIELD)))
-              .count();
-        }
-        Assert.assertNotEquals("Elasticsearch is not updated!", cnt, 0);
-        if (cnt == 0) {
-          Assert.fail("Elasticsearch is not updated!");
-        }
-      }
       {
-        //ensure meta alerts in ES are up-to-date
-        long cnt = 0;
-        for (int t = 0; t < MAX_RETRIES && cnt == 0; ++t, Thread.sleep(SLEEP_MS)) {
-          docs = es.getAllIndexedDocs(MetaAlertDao.METAALERTS_INDEX, MetaAlertDao.METAALERT_DOC);
-          cnt = docs
-              .stream()
-              .filter(d -> {
-                List<Map<String, Object>> alerts = (List<Map<String, Object>>) d
-                    .get(MetaAlertDao.ALERT_FIELD);
-
-                for (Map<String, Object> alert : alerts) {
-                  Object newField = alert.get(NEW_FIELD);
-                  if (newField != null && newField.equals(message0.get(NEW_FIELD))) {
-                    return true;
-                  }
-                }
-
-                return false;
-              }).count();
-        }
-        if (cnt == 0) {
-          Assert.fail("Elasticsearch metaalerts not updated!");
-        }
+        // Verify status changed to current status has no effect
+        Assert.assertFalse(metaDao.updateMetaAlertStatus("meta_alert", MetaAlertStatus.ACTIVE));
+
+        findUpdatedDoc(expectedMetaAlert, "meta_alert", METAALERT_TYPE);
+        findUpdatedDoc(expectedAlert0, "message_0", SENSOR_NAME);
+        findUpdatedDoc(expectedAlert1, "message_1", SENSOR_NAME);
+        findUpdatedDoc(expectedAlert2, "message_2", SENSOR_NAME);
       }
     }
   }
 
-
   @Test
   public void shouldSearchByStatus() throws Exception {
-    List<Map<String, Object>> metaInputData = new ArrayList<>();
-    Map<String, Object> activeMetaAlertJSON = JSONUtils.INSTANCE
-        .load(activeMetaAlert, new TypeReference<Map<String, Object>>() {
-        });
-    metaInputData.add(activeMetaAlertJSON);
-    Map<String, Object> inactiveMetaAlertJSON = JSONUtils.INSTANCE
-        .load(inactiveMetaAlert, new TypeReference<Map<String, Object>>() {
-        });
-    metaInputData.add(inactiveMetaAlertJSON);
+    // 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(metaInputData, MetaAlertDao.METAALERTS_INDEX, MetaAlertDao.METAALERT_TYPE);
-    // Wait for updates to persist
-    findUpdatedDoc(inactiveMetaAlertJSON, "inactive_metaalert", MetaAlertDao.METAALERT_TYPE);
+    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() {
       {
@@ -517,6 +647,8 @@ public class ElasticsearchMetaAlertIntegrationTest {
         }}));
       }
     });
+
+    // 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));
@@ -525,34 +657,19 @@ public class ElasticsearchMetaAlertIntegrationTest {
 
   @Test
   public void shouldSearchByNestedAlert() throws Exception {
-    // Create alerts
-    List<Map<String, Object>> alerts = new ArrayList<>();
-    Map<String, Object> searchByNestedAlertActive0JSON = JSONUtils.INSTANCE
-        .load(searchByNestedAlertActive0, new TypeReference<Map<String, Object>>() {
-        });
-    alerts.add(searchByNestedAlertActive0JSON);
-    Map<String, Object> searchByNestedAlertActive1JSON = JSONUtils.INSTANCE
-        .load(searchByNestedAlertActive1, new TypeReference<Map<String, Object>>() {
-        });
-    alerts.add(searchByNestedAlertActive1JSON);
-    Map<String, Object> searchByNestedAlertInactive0JSON = JSONUtils.INSTANCE
-        .load(searchByNestedAlertInactive0, new TypeReference<Map<String, Object>>() {
-        });
-    alerts.add(searchByNestedAlertInactive0JSON);
-    Map<String, Object> searchByNestedAlertInactive1JSON = JSONUtils.INSTANCE
-        .load(searchByNestedAlertInactive1, new TypeReference<Map<String, Object>>() {
-        });
-    alerts.add(searchByNestedAlertInactive1JSON);
+    // 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);
     elasticsearchAdd(alerts, INDEX, SENSOR_NAME);
-    // Wait for updates to persist
-    findUpdatedDoc(searchByNestedAlertActive0JSON, "search_by_nested_alert_active_0",
-        SENSOR_NAME);
-    findUpdatedDoc(searchByNestedAlertActive1JSON, "search_by_nested_alert_active_1",
-        SENSOR_NAME);
-    findUpdatedDoc(searchByNestedAlertInactive0JSON, "search_by_nested_alert_inactive_0",
-        SENSOR_NAME);
-    findUpdatedDoc(searchByNestedAlertInactive1JSON, "search_by_nested_alert_inactive_1",
-        SENSOR_NAME);
 
     // Put the nested type into the test index, so that it'll match appropriately
     ((ElasticsearchDao) esDao).getClient().admin().indices().preparePutMapping(INDEX)
@@ -560,23 +677,23 @@ public class ElasticsearchMetaAlertIntegrationTest {
         .setSource(nestedAlertMapping)
         .get();
 
-    // Create metaalerts
-    Map<String, Object> activeMetaAlertJSON = JSONUtils.INSTANCE
-        .load(activeMetaAlert, new TypeReference<Map<String, Object>>() {
-        });
-    activeMetaAlertJSON.put("alert",
-        Arrays.asList(searchByNestedAlertActive0JSON, searchByNestedAlertActive1JSON));
-    Map<String, Object> inactiveMetaAlertJSON = JSONUtils.INSTANCE
-        .load(inactiveMetaAlert, new TypeReference<Map<String, Object>>() {
-        });
-    inactiveMetaAlertJSON.put("alert",
-        Arrays.asList(searchByNestedAlertInactive0JSON, searchByNestedAlertInactive1JSON));
-
+    // 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.
-    elasticsearchAdd(Arrays.asList(activeMetaAlertJSON, inactiveMetaAlertJSON),
-        MetaAlertDao.METAALERTS_INDEX, MetaAlertDao.METAALERT_TYPE);
-    // Wait for updates to persist
-    findUpdatedDoc(activeMetaAlertJSON, "active_metaalert", MetaAlertDao.METAALERT_TYPE);
+    elasticsearchAdd(Arrays.asList(activeMetaAlert, inactiveMetaAlert), 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("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() {
       {
@@ -615,7 +732,7 @@ public class ElasticsearchMetaAlertIntegrationTest {
 
     // Nested query should match a nested alert
     Assert.assertEquals(1, searchResponse.getTotal());
-    Assert.assertEquals("active_metaalert",
+    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
@@ -638,28 +755,21 @@ public class ElasticsearchMetaAlertIntegrationTest {
 
     // Nested query should match a plain alert
     Assert.assertEquals(1, searchResponse.getTotal());
-    Assert.assertEquals("search_by_nested_alert_inactive_0",
+    Assert.assertEquals("message_2",
         searchResponse.getResults().get(0).getSource().get("guid"));
   }
 
   @Test
-  public void shouldGroupHidesAlert() throws Exception {
-    // Create alerts
-    List<Map<String, Object>> alerts = new ArrayList<>();
-    Map<String, Object> groupByChildAlertJson = JSONUtils.INSTANCE
-        .load(groupByChildAlert, new TypeReference<Map<String, Object>>() {
-        });
-    alerts.add(groupByChildAlertJson);
-    Map<String, Object> groupByStandaloneAlertJson = JSONUtils.INSTANCE
-        .load(groupByStandaloneAlert, new TypeReference<Map<String, Object>>() {
-        });
-    alerts.add(groupByStandaloneAlertJson);
+  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);
-    // Wait for updates to persist
-    findUpdatedDoc(groupByChildAlertJson, "group_by_child_alert",
-        SENSOR_NAME);
-    findUpdatedDoc(groupByStandaloneAlertJson, "group_by_standalone_alert",
-        SENSOR_NAME);
+
 
     // Put the nested type into the test index, so that it'll match appropriately
     ((ElasticsearchDao) esDao).getClient().admin().indices().preparePutMapping(INDEX)
@@ -669,6 +779,11 @@ public class ElasticsearchMetaAlertIntegrationTest {
 
     // 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");
@@ -690,24 +805,109 @@ public class ElasticsearchMetaAlertIntegrationTest {
     Assert.assertEquals(10.0d, result.getScore(), 0.0d);
   }
 
+  @SuppressWarnings("unchecked")
   @Test
-  public void testStatusChanges() throws Exception {
-    // Create alerts
-    List<Map<String, Object>> alerts = new ArrayList<>();
-    Map<String, Object> searchByNestedAlertActive0Json = JSONUtils.INSTANCE
-        .load(searchByNestedAlertActive0, new TypeReference<Map<String, Object>>() {
-        });
-    alerts.add(searchByNestedAlertActive0Json);
-    Map<String, Object> searchByNestedAlertActive1Json = JSONUtils.INSTANCE
-        .load(searchByNestedAlertActive1, new TypeReference<Map<String, Object>>() {
-        });
-    alerts.add(searchByNestedAlertActive1Json);
+  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);
-    // Wait for updates to persist
-    findUpdatedDoc(searchByNestedAlertActive0Json, "search_by_nested_alert_active_0",
-        SENSOR_NAME);
-    findUpdatedDoc(searchByNestedAlertActive1Json, "search_by_nested_alert_active_1",
-        SENSOR_NAME);
 
     // Put the nested type into the test index, so that it'll match appropriately
     ((ElasticsearchDao) esDao).getClient().admin().indices().preparePutMapping(INDEX)
@@ -715,199 +915,170 @@ public class ElasticsearchMetaAlertIntegrationTest {
         .setSource(nestedAlertMapping)
         .get();
 
-    // Create metaalerts
-    Map<String, Object> activeMetaAlertJSON = JSONUtils.INSTANCE
-        .load(activeMetaAlert, new TypeReference<Map<String, Object>>() {
-        });
-    activeMetaAlertJSON.put("alert",
-        Arrays.asList(searchByNestedAlertActive0Json, searchByNestedAlertActive1Json));
-
+    // 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(activeMetaAlertJSON),
-        MetaAlertDao.METAALERTS_INDEX, MetaAlertDao.METAALERT_TYPE);
-    // Wait for updates to persist
-    findUpdatedDoc(activeMetaAlertJSON, "active_metaalert", MetaAlertDao.METAALERT_TYPE);
-
-    // Build our update request to inactive status
-    Map<String, Object> documentMap = new HashMap<>();
-
-    documentMap.put("status", MetaAlertStatus.INACTIVE.getStatusString());
-    Document document = new Document(documentMap, "active_metaalert", MetaAlertDao.METAALERT_TYPE,
-        0L);
-    metaDao.update(document, Optional.of(MetaAlertDao.METAALERTS_INDEX));
-
-    Map<String, Object> expectedMetaDoc = new HashMap<>();
-    expectedMetaDoc.putAll(activeMetaAlertJSON);
-    expectedMetaDoc.put("status", MetaAlertStatus.INACTIVE.getStatusString());
-
-    // Make sure the update has gone through on the meta alert and the child alerts.
-    Assert.assertTrue(
-        findUpdatedDoc(expectedMetaDoc, "active_metaalert", MetaAlertDao.METAALERT_TYPE));
-
-    Map<String, Object> expectedAlertDoc0 = new HashMap<>();
-    expectedAlertDoc0.putAll(searchByNestedAlertActive0Json);
-    expectedAlertDoc0.put("metaalerts", new ArrayList<>());
-    Assert.assertTrue(
-        findUpdatedDoc(expectedAlertDoc0, "search_by_nested_alert_active_0", SENSOR_NAME));
-
-    Map<String, Object> expectedAlertDoc1 = new HashMap<>();
-    expectedAlertDoc1.putAll(searchByNestedAlertActive1Json);
-    expectedAlertDoc1.put("metaalerts", new ArrayList<>());
-    Assert.assertTrue(
-        findUpdatedDoc(expectedAlertDoc1, "search_by_nested_alert_active_1", SENSOR_NAME));
-
-    // Search against the indices. Should return the two alerts, but not the inactive metaalert.
-    SearchRequest searchRequest = new SearchRequest();
-    ArrayList<String> indices = new ArrayList<>();
-    indices.add(SENSOR_NAME);
-    indices.add(MetaAlertDao.METAALERT_TYPE);
-    searchRequest.setIndices(indices);
-    searchRequest.setSize(5);
-    searchRequest.setQuery("*");
-
-    // Validate our results
-    SearchResult expected0 = new SearchResult();
-    expected0.setId((String) expectedAlertDoc0.get(Constants.GUID));
-    expected0.setIndex(INDEX);
-    expected0.setSource(expectedAlertDoc0);
-    expected0.setScore(1.0f);
-
-    SearchResult expected1 = new SearchResult();
-    expected1.setId((String) expectedAlertDoc1.get(Constants.GUID));
-    expected1.setIndex(INDEX);
-    expected1.setSource(expectedAlertDoc1);
-    expected1.setScore(1.0f);
-
-    ArrayList<SearchResult> expectedResults = new ArrayList<>();
-    expectedResults.add(expected0);
-    expectedResults.add(expected1);
-
-    SearchResponse result = metaDao.search(searchRequest);
-    Assert.assertEquals(2, result.getTotal());
-    // Use set comparison to avoid ordering issues. We already checked counts.
-    Assert.assertEquals(new HashSet<>(expectedResults), new HashSet<>(result.getResults()));
-
-    // Build our update request back to active status
-    documentMap.put("status", MetaAlertStatus.ACTIVE.getStatusString());
-    document = new Document(documentMap, "active_metaalert", MetaAlertDao.METAALERT_TYPE, 0L);
-    metaDao.update(document, Optional.of(MetaAlertDao.METAALERTS_INDEX));
-
-    expectedMetaDoc = new HashMap<>();
-    expectedMetaDoc.putAll(activeMetaAlertJSON);
-
-    // Make sure the update has gone through on the meta alert and the child alerts.
-    Assert.assertTrue(
-        findUpdatedDoc(expectedMetaDoc, "active_metaalert", MetaAlertDao.METAALERT_TYPE));
-
-    expectedAlertDoc0 = new HashMap<>();
-    expectedAlertDoc0.putAll(searchByNestedAlertActive0Json);
-    Assert.assertTrue(
-        findUpdatedDoc(expectedAlertDoc0, "search_by_nested_alert_active_0", SENSOR_NAME));
-
-    expectedAlertDoc1 = new HashMap<>();
-    expectedAlertDoc1.putAll(searchByNestedAlertActive1Json);
-    Assert.assertTrue(
-        findUpdatedDoc(expectedAlertDoc1, "search_by_nested_alert_active_1", SENSOR_NAME));
-
-    // Search against the indices. Should return just the active metaalert.
-    SearchResult expectedMeta = new SearchResult();
-    expectedMeta.setId((String) activeMetaAlertJSON.get(Constants.GUID));
-    expectedMeta.setIndex(MetaAlertDao.METAALERTS_INDEX);
-    expectedMeta.setSource(activeMetaAlertJSON);
-    expectedMeta.setScore(1.0f);
-
-    expectedResults = new ArrayList<>();
-    expectedResults.add(expectedMeta);
-
-    result = metaDao.search(searchRequest);
-    Assert.assertEquals(1, result.getTotal());
-    Assert.assertEquals(expectedResults, result.getResults());
-  }
+    elasticsearchAdd(Collections.singletonList(metaAlert), METAALERTS_INDEX, MetaAlertDao.METAALERT_TYPE);
 
-  @Test
-  @SuppressWarnings("unchecked")
-  public void shouldUpdateMetaAlertOnAlertPatchOrReplace() throws Exception {
-    List<Map<String, Object>> inputData = new ArrayList<>();
-    Map<String, Object> updateMetaAlertAlert0JSON = JSONUtils.INSTANCE
-        .load(updateMetaAlertAlert0, new TypeReference<Map<String, Object>>() {
-        });
-    inputData.add(updateMetaAlertAlert0JSON);
-    Map<String, Object> updateMetaAlertAlert1JSON = JSONUtils.INSTANCE
-        .load(updateMetaAlertAlert1, new TypeReference<Map<String, Object>>() {
-        });
-    inputData.add(updateMetaAlertAlert1JSON);
-    elasticsearchAdd(inputData, INDEX, SENSOR_NAME);
-    // Wait for updates to persist
-    findUpdatedDoc(updateMetaAlertAlert1JSON, "update_metaalert_alert_1", SENSOR_NAME);
-
-    MetaAlertCreateResponse metaAlertCreateResponse = metaDao
-        .createMetaAlert(new MetaAlertCreateRequest() {{
-          setGuidToIndices(new HashMap<String, String>() {{
-            put("update_metaalert_alert_0", INDEX);
-            put("update_metaalert_alert_1", INDEX);
-          }});
-          setGroups(Collections.singletonList("group"));
-        }});
-    // Wait for updates to persist
-    findCreatedDoc(metaAlertCreateResponse.getGuid(), 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)));
 
-    // Patch alert
-    metaDao.patch(JSONUtils.INSTANCE.load(updateMetaAlertPatchRequest, PatchRequest.class),
-        Optional.empty());
+    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()));
 
-    // Wait for updates to persist
-    updateMetaAlertAlert0JSON.put("field", "patched value 0");
-    findUpdatedDoc(updateMetaAlertAlert0JSON, "update_metaalert_alert_0", SENSOR_NAME);
-
-    Map<String, Object> metaalert = metaDao
-        .getLatest(metaAlertCreateResponse.getGuid(), MetaAlertDao.METAALERT_TYPE).getDocument();
-    List<Map<String, Object>> alerts = (List<Map<String, Object>>) metaalert.get("alert");
-    Assert.assertEquals(2, alerts.size());
-    Assert.assertEquals("update_metaalert_alert_1", alerts.get(0).get("guid"));
-    Assert.assertEquals("value 1", alerts.get(0).get("field"));
-    Assert.assertEquals("update_metaalert_alert_0", alerts.get(1).get("guid"));
-    Assert.assertEquals("patched value 0", alerts.get(1).get("field"));
-
-    // Replace alert
-    metaDao.replace(JSONUtils.INSTANCE.load(updateMetaAlertReplaceRequest, ReplaceRequest.class),
-        Optional.empty());
+      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);
+    }
 
-    // Wait for updates to persist
-    updateMetaAlertAlert0JSON.put("field", "replaced value 0");
-    findUpdatedDoc(updateMetaAlertAlert0JSON, "update_metaalert_alert_0", SENSOR_NAME);
-
-    metaalert = metaDao.getLatest(metaAlertCreateResponse.getGuid(), MetaAlertDao.METAALERT_TYPE)
-        .getDocument();
-    alerts = (List<Map<String, Object>>) metaalert.get("alert");
-    Assert.assertEquals(2, alerts.size());
-    Assert.assertEquals("update_metaalert_alert_1", alerts.get(0).get("guid"));
-    Assert.assertEquals("value 1", alerts.get(0).get("field"));
-    Assert.assertEquals("update_metaalert_alert_0", alerts.get(1).get("guid"));
-    Assert.assertEquals("replaced value 0", alerts.get(1).get("field"));
+    {
+      // 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 boolean findUpdatedDoc(Map<String, Object> message0, String guid, String sensorType)
-      throws InterruptedException, IOException {
-    boolean found = false;
-    for (int t = 0; t < MAX_RETRIES && !found; ++t, Thread.sleep(SLEEP_MS)) {
+  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");
+      cnt = docs
+          .stream()
+          .filter(d -> {
+            Object newfield = d.get(fieldName);
+            return newfield != null && newfield.equals(fieldValue);
+          }).count();
+    }
+    return cnt;
+  }
+
+  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);
+      cnt = docs
+          .stream()
+          .filter(d -> {
+            List<Map<String, Object>> alerts = (List<Map<String, Object>>) d
+                .get(MetaAlertDao.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;
+  }
+
+  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())) {
-        found = true;
+        return;
       }
     }
-    return found;
+    throw new OriginalNotFoundException("Count not find " + guid + " after " + MAX_RETRIES + "tries");
   }
 
   protected boolean findCreatedDoc(String guid, String sensorType)
-      throws InterruptedException, IOException {
-    boolean found = false;
-    for (int t = 0; t < MAX_RETRIES && !found; ++t, Thread.sleep(SLEEP_MS)) {
+      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) {
-        found = true;
+        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;
+        }
       }
     }
-    return found;
+    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("source:type", SENSOR_NAME);
+      alerts.put(MetaAlertDao.THREAT_FIELD_DEFAULT, 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("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;
   }
 
   protected void elasticsearchAdd(List<Map<String, Object>> inputData, String index, String docType)

http://git-wip-us.apache.org/repos/asf/metron/blob/fd896fbe/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 fddf056..116ee4b 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
@@ -152,7 +152,7 @@ public class ElasticsearchUpdateIntegrationTest {
       Assert.assertEquals(message0, doc.getDocument());
       {
         //ensure hbase is up to date
-        Get g = new Get(guid.getBytes());
+        Get g = new Get(HBaseDao.Key.toBytes(new HBaseDao.Key(guid, SENSOR_NAME)));
         Result r = table.get(g);
         NavigableMap<byte[], byte[]> columns = r.getFamilyMap(CF.getBytes());
         Assert.assertEquals(1, columns.size());
@@ -190,7 +190,7 @@ public class ElasticsearchUpdateIntegrationTest {
       Assert.assertEquals(message0, doc.getDocument());
       {
         //ensure hbase is up to date
-        Get g = new Get(guid.getBytes());
+        Get g = new Get(HBaseDao.Key.toBytes(new HBaseDao.Key(guid, SENSOR_NAME)));
         Result r = table.get(g);
         NavigableMap<byte[], byte[]> columns = r.getFamilyMap(CF.getBytes());
         Assert.assertEquals(2, columns.size());

http://git-wip-us.apache.org/repos/asf/metron/blob/fd896fbe/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/converter/EnrichmentKey.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/converter/EnrichmentKey.java b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/converter/EnrichmentKey.java
index cab0dfc..a91def2 100644
--- a/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/converter/EnrichmentKey.java
+++ b/metron-platform/metron-enrichment/src/main/java/org/apache/metron/enrichment/converter/EnrichmentKey.java
@@ -21,20 +21,12 @@ import com.google.common.hash.HashFunction;
 import com.google.common.hash.Hasher;
 import com.google.common.hash.Hashing;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metron.common.utils.KeyUtil;
 import org.apache.metron.enrichment.lookup.LookupKey;
 
 import java.io.*;
 
 public class EnrichmentKey implements LookupKey {
-  private static final int SEED = 0xDEADBEEF;
-  private static final int HASH_PREFIX_SIZE=16;
-  ThreadLocal<HashFunction> hFunction= new ThreadLocal<HashFunction>() {
-    @Override
-    protected HashFunction initialValue() {
-      return Hashing.murmur3_128(SEED);
-    }
-  };
-
   public String indicator;
   public String type;
 
@@ -63,21 +55,14 @@ public class EnrichmentKey implements LookupKey {
     } catch (IOException e) {
       throw new RuntimeException("Unable to convert type and indicator to bytes", e);
     }
-    Hasher hasher = hFunction.get().newHasher();
-    hasher.putBytes(Bytes.toBytes(indicator));
-    byte[] prefix = hasher.hash().asBytes();
-    byte[] val = new byte[indicatorBytes.length + prefix.length];
-    int offset = 0;
-    System.arraycopy(prefix, 0, val, offset, prefix.length);
-    offset += prefix.length;
-    System.arraycopy(indicatorBytes, 0, val, offset, indicatorBytes.length);
-    return val;
+    byte[] prefix = KeyUtil.INSTANCE.getPrefix(Bytes.toBytes(indicator));
+    return KeyUtil.INSTANCE.merge(prefix, indicatorBytes);
   }
 
   @Override
   public void fromBytes(byte[] row) {
     ByteArrayInputStream baos = new ByteArrayInputStream(row);
-    baos.skip(HASH_PREFIX_SIZE);
+    baos.skip(KeyUtil.HASH_PREFIX_SIZE);
     DataInputStream w = new DataInputStream(baos);
     try {
       type = w.readUTF();

http://git-wip-us.apache.org/repos/asf/metron/blob/fd896fbe/metron-platform/metron-indexing/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/README.md b/metron-platform/metron-indexing/README.md
index 86c8d37..d8c5beb 100644
--- a/metron-platform/metron-indexing/README.md
+++ b/metron-platform/metron-indexing/README.md
@@ -152,9 +152,12 @@ in parallel.  This enables a flexible strategy for specifying your backing store
 For instance, currently the REST API supports the update functionality and may be configured with a list of
 IndexDao implementations to use to support the updates.
 
+Updates with the IndexDao.update method replace the current object with the new object.  For partial updates,
+use IndexDao.patch instead.
+
 ### The `HBaseDao`
 
-Updates will be written to HBase. The key structure is the GUID and
+Updates will be written to HBase. The key structure includes the GUID and sensor type and
 for each new version, a new column is created with value as the message.
 
 The HBase table and column family are configured via fields in the global configuration.
@@ -169,17 +172,23 @@ The HBase column family to use for message updates.
 
 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
-enables a couple extra features: creation of a meta alert and the ability to get all meta alerts associated with an alert.
+enables several features: 
+* the ability to get all meta alerts associated with an alert
+* creation of a meta alert
+* adding alerts to a meta alert
+* removing alerts from a meta alert
+* changing a meta alert's status
 
 The implementation of this is to denormalize the relationship between alerts and meta alerts, and store alerts as a nested field within a meta alert.
 The use of nested fields is to avoid the limitations of parent-child relationships (one-to-many) and merely linking by IDs
-(which causes issues with pagination as a result of being unable to join indices).
+(which causes issues with pagination as a result of being unable to join indices).  A list of containing meta alerts is stored 
+on an alert for the purpose of keeping source alerts and alerts contained in meta alerts in sync.
 
 The search functionality of `IndexDao` is wrapped by the `MetaAlertDao` in order to provide both regular and meta alerts side-by-side with sorting.
 The updating capabilities are similarly wrapped, in order to ensure updates are carried through both the alerts and associated meta alerts.
 Both of these functions are handled under the hood.
 
-In addition, an API endpoint is added for the meta alert specific features of creation and going from meta alert to alert.
+In addition, API endpoints have been added to expose the features listed above.
 The denormalization handles the case of going from meta alert to alert automatically.
 
 # Notes on Performance Tuning

http://git-wip-us.apache.org/repos/asf/metron/blob/fd896fbe/metron-platform/metron-indexing/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-indexing/pom.xml b/metron-platform/metron-indexing/pom.xml
index 35dfb2b..5e64118 100644
--- a/metron-platform/metron-indexing/pom.xml
+++ b/metron-platform/metron-indexing/pom.xml
@@ -167,6 +167,13 @@
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>org.apache.metron</groupId>
+            <artifactId>metron-hbase</artifactId>
+            <version>${project.parent.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
     <build>
         <plugins>