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/04/02 23:07:37 UTC

svn commit: r761426 - in /incubator/cassandra/trunk/src/org/apache/cassandra: gms/FailureDetector.java net/EndPoint.java utils/FBUtilities.java

Author: jbellis
Date: Thu Apr  2 21:07:37 2009
New Revision: 761426

URL: http://svn.apache.org/viewvc?rev=761426&view=rev
Log:
replace getLocalHostName with getHostName.  localHostName caused problems when it was not equal to hostName by making endpoints that should be equal, not.

Modified:
    incubator/cassandra/trunk/src/org/apache/cassandra/gms/FailureDetector.java
    incubator/cassandra/trunk/src/org/apache/cassandra/net/EndPoint.java
    incubator/cassandra/trunk/src/org/apache/cassandra/utils/FBUtilities.java

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/gms/FailureDetector.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/gms/FailureDetector.java?rev=761426&r1=761425&r2=761426&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/gms/FailureDetector.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/gms/FailureDetector.java Thu Apr  2 21:07:37 2009
@@ -140,7 +140,7 @@
         try
         {
             /* If the endpoint in question is the local endpoint return true. */
-            String localHost = FBUtilities.getLocalHostName();
+            String localHost = FBUtilities.getHostName();
             if ( localHost.equals( ep.getHost() ) )
                     return true;
         }

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/net/EndPoint.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/net/EndPoint.java?rev=761426&r1=761425&r2=761426&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/net/EndPoint.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/net/EndPoint.java Thu Apr  2 21:07:37 2009
@@ -1 +1 @@
-/**
 * 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.net;


import java.io.IOException;
import java.io.Serializable;
import java.net.*;
import java.nio.ByteBuffer;
import java.nio.CharBuffer;
import java.u
 til.HashMap;
import java.util.Map;

import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.LogUtil;
import org.apache.log4j.Logger;

/**
 * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com )
 */

public class EndPoint implements Serializable, Comparable<EndPoint>
{
	// logging and profiling.
	private static Logger logger_ = Logger.getLogger(EndPoint.class);
	private static final long serialVersionUID = -4962625949179835907L;
	private static Map<CharBuffer, String> hostNames_ = new HashMap<CharBuffer, String>();
    protected static final int randomPort_ = 5555;
    public static EndPoint randomLocalEndPoint_;
    
    static
    {
        try
        {
            randomLocalEndPoint_ = new EndPoint(FBUtilities.getHostName(), EndPoint.randomPort_);
        }        
        catch ( IOException ex )
        {
            logger_.warn(LogUtil.throwableToString(ex));
        }
    }

	private String host_;
	pri
 vate int port_;

	private transient InetSocketAddress ia_;

	/* Ctor for JAXB. DO NOT DELETE */
	private EndPoint()
	{
	}

	public EndPoint(String host, int port)
	{
		/*
		 * Attempts to resolve the host, but does not fail if it cannot.
		 */
		host_ = host;
		port_ = port;
	}

	// create a local endpoint id
	public EndPoint(int port)
	{
		try
		{
			host_ = FBUtilities.getLocalHostName();
			port_ = port;
		}
		catch (UnknownHostException e)
		{
			logger_.warn(LogUtil.throwableToString(e));
		}
	}

	public String getHost()
	{
		return host_;
	}

	public int getPort()
	{
		return port_;
	}

	public void setPort(int port)
	{
		port_ = port;
	}

	public InetSocketAddress getInetAddress()
	{
		if (ia_ == null || ia_.isUnresolved())
		{
			ia_ = new InetSocketAddress(host_, port_);
		}
		return ia_;
	}

	public boolean equals(Object o)
	{
		if (!(o instanceof EndPoint))
			return false;

		EndPoint rhs = (EndPoint) o;
		return (host_.equals(rhs.host_) && port_ == rhs.port_);
	
 }

	public int hashCode()
	{
		return (host_ + port_).hashCode();
	}

	public int compareTo(EndPoint rhs)
	{
		return host_.compareTo(rhs.host_);
	}

	public String toString()
	{
		return (host_ + ":" + port_);
	}

	public static EndPoint fromString(String str)
	{
		String[] values = str.split(":");
		return new EndPoint(values[0], Integer.parseInt(values[1]));
	}

	public static byte[] toBytes(EndPoint ep)
	{
		ByteBuffer buffer = ByteBuffer.allocate(6);
		byte[] iaBytes = ep.getInetAddress().getAddress().getAddress();
		buffer.put(iaBytes);
		buffer.put(MessagingService.toByteArray((short) ep.getPort()));
		buffer.flip();
		return buffer.array();
	}

