You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by mm...@apache.org on 2018/07/11 01:32:23 UTC

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

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

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrUtilities.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrUtilities.java b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrUtilities.java
new file mode 100644
index 0000000..ce0ae84
--- /dev/null
+++ b/metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrUtilities.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.metron.solr.dao;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.metron.common.Constants;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertConstants;
+import org.apache.metron.indexing.dao.search.SearchResult;
+import org.apache.metron.indexing.dao.update.Document;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrInputDocument;
+
+public class SolrUtilities {
+
+  public static SearchResult getSearchResult(SolrDocument solrDocument, List<String> fields) {
+    SearchResult searchResult = new SearchResult();
+    searchResult.setId((String) solrDocument.getFieldValue(Constants.GUID));
+    Map<String, Object> docSource = toDocument(solrDocument).getDocument();
+    final Map<String, Object> source = new HashMap<>();
+    if (fields != null) {
+      fields.forEach(field -> source.put(field, docSource.get(field)));
+    } else {
+      source.putAll(docSource);
+    }
+    searchResult.setSource(source);
+    return searchResult;
+  }
+
+  public static Document toDocument(SolrDocument solrDocument) {
+    Map<String, Object> document = new HashMap<>();
+    solrDocument.getFieldNames().stream()
+        .filter(name -> !name.equals(SolrDao.VERSION_FIELD))
+        .forEach(name -> document.put(name, solrDocument.getFieldValue(name)));
+    // Make sure to put child alerts in
+    if (solrDocument.hasChildDocuments() && solrDocument
+        .getFieldValue(Constants.SENSOR_TYPE)
+        .equals(MetaAlertConstants.METAALERT_TYPE)) {
+      List<Map<String, Object>> childDocuments = new ArrayList<>();
+      for (SolrDocument childDoc : solrDocument.getChildDocuments()) {
+        Map<String, Object> childDocMap = new HashMap<>();
+        childDoc.getFieldNames().stream()
+            .filter(name -> !name.equals(SolrDao.VERSION_FIELD))
+            .forEach(name -> childDocMap.put(name, childDoc.getFieldValue(name)));
+        childDocuments.add(childDocMap);
+      }
+
+      document.put(MetaAlertConstants.ALERT_FIELD, childDocuments);
+    }
+    return new Document(document,
+        (String) solrDocument.getFieldValue(Constants.GUID),
+        (String) solrDocument.getFieldValue(Constants.SENSOR_TYPE), 0L);
+  }
+
+  public static SolrInputDocument toSolrInputDocument(Document document) {
+    SolrInputDocument solrInputDocument = new SolrInputDocument();
+    for (Map.Entry<String, Object> field : document.getDocument().entrySet()) {
+      if (field.getKey().equals(MetaAlertConstants.ALERT_FIELD)) {
+        // We have a children, that needs to be translated as a child doc, not a field.
+        List<Map<String, Object>> alerts = (List<Map<String, Object>>) field.getValue();
+        for (Map<String, Object> alert : alerts) {
+          SolrInputDocument childDocument = new SolrInputDocument();
+          for (Map.Entry<String, Object> alertField : alert.entrySet()) {
+            childDocument.addField(alertField.getKey(), alertField.getValue());
+          }
+          solrInputDocument.addChildDocument(childDocument);
+        }
+      } else {
+        solrInputDocument.addField(field.getKey(), field.getValue());
+      }
+    }
+    return solrInputDocument;
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/metron/blob/49f851e0/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrMetaAlertDaoTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrMetaAlertDaoTest.java b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrMetaAlertDaoTest.java
new file mode 100644
index 0000000..5378467
--- /dev/null
+++ b/metron-platform/metron-solr/src/test/java/org/apache/metron/solr/dao/SolrMetaAlertDaoTest.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.metron.solr.dao;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.metron.indexing.dao.AccessConfig;
+import org.apache.metron.indexing.dao.HBaseDao;
+import org.apache.metron.indexing.dao.IndexDao;
+import org.apache.metron.indexing.dao.MultiIndexDao;
+import org.apache.metron.indexing.dao.RetrieveLatestDao;
+import org.apache.metron.indexing.dao.metaalert.MetaAlertCreateRequest;
+import org.apache.metron.indexing.dao.search.FieldType;
+import org.apache.metron.indexing.dao.search.GetRequest;
+import org.apache.metron.indexing.dao.search.GroupRequest;
+import org.apache.metron.indexing.dao.search.GroupResponse;
+import org.apache.metron.indexing.dao.search.InvalidCreateException;
+import org.apache.metron.indexing.dao.search.SearchRequest;
+import org.apache.metron.indexing.dao.search.SearchResponse;
+import org.apache.metron.indexing.dao.update.Document;
+import org.apache.metron.indexing.dao.update.OriginalNotFoundException;
+import org.apache.metron.indexing.dao.update.PatchRequest;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class SolrMetaAlertDaoTest {
+  private static AccessConfig accessConfig = new AccessConfig();
+
+  @BeforeClass
+  public static void setupBefore() {
+    accessConfig.setGlobalConfigSupplier(() ->
+        new HashMap<String, Object>() {{
+          put("solr.zookeeper", "zookeeper:2181");
+        }}
+    );
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidInit() {
+    IndexDao dao = new IndexDao() {
+      @Override
+      public SearchResponse search(SearchRequest searchRequest) {
+        return null;
+      }
+
+      @Override
+      public GroupResponse group(GroupRequest groupRequest) {
+        return null;
+      }
+
+      @Override
+      public void init(AccessConfig config) {
+      }
+
+      @Override
+      public Document getLatest(String guid, String sensorType) {
+        return null;
+      }
+
+      @Override
+      public Iterable<Document> getAllLatest(
+          List<GetRequest> getRequests) {
+        return null;
+      }
+
+      @Override
+      public void update(Document update, Optional<String> index) {
+      }
+
+      @Override
+      public void batchUpdate(Map<Document, Optional<String>> updates) {
+      }
+
+      @Override
+      public void patch(RetrieveLatestDao dao, PatchRequest request, Optional<Long> timestamp) {
+      }
+
+      @Override
+      public Map<String, FieldType> getColumnMetadata(List<String> indices) {
+        return null;
+      }
+    };
+    SolrMetaAlertDao metaAlertDao = new SolrMetaAlertDao();
+    metaAlertDao.init(dao);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInitInvalidDao() {
+    HBaseDao dao = new HBaseDao();
+    SolrMetaAlertDao solrDao = new SolrMetaAlertDao();
+    solrDao.init(dao, Optional.empty());
+  }
+
+  @Test(expected = InvalidCreateException.class)
+  public void testCreateMetaAlertEmptyGuids() throws InvalidCreateException, IOException {
+    SolrDao solrDao = new SolrDao();
+    solrDao.init(accessConfig);
+    SolrMetaAlertDao emaDao = new SolrMetaAlertDao();
+    emaDao.init(solrDao);
+
+    MetaAlertCreateRequest createRequest = new MetaAlertCreateRequest();
+    emaDao.createMetaAlert(createRequest);
+  }
+
+  @Test(expected = InvalidCreateException.class)
+  public void testCreateMetaAlertEmptyGroups() throws InvalidCreateException, IOException {
+    SolrDao solrDao = new SolrDao();
+    solrDao.init(accessConfig);
+    MultiIndexDao miDao = new MultiIndexDao(solrDao);
+    SolrMetaAlertDao emaDao = new SolrMetaAlertDao();
+    emaDao.init(miDao);
+
+    MetaAlertCreateRequest createRequest = new MetaAlertCreateRequest();
+    createRequest.setAlerts(Collections.singletonList(new GetRequest("don't", "care")));
+    emaDao.createMetaAlert(createRequest);
+  }
+}

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

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

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

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

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

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

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