source: proiecte/HadoopJUnit/hadoop-0.20.1/src/core/org/apache/hadoop/net/NetUtils.java @ 120

Last change on this file since 120 was 120, checked in by (none), 14 years ago

Added the mail files for the Hadoop JUNit Project

  • Property svn:executable set to *
File size: 16.2 KB
Line 
1/**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements.  See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership.  The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License.  You may obtain a copy of the License at
9 *
10 *     http://www.apache.org/licenses/LICENSE-2.0
11 *
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
17 */
18package org.apache.hadoop.net;
19
20import java.io.IOException;
21import java.io.InputStream;
22import java.io.OutputStream;
23import java.net.InetAddress;
24import java.net.InetSocketAddress;
25import java.net.Socket;
26import java.net.SocketAddress;
27import java.net.URI;
28import java.net.UnknownHostException;
29import java.nio.channels.SocketChannel;
30import java.util.Map.Entry;
31import java.util.*;
32
33import javax.net.SocketFactory;
34
35import org.apache.commons.logging.Log;
36import org.apache.commons.logging.LogFactory;
37import org.apache.hadoop.conf.Configuration;
38import org.apache.hadoop.fs.Path;
39import org.apache.hadoop.ipc.Server;
40import org.apache.hadoop.ipc.VersionedProtocol;
41import org.apache.hadoop.util.ReflectionUtils;
42
43public class NetUtils {
44  private static final Log LOG = LogFactory.getLog(NetUtils.class);
45 
46  private static Map<String, String> hostToResolved = 
47                                     new HashMap<String, String>();
48
49  /**
50   * Get the socket factory for the given class according to its
51   * configuration parameter
52   * <tt>hadoop.rpc.socket.factory.class.&lt;ClassName&gt;</tt>. When no
53   * such parameter exists then fall back on the default socket factory as
54   * configured by <tt>hadoop.rpc.socket.factory.class.default</tt>. If
55   * this default socket factory is not configured, then fall back on the JVM
56   * default socket factory.
57   *
58   * @param conf the configuration
59   * @param clazz the class (usually a {@link VersionedProtocol})
60   * @return a socket factory
61   */
62  public static SocketFactory getSocketFactory(Configuration conf,
63      Class<?> clazz) {
64
65    SocketFactory factory = null;
66
67    String propValue =
68        conf.get("hadoop.rpc.socket.factory.class." + clazz.getSimpleName());
69    if ((propValue != null) && (propValue.length() > 0))
70      factory = getSocketFactoryFromProperty(conf, propValue);
71
72    if (factory == null)
73      factory = getDefaultSocketFactory(conf);
74
75    return factory;
76  }
77
78  /**
79   * Get the default socket factory as specified by the configuration
80   * parameter <tt>hadoop.rpc.socket.factory.default</tt>
81   *
82   * @param conf the configuration
83   * @return the default socket factory as specified in the configuration or
84   *         the JVM default socket factory if the configuration does not
85   *         contain a default socket factory property.
86   */
87  public static SocketFactory getDefaultSocketFactory(Configuration conf) {
88
89    String propValue = conf.get("hadoop.rpc.socket.factory.class.default");
90    if ((propValue == null) || (propValue.length() == 0))
91      return SocketFactory.getDefault();
92
93    return getSocketFactoryFromProperty(conf, propValue);
94  }
95
96  /**
97   * Get the socket factory corresponding to the given proxy URI. If the
98   * given proxy URI corresponds to an absence of configuration parameter,
99   * returns null. If the URI is malformed raises an exception.
100   *
101   * @param propValue the property which is the class name of the
102   *        SocketFactory to instantiate; assumed non null and non empty.
103   * @return a socket factory as defined in the property value.
104   */
105  public static SocketFactory getSocketFactoryFromProperty(
106      Configuration conf, String propValue) {
107
108    try {
109      Class<?> theClass = conf.getClassByName(propValue);
110      return (SocketFactory) ReflectionUtils.newInstance(theClass, conf);
111
112    } catch (ClassNotFoundException cnfe) {
113      throw new RuntimeException("Socket Factory class not found: " + cnfe);
114    }
115  }
116
117  /**
118   * Util method to build socket addr from either:
119   *   <host>:<post>
120   *   <fs>://<host>:<port>/<path>
121   */
122  public static InetSocketAddress createSocketAddr(String target) {
123    return createSocketAddr(target, -1);
124  }
125
126  /**
127   * Util method to build socket addr from either:
128   *   <host>
129   *   <host>:<post>
130   *   <fs>://<host>:<port>/<path>
131   */
132  public static InetSocketAddress createSocketAddr(String target,
133                                                   int defaultPort) {
134    int colonIndex = target.indexOf(':');
135    if (colonIndex < 0 && defaultPort == -1) {
136      throw new RuntimeException("Not a host:port pair: " + target);
137    }
138    String hostname;
139    int port = -1;
140    if (!target.contains("/")) {
141      if (colonIndex == -1) {
142        hostname = target;
143      } else {
144        // must be the old style <host>:<port>
145        hostname = target.substring(0, colonIndex);
146        port = Integer.parseInt(target.substring(colonIndex + 1));
147      }
148    } else {
149      // a new uri
150      URI addr = new Path(target).toUri();
151      hostname = addr.getHost();
152      port = addr.getPort();
153    }
154
155    if (port == -1) {
156      port = defaultPort;
157    }
158 
159    if (getStaticResolution(hostname) != null) {
160      hostname = getStaticResolution(hostname);
161    }
162    return new InetSocketAddress(hostname, port);
163  }
164
165  /**
166   * Handle the transition from pairs of attributes specifying a host and port
167   * to a single colon separated one.
168   * @param conf the configuration to check
169   * @param oldBindAddressName the old address attribute name
170   * @param oldPortName the old port attribute name
171   * @param newBindAddressName the new combined name
172   * @return the complete address from the configuration
173   */
174  @Deprecated
175  public static String getServerAddress(Configuration conf,
176                                        String oldBindAddressName,
177                                        String oldPortName,
178                                        String newBindAddressName) {
179    String oldAddr = conf.get(oldBindAddressName);
180    String oldPort = conf.get(oldPortName);
181    String newAddrPort = conf.get(newBindAddressName);
182    if (oldAddr == null && oldPort == null) {
183      return newAddrPort;
184    }
185    String[] newAddrPortParts = newAddrPort.split(":",2);
186    if (newAddrPortParts.length != 2) {
187      throw new IllegalArgumentException("Invalid address/port: " + 
188                                         newAddrPort);
189    }
190    if (oldAddr == null) {
191      oldAddr = newAddrPortParts[0];
192    } else {
193      LOG.warn("Configuration parameter " + oldBindAddressName +
194               " is deprecated. Use " + newBindAddressName + " instead.");
195    }
196    if (oldPort == null) {
197      oldPort = newAddrPortParts[1];
198    } else {
199      LOG.warn("Configuration parameter " + oldPortName +
200               " is deprecated. Use " + newBindAddressName + " instead.");     
201    }
202    return oldAddr + ":" + oldPort;
203  }
204 
205  /**
206   * Adds a static resolution for host. This can be used for setting up
207   * hostnames with names that are fake to point to a well known host. For e.g.
208   * in some testcases we require to have daemons with different hostnames
209   * running on the same machine. In order to create connections to these
210   * daemons, one can set up mappings from those hostnames to "localhost".
211   * {@link NetUtils#getStaticResolution(String)} can be used to query for
212   * the actual hostname.
213   * @param host
214   * @param resolvedName
215   */
216  public static void addStaticResolution(String host, String resolvedName) {
217    synchronized (hostToResolved) {
218      hostToResolved.put(host, resolvedName);
219    }
220  }
221 
222  /**
223   * Retrieves the resolved name for the passed host. The resolved name must
224   * have been set earlier using
225   * {@link NetUtils#addStaticResolution(String, String)}
226   * @param host
227   * @return the resolution
228   */
229  public static String getStaticResolution(String host) {
230    synchronized (hostToResolved) {
231      return hostToResolved.get(host);
232    }
233  }
234 
235  /**
236   * This is used to get all the resolutions that were added using
237   * {@link NetUtils#addStaticResolution(String, String)}. The return
238   * value is a List each element of which contains an array of String
239   * of the form String[0]=hostname, String[1]=resolved-hostname
240   * @return the list of resolutions
241   */
242  public static List <String[]> getAllStaticResolutions() {
243    synchronized (hostToResolved) {
244      Set <Entry <String, String>>entries = hostToResolved.entrySet();
245      if (entries.size() == 0) {
246        return null;
247      }
248      List <String[]> l = new ArrayList<String[]>(entries.size());
249      for (Entry<String, String> e : entries) {
250        l.add(new String[] {e.getKey(), e.getValue()});
251      }
252    return l;
253    }
254  }
255 
256  /**
257   * Returns InetSocketAddress that a client can use to
258   * connect to the server. Server.getListenerAddress() is not correct when
259   * the server binds to "0.0.0.0". This returns "127.0.0.1:port" when
260   * the getListenerAddress() returns "0.0.0.0:port".
261   *
262   * @param server
263   * @return socket address that a client can use to connect to the server.
264   */
265  public static InetSocketAddress getConnectAddress(Server server) {
266    InetSocketAddress addr = server.getListenerAddress();
267    if (addr.getAddress().getHostAddress().equals("0.0.0.0")) {
268      addr = new InetSocketAddress("127.0.0.1", addr.getPort());
269    }
270    return addr;
271  }
272 
273  /**
274   * Same as getInputStream(socket, socket.getSoTimeout()).<br><br>
275   *
276   * From documentation for {@link #getInputStream(Socket, long)}:<br>
277   * Returns InputStream for the socket. If the socket has an associated
278   * SocketChannel then it returns a
279   * {@link SocketInputStream} with the given timeout. If the socket does not
280   * have a channel, {@link Socket#getInputStream()} is returned. In the later
281   * case, the timeout argument is ignored and the timeout set with
282   * {@link Socket#setSoTimeout(int)} applies for reads.<br><br>
283   *
284   * Any socket created using socket factories returned by {@link #NetUtils},
285   * must use this interface instead of {@link Socket#getInputStream()}.
286   *     
287   * @see #getInputStream(Socket, long)
288   *
289   * @param socket
290   * @return InputStream for reading from the socket.
291   * @throws IOException
292   */
293  public static InputStream getInputStream(Socket socket) 
294                                           throws IOException {
295    return getInputStream(socket, socket.getSoTimeout());
296  }
297 
298  /**
299   * Returns InputStream for the socket. If the socket has an associated
300   * SocketChannel then it returns a
301   * {@link SocketInputStream} with the given timeout. If the socket does not
302   * have a channel, {@link Socket#getInputStream()} is returned. In the later
303   * case, the timeout argument is ignored and the timeout set with
304   * {@link Socket#setSoTimeout(int)} applies for reads.<br><br>
305   *
306   * Any socket created using socket factories returned by {@link #NetUtils},
307   * must use this interface instead of {@link Socket#getInputStream()}.
308   *     
309   * @see Socket#getChannel()
310   *
311   * @param socket
312   * @param timeout timeout in milliseconds. This may not always apply. zero
313   *        for waiting as long as necessary.
314   * @return InputStream for reading from the socket.
315   * @throws IOException
316   */
317  public static InputStream getInputStream(Socket socket, long timeout) 
318                                           throws IOException {
319    return (socket.getChannel() == null) ? 
320          socket.getInputStream() : new SocketInputStream(socket, timeout);
321  }
322 
323  /**
324   * Same as getOutputStream(socket, 0). Timeout of zero implies write will
325   * wait until data is available.<br><br>
326   *
327   * From documentation for {@link #getOutputStream(Socket, long)} : <br>
328   * Returns OutputStream for the socket. If the socket has an associated
329   * SocketChannel then it returns a
330   * {@link SocketOutputStream} with the given timeout. If the socket does not
331   * have a channel, {@link Socket#getOutputStream()} is returned. In the later
332   * case, the timeout argument is ignored and the write will wait until
333   * data is available.<br><br>
334   *
335   * Any socket created using socket factories returned by {@link #NetUtils},
336   * must use this interface instead of {@link Socket#getOutputStream()}.
337   *
338   * @see #getOutputStream(Socket, long)
339   *
340   * @param socket
341   * @return OutputStream for writing to the socket.
342   * @throws IOException
343   */ 
344  public static OutputStream getOutputStream(Socket socket) 
345                                             throws IOException {
346    return getOutputStream(socket, 0);
347  }
348 
349  /**
350   * Returns OutputStream for the socket. If the socket has an associated
351   * SocketChannel then it returns a
352   * {@link SocketOutputStream} with the given timeout. If the socket does not
353   * have a channel, {@link Socket#getOutputStream()} is returned. In the later
354   * case, the timeout argument is ignored and the write will wait until
355   * data is available.<br><br>
356   *
357   * Any socket created using socket factories returned by {@link #NetUtils},
358   * must use this interface instead of {@link Socket#getOutputStream()}.
359   *
360   * @see Socket#getChannel()
361   *
362   * @param socket
363   * @param timeout timeout in milliseconds. This may not always apply. zero
364   *        for waiting as long as necessary.
365   * @return OutputStream for writing to the socket.
366   * @throws IOException   
367   */
368  public static OutputStream getOutputStream(Socket socket, long timeout) 
369                                             throws IOException {
370    return (socket.getChannel() == null) ? 
371            socket.getOutputStream() : new SocketOutputStream(socket, timeout);           
372  }
373 
374  /**
375   * This is a drop-in replacement for
376   * {@link Socket#connect(SocketAddress, int)}.
377   * In the case of normal sockets that don't have associated channels, this
378   * just invokes <code>socket.connect(endpoint, timeout)</code>. If
379   * <code>socket.getChannel()</code> returns a non-null channel,
380   * connect is implemented using Hadoop's selectors. This is done mainly
381   * to avoid Sun's connect implementation from creating thread-local
382   * selectors, since Hadoop does not have control on when these are closed
383   * and could end up taking all the available file descriptors.
384   *
385   * @see java.net.Socket#connect(java.net.SocketAddress, int)
386   *
387   * @param socket
388   * @param endpoint
389   * @param timeout - timeout in milliseconds
390   */
391  public static void connect(Socket socket, 
392                             SocketAddress endpoint, 
393                             int timeout) throws IOException {
394    if (socket == null || endpoint == null || timeout < 0) {
395      throw new IllegalArgumentException("Illegal argument for connect()");
396    }
397   
398    SocketChannel ch = socket.getChannel();
399   
400    if (ch == null) {
401      // let the default implementation handle it.
402      socket.connect(endpoint, timeout);
403    } else {
404      SocketIOWithTimeout.connect(ch, endpoint, timeout);
405    }
406  }
407 
408  /**
409   * Given a string representation of a host, return its ip address
410   * in textual presentation.
411   *
412   * @param name a string representation of a host:
413   *             either a textual representation its IP address or its host name
414   * @return its IP address in the string format
415   */
416  public static String normalizeHostName(String name) {
417    if (Character.digit(name.charAt(0), 16) != -1) { // it is an IP
418      return name;
419    } else {
420      try {
421        InetAddress ipAddress = InetAddress.getByName(name);
422        return ipAddress.getHostAddress();
423      } catch (UnknownHostException e) {
424        return name;
425      }
426    }
427  }
428 
429  /**
430   * Given a collection of string representation of hosts, return a list of
431   * corresponding IP addresses in the textual representation.
432   *
433   * @param names a collection of string representations of hosts
434   * @return a list of corresponding IP addresses in the string format
435   * @see #normalizeHostName(String)
436   */
437  public static List<String> normalizeHostNames(Collection<String> names) {
438    List<String> hostNames = new ArrayList<String>(names.size());
439    for (String name : names) {
440      hostNames.add(normalizeHostName(name));
441    }
442    return hostNames;
443  }
444}
Note: See TracBrowser for help on using the repository browser.