You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/05/14 23:48:46 UTC

[GitHub] [lucene-solr] megancarey commented on a change in pull request #1504: SOLR-14462: cache more than one autoscaling session

megancarey commented on a change in pull request #1504:
URL: https://github.com/apache/lucene-solr/pull/1504#discussion_r425424095



##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
##########
@@ -382,45 +383,55 @@ static MapWriter loggingInfo(Policy policy, SolrCloudManager cloudManager, Sugge
   }
 
   public enum Status {
-    NULL,
-    //it is just created and not yet used or all operations on it has been completed fully
-    UNUSED,
-    COMPUTING, EXECUTING
+    COMPUTING, // A command is actively using and modifying the session to compute placements
+    EXECUTING // A command is not done yet processing its changes but no longer uses the session
   }
 
   /**
-   * This class stores a session for sharing purpose. If a process creates a session to
-   * compute operations,
-   * 1) see if there is a session that is available in the cache,
-   * 2) if yes, check if it is expired
-   * 3) if it is expired, create a new session
-   * 4) if it is not expired, borrow it
-   * 5) after computing operations put it back in the cache
+   * This class stores sessions for sharing purposes. If a process requirees a session to

Review comment:
       Minor: "requirees" -> requires

##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
##########
@@ -429,87 +440,124 @@ private void release(SessionWrapper sessionWrapper) {
      * The session can be used by others while the caller is performing operations
      */
     private void returnSession(SessionWrapper sessionWrapper) {
-      TimeSource timeSource = sessionWrapper.session != null ? sessionWrapper.session.cloudManager.getTimeSource() : TimeSource.NANO_TIME;
+      boolean present;
       synchronized (lockObj) {
         sessionWrapper.status = Status.EXECUTING;
-        if (log.isDebugEnabled()) {
-          log.debug("returnSession, curr-time {} sessionWrapper.createTime {}, this.sessionWrapper.createTime {} "
-              , time(timeSource, MILLISECONDS),
-              sessionWrapper.createTime,
-              this.sessionWrapper.createTime);
-        }
-        if (sessionWrapper.createTime == this.sessionWrapper.createTime) {
-          //this session was used for computing new operations and this can now be used for other
-          // computing
-          this.sessionWrapper = sessionWrapper;
+        present = sessionWrapperSet.contains(sessionWrapper);
 
-          //one thread who is waiting for this need to be notified.
-          lockObj.notify();
-        } else {
-          log.debug("create time NOT SAME {} ", SessionWrapper.DEFAULT_INSTANCE.createTime);
-          //else just ignore it
-        }
+        // wake up single thread waiting for a session return (ok if not woken up, wait is short)
+        lockObj.notify();
       }
 
+      // Logging
+      if (present) {
+        if (log.isDebugEnabled()) {
+          log.debug("returnSession {}", sessionWrapper.getCreateTime());
+        }
+      } else {
+        log.warn("returning unknown session {} ", sessionWrapper.getCreateTime());
+      }
     }
 
 
