You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2009/05/07 00:40:16 UTC
svn commit: r772455 - in
/incubator/cassandra/trunk/src/java/org/apache/cassandra:
net/http/HttpRequestHandler.java service/LeaderElector.java
service/StorageService.java service/ZookeeperWatcher.java
Author: jbellis
Date: Wed May 6 22:40:16 2009
New Revision: 772455
URL: http://svn.apache.org/viewvc?rev=772455&view=rev
Log:
r/m orphaned Leader code. patch by jbellis; reviewed by Eric Evans for CASSANDRA-143
Removed:
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/LeaderElector.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ZookeeperWatcher.java
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/net/http/HttpRequestHandler.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/net/http/HttpRequestHandler.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/net/http/HttpRequestHandler.java?rev=772455&r1=772454&r2=772455&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/net/http/HttpRequestHandler.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/net/http/HttpRequestHandler.java Wed May 6 22:40:16 2009
@@ -312,7 +312,7 @@
EndPoint[] liveNodes = liveNodeList.toArray(new EndPoint[0]);
Arrays.sort(liveNodes);
- String[] sHeaders = {"Node No.", "Host:Port", "Status", "Leader", "Load Info", "Token", "Generation No."};
+ String[] sHeaders = {"Node No.", "Host:Port", "Status", "Load Info", "Token", "Generation No."};
formatter.startTable();
formatter.addHeaders(sHeaders);
int iNodeNumber = 0;
@@ -328,9 +328,6 @@
//Status
String status = ( FailureDetector.instance().isAlive(curNode) ) ? "Up" : "Down";
formatter.addCol(status);
- //Leader
- boolean isLeader = StorageService.instance().isLeader(curNode);
- formatter.addCol(Boolean.toString(isLeader));
//Load Info
String loadInfo = getLoadInfo(curNode);
formatter.addCol(loadInfo);
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=772455&r1=772454&r2=772455&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java Wed May 6 22:40:16 2009
@@ -317,91 +317,12 @@
else
nodePicker_ = new RackUnawareStrategy(tokenMetadata_, partitioner_, DatabaseDescriptor.getReplicationFactor(), DatabaseDescriptor.getStoragePort());
}
-
- private void reportToZookeeper() throws Throwable
- {
- try
- {
- zk_ = new ZooKeeper(DatabaseDescriptor.getZkAddress(), DatabaseDescriptor.getZkSessionTimeout(), new Watcher()
- {
- public void process(WatchedEvent we)
- {
- String path = "/Cassandra/" + DatabaseDescriptor.getClusterName() + "/Leader";
- String eventPath = we.getPath();
- logger_.debug("PROCESS EVENT : " + eventPath);
- if (eventPath != null && (eventPath.contains(path)))
- {
- logger_.debug("Signalling the leader instance ...");
- LeaderElector.instance().signal();
- }
- }
- });
-
- Stat stat = zk_.exists("/", false);
- if ( stat != null )
- {
- stat = zk_.exists("/Cassandra", false);
- if ( stat == null )
- {
- logger_.debug("Creating the Cassandra znode ...");
- zk_.create("/Cassandra", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- }
-
- String path = "/Cassandra/" + DatabaseDescriptor.getClusterName();
- stat = zk_.exists(path, false);
- if ( stat == null )
- {
- logger_.debug("Creating the cluster znode " + path);
- zk_.create(path, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- }
-
- /* Create the Leader, Locks and Misc znode */
- stat = zk_.exists(path + "/Leader", false);
- if ( stat == null )
- {
- logger_.debug("Creating the leader znode " + path);
- zk_.create(path + "/Leader", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- }
-
- stat = zk_.exists(path + "/Locks", false);
- if ( stat == null )
- {
- logger_.debug("Creating the locks znode " + path);
- zk_.create(path + "/Locks", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- }
-
- stat = zk_.exists(path + "/Misc", false);
- if ( stat == null )
- {
- logger_.debug("Creating the misc znode " + path);
- zk_.create(path + "/Misc", new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- }
- }
- }
- catch ( KeeperException ke )
- {
- LogUtil.throwableToString(ke);
- /* do the re-initialize again. */
- reportToZookeeper();
- }
- }
-
+
protected ZooKeeper getZooKeeperHandle()
{
return zk_;
}
- public boolean isLeader(EndPoint endpoint)
- {
- EndPoint leader = getLeader();
- return leader.equals(endpoint);
- }
-
- public EndPoint getLeader()
- {
- return LeaderElector.instance().getLeader();
- }
-
public void registerComponentForShutdown(IComponentShutdown component)
{
components_.add(component);
@@ -440,14 +361,6 @@
/* starts a load timer thread */
loadTimer_.schedule( new LoadDisseminator(), StorageService.threshold_, StorageService.threshold_);
- /* report our existence to ZooKeeper instance and start the leader election service */
-
- //reportToZookeeper();
- /* start the leader election algorithm */
- //LeaderElector.instance().start();
- /* start the map reduce framework */
- //startMapReduceFramework();
-
/* Start the storage load balancer */
storageLoadBalancer_.start();
/* Register with the Gossiper for EndPointState notifications */