You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@solr.apache.org by GitBox <gi...@apache.org> on 2022/01/03 10:08:25 UTC

[GitHub] [solr] arobinski opened a new pull request #488: Fix a problem in OpenExchangeRatesOrgProvider: introduce a parameter so that it downloads currency rates while indexing not while searching

arobinski opened a new pull request #488:
URL: https://github.com/apache/solr/pull/488


   https://issues.apache.org/jira/browse/SOLR-15883
   
   # Description
   
   Currently the currency fields (specifically: fields of type `solr.CurrencyFieldType` that use `OpenExchangeRatesOrgProvider`) refresh the currency rates during a search request. If the openexchangerates.org service is unavailable or slow, all search requests (that use currencies) may fail.
   
   This MR introduces a new boolean parameter `refreshWhileSearching` to fields of type `solr.CurrencyFieldType` that have `providerClass` equal to `solr.OpenExchangeRatesOrgProvider`. If the parameter is true (default), it will work as before. If it's false, then currency rates will NOT be refreshed during a search request.
   
   Additionally, the MR introduces a new class: OpenExchangeRatesOrgReloader, which can listen to `newSearcher` and `firstSearcher` events and reload currency rates when a new searcher is opened. If you want to use the OpenExchangeRatesOrgReloader, it has to be added to solrconfig.xml.
   
   This is my first MR to the SOLR project and I am not sure if I did everything properly. I'd be grateful for a review from someone who knows the SOLR code better.
   
   # Solution
   
   Instead of refreshing currency rates during a search request, refresh them during commit. More information above.
   
   # Tests
   
   I modified one test and added one test to the OpenExchangeRatesOrgProviderTest class.
   
   I have run `./gradlew check` and the test ChaosMonkeyNothingIsSafeWithPullReplicasTest failed, but it seems not related to my changes.
   
   # Checklist
   
   Please review the following and check all that apply:
   
   - [x] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/solr/HowToContribute) and my code conforms to the standards described there to the best of my ability.
   - [x] I have created a Jira issue and added the issue ID to my pull request title.
   - [x] I have given Solr maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended)
   - [x] I have developed this patch against the `main` branch.
   - [x] I have run `./gradlew check`.
   - [x] I have added tests for my changes.
   - [ ] I have added documentation for the [Reference Guide](https://github.com/apache/solr/tree/main/solr/solr-ref-guide)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] janhoy commented on pull request #488: SOLR-15883 Fix a problem in OpenExchangeRatesOrgProvider: introduce a parameter so that it downloads currency rates while indexing not while searching

Posted by GitBox <gi...@apache.org>.
janhoy commented on pull request #488:
URL: https://github.com/apache/solr/pull/488#issuecomment-1004811565


   > This seems like a much better solution than mine. I will try to implement it in this pull request.
   
   I'd look at creating a pool `ScheduledExecutorService.newMDCAwareSingleThreadExecutor(new SolrNamedThreadFactory("currencyProvider"))`, and in `reload()` wrap the logic inside a Runnable that is passed to the exector and then return immediately.
   
   That way, we only refresh rates if there are actual search requests, but once a request comes, it will only trigger a background refresh, while the request will succeed on the existing rates (stale).
   
   If you add proper synchronization on read/write of the `protected OpenExchangeRates rates` class, then the amount of time you need to lock while swapping in the new object should be minimal.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] arobinski commented on a change in pull request #488: SOLR-15883 Fix a problem in OpenExchangeRatesOrgProvider: introduce a parameter so that it downloads currency rates while indexing not while searching

Posted by GitBox <gi...@apache.org>.
arobinski commented on a change in pull request #488:
URL: https://github.com/apache/solr/pull/488#discussion_r785324624



##########
File path: solr/core/src/java/org/apache/solr/schema/FileExchangeRateProvider.java
##########
@@ -31,6 +31,7 @@
 
 import org.apache.lucene.util.ResourceLoader;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.core.SolrCore;

Review comment:
       and I removed this one too




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #488: SOLR-15883 Fix a problem in OpenExchangeRatesOrgProvider: introduce a parameter so that it downloads currency rates while indexing not while searching

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #488:
URL: https://github.com/apache/solr/pull/488#discussion_r785339009



##########
File path: solr/core/src/java/org/apache/solr/schema/OpenExchangeRatesOrgProvider.java
##########
@@ -135,31 +144,47 @@ public String toString() {
     return rates.getRates().keySet();
   }
 
