Author: jbellis
Date: Mon Apr 19 19:58:10 2010
New Revision: 935717

URL: http://svn.apache.org/viewvc?rev=935717&view=rev
Log:
rename EndPoint classes -> Endpoint, part 2.
patch by Erick Tryzelaar; reviewed by jbellis for CASSANDRA-994

Added:
    
cassandra/trunk/contrib/property_snitch/src/java/org/apache/cassandra/locator/PropertyFileEndpointSnitch.java
    
cassandra/trunk/contrib/property_snitch/src/java/org/apache/cassandra/locator/PropertyFileEndpointSnitchMBean.java
Removed:
    
cassandra/trunk/contrib/property_snitch/src/java/org/apache/cassandra/locator/PropertyFileEndPointSnitch.java
    
cassandra/trunk/contrib/property_snitch/src/java/org/apache/cassandra/locator/PropertyFileEndPointSnitchMBean.java
Modified:
    cassandra/trunk/conf/cassandra.xml
    cassandra/trunk/contrib/pig/storage-conf.xml
    cassandra/trunk/contrib/property_snitch/README.txt
    cassandra/trunk/contrib/word_count/storage-conf.xml
    cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java
    cassandra/trunk/src/java/org/apache/cassandra/gms/EndpointState.java
    cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigest.java
    
cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestAck2Message.java
    
cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestAckMessage.java
    
cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestSynMessage.java
    cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java
    
cassandra/trunk/src/java/org/apache/cassandra/gms/IEndpointStateChangePublisher.java
    
cassandra/trunk/src/java/org/apache/cassandra/gms/IEndpointStateChangeSubscriber.java
    
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterEndpointSnitch.java
    
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java
    cassandra/trunk/src/java/org/apache/cassandra/locator/RackAwareStrategy.java
    cassandra/trunk/test/conf/cassandra.xml
    cassandra/trunk/test/system/__init__.py
    cassandra/trunk/test/system/test_thrift_server.py

Modified: cassandra/trunk/conf/cassandra.xml
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/conf/cassandra.xml?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- cassandra/trunk/conf/cassandra.xml (original)
+++ cassandra/trunk/conf/cassandra.xml Mon Apr 19 19:58:10 2010
@@ -49,8 +49,8 @@
    ~ EndpointSnitch: Setting this to the class that implements
    ~ AbstractEndpointSnitch, which lets Cassandra know enough
    ~ about your network topology to route requests efficiently.
-   ~ Out of the box, Cassandra provides 
org.apache.cassandra.locator.EndPointSnitch,
-   ~ and PropertyFileEndPointSnitch is available in contrib/.
+   ~ Out of the box, Cassandra provides 
org.apache.cassandra.locator.EndpointSnitch,
+   ~ and PropertyFileEndpointSnitch is available in contrib/.
   -->
   <EndpointSnitch>org.apache.cassandra.locator.EndpointSnitch</EndpointSnitch>
 

Modified: cassandra/trunk/contrib/pig/storage-conf.xml
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/contrib/pig/storage-conf.xml?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- cassandra/trunk/contrib/pig/storage-conf.xml (original)
+++ cassandra/trunk/contrib/pig/storage-conf.xml Mon Apr 19 19:58:10 2010
@@ -133,7 +133,7 @@
        ~ Out of the box, Cassandra provides 
org.apache.cassandra.locator.EndpointSnitch,
        ~ and PropertyFileEndpointSnitch is available in contrib/.
       -->
-      
<EndpointSnitch>org.apache.cassandra.locator.EndPointSnitch</EndpointSnitch>
+      
<EndpointSnitch>org.apache.cassandra.locator.EndpointSnitch</EndpointSnitch>
     </Keyspace>
   </Keyspaces>
 

Modified: cassandra/trunk/contrib/property_snitch/README.txt
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/contrib/property_snitch/README.txt?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- cassandra/trunk/contrib/property_snitch/README.txt (original)
+++ cassandra/trunk/contrib/property_snitch/README.txt Mon Apr 19 19:58:10 2010
@@ -1,15 +1,15 @@
-PropertyFileEndPointSnitch
+PropertyFileEndpointSnitch
 ==========================
 
 Cassandra's RackAwareStrategy can be used to have replication take
 data-center and/or rack location into account when placing replicas. The
