You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by 乱麻的魅力 <80...@qq.com> on 2012/06/08 10:00:30 UTC

about zookeeper-cli have bug or some doubt

hi dev:
     I now try to use the zookeeper cli (c code version)to connect the zookeeper server, but i find only can connect to ZK,but cann't send any cmd to ZK, like "ls /".  if i send cmd ,then zk-cli goto deadlock at this line  lock_buffer_list(list)   {//LINE 00945 dequeue_buffer() function of zookeeper.c};   then i try to locate this case.
  
    i download  the zk cli (ver 3.4.3) from http://labs.renren.com/apache-mirror/zookeeper/ ,  buid the project again, find bug locate the line 00945   in zookeeper-3.4.3.tar.gz\zookeeper-3.4.3\src\c\src\zookeeper.c too. now i describe this case below:
  
 1 if client send cmd to ZKserver, client need call some function to send the cmd ,like zoo_awget,send_ping,zoo_aget,etc.., all this function need call  adaptor_send_queue(zh, 0); then below...
  
 2 adaptor_send_queue(zh, 0) call  flush_send_queue(zh, timeout);
  
  int flush_send_queue(zhandle_t*zh, int timeout)
{
    int rc= ZOK;
    struct timeval started;
#ifdef WIN32
    fd_set pollSet; 
    struct timeval wait;
#endif
    gettimeofday(&started,0);
    // we can't use dequeue_buffer() here because if (non-blocking) send_buffer()
    // returns EWOULDBLOCK we'd have to put the buffer back on the queue.
    // we use a recursive lock instead and only dequeue the buffer if a send was
    // successful
    lock_buffer_list(&zh->to_send);  /*first time lock the buffer, wfs 20120608 */
    while (zh->to_send.head != 0&& zh->state == ZOO_CONNECTED_STATE) {
        if(timeout!=0){
            int elapsed;
            struct timeval now;
            gettimeofday(&now,0);
            elapsed=calculate_interval(&started,&now);
            if (elapsed>timeout) {
                rc = ZOPERATIONTIMEOUT;
                break;
            }
 #ifdef WIN32
            wait = get_timeval(timeout-elapsed);
            FD_ZERO(&pollSet);
            FD_SET(zh->fd, &pollSet);
            // Poll the socket
            rc = select((int)(zh->fd)+1, NULL,  &pollSet, NULL, &wait);      
#else
            struct pollfd fds;
            fds.fd = zh->fd;
            fds.events = POLLOUT;
            fds.revents = 0;
            rc = poll(&fds, 1, timeout-elapsed);
#endif
            if (rc<=0) {
                /* timed out or an error or POLLERR */
                rc = rc==0 ? ZOPERATIONTIMEOUT : ZSYSTEMERROR;
                break;
            }
        }
         rc = send_buffer(zh->fd, zh->to_send.head);
        if(rc==0 && timeout==0){
            /* send_buffer would block while sending this buffer */
            rc = ZOK;
            break;
        }
        if (rc < 0) {
            rc = ZCONNECTIONLOSS;
            break;
        }
        // if the buffer has been sent successfully, remove it from the queue
        if (rc > 0)
            remove_buffer(&zh->to_send); /*this function will second time lock the buffer with lock under locked status, wfs 20120608 */

        gettimeofday(&zh->last_send, 0);
        rc = ZOK;
    }
    unlock_buffer_list(&zh->to_send);
    return rc;
} 
  
 static int remove_buffer(buffer_head_t *list)
{
    buffer_list_t *b = dequeue_buffer(list);
    if (!b) {
        return 0;
    }
    free_buffer(b);
    return 1;
}
  
 static buffer_list_t *dequeue_buffer(buffer_head_t *list)
{
    buffer_list_t *b;
    lock_buffer_list(list);  /*this function second time lock the buffer with lock under locked status 20120608 , then will lead the function to deadlock at this line;
    
     if i re-write a new function like *dequeue_buffer(buffer_head_t *list)  and remove_buffer function without lock and unlock to be callback by flush_send_queue, then zk-cli can send cmd to the zookkeeper server, clie don't deadlock*/

    b = list->head;
    if (b) {
        list->head = b->next;
        if (!list->head) {
            assert(b == list->last);
            list->last = 0;
        }
    }
    unlock_buffer_list(list);
    return b;
}
  
  i don't known whether I detailly describe this case,  and i find old version 3.3.3 have this bug too,i think this c source-code maybe never be tested or i use wrong way, can you help me clear this case。
  
  thanks!
    wfs fr china 20120608

Re: about zookeeper-cli have bug or some doubt

Posted by Patrick Hunt <ph...@apache.org>.
Try builds@apache.org, they are responsible for the CI.

