Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/433#discussion_r212679352
  
    --- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
    @@ -230,6 +188,95 @@ public void processAdd(AddUpdateCommand cmd) throws 
IOException {
         }
       }
     
    +
    +  private String createCollectionsIfRequired(Instant docTimestamp, String 
targetCollection, String printableId) {
    +    // Even though it is possible that multiple requests hit this code in 
the 1-2 sec that
    +    // it takes to create a collection, it's an established anti-pattern 
to feed data with a very large number
    +    // of client connections. This in mind, we only guard against spamming 
the overseer within a batch of
    +    // updates. We are intentionally tolerating a low level of redundant 
requests in favor of simpler code. Most
    +    // super-sized installations with many update clients will likely be 
multi-tenant and multiple tenants
    +    // probably don't write to the same alias. As such, we have deferred 
any solution to the "many clients causing
    +    // collection creation simultaneously" problem until such time as 
someone actually has that problem in a
    +    // real world use case that isn't just an anti-pattern.
    +    try {
    +      CreationType creationType = requiresCreateCollection(docTimestamp, 
timeRoutedAlias.getPreemptiveCreateWindow());
    +      switch (creationType) {
    +        case SYNCHRONOUS:
    +          // This next line blocks until all collections required by the 
current document have been created
    +          return maintain(targetCollection, docTimestamp, printableId, 
false);
    +        case ASYNC_PREEMPTIVE:
    +          // Note: creating an executor and throwing it away is slightly 
expensive, but this is only likely to happen
    +          // once per hour/day/week (depending on time slice size for the 
TRA). If the executor were retained, it
    +          // would need to be shut down in a close hook to avoid test 
failures due to thread leaks which is slightly
    +          // more complicated from a code maintenance and readability 
stand point. An executor must used instead of a
    +          // thread to ensure we pick up the proper MDC logging stuff from 
ExecutorUtil. T
    +          if (preemptiveCreationExecutor == null) {
    +            DefaultSolrThreadFactory threadFactory = new 
DefaultSolrThreadFactory("TRA-preemptive-creation");
    +            preemptiveCreationExecutor = 
newMDCAwareSingleThreadExecutor(threadFactory);
    +            preemptiveCreationExecutor.execute(() -> {
    +              maintain(targetCollection, docTimestamp, printableId, true);
    +              preemptiveCreationExecutor.shutdown();
    +              preemptiveCreationExecutor = null;
    +            });
    +          }
    +          return targetCollection;
    +        case NONE:
    +          return targetCollection; // just for clarity...
    +        default:
    +          return targetCollection; // could use fall through, but fall 
through is fiddly for later editors.
    +      }
    +      // do nothing if creationType == NONE
    +    } catch (SolrException e) {
    +      throw e;
    +    } catch (Exception e) {
    +      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
    +    }
    +  }
    +
    +  /**
    +   * Determine if the a new collection will be required based on the 
document timestamp. Passing null for
    +   * preemptiveCreateInterval tells you if the document is beyond all 
existing collections with a response of
    +   * {@link CreationType#NONE} or {@link CreationType#SYNCHRONOUS}, and 
passing a valid date math for
    +   * preemptiveCreateMath additionally distinguishes the case where the 
document is close enough to the end of
    +   * the TRA to trigger preemptive creation but not beyond all existing 
collections with a value of
    +   * {@link CreationType#ASYNC_PREEMPTIVE}.
    +   *
    +   * @param routeTimestamp The timestamp from the document
    +   * @param preemptiveCreateMath The date math indicating the {@link 
TimeRoutedAlias#preemptiveCreateMath}
    +   * @return a {@code CreationType} indicating if and how to create a 
collection
    +   */
    +  private CreationType requiresCreateCollection(Instant routeTimestamp,  
String preemptiveCreateMath) {
    --- End diff --
    
    don't need to pass preemptiveCreateMath... consider it's easily available 
from the timeRoutedAlias field which is already used in the code below.  Since 
elsewhere you've renamed "routeTimestamp" to be "docTimestamp", perhaps rename 
that as well?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to