فهرست منبع

ZOOKEEPER-29. Flexible quorums (flavio via mahadev)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/zookeeper/trunk@768390 13f79535-47bb-0310-9956-ffa450edef68
Mahadev Konar 16 سال پیش
والد
کامیت
cfdc76c1ee
31فایلهای تغییر یافته به همراه896 افزوده شده و 81 حذف شده
  1. 1 0
      CHANGES.txt
  2. 1 1
      docs/index.html
  3. 1 1
      docs/javaExample.html
  4. 1 1
      docs/linkmap.html
  5. 1 1
      docs/recipes.html
  6. 1 1
      docs/releasenotes.html
  7. 37 18
      docs/zookeeperAdmin.html
  8. 3 3
      docs/zookeeperAdmin.pdf
  9. 42 5
      docs/zookeeperInternals.html
  10. 15 4
      docs/zookeeperInternals.pdf
  11. 1 1
      docs/zookeeperJMX.html
  12. 1 1
      docs/zookeeperOver.html
  13. 1 1
      docs/zookeeperProgrammers.html
  14. 1 1
      docs/zookeeperQuotas.html
  15. 1 1
      docs/zookeeperStarted.html
  16. 1 1
      docs/zookeeperTutorial.html
  17. 30 1
      src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
  18. 36 0
      src/docs/src/documentation/content/xdocs/zookeeperInternals.xml
  19. 1 1
      src/java/main/org/apache/zookeeper/server/quorum/AckRequestProcessor.java
  20. 2 1
      src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java
  21. 8 6
      src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java
  22. 15 1
      src/java/main/org/apache/zookeeper/server/quorum/Follower.java
  23. 25 6
      src/java/main/org/apache/zookeeper/server/quorum/FollowerHandler.java
  24. 37 21
      src/java/main/org/apache/zookeeper/server/quorum/Leader.java
  25. 57 1
      src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
  26. 60 2
      src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
  27. 1 0
      src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java
  28. 249 0
      src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumHierarchical.java
  29. 46 0
      src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumMaj.java
  30. 32 0
      src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumVerifier.java
  31. 188 0
      src/java/test/org/apache/zookeeper/test/HierarchicalQuorumTest.java

+ 1 - 0
CHANGES.txt

@@ -112,6 +112,7 @@ NEW FEATURES:
   ZOOKEEPER-78. added a high level protocol/feature - for easy Leader
   ZOOKEEPER-78. added a high level protocol/feature - for easy Leader
   Election or exclusive Write Lock creation (mahadev via phunt)
   Election or exclusive Write Lock creation (mahadev via phunt)
 
 
+  ZOOKEEPER-29. Flexible quorums (flavio via mahadev) 
 
 
 Release 3.1.0 - 2009-02-06
 Release 3.1.0 - 2009-02-06
   
   

+ 1 - 1
docs/index.html

@@ -67,7 +67,7 @@
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 </li>
 </li>
 <li class="current">
 <li class="current">
-<a class="selected" href="index.html">ZooKeeper 3.1 Documentation</a>
+<a class="selected" href="index.html">ZooKeeper 3.2 Documentation</a>
 </li>
 </li>
 </ul>
 </ul>
 <!--+
 <!--+

+ 1 - 1
docs/javaExample.html

@@ -67,7 +67,7 @@
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 </li>
 </li>
 <li class="current">
 <li class="current">
-<a class="selected" href="index.html">ZooKeeper 3.1 Documentation</a>
+<a class="selected" href="index.html">ZooKeeper 3.2 Documentation</a>
 </li>
 </li>
 </ul>
 </ul>
 <!--+
 <!--+

+ 1 - 1
docs/linkmap.html

@@ -67,7 +67,7 @@
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 </li>
 </li>
 <li class="current">
 <li class="current">
-<a class="selected" href="index.html">ZooKeeper 3.1 Documentation</a>
+<a class="selected" href="index.html">ZooKeeper 3.2 Documentation</a>
 </li>
 </li>
 </ul>
 </ul>
 <!--+
 <!--+

+ 1 - 1
docs/recipes.html

@@ -67,7 +67,7 @@
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 </li>
 </li>
 <li class="current">
 <li class="current">
-<a class="selected" href="index.html">ZooKeeper 3.1 Documentation</a>
+<a class="selected" href="index.html">ZooKeeper 3.2 Documentation</a>
 </li>
 </li>
 </ul>
 </ul>
 <!--+
 <!--+

+ 1 - 1
docs/releasenotes.html

@@ -67,7 +67,7 @@
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 </li>
 </li>
 <li class="current">
 <li class="current">
-<a class="selected" href="index.html">ZooKeeper 3.1 Documentation</a>
+<a class="selected" href="index.html">ZooKeeper 3.2 Documentation</a>
 </li>
 </li>
 </ul>
 </ul>
 <!--+
 <!--+

+ 37 - 18
docs/zookeeperAdmin.html

@@ -67,7 +67,7 @@
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 </li>
 </li>
 <li class="current">
 <li class="current">
-<a class="selected" href="index.html">ZooKeeper 3.1 Documentation</a>
+<a class="selected" href="index.html">ZooKeeper 3.2 Documentation</a>
 </li>
 </li>
 </ul>
 </ul>
 <!--+
 <!--+
@@ -1107,10 +1107,38 @@ server.3=zoo3:2888:3888</span>
               with ZooKeeper. If followers fall too far behind a leader, they
               with ZooKeeper. If followers fall too far behind a leader, they
               will be dropped.</p>
               will be dropped.</p>
 </dd>
 </dd>
+
+          
+<dt>
+<term>group.x=nnnnn[:nnnnn]</term>
+</dt>
+<dd>
+<p>(No Java system property)</p>
+<p>Enables a hierarchical quorum construction."x" is a group identifier
+              and the numbers following the "=" sign correspond to server identifiers. 
+              The left-hand side of the assignment is a colon-separated list of server
+              identifiers. Note that groups must be disjoint and the union of all groups
+              must be the ZooKeeper ensemble. </p>
+</dd>
+
+	
+<dt>
+<term>weight.x=nnnnn</term>
+</dt>
+<dd>
+<p>(No Java system property)</p>
+<p>Used along with "group", it assigns a weight to a server when
+              forming quorums. Such a value corresponds to the weight of a server
+              when voting. There are a few parts of ZooKeeper that require voting
+              such as leader election and the atomic broadcast protocol. By default
+              the weight of server is 1. If the configuration defines groups, but not
+              weights, then a value of 1 will be assigned to all servers.  
+              </p>
+</dd>
         
         
 </dl>
 </dl>
 <p></p>
 <p></p>
-<a name="N10368"></a><a name="Unsafe+Options"></a>
+<a name="N1037A"></a><a name="Unsafe+Options"></a>
 <h4>Unsafe Options</h4>
 <h4>Unsafe Options</h4>
 <p>The following options can be useful, but be careful when you use
 <p>The following options can be useful, but be careful when you use
         them. The risk of each is explained along with the explanation of what
         them. The risk of each is explained along with the explanation of what
@@ -1155,7 +1183,7 @@ server.3=zoo3:2888:3888</span>
 </dd>
 </dd>
         
         
 </dl>
 </dl>
-<a name="N1039A"></a><a name="sc_zkCommands"></a>
+<a name="N103AC"></a><a name="sc_zkCommands"></a>
 <h3 class="h4">ZooKeeper Commands: The Four Letter Words</h3>
 <h3 class="h4">ZooKeeper Commands: The Four Letter Words</h3>
 <p>ZooKeeper responds to a small set of commands. Each command is
 <p>ZooKeeper responds to a small set of commands. Each command is
       composed of four letters. You issue the commands to ZooKeeper via telnet
       composed of four letters. You issue the commands to ZooKeeper via telnet
@@ -1179,15 +1207,6 @@ server.3=zoo3:2888:3888</span>
 </dd>
 </dd>
 
 
         
         
-<dt>
-<term>kill</term>
-</dt>
-<dd>
-<p>Shuts down the server. This must be issued from the machine
-            the ZooKeeper server is running on.</p>
-</dd>
-
-        
 <dt>
 <dt>
 <term>reqs</term>
 <term>reqs</term>
 </dt>
 </dt>