Patrick

On Sun, Jun 10, 2012 at 2:32 AM, Michi Mutsuzaki <mi...@cs.stanford.edu> wrote:
> It looks like "msbuild" command is missing.
>
> f:\hudson\hudson-slave\workspace\ZooKeeper-trunk-WinVS2008>msbuild
> trunk/src/c/zookeeper.sln /p:Configuration=Release
> 'msbuild' is not recognized as an internal or external command,
> operable program or batch file.
>
> Here is the output from the last successful run.
>
> f:\hudson\hudson-slave\workspace\ZooKeeper-trunk-WinVS2008>msbuild
> trunk/src/c/zookeeper.sln /p:Configuration=Release
> Microsoft (R) Build Engine Version 3.5.30729.1
> [Microsoft .NET Framework, Version 2.0.50727.4223]
> Copyright (C) Microsoft Corporation 2007. All rights reserved.
>
> Who should I contact to fix this?
>
> Thanks!
> --Michi
>
> On Sat, Jun 9, 2012 at 11:28 AM, Michi Mutsuzaki <mi...@cs.stanford.edu> wrote:
>> Ok, I'll take a look.
>>
>> --Michi
>>
>> On Fri, Jun 8, 2012 at 3:17 PM, Patrick Hunt <ph...@apache.org> wrote:
>>> Speaking of windows, Michi can you take a look why the windows job has
>>> started failing of late? Perhaps an environment change? (you might
>>> look at other windows jobs on that box to get an idea)
>>>
>>> https://builds.apache.org//view/S-Z/view/ZooKeeper/job/ZooKeeper-trunk-WinVS2008/
>>>
>>> Thanks!
>>>
>>> Patrick
>>>
>>> On Fri, Jun 8, 2012 at 10:16 AM, Michi Mutsuzaki <mi...@cs.stanford.edu> wrote:
>>>> I think there is a bug in windows port (are you on windows?) that
>>>> doesn't set recursive attribute for the to_send mutex. Please open a
>>>> jira:
>>>>
>>>> https://issues.apache.org/jira/browse/ZOOKEEPER
>>>>
>>>> Thanks!
>>>> --Michi
>>>>
>>>> On Fri, Jun 8, 2012 at 1:00 AM, 乱麻的魅力 <80...@qq.com> wrote:
>>>>> hi dev:
>>>>>     I now try to use the zookeeper cli (c code version)to connect the zookeeper server, but i find only can connect to ZK,but cann't send any cmd to ZK, like "ls /".  if i send cmd ,then zk-cli goto deadlock at this line  lock_buffer_list(list)   {//LINE 00945 dequeue_buffer() function of zookeeper.c};   then i try to locate this case.
>>>>>
>>>>>    i download  the zk cli (ver 3.4.3) from http://labs.renren.com/apache-mirror/zookeeper/ ,  buid the project again, find bug locate the line 00945   in zookeeper-3.4.3.tar.gz\zookeeper-3.4.3\src\c\src\zookeeper.c too. now i describe this case below:
>>>>>
>>>>>  1 if client send cmd to ZKserver, client need call some function to send the cmd ,like zoo_awget,send_ping,zoo_aget,etc.., all this function need call  adaptor_send_queue(zh, 0); then below...
>>>>>
>>>>>  2 adaptor_send_queue(zh, 0) call  flush_send_queue(zh, timeout);
>>>>>
>>>>>  int flush_send_queue(zhandle_t*zh, int timeout)
>>>>> {
>>>>>    int rc= ZOK;
>>>>>    struct timeval started;
>>>>> #ifdef WIN32
>>>>>    fd_set pollSet;
>>>>>    struct timeval wait;
>>>>> #endif
>>>>>    gettimeofday(&started,0);
>>>>>    // we can't use dequeue_buffer() here because if (non-blocking) send_buffer()
>>>>>    // returns EWOULDBLOCK we'd have to put the buffer back on the queue.
>>>>>    // we use a recursive lock instead and only dequeue the buffer if a send was
>>>>>    // successful
>>>>>    lock_buffer_list(&zh->to_send);  /*first time lock the buffer, wfs 20120608 */
>>>>>    while (zh->to_send.head != 0&& zh->state == ZOO_CONNECTED_STATE) {
>>>>>        if(timeout!=0){
>>>>>            int elapsed;
>>>>>            struct timeval now;
>>>>>            gettimeofday(&now,0);
>>>>>            elapsed=calculate_interval(&started,&now);
>>>>>            if (elapsed>timeout) {
>>>>>                rc = ZOPERATIONTIMEOUT;
>>>>>                break;
>>>>>            }
>>>>>  #ifdef WIN32
>>>>>            wait = get_timeval(timeout-elapsed);
>>>>>            FD_ZERO(&pollSet);
>>>>>            FD_SET(zh->fd, &pollSet);
>>>>>            // Poll the socket
>>>>>            rc = select((int)(zh->fd)+1, NULL,  &pollSet, NULL, &wait);
>>>>> #else
>>>>>            struct pollfd fds;
>>>>>            fds.fd = zh->fd;
>>>>>            fds.events = POLLOUT;
>>>>>            fds.revents = 0;
>>>>>            rc = poll(&fds, 1, timeout-elapsed);
>>>>> #endif
>>>>>            if (rc<=0) {
>>>>>                /* timed out or an error or POLLERR */
>>>>>                rc = rc==0 ? ZOPERATIONTIMEOUT : ZSYSTEMERROR;
>>>>>                break;
>>>>>            }
>>>>>        }
>>>>>         rc = send_buffer(zh->fd, zh->to_send.head);
>>>>>        if(rc==0 && timeout==0){
>>>>>            /* send_buffer would block while sending this buffer */
>>>>>            rc = ZOK;
>>>>>            break;
>>>>>        }
>>>>>        if (rc < 0) {
>>>>>            rc = ZCONNECTIONLOSS;
>>>>>            break;
>>>>>        }
>>>>>        // if the buffer has been sent successfully, remove it from the queue
>>>>>        if (rc > 0)
>>>>>            remove_buffer(&zh->to_send); /*this function will second time lock the buffer with lock under locked status, wfs 20120608 */
>>>>>
>>>>>        gettimeofday(&zh->last_send, 0);
>>>>>        rc = ZOK;
>>>>>    }
>>>>>    unlock_buffer_list(&zh->to_send);
>>>>>    return rc;
>>>>> }
>>>>>
>>>>>  static int remove_buffer(buffer_head_t *list)
>>>>> {
>>>>>    buffer_list_t *b = dequeue_buffer(list);
>>>>>    if (!b) {
>>>>>        return 0;
>>>>>    }
>>>>>    free_buffer(b);
>>>>>    return 1;
>>>>> }
>>>>>
>>>>>  static buffer_list_t *dequeue_buffer(buffer_head_t *list)
>>>>> {
>>>>>    buffer_list_t *b;
>>>>>    lock_buffer_list(list);  /*this function second time lock the buffer with lock under locked status 20120608 , then will lead the function to deadlock at this line;
>>>>>
>>>>>     if i re-write a new function like *dequeue_buffer(buffer_head_t *list)  and remove_buffer function without lock and unlock to be callback by flush_send_queue, then zk-cli can send cmd to the zookkeeper server, clie don't deadlock*/
>>>>>
>>>>>    b = list->head;
>>>>>    if (b) {
>>>>>        list->head = b->next;
>>>>>        if (!list->head) {
>>>>>            assert(b == list->last);
>>>>>            list->last = 0;
>>>>>        }
>>>>>    }
>>>>>    unlock_buffer_list(list);
>>>>>    return b;
>>>>> }
>>>>>
>>>>>  i don't known whether I detailly describe this case,  and i find old version 3.3.3 have this bug too,i think this c source-code maybe never be tested or i use wrong way, can you help me clear this case。
>>>>>
>>>>>  thanks!
>>>>>    wfs fr china 20120608

