[jira] Commented: (ZOOKEEPER-733) use netty to handle client connections

2010-06-14 Thread Kapil Thangavelu (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-733?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12878607#action_12878607
 ] 

Kapil Thangavelu commented on ZOOKEEPER-733:


Hi folks, I just wanted to note that we're really interested in the netty 
integration esp. to allow for secure communications. We don't have much by way 
of java developers, but we are available to help out with testing. We've 
successfully run our client test suite against the netty patch applied to trunk.


> use netty to handle client connections
> --
>
> Key: ZOOKEEPER-733
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-733
> Project: Zookeeper
>  Issue Type: Improvement
>Reporter: Benjamin Reed
> Attachments: accessive.jar, flowctl.zip, moved.zip, 
> QuorumTestFailed_sessionmoved_TRACE_LOG.txt.gz, ZOOKEEPER-733.patch, 
> ZOOKEEPER-733.patch, ZOOKEEPER-733.patch
>
>
> we currently have our own asynchronous NIO socket engine to be able to handle 
> lots of clients with a single thread. over time the engine has become more 
> complicated. we would also like the engine to use multiple threads on 
> machines with lots of cores. plus, we would like to be able to support things 
> like SSL. if we switch to netty, we can simplify our code and get the 
> previously mentioned benefits.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Commented: (ZOOKEEPER-733) use netty to handle client connections

2010-06-14 Thread Patrick Hunt (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-733?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12878631#action_12878631
 ] 

Patrick Hunt commented on ZOOKEEPER-733:


Great, thanks for the fb Kapil. There's still one serious issue that we know 
about with the patch, it's due to the way 
netty handles buffering. Our old server nio code does flow control by disabling 
read on the inbound sockets, but
netty aggressively buffers which means that the data's already been read by the 
time we disable read. We're working
on a solution for that. Subsequently we'll have some addl work re documentation 
and such. Also need to test out
encryption (ssl) support on the channels. It's in progress but I've been 
distracted by other things (incl the upcoming
summit). After that we also need to add netty support on the java client side, 
I'm not sure what we'll do on the c client
(to add ssl encrypt/cert support) but if you have any ideas on that we'd 
appreciate the help.

> use netty to handle client connections
> --
>
> Key: ZOOKEEPER-733
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-733
> Project: Zookeeper
>  Issue Type: Improvement
>Reporter: Benjamin Reed
> Attachments: accessive.jar, flowctl.zip, moved.zip, 
> QuorumTestFailed_sessionmoved_TRACE_LOG.txt.gz, ZOOKEEPER-733.patch, 
> ZOOKEEPER-733.patch, ZOOKEEPER-733.patch
>
>
> we currently have our own asynchronous NIO socket engine to be able to handle 
> lots of clients with a single thread. over time the engine has become more 
> complicated. we would also like the engine to use multiple threads on 
> machines with lots of cores. plus, we would like to be able to support things 
> like SSL. if we switch to netty, we can simplify our code and get the 
> previously mentioned benefits.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



Re: enhance zookeeper lock function in cocurrent condition

2010-06-14 Thread Patrick Hunt
One additional drawback of doing this is that more computational load 
will be pushed to the server. One reason we require the client to sort 
the child list is to reduce load on the server. We don't maintain the 
child list in sorted order (iirc) and adding something like "get first" 
will now require the server to do ordering. This increases latency for 
all users and is something we try to avoid.


Patrick

On 06/03/2010 12:03 PM, Benjamin Reed wrote:

if i understand your situation correctly, you have a lock that may have
more than 100,000 processes contending for a lock. since this can cause
a problem for getChildren, you want to have a way to get the server to
do it for you without returning everything.

the isFirst method would return true if you are first (sorted in utf8
order?) in the list of children. and you can set a watch on that
condition. what do the path and type arguments do?

ben

On 06/03/2010 03:20 AM, Joe Zou wrote:


Hi All:

Use zookeeper to build distribute lock is main feature. now implement
the lock function as below code:

