001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.net.unix;
019
020import java.io.Closeable;
021import java.io.EOFException;
022
023import org.apache.hadoop.classification.InterfaceAudience;
024import org.apache.hadoop.io.IOUtils;
025
026import java.io.IOException;
027import java.nio.channels.ClosedChannelException;
028import java.util.Iterator;
029import java.util.LinkedList;
030import java.util.TreeMap;
031import java.util.Map;
032import java.util.concurrent.locks.Condition;
033import java.util.concurrent.locks.ReentrantLock;
034
035import org.apache.commons.lang.SystemUtils;
036import org.apache.commons.logging.Log;
037import org.apache.commons.logging.LogFactory;
038import org.apache.hadoop.util.NativeCodeLoader;
039
040import com.google.common.annotations.VisibleForTesting;
041import com.google.common.base.Preconditions;
042import com.google.common.util.concurrent.Uninterruptibles;
043
044/**
045 * The DomainSocketWatcher watches a set of domain sockets to see when they
046 * become readable, or closed.  When one of those events happens, it makes a
047 * callback.
048 *
049 * See {@link DomainSocket} for more information about UNIX domain sockets.
050 */
051@InterfaceAudience.LimitedPrivate("HDFS")
052public final class DomainSocketWatcher implements Closeable {
053  static {
054    if (SystemUtils.IS_OS_WINDOWS) {
055      loadingFailureReason = "UNIX Domain sockets are not available on Windows.";
056    } else if (!NativeCodeLoader.isNativeCodeLoaded()) {
057      loadingFailureReason = "libhadoop cannot be loaded.";
058    } else {
059      String problem;
060      try {
061        anchorNative();
062        problem = null;
063      } catch (Throwable t) {
064        problem = "DomainSocketWatcher#anchorNative got error: " +
065          t.getMessage();
066      }
067      loadingFailureReason = problem;
068    }
069  }
070
071  static Log LOG = LogFactory.getLog(DomainSocketWatcher.class);
072
073  /**
074   * The reason why DomainSocketWatcher is not available, or null if it is
075   * available.
076   */
077  private final static String loadingFailureReason;
078
079  /**
080   * Initializes the native library code.
081   */
082  private static native void anchorNative();
083
084  public static String getLoadingFailureReason() {
085    return loadingFailureReason;
086  }
087
088  public interface Handler {
089    /**
090     * Handles an event on a socket.  An event may be the socket becoming
091     * readable, or the remote end being closed.
092     *
093     * @param sock    The socket that the event occurred on.
094     * @return        Whether we should close the socket.
095     */
096    boolean handle(DomainSocket sock);
097  }
098
099  /**
100   * Handler for {DomainSocketWatcher#notificationSockets[1]}
101   */
102  private class NotificationHandler implements Handler {
103    public boolean handle(DomainSocket sock) {
104      assert(lock.isHeldByCurrentThread());
105      try {
106        kicked = false;
107        if (LOG.isTraceEnabled()) {
108          LOG.trace(this + ": NotificationHandler: doing a read on " +
109            sock.fd);
110        }
111        if (sock.getInputStream().read() == -1) {
112          if (LOG.isTraceEnabled()) {
113            LOG.trace(this + ": NotificationHandler: got EOF on " + sock.fd);
114          }
115          throw new EOFException();
116        }
117        if (LOG.isTraceEnabled()) {
118          LOG.trace(this + ": NotificationHandler: read succeeded on " +
119            sock.fd);
120        }
121        return false;
122      } catch (IOException e) {
123        if (LOG.isTraceEnabled()) {
124          LOG.trace(this + ": NotificationHandler: setting closed to " +
125              "true for " + sock.fd);
126        }
127        closed = true;
128        return true;
129      }
130    }
131  }
132
133  private static class Entry {
134    final DomainSocket socket;
135    final Handler handler;
136
137    Entry(DomainSocket socket, Handler handler) {
138      this.socket = socket;
139      this.handler = handler;
140    }
141
142    DomainSocket getDomainSocket() {
143      return socket;
144    }
145
146    Handler getHandler() {
147      return handler;
148    }
149  }
150
151  /**
152   * The FdSet is a set of file descriptors that gets passed to poll(2).
153   * It contains a native memory segment, so that we don't have to copy
154   * in the poll0 function.
155   */
156  private static class FdSet {
157    private long data;
158
159    private native static long alloc0();
160
161    FdSet() {
162      data = alloc0();
163    }
164
165    /**
166     * Add a file descriptor to the set.
167     *
168     * @param fd   The file descriptor to add.
169     */
170    native void add(int fd);
171
172    /**
173     * Remove a file descriptor from the set.
174     *
175     * @param fd   The file descriptor to remove.
176     */
177    native void remove(int fd);
178
179    /**
180     * Get an array containing all the FDs marked as readable.
181     * Also clear the state of all FDs.
182     *
183     * @return     An array containing all of the currently readable file
184     *             descriptors.
185     */
186    native int[] getAndClearReadableFds();
187
188    /**
189     * Close the object and de-allocate the memory used.
190     */
191    native void close();
192  }
193
194  /**
195   * Lock which protects toAdd, toRemove, and closed.
196   */
197  private final ReentrantLock lock = new ReentrantLock();
198
199  /**
200   * Condition variable which indicates that toAdd and toRemove have been
201   * processed.
202   */
203  private final Condition processedCond = lock.newCondition();
204
205  /**
206   * Entries to add.
207   */
208  private final LinkedList<Entry> toAdd =
209      new LinkedList<Entry>();
210
211  /**
212   * Entries to remove.
213   */
214  private final TreeMap<Integer, DomainSocket> toRemove =
215      new TreeMap<Integer, DomainSocket>();
216
217  /**
218   * Maximum length of time to go between checking whether the interrupted
219   * bit has been set for this thread.
220   */
221  private final int interruptCheckPeriodMs;
222
223  /**
224   * A pair of sockets used to wake up the thread after it has called poll(2).
225   */
226  private final DomainSocket notificationSockets[];
227
228  /**
229   * Whether or not this DomainSocketWatcher is closed.
230   */
231  private boolean closed = false;
232  
233  /**
234   * True if we have written a byte to the notification socket. We should not
235   * write anything else to the socket until the notification handler has had a
236   * chance to run. Otherwise, our thread might block, causing deadlock. 
237   * See HADOOP-11333 for details.
238   */
239  private boolean kicked = false;
240
241  public DomainSocketWatcher(int interruptCheckPeriodMs) throws IOException {
242    if (loadingFailureReason != null) {
243      throw new UnsupportedOperationException(loadingFailureReason);
244    }
245    Preconditions.checkArgument(interruptCheckPeriodMs > 0);
246    this.interruptCheckPeriodMs = interruptCheckPeriodMs;
247    notificationSockets = DomainSocket.socketpair();
248    watcherThread.setDaemon(true);
249    watcherThread.setUncaughtExceptionHandler(
250        new Thread.UncaughtExceptionHandler() {
251          @Override
252          public void uncaughtException(Thread thread, Throwable t) {
253            LOG.error(thread + " terminating on unexpected exception", t);
254          }
255        });
256    watcherThread.start();
257  }
258
259  /**
260   * Close the DomainSocketWatcher and wait for its thread to terminate.
261   *
262   * If there is more than one close, all but the first will be ignored.
263   */
264  @Override
265  public void close() throws IOException {
266    lock.lock();
267    try {
268      if (closed) return;
269      if (LOG.isDebugEnabled()) {
270        LOG.debug(this + ": closing");
271      }
272      closed = true;
273    } finally {
274      lock.unlock();
275    }
276    // Close notificationSockets[0], so that notificationSockets[1] gets an EOF
277    // event.  This will wake up the thread immediately if it is blocked inside
278    // the select() system call.
279    notificationSockets[0].close();
280    // Wait for the select thread to terminate.
281    Uninterruptibles.joinUninterruptibly(watcherThread);
282  }
283
284  @VisibleForTesting
285  public boolean isClosed() {
286    lock.lock();
287    try {
288      return closed;
289    } finally {
290      lock.unlock();
291    }
292  }
293
294  /**
295   * Add a socket.
296   *
297   * @param sock     The socket to add.  It is an error to re-add a socket that
298   *                   we are already watching.
299   * @param handler  The handler to associate with this socket.  This may be
300   *                   called any time after this function is called.
301   */
302  public void add(DomainSocket sock, Handler handler) {
303    lock.lock();
304    try {
305      if (closed) {
306        handler.handle(sock);
307        IOUtils.cleanup(LOG, sock);
308        return;
309      }
310      Entry entry = new Entry(sock, handler);
311      try {
312        sock.refCount.reference();
313      } catch (ClosedChannelException e1) {
314        // If the socket is already closed before we add it, invoke the
315        // handler immediately.  Then we're done.
316        handler.handle(sock);
317        return;
318      }
319      toAdd.add(entry);
320      kick();
321      while (true) {
322        try {
323          processedCond.await();
324        } catch (InterruptedException e) {
325          Thread.currentThread().interrupt();
326        }
327        if (!toAdd.contains(entry)) {
328          break;
329        }
330      }
331    } finally {
332      lock.unlock();
333    }
334  }
335
336  /**
337   * Remove a socket.  Its handler will be called.
338   *
339   * @param sock     The socket to remove.
340   */
341  public void remove(DomainSocket sock) {
342    lock.lock();
343    try {
344      if (closed) return;
345      toRemove.put(sock.fd, sock);
346      kick();
347      while (true) {
348        try {
349          processedCond.await();
350        } catch (InterruptedException e) {
351          Thread.currentThread().interrupt();
352        }
353        if (!toRemove.containsKey(sock.fd)) {
354          break;
355        }
356      }
357    } finally {
358      lock.unlock();
359    }
360  }
361
362  /**
363   * Wake up the DomainSocketWatcher thread.
364   */
365  private void kick() {
366    assert(lock.isHeldByCurrentThread());
367    
368    if (kicked) {
369      return;
370    }
371    
372    try {
373      notificationSockets[0].getOutputStream().write(0);
374      kicked = true;
375    } catch (IOException e) {
376      if (!closed) {
377        LOG.error(this + ": error writing to notificationSockets[0]", e);
378      }
379    }
380  }
381
382  /**
383   * Send callback and return whether or not the domain socket was closed as a
384   * result of processing.
385   *
386   * @param caller reason for call
387   * @param entries mapping of file descriptor to entry
388   * @param fdSet set of file descriptors
389   * @param fd file descriptor
390   * @return true if the domain socket was closed as a result of processing
391   */
392  private boolean sendCallback(String caller, TreeMap<Integer, Entry> entries,
393      FdSet fdSet, int fd) {
394    if (LOG.isTraceEnabled()) {
395      LOG.trace(this + ": " + caller + " starting sendCallback for fd " + fd);
396    }
397    Entry entry = entries.get(fd);
398    Preconditions.checkNotNull(entry,
399        this + ": fdSet contained " + fd + ", which we were " +
400        "not tracking.");
401    DomainSocket sock = entry.getDomainSocket();
402    if (entry.getHandler().handle(sock)) {
403      if (LOG.isTraceEnabled()) {
404        LOG.trace(this + ": " + caller + ": closing fd " + fd +
405            " at the request of the handler.");
406      }
407      if (toRemove.remove(fd) != null) {
408        if (LOG.isTraceEnabled()) {
409          LOG.trace(this + ": " + caller + " : sendCallback processed fd " +
410            fd  + " in toRemove.");
411        }
412      }
413      try {
414        sock.refCount.unreferenceCheckClosed();
415      } catch (IOException e) {
416        Preconditions.checkArgument(false,
417            this + ": file descriptor " + sock.fd + " was closed while " +
418            "still in the poll(2) loop.");
419      }
420      IOUtils.cleanup(LOG, sock);
421      fdSet.remove(fd);
422      return true;
423    } else {
424      if (LOG.isTraceEnabled()) {
425        LOG.trace(this + ": " + caller + ": sendCallback not " +
426            "closing fd " + fd);
427      }
428      return false;
429    }
430  }
431
432  /**
433   * Send callback, and if the domain socket was closed as a result of
434   * processing, then also remove the entry for the file descriptor.
435   *
436   * @param caller reason for call
437   * @param entries mapping of file descriptor to entry
438   * @param fdSet set of file descriptors
439   * @param fd file descriptor
440   */
441  private void sendCallbackAndRemove(String caller,
442      TreeMap<Integer, Entry> entries, FdSet fdSet, int fd) {
443    if (sendCallback(caller, entries, fdSet, fd)) {
444      entries.remove(fd);
445    }
446  }
447
448  @VisibleForTesting
449  final Thread watcherThread = new Thread(new Runnable() {
450    @Override
451    public void run() {
452      if (LOG.isDebugEnabled()) {
453        LOG.debug(this + ": starting with interruptCheckPeriodMs = " +
454            interruptCheckPeriodMs);
455      }
456      final TreeMap<Integer, Entry> entries = new TreeMap<Integer, Entry>();
457      FdSet fdSet = new FdSet();
458      addNotificationSocket(entries, fdSet);
459      try {
460        while (true) {
461          lock.lock();
462          try {
463            for (int fd : fdSet.getAndClearReadableFds()) {
464              sendCallbackAndRemove("getAndClearReadableFds", entries, fdSet,
465                  fd);
466            }
467            if (!(toAdd.isEmpty() && toRemove.isEmpty())) {
468              // Handle pending additions (before pending removes).
469              for (Iterator<Entry> iter = toAdd.iterator(); iter.hasNext(); ) {
470                Entry entry = iter.next();
471                DomainSocket sock = entry.getDomainSocket();
472                Entry prevEntry = entries.put(sock.fd, entry);
473                Preconditions.checkState(prevEntry == null,
474                    this + ": tried to watch a file descriptor that we " +
475                    "were already watching: " + sock);
476                if (LOG.isTraceEnabled()) {
477                  LOG.trace(this + ": adding fd " + sock.fd);
478                }
479                fdSet.add(sock.fd);
480                iter.remove();
481              }
482              // Handle pending removals
483              while (true) {
484                Map.Entry<Integer, DomainSocket> entry = toRemove.firstEntry();
485                if (entry == null) break;
486                sendCallbackAndRemove("handlePendingRemovals",
487                    entries, fdSet, entry.getValue().fd);
488              }
489              processedCond.signalAll();
490            }
491            // Check if the thread should terminate.  Doing this check now is
492            // easier than at the beginning of the loop, since we know toAdd and
493            // toRemove are now empty and processedCond has been notified if it
494            // needed to be.
495            if (closed) {
496              if (LOG.isDebugEnabled()) {
497                LOG.debug(toString() + " thread terminating.");
498              }
499              return;
500            }
501            // Check if someone sent our thread an InterruptedException while we
502            // were waiting in poll().
503            if (Thread.interrupted()) {
504              throw new InterruptedException();
505            }
506          } finally {
507            lock.unlock();
508          }
509          doPoll0(interruptCheckPeriodMs, fdSet);
510        }
511      } catch (InterruptedException e) {
512        LOG.info(toString() + " terminating on InterruptedException");
513      } catch (Throwable e) {
514        LOG.error(toString() + " terminating on exception", e);
515      } finally {
516        lock.lock();
517        try {
518          kick(); // allow the handler for notificationSockets[0] to read a byte
519          for (Entry entry : entries.values()) {
520            // We do not remove from entries as we iterate, because that can
521            // cause a ConcurrentModificationException.
522            sendCallback("close", entries, fdSet, entry.getDomainSocket().fd);
523          }
524          entries.clear();
525          fdSet.close();
526        } finally {
527          lock.unlock();
528        }
529      }
530    }
531  });
532
533  private void addNotificationSocket(final TreeMap<Integer, Entry> entries,
534      FdSet fdSet) {
535    entries.put(notificationSockets[1].fd, 
536        new Entry(notificationSockets[1], new NotificationHandler()));
537    try {
538      notificationSockets[1].refCount.reference();
539    } catch (IOException e) {
540      throw new RuntimeException(e);
541    }
542    fdSet.add(notificationSockets[1].fd);
543    if (LOG.isTraceEnabled()) {
544      LOG.trace(this + ": adding notificationSocket " +
545          notificationSockets[1].fd + ", connected to " +
546          notificationSockets[0].fd);
547    }
548  }
549
550  public String toString() {
551    return "DomainSocketWatcher(" + System.identityHashCode(this) + ")"; 
552  }
553
554  private static native int doPoll0(int maxWaitMs, FdSet readFds)
555      throws IOException;
556}