Re: about zookeeper-cli have bug or some doubt

Posted by Michi Mutsuzaki <mi...@cs.stanford.edu>.
It looks like "msbuild" command is missing.

f:\hudson\hudson-slave\workspace\ZooKeeper-trunk-WinVS2008>msbuild
trunk/src/c/zookeeper.sln /p:Configuration=Release
'msbuild' is not recognized as an internal or external command,
operable program or batch file.

Here is the output from the last successful run.

f:\hudson\hudson-slave\workspace\ZooKeeper-trunk-WinVS2008>msbuild
trunk/src/c/zookeeper.sln /p:Configuration=Release
Microsoft (R) Build Engine Version 3.5.30729.1
[Microsoft .NET Framework, Version 2.0.50727.4223]
Copyright (C) Microsoft Corporation 2007. All rights reserved.

Who should I contact to fix this?

Thanks!
--Michi

On Sat, Jun 9, 2012 at 11:28 AM, Michi Mutsuzaki <mi...@cs.stanford.edu> wrote:
> Ok, I'll take a look.
>
> --Michi
>
> On Fri, Jun 8, 2012 at 3:17 PM, Patrick Hunt <ph...@apache.org> wrote:
>> Speaking of windows, Michi can you take a look why the windows job has
>> started failing of late? Perhaps an environment change? (you might
>> look at other windows jobs on that box to get an idea)
>>
>> https://builds.apache.org//view/S-Z/view/ZooKeeper/job/ZooKeeper-trunk-WinVS2008/
>>
>> Thanks!
>>
>> Patrick
>>
>> On Fri, Jun 8, 2012 at 10:16 AM, Michi Mutsuzaki <mi...@cs.stanford.edu> wrote:
>>> I think there is a bug in windows port (are you on windows?) that
>>> doesn't set recursive attribute for the to_send mutex. Please open a
>>> jira:
>>>
>>> https://issues.apache.org/jira/browse/ZOOKEEPER
>>>
>>> Thanks!
>>> --Michi
>>>
>>> On Fri, Jun 8, 2012 at 1:00 AM, 乱麻的魅力 <80...@qq.com> wrote:
>>>> hi dev:
>>>>     I now try to use the zookeeper cli (c code version)to connect the zookeeper server, but i find only can connect to ZK,but cann't send any cmd to ZK, like "ls /".  if i send cmd ,then zk-cli goto deadlock at this line  lock_buffer_list(list)   {//LINE 00945 dequeue_buffer() function of zookeeper.c};   then i try to locate this case.
>>>>
>>>>    i download  the zk cli (ver 3.4.3) from http://labs.renren.com/apache-mirror/zookeeper/ ,  buid the project again, find bug locate the line 00945   in zookeeper-3.4.3.tar.gz\zookeeper-3.4.3\src\c\src\zookeeper.c too. now i describe this case below:
>>>>
>>>>  1 if client send cmd to ZKserver, client need call some function to send the cmd ,like zoo_awget,send_ping,zoo_aget,etc.., all this function need call  adaptor_send_queue(zh, 0); then below...
>>>>
>>>>  2 adaptor_send_queue(zh, 0) call  flush_send_queue(zh, timeout);
>>>>
>>>>  int flush_send_queue(zhandle_t*zh, int timeout)
>>>> {
>>>>    int rc= ZOK;
>>>>    struct timeval started;
>>>> #ifdef WIN32
>>>>    fd_set pollSet;
>>>>    struct timeval wait;
>>>> #endif
>>>>    gettimeofday(&started,0);
>>>>    // we can't use dequeue_buffer() here because if (non-blocking) send_buffer()
>>>>    // returns EWOULDBLOCK we'd have to put the buffer back on the queue.
>>>>    // we use a recursive lock instead and only dequeue the buffer if a send was
>>>>    // successful
>>>>    lock_buffer_list(&zh->to_send);  /*first time lock the buffer, wfs 20120608 */
>>>>    while (zh->to_send.head != 0&& zh->state == ZOO_CONNECTED_STATE) {
>>>>        if(timeout!=0){
>>>>            int elapsed;
>>>>            struct timeval now;
>>>>            gettimeofday(&now,0);
>>>>            elapsed=calculate_interval(&started,&now);
>>>>            if (elapsed>timeout) {
>>>>                rc = ZOPERATIONTIMEOUT;
>>>>                break;
>>>>            }
>>>>  #ifdef WIN32
>>>>            wait = get_timeval(timeout-elapsed);
>>>>            FD_ZERO(&pollSet);
>>>>            FD_SET(zh->fd, &pollSet);
>>>>            // Poll the socket
>>>>            rc = select((int)(zh->fd)+1, NULL,  &pollSet, NULL, &wait);
>>>> #else
>>>>            struct pollfd fds;
>>>>            fds.fd = zh->fd;
>>>>            fds.events = POLLOUT;
>>>>            fds.revents = 0;
>>>>            rc = poll(&fds, 1, timeout-elapsed);
>>>> #endif
>>>>            if (rc<=0) {
>>>>                /* timed out or an error or POLLERR */
>>>>                rc = rc==0 ? ZOPERATIONTIMEOUT : ZSYSTEMERROR;
>>>>                break;
>>>>            }
>>>>        }
>>>>         rc = send_buffer(zh->fd, zh->to_send.head);
>>>>        if(rc==0 && timeout==0){
>>>>            /* send_buffer would block while sending this buffer */
>>>>            rc = ZOK;
>>>>            break;
>>>>        }
>>>>        if (rc < 0) {
>>>>            rc = ZCONNECTIONLOSS;
>>>>            break;
>>>>        }
>>>>        // if the buffer has been sent successfully, remove it from the queue
>>>>        if (rc > 0)
>>>>            remove_buffer(&zh->to_send); /*this function will second time lock the buffer with lock under locked status, wfs 20120608 */
>>>>
>>>>        gettimeofday(&zh->last_send, 0);
>>>>        rc = ZOK;
>>>>    }
>>>>    unlock_buffer_list(&zh->to_send);
>>>>    return rc;
>>>> }
>>>>
>>>>  static int remove_buffer(buffer_head_t *list)
>>>> {
>>>>    buffer_list_t *b = dequeue_buffer(list);
>>>>    if (!b) {
>>>>        return 0;
>>>>    }
>>>>    free_buffer(b);
>>>>    return 1;
>>>> }
>>>>
>>>>  static buffer_list_t *dequeue_buffer(buffer_head_t *list)
>>>> {
>>>>    buffer_list_t *b;
>>>>    lock_buffer_list(list);  /*this function second time lock the buffer with lock under locked status 20120608 , then will lead the function to deadlock at this line;
>>>>
>>>>     if i re-write a new function like *dequeue_buffer(buffer_head_t *list)  and remove_buffer function without lock and unlock to be callback by flush_send_queue, then zk-cli can send cmd to the zookkeeper server, clie don't deadlock*/
>>>>
>>>>    b = list->head;
>>>>    if (b) {
>>>>        list->head = b->next;
>>>>        if (!list->head) {
>>>>            assert(b == list->last);
>>>>            list->last = 0;
>>>>        }
>>>>    }
>>>>    unlock_buffer_list(list);
>>>>    return b;
>>>> }
>>>>
>>>>  i don't known whether I detailly describe this case,  and i find old version 3.3.3 have this bug too,i think this c source-code maybe never be tested or i use wrong way, can you help me clear this case。
>>>>
>>>>  thanks!
>>>>    wfs fr china 20120608