-ProperyFileEndPointSnitch allows you to make use of RackAwareStrategy by
+ProperyFileEndpointSnitch allows you to make use of RackAwareStrategy by
 specifying node locations in a standard key/value properties file.
 
 
 Properties File
 ---------------
-The EndPointSnitch expects to find a file on the classapth named
+The EndpointSnitch expects to find a file on the classapth named
 cassandra-rack.properties in the following format:
 
   <node IP>\:<port>=<data center name>:<rack name>
@@ -28,8 +28,8 @@ Installing
  * Copy build/cassandra-propsnitch.jar to your Cassandra lib/
    directory, or otherwise add it to the CLASSPATH
    (see http://wiki.apache.org/cassandra/RunningCassandra)
- * Edit the EndPointSnitch element of storage-conf.xml to use
-   org.apache.cassandra.locator.PropertyFileEndPointSnitch
+ * Edit the EndpointSnitch element of storage-conf.xml to use
+   org.apache.cassandra.locator.PropertyFileEndpointSnitch
  * Create the cassandra-rack.properties in your classpath
    (e.g. in your Cassandra conf/ directory)
  * Optionally set ReplicaPlacementStrategy in cassandra.xml to

Added: 
cassandra/trunk/contrib/property_snitch/src/java/org/apache/cassandra/locator/PropertyFileEndpointSnitch.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/contrib/property_snitch/src/java/org/apache/cassandra/locator/PropertyFileEndpointSnitch.java?rev=935717&view=auto
==============================================================================
--- 
cassandra/trunk/contrib/property_snitch/src/java/org/apache/cassandra/locator/PropertyFileEndpointSnitch.java
 (added)
+++ 
cassandra/trunk/contrib/property_snitch/src/java/org/apache/cassandra/locator/PropertyFileEndpointSnitch.java
 Mon Apr 19 19:58:10 2010
@@ -0,0 +1,180 @@
+/**
+ * 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.cassandra.locator;
+
+import java.io.FileReader;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.net.UnknownHostException;
+import java.net.URL;
+import java.util.Properties;
+import java.util.StringTokenizer;
+
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
+import java.net.InetAddress;
+
+import org.apache.cassandra.config.ConfigurationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * PropertyFileEndpointSnitch
+ *
+ * PropertyFileEndpointSnitch is used by Digg to determine if two IP's are in 
the same
+ * datacenter or on the same rack.
+ *
+ */
+public class PropertyFileEndpointSnitch extends EndpointSnitch implements 
PropertyFileEndpointSnitchMBean {
+    /**
+     * A list of properties with keys being host:port and values being 
datacenter:rack
+     */
+    private Properties hostProperties = new Properties();
+
+    /**
+     * The default rack property file to be read.
+     */
+    private static String RACK_PROPERTY_FILENAME = "cassandra-rack.properties";
+
+    /**
+     * Whether to use the parent for detection of same node
+     */
+    private boolean runInBaseMode = false;
+
+    /**
+     * Reference to the logger.
+     */
+    private static Logger logger_ = 
LoggerFactory.getLogger(PropertyFileEndpointSnitch.class);
+
+    public PropertyFileEndpointSnitch() throws ConfigurationException
+    {
+        reloadConfiguration();
+        try
+        {
+            MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+            mbs.registerMBean(this, new ObjectName(MBEAN_OBJECT_NAME));
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Get the raw information about an end point
+     * 
+     * @param endpoint endpoint to process
+     * 
+     * @return a array of string with the first index being the data center 
and the second being the rack
+     */
+    public String[] getEndpointInfo(InetAddress endpoint) {
+        String key = endpoint.toString();
+        String value = hostProperties.getProperty(key);
+        if (value == null)
+        {
+            logger_.error("Could not find end point information for {}, will 
use default.", key);
+            value = hostProperties.getProperty("default");
+        }
+        StringTokenizer st = new StringTokenizer(value, ":");
+        if (st.countTokens() < 2)
+        {
+            logger_.error("Value for " + key + " is invalid: " + value);
+            return new String [] {"default", "default"};
+        }
+        return new String[] {st.nextToken(), st.nextToken()};
+    }
+
+    /**
+     * Return the data center for which an endpoint resides in
+     *  
+     * @param endpoint the endpoint to process
+     * @return string of data center
+     */
+    public String getDataCenterForEndpoint(InetAddress endpoint) {
+        return getEndpointInfo(endpoint)[0];
+    }
+
+    /**
+     * Return the rack for which an endpoint resides in
+     *  
+     * @param endpoint the endpoint to process
+     * 
+     * @return string of rack
+     */
+    public String getRackForEndpoint(InetAddress endpoint) {
+        return getEndpointInfo(endpoint)[1];
+    }
+
+    @Override
+    public boolean isInSameDataCenter(InetAddress host, InetAddress host2)
+            throws UnknownHostException {
+        if (runInBaseMode)
+        {
+            return super.isInSameDataCenter(host, host2);
+        }
+        return 
getDataCenterForEndpoint(host).equals(getDataCenterForEndpoint(host2));
+    }
+
+    @Override
+    public boolean isOnSameRack(InetAddress host, InetAddress host2)
+            throws UnknownHostException {
+        if (runInBaseMode)
+        {
+            return super.isOnSameRack(host, host2);
+        }
+        if (!isInSameDataCenter(host, host2))
+        {
+            return false;
+        }
+        return getRackForEndpoint(host).equals(getRackForEndpoint(host2));
+    }
+
+    public String displayConfiguration() {
+        StringBuffer configurationString = new StringBuffer("Current rack 
configuration\n=================\n");
+        for (Object key: hostProperties.keySet()) {
+            String endpoint = (String) key;
+            String value = hostProperties.getProperty(endpoint);
+            
configurationString.append(endpoint).append("=").append(value).append("\n");
+        }
+        return configurationString.toString();
+    }
+
+    public void reloadConfiguration() throws ConfigurationException
+    {
+        ClassLoader loader = PropertyFileEndpointSnitch.class.getClassLoader();
+        URL scpurl = loader.getResource(RACK_PROPERTY_FILENAME);
+        if (scpurl == null)
+            throw new ConfigurationException("unable to locate " + 
RACK_PROPERTY_FILENAME);
+
+        String rackPropertyFilename = scpurl.getFile();
+
+        try
+        {
+            Properties localHostProperties = new Properties();
+            localHostProperties.load(new FileReader(rackPropertyFilename));
+            hostProperties = localHostProperties;
+            runInBaseMode = false;
+        }
+        catch (IOException ioe)
+        {
+            throw new ConfigurationException("Could not process " + 
rackPropertyFilename, ioe);
+        }
+    }
+}

Added: 
cassandra/trunk/contrib/property_snitch/src/java/org/apache/cassandra/locator/PropertyFileEndpointSnitchMBean.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/contrib/property_snitch/src/java/org/apache/cassandra/locator/PropertyFileEndpointSnitchMBean.java?rev=935717&view=auto
==============================================================================
--- 
cassandra/trunk/contrib/property_snitch/src/java/org/apache/cassandra/locator/PropertyFileEndpointSnitchMBean.java
 (added)
+++ 
cassandra/trunk/contrib/property_snitch/src/java/org/apache/cassandra/locator/PropertyFileEndpointSnitchMBean.java
 Mon Apr 19 19:58:10 2010
@@ -0,0 +1,49 @@
+/**
+ * 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.cassandra.locator;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.ConfigurationException;
+
+/**
+ * PropertyFileEndpointSnitchMBean
+ * 
+ * PropertyFileEndpointSnitchMBean is the management interface for Digg's 
EndpointSnitch MBean
+ * 
+ * @author Sammy Yu <s...@sammyyu.net>
+ * 
+ */
+public interface PropertyFileEndpointSnitchMBean {
+    /**
+     * The object name of the mbean.
+     */
+    public static String MBEAN_OBJECT_NAME = 
"org.apache.cassandra.locator:type=EndpointSnitch";
+    
+    /**
+     * Reload the rack configuration
+     */
+    public void reloadConfiguration() throws ConfigurationException;
+    
+    /**
+     * Display the current configuration
+     */
+    public String displayConfiguration();
+
+}

Modified: cassandra/trunk/contrib/word_count/storage-conf.xml
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/contrib/word_count/storage-conf.xml?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- cassandra/trunk/contrib/word_count/storage-conf.xml (original)
+++ cassandra/trunk/contrib/word_count/storage-conf.xml Mon Apr 19 19:58:10 2010
@@ -130,10 +130,10 @@
        ~ EndpointSnitch: Setting this to the class that implements
        ~ AbstractEndpointSnitch, which lets Cassandra know enough
        ~ about your network topology to route requests efficiently.
-       ~ Out of the box, Cassandra provides 
org.apache.cassandra.locator.EndPointSnitch,
-       ~ and PropertyFileEndPointSnitch is available in contrib/.
+       ~ Out of the box, Cassandra provides 
org.apache.cassandra.locator.EndpointSnitch,
+       ~ and PropertyFileEndpointSnitch is available in contrib/.
       -->
-      
<EndpointSnitch>org.apache.cassandra.locator.EndPointSnitch</EndpointSnitch>
+      
<EndpointSnitch>org.apache.cassandra.locator.EndpointSnitch</EndpointSnitch>
     </Keyspace>
   </Keyspaces>
 

Modified: cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/dht/BootStrapper.java Mon Apr 
19 19:58:10 2010
@@ -141,7 +141,7 @@ public class BootStrapper
         return maxEndpoint;
     }
 
-    /** get potential sources for each range, ordered by proximity (as 
determined by EndPointSnitch) */
+    /** get potential sources for each range, ordered by proximity (as 
determined by EndpointSnitch) */
     Multimap<Range, InetAddress> getRangesWithSources(String table)
     {
         assert tokenMetadata.sortedTokens().size() > 0;

Modified: cassandra/trunk/src/java/org/apache/cassandra/gms/EndpointState.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/gms/EndpointState.java?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/gms/EndpointState.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/gms/EndpointState.java Mon 
Apr 19 19:58:10 2010
@@ -28,7 +28,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * This abstraction represents both the HeartBeatState and the 
ApplicationState in an EndPointState
+ * This abstraction represents both the HeartBeatState and the 
ApplicationState in an EndpointState
  * instance. Any state for a given endpoint can be retrieved from this 
instance.
  */
 
@@ -37,7 +37,7 @@ public class EndpointState
     private static ICompactSerializer<EndpointState> serializer_;
     static
     {
-        serializer_ = new EndPointStateSerializer();
+        serializer_ = new EndpointStateSerializer();
     }
     
     HeartBeatState hbState_;
@@ -154,9 +154,9 @@ public class EndpointState
 
 }
 
-class EndPointStateSerializer implements ICompactSerializer<EndpointState>
+class EndpointStateSerializer implements ICompactSerializer<EndpointState>
 {
-    private static Logger logger_ = 
LoggerFactory.getLogger(EndPointStateSerializer.class);
+    private static Logger logger_ = 
LoggerFactory.getLogger(EndpointStateSerializer.class);
     
     public void serialize(EndpointState epState, DataOutputStream dos) throws 
IOException
     {
@@ -177,7 +177,7 @@ class EndPointStateSerializer implements
             {
                 if ( Gossiper.MAX_GOSSIP_PACKET_SIZE - dos.size() < estimate )
                 {
-                    logger_.info("@@@@ Breaking out to respect the MTU size in 
EndPointState serializer. Estimate is {} @@@@", estimate);;
+                    logger_.info("@@@@ Breaking out to respect the MTU size in 
EndpointState serializer. Estimate is {} @@@@", estimate);;
                     break;
                 }
             

Modified: cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigest.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigest.java?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigest.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigest.java Mon Apr 
19 19:58:10 2010
@@ -27,7 +27,7 @@ import org.apache.cassandra.net.CompactE
 import java.net.InetAddress;
 
 /**
- * Contains information about a specified list of EndPoints and the largest 
version 
+ * Contains information about a specified list of Endpoints and the largest 
version 
  * of the state they have generated as known by the local endpoint.
  */
 

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestAck2Message.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestAck2Message.java?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestAck2Message.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestAck2Message.java 
Mon Apr 19 19:58:10 2010
@@ -61,13 +61,13 @@ class GossipDigestAck2MessageSerializer 
 {
     public void serialize(GossipDigestAck2Message gDigestAck2Message, 
DataOutputStream dos) throws IOException
     {
-        /* Use the EndPointState */
-        
EndPointStatesSerializationHelper.serialize(gDigestAck2Message.epStateMap_, 
dos);
+        /* Use the EndpointState */
+        
EndpointStatesSerializationHelper.serialize(gDigestAck2Message.epStateMap_, 
dos);
     }
 
     public GossipDigestAck2Message deserialize(DataInputStream dis) throws 
IOException
     {
-        Map<InetAddress, EndpointState> epStateMap = 
EndPointStatesSerializationHelper.deserialize(dis);
+        Map<InetAddress, EndpointState> epStateMap = 
EndpointStatesSerializationHelper.deserialize(dis);
         return new GossipDigestAck2Message(epStateMap);        
     }
 }

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestAckMessage.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestAckMessage.java?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestAckMessage.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestAckMessage.java 
Mon Apr 19 19:58:10 2010
@@ -73,10 +73,10 @@ class GossipDigestAckMessageSerializer i
         /* Use the helper to serialize the GossipDigestList */
         boolean bContinue = 
GossipDigestSerializationHelper.serialize(gDigestAckMessage.gDigestList_, dos);
         dos.writeBoolean(bContinue);
-        /* Use the EndPointState */
+        /* Use the EndpointState */
         if ( bContinue )
         {
-            
EndPointStatesSerializationHelper.serialize(gDigestAckMessage.epStateMap_, 
dos);            
+            
EndpointStatesSerializationHelper.serialize(gDigestAckMessage.epStateMap_, 
dos);            
         }
     }
 
@@ -88,7 +88,7 @@ class GossipDigestAckMessageSerializer i
 
         if ( bContinue )
         {
-            epStateMap = EndPointStatesSerializationHelper.deserialize(dis);   
                                 
+            epStateMap = EndpointStatesSerializationHelper.deserialize(dis);   
                                 
         }
         return new GossipDigestAckMessage(gDigestList, epStateMap);
     }

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestSynMessage.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestSynMessage.java?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestSynMessage.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/gms/GossipDigestSynMessage.java 
Mon Apr 19 19:58:10 2010
@@ -112,9 +112,9 @@ class GossipDigestSerializationHelper
     }
 }
 
-class EndPointStatesSerializationHelper
+class EndpointStatesSerializationHelper
 {
-    private static final Logger logger_ = 
LoggerFactory.getLogger(EndPointStatesSerializationHelper.class);
+    private static final Logger logger_ = 
LoggerFactory.getLogger(EndpointStatesSerializationHelper.class);
 
     static boolean serialize(Map<InetAddress, EndpointState> epStateMap, 
DataOutputStream dos) throws IOException
     {
@@ -151,7 +151,7 @@ class EndPointStatesSerializationHelper
         {
             if ( dis.available() == 0 )
             {
-                logger_.info("Remaining bytes zero. Stopping deserialization 
in EndPointState.");
+                logger_.info("Remaining bytes zero. Stopping deserialization 
in EndpointState.");
                 break;
             }
             // int length = dis.readInt();            

Modified: cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/gms/Gossiper.java Mon Apr 19 
19:58:10 2010
@@ -111,7 +111,7 @@ public class Gossiper implements IFailur
     private long FatClientTimeout_;
     private Random random_ = new Random();
 
-    /* subscribers for interest in EndPointState change */
+    /* subscribers for interest in EndpointState change */
     private List<IEndpointStateChangeSubscriber> subscribers_ = new 
ArrayList<IEndpointStateChangeSubscriber>();
 
     /* live member set */
@@ -143,7 +143,7 @@ public class Gossiper implements IFailur
         FailureDetector.instance.registerFailureDetectionEventListener(this);
     }
 
-    /** Register with the Gossiper for EndPointState notifications */
+    /** Register with the Gossiper for EndpointState notifications */
     public synchronized void register(IEndpointStateChangeSubscriber 
subscriber)
     {
         subscribers_.add(subscriber);
@@ -577,7 +577,7 @@ public class Gossiper implements IFailur
      * immediately mark it alive.
      *
      * @param ep endpoint
-     * @param epState EndPointState for the endpoint
+     * @param epState EndpointState for the endpoint
      * @param isKnownNode is this node familiar to us already (present in 
endpointStateMap)
      */
     private void handleMajorStateChange(InetAddress ep, EndpointState epState, 
boolean isKnownNode)
@@ -914,7 +914,7 @@ public class Gossiper implements IFailur
 
             /*
              * These digests have their maxVersion set to the difference of 
the version
-             * of the local EndPointState and the version found in the 
GossipDigest.
+             * of the local EndpointState and the version found in the 
GossipDigest.
             */
             List<GossipDigest> diffDigests = new ArrayList<GossipDigest>();
             for ( GossipDigest gDigest : gDigestList )

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/gms/IEndpointStateChangePublisher.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/gms/IEndpointStateChangePublisher.java?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/gms/IEndpointStateChangePublisher.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/gms/IEndpointStateChangePublisher.java
 Mon Apr 19 19:58:10 2010
@@ -27,13 +27,13 @@ public interface IEndpointStateChangePub
 {
     /**
      * Register for interesting state changes.
-     * @param subcriber module which implements the 
IEndPointStateChangeSubscriber
+     * @param subcriber module which implements the 
IEndpointStateChangeSubscriber
      */
     public void register(IEndpointStateChangeSubscriber subcriber);
     
     /**
      * Unregister interest for state changes.
-     * @param subcriber module which implements the 
IEndPointStateChangeSubscriber
+     * @param subcriber module which implements the 
IEndpointStateChangeSubscriber
      */
     public void unregister(IEndpointStateChangeSubscriber subcriber);
 }

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/gms/IEndpointStateChangeSubscriber.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/gms/IEndpointStateChangeSubscriber.java?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/gms/IEndpointStateChangeSubscriber.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/gms/IEndpointStateChangeSubscriber.java
 Mon Apr 19 19:58:10 2010
@@ -21,7 +21,7 @@ package org.apache.cassandra.gms;
 import java.net.InetAddress;
 
 /**
- * This is called by an instance of the IEndPointStateChangePublisher to notify
+ * This is called by an instance of the IEndpointStateChangePublisher to notify
  * interested parties about changes in the the state associated with any 
endpoint.
  * For instance if node A figures there is a changes in state for an endpoint B
  * it notifies all interested parties of this change. It is upto to the 
registered

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterEndpointSnitch.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterEndpointSnitch.java?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterEndpointSnitch.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterEndpointSnitch.java
 Mon Apr 19 19:58:10 2010
@@ -34,17 +34,17 @@ import org.apache.cassandra.utils.XMLUti
 import org.xml.sax.SAXException;
 
 /**
- * DataCenterEndPointSnitch
+ * DataCenterEndpointSnitch
  * <p/>
  * This class basically reads the configuration and sets the IP Ranges to a
  * hashMap which can be read later. this class also provides a way to compare 2
- * EndPoints and also get details from the same.
+ * Endpoints and also get details from the same.
  */
 
 public class DatacenterEndpointSnitch extends AbstractEndpointSnitch
 {
     /**
-     * This Map will contain the information of the EndPoints and its Location
+     * This Map will contain the information of the Endpoints and its Location
      * (Datacenter and RAC)
      */
     private Map<Byte, Map<Byte, String>> ipDC = new HashMap<Byte, Map<Byte, 
String>>();

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java
 Mon Apr 19 19:58:10 2010
@@ -38,7 +38,7 @@ import org.apache.cassandra.thrift.Consi
  * other Datacenter.... For example if we have 5 nodes this stategy will make
  * sure to make 2 copies out of 5 in other dataceneter.
  * <p/>
- * This class also caches the EndPoints and invalidates the cache if there is a
+ * This class also caches the Endpoints and invalidates the cache if there is a
  * change in the number of tokens.
  */
 public class DatacenterShardStategy extends AbstractReplicationStrategy
@@ -62,8 +62,8 @@ public class DatacenterShardStategy exte
     }
 
     /**
-     * This Method will get the required information of the EndPoint from the
-     * DataCenterEndPointSnitch and poopulates this singleton class.
+     * This Method will get the required information of the Endpoint from the
+     * DataCenterEndpointSnitch and poopulates this singleton class.
      */
     private synchronized void loadEndpoints(TokenMetadata metadata) throws 
IOException
     {
@@ -111,7 +111,7 @@ public class DatacenterShardStategy exte
         super(tokenMetadata, snitch);
         if ((!(snitch instanceof DatacenterEndpointSnitch)))
         {
-            throw new IllegalArgumentException("DatacenterShardStrategy 
requires DatacenterEndPointSnitch");
+            throw new IllegalArgumentException("DatacenterShardStrategy 
requires DatacenterEndpointSnitch");
         }
     }
 

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/locator/RackAwareStrategy.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/locator/RackAwareStrategy.java?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/locator/RackAwareStrategy.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/locator/RackAwareStrategy.java 
Mon Apr 19 19:58:10 2010
@@ -40,7 +40,7 @@ public class RackAwareStrategy extends A
     {
         super(tokenMetadata, snitch);
         if (!(snitch instanceof EndpointSnitch))
-            throw new IllegalArgumentException(("RackAwareStrategy requires 
EndPointSnitch."));
+            throw new IllegalArgumentException(("RackAwareStrategy requires 
EndpointSnitch."));
     }
 
     public ArrayList<InetAddress> getNaturalEndpoints(Token token, 
TokenMetadata metadata, String table)

Modified: cassandra/trunk/test/conf/cassandra.xml
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/test/conf/cassandra.xml?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- cassandra/trunk/test/conf/cassandra.xml (original)
+++ cassandra/trunk/test/conf/cassandra.xml Mon Apr 19 19:58:10 2010
@@ -58,13 +58,13 @@
        <ColumnFamily ColumnType="Super" CompareSubcolumnsWith="TimeUUIDType" 
Name="Super4"/>
        
<ReplicaPlacementStrategy>org.apache.cassandra.locator.RackUnawareStrategy</ReplicaPlacementStrategy>
        <ReplicationFactor>1</ReplicationFactor>
-       
<EndpointSnitch>org.apache.cassandra.locator.EndPointSnitch</EndpointSnitch>
+       
<EndpointSnitch>org.apache.cassandra.locator.EndpointSnitch</EndpointSnitch>
      </Keyspace>
      <Keyspace Name = "Keyspace3">
        <ColumnFamily Name="Standard1"/>
        
<ReplicaPlacementStrategy>org.apache.cassandra.locator.RackUnawareStrategy</ReplicaPlacementStrategy>
        <ReplicationFactor>5</ReplicationFactor>
-       
<EndpointSnitch>org.apache.cassandra.locator.EndPointSnitch</EndpointSnitch>
+       
<EndpointSnitch>org.apache.cassandra.locator.EndpointSnitch</EndpointSnitch>
      </Keyspace>
      <Keyspace Name = "Keyspace4">
        <ColumnFamily Name="Standard1"/>
@@ -73,7 +73,7 @@
        <ColumnFamily ColumnType="Super" CompareSubcolumnsWith="TimeUUIDType" 
Name="Super4"/>
        
<ReplicaPlacementStrategy>org.apache.cassandra.locator.RackUnawareStrategy</ReplicaPlacementStrategy>
        <ReplicationFactor>3</ReplicationFactor>
-       
<EndpointSnitch>org.apache.cassandra.locator.EndPointSnitch</EndpointSnitch>
+       
<EndpointSnitch>org.apache.cassandra.locator.EndpointSnitch</EndpointSnitch>
      </Keyspace>
    </Keyspaces>
    <Seeds>

Modified: cassandra/trunk/test/system/__init__.py
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/test/system/__init__.py?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- cassandra/trunk/test/system/__init__.py (original)
+++ cassandra/trunk/test/system/__init__.py Mon Apr 19 19:58:10 2010
@@ -142,7 +142,7 @@ class ThriftTester(BaseTester):
         self.client.transport.close()
         
     def define_schema(self):
-        keyspace1 = Cassandra.KsDef('Keyspace1', 
'org.apache.cassandra.locator.RackUnawareStrategy', 1, 
'org.apache.cassandra.locator.EndPointSnitch',
+        keyspace1 = Cassandra.KsDef('Keyspace1', 
'org.apache.cassandra.locator.RackUnawareStrategy', 1, 
'org.apache.cassandra.locator.EndpointSnitch',
         [
             Cassandra.CfDef('Keyspace1', 'Standard1'),
             Cassandra.CfDef('Keyspace1', 'Standard2'), 
@@ -154,7 +154,7 @@ class ThriftTester(BaseTester):
             Cassandra.CfDef('Keyspace1', 'Super4', column_type='Super', 
subcomparator_type='UTF8Type')
         ])
 
-        keyspace2 = Cassandra.KsDef('Keyspace2', 
'org.apache.cassandra.locator.RackUnawareStrategy', 1, 
'org.apache.cassandra.locator.EndPointSnitch',
+        keyspace2 = Cassandra.KsDef('Keyspace2', 
'org.apache.cassandra.locator.RackUnawareStrategy', 1, 
'org.apache.cassandra.locator.EndpointSnitch',
         [
             Cassandra.CfDef('Keyspace2', 'Standard1'),
             Cassandra.CfDef('Keyspace2', 'Standard3'),
@@ -162,12 +162,12 @@ class ThriftTester(BaseTester):
             Cassandra.CfDef('Keyspace2', 'Super4', column_type='Super', 
subcomparator_type='TimeUUIDType'),
         ])
 
-        keyspace3 = Cassandra.KsDef('Keyspace3', 
'org.apache.cassandra.locator.RackUnawareStrategy', 5, 
'org.apache.cassandra.locator.EndPointSnitch',
+        keyspace3 = Cassandra.KsDef('Keyspace3', 
'org.apache.cassandra.locator.RackUnawareStrategy', 5, 
'org.apache.cassandra.locator.EndpointSnitch',
         [
             Cassandra.CfDef('Keyspace3', 'Standard1'),
         ])
 
-        keyspace4 = Cassandra.KsDef('Keyspace4', 
'org.apache.cassandra.locator.RackUnawareStrategy', 3, 
'org.apache.cassandra.locator.EndPointSnitch',
+        keyspace4 = Cassandra.KsDef('Keyspace4', 
'org.apache.cassandra.locator.RackUnawareStrategy', 3, 
'org.apache.cassandra.locator.EndpointSnitch',
         [
             Cassandra.CfDef('Keyspace4', 'Standard1'),
             Cassandra.CfDef('Keyspace4', 'Standard3'),

Modified: cassandra/trunk/test/system/test_thrift_server.py
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/test/system/test_thrift_server.py?rev=935717&r1=935716&r2=935717&view=diff
==============================================================================
--- cassandra/trunk/test/system/test_thrift_server.py (original)
+++ cassandra/trunk/test/system/test_thrift_server.py Mon Apr 19 19:58:10 2010
@@ -949,7 +949,7 @@ class TestMutations(ThriftTester):
     def test_system_keyspace_operations(self):
         """ Test keyspace (add, drop, rename) operations """
         # create
-        keyspace = KsDef('CreateKeyspace', 
'org.apache.cassandra.locator.RackUnawareStrategy', 1, 
'org.apache.cassandra.locator.EndPointSnitch',
+        keyspace = KsDef('CreateKeyspace', 
'org.apache.cassandra.locator.RackUnawareStrategy', 1, 
'org.apache.cassandra.locator.EndpointSnitch',
         [
             CfDef('CreateKeyspace', 'CreateKsCf')
         ])


Reply via email to