@@ -1228,7 +1247,7 @@ server.3=zoo3:2888:3888</span>
 <pre class="code">$ echo ruok | nc 127.0.0.1 5111
 <pre class="code">$ echo ruok | nc 127.0.0.1 5111
 imok
 imok
 </pre>
 </pre>
-<a name="N103E1"></a><a name="sc_dataFileManagement"></a>
+<a name="N103EC"></a><a name="sc_dataFileManagement"></a>
 <h3 class="h4">Data File Management</h3>
 <h3 class="h4">Data File Management</h3>
 <p>ZooKeeper stores its data in a data directory and its transaction
 <p>ZooKeeper stores its data in a data directory and its transaction
       log in a transaction log directory. By default these two directories are
       log in a transaction log directory. By default these two directories are
@@ -1236,7 +1255,7 @@ imok
       transaction log files in a separate directory than the data files.
       transaction log files in a separate directory than the data files.
       Throughput increases and latency decreases when transaction logs reside
       Throughput increases and latency decreases when transaction logs reside
       on a dedicated log devices.</p>
       on a dedicated log devices.</p>
-<a name="N103EA"></a><a name="The+Data+Directory"></a>
+<a name="N103F5"></a><a name="The+Data+Directory"></a>
 <h4>The Data Directory</h4>
 <h4>The Data Directory</h4>
 <p>This directory has two files in it:</p>
 <p>This directory has two files in it:</p>
 <ul>
 <ul>
@@ -1282,14 +1301,14 @@ imok
         idempotent nature of its updates. By replaying the transaction log
         idempotent nature of its updates. By replaying the transaction log
         against fuzzy snapshots ZooKeeper gets the state of the system at the
         against fuzzy snapshots ZooKeeper gets the state of the system at the
         end of the log.</p>
         end of the log.</p>
-<a name="N10426"></a><a name="The+Log+Directory"></a>
+<a name="N10431"></a><a name="The+Log+Directory"></a>
 <h4>The Log Directory</h4>
 <h4>The Log Directory</h4>
 <p>The Log Directory contains the ZooKeeper transaction logs.
 <p>The Log Directory contains the ZooKeeper transaction logs.
         Before any update takes place, ZooKeeper ensures that the transaction
         Before any update takes place, ZooKeeper ensures that the transaction
         that represents the update is written to non-volatile storage. A new
         that represents the update is written to non-volatile storage. A new
         log file is started each time a snapshot is begun. The log file's
         log file is started each time a snapshot is begun. The log file's
         suffix is the first zxid written to that log.</p>
         suffix is the first zxid written to that log.</p>
-<a name="N10430"></a><a name="sc_filemanagement"></a>
+<a name="N1043B"></a><a name="sc_filemanagement"></a>
 <h4>File Management</h4>
 <h4>File Management</h4>
 <p>The format of snapshot and log files does not change between
 <p>The format of snapshot and log files does not change between
         standalone ZooKeeper servers and different configurations of
         standalone ZooKeeper servers and different configurations of
@@ -1309,7 +1328,7 @@ imok
         this document for more details on setting a retention policy
         this document for more details on setting a retention policy
         and maintenance of ZooKeeper storage.
         and maintenance of ZooKeeper storage.
         </p>
         </p>
-<a name="N10445"></a><a name="sc_commonProblems"></a>
+<a name="N10450"></a><a name="sc_commonProblems"></a>
 <h3 class="h4">Things to Avoid</h3>
 <h3 class="h4">Things to Avoid</h3>
 <p>Here are some common problems you can avoid by configuring
 <p>Here are some common problems you can avoid by configuring
       ZooKeeper correctly:</p>
       ZooKeeper correctly:</p>
@@ -1363,7 +1382,7 @@ imok
 </dd>
 </dd>
       
       
 </dl>
 </dl>
-<a name="N10469"></a><a name="sc_bestPractices"></a>
+<a name="N10474"></a><a name="sc_bestPractices"></a>
 <h3 class="h4">Best Practices</h3>
 <h3 class="h4">Best Practices</h3>
 <p>For best results, take note of the following list of good
 <p>For best results, take note of the following list of good
       Zookeeper practices. <em>[tbd...]</em>
       Zookeeper practices. <em>[tbd...]</em>

تفاوت فایلی نمایش داده نمی شود زیرا این فایل بسیار بزرگ است
+ 3 - 3
docs/zookeeperAdmin.pdf


+ 42 - 5
docs/zookeeperInternals.html

@@ -67,7 +67,7 @@
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 </li>
 </li>
 <li class="current">
 <li class="current">
-<a class="selected" href="index.html">ZooKeeper 3.1 Documentation</a>
+<a class="selected" href="index.html">ZooKeeper 3.2 Documentation</a>
 </li>
 </li>
 </ul>
 </ul>
 <!--+
 <!--+
@@ -211,6 +211,9 @@ document.write("Last Published: " + document.lastModified);
 </ul>
 </ul>
 </li>
 </li>
 <li>
 <li>
+<a href="#sc_quorum">Quorums</a>
+</li>
+<li>
 <a href="#sc_logging">Logging</a>
 <a href="#sc_logging">Logging</a>
 <ul class="minitoc">
 <ul class="minitoc">
 <li>
 <li>
@@ -617,8 +620,42 @@ proposals and to not worry about duplicate proposals for a given zxid.
 </div>
 </div>
 
 
 
 
+<a name="N10134"></a><a name="sc_quorum"></a>
+<h2 class="h3">Quorums</h2>
+<div class="section">
+<p>
+Atomic broadcast and leader election use the notion of quorum to guarantee a consistent
+view of the system. By default, ZooKeeper uses majority quorums, which means that every
+voting that happens in one of these protocols requires a majority to vote on. One example is
+acknowledging a leader proposal: the leader can only commit once it receives an
+acknowledgement from a quorum of servers.
+</p>
+<p>
+If we extract the properties that we really need from our use of majorities, we have that we only
+need to guarantee that groups of processes used to validate an operation by voting (e.g., acknowledging
+a leader proposal) pairwise intersect in at least one server. Using majorities guarantees such a property.
+However, there are other ways of constructing quorums different from majorities. For example, we can assign
+weights to the votes of servers, and say that the votes of some servers are more important. To obtain a quorum,
+we get enough votes so that the sum of weights of all votes is larger than half of the total sum of all weights.    
+</p>
+<p>
+A different construction that uses weights and is useful in wide-area deployments (co-locations) is a hierarchical
+one. With this construction, we split the servers into disjoint groups and assign weights to processes. To form 
+a quorum, we have to get a hold of enough servers from a majority of groups G, such that for each group g in G,
+the sum of votes from g is larger than half of the sum of weights in g. Interestingly, this construction enables
+smaller quorums. If we have, for example, 9 servers, we split them into 3 groups, and assign a weight of 1 to each
+server, then we are able to form quorums of size 4. Note that two subsets of processes composed each of a majority
+of servers from each of a majority of groups necessarily have a non-empty intersection. It is reasonable to expect
+that a majority of co-locations will have a majority of servers available with high probability. 
+</p>
+<p>
+With ZooKeeper, we provide a user with the ability of configuring servers to use majority quorums, weights, or a 
+hierarchy of groups.
+</p>
+</div>
+
 
 
-<a name="N10134"></a><a name="sc_logging"></a>
+<a name="N10147"></a><a name="sc_logging"></a>
 <h2 class="h3">Logging</h2>
 <h2 class="h3">Logging</h2>
 <div class="section">
 <div class="section">
 <p>
 <p>
@@ -629,10 +666,10 @@ ZooKeeper, see the <a href="zookeeperAdmin.html#sc_logging">Logging</a> section
 of the <a href="zookeeperAdmin.html">ZooKeeper Administrator's Guide.</a>
 of the <a href="zookeeperAdmin.html">ZooKeeper Administrator's Guide.</a>
 
 
 </p>
 </p>