Re: about zookeeper-cli have bug or some doubt

Posted by Michi Mutsuzaki <mi...@cs.stanford.edu>.
Ok, I'll take a look.

--Michi

On Fri, Jun 8, 2012 at 3:17 PM, Patrick Hunt <ph...@apache.org> wrote:
> Speaking of windows, Michi can you take a look why the windows job has
> started failing of late? Perhaps an environment change? (you might
> look at other windows jobs on that box to get an idea)
>
> https://builds.apache.org//view/S-Z/view/ZooKeeper/job/ZooKeeper-trunk-WinVS2008/
>
> Thanks!
>
> Patrick
>
> On Fri, Jun 8, 2012 at 10:16 AM, Michi Mutsuzaki <mi...@cs.stanford.edu> wrote:
>> I think there is a bug in windows port (are you on windows?) that
>> doesn't set recursive attribute for the to_send mutex. Please open a
>> jira:
>>
>> https://issues.apache.org/jira/browse/ZOOKEEPER
>>
>> Thanks!
>> --Michi
>>
>> On Fri, Jun 8, 2012 at 1:00 AM, 乱麻的魅力 <80...@qq.com> wrote:
>>> hi dev:
>>>     I now try to use the zookeeper cli (c code version)to connect the zookeeper server, but i find only can connect to ZK,but cann't send any cmd to ZK, like "ls /".  if i send cmd ,then zk-cli goto deadlock at this line  lock_buffer_list(list)   {//LINE 00945 dequeue_buffer() function of zookeeper.c};   then i try to locate this case.
>>>
>>>    i download  the zk cli (ver 3.4.3) from http://labs.renren.com/apache-mirror/zookeeper/ ,  buid the project again, find bug locate the line 00945   in zookeeper-3.4.3.tar.gz\zookeeper-3.4.3\src\c\src\zookeeper.c too. now i describe this case below:
>>>
>>>  1 if client send cmd to ZKserver, client need call some function to send the cmd ,like zoo_awget,send_ping,zoo_aget,etc.., all this function need call  adaptor_send_queue(zh, 0); then below...
>>>
>>>  2 adaptor_send_queue(zh, 0) call  flush_send_queue(zh, timeout);
>>>
>>>  int flush_send_queue(zhandle_t*zh, int timeout)
>>> {
>>>    int rc= ZOK;
>>>    struct timeval started;
>>> #ifdef WIN32
>>>    fd_set pollSet;
>>>    struct timeval wait;
>>> #endif
>>>    gettimeofday(&started,0);
>>>    // we can't use dequeue_buffer() here because if (non-blocking) send_buffer()
>>>    // returns EWOULDBLOCK we'd have to put the buffer back on the queue.
>>>    // we use a recursive lock instead and only dequeue the buffer if a send was
>>>    // successful
>>>    lock_buffer_list(&zh->to_send);  /*first time lock the buffer, wfs 20120608 */
>>>    while (zh->to_send.head != 0&& zh->state == ZOO_CONNECTED_STATE) {
>>>        if(timeout!=0){
>>>            int elapsed;
>>>            struct timeval now;
>>>            gettimeofday(&now,0);
>>>            elapsed=calculate_interval(&started,&now);
>>>            if (elapsed>timeout) {
>>>                rc = ZOPERATIONTIMEOUT;
>>>                break;
>>>            }
>>>  #ifdef WIN32
>>>            wait = get_timeval(timeout-elapsed);
>>>            FD_ZERO(&pollSet);
>>>            FD_SET(zh->fd, &pollSet);
>>>            // Poll the socket
>>>            rc = select((int)(zh->fd)+1, NULL,  &pollSet, NULL, &wait);
>>> #else
>>>            struct pollfd fds;
>>>            fds.fd = zh->fd;
>>>            fds.events = POLLOUT;
>>>            fds.revents = 0;
>>>            rc = poll(&fds, 1, timeout-elapsed);
>>> #endif
>>>            if (rc<=0) {
>>>                /* timed out or an error or POLLERR */
>>>                rc = rc==0 ? ZOPERATIONTIMEOUT : ZSYSTEMERROR;
>>>                break;
>>>            }
>>>        }
>>>         rc = send_buffer(zh->fd, zh->to_send.head);
>>>        if(rc==0 && timeout==0){
>>>            /* send_buffer would block while sending this buffer */
>>>            rc = ZOK;
>>>            break;
>>>        }
>>>        if (rc < 0) {
>>>            rc = ZCONNECTIONLOSS;
>>>            break;
>>>        }
>>>        // if the buffer has been sent successfully, remove it from the queue
>>>        if (rc > 0)
>>>            remove_buffer(&zh->to_send); /*this function will second time lock the buffer with lock under locked status, wfs 20120608 */
>>>
>>>        gettimeofday(&zh->last_send, 0);
>>>        rc = ZOK;
>>>    }
>>>    unlock_buffer_list(&zh->to_send);
>>>    return rc;
>>> }
>>>
>>>  static int remove_buffer(buffer_head_t *list)
>>> {
>>>    buffer_list_t *b = dequeue_buffer(list);
>>>    if (!b) {
>>>        return 0;
>>>    }
>>>    free_buffer(b);
>>>    return 1;
>>> }
>>>
>>>  static buffer_list_t *dequeue_buffer(buffer_head_t *list)
>>> {
>>>    buffer_list_t *b;
>>>    lock_buffer_list(list);  /*this function second time lock the buffer with lock under locked status 20120608 , then will lead the function to deadlock at this line;
>>>
>>>     if i re-write a new function like *dequeue_buffer(buffer_head_t *list)  and remove_buffer function without lock and unlock to be callback by flush_send_queue, then zk-cli can send cmd to the zookkeeper server, clie don't deadlock*/
>>>
>>>    b = list->head;
>>>    if (b) {
>>>        list->head = b->next;
>>>        if (!list->head) {
>>>            assert(b == list->last);
>>>            list->last = 0;
>>>        }
>>>    }
>>>    unlock_buffer_list(list);
>>>    return b;
>>> }
>>>
>>>  i don't known whether I detailly describe this case,  and i find old version 3.3.3 have this bug too,i think this c source-code maybe never be tested or i use wrong way, can you help me clear this case。
>>>
>>>  thanks!
>>>    wfs fr china 20120608

