View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  
21  package org.apache.hadoop.hbase.ipc;
22  
23  import org.apache.commons.logging.Log;
24  import org.apache.commons.logging.LogFactory;
25  import org.apache.hadoop.conf.Configuration;
26  import org.apache.hadoop.io.DataOutputBuffer;
27  import org.apache.hadoop.io.IOUtils;
28  import org.apache.hadoop.io.Writable;
29  import org.apache.hadoop.io.WritableUtils;
30  import org.apache.hadoop.ipc.RemoteException;
31  import org.apache.hadoop.net.NetUtils;
32  import org.apache.hadoop.security.UserGroupInformation;
33  import org.apache.hadoop.util.ReflectionUtils;
34  
35  import javax.net.SocketFactory;
36  import java.io.BufferedInputStream;
37  import java.io.BufferedOutputStream;
38  import java.io.DataInputStream;
39  import java.io.DataOutputStream;
40  import java.io.FilterInputStream;
41  import java.io.IOException;
42  import java.io.InputStream;
43  import java.net.ConnectException;
44  import java.net.InetSocketAddress;
45  import java.net.Socket;
46  import java.net.SocketTimeoutException;
47  import java.net.UnknownHostException;
48  import java.util.Hashtable;
49  import java.util.Iterator;
50  import java.util.Map.Entry;
51  import java.util.concurrent.atomic.AtomicBoolean;
52  import java.util.concurrent.atomic.AtomicLong;
53  
54  /** A client for an IPC service.  IPC calls take a single {@link Writable} as a
55   * parameter, and return a {@link Writable} as their value.  A service runs on
56   * a port and is defined by a parameter class and a value class.
57   *
58   * <p>This is the org.apache.hadoop.ipc.Client renamed as HBaseClient and
59   * moved into this package so can access package-private methods.
60   *
61   * @see HBaseServer
62   */
63  public class HBaseClient {
64  
65    private static final Log LOG =
66      LogFactory.getLog("org.apache.hadoop.ipc.HBaseClient");
67    protected final Hashtable<ConnectionId, Connection> connections =
68      new Hashtable<ConnectionId, Connection>();
69  
70    protected final Class<? extends Writable> valueClass;   // class of call values
71    protected int counter;                            // counter for call ids
72    protected final AtomicBoolean running = new AtomicBoolean(true); // if client runs
73    final protected Configuration conf;
74    final protected int maxIdleTime; // connections will be culled if it was idle for
75                             // maxIdleTime microsecs
76    final protected int maxRetries; //the max. no. of retries for socket connections
77    final protected long failureSleep; // Time to sleep before retry on failure.
78    protected final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
79    protected final boolean tcpKeepAlive; // if T then use keepalives
80    protected final int pingInterval; // how often sends ping to the server in msecs
81  
82    protected final SocketFactory socketFactory;           // how to create sockets
83    private int refCount = 1;
84  
85    final private static String PING_INTERVAL_NAME = "ipc.ping.interval";
86    final static int DEFAULT_PING_INTERVAL = 60000; // 1 min
87    final static int PING_CALL_ID = -1;
88  
89    /**
90     * set the ping interval value in configuration
91     *
92     * @param conf Configuration
93     * @param pingInterval the ping interval
94     */
95    @SuppressWarnings({"UnusedDeclaration"})
96    public static void setPingInterval(Configuration conf, int pingInterval) {
97      conf.setInt(PING_INTERVAL_NAME, pingInterval);
98    }
99  
100   /**
101    * Get the ping interval from configuration;
102    * If not set in the configuration, return the default value.
103    *
104    * @param conf Configuration
105    * @return the ping interval
106    */
107   static int getPingInterval(Configuration conf) {
108     return conf.getInt(PING_INTERVAL_NAME, DEFAULT_PING_INTERVAL);
109   }
110 
111   /**
112    * Increment this client's reference count
113    *
114    */
115   synchronized void incCount() {
116     refCount++;
117   }
118 
119   /**
120    * Decrement this client's reference count
121    *
122    */
123   synchronized void decCount() {
124     refCount--;
125   }
126 
127   /**
128    * Return if this client has no reference
129    *
130    * @return true if this client has no reference; false otherwise
131    */
132   synchronized boolean isZeroReference() {
133     return refCount==0;
134   }
135 
136   /** A call waiting for a value. */
137   private class Call {
138     final int id;                                       // call id
139     final Writable param;                               // parameter
140     Writable value;                               // value, null if error
141     IOException error;                            // exception, null if value
142     boolean done;                                 // true when call is done
143 
144     protected Call(Writable param) {
145       this.param = param;
146       synchronized (HBaseClient.this) {
147         this.id = counter++;
148       }
149     }
150 
151     /** Indicate when the call is complete and the
152      * value or error are available.  Notifies by default.  */
153     protected synchronized void callComplete() {
154       this.done = true;
155       notify();                                 // notify caller
156     }
157 
158     /** Set the exception when there is an error.
159      * Notify the caller the call is done.
160      *
161      * @param error exception thrown by the call; either local or remote
162      */
163     public synchronized void setException(IOException error) {
164       this.error = error;
165       callComplete();
166     }
167 
168     /** Set the return value when there is no error.
169      * Notify the caller the call is done.
170      *
171      * @param value return value of the call.
172      */
173     public synchronized void setValue(Writable value) {
174       this.value = value;
175       callComplete();
176     }
177   }
178 
179   /** Thread that reads responses and notifies callers.  Each connection owns a
180    * socket connected to a remote address.  Calls are multiplexed through this
181    * socket: responses may be delivered out of order. */
182   private class Connection extends Thread {
183     private ConnectionId remoteId;
184     private Socket socket = null;                 // connected socket
185     private DataInputStream in;
186     private DataOutputStream out;
187 
188     // currently active calls
189     private final Hashtable<Integer, Call> calls = new Hashtable<Integer, Call>();
190     private final AtomicLong lastActivity = new AtomicLong();// last I/O activity time
191     protected final AtomicBoolean shouldCloseConnection = new AtomicBoolean();  // indicate if the connection is closed
192     private IOException closeException; // close reason
193 
194     public Connection(InetSocketAddress address) throws IOException {
195       this(new ConnectionId(address, null));
196     }
197 
198     public Connection(ConnectionId remoteId) throws IOException {
199       if (remoteId.getAddress().isUnresolved()) {
200         throw new UnknownHostException("unknown host: " +
201                                        remoteId.getAddress().getHostName());
202       }
203       this.remoteId = remoteId;
204       UserGroupInformation ticket = remoteId.getTicket();
205       this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
206         remoteId.getAddress().toString() +
207         ((ticket==null)?" from an unknown user": (" from " + ticket.getUserName())));
208       this.setDaemon(true);
209     }
210 
211     /** Update lastActivity with the current time. */
212     private void touch() {
213       lastActivity.set(System.currentTimeMillis());
214     }
215 
216     /**
217      * Add a call to this connection's call queue and notify
218      * a listener; synchronized.
219      * Returns false if called during shutdown.
220      * @param call to add
221      * @return true if the call was added.
222      */
223     protected synchronized boolean addCall(Call call) {
224       if (shouldCloseConnection.get())
225         return false;
226       calls.put(call.id, call);
227       notify();
228       return true;
229     }
230 
231     /** This class sends a ping to the remote side when timeout on
232      * reading. If no failure is detected, it retries until at least
233      * a byte is read.
234      */
235     private class PingInputStream extends FilterInputStream {
236       /* constructor */
237       protected PingInputStream(InputStream in) {
238         super(in);
239       }
240 
241       /* Process timeout exception
242        * if the connection is not going to be closed, send a ping.
243        * otherwise, throw the timeout exception.
244        */
245       private void handleTimeout(SocketTimeoutException e) throws IOException {
246         if (shouldCloseConnection.get() || !running.get()) {
247           throw e;
248         }
249         sendPing();
250       }
251 
252       /** Read a byte from the stream.
253        * Send a ping if timeout on read. Retries if no failure is detected
254        * until a byte is read.
255        * @throws IOException for any IO problem other than socket timeout
256        */
257       @Override
258       public int read() throws IOException {
259         do {
260           try {
261             return super.read();
262           } catch (SocketTimeoutException e) {
263             handleTimeout(e);
264           }
265         } while (true);
266       }
267 
268       /** Read bytes into a buffer starting from offset <code>off</code>
269        * Send a ping if timeout on read. Retries if no failure is detected
270        * until a byte is read.
271        *
272        * @return the total number of bytes read; -1 if the connection is closed.
273        */
274       @Override
275       public int read(byte[] buf, int off, int len) throws IOException {
276         do {
277           try {
278             return super.read(buf, off, len);
279           } catch (SocketTimeoutException e) {
280             handleTimeout(e);
281           }
282         } while (true);
283       }
284     }
285 
286     /** Connect to the server and set up the I/O streams. It then sends
287      * a header to the server and starts
288      * the connection thread that waits for responses.
289      * @throws java.io.IOException e
290      */
291     protected synchronized void setupIOstreams() throws IOException {
292       if (socket != null || shouldCloseConnection.get()) {
293         return;
294       }
295 
296       short ioFailures = 0;
297       short timeoutFailures = 0;
298       try {
299         if (LOG.isDebugEnabled()) {
300           LOG.debug("Connecting to "+remoteId.getAddress());
301         }
302         while (true) {
303           try {
304             this.socket = socketFactory.createSocket();
305             this.socket.setTcpNoDelay(tcpNoDelay);
306             this.socket.setKeepAlive(tcpKeepAlive);
307             // connection time out is 20s
308             NetUtils.connect(this.socket, remoteId.getAddress(), 20000);
309             this.socket.setSoTimeout(pingInterval);
310             break;
311           } catch (SocketTimeoutException toe) {
312             handleConnectionFailure(timeoutFailures++, maxRetries, toe);
313           } catch (IOException ie) {
314             handleConnectionFailure(ioFailures++, maxRetries, ie);
315           }
316         }
317         this.in = new DataInputStream(new BufferedInputStream
318             (new PingInputStream(NetUtils.getInputStream(socket))));
319         this.out = new DataOutputStream
320             (new BufferedOutputStream(NetUtils.getOutputStream(socket)));
321         writeHeader();
322 
323         // update last activity time
324         touch();
325 
326         // start the receiver thread after the socket connection has been set up
327         start();
328       } catch (IOException e) {
329         markClosed(e);
330         close();
331 
332         throw e;
333       }
334     }
335 
336     /* Handle connection failures
337      *
338      * If the current number of retries is equal to the max number of retries,
339      * stop retrying and throw the exception; Otherwise backoff N seconds and
340      * try connecting again.
341      *
342      * This Method is only called from inside setupIOstreams(), which is
343      * synchronized. Hence the sleep is synchronized; the locks will be retained.
344      *
345      * @param curRetries current number of retries
346      * @param maxRetries max number of retries allowed
347      * @param ioe failure reason
348      * @throws IOException if max number of retries is reached
349      */
350     private void handleConnectionFailure(
351         int curRetries, int maxRetries, IOException ioe) throws IOException {
352       // close the current connection
353       if (socket != null) { // could be null if the socket creation failed
354         try {
355           socket.close();
356         } catch (IOException e) {
357           LOG.warn("Not able to close a socket", e);
358         }
359       }
360       // set socket to null so that the next call to setupIOstreams
361       // can start the process of connect all over again.
362       socket = null;
363 
364       // throw the exception if the maximum number of retries is reached
365       if (curRetries >= maxRetries) {
366         throw ioe;
367       }
368 
369       // otherwise back off and retry
370       try {
371         Thread.sleep(failureSleep);
372       } catch (InterruptedException ignored) {}
373 
374       LOG.info("Retrying connect to server: " + remoteId.getAddress() +
375         " after sleeping " + failureSleep + "ms. Already tried " + curRetries +
376         " time(s).");
377     }
378 
379     /* Write the header for each connection
380      * Out is not synchronized because only the first thread does this.
381      */
382     private void writeHeader() throws IOException {
383       out.write(HBaseServer.HEADER.array());
384       out.write(HBaseServer.CURRENT_VERSION);
385       //When there are more fields we can have ConnectionHeader Writable.
386       DataOutputBuffer buf = new DataOutputBuffer();
387       WritableUtils.writeString(buf, remoteId.getTicket().getUserName());
388       int bufLen = buf.getLength();
389       out.writeInt(bufLen);
390       out.write(buf.getData(), 0, bufLen);
391     }
392 
393     /* wait till someone signals us to start reading RPC response or
394      * it is idle too long, it is marked as to be closed,
395      * or the client is marked as not running.
396      *
397      * Return true if it is time to read a response; false otherwise.
398      */
399     @SuppressWarnings({"ThrowableInstanceNeverThrown"})
400     private synchronized boolean waitForWork() {
401       if (calls.isEmpty() && !shouldCloseConnection.get()  && running.get())  {
402         long timeout = maxIdleTime-
403               (System.currentTimeMillis()-lastActivity.get());
404         if (timeout>0) {
405           try {
406             wait(timeout);
407           } catch (InterruptedException ignored) {}
408         }
409       }
410 
411       if (!calls.isEmpty() && !shouldCloseConnection.get() && running.get()) {
412         return true;
413       } else if (shouldCloseConnection.get()) {
414         return false;
415       } else if (calls.isEmpty()) { // idle connection closed or stopped
416         markClosed(null);
417         return false;
418       } else { // get stopped but there are still pending requests
419         markClosed((IOException)new IOException().initCause(
420             new InterruptedException()));
421         return false;
422       }
423     }
424 
425     public InetSocketAddress getRemoteAddress() {
426       return remoteId.getAddress();
427     }
428 
429     /* Send a ping to the server if the time elapsed
430      * since last I/O activity is equal to or greater than the ping interval
431      */
432     protected synchronized void sendPing() throws IOException {
433       long curTime = System.currentTimeMillis();
434       if ( curTime - lastActivity.get() >= pingInterval) {
435         lastActivity.set(curTime);
436         //noinspection SynchronizeOnNonFinalField
437         synchronized (this.out) {
438           out.writeInt(PING_CALL_ID);
439           out.flush();
440         }
441       }
442     }
443 
444     @Override
445     public void run() {
446       if (LOG.isDebugEnabled())
447         LOG.debug(getName() + ": starting, having connections "
448             + connections.size());
449 
450       try {
451         while (waitForWork()) {//wait here for work - read or close connection
452           receiveResponse();
453         }
454       } catch (Throwable t) {
455         LOG.warn("Unexpected exception receiving call responses", t);
456         markClosed(new IOException("Unexpected exception receiving call responses", t));
457       }
458 
459       close();
460 
461       if (LOG.isDebugEnabled())
462         LOG.debug(getName() + ": stopped, remaining connections "
463             + connections.size());
464     }
465 
466     /* Initiates a call by sending the parameter to the remote server.
467      * Note: this is not called from the Connection thread, but by other
468      * threads.
469      */
470     protected void sendParam(Call call) {
471       if (shouldCloseConnection.get()) {
472         return;
473       }
474 
475       DataOutputBuffer d=null;
476       try {
477         //noinspection SynchronizeOnNonFinalField
478         synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC
479           if (LOG.isDebugEnabled())
480             LOG.debug(getName() + " sending #" + call.id);
481 
482           //for serializing the
483           //data to be written
484           d = new DataOutputBuffer();
485           d.writeInt(call.id);
486           call.param.write(d);
487           byte[] data = d.getData();
488           int dataLength = d.getLength();
489           out.writeInt(dataLength);      //first put the data length
490           out.write(data, 0, dataLength);//write the data
491           out.flush();
492         }
493       } catch(IOException e) {
494         markClosed(e);
495       } finally {
496         //the buffer is just an in-memory buffer, but it is still polite to
497         // close early
498         IOUtils.closeStream(d);
499       }
500     }
501 
502     /* Receive a response.
503      * Because only one receiver, so no synchronization on in.
504      */
505     private void receiveResponse() {
506       if (shouldCloseConnection.get()) {
507         return;
508       }
509       touch();
510 
511       try {
512         int id = in.readInt();                    // try to read an id
513 
514         if (LOG.isDebugEnabled())
515           LOG.debug(getName() + " got value #" + id);
516 
517         Call call = calls.get(id);
518 
519         boolean isError = in.readBoolean();     // read if error
520         if (isError) {
521           //noinspection ThrowableInstanceNeverThrown
522           call.setException(new RemoteException( WritableUtils.readString(in),
523               WritableUtils.readString(in)));
524           calls.remove(id);
525         } else {
526           Writable value = ReflectionUtils.newInstance(valueClass, conf);
527           value.readFields(in);                 // read value
528           call.setValue(value);
529           calls.remove(id);
530         }
531       } catch (IOException e) {
532         markClosed(e);
533       }
534     }
535 
536     private synchronized void markClosed(IOException e) {
537       if (shouldCloseConnection.compareAndSet(false, true)) {
538         closeException = e;
539         notifyAll();
540       }
541     }
542 
543     /** Close the connection. */
544     private synchronized void close() {
545       if (!shouldCloseConnection.get()) {
546         LOG.error("The connection is not in the closed state");
547         return;
548       }
549 
550       // release the resources
551       // first thing to do;take the connection out of the connection list
552       synchronized (connections) {
553         if (connections.get(remoteId) == this) {
554           connections.remove(remoteId);
555         }
556       }
557 
558       // close the streams and therefore the socket
559       IOUtils.closeStream(out);
560       IOUtils.closeStream(in);
561 
562       // clean up all calls
563       if (closeException == null) {
564         if (!calls.isEmpty()) {
565           LOG.warn(
566               "A connection is closed for no cause and calls are not empty");
567 
568           // clean up calls anyway
569           closeException = new IOException("Unexpected closed connection");
570           cleanupCalls();
571         }
572       } else {
573         // log the info
574         if (LOG.isDebugEnabled()) {
575           LOG.debug("closing ipc connection to " + remoteId.address + ": " +
576               closeException.getMessage(),closeException);
577         }
578 
579         // cleanup calls
580         cleanupCalls();
581       }
582       if (LOG.isDebugEnabled())
583         LOG.debug(getName() + ": closed");
584     }
585 
586     /* Cleanup all calls and mark them as done */
587     private void cleanupCalls() {
588       Iterator<Entry<Integer, Call>> itor = calls.entrySet().iterator() ;
589       while (itor.hasNext()) {
590         Call c = itor.next().getValue();
591         c.setException(closeException); // local exception
592         itor.remove();
593       }
594     }
595   }
596 
597   /** Call implementation used for parallel calls. */
598   private class ParallelCall extends Call {
599     private final ParallelResults results;
600     protected final int index;
601 
602     public ParallelCall(Writable param, ParallelResults results, int index) {
603       super(param);
604       this.results = results;
605       this.index = index;
606     }
607 
608     /** Deliver result to result collector. */
609     @Override
610     protected void callComplete() {
611       results.callComplete(this);
612     }
613   }
614 
615   /** Result collector for parallel calls. */
616   private static class ParallelResults {
617     protected final Writable[] values;
618     protected int size;
619     protected int count;
620 
621     public ParallelResults(int size) {
622       this.values = new Writable[size];
623       this.size = size;
624     }
625 
626     /*
627      * Collect a result.
628      */
629     synchronized void callComplete(ParallelCall call) {
630       // FindBugs IS2_INCONSISTENT_SYNC
631       values[call.index] = call.value;            // store the value
632       count++;                                    // count it
633       if (count == size)                          // if all values are in
634         notify();                                 // then notify waiting caller
635     }
636   }
637 
638   /**
639    * Construct an IPC client whose values are of the given {@link Writable}
640    * class.
641    * @param valueClass value class
642    * @param conf configuration
643    * @param factory socket factory
644    */
645   public HBaseClient(Class<? extends Writable> valueClass, Configuration conf,
646       SocketFactory factory) {
647     this.valueClass = valueClass;
648     this.maxIdleTime =
649       conf.getInt("hbase.ipc.client.connection.maxidletime", 10000); //10s
650     this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries", 0);
651     this.failureSleep = conf.getInt("hbase.client.pause", 2000);
652     this.tcpNoDelay = conf.getBoolean("hbase.ipc.client.tcpnodelay", false);
653     this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true);
654     this.pingInterval = getPingInterval(conf);
655     if (LOG.isDebugEnabled()) {
656       LOG.debug("The ping interval is" + this.pingInterval + "ms.");
657     }
658     this.conf = conf;
659     this.socketFactory = factory;
660   }
661 
662   /**
663    * Construct an IPC client with the default SocketFactory
664    * @param valueClass value class
665    * @param conf configuration
666    */
667   public HBaseClient(Class<? extends Writable> valueClass, Configuration conf) {
668     this(valueClass, conf, NetUtils.getDefaultSocketFactory(conf));
669   }
670 
671   /** Return the socket factory of this client
672    *
673    * @return this client's socket factory
674    */
675   SocketFactory getSocketFactory() {
676     return socketFactory;
677   }
678 
679   /** Stop all threads related to this client.  No further calls may be made
680    * using this client. */
681   public void stop() {
682     if (LOG.isDebugEnabled()) {
683       LOG.debug("Stopping client");
684     }
685 
686     if (!running.compareAndSet(true, false)) {
687       return;
688     }
689 
690     // wake up all connections
691     synchronized (connections) {
692       for (Connection conn : connections.values()) {
693         conn.interrupt();
694       }
695     }
696 
697     // wait until all connections are closed
698     while (!connections.isEmpty()) {
699       try {
700         Thread.sleep(100);
701       } catch (InterruptedException ignored) {
702       }
703     }
704   }
705 
706   /** Make a call, passing <code>param</code>, to the IPC server running at
707    * <code>address</code>, returning the value.  Throws exceptions if there are
708    * network problems or if the remote code threw an exception.
709    * @param param writable parameter
710    * @param address network address
711    * @return Writable
712    * @throws IOException e
713    */
714   public Writable call(Writable param, InetSocketAddress address)
715   throws IOException {
716       return call(param, address, null);
717   }
718 
719   public Writable call(Writable param, InetSocketAddress addr,
720                        UserGroupInformation ticket)
721                        throws IOException {
722     Call call = new Call(param);
723     Connection connection = getConnection(addr, ticket, call);
724     connection.sendParam(call);                 // send the parameter
725     boolean interrupted = false;
726     //noinspection SynchronizationOnLocalVariableOrMethodParameter
727     synchronized (call) {
728       while (!call.done) {
729         try {
730           call.wait();                           // wait for the result
731         } catch (InterruptedException ignored) {
732           // save the fact that we were interrupted
733           interrupted = true;
734         }
735       }
736 
737       if (interrupted) {
738         // set the interrupt flag now that we are done waiting
739         Thread.currentThread().interrupt();
740       }
741 
742       if (call.error != null) {
743         if (call.error instanceof RemoteException) {
744           call.error.fillInStackTrace();
745           throw call.error;
746         }
747         // local exception
748         throw wrapException(addr, call.error);
749       }
750       return call.value;
751     }
752   }
753 
754   /**
755    * Take an IOException and the address we were trying to connect to
756    * and return an IOException with the input exception as the cause.
757    * The new exception provides the stack trace of the place where
758    * the exception is thrown and some extra diagnostics information.
759    * If the exception is ConnectException or SocketTimeoutException,
760    * return a new one of the same type; Otherwise return an IOException.
761    *
762    * @param addr target address
763    * @param exception the relevant exception
764    * @return an exception to throw
765    */
766   @SuppressWarnings({"ThrowableInstanceNeverThrown"})
767   private IOException wrapException(InetSocketAddress addr,
768                                          IOException exception) {
769     if (exception instanceof ConnectException) {
770       //connection refused; include the host:port in the error
771       return (ConnectException)new ConnectException(
772            "Call to " + addr + " failed on connection exception: " + exception)
773                     .initCause(exception);
774     } else if (exception instanceof SocketTimeoutException) {
775       return (SocketTimeoutException)new SocketTimeoutException(
776            "Call to " + addr + " failed on socket timeout exception: "
777                       + exception).initCause(exception);
778     } else {
779       return (IOException)new IOException(
780            "Call to " + addr + " failed on local exception: " + exception)
781                                  .initCause(exception);
782 
783     }
784   }
785 
786   /** Makes a set of calls in parallel.  Each parameter is sent to the
787    * corresponding address.  When all values are available, or have timed out
788    * or errored, the collected results are returned in an array.  The array
789    * contains nulls for calls that timed out or errored.
790    * @param params writable parameters
791    * @param addresses socket addresses
792    * @return  Writable[]
793    * @throws IOException e
794    */
795   public Writable[] call(Writable[] params, InetSocketAddress[] addresses)
796     throws IOException {
797     if (addresses.length == 0) return new Writable[0];
798 
799     ParallelResults results = new ParallelResults(params.length);
800     // TODO this synchronization block doesnt make any sense, we should possibly fix it
801     //noinspection SynchronizationOnLocalVariableOrMethodParameter
802     synchronized (results) {
803       for (int i = 0; i < params.length; i++) {
804         ParallelCall call = new ParallelCall(params[i], results, i);
805         try {
806           Connection connection = getConnection(addresses[i], null, call);
807           connection.sendParam(call);             // send each parameter
808         } catch (IOException e) {
809           // log errors
810           LOG.info("Calling "+addresses[i]+" caught: " +
811                    e.getMessage(),e);
812           results.size--;                         //  wait for one fewer result
813         }
814       }
815       while (results.count != results.size) {
816         try {
817           results.wait();                    // wait for all results
818         } catch (InterruptedException ignored) {}
819       }
820 
821       return results.values;
822     }
823   }
824 
825   /* Get a connection from the pool, or create a new one and add it to the
826    * pool.  Connections to a given host/port are reused. */
827   private Connection getConnection(InetSocketAddress addr,
828                                    UserGroupInformation ticket,
829                                    Call call)
830                                    throws IOException {
831     if (!running.get()) {
832       // the client is stopped
833       throw new IOException("The client is stopped");
834     }
835     Connection connection;
836     /* we could avoid this allocation for each RPC by having a
837      * connectionsId object and with set() method. We need to manage the
838      * refs for keys in HashMap properly. For now its ok.
839      */
840     ConnectionId remoteId = new ConnectionId(addr, ticket);
841     do {
842       synchronized (connections) {
843         connection = connections.get(remoteId);
844         if (connection == null) {
845           connection = new Connection(remoteId);
846           connections.put(remoteId, connection);
847         }
848       }
849     } while (!connection.addCall(call));
850 
851     //we don't invoke the method below inside "synchronized (connections)"
852     //block above. The reason for that is if the server happens to be slow,
853     //it will take longer to establish a connection and that will slow the
854     //entire system down.
855     connection.setupIOstreams();
856     return connection;
857   }
858 
859   /**
860    * This class holds the address and the user ticket. The client connections
861    * to servers are uniquely identified by <remoteAddress, ticket>
862    */
863   private static class ConnectionId {
864     final InetSocketAddress address;
865     final UserGroupInformation ticket;
866 
867     ConnectionId(InetSocketAddress address, UserGroupInformation ticket) {
868       this.address = address;
869       this.ticket = ticket;
870     }
871 
872     InetSocketAddress getAddress() {
873       return address;
874     }
875     UserGroupInformation getTicket() {
876       return ticket;
877     }
878 
879     @Override
880     public boolean equals(Object obj) {
881      if (obj instanceof ConnectionId) {
882        ConnectionId id = (ConnectionId) obj;
883        return address.equals(id.address) && ticket == id.ticket;
884        //Note : ticket is a ref comparision.
885      }
886      return false;
887     }
888 
889     @Override
890     public int hashCode() {
891       return address.hashCode() ^ System.identityHashCode(ticket);
892     }
893   }
894 }