-<a name="N10149"></a><a name="sc_developerGuidelines"></a>
+<a name="N1015C"></a><a name="sc_developerGuidelines"></a>
 <h3 class="h4">Developer Guidelines</h3>
 <h3 class="h4">Developer Guidelines</h3>
 <p>Please follow these guidelines when submitting code. Patch reviewers will look for the following:</p>
 <p>Please follow these guidelines when submitting code. Patch reviewers will look for the following:</p>
-<a name="N10151"></a><a name="sc_rightLevel"></a>
+<a name="N10164"></a><a name="sc_rightLevel"></a>
 <h4>Logging at the Right Level</h4>
 <h4>Logging at the Right Level</h4>
 <p>
 <p>
 There are <a href="http://logging.apache.org/log4j/1.2/apidocs/org/apache/log4j/Level.html#FATAL">6 levels of logging in log4j</a>. 
 There are <a href="http://logging.apache.org/log4j/1.2/apidocs/org/apache/log4j/Level.html#FATAL">6 levels of logging in log4j</a>. 
@@ -667,7 +704,7 @@ It's important to pick the right one. In order of higher to lower severity:</p>
 <p>
 <p>
 ZooKeeper is typically run in production such that log messages of INFO level 
 ZooKeeper is typically run in production such that log messages of INFO level 
 severity and higher (more severe) are output to the log.</p>
 severity and higher (more severe) are output to the log.</p>
-<a name="N1017C"></a><a name="sc_log4jIdioms"></a>
+<a name="N1018F"></a><a name="sc_log4jIdioms"></a>
 <h4>Use of Standard log4j Idioms</h4>
 <h4>Use of Standard log4j Idioms</h4>
 <p>
 <p>
 <em>Static Message Logging</em>
 <em>Static Message Logging</em>

تفاوت فایلی نمایش داده نمی شود زیرا این فایل بسیار بزرگ است
+ 15 - 4
docs/zookeeperInternals.pdf


+ 1 - 1
docs/zookeeperJMX.html

@@ -67,7 +67,7 @@
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 </li>
 </li>
 <li class="current">
 <li class="current">
-<a class="selected" href="index.html">ZooKeeper 3.1 Documentation</a>
+<a class="selected" href="index.html">ZooKeeper 3.2 Documentation</a>
 </li>
 </li>
 </ul>
 </ul>
 <!--+
 <!--+

+ 1 - 1
docs/zookeeperOver.html

@@ -67,7 +67,7 @@
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 </li>
 </li>
 <li class="current">
 <li class="current">
-<a class="selected" href="index.html">ZooKeeper 3.1 Documentation</a>
+<a class="selected" href="index.html">ZooKeeper 3.2 Documentation</a>
 </li>
 </li>
 </ul>
 </ul>
 <!--+
 <!--+

+ 1 - 1
docs/zookeeperProgrammers.html

@@ -67,7 +67,7 @@
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 </li>
 </li>
 <li class="current">
 <li class="current">
-<a class="selected" href="index.html">ZooKeeper 3.1 Documentation</a>
+<a class="selected" href="index.html">ZooKeeper 3.2 Documentation</a>
 </li>
 </li>
 </ul>
 </ul>
 <!--+
 <!--+

+ 1 - 1
docs/zookeeperQuotas.html

@@ -67,7 +67,7 @@
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 </li>
 </li>
 <li class="current">
 <li class="current">
-<a class="selected" href="index.html">ZooKeeper 3.1 Documentation</a>
+<a class="selected" href="index.html">ZooKeeper 3.2 Documentation</a>
 </li>
 </li>
 </ul>
 </ul>
 <!--+
 <!--+

+ 1 - 1
docs/zookeeperStarted.html

@@ -67,7 +67,7 @@
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 </li>
 </li>
 <li class="current">
 <li class="current">
-<a class="selected" href="index.html">ZooKeeper 3.1 Documentation</a>
+<a class="selected" href="index.html">ZooKeeper 3.2 Documentation</a>
 </li>
 </li>
 </ul>
 </ul>
 <!--+
 <!--+

+ 1 - 1
docs/zookeeperTutorial.html

@@ -67,7 +67,7 @@
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 <a class="unselected" href="http://wiki.apache.org/hadoop/ZooKeeper">Wiki</a>
 </li>
 </li>
 <li class="current">
 <li class="current">
-<a class="selected" href="index.html">ZooKeeper 3.1 Documentation</a>
+<a class="selected" href="index.html">ZooKeeper 3.2 Documentation</a>
 </li>
 </li>
 </ul>
 </ul>
 <!--+
 <!--+

+ 30 - 1
src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml

@@ -775,8 +775,37 @@ server.3=zoo3:2888:3888</computeroutput></para>
               will be dropped.</para>
               will be dropped.</para>
             </listitem>
             </listitem>
           </varlistentry>
           </varlistentry>
-        </variablelist>
 
 
+          <varlistentry>
+            <term>group.x=nnnnn[:nnnnn]</term>
+
+            <listitem>
+              <para>(No Java system property)</para>
+
+              <para>Enables a hierarchical quorum construction."x" is a group identifier
+              and the numbers following the "=" sign correspond to server identifiers. 
+              The left-hand side of the assignment is a colon-separated list of server
+              identifiers. Note that groups must be disjoint and the union of all groups
+              must be the ZooKeeper ensemble. </para>
+            </listitem>
+          </varlistentry>
+
+	<varlistentry>
+            <term>weight.x=nnnnn</term>
+
+            <listitem>
+              <para>(No Java system property)</para>
+
+              <para>Used along with "group", it assigns a weight to a server when
+              forming quorums. Such a value corresponds to the weight of a server
+              when voting. There are a few parts of ZooKeeper that require voting
+              such as leader election and the atomic broadcast protocol. By default
+              the weight of server is 1. If the configuration defines groups, but not
+              weights, then a value of 1 will be assigned to all servers.  
+              </para>
+            </listitem>
+          </varlistentry>
+        </variablelist>
         <para></para>
         <para></para>
       </section>
       </section>
 
 

+ 36 - 0
src/docs/src/documentation/content/xdocs/zookeeperInternals.xml

@@ -356,6 +356,42 @@ proposals and to not worry about duplicate proposals for a given zxid.
 
 
 </section>
 </section>
 
 
+<section id="sc_quorum">
+<title>Quorums</title>
+
+<para>
+Atomic broadcast and leader election use the notion of quorum to guarantee a consistent
+view of the system. By default, ZooKeeper uses majority quorums, which means that every
+voting that happens in one of these protocols requires a majority to vote on. One example is
+acknowledging a leader proposal: the leader can only commit once it receives an
+acknowledgement from a quorum of servers.
+</para>
+
+<para>
+If we extract the properties that we really need from our use of majorities, we have that we only
+need to guarantee that groups of processes used to validate an operation by voting (e.g., acknowledging
+a leader proposal) pairwise intersect in at least one server. Using majorities guarantees such a property.
+However, there are other ways of constructing quorums different from majorities. For example, we can assign
+weights to the votes of servers, and say that the votes of some servers are more important. To obtain a quorum,
+we get enough votes so that the sum of weights of all votes is larger than half of the total sum of all weights.    
+</para>
+
+<para>
+A different construction that uses weights and is useful in wide-area deployments (co-locations) is a hierarchical
+one. With this construction, we split the servers into disjoint groups and assign weights to processes. To form 
+a quorum, we have to get a hold of enough servers from a majority of groups G, such that for each group g in G,
+the sum of votes from g is larger than half of the sum of weights in g. Interestingly, this construction enables
+smaller quorums. If we have, for example, 9 servers, we split them into 3 groups, and assign a weight of 1 to each
+server, then we are able to form quorums of size 4. Note that two subsets of processes composed each of a majority
+of servers from each of a majority of groups necessarily have a non-empty intersection. It is reasonable to expect
+that a majority of co-locations will have a majority of servers available with high probability. 
+</para>  
+
+<para>
+With ZooKeeper, we provide a user with the ability of configuring servers to use majority quorums, weights, or a 
+hierarchy of groups.
+</para>
+</section>
 
 
 <section id="sc_logging">
 <section id="sc_logging">
 
 

+ 1 - 1
src/java/main/org/apache/zookeeper/server/quorum/AckRequestProcessor.java