Re: about zookeeper-cli have bug or some doubt

Posted by Patrick Hunt <ph...@apache.org>.
Speaking of windows, Michi can you take a look why the windows job has
started failing of late? Perhaps an environment change? (you might
look at other windows jobs on that box to get an idea)

https://builds.apache.org//view/S-Z/view/ZooKeeper/job/ZooKeeper-trunk-WinVS2008/

Thanks!

Patrick

On Fri, Jun 8, 2012 at 10:16 AM, Michi Mutsuzaki <mi...@cs.stanford.edu> wrote:
> I think there is a bug in windows port (are you on windows?) that
> doesn't set recursive attribute for the to_send mutex. Please open a
> jira:
>
> https://issues.apache.org/jira/browse/ZOOKEEPER
>
> Thanks!
> --Michi
>
> On Fri, Jun 8, 2012 at 1:00 AM, 乱麻的魅力 <80...@qq.com> wrote:
>> hi dev:
>>     I now try to use the zookeeper cli (c code version)to connect the zookeeper server, but i find only can connect to ZK,but cann't send any cmd to ZK, like "ls /".  if i send cmd ,then zk-cli goto deadlock at this line  lock_buffer_list(list)   {//LINE 00945 dequeue_buffer() function of zookeeper.c};   then i try to locate this case.
>>
>>    i download  the zk cli (ver 3.4.3) from http://labs.renren.com/apache-mirror/zookeeper/ ,  buid the project again, find bug locate the line 00945   in zookeeper-3.4.3.tar.gz\zookeeper-3.4.3\src\c\src\zookeeper.c too. now i describe this case below:
>>
>>  1 if client send cmd to ZKserver, client need call some function to send the cmd ,like zoo_awget,send_ping,zoo_aget,etc.., all this function need call  adaptor_send_queue(zh, 0); then below...
>>
>>  2 adaptor_send_queue(zh, 0) call  flush_send_queue(zh, timeout);
>>
>>  int flush_send_queue(zhandle_t*zh, int timeout)
>> {
>>    int rc= ZOK;
>>    struct timeval started;
>> #ifdef WIN32
>>    fd_set pollSet;
>>    struct timeval wait;
>> #endif
>>    gettimeofday(&started,0);
>>    // we can't use dequeue_buffer() here because if (non-blocking) send_buffer()
>>    // returns EWOULDBLOCK we'd have to put the buffer back on the queue.
>>    // we use a recursive lock instead and only dequeue the buffer if a send was
>>    // successful
>>    lock_buffer_list(&zh->to_send);  /*first time lock the buffer, wfs 20120608 */
>>    while (zh->to_send.head != 0&& zh->state == ZOO_CONNECTED_STATE) {
>>        if(timeout!=0){
>>            int elapsed;
>>            struct timeval now;
>>            gettimeofday(&now,0);
>>            elapsed=calculate_interval(&started,&now);
>>            if (elapsed>timeout) {
>>                rc = ZOPERATIONTIMEOUT;
>>                break;
>>            }
>>  #ifdef WIN32
>>            wait = get_timeval(timeout-elapsed);
>>            FD_ZERO(&pollSet);
>>            FD_SET(zh->fd, &pollSet);
>>            // Poll the socket
>>            rc = select((int)(zh->fd)+1, NULL,  &pollSet, NULL, &wait);
>> #else
>>            struct pollfd fds;
>>            fds.fd = zh->fd;
>>            fds.events = POLLOUT;
>>            fds.revents = 0;
>>            rc = poll(&fds, 1, timeout-elapsed);
>> #endif
>>            if (rc<=0) {
>>                /* timed out or an error or POLLERR */
>>                rc = rc==0 ? ZOPERATIONTIMEOUT : ZSYSTEMERROR;
>>                break;
>>            }
>>        }
>>         rc = send_buffer(zh->fd, zh->to_send.head);
>>        if(rc==0 && timeout==0){
>>            /* send_buffer would block while sending this buffer */
>>            rc = ZOK;
>>            break;
>>        }
>>        if (rc < 0) {
>>            rc = ZCONNECTIONLOSS;
>>            break;
>>        }
>>        // if the buffer has been sent successfully, remove it from the queue
>>        if (rc > 0)
>>            remove_buffer(&zh->to_send); /*this function will second time lock the buffer with lock under locked status, wfs 20120608 */
>>
>>        gettimeofday(&zh->last_send, 0);
>>        rc = ZOK;
>>    }
>>    unlock_buffer_list(&zh->to_send);
>>    return rc;
>> }
>>
>>  static int remove_buffer(buffer_head_t *list)
>> {
>>    buffer_list_t *b = dequeue_buffer(list);
>>    if (!b) {
>>        return 0;
>>    }
>>    free_buffer(b);
>>    return 1;
>> }
>>
>>  static buffer_list_t *dequeue_buffer(buffer_head_t *list)
>> {
>>    buffer_list_t *b;
>>    lock_buffer_list(list);  /*this function second time lock the buffer with lock under locked status 20120608 , then will lead the function to deadlock at this line;
>>
>>     if i re-write a new function like *dequeue_buffer(buffer_head_t *list)  and remove_buffer function without lock and unlock to be callback by flush_send_queue, then zk-cli can send cmd to the zookkeeper server, clie don't deadlock*/
>>
>>    b = list->head;
>>    if (b) {
>>        list->head = b->next;
>>        if (!list->head) {
>>            assert(b == list->last);
>>            list->last = 0;
>>        }
>>    }
>>    unlock_buffer_list(list);
>>    return b;
>> }
>>
>>  i don't known whether I detailly describe this case,  and i find old version 3.3.3 have this bug too,i think this c source-code maybe never be tested or i use wrong way, can you help me clear this case。
>>
>>  thanks!
>>    wfs fr china 20120608