	public static EndPoint fromBytes(byte[] bytes)
	{
		ByteBuffer buffer = ByteBuffer.allocate(4);
		System.arraycopy(bytes, 0, buffer.array(), 0, 4);
		byte[] portBytes = new byte[2];
		System.arraycopy(bytes, 4, portBytes, 0, portBytes.length);
		try
		{
			CharBuffer charBuffer = buffer.asCharBuffer();
			String host = hostNa
 mes_.get(charBuffer);
			if (host == null)
			{				
				host = InetAddress.getByAddress(buffer.array()).getHostName();				
				hostNames_.put(charBuffer, host);
			}
			int port = (int) MessagingService.byteArrayToShort(portBytes);
			return new EndPoint(host, port);
		}
		catch (UnknownHostException e)
		{
			throw new IllegalArgumentException(e);
		}
	}
}
\ No newline at end of file
+/**
 * 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.net;


import java.io.IOException;
import java.io.Serializable;
import java.net.*;
import java.nio.ByteBuffer;
import java.nio.CharBuffer;
import java.u
 til.HashMap;
import java.util.Map;

import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.LogUtil;
import org.apache.log4j.Logger;

/**
 * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com )
 */

public class EndPoint implements Serializable, Comparable<EndPoint>
{
	// logging and profiling.
	private static Logger logger_ = Logger.getLogger(EndPoint.class);
	private static final long serialVersionUID = -4962625949179835907L;
	private static Map<CharBuffer, String> hostNames_ = new HashMap<CharBuffer, String>();
    protected static final int randomPort_ = 5555;
    public static EndPoint randomLocalEndPoint_;
    
    static
    {
        try
        {
            randomLocalEndPoint_ = new EndPoint(FBUtilities.getHostName(), EndPoint.randomPort_);
        }        
        catch ( IOException ex )
        {
            logger_.warn(LogUtil.throwableToString(ex));
        }
    }

	private String host_;
	pri
 vate int port_;

	private transient InetSocketAddress ia_;

	/* Ctor for JAXB. DO NOT DELETE */
	private EndPoint()
	{
	}

	public EndPoint(String host, int port)
	{
		/*
		 * Attempts to resolve the host, but does not fail if it cannot.
		 */
		host_ = host;
		port_ = port;
	}

	// create a local endpoint id
	public EndPoint(int port)
	{
		try
		{
			host_ = FBUtilities.getHostName();
			port_ = port;
		}
		catch (UnknownHostException e)
		{
			logger_.warn(LogUtil.throwableToString(e));
		}
	}

	public String getHost()
	{
		return host_;
	}

	public int getPort()
	{
		return port_;
	}

	public void setPort(int port)
	{
		port_ = port;
	}

	public InetSocketAddress getInetAddress()
	{
		if (ia_ == null || ia_.isUnresolved())
		{
			ia_ = new InetSocketAddress(host_, port_);
		}
		return ia_;
	}

	public boolean equals(Object o)
	{
		if (!(o instanceof EndPoint))
			return false;

		EndPoint rhs = (EndPoint) o;
		return (host_.equals(rhs.host_) && port_ == rhs.port_);
	}

	p
 ublic int hashCode()
	{
		return (host_ + port_).hashCode();
	}

	public int compareTo(EndPoint rhs)
	{
		return host_.compareTo(rhs.host_);
	}

	public String toString()
	{
		return (host_ + ":" + port_);
	}

	public static EndPoint fromString(String str)
	{
		String[] values = str.split(":");
		return new EndPoint(values[0], Integer.parseInt(values[1]));
	}

	public static byte[] toBytes(EndPoint ep)
	{
		ByteBuffer buffer = ByteBuffer.allocate(6);
		byte[] iaBytes = ep.getInetAddress().getAddress().getAddress();
		buffer.put(iaBytes);
		buffer.put(MessagingService.toByteArray((short) ep.getPort()));
		buffer.flip();
		return buffer.array();
	}

	public static EndPoint fromBytes(byte[] bytes)
	{
		ByteBuffer buffer = ByteBuffer.allocate(4);
		System.arraycopy(bytes, 0, buffer.array(), 0, 4);
		byte[] portBytes = new byte[2];
		System.arraycopy(bytes, 4, portBytes, 0, portBytes.length);
		try
		{
			CharBuffer charBuffer = buffer.asCharBuffer();
			String host = hostNames_.
 get(charBuffer);
			if (host == null)
			{				
				host = InetAddress.getByAddress(buffer.array()).getHostName();				
				hostNames_.put(charBuffer, host);
			}
			int port = (int) MessagingService.byteArrayToShort(portBytes);
			return new EndPoint(host, port);
		}
		catch (UnknownHostException e)
		{
			throw new IllegalArgumentException(e);
		}
	}
}
\ No newline at end of file

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/utils/FBUtilities.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/utils/FBUtilities.java?rev=761426&r1=761425&r2=761426&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/utils/FBUtilities.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/utils/FBUtilities.java Thu Apr  2 21:07:37 2009
@@ -146,15 +146,6 @@
         return localInetAddress_;
     }
 
-    public static String getLocalHostName() throws UnknownHostException
-    {
-	if ( host_ == null )
-	{
-		host_ = getLocalAddress().getHostName();
-	}
-	return host_;
-    }
-
     public static String getHostName() throws UnknownHostException
     {
         return getLocalAddress().getCanonicalHostName();