You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@metron.apache.org by justinleet <gi...@git.apache.org> on 2018/06/05 18:52:46 UTC

[GitHub] metron pull request #1050: METRON-1585: SolrRetrieveLatestDao does not use t...

GitHub user justinleet opened a pull request:

    https://github.com/apache/metron/pull/1050

    METRON-1585: SolrRetrieveLatestDao does not use the collection lookup

    ## Contributor Comments
    
    - Fixed the naming of the parameters in `getLatest()`.
    - Updated the `SolrRetrieveLatestDao` to use the collection lookup.  Includes updating constructor to have access to the appropriate config.
    - Moved the utility function for the collection lookup to `SolrUtilities`, and changed `SolrUpdateDao` to use it.
    - Added integration tests for the `SolrRetrieveLatestDao` to make sure this works as expected.
    
    I also ran it up in full dev and did
    ```
    curl -u user:password -X POST --header 'Content-Type: application/json' --header 'Accept: application/json' -d '{
      "guid": "b42981ad-8b8f-493d-84fb-2c792d9fa237",
      "sensorType": "bro"
    }' 'http://node1:8082/api/v1/search/findOne' | jq '.'
      % Total    % Received % Xferd  Average Speed   Time    Time     Time  Current
                                     Dload  Upload   Total   Spent    Left  Speed
    ```
    
    The response (after passing through jq for formatting):
    ```
    {
      "adapter.threatinteladapter.end.ts": 1528224091155,
      "bro_timestamp": "1528224089.932079",
      "ip_dst_port": 8080,
      "enrichmentsplitterbolt.splitter.end.ts": 1528224091144,
      "enrichmentsplitterbolt.splitter.begin.ts": 1528224091144,
      "adapter.hostfromjsonlistadapter.end.ts": 1528224091147,
      "adapter.geoadapter.begin.ts": 1528224091146,
      "uid": "CUrRne3iLIxXavQtci",
      "trans_depth": 225,
      "protocol": "http",
      "original_string": "HTTP | id.orig_p:50451 method:GET request_body_len:0 id.resp_p:8080 uri:/api/v1/persist/wizard-data?_=1484169340974 tags:[] uid:CUrRne3iLIxXavQtci referrer:http://node1:8080/ trans_depth:225 host:node1 id.orig_h:192.168.66.1 response_body_len:0 user_agent:Mozilla/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/55.0.2883.95 Safari/537.36 ts:1528224089.932079 id.resp_h:192.168.66.121",
      "ip_dst_addr": "192.168.66.121",
      "threatinteljoinbolt.joiner.ts": 1528224091158,
      "host": "node1",
      "enrichmentjoinbolt.joiner.ts": 1528224091149,
      "adapter.hostfromjsonlistadapter.begin.ts": 1528224091147,
      "threatintelsplitterbolt.splitter.begin.ts": 1528224091153,
      "ip_src_addr": "192.168.66.1",
      "user_agent": "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_12_2) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/55.0.2883.95 Safari/537.36",
      "timestamp": 1528224089932,
      "method": "GET",
      "request_body_len": 0,
      "uri": "/api/v1/persist/wizard-data?_=1484169340974",
      "source.type": "bro",
      "adapter.geoadapter.end.ts": 1528224091146,
      "referrer": "http://node1:8080/",
      "threatintelsplitterbolt.splitter.end.ts": 1528224091153,
      "adapter.threatinteladapter.begin.ts": 1528224091155,
      "ip_src_port": 50451,
      "guid": "b42981ad-8b8f-493d-84fb-2c792d9fa237",
      "response_body_len": 0
    }
    ```
    
    ## Pull Request Checklist
    
    Thank you for submitting a contribution to Apache Metron.  
    Please refer to our [Development Guidelines](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=61332235) for the complete guide to follow for contributions.  
    Please refer also to our [Build Verification Guidelines](https://cwiki.apache.org/confluence/display/METRON/Verifying+Builds?show-miniview) for complete smoke testing guides.  
    
    
    In order to streamline the review of the contribution we ask you follow these guidelines and ask you to double check the following:
    
    ### For all changes:
    - [x] Is there a JIRA ticket associated with this PR? If not one needs to be created at [Metron Jira](https://issues.apache.org/jira/browse/METRON/?selectedTab=com.atlassian.jira.jira-projects-plugin:summary-panel).
    - [x] Does your PR title start with METRON-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
    - [x] Has your PR been rebased against the latest commit within the target branch (typically master)?
    
    
    ### For code changes:
    - [x] Have you included steps to reproduce the behavior or problem that is being changed or addressed?
    - [x] Have you included steps or a guide to how the change may be verified and tested manually?
    - [x] Have you ensured that the full suite of tests and checks have been executed in the root metron folder via:
      ```
      mvn -q clean integration-test install && dev-utilities/build-utils/verify_licenses.sh 
      ```
    
    - [x] Have you written or updated unit tests and or integration tests to verify your changes?
    - [x] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
    - [x] Have you verified the basic functionality of the build by building and running locally with Vagrant full-dev environment or the equivalent?
    
    ### For documentation related changes:
    - [x] Have you ensured that format looks appropriate for the output in which it is rendered by building and verifying the site-book? If not then run the following commands and the verify changes via `site-book/target/site/index.html`:
    
      ```
      cd site-book
      mvn site
      ```
    
    #### Note:
    Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.
    It is also recommended that [travis-ci](https://travis-ci.org) is set up for your personal repository such that your branches are built there before submitting a pull request.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/justinleet/metron retrieveLatest

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/metron/pull/1050.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1050
    
----
commit b58ca1f239e4756d00903b9c2091175417242bf0
Author: justinjleet <ju...@...>
Date:   2018-06-05T17:30:56Z

    Updating solr retrieve latest dao to use lookup

commit 050abf99a69654782ec6b40546bbec4667bc956a
Author: justinjleet <ju...@...>
Date:   2018-06-05T18:23:33Z

    fixing test teardown

----


---

[GitHub] metron pull request #1050: METRON-1585: SolrRetrieveLatestDao does not use t...

Posted by justinleet <gi...@git.apache.org>.
Github user justinleet commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1050#discussion_r193207026
  
    --- Diff: metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java ---
    @@ -0,0 +1,168 @@
    +/*
    + * 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.solr.SolrConstants.SOLR_ZOOKEEPER;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +
    +import com.google.common.collect.Iterables;
    +import java.io.IOException;
    +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.AccessConfig;
    +import org.apache.metron.indexing.dao.IndexDao;
    +import org.apache.metron.indexing.dao.search.GetRequest;
    +import org.apache.metron.indexing.dao.update.Document;
    +import org.apache.metron.solr.dao.SolrDao;
    +import org.apache.metron.solr.integration.components.SolrComponent;
    +import org.apache.solr.client.solrj.SolrServerException;
    +import org.junit.After;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +public class SolrRetrieveLatestIntegrationTest {
    +
    +  private static SolrComponent solrComponent;
    +
    +  protected static final String TEST_COLLECTION = "test";
    +  protected static final String TEST_SENSOR = "test_sensor";
    +  protected static final String BRO_SENSOR = "bro";
    +
    +  private static IndexDao dao;
    +
    +  @BeforeClass
    +  public static void setupBeforeClass() throws Exception {
    +    solrComponent = new SolrComponent.Builder().build();
    +    solrComponent.start();
    +  }
    +
    +  @Before
    +  public void setup() throws Exception {
    +    solrComponent
    +        .addCollection(TEST_COLLECTION, "../metron-solr/src/test/resources/config/test/conf");
    +    solrComponent.addCollection(BRO_SENSOR, "../metron-solr/src/main/config/schema/bro");
    +
    +    AccessConfig accessConfig = new AccessConfig();
    +    Map<String, Object> globalConfig = new HashMap<>();
    +    globalConfig.put(SOLR_ZOOKEEPER, solrComponent.getZookeeperUrl());
    +    accessConfig.setGlobalConfigSupplier(() -> globalConfig);
    +    // Map the sensor name to the collection name for test.
    +    accessConfig.setIndexSupplier(s -> s.equals(TEST_SENSOR) ? TEST_COLLECTION : s);
    +
    +    dao = new SolrDao();
    +    dao.init(accessConfig);
    +    addData(BRO_SENSOR, BRO_SENSOR);
    +    addData(TEST_COLLECTION, TEST_SENSOR);
    +  }
    +
    +  @After
    +  public void reset() {
    +    solrComponent.reset();
    +  }
    +
    +  @AfterClass
    +  public static void teardown() {
    +    solrComponent.stop();
    +  }
    +
    +  @Test
    +  public void testGetLatest() throws IOException {
    --- End diff --
    
    I'll add a case for it.


---

[GitHub] metron pull request #1050: METRON-1585: SolrRetrieveLatestDao does not use t...

Posted by justinleet <gi...@git.apache.org>.
Github user justinleet commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1050#discussion_r193208812
  
    --- Diff: metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java ---
    @@ -0,0 +1,168 @@
    +/*
    + * 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.solr.SolrConstants.SOLR_ZOOKEEPER;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +
    +import com.google.common.collect.Iterables;
    +import java.io.IOException;
    +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.AccessConfig;
    +import org.apache.metron.indexing.dao.IndexDao;
    +import org.apache.metron.indexing.dao.search.GetRequest;
    +import org.apache.metron.indexing.dao.update.Document;
    +import org.apache.metron.solr.dao.SolrDao;
    +import org.apache.metron.solr.integration.components.SolrComponent;
    +import org.apache.solr.client.solrj.SolrServerException;
    +import org.junit.After;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +public class SolrRetrieveLatestIntegrationTest {
    +
    +  private static SolrComponent solrComponent;
    +
    +  protected static final String TEST_COLLECTION = "test";
    +  protected static final String TEST_SENSOR = "test_sensor";
    +  protected static final String BRO_SENSOR = "bro";
    +
    +  private static IndexDao dao;
    +
    +  @BeforeClass
    +  public static void setupBeforeClass() throws Exception {
    +    solrComponent = new SolrComponent.Builder().build();
    +    solrComponent.start();
    +  }
    +
    +  @Before
    +  public void setup() throws Exception {
    +    solrComponent
    +        .addCollection(TEST_COLLECTION, "../metron-solr/src/test/resources/config/test/conf");
    +    solrComponent.addCollection(BRO_SENSOR, "../metron-solr/src/main/config/schema/bro");
    +
    +    AccessConfig accessConfig = new AccessConfig();
    +    Map<String, Object> globalConfig = new HashMap<>();
    +    globalConfig.put(SOLR_ZOOKEEPER, solrComponent.getZookeeperUrl());
    +    accessConfig.setGlobalConfigSupplier(() -> globalConfig);
    +    // Map the sensor name to the collection name for test.
    +    accessConfig.setIndexSupplier(s -> s.equals(TEST_SENSOR) ? TEST_COLLECTION : s);
    +
    +    dao = new SolrDao();
    +    dao.init(accessConfig);
    +    addData(BRO_SENSOR, BRO_SENSOR);
    +    addData(TEST_COLLECTION, TEST_SENSOR);
    +  }
    +
    +  @After
    +  public void reset() {
    +    solrComponent.reset();
    +  }
    +
    +  @AfterClass
    +  public static void teardown() {
    +    solrComponent.stop();
    +  }
    +
    +  @Test
    +  public void testGetLatest() throws IOException {
    +    Document actual = dao.getLatest("message_1_bro", BRO_SENSOR);
    +    assertEquals(buildExpectedDocument(BRO_SENSOR, 1), actual);
    +  }
    +
    +  @Test
    +  public void testGetLatestCollectionSensorDiffer() throws IOException {
    +    Document actual = dao.getLatest("message_1_test_sensor", TEST_SENSOR);
    +    assertEquals(buildExpectedDocument(TEST_SENSOR, 1), actual);
    +  }
    +
    +  @Test
    +  public void testGetAllLatest() throws IOException {
    +    List<GetRequest> requests = new ArrayList<>();
    +    requests.add(buildGetRequest(BRO_SENSOR, 1));
    +    requests.add(buildGetRequest(BRO_SENSOR, 2));
    +
    +    Iterable<Document> actual = dao.getAllLatest(requests);
    +    for (Document doc : actual) {
    --- End diff --
    
    I'll kill this while I'm in there.  Oops.


---

[GitHub] metron pull request #1050: METRON-1585: SolrRetrieveLatestDao does not use t...

Posted by justinleet <gi...@git.apache.org>.
Github user justinleet closed the pull request at:

    https://github.com/apache/metron/pull/1050


---

[GitHub] metron issue #1050: METRON-1585: SolrRetrieveLatestDao does not use the coll...

Posted by justinleet <gi...@git.apache.org>.
Github user justinleet commented on the issue:

    https://github.com/apache/metron/pull/1050
  
    Closing feature branch PR


---

[GitHub] metron pull request #1050: METRON-1585: SolrRetrieveLatestDao does not use t...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1050#discussion_r193204450
  
    --- Diff: metron-platform/metron-solr/src/test/java/org/apache/metron/solr/integration/SolrRetrieveLatestIntegrationTest.java ---
    @@ -0,0 +1,168 @@
    +/*
    + * 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.solr.SolrConstants.SOLR_ZOOKEEPER;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +
    +import com.google.common.collect.Iterables;
    +import java.io.IOException;
    +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.AccessConfig;
    +import org.apache.metron.indexing.dao.IndexDao;
    +import org.apache.metron.indexing.dao.search.GetRequest;
    +import org.apache.metron.indexing.dao.update.Document;
    +import org.apache.metron.solr.dao.SolrDao;
    +import org.apache.metron.solr.integration.components.SolrComponent;
    +import org.apache.solr.client.solrj.SolrServerException;
    +import org.junit.After;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +public class SolrRetrieveLatestIntegrationTest {
    +
    +  private static SolrComponent solrComponent;
    +
    +  protected static final String TEST_COLLECTION = "test";
    +  protected static final String TEST_SENSOR = "test_sensor";
    +  protected static final String BRO_SENSOR = "bro";
    +
    +  private static IndexDao dao;
    +
    +  @BeforeClass
    +  public static void setupBeforeClass() throws Exception {
    +    solrComponent = new SolrComponent.Builder().build();
    +    solrComponent.start();
    +  }
    +
    +  @Before
    +  public void setup() throws Exception {
    +    solrComponent
    +        .addCollection(TEST_COLLECTION, "../metron-solr/src/test/resources/config/test/conf");
    +    solrComponent.addCollection(BRO_SENSOR, "../metron-solr/src/main/config/schema/bro");
    +
    +    AccessConfig accessConfig = new AccessConfig();
    +    Map<String, Object> globalConfig = new HashMap<>();
    +    globalConfig.put(SOLR_ZOOKEEPER, solrComponent.getZookeeperUrl());
    +    accessConfig.setGlobalConfigSupplier(() -> globalConfig);
    +    // Map the sensor name to the collection name for test.
    +    accessConfig.setIndexSupplier(s -> s.equals(TEST_SENSOR) ? TEST_COLLECTION : s);
    +
    +    dao = new SolrDao();
    +    dao.init(accessConfig);
    +    addData(BRO_SENSOR, BRO_SENSOR);
    +    addData(TEST_COLLECTION, TEST_SENSOR);
    +  }
    +
    +  @After
    +  public void reset() {
    +    solrComponent.reset();
    +  }
    +
    +  @AfterClass
    +  public static void teardown() {
    +    solrComponent.stop();
    +  }
    +
    +  @Test
    +  public void testGetLatest() throws IOException {
    --- End diff --
    
    Do we have a test that covers when a sensor type cannot be mapped to an index?  Seems like we should validate how that condition is handled.


---

[GitHub] metron pull request #1050: METRON-1585: SolrRetrieveLatestDao does not use t...

Posted by justinleet <gi...@git.apache.org>.
Github user justinleet commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1050#discussion_r193207415
  
    --- Diff: metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrRetrieveLatestDao.java ---
    @@ -61,10 +70,13 @@ public Document getLatest(String guid, String collection) throws IOException {
       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);
    +      Optional<String> index = SolrUtilities
    +          .getIndex(config.getIndexSupplier(), getRequest.getSensorType(), Optional.empty());
    +      if (index.isPresent()) {
    +        Collection<String> ids = collectionIdMap.getOrDefault(index.get(), new HashSet<>());
    +        ids.add(getRequest.getGuid());
    +        collectionIdMap.put(index.get(), ids);
    +      }
    --- End diff --
    
    It means that someone requested something bogus (e.g. a GUID against the wrong sensor).  Logging it is a good idea, though.


---

[GitHub] metron issue #1050: METRON-1585: SolrRetrieveLatestDao does not use the coll...

Posted by justinleet <gi...@git.apache.org>.
Github user justinleet commented on the issue:

    https://github.com/apache/metron/pull/1050
  
    @nickwallen Added some logging for when sensor type doesn't map to an index.  Also added a couple test cases for things like
    
    - When a mapping is broken and the function returns nothing
    - When we query for something outright missing (the test case is a bro guid in the test sensor)
    - When we provide an explicit index to `getAllLatest`. This is done by providing a mismatched explicit index (we say it's bro, even though source type is for test).  This means that nothing will be found.
    - A test case for not all of the GetRequests have a corresponding result.


---

[GitHub] metron pull request #1050: METRON-1585: SolrRetrieveLatestDao does not use t...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1050#discussion_r193200033
  
    --- Diff: metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrRetrieveLatestDao.java ---
    @@ -61,10 +70,13 @@ public Document getLatest(String guid, String collection) throws IOException {
       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);
    +      Optional<String> index = SolrUtilities
    +          .getIndex(config.getIndexSupplier(), getRequest.getSensorType(), Optional.empty());
    +      if (index.isPresent()) {
    +        Collection<String> ids = collectionIdMap.getOrDefault(index.get(), new HashSet<>());
    +        ids.add(getRequest.getGuid());
    +        collectionIdMap.put(index.get(), ids);
    +      }
    --- End diff --
    
    What does it mean if there is no index present?  Should we log something in that case?


---

[GitHub] metron pull request #1050: METRON-1585: SolrRetrieveLatestDao does not use t...

Posted by justinleet <gi...@git.apache.org>.
Github user justinleet commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1050#discussion_r193207117
  
    --- Diff: metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrRetrieveLatestDao.java ---
    @@ -38,16 +40,23 @@
     public class SolrRetrieveLatestDao implements RetrieveLatestDao {
     
       private transient SolrClient client;
    +  private AccessConfig config;
     
    -  public SolrRetrieveLatestDao(SolrClient client) {
    +  public SolrRetrieveLatestDao(SolrClient client, AccessConfig config) {
         this.client = client;
    +    this.config = config;
       }
     
       @Override
    -  public Document getLatest(String guid, String collection) throws IOException {
    -
    +  public Document getLatest(String guid, String sensorType) throws IOException {
         try {
    -      SolrDocument solrDocument = client.getById(collection, guid);
    +      Optional<String> index = SolrUtilities
    +          .getIndex(config.getIndexSupplier(), sensorType, Optional.empty());
    +      if (!index.isPresent()) {
    +        return null;
    --- End diff --
    
    Probably a good idea to log, I'll add it.


---

[GitHub] metron issue #1050: METRON-1585: SolrRetrieveLatestDao does not use the coll...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on the issue:

    https://github.com/apache/metron/pull/1050
  
    +1 Looks great.  Thanks!


---

[GitHub] metron issue #1050: METRON-1585: SolrRetrieveLatestDao does not use the coll...

Posted by merrimanr <gi...@git.apache.org>.
Github user merrimanr commented on the issue:

    https://github.com/apache/metron/pull/1050
  
    This has been merged.  Can you close @justinleet?


---

[GitHub] metron pull request #1050: METRON-1585: SolrRetrieveLatestDao does not use t...

Posted by nickwallen <gi...@git.apache.org>.
Github user nickwallen commented on a diff in the pull request:

    https://github.com/apache/metron/pull/1050#discussion_r193201422
  
    --- Diff: metron-platform/metron-solr/src/main/java/org/apache/metron/solr/dao/SolrRetrieveLatestDao.java ---
    @@ -38,16 +40,23 @@
     public class SolrRetrieveLatestDao implements RetrieveLatestDao {
     
       private transient SolrClient client;
    +  private AccessConfig config;
     
    -  public SolrRetrieveLatestDao(SolrClient client) {
    +  public SolrRetrieveLatestDao(SolrClient client, AccessConfig config) {
         this.client = client;
    +    this.config = config;
       }
     
       @Override
    -  public Document getLatest(String guid, String collection) throws IOException {
    -
    +  public Document getLatest(String guid, String sensorType) throws IOException {
         try {
    -      SolrDocument solrDocument = client.getById(collection, guid);
    +      Optional<String> index = SolrUtilities
    +          .getIndex(config.getIndexSupplier(), sensorType, Optional.empty());
    +      if (!index.isPresent()) {
    +        return null;
    --- End diff --
    
    Should we log if we are not able to map a sensor to its index?  That should not happen if everything is working as intended, right?


---