You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@zookeeper.apache.org by neptune <op...@gmail.com> on 2010/02/08 23:37:16 UTC

When session expired event fired?

Hi all.
I have a question. I started zookeeper(3.2.2) on three servers.
When session expired event fired in following code?
I expected that if client can't connect to server(disconnected) for session
timeout, zookeeper fires session expired event.
I killed three zookeeper server sequentially. Client retry to connect
zookeeper server. Never occured Expired event.

*class WatcherTest {
  public static void main(String[] args) {
    (new **WatcherTest*()).exec();
*  }

  private WatcherTest() throws Exception {
    zk = new ZooKeeper("server1:2181,server2:2181:server3:2181", 10 * 1000,
this);
  }
  private void exec() {
    while(ture) {
      //do something
    }
  }
  public void process(WatchedEvent event) {
    if (event.getType() == Event.EventType.None) {
      switch (event.getState()) {
      case SyncConnected:
        System.out.println("ZK SyncConnected");
        break;
      case Disconnected:
        System.out.println("ZK Disconnected");
        break;
      case Expired:
        System.out.println("ZK Session Expired");
        System.exit(0);
        break;
      }
    }
}
*

Re: When session expired event fired?

Posted by Patrick Hunt <ph...@apache.org>.
Here's one that I know of that's open - LinkedIn's Norbert
http://wiki.github.com/rhavyn/norbert/

Patrick

neptune wrote:
> Thanks for your quick answer.
> I will make cluster member ship service using zookeeper.
> If a node in a cluster can't connect to zookeeper cluster, the node killed
> by oneself (like google bigtables' tabletserver).
> If no session expired, client check disconnected time.
> Any idea or usage for cluster membership service using zookeeper?
> 
> Thanks.
> 
> 2010/2/9 Mahadev Konar <ma...@yahoo-inc.com>
> 
>> Hi,
>>  a zookeeper client does not expire a session until and unless it is able
>> to
>> connect to one of the servers. In your case if you kill all the servers,
>> the
>> client is not able to connect to any of the servers and will keep trying to
>> connect to the three servers. It cannot expire a session on its own and
>> needs to hear from the server to know if the session is expired or not.
>>
>> Does that help?
>>
>> Thanks
>> mahadev
>>
>>
>> On 2/8/10 2:37 PM, "neptune" <op...@gmail.com> wrote:
>>
>>> Hi all.
>>> I have a question. I started zookeeper(3.2.2) on three servers.
>>> When session expired event fired in following code?
>>> I expected that if client can't connect to server(disconnected) for
>> session
>>> timeout, zookeeper fires session expired event.
>>> I killed three zookeeper server sequentially. Client retry to connect
>>> zookeeper server. Never occured Expired event.
>>>
>>> *class WatcherTest {
>>>   public static void main(String[] args) {
>>>     (new **WatcherTest*()).exec();
>>> *  }
>>>
>>>   private WatcherTest() throws Exception {
>>>     zk = new ZooKeeper("server1:2181,server2:2181:server3:2181", 10 *
>> 1000,
>>> this);
>>>   }
>>>   private void exec() {
>>>     while(ture) {
>>>       //do something
>>>     }
>>>   }
>>>   public void process(WatchedEvent event) {
>>>     if (event.getType() == Event.EventType.None) {
>>>       switch (event.getState()) {
>>>       case SyncConnected:
>>>         System.out.println("ZK SyncConnected");
>>>         break;
>>>       case Disconnected:
>>>         System.out.println("ZK Disconnected");
>>>         break;
>>>       case Expired:
>>>         System.out.println("ZK Session Expired");
>>>         System.exit(0);
>>>         break;
>>>       }
>>>     }
>>> }
>>> *
>>
> 

Re: When session expired event fired?

Posted by neptune <op...@gmail.com>.
Thanks for your quick answer.
I will make cluster member ship service using zookeeper.
If a node in a cluster can't connect to zookeeper cluster, the node killed
by oneself (like google bigtables' tabletserver).
If no session expired, client check disconnected time.
Any idea or usage for cluster membership service using zookeeper?

Thanks.

2010/2/9 Mahadev Konar <ma...@yahoo-inc.com>

> Hi,
>  a zookeeper client does not expire a session until and unless it is able
> to
> connect to one of the servers. In your case if you kill all the servers,
> the
> client is not able to connect to any of the servers and will keep trying to
> connect to the three servers. It cannot expire a session on its own and
> needs to hear from the server to know if the session is expired or not.
>
> Does that help?
>
> Thanks
> mahadev
>
>
> On 2/8/10 2:37 PM, "neptune" <op...@gmail.com> wrote:
>
> > Hi all.
> > I have a question. I started zookeeper(3.2.2) on three servers.
> > When session expired event fired in following code?
> > I expected that if client can't connect to server(disconnected) for
> session
> > timeout, zookeeper fires session expired event.
> > I killed three zookeeper server sequentially. Client retry to connect
> > zookeeper server. Never occured Expired event.
> >
> > *class WatcherTest {
> >   public static void main(String[] args) {
> >     (new **WatcherTest*()).exec();
> > *  }
> >
> >   private WatcherTest() throws Exception {
> >     zk = new ZooKeeper("server1:2181,server2:2181:server3:2181", 10 *
> 1000,
> > this);
> >   }
> >   private void exec() {
> >     while(ture) {
> >       //do something
> >     }
> >   }
> >   public void process(WatchedEvent event) {
> >     if (event.getType() == Event.EventType.None) {
> >       switch (event.getState()) {
> >       case SyncConnected:
> >         System.out.println("ZK SyncConnected");
> >         break;
> >       case Disconnected:
> >         System.out.println("ZK Disconnected");
> >         break;
> >       case Expired:
> >         System.out.println("ZK Session Expired");
> >         System.exit(0);
> >         break;
> >       }
> >     }
> > }
> > *
>
>

Re: When session expired event fired?

Posted by Patrick Hunt <ph...@apache.org>.
No, disconnected means that you don't have a connection to the cluster, 
it does not inform the status of the session. The expired indicates that 
the session itself is no longer valid (and therefore any ephemerals are 
cleaned up).

Perhaps this is one area we should work on wrt the docs, ensuring that 
we make clear the diff btw connection and session. I have done a number 
of changes for 3.3.0 logging specifically in this area based on hbase 
user feedback (ensuring we are clear in our logging what is a connection 
related log and what is a session related log).

Hope this helps,

Patrick

neptune wrote:
> Hi all,
> Here's DisconnectedTimeoutHandler sample code. I thought Session Expired is
> similar DisconnectedTimeoutHandler.
> 
> *public class WatcherTest implements Watcher {
>   private static final String ZK_HOSTS = "127.0.0.1:2181,127.0.0.1:2182,
> 127.0.0.1:2183";
> 
>   private static final int ZK_SESSION_TIMEOUT = 10 * 1000;
> 
>   private ZooKeeper zk;
> 
>   DisconnectedTimeoutHandler timeoutHandler;
> 
>   private WatcherTest() throws Exception {
>     timeoutHandler = new DisconnectedTimeoutHandler(this,
> ZK_SESSION_TIMEOUT);
>     zk = new ZooKeeper(ZK_HOSTS, ZK_SESSION_TIMEOUT, this);
>     timeoutHandler.start();
>   }
> 
>   private void exec() {
>     while(true) {
>       try {
>         Thread.sleep(5 * 1000);
> //        long sid = zk.getSessionId();
> //        byte[] passwd = zk.getSessionPasswd();
> //        zk.close();
> //        zk = new ZooKeeper(ZK_HOSTS, ZK_SESSION_TIMEOUT, this, sid,
> passwd);
>       } catch (Exception e) {
>         e.printStackTrace();
>       }
>     }
>   }
> 
>   @Override
>   public void process(WatchedEvent event) {
>     if (event.getType() == Event.EventType.None) {
>       switch (event.getState()) {
>       case SyncConnected:
>         timeoutHandler.disconnected(false);
>         System.out.println("ZK SyncConnected");
>         break;
>       case Disconnected:
>         timeoutHandler.disconnected(true);
>         System.out.println("ZK Disconnected");
>         break;
>       case Expired:
>         System.out.println("ZK Session Expired");
>         shutdown();
>         break;
>       }
>     }
>   }
> 
>   private void shutdown() {
>     System.exit(0);
>   }
> 
>   public static void main(String[] args) throws Exception {
>     WatcherTest test = new WatcherTest();
>     test.exec();
>   }
> }
> 
> class DisconnectedTimeoutHandler extends Thread {
>   private boolean disconnected = false;
>   private long lastDisconnectedTime;
>   private long timeout;
>   private Watcher watcher;
> 
>   public DisconnectedTimeoutHandler(Watcher watcher, long timeout) {
>     this.watcher = watcher;
>     this.timeout = timeout;
>   }
> 
>   public synchronized void disconnected(boolean disconnected) {
>     if(!this.disconnected && disconnected) {
>       this.lastDisconnectedTime = System.currentTimeMillis();;
>     }
>     this.disconnected = disconnected;
>     synchronized(this) {
>       this.notify();
>     }
>   }
> 
>   public void start() {
>     while(true) {
>       synchronized(this) {
>         try {
>           this.wait();
>         } catch (InterruptedException e) {
>         }
>       }
> 
>       while(disconnected) {
>         if(System.currentTimeMillis() - lastDisconnectedTime >= timeout) {
>           watcher.process(new WatchedEvent(Watcher.Event.EventType.None,
> Watcher.Event.KeeperState.Expired, null));
>           disconnected = false;
>           lastDisconnectedTime = 0;
>         }
>         try {
>           Thread.sleep(1000);
>         } catch (InterruptedException e) {
>           break;
>         }
>       }
>     }
>   }
> }*
> 
> 2010/2/9 neptune <op...@gmail.com>
> 
>> I saw that wiki page. And I read this paragraph "It means that the client
>> was partitioned off from the ZooKeeper service for more the the session
>> timeout ..."
>> That's why I use Expired event for cluster membership. In my case client is
>> a node in a cluster.
>>
>> 2010/2/9 Patrick Hunt <ph...@apache.org>
>>
>> this? "How should I handle SESSION_EXPIRED?"
>>> http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A3
>>>
>>>
>>> Benjamin Reed wrote:
>>>
>>>> i was looking through the docs to see if we talk about handling session
>>>> expired, but i couldn't find anything. we should probably open a jira to add
>>>> to the docs, unless i missed something. did i?
>>>>
>>>> ben
>>>>
>>>> -----Original Message-----
>>>> From: Mahadev Konar [mailto:mahadev@yahoo-inc.com] Sent: Monday,
>>>> February 08, 2010 2:43 PM
>>>> To: zookeeper-user@hadoop.apache.org
>>>> Subject: Re: When session expired event fired?
>>>>
>>>> Hi,
>>>>  a zookeeper client does not expire a session until and unless it is able
>>>> to
>>>> connect to one of the servers. In your case if you kill all the servers,
>>>> the
>>>> client is not able to connect to any of the servers and will keep trying
>>>> to
>>>> connect to the three servers. It cannot expire a session on its own and
>>>> needs to hear from the server to know if the session is expired or not.
>>>>
>>>> Does that help?
>>>> Thanks
>>>> mahadev
>>>>
>>>>
>>>> On 2/8/10 2:37 PM, "neptune" <op...@gmail.com> wrote:
>>>>
>>>>  Hi all.
>>>>> I have a question. I started zookeeper(3.2.2) on three servers.
>>>>> When session expired event fired in following code?
>>>>> I expected that if client can't connect to server(disconnected) for
>>>>> session
>>>>> timeout, zookeeper fires session expired event.
>>>>> I killed three zookeeper server sequentially. Client retry to connect
>>>>> zookeeper server. Never occured Expired event.
>>>>>
>>>>> *class WatcherTest {
>>>>>  public static void main(String[] args) {
>>>>>    (new **WatcherTest*()).exec();
>>>>> *  }
>>>>>
>>>>>  private WatcherTest() throws Exception {
>>>>>    zk = new ZooKeeper("server1:2181,server2:2181:server3:2181", 10 *
>>>>> 1000,
>>>>> this);
>>>>>  }
>>>>>  private void exec() {
>>>>>    while(ture) {
>>>>>      //do something
>>>>>    }
>>>>>  }
>>>>>  public void process(WatchedEvent event) {
>>>>>    if (event.getType() == Event.EventType.None) {
>>>>>      switch (event.getState()) {
>>>>>      case SyncConnected:
>>>>>        System.out.println("ZK SyncConnected");
>>>>>        break;
>>>>>      case Disconnected:
>>>>>        System.out.println("ZK Disconnected");
>>>>>        break;
>>>>>      case Expired:
>>>>>        System.out.println("ZK Session Expired");
>>>>>        System.exit(0);
>>>>>        break;
>>>>>      }
>>>>>    }
>>>>> }
>>>>> *
>>>>>
>>>>
> 

Re: When session expired event fired?

Posted by neptune <op...@gmail.com>.
Hi all,
Here's DisconnectedTimeoutHandler sample code. I thought Session Expired is
similar DisconnectedTimeoutHandler.

*public class WatcherTest implements Watcher {
  private static final String ZK_HOSTS = "127.0.0.1:2181,127.0.0.1:2182,
127.0.0.1:2183";

  private static final int ZK_SESSION_TIMEOUT = 10 * 1000;

  private ZooKeeper zk;

  DisconnectedTimeoutHandler timeoutHandler;

  private WatcherTest() throws Exception {
    timeoutHandler = new DisconnectedTimeoutHandler(this,
ZK_SESSION_TIMEOUT);
    zk = new ZooKeeper(ZK_HOSTS, ZK_SESSION_TIMEOUT, this);
    timeoutHandler.start();
  }

  private void exec() {
    while(true) {
      try {
        Thread.sleep(5 * 1000);
//        long sid = zk.getSessionId();
//        byte[] passwd = zk.getSessionPasswd();
//        zk.close();
//        zk = new ZooKeeper(ZK_HOSTS, ZK_SESSION_TIMEOUT, this, sid,
passwd);
      } catch (Exception e) {
        e.printStackTrace();
      }
    }
  }

  @Override
  public void process(WatchedEvent event) {
    if (event.getType() == Event.EventType.None) {
      switch (event.getState()) {
      case SyncConnected:
        timeoutHandler.disconnected(false);
        System.out.println("ZK SyncConnected");
        break;
      case Disconnected:
        timeoutHandler.disconnected(true);
        System.out.println("ZK Disconnected");
        break;
      case Expired:
        System.out.println("ZK Session Expired");
        shutdown();
        break;
      }
    }
  }

  private void shutdown() {
    System.exit(0);
  }

  public static void main(String[] args) throws Exception {
    WatcherTest test = new WatcherTest();
    test.exec();
  }
}

class DisconnectedTimeoutHandler extends Thread {
  private boolean disconnected = false;
  private long lastDisconnectedTime;
  private long timeout;
  private Watcher watcher;

  public DisconnectedTimeoutHandler(Watcher watcher, long timeout) {
    this.watcher = watcher;
    this.timeout = timeout;
  }

  public synchronized void disconnected(boolean disconnected) {
    if(!this.disconnected && disconnected) {
      this.lastDisconnectedTime = System.currentTimeMillis();;
    }
    this.disconnected = disconnected;
    synchronized(this) {
      this.notify();
    }
  }

  public void start() {
    while(true) {
      synchronized(this) {
        try {
          this.wait();
        } catch (InterruptedException e) {
        }
      }

      while(disconnected) {
        if(System.currentTimeMillis() - lastDisconnectedTime >= timeout) {
          watcher.process(new WatchedEvent(Watcher.Event.EventType.None,
Watcher.Event.KeeperState.Expired, null));
          disconnected = false;
          lastDisconnectedTime = 0;
        }
        try {
          Thread.sleep(1000);
        } catch (InterruptedException e) {
          break;
        }
      }
    }
  }
}*

2010/2/9 neptune <op...@gmail.com>

> I saw that wiki page. And I read this paragraph "It means that the client
> was partitioned off from the ZooKeeper service for more the the session
> timeout ..."
> That's why I use Expired event for cluster membership. In my case client is
> a node in a cluster.
>
> 2010/2/9 Patrick Hunt <ph...@apache.org>
>
> this? "How should I handle SESSION_EXPIRED?"
>> http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A3
>>
>>
>> Benjamin Reed wrote:
>>
>>> i was looking through the docs to see if we talk about handling session
>>> expired, but i couldn't find anything. we should probably open a jira to add
>>> to the docs, unless i missed something. did i?
>>>
>>> ben
>>>
>>> -----Original Message-----
>>> From: Mahadev Konar [mailto:mahadev@yahoo-inc.com] Sent: Monday,
>>> February 08, 2010 2:43 PM
>>> To: zookeeper-user@hadoop.apache.org
>>> Subject: Re: When session expired event fired?
>>>
>>> Hi,
>>>  a zookeeper client does not expire a session until and unless it is able
>>> to
>>> connect to one of the servers. In your case if you kill all the servers,
>>> the
>>> client is not able to connect to any of the servers and will keep trying
>>> to
>>> connect to the three servers. It cannot expire a session on its own and
>>> needs to hear from the server to know if the session is expired or not.
>>>
>>> Does that help?
>>> Thanks
>>> mahadev
>>>
>>>
>>> On 2/8/10 2:37 PM, "neptune" <op...@gmail.com> wrote:
>>>
>>>  Hi all.
>>>> I have a question. I started zookeeper(3.2.2) on three servers.
>>>> When session expired event fired in following code?
>>>> I expected that if client can't connect to server(disconnected) for
>>>> session
>>>> timeout, zookeeper fires session expired event.
>>>> I killed three zookeeper server sequentially. Client retry to connect
>>>> zookeeper server. Never occured Expired event.
>>>>
>>>> *class WatcherTest {
>>>>  public static void main(String[] args) {
>>>>    (new **WatcherTest*()).exec();
>>>> *  }
>>>>
>>>>  private WatcherTest() throws Exception {
>>>>    zk = new ZooKeeper("server1:2181,server2:2181:server3:2181", 10 *
>>>> 1000,
>>>> this);
>>>>  }
>>>>  private void exec() {
>>>>    while(ture) {
>>>>      //do something
>>>>    }
>>>>  }
>>>>  public void process(WatchedEvent event) {
>>>>    if (event.getType() == Event.EventType.None) {
>>>>      switch (event.getState()) {
>>>>      case SyncConnected:
>>>>        System.out.println("ZK SyncConnected");
>>>>        break;
>>>>      case Disconnected:
>>>>        System.out.println("ZK Disconnected");
>>>>        break;
>>>>      case Expired:
>>>>        System.out.println("ZK Session Expired");
>>>>        System.exit(0);
>>>>        break;
>>>>      }
>>>>    }
>>>> }
>>>> *
>>>>
>>>
>>>
>

Re: When session expired event fired?

Posted by Patrick Hunt <ph...@apache.org>.
In this case all of the _other_ zookeeper clients will see the client 
session is closed (actually they see any ephemeral znodes that the 
client created are removed, which is typically what you do for group 
membership - create an ephemeral znode that represents your client, a 
member of the cluster) and can act appropriately.

Your client will see a disconnected event:
http://hadoop.apache.org/zookeeper/docs/current/zookeeperProgrammers.html#ch_zkSessions

Given what Mahadev mentioned (you don't know if you are expired until 
you reconnect) what you do now in the disco client is up to your use 
case. If it's just group membership then it's probably fine, your client 
will be disconnected, perhaps expired. It won't know, but the rest of 
the cluster (incl any monitoring app) will know and can act 
appropriately. The disconnected client needs to decide what to do 
(perhaps, or perhaps it doesn't do anything, it's just that the other 
members of the cluster no longer consider it active, and therefore don't 
involve it in processing)

Patrick

neptune wrote:
> I saw that wiki page. And I read this paragraph "It means that the client
> was partitioned off from the ZooKeeper service for more the the session
> timeout ..."
> That's why I use Expired event for cluster membership. In my case client is
> a node in a cluster.
> 
> 2010/2/9 Patrick Hunt <ph...@apache.org>
> 
>> this? "How should I handle SESSION_EXPIRED?"
>> http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A3
>>
>>
>> Benjamin Reed wrote:
>>
>>> i was looking through the docs to see if we talk about handling session
>>> expired, but i couldn't find anything. we should probably open a jira to add
>>> to the docs, unless i missed something. did i?
>>>
>>> ben
>>>
>>> -----Original Message-----
>>> From: Mahadev Konar [mailto:mahadev@yahoo-inc.com] Sent: Monday, February
>>> 08, 2010 2:43 PM
>>> To: zookeeper-user@hadoop.apache.org
>>> Subject: Re: When session expired event fired?
>>>
>>> Hi,
>>>  a zookeeper client does not expire a session until and unless it is able
>>> to
>>> connect to one of the servers. In your case if you kill all the servers,
>>> the
>>> client is not able to connect to any of the servers and will keep trying
>>> to
>>> connect to the three servers. It cannot expire a session on its own and
>>> needs to hear from the server to know if the session is expired or not.
>>>
>>> Does that help?
>>> Thanks
>>> mahadev
>>>
>>>
>>> On 2/8/10 2:37 PM, "neptune" <op...@gmail.com> wrote:
>>>
>>>  Hi all.
>>>> I have a question. I started zookeeper(3.2.2) on three servers.
>>>> When session expired event fired in following code?
>>>> I expected that if client can't connect to server(disconnected) for
>>>> session
>>>> timeout, zookeeper fires session expired event.
>>>> I killed three zookeeper server sequentially. Client retry to connect
>>>> zookeeper server. Never occured Expired event.
>>>>
>>>> *class WatcherTest {
>>>>  public static void main(String[] args) {
>>>>    (new **WatcherTest*()).exec();
>>>> *  }
>>>>
>>>>  private WatcherTest() throws Exception {
>>>>    zk = new ZooKeeper("server1:2181,server2:2181:server3:2181", 10 *
>>>> 1000,
>>>> this);
>>>>  }
>>>>  private void exec() {
>>>>    while(ture) {
>>>>      //do something
>>>>    }
>>>>  }
>>>>  public void process(WatchedEvent event) {
>>>>    if (event.getType() == Event.EventType.None) {
>>>>      switch (event.getState()) {
>>>>      case SyncConnected:
>>>>        System.out.println("ZK SyncConnected");
>>>>        break;
>>>>      case Disconnected:
>>>>        System.out.println("ZK Disconnected");
>>>>        break;
>>>>      case Expired:
>>>>        System.out.println("ZK Session Expired");
>>>>        System.exit(0);
>>>>        break;
>>>>      }
>>>>    }
>>>> }
>>>> *
>>>>
>>>
> 

Re: When session expired event fired?

Posted by neptune <op...@gmail.com>.
I saw that wiki page. And I read this paragraph "It means that the client
was partitioned off from the ZooKeeper service for more the the session
timeout ..."
That's why I use Expired event for cluster membership. In my case client is
a node in a cluster.

2010/2/9 Patrick Hunt <ph...@apache.org>

> this? "How should I handle SESSION_EXPIRED?"
> http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A3
>
>
> Benjamin Reed wrote:
>
>> i was looking through the docs to see if we talk about handling session
>> expired, but i couldn't find anything. we should probably open a jira to add
>> to the docs, unless i missed something. did i?
>>
>> ben
>>
>> -----Original Message-----
>> From: Mahadev Konar [mailto:mahadev@yahoo-inc.com] Sent: Monday, February
>> 08, 2010 2:43 PM
>> To: zookeeper-user@hadoop.apache.org
>> Subject: Re: When session expired event fired?
>>
>> Hi,
>>  a zookeeper client does not expire a session until and unless it is able
>> to
>> connect to one of the servers. In your case if you kill all the servers,
>> the
>> client is not able to connect to any of the servers and will keep trying
>> to
>> connect to the three servers. It cannot expire a session on its own and
>> needs to hear from the server to know if the session is expired or not.
>>
>> Does that help?
>> Thanks
>> mahadev
>>
>>
>> On 2/8/10 2:37 PM, "neptune" <op...@gmail.com> wrote:
>>
>>  Hi all.
>>> I have a question. I started zookeeper(3.2.2) on three servers.
>>> When session expired event fired in following code?
>>> I expected that if client can't connect to server(disconnected) for
>>> session
>>> timeout, zookeeper fires session expired event.
>>> I killed three zookeeper server sequentially. Client retry to connect
>>> zookeeper server. Never occured Expired event.
>>>
>>> *class WatcherTest {
>>>  public static void main(String[] args) {
>>>    (new **WatcherTest*()).exec();
>>> *  }
>>>
>>>  private WatcherTest() throws Exception {
>>>    zk = new ZooKeeper("server1:2181,server2:2181:server3:2181", 10 *
>>> 1000,
>>> this);
>>>  }
>>>  private void exec() {
>>>    while(ture) {
>>>      //do something
>>>    }
>>>  }
>>>  public void process(WatchedEvent event) {
>>>    if (event.getType() == Event.EventType.None) {
>>>      switch (event.getState()) {
>>>      case SyncConnected:
>>>        System.out.println("ZK SyncConnected");
>>>        break;
>>>      case Disconnected:
>>>        System.out.println("ZK Disconnected");
>>>        break;
>>>      case Expired:
>>>        System.out.println("ZK Session Expired");
>>>        System.exit(0);
>>>        break;
>>>      }
>>>    }
>>> }
>>> *
>>>
>>
>>

Re: When session expired event fired?

Posted by Patrick Hunt <ph...@apache.org>.
this? "How should I handle SESSION_EXPIRED?"
http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A3

Benjamin Reed wrote:
> i was looking through the docs to see if we talk about handling session expired, but i couldn't find anything. we should probably open a jira to add to the docs, unless i missed something. did i?
> 
> ben
> 
> -----Original Message-----
> From: Mahadev Konar [mailto:mahadev@yahoo-inc.com] 
> Sent: Monday, February 08, 2010 2:43 PM
> To: zookeeper-user@hadoop.apache.org
> Subject: Re: When session expired event fired?
> 
> Hi,
>  a zookeeper client does not expire a session until and unless it is able to
> connect to one of the servers. In your case if you kill all the servers, the
> client is not able to connect to any of the servers and will keep trying to
> connect to the three servers. It cannot expire a session on its own and
> needs to hear from the server to know if the session is expired or not.
> 
> Does that help? 
> 
> Thanks
> mahadev
> 
> 
> On 2/8/10 2:37 PM, "neptune" <op...@gmail.com> wrote:
> 
>> Hi all.
>> I have a question. I started zookeeper(3.2.2) on three servers.
>> When session expired event fired in following code?
>> I expected that if client can't connect to server(disconnected) for session
>> timeout, zookeeper fires session expired event.
>> I killed three zookeeper server sequentially. Client retry to connect
>> zookeeper server. Never occured Expired event.
>>
>> *class WatcherTest {
>>   public static void main(String[] args) {
>>     (new **WatcherTest*()).exec();
>> *  }
>>
>>   private WatcherTest() throws Exception {
>>     zk = new ZooKeeper("server1:2181,server2:2181:server3:2181", 10 * 1000,
>> this);
>>   }
>>   private void exec() {
>>     while(ture) {
>>       //do something
>>     }
>>   }
>>   public void process(WatchedEvent event) {
>>     if (event.getType() == Event.EventType.None) {
>>       switch (event.getState()) {
>>       case SyncConnected:
>>         System.out.println("ZK SyncConnected");
>>         break;
>>       case Disconnected:
>>         System.out.println("ZK Disconnected");
>>         break;
>>       case Expired:
>>         System.out.println("ZK Session Expired");
>>         System.exit(0);
>>         break;
>>       }
>>     }
>> }
>> *
> 

RE: When session expired event fired?

Posted by Benjamin Reed <br...@yahoo-inc.com>.
i was looking through the docs to see if we talk about handling session expired, but i couldn't find anything. we should probably open a jira to add to the docs, unless i missed something. did i?

ben

-----Original Message-----
From: Mahadev Konar [mailto:mahadev@yahoo-inc.com] 
Sent: Monday, February 08, 2010 2:43 PM
To: zookeeper-user@hadoop.apache.org
Subject: Re: When session expired event fired?

Hi,
 a zookeeper client does not expire a session until and unless it is able to
connect to one of the servers. In your case if you kill all the servers, the
client is not able to connect to any of the servers and will keep trying to
connect to the three servers. It cannot expire a session on its own and
needs to hear from the server to know if the session is expired or not.

Does that help? 

Thanks
mahadev


On 2/8/10 2:37 PM, "neptune" <op...@gmail.com> wrote:

> Hi all.
> I have a question. I started zookeeper(3.2.2) on three servers.
> When session expired event fired in following code?
> I expected that if client can't connect to server(disconnected) for session
> timeout, zookeeper fires session expired event.
> I killed three zookeeper server sequentially. Client retry to connect
> zookeeper server. Never occured Expired event.
> 
> *class WatcherTest {
>   public static void main(String[] args) {
>     (new **WatcherTest*()).exec();
> *  }
> 
>   private WatcherTest() throws Exception {
>     zk = new ZooKeeper("server1:2181,server2:2181:server3:2181", 10 * 1000,
> this);
>   }
>   private void exec() {
>     while(ture) {
>       //do something
>     }
>   }
>   public void process(WatchedEvent event) {
>     if (event.getType() == Event.EventType.None) {
>       switch (event.getState()) {
>       case SyncConnected:
>         System.out.println("ZK SyncConnected");
>         break;
>       case Disconnected:
>         System.out.println("ZK Disconnected");
>         break;
>       case Expired:
>         System.out.println("ZK Session Expired");
>         System.exit(0);
>         break;
>       }
>     }
> }
> *


Re: When session expired event fired?

Posted by Mahadev Konar <ma...@yahoo-inc.com>.
Hi,
 a zookeeper client does not expire a session until and unless it is able to
connect to one of the servers. In your case if you kill all the servers, the
client is not able to connect to any of the servers and will keep trying to
connect to the three servers. It cannot expire a session on its own and
needs to hear from the server to know if the session is expired or not.

Does that help? 

Thanks
mahadev


On 2/8/10 2:37 PM, "neptune" <op...@gmail.com> wrote:

> Hi all.
> I have a question. I started zookeeper(3.2.2) on three servers.
> When session expired event fired in following code?
> I expected that if client can't connect to server(disconnected) for session
> timeout, zookeeper fires session expired event.
> I killed three zookeeper server sequentially. Client retry to connect
> zookeeper server. Never occured Expired event.
> 
> *class WatcherTest {
>   public static void main(String[] args) {
>     (new **WatcherTest*()).exec();
> *  }
> 
>   private WatcherTest() throws Exception {
>     zk = new ZooKeeper("server1:2181,server2:2181:server3:2181", 10 * 1000,
> this);
>   }
>   private void exec() {
>     while(ture) {
>       //do something
>     }
>   }
>   public void process(WatchedEvent event) {
>     if (event.getType() == Event.EventType.None) {
>       switch (event.getState()) {
>       case SyncConnected:
>         System.out.println("ZK SyncConnected");
>         break;
>       case Disconnected:
>         System.out.println("ZK Disconnected");
>         break;
>       case Expired:
>         System.out.println("ZK Session Expired");
>         System.exit(0);
>         break;
>       }
>     }
> }
> *