Public void lock() throws InterruptedException{

Do{

If(path == null){

Path = zk.create(lockPrefix,null,acl, CreateMode./EPHEMERAL_SEQUENTIAL/)

}

List children = zk.getChildren(parentPath);

If(isFirst(children,path)){

Return;

}else{

Final CowntDownLatch latch = new CountDownLatch(1);

String nestestChild = findLastBefore(children,path);

If(zk.exist(nestestChildPath,new Watcher(Event){

Latch.countDown();

}) != null){

Latch.await();

}else{

//acquire lock success

Return;

}

}

}while(true);

}

In high concurrent case, lock node may need to get a big ephemeral
children nodes. So that the GetChildren may cause the package
exceeding the limitation(4MB as default), and also this would cause
the performance issue. To avoid the issue, I plan to add a new
interface isFirst for zeekeeper. I don’t know if it is useful as a
common usage, but I do think it should help a little bit in the
concurrent situation. Below is snippet of the code change, and the
attachment is full list of it.

Public void lock() throws InterruptedException{

Do{

If(path == null){

Path = zk.create(lockPrefix,null,acl, CreateMode./EPHEMERAL_SEQUENTIAL/)

}

Final CowntDownLatch latch = new CountDownLatch(1);

If(!Zk.isFirst(parentPath,path,Type,new Watcher(Event){

Latch.countDown();

})){

Latch.countDown()

}else{

//acquire success.

Return;

}

}while(true);

}

As we know, only the first node can aquire the lock success, so when
lock Type parent node remove child node, it need trigger the the
wather to notify the first node.

the second lock requirement is:

in our current project, each save need require multiple lock. In
distribute Env, it very maybe cause dead lock or lock starve. So we
need a stateLock, in the lock node, it keep the multiple states to
judge the node if acquire the lock or not. Example:

Client1: lock( id1,id2,id3) ->zdnode---01

Client2: lock(id2,id3) ->zdnode---02

Client3: lock(id4) ->zdnode---03

We need client2 need wait the lock until the client1 unlock lock. But
client 3 can acquire the lock at once. These judge logic in zookeeper
server. We add a LockState interface:

*public* *interface* LockState{

String /PATH_SEPERATOR/ = "/";

String /PATH_DELIMIT/ = "|";

*boolean* isConflict(LockState state);

*byte*[] getBytes();

}

Any new lock strategy can be added by implement the interface.

Attached is my code diff from 3.2.2 and the use lock some case.

Best Regards

Joe Zou






[jira] Updated: (ZOOKEEPER-767) Submitting Demo/Recipe Shared / Exclusive Lock Code

2010-06-14 Thread Sam Baskinger (JIRA)

 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-767?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sam Baskinger updated ZOOKEEPER-767:


Attachment: ZOOKEEPER-767.patch

1) Moving lock-ownership checks before we perform a zookeeper action.
2) Wrapping any code that may throw an exception after creating a lock file 
with a catch block that will delete the lock file and propagate the exception 
up the call stack.

> Submitting Demo/Recipe Shared / Exclusive Lock Code
> ---
>
> Key: ZOOKEEPER-767
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-767
> Project: Zookeeper
>  Issue Type: Improvement
>  Components: recipes
>Affects Versions: 3.3.0
>Reporter: Sam Baskinger
>Assignee: Sam Baskinger
>Priority: Minor
> Fix For: 3.4.0
>
> Attachments: ZOOKEEPER-767.patch, ZOOKEEPER-767.patch, 
> ZOOKEEPER-767.patch
>
>
> Networked Insights would like to share-back some code for shared/exclusive 
> locking that we are using in our labs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Commented: (ZOOKEEPER-733) use netty to handle client connections

2010-06-14 Thread Gustavo Niemeyer (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-733?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12878710#action_12878710
 ] 

Gustavo Niemeyer commented on ZOOKEEPER-733:


Hi Patrick,

Our most pressing need is actually on securing the client=>server communication 
indeed.  The basic need is simply to require a specific certificate to connect 
and communicate securely with the server.  Any clients which don't hold the 
certificate should be denied access.

We're certainly available for debating on any open issues you'd like to debate 
on around this.