-  @Override
-  @SuppressWarnings("resource")
-  public boolean reload() throws SolrException {
-    InputStream ratesJsonStream = null;
+  public void reloadNow() throws SolrException {
     try {
       log.debug("Reloading exchange rates from {}", ratesFileLocation);
-      try {
-        ratesJsonStream = (new URL(ratesFileLocation)).openStream();
-      } catch (Exception e) {
-        ratesJsonStream = resourceLoader.openResource(ratesFileLocation);
-      }
-        
-      rates = new OpenExchangeRates(ratesJsonStream);
-      return true;
+
+      // We set the timestamp based on the monotonic time not the time from openexchangerates.com because
+      // in the reload() method we will be comparing it to the monotonic time. If we took the time
+      // from openexchangerates.com and the timestamp was off or the system clock was set to a different time, we could
+      // be refreshing the exchange rates too often (even on every search request) or too rarely. Also, it's necessary
+      // to set the timestamp to the current time and to do it before the actual reload, so in the case
+      // when the openexchangerates.com server is down for more than 60 minutes, we don't try to refresh the rates
+      // on every search request.
+      lastReloadTimestamp = TimeUnit.NANOSECONDS.toSeconds(System.nanoTime());
+
+      rates = new OpenExchangeRates();
+      log.debug("Successfully reloaded exchange rates from {}", ratesFileLocation);
     } catch (Exception e) {
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Error reloading exchange rates", e);
+      log.error("Error reloading exchange rates", e);
     } finally {
-      if (ratesJsonStream != null) {
-        try {
-          ratesJsonStream.close();
-        } catch (IOException e) {
-          throw new SolrException(ErrorCode.SERVER_ERROR, "Error closing stream", e);
-        }
+      reloading = false;
+    }
+  }
+
+  @Override
+  public boolean reload() throws SolrException {
+    if ((lastReloadTimestamp + refreshIntervalSeconds) >= TimeUnit.NANOSECONDS.toSeconds(System.nanoTime())) {
+      return true;
+    }
+
+    synchronized (this) {
+      if (!reloading) {
+        log.debug("Refresh interval has expired. Refreshing exchange rates (in a separate thread).");
+        reloading = true;
+        executorService.submit(this::reloadNow);

Review comment:
       *FutureReturnValueIgnored:*  Return value of methods returning Future must be checked. Ignoring returned Futures suppresses exceptions thrown from the code that completes the Future. [(details)](https://errorprone.info/bugpattern/FutureReturnValueIgnored)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/core/src/java/org/apache/solr/schema/OpenExchangeRatesOrgProvider.java
##########
@@ -135,31 +144,47 @@ public String toString() {
     return rates.getRates().keySet();
   }
 
-  @Override
-  @SuppressWarnings("resource")
-  public boolean reload() throws SolrException {
-    InputStream ratesJsonStream = null;
+  public void reloadNow() throws SolrException {
     try {
       log.debug("Reloading exchange rates from {}", ratesFileLocation);
-      try {
-        ratesJsonStream = (new URL(ratesFileLocation)).openStream();
-      } catch (Exception e) {
-        ratesJsonStream = resourceLoader.openResource(ratesFileLocation);
-      }
-        
-      rates = new OpenExchangeRates(ratesJsonStream);
-      return true;
+
+      // We set the timestamp based on the monotonic time not the time from openexchangerates.com because
+      // in the reload() method we will be comparing it to the monotonic time. If we took the time
+      // from openexchangerates.com and the timestamp was off or the system clock was set to a different time, we could
+      // be refreshing the exchange rates too often (even on every search request) or too rarely. Also, it's necessary
+      // to set the timestamp to the current time and to do it before the actual reload, so in the case
+      // when the openexchangerates.com server is down for more than 60 minutes, we don't try to refresh the rates
+      // on every search request.
+      lastReloadTimestamp = TimeUnit.NANOSECONDS.toSeconds(System.nanoTime());
+
+      rates = new OpenExchangeRates();
+      log.debug("Successfully reloaded exchange rates from {}", ratesFileLocation);
     } catch (Exception e) {
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Error reloading exchange rates", e);
+      log.error("Error reloading exchange rates", e);
     } finally {
-      if (ratesJsonStream != null) {
-        try {
-          ratesJsonStream.close();
-        } catch (IOException e) {
-          throw new SolrException(ErrorCode.SERVER_ERROR, "Error closing stream", e);
-        }
+      reloading = false;
+    }
+  }
+
+  @Override
+  public boolean reload() throws SolrException {
+    if ((lastReloadTimestamp + refreshIntervalSeconds) >= TimeUnit.NANOSECONDS.toSeconds(System.nanoTime())) {

Review comment:
       *THREAD_SAFETY_VIOLATION:*  Read/Write race. Non-private method `OpenExchangeRatesOrgProvider.reload()` reads without synchronization from `this.refreshIntervalSeconds`. Potentially races with write in method `OpenExchangeRatesOrgProvider.init(...)`.
    Reporting because this access may occur on a background thread.
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] SadatAnwar commented on a change in pull request #488: SOLR-15883 Fix a problem in OpenExchangeRatesOrgProvider: introduce a parameter so that it downloads currency rates while indexing not while searching

Posted by GitBox <gi...@apache.org>.
SadatAnwar commented on a change in pull request #488:
URL: https://github.com/apache/solr/pull/488#discussion_r784801442



##########
File path: solr/core/src/java/org/apache/solr/schema/FileExchangeRateProvider.java
##########
@@ -31,6 +31,7 @@
 
 import org.apache.lucene.util.ResourceLoader;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.core.SolrCore;

Review comment:
       Unused import?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] arobinski commented on pull request #488: SOLR-15883 Fix a problem in OpenExchangeRatesOrgProvider: introduce a parameter so that it downloads currency rates while indexing not while searching

Posted by GitBox <gi...@apache.org>.
arobinski commented on pull request #488:
URL: https://github.com/apache/solr/pull/488#issuecomment-1004694776


   This seems like a much better solution than mine. I will try to implement it in this pull request.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] SadatAnwar commented on a change in pull request #488: SOLR-15883 Fix a problem in OpenExchangeRatesOrgProvider: introduce a parameter so that it downloads currency rates while indexing not while searching

Posted by GitBox <gi...@apache.org>.
SadatAnwar commented on a change in pull request #488:
URL: https://github.com/apache/solr/pull/488#discussion_r784801251



##########
File path: solr/core/src/java/org/apache/solr/schema/ExchangeRateProvider.java
##########
@@ -20,6 +20,7 @@
 
 import org.apache.lucene.util.ResourceLoader;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.core.SolrCore;

Review comment:
       is this an unused import? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] arobinski commented on a change in pull request #488: SOLR-15883 Fix a problem in OpenExchangeRatesOrgProvider: introduce a parameter so that it downloads currency rates while indexing not while searching

Posted by GitBox <gi...@apache.org>.
arobinski commented on a change in pull request #488:
URL: https://github.com/apache/solr/pull/488#discussion_r785324608



##########
File path: solr/core/src/java/org/apache/solr/schema/ExchangeRateProvider.java
##########
@@ -20,6 +20,7 @@
 
 import org.apache.lucene.util.ResourceLoader;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.core.SolrCore;

Review comment:
       yes. I removed it. Thanks.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] janhoy commented on pull request #488: SOLR-15883 Fix a problem in OpenExchangeRatesOrgProvider: introduce a parameter so that it downloads currency rates while indexing not while searching

Posted by GitBox <gi...@apache.org>.
janhoy commented on pull request #488:
URL: https://github.com/apache/solr/pull/488#issuecomment-1004068228


   Interesting solution to this problem. Another way to fix it would be for the CurrencyFieldType to register `ScheduledExecutorService` and do the reloading in a tread-safe manner in that thread. It would be more automatic without new params or classes, and it would respect the reloadInterval setting as before, only it would do the reload in the executor instead of in the query thread. I think it is quite resource efficient as well, you won't occupy a thread except for each run of the executor. WDYT?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] arobinski commented on pull request #488: SOLR-15883 Fix a problem in OpenExchangeRatesOrgProvider: introduce a parameter so that it downloads currency rates while indexing not while searching

Posted by GitBox <gi...@apache.org>.
arobinski commented on pull request #488:
URL: https://github.com/apache/solr/pull/488#issuecomment-1011258926


   @janhoy I tried doing as you wrote. I created an ExecutorService using `ExecutorUtil.newMDCAwareSingleThreadExecutor` . It's on line 70 in OpenExchangeRatesOrgProvider.java. It generally seems to work, but the problem is that SOLR does not terminate properly now. The reason is that I never call `executorService.shutdown()`. I don't know from where I can call the `shutdown` method. Maybe there is a SOLR shutdown event that I can subscribe to? Could you give me a hint, please?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org