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:30 UTC

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

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);
+    }
+  }
+}