> use netty to handle client connections
> --
>
> Key: ZOOKEEPER-733
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-733
> Project: Zookeeper
>  Issue Type: Improvement
>Reporter: Benjamin Reed
> Attachments: accessive.jar, flowctl.zip, moved.zip, 
> QuorumTestFailed_sessionmoved_TRACE_LOG.txt.gz, ZOOKEEPER-733.patch, 
> ZOOKEEPER-733.patch, ZOOKEEPER-733.patch
>
>
> we currently have our own asynchronous NIO socket engine to be able to handle 
> lots of clients with a single thread. over time the engine has become more 
> complicated. we would also like the engine to use multiple threads on 
> machines with lots of cores. plus, we would like to be able to support things 
> like SSL. if we switch to netty, we can simplify our code and get the 
> previously mentioned benefits.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Commented: (ZOOKEEPER-767) Submitting Demo/Recipe Shared / Exclusive Lock Code

2010-06-14 Thread Sam Baskinger (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-767?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12878717#action_12878717
 ] 

Sam Baskinger commented on ZOOKEEPER-767:
-

Thanks of the code-snippet Benjamin. You're absolutely right. Fixed #1.

Regarding #2, when getting a shared lock we ignore existing shared locks and 
only look for exclusive locks. Line 225 of the new patch has:

