[ 
https://issues.apache.org/jira/browse/SOLR-8777?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15353715#comment-15353715
 ] 

Scott Blum commented on SOLR-8777:
----------------------------------

LGTM.  One suggestion, it's almost as easy to make 
checkForExistingEphemeralNode() to use a watcher instead of a loop.

{code}
private void checkForExistingEphemeralNode() throws KeeperException, 
InterruptedException {
  if (zkRunOnly) {
    return;
  }
  String nodeName = getNodeName();
  String nodePath = ZkStateReader.LIVE_NODES_ZKNODE + "/" + nodeName;

  if (!zkClient.exists(nodePath, true)) {
    return;
  }

  final CountDownLatch deletedLatch = new CountDownLatch(1);
  Stat stat = zkClient.exists(nodePath, new Watcher() {
    @Override
    public void process(WatchedEvent event) {
      if (Event.EventType.None.equals(event.getType())) {
        return;
      }
      if (Event.EventType.NodeDeleted.equals(event.getType())) {
        deletedLatch.countDown();
      }
    }
  }, true);

  if (stat == null) {
    // suddenly disappeared
    return;
  }

  boolean deleted = 
deletedLatch.await(zkClient.getSolrZooKeeper().getSessionTimeout() * 2, 
TimeUnit.MILLISECONDS);
  if (!deleted) {
    throw new SolrException(ErrorCode.SERVER_ERROR, "A previous ephemeral live 
node still exists. " +
        "Solr cannot continue. Please ensure that no other Solr process using 
the same port is running already.");
  }
}
{code}


> Duplicate Solr process can cripple a running process
> ----------------------------------------------------
>
>                 Key: SOLR-8777
>                 URL: https://issues.apache.org/jira/browse/SOLR-8777
>             Project: Solr
>          Issue Type: Bug
>          Components: SolrCloud
>    Affects Versions: 5.3.1
>            Reporter: Shalin Shekhar Mangar
>         Attachments: SOLR-8777.patch
>
>
> Thanks to [~mewmewball] for catching this one.
> Accidentally executing the same instance of Solr twice causes the second 
> start instance to die with an "Address already in use", but not before 
> deleting the first instance's live_node entry, emitting "Found a previous 
> node that still exists while trying to register a new live node <node> - 
> removing existing node to create another".
> The second start instance dies and its ephemeral node is then removed, 
> causing /live_nodes/<node> to be empty since the first start instance's 
> live_node was deleted by the second.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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

Reply via email to