Re: about zookeeper-cli have bug or some doubt

Posted by Michi Mutsuzaki <mi...@cs.stanford.edu>.
I think there is a bug in windows port (are you on windows?) that
doesn't set recursive attribute for the to_send mutex. Please open a
jira:

https://issues.apache.org/jira/browse/ZOOKEEPER

Thanks!
--Michi

On Fri, Jun 8, 2012 at 1:00 AM, 乱麻的魅力 <80...@qq.com> wrote:
> hi dev:
>     I now try to use the zookeeper cli (c code version)to connect the zookeeper server, but i find only can connect to ZK,but cann't send any cmd to ZK, like "ls /".  if i send cmd ,then zk-cli goto deadlock at this line  lock_buffer_list(list)   {//LINE 00945 dequeue_buffer() function of zookeeper.c};   then i try to locate this case.
>
>    i download  the zk cli (ver 3.4.3) from http://labs.renren.com/apache-mirror/zookeeper/ ,  buid the project again, find bug locate the line 00945   in zookeeper-3.4.3.tar.gz\zookeeper-3.4.3\src\c\src\zookeeper.c too. now i describe this case below:
>
>  1 if client send cmd to ZKserver, client need call some function to send the cmd ,like zoo_awget,send_ping,zoo_aget,etc.., all this function need call  adaptor_send_queue(zh, 0); then below...
>
>  2 adaptor_send_queue(zh, 0) call  flush_send_queue(zh, timeout);
>
>  int flush_send_queue(zhandle_t*zh, int timeout)
> {
>    int rc= ZOK;
>    struct timeval started;
> #ifdef WIN32
>    fd_set pollSet;
>    struct timeval wait;
> #endif
>    gettimeofday(&started,0);
>    // we can't use dequeue_buffer() here because if (non-blocking) send_buffer()
>    // returns EWOULDBLOCK we'd have to put the buffer back on the queue.
>    // we use a recursive lock instead and only dequeue the buffer if a send was
>    // successful
>    lock_buffer_list(&zh->to_send);  /*first time lock the buffer, wfs 20120608 */
>    while (zh->to_send.head != 0&& zh->state == ZOO_CONNECTED_STATE) {
>        if(timeout!=0){
>            int elapsed;
>            struct timeval now;
>            gettimeofday(&now,0);
>            elapsed=calculate_interval(&started,&now);
>            if (elapsed>timeout) {
>                rc = ZOPERATIONTIMEOUT;
>                break;
>            }
>  #ifdef WIN32
>            wait = get_timeval(timeout-elapsed);
>            FD_ZERO(&pollSet);
>            FD_SET(zh->fd, &pollSet);
>            // Poll the socket
>            rc = select((int)(zh->fd)+1, NULL,  &pollSet, NULL, &wait);
> #else
>            struct pollfd fds;
>            fds.fd = zh->fd;
>            fds.events = POLLOUT;
>            fds.revents = 0;
>            rc = poll(&fds, 1, timeout-elapsed);
> #endif
>            if (rc<=0) {
>                /* timed out or an error or POLLERR */
>                rc = rc==0 ? ZOPERATIONTIMEOUT : ZSYSTEMERROR;
>                break;
>            }
>        }
>         rc = send_buffer(zh->fd, zh->to_send.head);
>        if(rc==0 && timeout==0){
>            /* send_buffer would block while sending this buffer */
>            rc = ZOK;
>            break;
>        }
>        if (rc < 0) {
>            rc = ZCONNECTIONLOSS;
>            break;
>        }
>        // if the buffer has been sent successfully, remove it from the queue
>        if (rc > 0)
>            remove_buffer(&zh->to_send); /*this function will second time lock the buffer with lock under locked status, wfs 20120608 */
>
>        gettimeofday(&zh->last_send, 0);
>        rc = ZOK;
>    }
>    unlock_buffer_list(&zh->to_send);
>    return rc;
> }
>
>  static int remove_buffer(buffer_head_t *list)
> {
>    buffer_list_t *b = dequeue_buffer(list);
>    if (!b) {
>        return 0;
>    }
>    free_buffer(b);
>    return 1;
> }
>
>  static buffer_list_t *dequeue_buffer(buffer_head_t *list)
> {
>    buffer_list_t *b;
>    lock_buffer_list(list);  /*this function second time lock the buffer with lock under locked status 20120608 , then will lead the function to deadlock at this line;
>
>     if i re-write a new function like *dequeue_buffer(buffer_head_t *list)  and remove_buffer function without lock and unlock to be callback by flush_send_queue, then zk-cli can send cmd to the zookkeeper server, clie don't deadlock*/
>
>    b = list->head;
>    if (b) {
>        list->head = b->next;
>        if (!list->head) {
>            assert(b == list->last);
>            list->last = 0;
>        }
>    }
>    unlock_buffer_list(list);
>    return b;
> }
>
>  i don't known whether I detailly describe this case,  and i find old version 3.3.3 have this bug too,i think this c source-code maybe never be tested or i use wrong way, can you help me clear this case。
>
>  thanks!
>    wfs fr china 20120608