{noformat}
if (child.startsWith(EXLOCK)) { ...
{noformat}

If I'm not confusing the matter, while a single "exclusive lock" node 
represents a single exclusive lock, a series of contiguous "shared lock" nodes 
make up the total of a shared lock. I took some time to stare at the code in 
question and corresponding code in the getExclusiveLock() call and I think they 
are as we intended them. 

As for #3, wow, I fell asleep at the IDE for that one. Thank you. Any exception 
will result in a "roll back" of the lock file creation and the Exception is 
propagated up the stack.

Now, the larger question of the existing lock implementation, the existing 
{{WriteLock.java}} doesn't appear to closely follow the recipe (I'm reading 
http://hadoop.apache.org/zookeeper/docs/current/recipes.html#sc_recipes_Locks ) 
. What would prevent us from using it is the lack of first scheduling a lock 
(creating the node) and then doing the blocking logic. We realize this is 
potentially more work, but there may be some very high reader contention and we 
need to ensure that a single writer process doesn't starve. There is the added 
benefit of being able to observe the finite list of readers that must complete 
before the writer can lock.

Other than that, if the existing WriteLock had shared/exclusive coexisting and 
a block-until-timeout construct, we would probably prefer to spend our time 
integrating that code than crafting up our own. It may well be that the 
{{SharedExclusiveLock.java}} file has too many production concerns in it and 
doesn't suite the goal of a recipe file.

> Submitting Demo/Recipe Shared / Exclusive Lock Code
> ---
>
> Key: ZOOKEEPER-767
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-767
> Project: Zookeeper
>  Issue Type: Improvement
>  Components: recipes
>Affects Versions: 3.3.0
>Reporter: Sam Baskinger
>Assignee: Sam Baskinger
>Priority: Minor
> Fix For: 3.4.0
>
> Attachments: ZOOKEEPER-767.patch, ZOOKEEPER-767.patch, 
> ZOOKEEPER-767.patch
>
>
> Networked Insights would like to share-back some code for shared/exclusive 
> locking that we are using in our labs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Updated: (ZOOKEEPER-767) Submitting Demo/Recipe Shared / Exclusive Lock Code

2010-06-14 Thread Sam Baskinger (JIRA)

 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-767?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sam Baskinger updated ZOOKEEPER-767:


Status: Patch Available  (was: Open)

Fixed failure cases in the event of a thrown exception after requesting a lock. 
Optimization in returning earlier (and avoiding a needless create operation) 
when a lock file is already held by the caller. Also, some variable names were 
updated from the read/write to shared/exclusive.

> Submitting Demo/Recipe Shared / Exclusive Lock Code
> ---
>
> Key: ZOOKEEPER-767
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-767
> Project: Zookeeper
>  Issue Type: Improvement
>  Components: recipes
>Affects Versions: 3.3.0
>Reporter: Sam Baskinger
>Assignee: Sam Baskinger
>Priority: Minor
> Fix For: 3.4.0
>
> Attachments: ZOOKEEPER-767.patch, ZOOKEEPER-767.patch, 
> ZOOKEEPER-767.patch
>
>
> Networked Insights would like to share-back some code for shared/exclusive 
> locking that we are using in our labs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Commented: (ZOOKEEPER-767) Submitting Demo/Recipe Shared / Exclusive Lock Code

2010-06-14 Thread Hadoop QA (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-767?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12878729#action_12878729
 ] 

Hadoop QA commented on ZOOKEEPER-767:
-

+1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12447056/ZOOKEEPER-767.patch
  against trunk revision 953041.

+1 @author.  The patch does not contain any @author tags.

+1 tests included.  The patch appears to include 3 new or modified tests.

+1 javadoc.  The javadoc tool did not generate any warning messages.

+1 javac.  The applied patch does not increase the total number of javac 
compiler warnings.

+1 findbugs.  The patch does not introduce any new Findbugs warnings.

+1 release audit.  The applied patch does not increase the total number of 
release audit warnings.

+1 core tests.  The patch passed core unit tests.

+1 contrib tests.  The patch passed contrib unit tests.

Test results: 
http://hudson.zones.apache.org/hudson/job/Zookeeper-Patch-h1.grid.sp2.yahoo.net/115/testReport/
Findbugs warnings: 
http://hudson.zones.apache.org/hudson/job/Zookeeper-Patch-h1.grid.sp2.yahoo.net/115/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: 
http://hudson.zones.apache.org/hudson/job/Zookeeper-Patch-h1.grid.sp2.yahoo.net/115/console

This message is automatically generated.

> Submitting Demo/Recipe Shared / Exclusive Lock Code
> ---
>
> Key: ZOOKEEPER-767
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-767
> Project: Zookeeper
>  Issue Type: Improvement
>  Components: recipes
>Affects Versions: 3.3.0
>Reporter: Sam Baskinger
>Assignee: Sam Baskinger
>Priority: Minor
> Fix For: 3.4.0
>
> Attachments: ZOOKEEPER-767.patch, ZOOKEEPER-767.patch, 
> ZOOKEEPER-767.patch
>
>
> Networked Insights would like to share-back some code for shared/exclusive 
> locking that we are using in our labs.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Updated: (ZOOKEEPER-744) Add monitoring four-letter word

2010-06-14 Thread Savu Andrei (JIRA)

 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-744?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Savu Andrei updated ZOOKEEPER-744:
--

Attachment: ZOOKEEPER-744.patch

I've updated the patch and added a new the test for getWatchCount(). I'm not 
yet recording the time of the last election, I'm thinking about open a JIRA 
later for this. I want to move on and work on ZOOKEEPER-613. 

> Add monitoring four-letter word
> ---
>
> Key: ZOOKEEPER-744
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-744
> Project: Zookeeper
>  Issue Type: New Feature
>  Components: server
>Affects Versions: 3.4.0
>Reporter: Travis Crawford
>Assignee: Savu Andrei
> Fix For: 3.4.0
>
> Attachments: zk-ganglia.png, ZOOKEEPER-744.patch, 
> ZOOKEEPER-744.patch, ZOOKEEPER-744.patch
>
>
> Filing a feature request based on a zookeeper-user discussion.
> Zookeeper should have a new four-letter word that returns key-value pairs 
> appropriate for importing to a monitoring system (such as Ganglia which has a 
> large installed base)
> This command should initially export the following:
> (a) Count of instances in the ensemble.
> (b) Count of up-to-date instances in the ensemble.
> But be designed such that in the future additional data can be added. For 
> example, the output could define the statistic in a comment, then print a key 
> "space character" value line:
> """
> # Total number of instances in the ensemble
> zk_ensemble_instances_total 5
> # Number of instances currently participating in the quorum.
> zk_ensemble_instances_active 4
> """
> From the mailing list:
> """
> Date: Mon, 19 Apr 2010 12:10:44 -0700
> From: Patrick Hunt 
> To: zookeeper-u...@hadoop.apache.org
> Subject: Re: Recovery issue - how to debug?
> On 04/19/2010 11:55 AM, Travis Crawford wrote:
> > It would be a lot easier from the operations perspective if the leader
> > explicitly published some health stats:
> >
> > (a) Count of instances in the ensemble.
> > (b) Count of up-to-date instances in the ensemble.
> >
> > This would greatly simplify monitoring&  alerting - when an instance
> > falls behind one could configure their monitoring system to let
> > someone know and take a look at the logs.
> That's a great idea. Please enter a JIRA for this - a new 4 letter word 
> and JMX support. It would also be a great starter project for someone 
> interested in becoming more familiar with the server code.
> Patrick
> """

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Updated: (ZOOKEEPER-744) Add monitoring four-letter word

2010-06-14 Thread Savu Andrei (JIRA)

 [ 
https://issues.apache.org/jira/browse/ZOOKEEPER-744?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Savu Andrei updated ZOOKEEPER-744:
--

Status: Patch Available  (was: Open)

> Add monitoring four-letter word
> ---
>
> Key: ZOOKEEPER-744
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-744
> Project: Zookeeper
>  Issue Type: New Feature
>  Components: server
>Affects Versions: 3.4.0
>Reporter: Travis Crawford
>Assignee: Savu Andrei
> Fix For: 3.4.0
>
> Attachments: zk-ganglia.png, ZOOKEEPER-744.patch, 
> ZOOKEEPER-744.patch, ZOOKEEPER-744.patch
>
>
> Filing a feature request based on a zookeeper-user discussion.
> Zookeeper should have a new four-letter word that returns key-value pairs 
> appropriate for importing to a monitoring system (such as Ganglia which has a 
> large installed base)
> This command should initially export the following:
> (a) Count of instances in the ensemble.
> (b) Count of up-to-date instances in the ensemble.
> But be designed such that in the future additional data can be added. For 
> example, the output could define the statistic in a comment, then print a key 
> "space character" value line:
> """
> # Total number of instances in the ensemble
> zk_ensemble_instances_total 5
> # Number of instances currently participating in the quorum.
> zk_ensemble_instances_active 4
> """
> From the mailing list:
> """
> Date: Mon, 19 Apr 2010 12:10:44 -0700
> From: Patrick Hunt 
> To: zookeeper-u...@hadoop.apache.org
> Subject: Re: Recovery issue - how to debug?
> On 04/19/2010 11:55 AM, Travis Crawford wrote:
> > It would be a lot easier from the operations perspective if the leader
> > explicitly published some health stats:
> >
> > (a) Count of instances in the ensemble.
> > (b) Count of up-to-date instances in the ensemble.
> >
> > This would greatly simplify monitoring&  alerting - when an instance
> > falls behind one could configure their monitoring system to let
> > someone know and take a look at the logs.
> That's a great idea. Please enter a JIRA for this - a new 4 letter word 
> and JMX support. It would also be a great starter project for someone 
> interested in becoming more familiar with the server code.
> Patrick
> """

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Commented: (ZOOKEEPER-744) Add monitoring four-letter word

2010-06-14 Thread Hadoop QA (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-744?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12878793#action_12878793
 ] 

Hadoop QA commented on ZOOKEEPER-744:
-

+1 overall.  Here are the results of testing the latest attachment 
  http://issues.apache.org/jira/secure/attachment/12447082/ZOOKEEPER-744.patch
  against trunk revision 953041.

+1 @author.  The patch does not contain any @author tags.

+1 tests included.  The patch appears to include 16 new or modified tests.

+1 javadoc.  The javadoc tool did not generate any warning messages.

+1 javac.  The applied patch does not increase the total number of javac 
compiler warnings.

+1 findbugs.  The patch does not introduce any new Findbugs warnings.

+1 release audit.  The applied patch does not increase the total number of 
release audit warnings.

+1 core tests.  The patch passed core unit tests.

+1 contrib tests.  The patch passed contrib unit tests.

Test results: 
http://hudson.zones.apache.org/hudson/job/Zookeeper-Patch-h1.grid.sp2.yahoo.net/116/testReport/
Findbugs warnings: 
http://hudson.zones.apache.org/hudson/job/Zookeeper-Patch-h1.grid.sp2.yahoo.net/116/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
Console output: 
http://hudson.zones.apache.org/hudson/job/Zookeeper-Patch-h1.grid.sp2.yahoo.net/116/console

This message is automatically generated.

> Add monitoring four-letter word
> ---
>
> Key: ZOOKEEPER-744
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-744
> Project: Zookeeper
>  Issue Type: New Feature
>  Components: server
>Affects Versions: 3.4.0
>Reporter: Travis Crawford
>Assignee: Savu Andrei
> Fix For: 3.4.0
>
> Attachments: zk-ganglia.png, ZOOKEEPER-744.patch, 
> ZOOKEEPER-744.patch, ZOOKEEPER-744.patch
>
>
> Filing a feature request based on a zookeeper-user discussion.
> Zookeeper should have a new four-letter word that returns key-value pairs 
> appropriate for importing to a monitoring system (such as Ganglia which has a 
> large installed base)
> This command should initially export the following:
> (a) Count of instances in the ensemble.
> (b) Count of up-to-date instances in the ensemble.
> But be designed such that in the future additional data can be added. For 
> example, the output could define the statistic in a comment, then print a key 
> "space character" value line:
> """
> # Total number of instances in the ensemble
> zk_ensemble_instances_total 5
> # Number of instances currently participating in the quorum.
> zk_ensemble_instances_active 4
> """
> From the mailing list:
> """
> Date: Mon, 19 Apr 2010 12:10:44 -0700
> From: Patrick Hunt 
> To: zookeeper-u...@hadoop.apache.org
> Subject: Re: Recovery issue - how to debug?
> On 04/19/2010 11:55 AM, Travis Crawford wrote:
> > It would be a lot easier from the operations perspective if the leader
> > explicitly published some health stats:
> >
> > (a) Count of instances in the ensemble.
> > (b) Count of up-to-date instances in the ensemble.
> >
> > This would greatly simplify monitoring&  alerting - when an instance
> > falls behind one could configure their monitoring system to let
> > someone know and take a look at the logs.
> That's a great idea. Please enter a JIRA for this - a new 4 letter word 
> and JMX support. It would also be a great starter project for someone 
> interested in becoming more familiar with the server code.
> Patrick
> """

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Commented: (ZOOKEEPER-119) Reserve ".zookeeper" node for server use.

2010-06-14 Thread Savu Andrei (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-119?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12878802#action_12878802
 ] 

Savu Andrei commented on ZOOKEEPER-119:
---

I believe this JIRA should be closed because ZOOKEEPER-231 already adds 
"zookeeper" as a reserved node name when creating the hierarchy for quotas. 

> Reserve ".zookeeper" node for server use.
> -
>
> Key: ZOOKEEPER-119
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-119
> Project: Zookeeper
>  Issue Type: New Feature
>  Components: server
>Reporter: Patrick Hunt
> Attachments: ZOOKEEPER-119.patch
>
>
> The server should not allow clients to create .zookeeper nodes in the node 
> hierarchy. These nodes should be reserved for zk future use, some ideas:
> * /.zookeeper/proc
> * /.zookeeper/stats
> * /.zookeeper/...
> * /.../.zookeeper/... (disallow both at root as well as child nodes)

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.



[jira] Commented: (ZOOKEEPER-719) Add throttling to BookKeeper client

2010-06-14 Thread Benjamin Reed (JIRA)

[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-719?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12878850#action_12878850
 ] 

Benjamin Reed commented on ZOOKEEPER-719:
-

i think using a system property is still the easiest, but i'm fine with the 
set/get if you want to do it. you just need to make it thread safe.

> Add throttling to BookKeeper client
> ---
>
> Key: ZOOKEEPER-719
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-719
> Project: Zookeeper
>  Issue Type: Bug
>  Components: contrib-bookkeeper
>Affects Versions: 3.3.0
>Reporter: Flavio Paiva Junqueira
>Assignee: Flavio Paiva Junqueira
> Fix For: 3.4.0
>
> Attachments: ZOOKEEPER-719.patch, ZOOKEEPER-719.patch
>
>
> Add throttling to client to control the rate of operations to bookies. 

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.