[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-09-06 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/lucene-solr/pull/433


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-30 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r214199163
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -437,45 +461,46 @@ protected void doClose() {
 
 
   /**
-   * Create as many collections as required. This method loops to allow 
for the possibility that the routeTimestamp
+   * Create as many collections as required. This method loops to allow 
for the possibility that the docTimestamp
* requires more than one collection to be created. Since multiple 
threads may be invoking maintain on separate
* requests to the same alias, we must pass in the name of the 
collection that this thread believes to be the most
* recent collection. This assumption is checked when the command is 
executed in the overseer. When this method
* finds that all collections required have been created it returns the 
(possibly new) most recent collection.
* The return value is ignored by the calling code in the async 
preemptive case.
*
-   * @param targetCollection the initial notion of the latest collection 
available.
* @param docTimestamp the timestamp from the document that determines 
routing
* @param printableId an identifier for the add command used in error 
messages
+   * @param targetCollectionDesc the descriptor for the presently selected 
collection which should also be
+   * the most recent collection in all cases 
where this method is invoked.
* @return The latest collection, including collections created during 
maintenance
*/
-  public String maintain(String targetCollection, Instant docTimestamp, 
String printableId, boolean asyncSinglePassOnly) {
-do { // typically we don't loop; it's only when we need to create a 
collection
-
-  // Note: This code no longer short circuits immediately when it sees 
that the expected latest
-  // collection is the current latest collection. With the advent of 
preemptive collection creation
-  // we always need to do the time based checks. Otherwise, we cannot 
handle the case where the
-  // preemptive window is larger than our TRA's time slices
-
-  // Check the doc isn't too far in the future
-
-  if (NONE == requiresCreateCollection(docTimestamp, 
timeRoutedAlias.getPreemptiveCreateWindow()))
-return targetCollection; // thus we don't need another collection
+  private String createAllRequiredCollections( Instant docTimestamp, 
String printableId,
+   Map.Entry 
targetCollectionDesc) {
+do {
+  switch(typeOfCreationRequired(docTimestamp, 
targetCollectionDesc.getKey())) {
+case NONE:
+  return targetCollectionDesc.getValue(); // we don't need another 
collection
+case ASYNC_PREEMPTIVE:
+  // can happen when preemptive interval is longer than one time 
slice
+  preemptiveAsync(() -> 
createNextCollection(this.parsedCollectionsDesc.get(0).getValue()));
+  return targetCollectionDesc.getValue();
+case SYNCHRONOUS:
+  createNextCollection(targetCollectionDesc.getValue()); // 
*should* throw if fails for some reason but...
+  if (!updateParsedCollectionAliases()) { // thus we didn't make 
progress...
+// this is not expected, even in known failure cases, but we 
check just in case
+throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+"We need to create a new time routed collection but for 
unknown reasons were unable to do so.");
+  }
+  // then retry the loop ... have to do find again in case other 
requests also added collections
+  // that were made visible when we called 
updateParsedCollectionAliases()
+  targetCollectionDesc = findCandidateGivenTimestamp(docTimestamp, 
printableId);
+  break;
+default:
+  throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
"Unknown creation type while adding " +
--- End diff --

After playing with it I decided to just factor it into a helper method 
(reused in both places) since it's only real draw back is cluttering up the 
code visually. (I've already done the typing, and the assert just looks a bit 
odd to my eye)


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-30 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r214173448
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -437,45 +461,46 @@ protected void doClose() {
 
 
   /**
-   * Create as many collections as required. This method loops to allow 
for the possibility that the routeTimestamp
+   * Create as many collections as required. This method loops to allow 
for the possibility that the docTimestamp
* requires more than one collection to be created. Since multiple 
threads may be invoking maintain on separate
* requests to the same alias, we must pass in the name of the 
collection that this thread believes to be the most
* recent collection. This assumption is checked when the command is 
executed in the overseer. When this method
* finds that all collections required have been created it returns the 
(possibly new) most recent collection.
* The return value is ignored by the calling code in the async 
preemptive case.
*
-   * @param targetCollection the initial notion of the latest collection 
available.
* @param docTimestamp the timestamp from the document that determines 
routing
* @param printableId an identifier for the add command used in error 
messages
+   * @param targetCollectionDesc the descriptor for the presently selected 
collection which should also be
+   * the most recent collection in all cases 
where this method is invoked.
* @return The latest collection, including collections created during 
maintenance
*/
-  public String maintain(String targetCollection, Instant docTimestamp, 
String printableId, boolean asyncSinglePassOnly) {
-do { // typically we don't loop; it's only when we need to create a 
collection
-
-  // Note: This code no longer short circuits immediately when it sees 
that the expected latest
-  // collection is the current latest collection. With the advent of 
preemptive collection creation
-  // we always need to do the time based checks. Otherwise, we cannot 
handle the case where the
-  // preemptive window is larger than our TRA's time slices
-
-  // Check the doc isn't too far in the future
-
-  if (NONE == requiresCreateCollection(docTimestamp, 
timeRoutedAlias.getPreemptiveCreateWindow()))
-return targetCollection; // thus we don't need another collection
+  private String createAllRequiredCollections( Instant docTimestamp, 
String printableId,
+   Map.Entry 
targetCollectionDesc) {
+do {
+  switch(typeOfCreationRequired(docTimestamp, 
targetCollectionDesc.getKey())) {
+case NONE:
+  return targetCollectionDesc.getValue(); // we don't need another 
collection
+case ASYNC_PREEMPTIVE:
+  // can happen when preemptive interval is longer than one time 
slice
+  preemptiveAsync(() -> 
createNextCollection(this.parsedCollectionsDesc.get(0).getValue()));
--- End diff --

Agree see above comment


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-30 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r214170180
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -165,31 +165,42 @@ private String getAliasName() {
 
   @Override
   public void processAdd(AddUpdateCommand cmd) throws IOException {
-SolrInputDocument solrInputDocument = cmd.getSolrInputDocument();
-final Object routeValue = 
solrInputDocument.getFieldValue(timeRoutedAlias.getRouteField());
-final Instant docTimestampToRoute = parseRouteKey(routeValue);
-updateParsedCollectionAliases();
-String candidateCollection = 
findCandidateCollectionGivenTimestamp(docTimestampToRoute, 
cmd.getPrintableId());
-final Instant maxFutureTime = 
Instant.now().plusMillis(timeRoutedAlias.getMaxFutureMs());
+final Instant docTimestamp =
+
parseRouteKey(cmd.getSolrInputDocument().getFieldValue(timeRoutedAlias.getRouteField()));
+
 // TODO: maybe in some cases the user would want to ignore/warn 
instead?
-if (docTimestampToRoute.isAfter(maxFutureTime)) {
+if 
(docTimestamp.isAfter(Instant.now().plusMillis(timeRoutedAlias.getMaxFutureMs(
 {
   throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-  "The document's time routed key of " + docTimestampToRoute + " 
is too far in the future given " +
+  "The document's time routed key of " + docTimestamp + " is too 
far in the future given " +
   TimeRoutedAlias.ROUTER_MAX_FUTURE + "=" + 
timeRoutedAlias.getMaxFutureMs());
 }
-String targetCollection = 
createCollectionsIfRequired(docTimestampToRoute, candidateCollection, 
cmd.getPrintableId());
+
+// to avoid potential for race conditions, this next method should not 
get called again unless
+// we have created a collection synchronously
+updateParsedCollectionAliases();
+
+String targetCollection = createCollectionsIfRequired(docTimestamp, 
cmd.getPrintableId(), cmd);
+
 if (thisCollection.equals(targetCollection)) {
   // pass on through; we've reached the right collection
   super.processAdd(cmd);
 } else {
   // send to the right collection
-  SolrCmdDistributor.Node targetLeaderNode = 
routeDocToSlice(targetCollection, solrInputDocument);
+  SolrCmdDistributor.Node targetLeaderNode = 
routeDocToSlice(targetCollection, cmd.getSolrInputDocument());
   cmdDistrib.distribAdd(cmd, 
Collections.singletonList(targetLeaderNode), new 
ModifiableSolrParams(outParamsToLeader));
 }
   }
 
-
-  private String createCollectionsIfRequired(Instant docTimestamp, String 
targetCollection, String printableId) {
+  /**
+   * Create any required collections and return the name of the collection 
to which the current document should be sent.
+   *
+   * @param docTimestamp the date for the document taken from the field 
specified in the TRA config
--- End diff --

nice catch, yeah I added the cmd late in the game. very happy to get rid of 
printable id.
Generally if I doc a method I doc it fully (goal at least) if not I don't 
do any of it. I generally add to private methods only where it seems to be 
adding something. In this case the method name doesn't actually indicate that 
we're also selecting the target collection as well. The method name is already 
long so I used docs.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-30 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r214172726
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -437,45 +461,46 @@ protected void doClose() {
 
 
   /**
-   * Create as many collections as required. This method loops to allow 
for the possibility that the routeTimestamp
+   * Create as many collections as required. This method loops to allow 
for the possibility that the docTimestamp
* requires more than one collection to be created. Since multiple 
threads may be invoking maintain on separate
* requests to the same alias, we must pass in the name of the 
collection that this thread believes to be the most
* recent collection. This assumption is checked when the command is 
executed in the overseer. When this method
* finds that all collections required have been created it returns the 
(possibly new) most recent collection.
* The return value is ignored by the calling code in the async 
preemptive case.
*
-   * @param targetCollection the initial notion of the latest collection 
available.
* @param docTimestamp the timestamp from the document that determines 
routing
* @param printableId an identifier for the add command used in error 
messages
+   * @param targetCollectionDesc the descriptor for the presently selected 
collection which should also be
+   * the most recent collection in all cases 
where this method is invoked.
* @return The latest collection, including collections created during 
maintenance
*/
-  public String maintain(String targetCollection, Instant docTimestamp, 
String printableId, boolean asyncSinglePassOnly) {
-do { // typically we don't loop; it's only when we need to create a 
collection
-
-  // Note: This code no longer short circuits immediately when it sees 
that the expected latest
-  // collection is the current latest collection. With the advent of 
preemptive collection creation
-  // we always need to do the time based checks. Otherwise, we cannot 
handle the case where the
-  // preemptive window is larger than our TRA's time slices
-
-  // Check the doc isn't too far in the future
-
-  if (NONE == requiresCreateCollection(docTimestamp, 
timeRoutedAlias.getPreemptiveCreateWindow()))
-return targetCollection; // thus we don't need another collection
+  private String createAllRequiredCollections( Instant docTimestamp, 
String printableId,
+   Map.Entry 
targetCollectionDesc) {
+do {
+  switch(typeOfCreationRequired(docTimestamp, 
targetCollectionDesc.getKey())) {
+case NONE:
+  return targetCollectionDesc.getValue(); // we don't need another 
collection
+case ASYNC_PREEMPTIVE:
+  // can happen when preemptive interval is longer than one time 
slice
+  preemptiveAsync(() -> 
createNextCollection(this.parsedCollectionsDesc.get(0).getValue()));
+  return targetCollectionDesc.getValue();
+case SYNCHRONOUS:
+  createNextCollection(targetCollectionDesc.getValue()); // 
*should* throw if fails for some reason but...
+  if (!updateParsedCollectionAliases()) { // thus we didn't make 
progress...
+// this is not expected, even in known failure cases, but we 
check just in case
+throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+"We need to create a new time routed collection but for 
unknown reasons were unable to do so.");
+  }
+  // then retry the loop ... have to do find again in case other 
requests also added collections
+  // that were made visible when we called 
updateParsedCollectionAliases()
+  targetCollectionDesc = findCandidateGivenTimestamp(docTimestamp, 
printableId);
+  break;
+default:
+  throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
"Unknown creation type while adding " +
--- End diff --

no not required, but I had a wrong default return 
targetCollectionDesc.getValue() previously and replaced it with this. The 
assert would be fine too.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-30 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r214171837
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -198,32 +209,32 @@ private String createCollectionsIfRequired(Instant 
docTimestamp, String targetCo
 // 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.
+Map.Entry candidateCollectionDesc = 
findCandidateGivenTimestamp(docTimestamp, cmd.getPrintableId());
+String candidateCollectionName = candidateCollectionDesc.getValue();
 try {
-  CreationType creationType = requiresCreateCollection(docTimestamp, 
timeRoutedAlias.getPreemptiveCreateWindow());
-  switch (creationType) {
+  switch (typeOfCreationRequired(docTimestamp, 
candidateCollectionDesc.getKey())) {
 case SYNCHRONOUS:
   // This next line blocks until all collections required by the 
current document have been created
-  return maintain(targetCollection, docTimestamp, printableId, 
false);
+  return createAllRequiredCollections(docTimestamp, printableId, 
candidateCollectionDesc);
 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;
-});
+// It's important not to add code between here and the prior 
call to findCandidateGivenTimestamp()
+// in processAdd() that invokes 
updateParsedCollectionAliases(). Doing so would update parsedCollectionsDesc
+// and create a race condition. We are relying on the fact 
that get(0) is returning the head of the parsed
+// collections that existed when candidateCollectionDesc was 
created. If this class updates it's notion of
+// parsedCollectionsDesc since candidateCollectionDesc was 
chosen, we could create collection n+2
+// instead of collection n+1.
+
+// This line does not block and the document can be added 
immediately
+preemptiveAsync(() -> 
createNextCollection(this.parsedCollectionsDesc.get(0).getValue()));
--- End diff --

argh I fixed this at one point (as evidenced by my comment) and somehow it 
got back in there. I think it got reconsolidated was when I fixed the async 
during sync bug (for which I added another section to the test). 


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-30 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r214121353
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -198,32 +209,32 @@ private String createCollectionsIfRequired(Instant 
docTimestamp, String targetCo
 // 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.
+Map.Entry candidateCollectionDesc = 
findCandidateGivenTimestamp(docTimestamp, cmd.getPrintableId());
+String candidateCollectionName = candidateCollectionDesc.getValue();
 try {
-  CreationType creationType = requiresCreateCollection(docTimestamp, 
timeRoutedAlias.getPreemptiveCreateWindow());
-  switch (creationType) {
+  switch (typeOfCreationRequired(docTimestamp, 
candidateCollectionDesc.getKey())) {
 case SYNCHRONOUS:
   // This next line blocks until all collections required by the 
current document have been created
-  return maintain(targetCollection, docTimestamp, printableId, 
false);
+  return createAllRequiredCollections(docTimestamp, printableId, 
candidateCollectionDesc);
 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;
-});
+// It's important not to add code between here and the prior 
call to findCandidateGivenTimestamp()
+// in processAdd() that invokes 
updateParsedCollectionAliases(). Doing so would update parsedCollectionsDesc
+// and create a race condition. We are relying on the fact 
that get(0) is returning the head of the parsed
+// collections that existed when candidateCollectionDesc was 
created. If this class updates it's notion of
+// parsedCollectionsDesc since candidateCollectionDesc was 
chosen, we could create collection n+2
+// instead of collection n+1.
+
+// This line does not block and the document can be added 
immediately
+preemptiveAsync(() -> 
createNextCollection(this.parsedCollectionsDesc.get(0).getValue()));
--- End diff --

bug: do not refer to this.parsedCollectionsDesc from the lambda runnable as 
it is not thread-safe (and I don't think we should try to make it so.  the 
input should be gathered in the immediate line prior (thus in the main thread) 
so that it's already resolved.

To make this bug harder (you did it twice), you could change 
preemptiveAsync() to take the collection name instead of taking a lambda.  It 
could then be named createNextCollectionAsync


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-30 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r214122537
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -165,31 +165,42 @@ private String getAliasName() {
 
   @Override
   public void processAdd(AddUpdateCommand cmd) throws IOException {
-SolrInputDocument solrInputDocument = cmd.getSolrInputDocument();
-final Object routeValue = 
solrInputDocument.getFieldValue(timeRoutedAlias.getRouteField());
-final Instant docTimestampToRoute = parseRouteKey(routeValue);
-updateParsedCollectionAliases();
-String candidateCollection = 
findCandidateCollectionGivenTimestamp(docTimestampToRoute, 
cmd.getPrintableId());
-final Instant maxFutureTime = 
Instant.now().plusMillis(timeRoutedAlias.getMaxFutureMs());
+final Instant docTimestamp =
+
parseRouteKey(cmd.getSolrInputDocument().getFieldValue(timeRoutedAlias.getRouteField()));
+
 // TODO: maybe in some cases the user would want to ignore/warn 
instead?
-if (docTimestampToRoute.isAfter(maxFutureTime)) {
+if 
(docTimestamp.isAfter(Instant.now().plusMillis(timeRoutedAlias.getMaxFutureMs(
 {
   throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-  "The document's time routed key of " + docTimestampToRoute + " 
is too far in the future given " +
+  "The document's time routed key of " + docTimestamp + " is too 
far in the future given " +
   TimeRoutedAlias.ROUTER_MAX_FUTURE + "=" + 
timeRoutedAlias.getMaxFutureMs());
 }
-String targetCollection = 
createCollectionsIfRequired(docTimestampToRoute, candidateCollection, 
cmd.getPrintableId());
+
+// to avoid potential for race conditions, this next method should not 
get called again unless
+// we have created a collection synchronously
+updateParsedCollectionAliases();
+
+String targetCollection = createCollectionsIfRequired(docTimestamp, 
cmd.getPrintableId(), cmd);
+
 if (thisCollection.equals(targetCollection)) {
   // pass on through; we've reached the right collection
   super.processAdd(cmd);
 } else {
   // send to the right collection
-  SolrCmdDistributor.Node targetLeaderNode = 
routeDocToSlice(targetCollection, solrInputDocument);
+  SolrCmdDistributor.Node targetLeaderNode = 
routeDocToSlice(targetCollection, cmd.getSolrInputDocument());
   cmdDistrib.distribAdd(cmd, 
Collections.singletonList(targetLeaderNode), new 
ModifiableSolrParams(outParamsToLeader));
 }
   }
 
-
-  private String createCollectionsIfRequired(Instant docTimestamp, String 
targetCollection, String printableId) {
+  /**
+   * Create any required collections and return the name of the collection 
to which the current document should be sent.
+   *
+   * @param docTimestamp the date for the document taken from the field 
specified in the TRA config
--- End diff --

These parameter-level docs are fine but FYI don't feel that you have to do 
this, especially for private methods, and especially for obvious parameters 
(i.e. you don't *have* to document all; you could just do some at your 
discretion, or none).  This is subjective of course; I'm sharing my opinion and 
not insisting on anything.  My preference leans towards... "if you having 
something helpful to say then say it, but avoid writing the obvious" (for 
various reasons)

Oh look... you don't need to pass printableId since you could get it from 
AddUpdateCommand if needed.  printableId is a wart on these method signatures 
IMO.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-30 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r214120233
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -437,45 +461,46 @@ protected void doClose() {
 
 
   /**
-   * Create as many collections as required. This method loops to allow 
for the possibility that the routeTimestamp
+   * Create as many collections as required. This method loops to allow 
for the possibility that the docTimestamp
* requires more than one collection to be created. Since multiple 
threads may be invoking maintain on separate
* requests to the same alias, we must pass in the name of the 
collection that this thread believes to be the most
* recent collection. This assumption is checked when the command is 
executed in the overseer. When this method
* finds that all collections required have been created it returns the 
(possibly new) most recent collection.
* The return value is ignored by the calling code in the async 
preemptive case.
*
-   * @param targetCollection the initial notion of the latest collection 
available.
* @param docTimestamp the timestamp from the document that determines 
routing
* @param printableId an identifier for the add command used in error 
messages
+   * @param targetCollectionDesc the descriptor for the presently selected 
collection which should also be
+   * the most recent collection in all cases 
where this method is invoked.
* @return The latest collection, including collections created during 
maintenance
*/
-  public String maintain(String targetCollection, Instant docTimestamp, 
String printableId, boolean asyncSinglePassOnly) {
-do { // typically we don't loop; it's only when we need to create a 
collection
-
-  // Note: This code no longer short circuits immediately when it sees 
that the expected latest
-  // collection is the current latest collection. With the advent of 
preemptive collection creation
-  // we always need to do the time based checks. Otherwise, we cannot 
handle the case where the
-  // preemptive window is larger than our TRA's time slices
-
-  // Check the doc isn't too far in the future
-
-  if (NONE == requiresCreateCollection(docTimestamp, 
timeRoutedAlias.getPreemptiveCreateWindow()))
-return targetCollection; // thus we don't need another collection
+  private String createAllRequiredCollections( Instant docTimestamp, 
String printableId,
+   Map.Entry 
targetCollectionDesc) {
+do {
+  switch(typeOfCreationRequired(docTimestamp, 
targetCollectionDesc.getKey())) {
+case NONE:
+  return targetCollectionDesc.getValue(); // we don't need another 
collection
+case ASYNC_PREEMPTIVE:
+  // can happen when preemptive interval is longer than one time 
slice
+  preemptiveAsync(() -> 
createNextCollection(this.parsedCollectionsDesc.get(0).getValue()));
--- End diff --

bug: do not refer to this.parsedCollectionsDesc from the lambda runnable as 
it is not thread-safe (and I don't think we should try to make it so.  the 
input should be gathered in the immediate line prior (thus in the main thread) 
so that it's already resolved.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-30 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r214118388
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -437,45 +461,46 @@ protected void doClose() {
 
 
   /**
-   * Create as many collections as required. This method loops to allow 
for the possibility that the routeTimestamp
+   * Create as many collections as required. This method loops to allow 
for the possibility that the docTimestamp
* requires more than one collection to be created. Since multiple 
threads may be invoking maintain on separate
* requests to the same alias, we must pass in the name of the 
collection that this thread believes to be the most
* recent collection. This assumption is checked when the command is 
executed in the overseer. When this method
* finds that all collections required have been created it returns the 
(possibly new) most recent collection.
* The return value is ignored by the calling code in the async 
preemptive case.
*
-   * @param targetCollection the initial notion of the latest collection 
available.
* @param docTimestamp the timestamp from the document that determines 
routing
* @param printableId an identifier for the add command used in error 
messages
+   * @param targetCollectionDesc the descriptor for the presently selected 
collection which should also be
+   * the most recent collection in all cases 
where this method is invoked.
* @return The latest collection, including collections created during 
maintenance
*/
-  public String maintain(String targetCollection, Instant docTimestamp, 
String printableId, boolean asyncSinglePassOnly) {
-do { // typically we don't loop; it's only when we need to create a 
collection
-
-  // Note: This code no longer short circuits immediately when it sees 
that the expected latest
-  // collection is the current latest collection. With the advent of 
preemptive collection creation
-  // we always need to do the time based checks. Otherwise, we cannot 
handle the case where the
-  // preemptive window is larger than our TRA's time slices
-
-  // Check the doc isn't too far in the future
-
-  if (NONE == requiresCreateCollection(docTimestamp, 
timeRoutedAlias.getPreemptiveCreateWindow()))
-return targetCollection; // thus we don't need another collection
+  private String createAllRequiredCollections( Instant docTimestamp, 
String printableId,
+   Map.Entry 
targetCollectionDesc) {
+do {
+  switch(typeOfCreationRequired(docTimestamp, 
targetCollectionDesc.getKey())) {
+case NONE:
+  return targetCollectionDesc.getValue(); // we don't need another 
collection
+case ASYNC_PREEMPTIVE:
+  // can happen when preemptive interval is longer than one time 
slice
+  preemptiveAsync(() -> 
createNextCollection(this.parsedCollectionsDesc.get(0).getValue()));
+  return targetCollectionDesc.getValue();
+case SYNCHRONOUS:
+  createNextCollection(targetCollectionDesc.getValue()); // 
*should* throw if fails for some reason but...
+  if (!updateParsedCollectionAliases()) { // thus we didn't make 
progress...
+// this is not expected, even in known failure cases, but we 
check just in case
+throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+"We need to create a new time routed collection but for 
unknown reasons were unable to do so.");
+  }
+  // then retry the loop ... have to do find again in case other 
requests also added collections
+  // that were made visible when we called 
updateParsedCollectionAliases()
+  targetCollectionDesc = findCandidateGivenTimestamp(docTimestamp, 
printableId);
+  break;
+default:
+  throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
"Unknown creation type while adding " +
--- End diff --

I wouldn't waste your virtual breath (lines of code with thoughtful 
explanation) on a default case of an enum switch.Another approach I like is 
to `assert ENUMNAME.values().length == 3;` before the switch which will be 
caught at test time.  I forget but if java still insists we have a default then 
throw IllegalStateException or something like that in a one-liner without 
explaination.  (keep it short & sweet)


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-28 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r213400327
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -405,4 +435,54 @@ protected void doClose() {
 collection, slice.getName(), 
DistributedUpdateProcessor.MAX_RETRIES_ON_FORWARD_DEAULT);
   }
 
+
+  /**
+   * Create as many collections as required. This method loops to allow 
for the possibility that the routeTimestamp
+   * requires more than one collection to be created. Since multiple 
threads may be invoking maintain on separate
+   * requests to the same alias, we must pass in the name of the 
collection that this thread believes to be the most
+   * recent collection. This assumption is checked when the command is 
executed in the overseer. When this method
+   * finds that all collections required have been created it returns the 
(possibly new) most recent collection.
+   * The return value is ignored by the calling code in the async 
preemptive case.
+   *
+   * @param targetCollection the initial notion of the latest collection 
available.
+   * @param docTimestamp the timestamp from the document that determines 
routing
+   * @param printableId an identifier for the add command used in error 
messages
+   * @return The latest collection, including collections created during 
maintenance
+   */
+  public String maintain(String targetCollection, Instant docTimestamp, 
String printableId, boolean asyncSinglePassOnly) {
+do { // typically we don't loop; it's only when we need to create a 
collection
+
+  // Note: This code no longer short circuits immediately when it sees 
that the expected latest
--- End diff --

That's fine, the comment mostly aimed at making the review process clearer. 
You're right it probably doesn't need to be carried forward


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-28 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r212806120
  
--- 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(() -> {
--- End diff --

I've not wanted to create 2 places in the code where we do the same thing, 
but I think I figured out how to factor it so it's both clearer and 
non-duplicative...


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-28 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r213419315
  
--- Diff: 
solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
 ---
@@ -392,29 +393,73 @@ public void testPreemptiveCreation() throws Exception 
{
 CollectionAdminRequest.setAliasProperty(alias)
 .addProperty(TimeRoutedAlias.ROUTER_PREEMPTIVE_CREATE_MATH, 
"3DAY").process(solrClient);
 
-Thread.sleep(1000); // a moment to be sure the alias change has taken 
effect
-
 assertUpdateResponse(add(alias, Collections.singletonList(
 sdoc("id", "7", "timestamp_dt", "2017-10-25T23:01:00Z")), // 
should cause preemptive creation now
 params));
 assertUpdateResponse(solrClient.commit(alias));
 waitCol("2017-10-27", numShards);
-waitCol("2017-10-28", numShards);
 
 cols = new 
CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
-assertEquals(6,cols.size());
+assertEquals(5,cols.size()); // only one created in async case
 assertNumDocs("2017-10-23", 1);
 assertNumDocs("2017-10-24", 1);
 assertNumDocs("2017-10-25", 5);
 assertNumDocs("2017-10-26", 0);
 assertNumDocs("2017-10-27", 0);
+
+assertUpdateResponse(add(alias, Collections.singletonList(
+sdoc("id", "8", "timestamp_dt", "2017-10-25T23:01:00Z")), // 
should cause preemptive creation now
+params));
+assertUpdateResponse(solrClient.commit(alias));
+waitCol("2017-10-27", numShards);
+waitCol("2017-10-28", numShards);
+
+cols = new 
CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
+assertEquals(6,cols.size()); // Subsequent documents continue to 
create up to limit
+assertNumDocs("2017-10-23", 1);
+assertNumDocs("2017-10-24", 1);
+assertNumDocs("2017-10-25", 6);
+assertNumDocs("2017-10-26", 0);
+assertNumDocs("2017-10-27", 0);
 assertNumDocs("2017-10-28", 0);
 
 QueryResponse resp;
 resp = solrClient.query(alias, params(
 "q", "*:*",
 "rows", "10"));
-assertEquals(7, resp.getResults().getNumFound());
+assertEquals(8, resp.getResults().getNumFound());
+
+assertUpdateResponse(add(alias, Arrays.asList(
--- End diff --

addDocsAndCommit contains a lot of logic I don't really want, especially 
the shuffling of the input documents! 


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-28 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r212805945
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -167,59 +167,17 @@ private String getAliasName() {
   public void processAdd(AddUpdateCommand cmd) throws IOException {
 SolrInputDocument solrInputDocument = cmd.getSolrInputDocument();
 final Object routeValue = 
solrInputDocument.getFieldValue(timeRoutedAlias.getRouteField());
-final Instant routeTimestamp = parseRouteKey(routeValue);
-
+final Instant docTimestampToRoute = parseRouteKey(routeValue);
 updateParsedCollectionAliases();
-String targetCollection;
-do { // typically we don't loop; it's only when we need to create a 
collection
-  targetCollection = 
findTargetCollectionGivenTimestamp(routeTimestamp);
-
-  if (targetCollection == null) {
-throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-"Doc " + cmd.getPrintableId() + " couldn't be routed with " + 
timeRoutedAlias.getRouteField() + "=" + routeTimestamp);
-  }
-
-  // Note: the following rule is tempting but not necessary and is not 
compatible with
-  // only using this URP when the alias distrib phase is NONE; 
otherwise a doc may be routed to from a non-recent
-  // collection to the most recent only to then go there directly 
instead of realizing a new collection is needed.
-  //  // If it's going to some other collection (not "this") then 
break to just send it there
-  //  if (!thisCollection.equals(targetCollection)) {
-  //break;
-  //  }
-  // Also tempting but not compatible:  check that we're the leader, 
if not then break
-
-  // If the doc goes to the most recent collection then do some checks 
below, otherwise break the loop.
-  final Instant mostRecentCollTimestamp = 
parsedCollectionsDesc.get(0).getKey();
-  final String mostRecentCollName = 
parsedCollectionsDesc.get(0).getValue();
-  if (!mostRecentCollName.equals(targetCollection)) {
-break;
-  }
-
-  // Check the doc isn't too far in the future
-  final Instant maxFutureTime = 
Instant.now().plusMillis(timeRoutedAlias.getMaxFutureMs());
-  if (routeTimestamp.isAfter(maxFutureTime)) {
-throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-"The document's time routed key of " + routeValue + " is too 
far in the future given " +
-TimeRoutedAlias.ROUTER_MAX_FUTURE + "=" + 
timeRoutedAlias.getMaxFutureMs());
-  }
-
-  // Create a new collection?
-  final Instant nextCollTimestamp = 
timeRoutedAlias.computeNextCollTimestamp(mostRecentCollTimestamp);
-  if (routeTimestamp.isBefore(nextCollTimestamp)) {
-break; // thus we don't need another collection
-  }
-
-  createCollectionAfter(mostRecentCollName); // *should* throw if 
fails for some reason but...
-  final boolean updated = updateParsedCollectionAliases();
-  if (!updated) { // thus we didn't make progress...
-// this is not expected, even in known failure cases, but we check 
just in case
-throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-"We need to create a new time routed collection but for 
unknown reasons were unable to do so.");
-  }
-  // then retry the loop ...
-} while(true);
-assert targetCollection != null;
-
+String candidateCollection = 
findCandidateCollectionGivenTimestamp(docTimestampToRoute, 
cmd.getPrintableId());
--- End diff --

+1


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-28 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r213405495
  
--- Diff: 
solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
 ---
@@ -322,6 +325,104 @@ public void testSliceRouting() throws Exception {
 }
   }
 
+  @Test
+  public void testPreemptiveCreation() throws Exception {
+String configName = TimeRoutedAliasUpdateProcessorTest.configName + 
getTestName();
+createConfigSet(configName);
+
+final int numShards = 1 ;
+final int numReplicas = 1 ;
+CollectionAdminRequest.createTimeRoutedAlias(alias, 
"2017-10-23T00:00:00Z", "+1DAY", timeField,
+CollectionAdminRequest.createCollection("_unused_", configName, 
numShards, numReplicas)
+
.setMaxShardsPerNode(numReplicas)).setPreemptiveCreateWindow("3HOUR")
+.process(solrClient);
+
+// cause some collections to be created
+assertUpdateResponse(solrClient.add(alias,
+sdoc("id","1","timestamp_dt", "2017-10-25T00:00:00Z")
--- End diff --

comments (now below) address the purpose of each collection


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-28 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r212805841
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -167,59 +167,17 @@ private String getAliasName() {
   public void processAdd(AddUpdateCommand cmd) throws IOException {
 SolrInputDocument solrInputDocument = cmd.getSolrInputDocument();
 final Object routeValue = 
solrInputDocument.getFieldValue(timeRoutedAlias.getRouteField());
-final Instant routeTimestamp = parseRouteKey(routeValue);
-
+final Instant docTimestampToRoute = parseRouteKey(routeValue);
--- End diff --

+1


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-28 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r213397248
  
--- 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 --

+1


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-28 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r213398526
  
--- 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) {
+final Instant mostRecentCollTimestamp = 
parsedCollectionsDesc.get(0).getKey();
+final Instant nextCollTimestamp = 
timeRoutedAlias.computeNextCollTimestamp(mostRecentCollTimestamp);
+if (!routeTimestamp.isBefore(nextCollTimestamp)) {
+  // current document is destined for a collection that doesn't exist, 
must create the destination
+  // to proceed with this add command
+  return SYNCHRONOUS;
+}
+
+if (isBlank(preemptiveCreateMath)) {
--- End diff --
   

[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-28 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r213415931
  
--- Diff: 
solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
 ---
@@ -322,6 +325,104 @@ public void testSliceRouting() throws Exception {
 }
   }
 
+  @Test
+  public void testPreemptiveCreation() throws Exception {
+String configName = TimeRoutedAliasUpdateProcessorTest.configName + 
getTestName();
+createConfigSet(configName);
+
+final int numShards = 1 ;
+final int numReplicas = 1 ;
+CollectionAdminRequest.createTimeRoutedAlias(alias, 
"2017-10-23T00:00:00Z", "+1DAY", timeField,
+CollectionAdminRequest.createCollection("_unused_", configName, 
numShards, numReplicas)
+
.setMaxShardsPerNode(numReplicas)).setPreemptiveCreateWindow("3HOUR")
+.process(solrClient);
+
+// cause some collections to be created
+assertUpdateResponse(solrClient.add(alias,
+sdoc("id","1","timestamp_dt", "2017-10-25T00:00:00Z")
+));
+assertUpdateResponse(solrClient.commit(alias));
+
+// wait for all the collections to exist...
+waitCol("2017-10-23", numShards);
+waitCol("2017-10-24", numShards);
+waitCol("2017-10-25", numShards);
+
+// normal update, nothing special, no collection creation required.
+List cols = new 
CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
+assertEquals(3,cols.size());
+
+assertNumDocs("2017-10-23", 0);
+assertNumDocs("2017-10-24", 0);
+assertNumDocs("2017-10-25", 1);
+
+// cause some collections to be created
+
+ModifiableSolrParams params = params();
+assertUpdateResponse(add(alias, Arrays.asList(
+sdoc("id", "2", "timestamp_dt", "2017-10-24T00:00:00Z"),
+sdoc("id", "3", "timestamp_dt", "2017-10-25T00:00:00Z"),
+sdoc("id", "4", "timestamp_dt", "2017-10-23T00:00:00Z"),
+sdoc("id", "5", "timestamp_dt", "2017-10-25T23:00:00Z")), // 
should cause preemptive creation
--- End diff --

I think this comment is obsolete, carried over from earlier versions?


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-28 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r213403086
  
--- Diff: 
solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java 
---
@@ -141,6 +145,9 @@ public TimeRoutedAlias(String aliasName, Map aliasMetadata) {
 
 //optional:
 maxFutureMs = params.getLong(ROUTER_MAX_FUTURE, 
TimeUnit.MINUTES.toMillis(10));
+// the date math configured is an interval to be subtracted from the 
most recent collection's time stamp
+preemptiveCreateMath = params.get(ROUTER_PREEMPTIVE_CREATE_MATH) != 
null ?
--- End diff --

ok


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-28 Thread nsoft
Github user nsoft commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r212805827
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -94,13 +92,15 @@
   private final SolrCmdDistributor cmdDistrib;
   private final CollectionsHandler collHandler;
   private final SolrParams outParamsToLeader;
+  @SuppressWarnings("FieldCanBeLocal")
   private final CloudDescriptor cloudDesc;
 
   private List> parsedCollectionsDesc; // 
k=timestamp (start), v=collection.  Sorted descending
   private Aliases parsedCollectionsAliases; // a cached reference to the 
source of what we parse into parsedCollectionsDesc
   private SolrQueryRequest req;
+  private ExecutorService preemptiveCreationExecutor;
--- End diff --

No that's a good point


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-25 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r212809165
  
--- Diff: 
solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
 ---
@@ -392,29 +393,73 @@ public void testPreemptiveCreation() throws Exception 
{
 CollectionAdminRequest.setAliasProperty(alias)
 .addProperty(TimeRoutedAlias.ROUTER_PREEMPTIVE_CREATE_MATH, 
"3DAY").process(solrClient);
 
-Thread.sleep(1000); // a moment to be sure the alias change has taken 
effect
-
 assertUpdateResponse(add(alias, Collections.singletonList(
 sdoc("id", "7", "timestamp_dt", "2017-10-25T23:01:00Z")), // 
should cause preemptive creation now
 params));
 assertUpdateResponse(solrClient.commit(alias));
 waitCol("2017-10-27", numShards);
-waitCol("2017-10-28", numShards);
 
 cols = new 
CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
-assertEquals(6,cols.size());
+assertEquals(5,cols.size()); // only one created in async case
 assertNumDocs("2017-10-23", 1);
 assertNumDocs("2017-10-24", 1);
 assertNumDocs("2017-10-25", 5);
 assertNumDocs("2017-10-26", 0);
 assertNumDocs("2017-10-27", 0);
+
+assertUpdateResponse(add(alias, Collections.singletonList(
+sdoc("id", "8", "timestamp_dt", "2017-10-25T23:01:00Z")), // 
should cause preemptive creation now
+params));
+assertUpdateResponse(solrClient.commit(alias));
+waitCol("2017-10-27", numShards);
+waitCol("2017-10-28", numShards);
+
+cols = new 
CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
+assertEquals(6,cols.size()); // Subsequent documents continue to 
create up to limit
+assertNumDocs("2017-10-23", 1);
+assertNumDocs("2017-10-24", 1);
+assertNumDocs("2017-10-25", 6);
+assertNumDocs("2017-10-26", 0);
+assertNumDocs("2017-10-27", 0);
 assertNumDocs("2017-10-28", 0);
 
 QueryResponse resp;
 resp = solrClient.query(alias, params(
 "q", "*:*",
 "rows", "10"));
-assertEquals(7, resp.getResults().getNumFound());
+assertEquals(8, resp.getResults().getNumFound());
+
+assertUpdateResponse(add(alias, Arrays.asList(
--- End diff --

Shouldn't we use `addDocsAndCommit` here and the other spots where multiple 
docs are added at a time?  I know you're passing params but it's empty.

Otherwise, tests look good!


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-25 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r212802626
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -94,13 +92,15 @@
   private final SolrCmdDistributor cmdDistrib;
   private final CollectionsHandler collHandler;
   private final SolrParams outParamsToLeader;
+  @SuppressWarnings("FieldCanBeLocal")
   private final CloudDescriptor cloudDesc;
 
   private List> parsedCollectionsDesc; // 
k=timestamp (start), v=collection.  Sorted descending
   private Aliases parsedCollectionsAliases; // a cached reference to the 
source of what we parse into parsedCollectionsDesc
   private SolrQueryRequest req;
+  private ExecutorService preemptiveCreationExecutor;
--- End diff --

Since it will be nulled out in another thread, we ought to declare this as 
volatile.  I know this is being a bit pedantic since I don't think it'd be a 
real problem.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-25 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r212802141
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -167,59 +167,17 @@ private String getAliasName() {
   public void processAdd(AddUpdateCommand cmd) throws IOException {
 SolrInputDocument solrInputDocument = cmd.getSolrInputDocument();
 final Object routeValue = 
solrInputDocument.getFieldValue(timeRoutedAlias.getRouteField());
-final Instant routeTimestamp = parseRouteKey(routeValue);
-
+final Instant docTimestampToRoute = parseRouteKey(routeValue);
 updateParsedCollectionAliases();
-String targetCollection;
-do { // typically we don't loop; it's only when we need to create a 
collection
-  targetCollection = 
findTargetCollectionGivenTimestamp(routeTimestamp);
-
-  if (targetCollection == null) {
-throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-"Doc " + cmd.getPrintableId() + " couldn't be routed with " + 
timeRoutedAlias.getRouteField() + "=" + routeTimestamp);
-  }
-
-  // Note: the following rule is tempting but not necessary and is not 
compatible with
-  // only using this URP when the alias distrib phase is NONE; 
otherwise a doc may be routed to from a non-recent
-  // collection to the most recent only to then go there directly 
instead of realizing a new collection is needed.
-  //  // If it's going to some other collection (not "this") then 
break to just send it there
-  //  if (!thisCollection.equals(targetCollection)) {
-  //break;
-  //  }
-  // Also tempting but not compatible:  check that we're the leader, 
if not then break
-
-  // If the doc goes to the most recent collection then do some checks 
below, otherwise break the loop.
-  final Instant mostRecentCollTimestamp = 
parsedCollectionsDesc.get(0).getKey();
-  final String mostRecentCollName = 
parsedCollectionsDesc.get(0).getValue();
-  if (!mostRecentCollName.equals(targetCollection)) {
-break;
-  }
-
-  // Check the doc isn't too far in the future
-  final Instant maxFutureTime = 
Instant.now().plusMillis(timeRoutedAlias.getMaxFutureMs());
-  if (routeTimestamp.isAfter(maxFutureTime)) {
-throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-"The document's time routed key of " + routeValue + " is too 
far in the future given " +
-TimeRoutedAlias.ROUTER_MAX_FUTURE + "=" + 
timeRoutedAlias.getMaxFutureMs());
-  }
-
-  // Create a new collection?
-  final Instant nextCollTimestamp = 
timeRoutedAlias.computeNextCollTimestamp(mostRecentCollTimestamp);
-  if (routeTimestamp.isBefore(nextCollTimestamp)) {
-break; // thus we don't need another collection
-  }
-
-  createCollectionAfter(mostRecentCollName); // *should* throw if 
fails for some reason but...
-  final boolean updated = updateParsedCollectionAliases();
-  if (!updated) { // thus we didn't make progress...
-// this is not expected, even in known failure cases, but we check 
just in case
-throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-"We need to create a new time routed collection but for 
unknown reasons were unable to do so.");
-  }
-  // then retry the loop ...
-} while(true);
-assert targetCollection != null;
-
+String candidateCollection = 
findCandidateCollectionGivenTimestamp(docTimestampToRoute, 
cmd.getPrintableId());
--- End diff --

You can move this line to immediately before its first use of the result.  
Presently, the maxFutureTime check is inbetween which breaks up the natural 
flow.
Hmm; even the "updateParsedCollectionAliases()" call can move down.
Finally, some newlines here & there would help separate separate steps.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-25 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r212802993
  
--- 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) {
+final Instant mostRecentCollTimestamp = 
parsedCollectionsDesc.get(0).getKey();
+final Instant nextCollTimestamp = 
timeRoutedAlias.computeNextCollTimestamp(mostRecentCollTimestamp);
+if (!routeTimestamp.isBefore(nextCollTimestamp)) {
+  // current document is destined for a collection that doesn't exist, 
must create the destination
+  // to proceed with this add command
+  return SYNCHRONOUS;
+}
+
+if (isBlank(preemptiveCreateMath)) {
--- End diff --
 

[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-25 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r212802686
  
--- 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(() -> {
--- End diff --

the code executed in the new thread should not call maintain() since I see 
you had to make maintain more complicated to tell if it's being called from 
"async".  It can call: 
```
  final String mostRecentCollName = 
parsedCollectionsDesc.get(0).getValue();
  createCollectionAfter(mostRecentCollName);
```


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-24 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r212675378
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -405,4 +435,54 @@ protected void doClose() {
 collection, slice.getName(), 
DistributedUpdateProcessor.MAX_RETRIES_ON_FORWARD_DEAULT);
   }
 
+
+  /**
+   * Create as many collections as required. This method loops to allow 
for the possibility that the routeTimestamp
--- End diff --

Minor: `routeTimestamp` is now known as `docTimestamp` based on your edits.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-24 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r212680119
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -167,59 +167,17 @@ private String getAliasName() {
   public void processAdd(AddUpdateCommand cmd) throws IOException {
 SolrInputDocument solrInputDocument = cmd.getSolrInputDocument();
 final Object routeValue = 
solrInputDocument.getFieldValue(timeRoutedAlias.getRouteField());
-final Instant routeTimestamp = parseRouteKey(routeValue);
-
+final Instant docTimestampToRoute = parseRouteKey(routeValue);
--- End diff --

elsewhere I notice you used "docTimestamp" which seems better to me than 
this long name.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-24 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r212677139
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -405,4 +435,54 @@ protected void doClose() {
 collection, slice.getName(), 
DistributedUpdateProcessor.MAX_RETRIES_ON_FORWARD_DEAULT);
   }
 
+
+  /**
+   * Create as many collections as required. This method loops to allow 
for the possibility that the routeTimestamp
+   * requires more than one collection to be created. Since multiple 
threads may be invoking maintain on separate
+   * requests to the same alias, we must pass in the name of the 
collection that this thread believes to be the most
+   * recent collection. This assumption is checked when the command is 
executed in the overseer. When this method
+   * finds that all collections required have been created it returns the 
(possibly new) most recent collection.
+   * The return value is ignored by the calling code in the async 
preemptive case.
+   *
+   * @param targetCollection the initial notion of the latest collection 
available.
+   * @param docTimestamp the timestamp from the document that determines 
routing
+   * @param printableId an identifier for the add command used in error 
messages
+   * @return The latest collection, including collections created during 
maintenance
+   */
+  public String maintain(String targetCollection, Instant docTimestamp, 
String printableId, boolean asyncSinglePassOnly) {
+do { // typically we don't loop; it's only when we need to create a 
collection
+
+  // Note: This code no longer short circuits immediately when it sees 
that the expected latest
--- End diff --

I don't think source code should refer to what the code used to do.  You 
could communicate this in GitHub/JIRA.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-24 Thread dsmiley
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: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-08 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r208719459
  
--- Diff: 
solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
 ---
@@ -343,8 +444,31 @@ public void testSliceRouting() throws Exception {
   }
 
   private void waitCol(final String datePart, int slices) {
-waitForState("waiting for collections to be created",alias + "_" + 
datePart,
-(liveNodes, collectionState) -> 
collectionState.getActiveSlices().size() == slices);
+// collection to exist
+String collection = alias + "_" + datePart;
+waitForState("waiting for collections to be created", collection,
+(liveNodes, collectionState) -> {
+  if (collectionState == null) {
+// per predicate javadoc, this is what we get if the 
collection doesn't exist at all.
+return false;
+  }
+  Collection activeSlices = 
collectionState.getActiveSlices();
+  int size = activeSlices.size();
+  return size == slices;
+});
+// and alias to be aware of collection
+long start = System.nanoTime(); // mumble mumble precommit mumble 
mumble...
+while 
(!cluster.getSolrClient().getZkStateReader().getAliases().getCollectionAliasListMap().get(alias).contains(collection))
 {
+  if (NANOSECONDS.toSeconds(System.nanoTime() - start) > 10) {
+fail("took over 10 seconds after collection creation to update 
aliases");
+  } else {
+try {
+  Thread.sleep(500);
+} catch (InterruptedException e) {
+  e.printStackTrace();
--- End diff --

-- and return or throw.  Interrupt means stop what you're doing.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-08 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r208716790
  
--- Diff: 
solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java 
---
@@ -141,6 +145,9 @@ public TimeRoutedAlias(String aliasName, Map aliasMetadata) {
 
 //optional:
 maxFutureMs = params.getLong(ROUTER_MAX_FUTURE, 
TimeUnit.MINUTES.toMillis(10));
+// the date math configured is an interval to be subtracted from the 
most recent collection's time stamp
+preemptiveCreateMath = params.get(ROUTER_PREEMPTIVE_CREATE_MATH) != 
null ?
--- End diff --

Perhaps this should detect if it already starts with a plus or minus and if 
so removes it, thus making it more friendly to people who like to include the 
operator?


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-08 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r208717805
  
--- Diff: 
solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
 ---
@@ -322,6 +325,104 @@ public void testSliceRouting() throws Exception {
 }
   }
 
+  @Test
+  public void testPreemptiveCreation() throws Exception {
+String configName = TimeRoutedAliasUpdateProcessorTest.configName + 
getTestName();
+createConfigSet(configName);
+
+final int numShards = 1 ;
+final int numReplicas = 1 ;
+CollectionAdminRequest.createTimeRoutedAlias(alias, 
"2017-10-23T00:00:00Z", "+1DAY", timeField,
+CollectionAdminRequest.createCollection("_unused_", configName, 
numShards, numReplicas)
+
.setMaxShardsPerNode(numReplicas)).setPreemptiveCreateWindow("3HOUR")
+.process(solrClient);
+
+// cause some collections to be created
+assertUpdateResponse(solrClient.add(alias,
+sdoc("id","1","timestamp_dt", "2017-10-25T00:00:00Z")
--- End diff --

To reduce the overhead of the test, can we start one one collection, maybe 
two, before getting into pre-emption?
And this test should very likely be annotated with \@Slow


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-08 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r208720134
  
--- Diff: 
solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
 ---
@@ -322,6 +325,104 @@ public void testSliceRouting() throws Exception {
 }
   }
 
+  @Test
+  public void testPreemptiveCreation() throws Exception {
+String configName = TimeRoutedAliasUpdateProcessorTest.configName + 
getTestName();
+createConfigSet(configName);
+
+final int numShards = 1 ;
+final int numReplicas = 1 ;
+CollectionAdminRequest.createTimeRoutedAlias(alias, 
"2017-10-23T00:00:00Z", "+1DAY", timeField,
+CollectionAdminRequest.createCollection("_unused_", configName, 
numShards, numReplicas)
+
.setMaxShardsPerNode(numReplicas)).setPreemptiveCreateWindow("3HOUR")
+.process(solrClient);
+
+// cause some collections to be created
+assertUpdateResponse(solrClient.add(alias,
+sdoc("id","1","timestamp_dt", "2017-10-25T00:00:00Z")
+));
+assertUpdateResponse(solrClient.commit(alias));
+
+// wait for all the collections to exist...
+waitCol("2017-10-23", numShards);
+waitCol("2017-10-24", numShards);
+waitCol("2017-10-25", numShards);
+
+// normal update, nothing special, no collection creation required.
+List cols = new 
CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
+assertEquals(3,cols.size());
+
+assertNumDocs("2017-10-23", 0);
+assertNumDocs("2017-10-24", 0);
+assertNumDocs("2017-10-25", 1);
+
+// cause some collections to be created
+
+ModifiableSolrParams params = params();
+assertUpdateResponse(add(alias, Arrays.asList(
+sdoc("id", "2", "timestamp_dt", "2017-10-24T00:00:00Z"),
+sdoc("id", "3", "timestamp_dt", "2017-10-25T00:00:00Z"),
+sdoc("id", "4", "timestamp_dt", "2017-10-23T00:00:00Z"),
+sdoc("id", "5", "timestamp_dt", "2017-10-25T23:00:00Z")), // 
should cause preemptive creation
--- End diff --

lets also add another document after this, one that would also cause 
pre-emptive creation if it weren't for there already being a request in-progress


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-08 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r208718224
  
--- Diff: 
solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
 ---
@@ -322,6 +325,104 @@ public void testSliceRouting() throws Exception {
 }
   }
 
+  @Test
+  public void testPreemptiveCreation() throws Exception {
+String configName = TimeRoutedAliasUpdateProcessorTest.configName + 
getTestName();
+createConfigSet(configName);
+
+final int numShards = 1 ;
+final int numReplicas = 1 ;
+CollectionAdminRequest.createTimeRoutedAlias(alias, 
"2017-10-23T00:00:00Z", "+1DAY", timeField,
+CollectionAdminRequest.createCollection("_unused_", configName, 
numShards, numReplicas)
+
.setMaxShardsPerNode(numReplicas)).setPreemptiveCreateWindow("3HOUR")
+.process(solrClient);
+
+// cause some collections to be created
+assertUpdateResponse(solrClient.add(alias,
+sdoc("id","1","timestamp_dt", "2017-10-25T00:00:00Z")
+));
+assertUpdateResponse(solrClient.commit(alias));
+
+// wait for all the collections to exist...
+waitCol("2017-10-23", numShards);
+waitCol("2017-10-24", numShards);
+waitCol("2017-10-25", numShards);
+
+// normal update, nothing special, no collection creation required.
+List cols = new 
CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
+assertEquals(3,cols.size());
+
+assertNumDocs("2017-10-23", 0);
+assertNumDocs("2017-10-24", 0);
+assertNumDocs("2017-10-25", 1);
+
+// cause some collections to be created
+
+ModifiableSolrParams params = params();
+assertUpdateResponse(add(alias, Arrays.asList(
+sdoc("id", "2", "timestamp_dt", "2017-10-24T00:00:00Z"),
+sdoc("id", "3", "timestamp_dt", "2017-10-25T00:00:00Z"),
+sdoc("id", "4", "timestamp_dt", "2017-10-23T00:00:00Z"),
+sdoc("id", "5", "timestamp_dt", "2017-10-25T23:00:00Z")), // 
should cause preemptive creation
+params));
+assertUpdateResponse(solrClient.commit(alias));
+
+cols = new 
CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
+assertEquals(3,cols.size());
+assertNumDocs("2017-10-23", 1);
+assertNumDocs("2017-10-24", 1);
+assertNumDocs("2017-10-25", 3);
+
+assertUpdateResponse(add(alias, Collections.singletonList(
+sdoc("id", "6", "timestamp_dt", "2017-10-25T23:01:00Z")), // might 
cause duplicate preemptive creation
+params));
+assertUpdateResponse(solrClient.commit(alias));
+
+waitCol("2017-10-26", numShards);
+cols = new 
CollectionAdminRequest.ListAliases().process(solrClient).getAliasesAsLists().get(alias);
+assertEquals(4,cols.size());
+assertNumDocs("2017-10-23", 1);
+assertNumDocs("2017-10-24", 1);
+assertNumDocs("2017-10-25", 4);
+assertNumDocs("2017-10-26", 0);
+
+// now test with pre-create window longer than time slice, and forcing 
multiple creations.
+CollectionAdminRequest.setAliasProperty(alias)
+.addProperty(TimeRoutedAlias.ROUTER_PREEMPTIVE_CREATE_MATH, 
"3DAY").process(solrClient);
+
+Thread.sleep(1000); // a moment to be sure the alias change has taken 
effect
--- End diff --

curious; has this been shown to be necessary?  I can imagine it might be... 
since a new TRA URP isn't doing to do a ZK sync to view the alias state.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-07 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r208446107
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -167,66 +173,121 @@ private String getAliasName() {
   public void processAdd(AddUpdateCommand cmd) throws IOException {
 SolrInputDocument solrInputDocument = cmd.getSolrInputDocument();
 final Object routeValue = 
solrInputDocument.getFieldValue(timeRoutedAlias.getRouteField());
-final Instant routeTimestamp = parseRouteKey(routeValue);
-
+final Instant docTimestampToRoute = parseRouteKey(routeValue);
 updateParsedCollectionAliases();
-String targetCollection;
-do { // typically we don't loop; it's only when we need to create a 
collection
-  targetCollection = 
findTargetCollectionGivenTimestamp(routeTimestamp);
+String candidateCollection = 
findCandidateCollectionGivenTimestamp(docTimestampToRoute, 
cmd.getPrintableId());
+String targetCollection = 
createCollectionsIfRequired(docTimestampToRoute, candidateCollection, 
cmd.getPrintableId());
+if (thisCollection.equals(targetCollection)) {
+  // pass on through; we've reached the right collection
+  super.processAdd(cmd);
+} else {
+  // send to the right collection
+  SolrCmdDistributor.Node targetLeaderNode = 
routeDocToSlice(targetCollection, solrInputDocument);
+  cmdDistrib.distribAdd(cmd, 
Collections.singletonList(targetLeaderNode), new 
ModifiableSolrParams(outParamsToLeader));
+}
+  }
 
-  if (targetCollection == null) {
-throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-"Doc " + cmd.getPrintableId() + " couldn't be routed with " + 
timeRoutedAlias.getRouteField() + "=" + routeTimestamp);
+  private String createCollectionsIfRequired(Instant docTimestamp, String 
targetCollection, String printableId) {
+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);
+case ASYNC_PREEMPTIVE:
+  // Note: creating an executor is slightly expensive, but only 
likely to happen once per hour/day/week
+  // (depending on time slice size for the TRA). Executor is used 
to ensure we pick up the MDC logging stuff
+  // from ExecutorUtil. 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, 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 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.
+  synchronized (execLock) {
+if (preemptiveCreationExecutor == null) {
+  preemptiveCreationExecutor = preemptiveCreationExecutor();
+}
+preemptiveCreationExecutor.execute(() -> {
+  maintain(targetCollection, docTimestamp, printableId);
+  preemptiveCreationExecutor = null;
+});
+preemptiveCreationExecutor.shutdown(); // shutdown immediately 
to ensure no new requests accepted
--- End diff --

Hmmm; I think we can shut down in finish() if it's non-null.  Wouldn't we 
want to keep this active for hypothetical more pre-emptive creation during the 
lifespan of this URP instance (however extremely unlikely that is I totally 
grant you).  I think it'd be easier to reason about.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-07 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r208449633
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -405,4 +449,58 @@ protected void doClose() {
 collection, slice.getName());
   }
 
+
+  /**
+   * Create as many collections as required. This method loops to allow 
for the possibility that the routeTimestamp
+   * requires more than one collection to be created. Since multiple 
threads may be invoking maintain on separate
+   * requests to the same alias, we must pass in the name of the 
collection that this thread believes to be the most
+   * recent collection. This assumption is checked when the command is 
executed in the overseer. When this method
+   * finds that all collections required have been created it returns the 
(possibly new) most recent collection.
+   * The return value is ignored by the calling code in the async 
preemptive case.
+   *
+   * @param targetCollection the initial notion of the latest collection 
available.
+   * @param docTimestamp the timestamp from the document that determines 
routing
+   * @param printableId an identifier for the add command used in error 
messages
+   * @return The latest collection, including collections created during 
maintenance
+   */
+  public String maintain(String targetCollection, Instant docTimestamp, 
String printableId) {
+do { // typically we don't loop; it's only when we need to create a 
collection
+
+  // Note: This code no longer short circuits immediately when it sees 
that the expected latest
+  // collection is the current latest collection. With the advent of 
preemptive collection creation
+  // we always need to do the time based checks. Otherwise, we cannot 
handle the case where the
+  // preemptive window is larger than our TRA's time slices
+
+  // Check the doc isn't too far in the future
+  // TODO: Instant.now() here seems wrong...
+  final Instant maxFutureTime = 
Instant.now().plusMillis(timeRoutedAlias.getMaxFutureMs());
--- End diff --

I think we can move this above the loop or maybe even super up-front.  It's 
documented in the ref guide and isn't tied to being associated with async/sync 
or near the edge.  If someone doesn't want this safety check they can 
initialize it in a way as to have no practical effect (and sure, ought to be 
more easily disabled by using say null or -1).


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-07 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r208448157
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -167,66 +173,121 @@ private String getAliasName() {
   public void processAdd(AddUpdateCommand cmd) throws IOException {
 SolrInputDocument solrInputDocument = cmd.getSolrInputDocument();
 final Object routeValue = 
solrInputDocument.getFieldValue(timeRoutedAlias.getRouteField());
-final Instant routeTimestamp = parseRouteKey(routeValue);
-
+final Instant docTimestampToRoute = parseRouteKey(routeValue);
 updateParsedCollectionAliases();
-String targetCollection;
-do { // typically we don't loop; it's only when we need to create a 
collection
-  targetCollection = 
findTargetCollectionGivenTimestamp(routeTimestamp);
+String candidateCollection = 
findCandidateCollectionGivenTimestamp(docTimestampToRoute, 
cmd.getPrintableId());
+String targetCollection = 
createCollectionsIfRequired(docTimestampToRoute, candidateCollection, 
cmd.getPrintableId());
+if (thisCollection.equals(targetCollection)) {
+  // pass on through; we've reached the right collection
+  super.processAdd(cmd);
+} else {
+  // send to the right collection
+  SolrCmdDistributor.Node targetLeaderNode = 
routeDocToSlice(targetCollection, solrInputDocument);
+  cmdDistrib.distribAdd(cmd, 
Collections.singletonList(targetLeaderNode), new 
ModifiableSolrParams(outParamsToLeader));
+}
+  }
 
-  if (targetCollection == null) {
-throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-"Doc " + cmd.getPrintableId() + " couldn't be routed with " + 
timeRoutedAlias.getRouteField() + "=" + routeTimestamp);
+  private String createCollectionsIfRequired(Instant docTimestamp, String 
targetCollection, String printableId) {
+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);
+case ASYNC_PREEMPTIVE:
+  // Note: creating an executor is slightly expensive, but only 
likely to happen once per hour/day/week
+  // (depending on time slice size for the TRA). Executor is used 
to ensure we pick up the MDC logging stuff
+  // from ExecutorUtil. 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, 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 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.
+  synchronized (execLock) {
+if (preemptiveCreationExecutor == null) {
+  preemptiveCreationExecutor = preemptiveCreationExecutor();
+}
+preemptiveCreationExecutor.execute(() -> {
+  maintain(targetCollection, docTimestamp, printableId);
+  preemptiveCreationExecutor = null;
+});
+preemptiveCreationExecutor.shutdown(); // shutdown immediately 
to ensure no new requests accepted
+  }
+  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);
+}
+  }
 
-  // Note: the following rule is tempting but not necessary and is not 
compatible with
-  // only using this URP when the alias distrib phase is NONE; 
otherwise a doc may be routed to from a non-recent
-  // collection to the most recent only to then go there 

[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-07 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r208445252
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -167,66 +173,121 @@ private String getAliasName() {
   public void processAdd(AddUpdateCommand cmd) throws IOException {
 SolrInputDocument solrInputDocument = cmd.getSolrInputDocument();
 final Object routeValue = 
solrInputDocument.getFieldValue(timeRoutedAlias.getRouteField());
-final Instant routeTimestamp = parseRouteKey(routeValue);
-
+final Instant docTimestampToRoute = parseRouteKey(routeValue);
 updateParsedCollectionAliases();
-String targetCollection;
-do { // typically we don't loop; it's only when we need to create a 
collection
-  targetCollection = 
findTargetCollectionGivenTimestamp(routeTimestamp);
+String candidateCollection = 
findCandidateCollectionGivenTimestamp(docTimestampToRoute, 
cmd.getPrintableId());
+String targetCollection = 
createCollectionsIfRequired(docTimestampToRoute, candidateCollection, 
cmd.getPrintableId());
+if (thisCollection.equals(targetCollection)) {
+  // pass on through; we've reached the right collection
+  super.processAdd(cmd);
+} else {
+  // send to the right collection
+  SolrCmdDistributor.Node targetLeaderNode = 
routeDocToSlice(targetCollection, solrInputDocument);
+  cmdDistrib.distribAdd(cmd, 
Collections.singletonList(targetLeaderNode), new 
ModifiableSolrParams(outParamsToLeader));
+}
+  }
 
-  if (targetCollection == null) {
-throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-"Doc " + cmd.getPrintableId() + " couldn't be routed with " + 
timeRoutedAlias.getRouteField() + "=" + routeTimestamp);
+  private String createCollectionsIfRequired(Instant docTimestamp, String 
targetCollection, String printableId) {
+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);
+case ASYNC_PREEMPTIVE:
+  // Note: creating an executor is slightly expensive, but only 
likely to happen once per hour/day/week
--- End diff --

ok but you can place this comment maybe somewhere higher as it applies to 
both sync & pre-emptive.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-07 Thread dsmiley
Github user dsmiley commented on a diff in the pull request:

https://github.com/apache/lucene-solr/pull/433#discussion_r208445857
  
--- Diff: 
solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
 ---
@@ -167,66 +173,121 @@ private String getAliasName() {
   public void processAdd(AddUpdateCommand cmd) throws IOException {
 SolrInputDocument solrInputDocument = cmd.getSolrInputDocument();
 final Object routeValue = 
solrInputDocument.getFieldValue(timeRoutedAlias.getRouteField());
-final Instant routeTimestamp = parseRouteKey(routeValue);
-
+final Instant docTimestampToRoute = parseRouteKey(routeValue);
 updateParsedCollectionAliases();
-String targetCollection;
-do { // typically we don't loop; it's only when we need to create a 
collection
-  targetCollection = 
findTargetCollectionGivenTimestamp(routeTimestamp);
+String candidateCollection = 
findCandidateCollectionGivenTimestamp(docTimestampToRoute, 
cmd.getPrintableId());
+String targetCollection = 
createCollectionsIfRequired(docTimestampToRoute, candidateCollection, 
cmd.getPrintableId());
+if (thisCollection.equals(targetCollection)) {
+  // pass on through; we've reached the right collection
+  super.processAdd(cmd);
+} else {
+  // send to the right collection
+  SolrCmdDistributor.Node targetLeaderNode = 
routeDocToSlice(targetCollection, solrInputDocument);
+  cmdDistrib.distribAdd(cmd, 
Collections.singletonList(targetLeaderNode), new 
ModifiableSolrParams(outParamsToLeader));
+}
+  }
 
-  if (targetCollection == null) {
-throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-"Doc " + cmd.getPrintableId() + " couldn't be routed with " + 
timeRoutedAlias.getRouteField() + "=" + routeTimestamp);
+  private String createCollectionsIfRequired(Instant docTimestamp, String 
targetCollection, String printableId) {
+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);
+case ASYNC_PREEMPTIVE:
+  // Note: creating an executor is slightly expensive, but only 
likely to happen once per hour/day/week
+  // (depending on time slice size for the TRA). Executor is used 
to ensure we pick up the MDC logging stuff
+  // from ExecutorUtil. 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, 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 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.
+  synchronized (execLock) {
--- End diff --

Why have a synchronize here?  URPs are single threaded and what we're 
locking on is local to this instance.


---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org



[GitHub] lucene-solr pull request #433: SOLR-12357 Premptive creation of collections ...

2018-08-07 Thread nsoft
GitHub user nsoft opened a pull request:

https://github.com/apache/lucene-solr/pull/433

SOLR-12357 Premptive creation of collections in Time Routed Aliases

new PR to clean up the strange merge conflicts

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

$ git pull https://github.com/nsoft/lucene-solr 12357-2

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

https://github.com/apache/lucene-solr/pull/433.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 #433


commit e4668036631adc7ecd094c6fd6b588d0f457ae1f
Author: Gus Heck 
Date:   2018-07-19T03:29:14Z

SOLR-12357 Premptively create collections in time routed aliases

(cherry picked from commit dcdc7c7)

commit c9f9872f79b84705daeb24ed6994c057d7d03de7
Author: Gus Heck 
Date:   2018-07-28T20:54:06Z

SOLR-12357 - Review Feedback, properly handle a preemptive
creation interval that happens to be longer than the time
slices for the TRA, further cleanup/organization.

(cherry picked from commit d07fcb4)

commit 613f8d50c1f1267c89152c140678e5243119bff1
Author: Gus Heck 
Date:   2018-08-08T01:08:24Z

SOLR-12357 - Simplified, removing static fields based on the realization
that the code path is rare and the case they guarded against is already
well known as an anti-pattern. Also moved some validation into the
TimeRoutedAlias class.




---

-
To unsubscribe, e-mail: dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: dev-h...@lucene.apache.org