@@ -36,7 +36,7 @@ class AckRequestProcessor implements RequestProcessor {
      * Forward the request as an ACK to the leader
      * Forward the request as an ACK to the leader
      */
      */
     public void processRequest(Request request) {
     public void processRequest(Request request) {
-        leader.processAck(request.zxid, null);
+        leader.processAck(leader.self.getId(), request.zxid, null);
     }
     }
 
 
     public void shutdown() {
     public void shutdown() {

+ 2 - 1
src/java/main/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java

@@ -789,8 +789,9 @@ public class AuthFastLeaderElection implements Election {
     private boolean termPredicate(HashMap<InetSocketAddress, Vote> votes,
     private boolean termPredicate(HashMap<InetSocketAddress, Vote> votes,
             long l, long zxid) {
             long l, long zxid) {
 
 
-        int count = 0;
+
         Collection<Vote> votesCast = votes.values();
         Collection<Vote> votesCast = votes.values();
+        int count = 0;
         /*
         /*
          * First make the views consistent. Sometimes peers will have different
          * First make the views consistent. Sometimes peers will have different
          * zxids for a server depending on timing.
          * zxids for a server depending on timing.

+ 8 - 6
src/java/main/org/apache/zookeeper/server/quorum/FastLeaderElection.java

@@ -21,7 +21,9 @@ package org.apache.zookeeper.server.quorum;
 
 
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.Collection;
+import java.util.Map;
 import java.util.HashMap;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 
 
@@ -454,19 +456,19 @@ public class FastLeaderElection implements Election {
             HashMap<Long, Vote> votes, 
             HashMap<Long, Vote> votes, 
             Vote vote) {
             Vote vote) {
 
 
-        Collection<Vote> votesCast = votes.values();
-        int count = 0;
+        HashSet<Long> set = new HashSet<Long>();
         
         
         /*
         /*
          * First make the views consistent. Sometimes peers will have
          * First make the views consistent. Sometimes peers will have
          * different zxids for a server depending on timing.
          * different zxids for a server depending on timing.
          */
          */
-        for (Vote v : votesCast) {
-            if (v.equals(vote))
-                count++;
+        for (Map.Entry<Long,Vote> entry : votes.entrySet()) {
+            if (vote.equals(entry.getValue())){
+                set.add(entry.getKey());
+            }
         }
         }
                       
                       
-        if (count > (self.quorumPeers.size() / 2))
+        if(self.getQuorumVerifier().containsQuorum(set))
             return true;
             return true;
         else
         else
             return false;
             return false;

+ 15 - 1
src/java/main/org/apache/zookeeper/server/quorum/Follower.java

@@ -170,10 +170,23 @@ public class Follower {
                         sock.getInputStream()));
                         sock.getInputStream()));
                 bufferedOutput = new BufferedOutputStream(sock.getOutputStream());
                 bufferedOutput = new BufferedOutputStream(sock.getOutputStream());
                 leaderOs = BinaryOutputArchive.getArchive(bufferedOutput);
                 leaderOs = BinaryOutputArchive.getArchive(bufferedOutput);
+                
+                /*
+                 * Send follower info, including last zxid and sid
+                 */
                 QuorumPacket qp = new QuorumPacket();
                 QuorumPacket qp = new QuorumPacket();
-                qp.setType(Leader.LASTZXID);
+                qp.setType(Leader.FOLLOWERINFO);
                 long sentLastZxid = self.getLastLoggedZxid();
                 long sentLastZxid = self.getLastLoggedZxid();
                 qp.setZxid(sentLastZxid);
                 qp.setZxid(sentLastZxid);
+                
+                /*
+                 * Add sid to payload
+                 */
+                ByteArrayOutputStream bsid = new ByteArrayOutputStream();
+                DataOutputStream dsid = new DataOutputStream(bsid);
+                dsid.writeLong(self.getId());
+                qp.setData(bsid.toByteArray());
+                
                 writePacket(qp, true);
                 writePacket(qp, true);
                 readPacket(qp);
                 readPacket(qp);
                 long newLeaderZxid = qp.getZxid();
                 long newLeaderZxid = qp.getZxid();
@@ -222,6 +235,7 @@ public class Follower {
                 writePacket(ack, true);
                 writePacket(ack, true);
                 sock.setSoTimeout(self.tickTime * self.syncLimit);
                 sock.setSoTimeout(self.tickTime * self.syncLimit);
                 zk.startup();
                 zk.startup();
+                
                 while (self.running) {
                 while (self.running) {
                     readPacket(qp);
                     readPacket(qp);
                     switch (qp.getType()) {
                     switch (qp.getType()) {

+ 25 - 6
src/java/main/org/apache/zookeeper/server/quorum/FollowerHandler.java

@@ -52,6 +52,15 @@ public class FollowerHandler extends Thread {
     final Leader leader;
     final Leader leader;
 
 
     long tickOfLastAck;
     long tickOfLastAck;
+    
+    /**
+     * ZooKeeper server identifier of this follower
+     */
+    protected long sid = 0;
+    
+    long getSid(){
+        return sid;
+    }
 
 
     /**
     /**
      * The packets to be sent to the follower
      * The packets to be sent to the follower
@@ -129,6 +138,7 @@ public class FollowerHandler extends Thread {
         String type = null;
         String type = null;
         String mess = null;
         String mess = null;
         Record txn = null;
         Record txn = null;
+        
         switch (p.getType()) {
         switch (p.getType()) {
         case Leader.ACK:
         case Leader.ACK:
             type = "ACK";
             type = "ACK";
@@ -136,8 +146,8 @@ public class FollowerHandler extends Thread {
         case Leader.COMMIT:
         case Leader.COMMIT:
             type = "COMMIT";
             type = "COMMIT";
             break;
             break;
-        case Leader.LASTZXID:
-            type = "LASTZXID";
+        case Leader.FOLLOWERINFO:
+            type = "FOLLOWERINFO";
             break;
             break;
         case Leader.NEWLEADER:
         case Leader.NEWLEADER:
             type = "NEWLEADER";
             type = "NEWLEADER";
@@ -200,12 +210,21 @@ public class FollowerHandler extends Thread {
 
 
             QuorumPacket qp = new QuorumPacket();
             QuorumPacket qp = new QuorumPacket();
             ia.readRecord(qp, "packet");
             ia.readRecord(qp, "packet");
-            if (qp.getType() != Leader.LASTZXID) {
-                LOG.error("First packet " + qp.toString()
-                        + " is not LASTZXID!");
+            if(qp.getType() != leader.FOLLOWERINFO){
+            	LOG.error("First packet " + qp.toString()
+                        + " is not FOLLOWERINFO!");
                 return;
                 return;
             }
             }
+            if (qp.getData() != null) {
+            	ByteBuffer bbsid = ByteBuffer.wrap(qp.getData());
+                this.sid = bbsid.getLong();
+            } else {
+            	this.sid = leader.followerCounter.getAndDecrement();
+            }
+            LOG.info("The follower sid: " + this.sid);
+            
             long peerLastZxid = qp.getZxid();
             long peerLastZxid = qp.getZxid();
+            
             int packetToSend = Leader.SNAP;
             int packetToSend = Leader.SNAP;
             boolean logTxns = true;
             boolean logTxns = true;
 
 
@@ -307,7 +326,7 @@ public class FollowerHandler extends Thread {
 
 
                 switch (qp.getType()) {
                 switch (qp.getType()) {
                 case Leader.ACK:
                 case Leader.ACK:
-                    leader.processAck(qp.getZxid(), sock.getLocalSocketAddress());
+                    leader.processAck(this.sid, qp.getZxid(), sock.getLocalSocketAddress());
                     break;
                     break;
                 case Leader.PING:
                 case Leader.PING:
                     // Process the touches
                     // Process the touches

+ 37 - 21
src/java/main/org/apache/zookeeper/server/quorum/Leader.java

@@ -20,6 +20,7 @@ package org.apache.zookeeper.server.quorum;
 
 
 import java.io.ByteArrayOutputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.IOException;
+import java.lang.StringBuffer;
 import java.net.BindException;
 import java.net.BindException;
 import java.net.ServerSocket;
 import java.net.ServerSocket;
 import java.net.Socket;
 import java.net.Socket;
@@ -31,6 +32,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentMap;
@@ -55,7 +57,7 @@ public class Leader {
     static public class Proposal {
     static public class Proposal {
         public QuorumPacket packet;
         public QuorumPacket packet;
 
 
-        public int ackCount;
+        public HashSet<Long> ackSet = new HashSet<Long>();
 
 
         public Request request;
         public Request request;
 
 
@@ -80,7 +82,9 @@ public class Leader {
     
     
     //Pending sync requests
     //Pending sync requests
     public HashMap<Long,List<FollowerSyncRequest>> pendingSyncs = new HashMap<Long,List<FollowerSyncRequest>>();
     public HashMap<Long,List<FollowerSyncRequest>> pendingSyncs = new HashMap<Long,List<FollowerSyncRequest>>();
-               
+    
+    //Follower counter
+    AtomicLong followerCounter = new AtomicLong(-1);
     /**
     /**
      * Adds follower to the leader.
      * Adds follower to the leader.
      * 
      * 
@@ -149,10 +153,10 @@ public class Leader {
     final static int NEWLEADER = 10;
     final static int NEWLEADER = 10;
 
 
     /**
     /**
-     * This message type is sent by a follower to indicate the last zxid in its
-     * log.
+     * This message type is sent by a follower to pass the last zxid. This is here
+     * for backward compatibility purposes.
      */
      */
-    final static int LASTZXID = 11;
+    final static int FOLLOWERINFO = 11;
 
 
     /**
     /**
      * This message type is sent by the leader to indicate that the follower is
      * This message type is sent by the leader to indicate that the follower is
@@ -198,7 +202,7 @@ public class Leader {
      * between the leader and the follower.
      * between the leader and the follower.
      */
      */
     final static int SYNC = 7;
     final static int SYNC = 7;
-     
+    
     private ConcurrentMap<Long, Proposal> outstandingProposals = new ConcurrentHashMap<Long, Proposal>();
     private ConcurrentMap<Long, Proposal> outstandingProposals = new ConcurrentHashMap<Long, Proposal>();
 
 
     ConcurrentLinkedQueue<Proposal> toBeApplied = new ConcurrentLinkedQueue<Proposal>();
     ConcurrentLinkedQueue<Proposal> toBeApplied = new ConcurrentLinkedQueue<Proposal>();
@@ -275,15 +279,23 @@ public class Leader {
             // We have to get at least a majority of servers in sync with
             // We have to get at least a majority of servers in sync with
             // us. We do this by waiting for the NEWLEADER packet to get
             // us. We do this by waiting for the NEWLEADER packet to get
             // acknowledged
             // acknowledged
-            newLeaderProposal.ackCount++;
-            while (newLeaderProposal.ackCount <= self.quorumPeers.size() / 2) {
+            newLeaderProposal.ackSet.add(self.getId());
+            while (!self.getQuorumVerifier().containsQuorum(newLeaderProposal.ackSet)){
+            //while (newLeaderProposal.ackCount <= self.quorumPeers.size() / 2) {
                 if (self.tick > self.initLimit) {
                 if (self.tick > self.initLimit) {
                     // Followers aren't syncing fast enough,
                     // Followers aren't syncing fast enough,
                     // renounce leadership!
                     // renounce leadership!
-                    shutdown("Waiting for " + (self.quorumPeers.size() / 2)
-                            + " followers, only synced with "
-                            + newLeaderProposal.ackCount);
-                    if (followers.size() >= self.quorumPeers.size() / 2) {
+                    StringBuffer ackToString = new StringBuffer();
+                    for(Long id : newLeaderProposal.ackSet)
+                        ackToString.append(id + ": ");
+                    
+                    shutdown("Waiting for a quorum of followers, only synced with: " + ackToString);
+                    HashSet<Long> followerSet = new HashSet<Long>();
+                    for(FollowerHandler f : followers)
+                        followerSet.add(f.getSid());
+                    
+                    if (self.getQuorumVerifier().containsQuorum(followerSet)) {
+                    //if (followers.size() >= self.quorumPeers.size() / 2) {
                         LOG.warn("Enough followers present. "+
                         LOG.warn("Enough followers present. "+
                                 "Perhaps the initTicks need to be increased.");
                                 "Perhaps the initTicks need to be increased.");
                     }
                     }
@@ -312,24 +324,29 @@ public class Leader {
                     self.tick++;
                     self.tick++;
                 }
                 }
                 int syncedCount = 0;
                 int syncedCount = 0;
+                HashSet<Long> syncedSet = new HashSet<Long>();
+                
                 // lock on the followers when we use it.
                 // lock on the followers when we use it.
+                syncedSet.add(self.getId());
                 synchronized (followers) {
                 synchronized (followers) {
                     for (FollowerHandler f : followers) {
                     for (FollowerHandler f : followers) {
                         if (f.synced()) {
                         if (f.synced()) {
                             syncedCount++;
                             syncedCount++;
+                            syncedSet.add(f.getSid());
                         }
                         }
                         f.ping();
                         f.ping();
                     }
                     }
                 }
                 }
-                if (!tickSkip && syncedCount < self.quorumPeers.size() / 2) {
+              if (!tickSkip && !self.getQuorumVerifier().containsQuorum(syncedSet)) {
+                //if (!tickSkip && syncedCount < self.quorumPeers.size() / 2) {
                     // Lost quorum, shutdown
                     // Lost quorum, shutdown
                     shutdown("Only " + syncedCount + " followers, need "
                     shutdown("Only " + syncedCount + " followers, need "
                             + (self.quorumPeers.size() / 2));
                             + (self.quorumPeers.size() / 2));
                     // make sure the order is the same!
                     // make sure the order is the same!
                     // the leader goes to looking
                     // the leader goes to looking
                     return;
                     return;
-                }
-                tickSkip = !tickSkip;
+              } 
+              tickSkip = !tickSkip;
             }
             }
         } finally {
         } finally {
             zk.unregisterJMX(this);
             zk.unregisterJMX(this);
@@ -385,7 +402,7 @@ public class Leader {
      *                the zxid of the proposal sent out
      *                the zxid of the proposal sent out
      * @param followerAddr
      * @param followerAddr
      */
      */
-    synchronized public void processAck(long zxid, SocketAddress followerAddr) {
+    synchronized public void processAck(long sid, long zxid, SocketAddress followerAddr) {
         boolean first = true;
         boolean first = true;
         
         
         if (LOG.isDebugEnabled()) {
         if (LOG.isDebugEnabled()) {
@@ -419,19 +436,18 @@ public class Leader {
                     + Long.toHexString(zxid) + " from " + followerAddr);
                     + Long.toHexString(zxid) + " from " + followerAddr);
             return;
             return;
         }
         }
-        p.ackCount++;
+        
+        p.ackSet.add(sid);
         if (LOG.isDebugEnabled()) {
         if (LOG.isDebugEnabled()) {
             LOG.debug("Count for zxid: 0x" + Long.toHexString(zxid)
             LOG.debug("Count for zxid: 0x" + Long.toHexString(zxid)
-                    + " is " + p.ackCount);
+                    + " is " + p.ackSet.size());
         }
         }
-                
-        if (p.ackCount > self.quorumPeers.size() / 2){
+        if (self.getQuorumVerifier().containsQuorum(p.ackSet)){        
             if (zxid != lastCommitted+1) {
             if (zxid != lastCommitted+1) {
                 LOG.warn("Commiting zxid 0x" + Long.toHexString(zxid)
                 LOG.warn("Commiting zxid 0x" + Long.toHexString(zxid)
                         + " from " + followerAddr + " not first!");
                         + " from " + followerAddr + " not first!");
                 LOG.warn("First is "
                 LOG.warn("First is "
                         + (lastCommitted+1));
                         + (lastCommitted+1));
-                //System.exit(13);
             }
             }
             outstandingProposals.remove(zxid);
             outstandingProposals.remove(zxid);
             if (p.request != null) {
             if (p.request != null) {

+ 57 - 1
src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java

@@ -34,6 +34,8 @@ import org.apache.zookeeper.jmx.ZKMBeanInfo;
 import org.apache.zookeeper.server.NIOServerCnxn;
 import org.apache.zookeeper.server.NIOServerCnxn;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
+import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
+import org.apache.zookeeper.server.quorum.flexible.QuorumMaj;
 
 
 /**
 /**
  * This class manages the quorum protocol. There are three states this server
  * This class manages the quorum protocol. There are three states this server
@@ -108,6 +110,13 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
     public int getQuorumSize(){
     public int getQuorumSize(){
         return quorumPeers.size();
         return quorumPeers.size();
     }
     }
+    
+    /**
+     * QuorumVerifier implementation; default (majority). 
+     */
+    
+    private QuorumVerifier quorumConfig;
+    
     /**
     /**
      * My id
      * My id
      */
      */
@@ -259,10 +268,24 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
         quorumStats = new QuorumStats(this);
         quorumStats = new QuorumStats(this);
     }
     }
     
     
+   
+    /**
+     * For backward compatibility purposes, we instantiate QuorumMaj by default.
+     */
+    
     public QuorumPeer(Map<Long, QuorumServer> quorumPeers, File dataDir,
     public QuorumPeer(Map<Long, QuorumServer> quorumPeers, File dataDir,
             File dataLogDir, int electionType,
             File dataLogDir, int electionType,
             long myid, int tickTime, int initLimit, int syncLimit,
             long myid, int tickTime, int initLimit, int syncLimit,
             NIOServerCnxn.Factory cnxnFactory) throws IOException {
             NIOServerCnxn.Factory cnxnFactory) throws IOException {
+        this(quorumPeers, dataDir, dataLogDir, electionType, myid, tickTime, 
+        		initLimit, syncLimit, cnxnFactory, new QuorumMaj(quorumPeers.size()));
+    }
+    
+    public QuorumPeer(Map<Long, QuorumServer> quorumPeers, File dataDir,
+            File dataLogDir, int electionType,
+            long myid, int tickTime, int initLimit, int syncLimit,
+            NIOServerCnxn.Factory cnxnFactory, 
+            QuorumVerifier quorumConfig) throws IOException {
         this();
         this();
         this.cnxnFactory = cnxnFactory;
         this.cnxnFactory = cnxnFactory;
         this.quorumPeers = quorumPeers;
         this.quorumPeers = quorumPeers;
@@ -272,6 +295,9 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
         this.initLimit = initLimit;
         this.initLimit = initLimit;
         this.syncLimit = syncLimit;        
         this.syncLimit = syncLimit;        
         this.logFactory = new FileTxnSnapLog(dataLogDir, dataDir);
         this.logFactory = new FileTxnSnapLog(dataLogDir, dataDir);
+        if(quorumConfig == null)
+            this.quorumConfig = new QuorumMaj(quorumPeers.size());
+        else this.quorumConfig = quorumConfig;
     }
     }
     
     
     QuorumStats quorumStats() {
     QuorumStats quorumStats() {
@@ -313,6 +339,7 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
         this.electionAlg = createElectionAlgorithm(electionType);
         this.electionAlg = createElectionAlgorithm(electionType);
     }
     }
     
     
+    
     /**
     /**
      * This constructor is only used by the existing unit test code.
      * This constructor is only used by the existing unit test code.
      * It defaults to FileLogProvider persistence provider.
      * It defaults to FileLogProvider persistence provider.
@@ -324,7 +351,23 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
     {
     {
         this(quorumPeers, snapDir, logDir, electionAlg,
         this(quorumPeers, snapDir, logDir, electionAlg,
                 myid,tickTime, initLimit,syncLimit,
                 myid,tickTime, initLimit,syncLimit,
-                new NIOServerCnxn.Factory(clientPort));
+                new NIOServerCnxn.Factory(clientPort),
+                new QuorumMaj(quorumPeers.size()));
+    }
+    
+    /**
+     * This constructor is only used by the existing unit test code.
+     * It defaults to FileLogProvider persistence provider.
+     */
+    public QuorumPeer(Map<Long,QuorumServer> quorumPeers, File snapDir,
+            File logDir, int clientPort, int electionAlg,
+            long myid, int tickTime, int initLimit, int syncLimit, 
+            QuorumVerifier quorumConfig)
+        throws IOException
+    {
+        this(quorumPeers, snapDir, logDir, electionAlg,
+                myid,tickTime, initLimit,syncLimit,
+                new NIOServerCnxn.Factory(clientPort), quorumConfig);
     }
     }
     
     
     public long getLastLoggedZxid(){
     public long getLastLoggedZxid(){
@@ -576,6 +619,19 @@ public class QuorumPeer extends Thread implements QuorumStats.Provider {
         return tick;
         return tick;
     }
     }
     
     
+    /**
+     * Return QuorumVerifier object
+     */
+    
+    public QuorumVerifier getQuorumVerifier(){
+        return quorumConfig;
+        
+    }
+    
+    public void setQuorumVerifier(QuorumVerifier quorumConfig){
+       this.quorumConfig = quorumConfig;
+    }
+    
     /**
     /**
      * Get an instance of LeaderElection
      * Get an instance of LeaderElection
      */
      */

+ 60 - 2
src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java

@@ -33,6 +33,9 @@ import java.util.Map.Entry;
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
 import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
+import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
+import org.apache.zookeeper.server.quorum.flexible.QuorumMaj;
+import org.apache.zookeeper.server.quorum.flexible.QuorumHierarchical;
 
 
 public class QuorumPeerConfig {
 public class QuorumPeerConfig {
     private static final Logger LOG = Logger.getLogger(QuorumPeerConfig.class);
     private static final Logger LOG = Logger.getLogger(QuorumPeerConfig.class);
@@ -50,6 +53,10 @@ public class QuorumPeerConfig {
         new HashMap<Long, QuorumServer>();
         new HashMap<Long, QuorumServer>();
 
 
     protected long serverId;
     protected long serverId;
+    protected HashMap<Long, Long> serverWeight = new HashMap<Long, Long>();
+    protected HashMap<Long, Long> serverGroup = new HashMap<Long, Long>();
+    protected int numGroups = 0;
+    protected QuorumVerifier quorumVerifier;
 
 
     @SuppressWarnings("serial")
     @SuppressWarnings("serial")
     public static class ConfigException extends Exception {
     public static class ConfigException extends Exception {
@@ -93,7 +100,8 @@ public class QuorumPeerConfig {
         }
         }
     }
     }
 
 
-    protected void parseProperties(Properties zkProp) throws IOException {
+    protected void parseProperties(Properties zkProp)
+    throws IOException, ConfigException {
         for (Entry<Object, Object> entry : zkProp.entrySet()) {
         for (Entry<Object, Object> entry : zkProp.entrySet()) {
             String key = entry.getKey().toString();
             String key = entry.getKey().toString();
             String value = entry.getValue().toString();
             String value = entry.getValue().toString();
@@ -129,6 +137,25 @@ public class QuorumPeerConfig {
                     servers.put(Long.valueOf(sid), new QuorumServer(sid, addr,
                     servers.put(Long.valueOf(sid), new QuorumServer(sid, addr,
                             electionAddr));
                             electionAddr));
                 }
                 }
+            } else if (key.startsWith("group")) {
+                int dot = key.indexOf('.');
+                long gid = Long.parseLong(key.substring(dot + 1));
+                
+                numGroups++;
+                
+                String parts[] = value.split(":");
+                for(String s : parts){
+                    long sid = Long.parseLong(s);
+                    if(serverGroup.containsKey(sid))
+                        throw new ConfigException("Server " + sid + "is in multiple groups");
+                    else
+                        serverGroup.put(sid, gid);
+                }       
+                
+            } else if(key.startsWith("weight")) {
+                int dot = key.indexOf('.');
+                long sid = Long.parseLong(key.substring(dot + 1));
+                serverWeight.put(sid, Long.parseLong(value));
             } else {
             } else {
                 System.setProperty("zookeeper." + key, value);
                 System.setProperty("zookeeper." + key, value);
             }
             }
@@ -169,6 +196,34 @@ public class QuorumPeerConfig {
                 }
                 }
             }
             }
 
 
+            /*
+             * Default of quorum config is majority
+             */
+            if(serverGroup.size() > 0){  
+                if(servers.size() != serverGroup.size())
+                    throw new ConfigException("Every server must be in exactly one group");
+            	/*
+            	 * The deafult weight of a server is 1
+            	 */
+            	for(QuorumServer s : servers.values()){
+            		if(!serverWeight.containsKey(s.id))
+            			serverWeight.put(s.id, (long) 1);
+            	}
+            	
+            	/*
+            	 * Set the quorumVerifier to be QuorumHierarchical 
+            	 */
+                quorumVerifier = new QuorumHierarchical(numGroups, 
+                        serverWeight, serverGroup);
+            } else {
+            	/*
+            	 * The default QuorumVerifier is QuorumMaj
+            	 */
+            	
+                LOG.info("Defaulting to majority quorums");
+                quorumVerifier = new QuorumMaj(servers.size());
+            }
+            
             File myIdFile = new File(dataDir, "myid");
             File myIdFile = new File(dataDir, "myid");
             if (!myIdFile.exists()) {
             if (!myIdFile.exists()) {
                 throw new IllegalArgumentException(myIdFile.toString()
                 throw new IllegalArgumentException(myIdFile.toString()
@@ -199,7 +254,10 @@ public class QuorumPeerConfig {
     public int getSyncLimit() { return syncLimit; }
     public int getSyncLimit() { return syncLimit; }
     public int getElectionAlg() { return electionAlg; }
     public int getElectionAlg() { return electionAlg; }
     public int getElectionPort() { return electionPort; }
     public int getElectionPort() { return electionPort; }
-
+    public QuorumVerifier getQuorumVerifier() {   
+        return quorumVerifier;
+    }
+    
     public Map<Long,QuorumServer> getServers() {
     public Map<Long,QuorumServer> getServers() {
         return Collections.unmodifiableMap(servers);
         return Collections.unmodifiableMap(servers);
     }
     }

+ 1 - 0
src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerMain.java

@@ -121,6 +121,7 @@ public class QuorumPeerMain {
                 quorumPeer.setTickTime(config.getTickTime());
                 quorumPeer.setTickTime(config.getTickTime());
                 quorumPeer.setInitLimit(config.getInitLimit());
                 quorumPeer.setInitLimit(config.getInitLimit());
                 quorumPeer.setSyncLimit(config.getSyncLimit());
                 quorumPeer.setSyncLimit(config.getSyncLimit());
+                quorumPeer.setQuorumVerifier(config.getQuorumVerifier());
                 quorumPeer.setCnxnFactory(cnxnFactory);
                 quorumPeer.setCnxnFactory(cnxnFactory);
 
 
                 quorumPeer.start();
                 quorumPeer.start();

+ 249 - 0
src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumHierarchical.java

@@ -0,0 +1,249 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.quorum.flexible;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Properties;
+import java.util.Map.Entry;
+
+
+import org.apache.log4j.Logger;
+
+import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
+
+
+/**
+ * This class implements a validator for hierarchical quorums. With this
+ * construction, zookeeper servers are split into disjoint groups, and 
+ * each server has a weight. We obtain a quorum if we get more than half
+ * of the total weight of a group for a majority of groups.
+ * 
+ * The configuration of quorums uses two parameters: group and weight. 
+ * Groups are sets of ZooKeeper servers, and we set a group by passing
+ * a colon-separated list of server ids. It is also necessary to assign
+ * weights to server. Here is an example of a configuration that creates
+ * three groups and assigns a weight of 1 to each server:
+ * 
+ *  group.1=1:2:3
+ *  group.2=4:5:6
+ *  group.3=7:8:9
+ *  
+ *  weight.1=1
+ *  weight.2=1
+ *  weight.3=1
+ *  weight.4=1
+ *  weight.5=1
+ *  weight.6=1
+ *  weight.7=1
+ *  weight.8=1
+ *  weight.9=1
+ * 
+ * Note that it is still necessary to define peers using the server keyword.
+ */
+
+public class QuorumHierarchical implements QuorumVerifier {
+    private static final Logger LOG = Logger.getLogger(QuorumHierarchical.class);
+
+    HashMap<Long, Long> serverWeight;
+    HashMap<Long, Long> serverGroup;
+    HashMap<Long, Long> groupWeight;
+    
+    int numGroups;
+   
+    /**
+     * This contructor requires the quorum configuration
+     * to be declared in a separate file, and it takes the
+     * file as an input parameter.
+     */
+    public QuorumHierarchical(String filename)
+    throws ConfigException {
+        this.serverWeight = new HashMap<Long, Long>();
+        this.serverGroup = new HashMap<Long, Long>();
+        this.groupWeight = new HashMap<Long, Long>();
+        this.numGroups = 0;
+        
+        readConfigFile(filename);
+    }
+    
+    /**
+     * This constructor takes a set of properties. We use
+     * it in the unit test for this feature.
+     */
+    
+    public QuorumHierarchical(Properties qp)
+    throws ConfigException {
+        this.serverWeight = new HashMap<Long, Long>();
+        this.serverGroup = new HashMap<Long, Long>();
+        this.groupWeight = new HashMap<Long, Long>();
+        this.numGroups = 0;
+        
+        parse(qp);
+        
+        LOG.info(serverWeight.size() + ", " + serverGroup.size() + ", " + groupWeight.size());
+    }
+    
+    /**
+     * This contructor takes the two hash maps needed to enable 
+     * validating quorums. We use it with QuorumPeerConfig. That is,
+     * we declare weights and groups in the server configuration
+     * file along with the other parameters.
+     */
+    public QuorumHierarchical(int numGroups,
+            HashMap<Long, Long> serverWeight,
+            HashMap<Long, Long> serverGroup)
+    {
+        this.serverWeight = serverWeight;
+        this.serverGroup = serverGroup;
+        this.groupWeight = new HashMap<Long, Long>();
+        
+        computeGroupWeight();
+        this.numGroups = numGroups;
+    }
+    
+    
+    /**
+     * Reads a configration file. Called from the constructor
+     * that takes a file as an input.
+     */
+    private void readConfigFile(String filename)
+    throws ConfigException{
+        File configFile = new File(filename);
+
+        LOG.info("Reading configuration from: " + configFile);
+
+        try {
+            if (!configFile.exists()) {
+                throw new IllegalArgumentException(configFile.toString()
+                        + " file is missing");
+            }
+    
+            Properties cfg = new Properties();
+            FileInputStream in = new FileInputStream(configFile);
+            try {
+                cfg.load(in);
+            } finally {
+                in.close();
+            }
+    
+            parse(cfg);
+        } catch (IOException e) {
+            throw new ConfigException("Error processing " + filename, e);
+        } catch (IllegalArgumentException e) {
+            throw new ConfigException("Error processing " + filename, e);
+        }
+        
+    }
+    
+    
+    /**
+     * Parse properties if configuration given in a separate file.
+     */
+    private void parse(Properties quorumProp){
+        for (Entry<Object, Object> entry : quorumProp.entrySet()) {
+            String key = entry.getKey().toString();
+            String value = entry.getValue().toString(); 
+            if (key.startsWith("group")) {
+                int dot = key.indexOf('.');
+                long gid = Long.parseLong(key.substring(dot + 1));
+                
+                numGroups++;
+                
+                String parts[] = value.split(":");
+                for(String s : parts){
+                    long sid = Long.parseLong(s);
+                    serverGroup.put(sid, gid);
+                }
+                    
+                
+            } else if(key.startsWith("weight")) {
+                int dot = key.indexOf('.');
+                long sid = Long.parseLong(key.substring(dot + 1));
+                serverWeight.put(sid, Long.parseLong(value));
+            }
+        }
+        
+        computeGroupWeight();
+    }
+    
+    /**
+     * This method pre-computes the weights of groups to speed up processing
+     * when validating a given set. We compute the weights of groups in 
+     * different places, so we have a separate method.
+     */
+    private void computeGroupWeight(){
+        for(long sid : serverGroup.keySet()){
+            Long gid = serverGroup.get(sid);
+            if(!groupWeight.containsKey(gid))
+                groupWeight.put(gid, serverWeight.get(sid));
+            else {
+                long totalWeight = serverWeight.get(sid) + groupWeight.get(gid);
+                groupWeight.put(gid, totalWeight);
+            }
+            
+        }        
+    }
+    
+    /**
+     * Verifies if a given set is a quorum.
+     */
+    public boolean containsQuorum(HashSet<Long> set){
+        HashMap<Long, Long> expansion = new HashMap<Long, Long>();
+        
+        /*
+         * Adds up weights per group
+         */
+        if(set.size() == 0) return false;
+        else LOG.info("Set size: " + set.size());
+        
+        for(long sid : set){
+            Long gid = serverGroup.get(sid);
+            if(!expansion.containsKey(gid))
+                expansion.put(gid, serverWeight.get(sid));
+            else {
+                long totalWeight = serverWeight.get(sid) + expansion.get(gid);
+                expansion.put(gid, totalWeight);
+            }
+        }
+  
+        /*
+         * Check if all groups have majority
+         */
+        boolean majPerGroup = true;
+        for(long gid : expansion.keySet()) {
+            LOG.info("gid: " + expansion.get(gid));
+            if(expansion.get(gid) <= (groupWeight.get(gid) / 2) )
+                majPerGroup = false;
+        }
+        
+        if((expansion.size() > (numGroups / 2)) && majPerGroup){
+            LOG.info("Positive set size: " + set.size());
+            return true;
+        }
+        else {
+            LOG.info("Negative set size: " + set.size());
+            return false;
+        }
+    }
+}

+ 46 - 0
src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumMaj.java

@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.quorum.flexible;
+
+import java.util.HashSet;
+
+
+/**
+ * This class implements a validator for majority quorums. The 
+ * implementation is straightforward.
+ *
+ */
+public class QuorumMaj implements QuorumVerifier {
+    int half;
+    
+    /**
+     * Defines a majority to avoid computing it every time.
+     */
+    public QuorumMaj(int n){
+        this.half = n/2;
+    }
+    
+    /**
+     * Verifies if a set is a majority.
+     */
+    public boolean containsQuorum(HashSet<Long> set){
+        return (set.size() > half);
+    }
+    
+}

+ 32 - 0
src/java/main/org/apache/zookeeper/server/quorum/flexible/QuorumVerifier.java

@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.server.quorum.flexible;
+
+import java.util.HashSet;
+
+/**
+ * All quorum validators have to implement a method called
+ * containsQuorum, which verifies if a HashSet of server 
+ * identifiers constitutes a quorum.
+ *
+ */
+
+public interface QuorumVerifier {
+    boolean containsQuorum(HashSet<Long> set);
+}

+ 188 - 0
src/java/test/org/apache/zookeeper/test/HierarchicalQuorumTest.java

@@ -0,0 +1,188 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.zookeeper.test;
+import java.io.IOException;
+import java.util.Properties;
+
+import java.io.File;
+import java.io.ByteArrayInputStream;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Random;
+
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.server.quorum.FastLeaderElection;
+import org.apache.zookeeper.server.quorum.QuorumPeer;
+import org.apache.zookeeper.server.quorum.Vote;
+import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer;
+import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState;
+import org.apache.zookeeper.server.quorum.flexible.QuorumHierarchical;
+
+import org.apache.log4j.Logger;
+import junit.framework.TestCase;
+import org.junit.Before;
+import org.junit.Test;
+
+public class HierarchicalQuorumTest extends TestCase {
+    private static final Logger LOG = Logger.getLogger(HierarchicalQuorumTest.class);
+    
+    class TestVote{
+        TestVote(int id, long leader){
+            this.leader = leader;
+            this.id = id;
+        }
+        long leader;
+        int id;
+    }
+    
+    Properties qp;
+    
+    int count;
+    int baseport;
+    int baseLEport;
+    HashMap<Long,QuorumServer> peers; 
+    ArrayList<LEThread> threads;
+    File tmpdir[];
+    int port[];
+    Object finalObj;
+    
+    volatile Vote votes[];
+    volatile boolean leaderDies;
+    volatile long leader = -1;
+    Random rand = new Random();
+    
+    
+    @Before
+    @Override
+    protected void setUp() throws Exception {
+        count = 9;
+        baseport= 33003;
+        baseLEport = 43003;
+        
+        peers = new HashMap<Long,QuorumServer>(count);
+        threads = new ArrayList<LEThread>(count);
+        votes = new Vote[count];
+        tmpdir = new File[count];
+        port = new int[count];
+        finalObj = new Object();
+        
+        String config = "group.1=0:1:2\n" +
+        "group.2=3:4:5\n" +
+        "group.3=6:7:8\n\n" + 
+        "weight.0=1\n" + 
+        "weight.1=1\n" +
+        "weight.2=1\n" +
+        "weight.3=1\n" +
+        "weight.4=1\n" +
+        "weight.5=1\n" +
+        "weight.6=1\n" +
+        "weight.7=1\n" +
+        "weight.8=1";
+        
+        ByteArrayInputStream is = new ByteArrayInputStream(config.getBytes());
+        this.qp = new Properties(); 
+        qp.load(is);
+        
+        LOG.info("SetUp " + getName());
+    }
+    
+    protected void tearDown() throws Exception {
+        for(int i = 0; i < threads.size(); i++) {
+            ((FastLeaderElection) threads.get(i).peer.getElectionAlg()).shutdown();
+        }
+        LOG.info("FINISHED " + getName());
+    }
+    
+    class LEThread extends Thread {
+        FastLeaderElection le;
+        int i;
+        QuorumPeer peer;
+        //int peerRound = 1;
+
+        LEThread(QuorumPeer peer, int i) {
+            this.i = i;
+            this.peer = peer;
+            LOG.info("Constructor: " + getName());
+        }
+
+        public void run() {
+            try {
+                Vote v = null;
+                while(true){
+                    
+                    //while(true) {
+                    peer.setPeerState(ServerState.LOOKING);
+                    LOG.info("Going to call leader election.");
+                    v = peer.getElectionAlg().lookForLeader();
+                    if(v == null){ 
+                        LOG.info("Thread " + i + " got a null vote");
+                        return;
+                    }
+                    
+                    /*
+                     * A real zookeeper would take care of setting the current vote. Here
+                     * we do it manually.
+                     */
+                    peer.setCurrentVote(v);
+            
+                    LOG.info("Finished election: " + i + ", " + v.id);
+                    votes[i] = v;
+                
+                    if((peer.getPeerState() == ServerState.FOLLOWING) ||
+                            (peer.getPeerState() == ServerState.LEADING)) break;
+                }
+                LOG.debug("Thread " + i + " votes " + v);
+            } catch (InterruptedException e) {
+                e.printStackTrace();
+            }
+        }
+    }
+    
+    @Test
+    public void testHierarchicalQuorum() throws Exception {
+        FastLeaderElection le[] = new FastLeaderElection[count];
+       
+        LOG.info("TestHierarchicalQuorum: " + getName()+ ", " + count);
+        for(int i = 0; i < count; i++) {
+            peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress(baseport+100+i), 
+                    new InetSocketAddress(baseLEport+100+i)));
+            tmpdir[i] = File.createTempFile("letest", "test");
+            port[i] = baseport+i;    
+        }
+        
+        for(int i = 0; i < le.length; i++) {
+            QuorumHierarchical hq = new QuorumHierarchical(qp);
+            QuorumPeer peer = new QuorumPeer(peers, tmpdir[i], tmpdir[i], port[i], 3, i, 2, 2, 2, hq);
+            peer.startLeaderElection();
+            LEThread thread = new LEThread(peer, i);
+            thread.start();
+            threads.add(thread);
+        }
+        LOG.info("Started threads " + getName());
+        
+        for(int i = 0; i < threads.size(); i++) {
+            threads.get(i).join(15000);
+            if (threads.get(i).isAlive()) {
+                fail("Threads didn't join");
+            }
+        }
+    }
+}

برخی فایل ها در این مقایسه diff نمایش داده نمی شوند زیرا تعداد فایل ها بسیار زیاد است