-    public SessionWrapper get(SolrCloudManager cloudManager) throws IOException, InterruptedException {
+    public SessionWrapper get(SolrCloudManager cloudManager, boolean allowWait) throws IOException, InterruptedException {
       TimeSource timeSource = cloudManager.getTimeSource();
+      long oldestUpdateTimeNs = TimeUnit.SECONDS.convert(timeSource.getTimeNs(), TimeUnit.NANOSECONDS) - SESSION_EXPIRY;
+      int zkVersion = cloudManager.getDistribStateManager().getAutoScalingConfig().getZkVersion();
+
       synchronized (lockObj) {
-        if (sessionWrapper.status == Status.NULL ||
-            sessionWrapper.zkVersion != cloudManager.getDistribStateManager().getAutoScalingConfig().getZkVersion() ||
-            TimeUnit.SECONDS.convert(timeSource.getTimeNs() - sessionWrapper.lastUpdateTime, TimeUnit.NANOSECONDS) > SESSION_EXPIRY) {
-          //no session available or the session is expired
+        // If nothing in the cache can possibly work, create a new session
+        if (!hasNonExpiredSession(zkVersion, oldestUpdateTimeNs)) {
           return createSession(cloudManager);
-        } else {
+        }
+
+        // Try to find a session available right away
+        SessionWrapper sw = getAvailableSession(zkVersion, oldestUpdateTimeNs);
+
+        if (sw != null) {
+          if (log.isDebugEnabled()) {
+            log.debug("reusing session {}", sw.getCreateTime());
+          }
+          return sw;
+        } else if (allowWait) {
+          // No session available, but if we wait a bit, maybe one can become available
+          // wait 1 to 10 secs in case a session is returned. Random to spread wakeup otherwise sessions not reused
+          long waitForMs = (long) (Math.random() * 9 * 1000 + 1000);
+
+          if (log.isDebugEnabled()) {
+            log.debug("No sessions are available, all busy COMPUTING. starting wait of {}ms", waitForMs);
+          }
           long waitStart = time(timeSource, MILLISECONDS);
-          //the session is not expired
-          log.debug("reusing a session {}", this.sessionWrapper.createTime);
-          if (this.sessionWrapper.status == Status.UNUSED || this.sessionWrapper.status == Status.EXECUTING) {
-            this.sessionWrapper.status = Status.COMPUTING;
-            return sessionWrapper;
-          } else {
-            //status= COMPUTING it's being used for computing. computing is
-            if (log.isDebugEnabled()) {
-              log.debug("session being used. waiting... current time {} ", time(timeSource, MILLISECONDS));
-            }
-            try {
-              lockObj.wait(10 * 1000);//wait for a max of 10 seconds
-            } catch (InterruptedException e) {
-              log.info("interrupted... ");
-            }
+          try {
+            lockObj.wait(waitForMs);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+          }
+
+          if (log.isDebugEnabled()) {
+            log.debug("out of waiting. wait of {}ms, actual time elapsed {}ms", waitForMs, timeElapsed(timeSource, waitStart, MILLISECONDS));
+          }
+
+          // Try again to find an available session
+          sw = getAvailableSession(zkVersion, oldestUpdateTimeNs);
+
+          if (sw != null) {
             if (log.isDebugEnabled()) {
-              log.debug("out of waiting curr-time:{} time-elapsed {}"
-                  , time(timeSource, MILLISECONDS), timeElapsed(timeSource, waitStart, MILLISECONDS));
-            }
-            // now this thread has woken up because it got timed out after 10 seconds or it is notified after
-            // the session was returned from another COMPUTING operation
-            if (this.sessionWrapper.status == Status.UNUSED || this.sessionWrapper.status == Status.EXECUTING) {
-              log.debug("Wait over. reusing the existing session ");
-              this.sessionWrapper.status = Status.COMPUTING;
-              return sessionWrapper;
-            } else {
-              //create a new Session
-              return createSession(cloudManager);
+              log.debug("Wait over. reusing an existing session {}", sw.getCreateTime());
             }
+            return sw;
+          } else {
+            return createSession(cloudManager);
           }
+        } else {
+          return createSession(cloudManager);
         }
       }
     }
 
+    /**
+     * Returns an available session from the cache (the best one once cache strategies are defined), or null if no session
+     * from the cache is available (i.e. all are still COMPUTING, are too old, wrong zk version or the cache is empty).<p>
+     * This method must be called while holding the monitor on {@link #lockObj}.<p>
+     * The method updates the session status to computing.
+     */
+    private SessionWrapper getAvailableSession(int zkVersion, long oldestUpdateTimeNs) {
+      for (SessionWrapper sw : sessionWrapperSet) {
+        if (sw.status == Status.EXECUTING && sw.getLastUpdateTime() >= oldestUpdateTimeNs && sw.zkVersion == zkVersion) {
+          sw.status = Status.COMPUTING;
+          return sw;
+        }
+      }
+      return null;
+    }
+
+    /**
+     * Returns true if there's a session in the cache that could be returned (if it was free). This is required to
+     * know if there's any point in waiting or if a new session should better be created right away.
+     */
+    private boolean hasNonExpiredSession(int zkVersion, long oldestUpdateTimeNs) {
+      for (SessionWrapper sw : sessionWrapperSet) {
+        if (sw.getLastUpdateTime() >= oldestUpdateTimeNs && sw.zkVersion == zkVersion) {
+          return true;
+        }
+      }
+      return false;
+    }
+
     private SessionWrapper createSession(SolrCloudManager cloudManager) throws InterruptedException, IOException {
-      synchronized (lockObj) {
+      if (log.isDebugEnabled()) {

Review comment:
       Assuming the thread could be waiting on the lockObj for a while, it might make more sense to leave at the beginning of the critical section?

##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
##########
@@ -429,87 +440,124 @@ private void release(SessionWrapper sessionWrapper) {
      * The session can be used by others while the caller is performing operations
      */
     private void returnSession(SessionWrapper sessionWrapper) {
-      TimeSource timeSource = sessionWrapper.session != null ? sessionWrapper.session.cloudManager.getTimeSource() : TimeSource.NANO_TIME;
+      boolean present;
       synchronized (lockObj) {
         sessionWrapper.status = Status.EXECUTING;
-        if (log.isDebugEnabled()) {
-          log.debug("returnSession, curr-time {} sessionWrapper.createTime {}, this.sessionWrapper.createTime {} "
-              , time(timeSource, MILLISECONDS),
-              sessionWrapper.createTime,
-              this.sessionWrapper.createTime);
-        }
-        if (sessionWrapper.createTime == this.sessionWrapper.createTime) {
-          //this session was used for computing new operations and this can now be used for other
-          // computing
-          this.sessionWrapper = sessionWrapper;
+        present = sessionWrapperSet.contains(sessionWrapper);
 
-          //one thread who is waiting for this need to be notified.
-          lockObj.notify();
-        } else {
-          log.debug("create time NOT SAME {} ", SessionWrapper.DEFAULT_INSTANCE.createTime);
-          //else just ignore it
-        }
+        // wake up single thread waiting for a session return (ok if not woken up, wait is short)
+        lockObj.notify();
       }
 
+      // Logging
+      if (present) {
+        if (log.isDebugEnabled()) {
+          log.debug("returnSession {}", sessionWrapper.getCreateTime());
+        }
+      } else {
+        log.warn("returning unknown session {} ", sessionWrapper.getCreateTime());
+      }
     }
 
 
-    public SessionWrapper get(SolrCloudManager cloudManager) throws IOException, InterruptedException {
+    public SessionWrapper get(SolrCloudManager cloudManager, boolean allowWait) throws IOException, InterruptedException {
       TimeSource timeSource = cloudManager.getTimeSource();
+      long oldestUpdateTimeNs = TimeUnit.SECONDS.convert(timeSource.getTimeNs(), TimeUnit.NANOSECONDS) - SESSION_EXPIRY;
+      int zkVersion = cloudManager.getDistribStateManager().getAutoScalingConfig().getZkVersion();
+
       synchronized (lockObj) {
-        if (sessionWrapper.status == Status.NULL ||
-            sessionWrapper.zkVersion != cloudManager.getDistribStateManager().getAutoScalingConfig().getZkVersion() ||
-            TimeUnit.SECONDS.convert(timeSource.getTimeNs() - sessionWrapper.lastUpdateTime, TimeUnit.NANOSECONDS) > SESSION_EXPIRY) {
-          //no session available or the session is expired
+        // If nothing in the cache can possibly work, create a new session
+        if (!hasNonExpiredSession(zkVersion, oldestUpdateTimeNs)) {
           return createSession(cloudManager);
-        } else {
+        }
+
+        // Try to find a session available right away
+        SessionWrapper sw = getAvailableSession(zkVersion, oldestUpdateTimeNs);
+
+        if (sw != null) {
+          if (log.isDebugEnabled()) {
+            log.debug("reusing session {}", sw.getCreateTime());
+          }
+          return sw;
+        } else if (allowWait) {
+          // No session available, but if we wait a bit, maybe one can become available
+          // wait 1 to 10 secs in case a session is returned. Random to spread wakeup otherwise sessions not reused
+          long waitForMs = (long) (Math.random() * 9 * 1000 + 1000);
+
+          if (log.isDebugEnabled()) {
+            log.debug("No sessions are available, all busy COMPUTING. starting wait of {}ms", waitForMs);
+          }
           long waitStart = time(timeSource, MILLISECONDS);
-          //the session is not expired
-          log.debug("reusing a session {}", this.sessionWrapper.createTime);
-          if (this.sessionWrapper.status == Status.UNUSED || this.sessionWrapper.status == Status.EXECUTING) {
-            this.sessionWrapper.status = Status.COMPUTING;
-            return sessionWrapper;
-          } else {
-            //status= COMPUTING it's being used for computing. computing is
-            if (log.isDebugEnabled()) {
-              log.debug("session being used. waiting... current time {} ", time(timeSource, MILLISECONDS));
-            }
-            try {
-              lockObj.wait(10 * 1000);//wait for a max of 10 seconds
-            } catch (InterruptedException e) {
-              log.info("interrupted... ");
-            }
+          try {
+            lockObj.wait(waitForMs);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+          }
+
+          if (log.isDebugEnabled()) {
+            log.debug("out of waiting. wait of {}ms, actual time elapsed {}ms", waitForMs, timeElapsed(timeSource, waitStart, MILLISECONDS));
+          }
+
+          // Try again to find an available session
+          sw = getAvailableSession(zkVersion, oldestUpdateTimeNs);
+
+          if (sw != null) {
             if (log.isDebugEnabled()) {
-              log.debug("out of waiting curr-time:{} time-elapsed {}"
-                  , time(timeSource, MILLISECONDS), timeElapsed(timeSource, waitStart, MILLISECONDS));
-            }
-            // now this thread has woken up because it got timed out after 10 seconds or it is notified after
-            // the session was returned from another COMPUTING operation
-            if (this.sessionWrapper.status == Status.UNUSED || this.sessionWrapper.status == Status.EXECUTING) {
-              log.debug("Wait over. reusing the existing session ");
-              this.sessionWrapper.status = Status.COMPUTING;
-              return sessionWrapper;
-            } else {
-              //create a new Session
-              return createSession(cloudManager);
+              log.debug("Wait over. reusing an existing session {}", sw.getCreateTime());
             }
+            return sw;
+          } else {
+            return createSession(cloudManager);
           }
+        } else {
+          return createSession(cloudManager);
         }
       }
     }
 
+    /**
+     * Returns an available session from the cache (the best one once cache strategies are defined), or null if no session
+     * from the cache is available (i.e. all are still COMPUTING, are too old, wrong zk version or the cache is empty).<p>
+     * This method must be called while holding the monitor on {@link #lockObj}.<p>
+     * The method updates the session status to computing.
+     */
+    private SessionWrapper getAvailableSession(int zkVersion, long oldestUpdateTimeNs) {
+      for (SessionWrapper sw : sessionWrapperSet) {
+        if (sw.status == Status.EXECUTING && sw.getLastUpdateTime() >= oldestUpdateTimeNs && sw.zkVersion == zkVersion) {
+          sw.status = Status.COMPUTING;
+          return sw;
+        }
+      }
+      return null;
+    }
+
+    /**
+     * Returns true if there's a session in the cache that could be returned (if it was free). This is required to
+     * know if there's any point in waiting or if a new session should better be created right away.
+     */
+    private boolean hasNonExpiredSession(int zkVersion, long oldestUpdateTimeNs) {

Review comment:
       Nit: rename to "hasViableSession" or something since it also requires zkVersion check

##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
##########
@@ -429,87 +440,124 @@ private void release(SessionWrapper sessionWrapper) {
      * The session can be used by others while the caller is performing operations
      */
     private void returnSession(SessionWrapper sessionWrapper) {
-      TimeSource timeSource = sessionWrapper.session != null ? sessionWrapper.session.cloudManager.getTimeSource() : TimeSource.NANO_TIME;
+      boolean present;
       synchronized (lockObj) {
         sessionWrapper.status = Status.EXECUTING;
-        if (log.isDebugEnabled()) {
-          log.debug("returnSession, curr-time {} sessionWrapper.createTime {}, this.sessionWrapper.createTime {} "
-              , time(timeSource, MILLISECONDS),
-              sessionWrapper.createTime,
-              this.sessionWrapper.createTime);
-        }
-        if (sessionWrapper.createTime == this.sessionWrapper.createTime) {
-          //this session was used for computing new operations and this can now be used for other
-          // computing
-          this.sessionWrapper = sessionWrapper;
+        present = sessionWrapperSet.contains(sessionWrapper);
 
-          //one thread who is waiting for this need to be notified.
-          lockObj.notify();
-        } else {
-          log.debug("create time NOT SAME {} ", SessionWrapper.DEFAULT_INSTANCE.createTime);
-          //else just ignore it
-        }
+        // wake up single thread waiting for a session return (ok if not woken up, wait is short)
+        lockObj.notify();
       }
 
+      // Logging
+      if (present) {
+        if (log.isDebugEnabled()) {
+          log.debug("returnSession {}", sessionWrapper.getCreateTime());
+        }
+      } else {
+        log.warn("returning unknown session {} ", sessionWrapper.getCreateTime());
+      }
     }
 
 
-    public SessionWrapper get(SolrCloudManager cloudManager) throws IOException, InterruptedException {
+    public SessionWrapper get(SolrCloudManager cloudManager, boolean allowWait) throws IOException, InterruptedException {
       TimeSource timeSource = cloudManager.getTimeSource();
+      long oldestUpdateTimeNs = TimeUnit.SECONDS.convert(timeSource.getTimeNs(), TimeUnit.NANOSECONDS) - SESSION_EXPIRY;
+      int zkVersion = cloudManager.getDistribStateManager().getAutoScalingConfig().getZkVersion();
+
       synchronized (lockObj) {
-        if (sessionWrapper.status == Status.NULL ||
-            sessionWrapper.zkVersion != cloudManager.getDistribStateManager().getAutoScalingConfig().getZkVersion() ||
-            TimeUnit.SECONDS.convert(timeSource.getTimeNs() - sessionWrapper.lastUpdateTime, TimeUnit.NANOSECONDS) > SESSION_EXPIRY) {
-          //no session available or the session is expired
+        // If nothing in the cache can possibly work, create a new session
+        if (!hasNonExpiredSession(zkVersion, oldestUpdateTimeNs)) {
           return createSession(cloudManager);
-        } else {
+        }
+
+        // Try to find a session available right away
+        SessionWrapper sw = getAvailableSession(zkVersion, oldestUpdateTimeNs);
+
+        if (sw != null) {
+          if (log.isDebugEnabled()) {
+            log.debug("reusing session {}", sw.getCreateTime());
+          }
+          return sw;
+        } else if (allowWait) {
+          // No session available, but if we wait a bit, maybe one can become available
+          // wait 1 to 10 secs in case a session is returned. Random to spread wakeup otherwise sessions not reused
+          long waitForMs = (long) (Math.random() * 9 * 1000 + 1000);
+
+          if (log.isDebugEnabled()) {
+            log.debug("No sessions are available, all busy COMPUTING. starting wait of {}ms", waitForMs);
+          }
           long waitStart = time(timeSource, MILLISECONDS);
-          //the session is not expired
-          log.debug("reusing a session {}", this.sessionWrapper.createTime);
-          if (this.sessionWrapper.status == Status.UNUSED || this.sessionWrapper.status == Status.EXECUTING) {
-            this.sessionWrapper.status = Status.COMPUTING;
-            return sessionWrapper;
-          } else {
-            //status= COMPUTING it's being used for computing. computing is
-            if (log.isDebugEnabled()) {
-              log.debug("session being used. waiting... current time {} ", time(timeSource, MILLISECONDS));
-            }
-            try {
-              lockObj.wait(10 * 1000);//wait for a max of 10 seconds
-            } catch (InterruptedException e) {
-              log.info("interrupted... ");
-            }
+          try {
+            lockObj.wait(waitForMs);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+          }
+
+          if (log.isDebugEnabled()) {
+            log.debug("out of waiting. wait of {}ms, actual time elapsed {}ms", waitForMs, timeElapsed(timeSource, waitStart, MILLISECONDS));
+          }
+
+          // Try again to find an available session
+          sw = getAvailableSession(zkVersion, oldestUpdateTimeNs);
+
+          if (sw != null) {
             if (log.isDebugEnabled()) {
-              log.debug("out of waiting curr-time:{} time-elapsed {}"
-                  , time(timeSource, MILLISECONDS), timeElapsed(timeSource, waitStart, MILLISECONDS));
-            }
-            // now this thread has woken up because it got timed out after 10 seconds or it is notified after
-            // the session was returned from another COMPUTING operation
-            if (this.sessionWrapper.status == Status.UNUSED || this.sessionWrapper.status == Status.EXECUTING) {
-              log.debug("Wait over. reusing the existing session ");
-              this.sessionWrapper.status = Status.COMPUTING;
-              return sessionWrapper;
-            } else {
-              //create a new Session
-              return createSession(cloudManager);
+              log.debug("Wait over. reusing an existing session {}", sw.getCreateTime());
             }
+            return sw;
+          } else {
+            return createSession(cloudManager);
           }
+        } else {
+          return createSession(cloudManager);
         }
       }
     }
 
+    /**
+     * Returns an available session from the cache (the best one once cache strategies are defined), or null if no session
+     * from the cache is available (i.e. all are still COMPUTING, are too old, wrong zk version or the cache is empty).<p>
+     * This method must be called while holding the monitor on {@link #lockObj}.<p>
+     * The method updates the session status to computing.
+     */
+    private SessionWrapper getAvailableSession(int zkVersion, long oldestUpdateTimeNs) {
+      for (SessionWrapper sw : sessionWrapperSet) {
+        if (sw.status == Status.EXECUTING && sw.getLastUpdateTime() >= oldestUpdateTimeNs && sw.zkVersion == zkVersion) {
+          sw.status = Status.COMPUTING;
+          return sw;
+        }
+      }
+      return null;
+    }
+
+    /**
+     * Returns true if there's a session in the cache that could be returned (if it was free). This is required to
+     * know if there's any point in waiting or if a new session should better be created right away.
+     */
+    private boolean hasNonExpiredSession(int zkVersion, long oldestUpdateTimeNs) {

Review comment:
       Do we know how many sessions we might have at any given time? Could this be expensive? I suppose it guarantees savings of 1-10 seconds in cases where allowWait is true and there's nothing worth waiting for; just wondering what average case scenario is

##########
File path: solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
##########
@@ -429,87 +440,124 @@ private void release(SessionWrapper sessionWrapper) {
      * The session can be used by others while the caller is performing operations
      */
     private void returnSession(SessionWrapper sessionWrapper) {
-      TimeSource timeSource = sessionWrapper.session != null ? sessionWrapper.session.cloudManager.getTimeSource() : TimeSource.NANO_TIME;
+      boolean present;
       synchronized (lockObj) {
         sessionWrapper.status = Status.EXECUTING;
-        if (log.isDebugEnabled()) {
-          log.debug("returnSession, curr-time {} sessionWrapper.createTime {}, this.sessionWrapper.createTime {} "
-              , time(timeSource, MILLISECONDS),
-              sessionWrapper.createTime,
-              this.sessionWrapper.createTime);
-        }
-        if (sessionWrapper.createTime == this.sessionWrapper.createTime) {
-          //this session was used for computing new operations and this can now be used for other
-          // computing
-          this.sessionWrapper = sessionWrapper;
+        present = sessionWrapperSet.contains(sessionWrapper);
 
-          //one thread who is waiting for this need to be notified.
-          lockObj.notify();
-        } else {
-          log.debug("create time NOT SAME {} ", SessionWrapper.DEFAULT_INSTANCE.createTime);
-          //else just ignore it
-        }
+        // wake up single thread waiting for a session return (ok if not woken up, wait is short)
+        lockObj.notify();
       }
 
+      // Logging
+      if (present) {
+        if (log.isDebugEnabled()) {
+          log.debug("returnSession {}", sessionWrapper.getCreateTime());
+        }
+      } else {
+        log.warn("returning unknown session {} ", sessionWrapper.getCreateTime());
+      }
     }
 
 
-    public SessionWrapper get(SolrCloudManager cloudManager) throws IOException, InterruptedException {
+    public SessionWrapper get(SolrCloudManager cloudManager, boolean allowWait) throws IOException, InterruptedException {

Review comment:
       Wrote up a *slightly* different implementation here: https://gist.github.com/megancarey/ae2bad53d320ef660ef45c8b003901e1
   
   No more redundant code but I suppose worse in terms of memory, since it makes a recursive call




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

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



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