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     */
018    
019    package org.apache.hadoop.service;
020    
021    import java.util.ArrayList;
022    import java.util.Collections;
023    import java.util.List;
024    
025    import org.apache.commons.logging.Log;
026    import org.apache.commons.logging.LogFactory;
027    import org.apache.hadoop.classification.InterfaceAudience.Public;
028    import org.apache.hadoop.classification.InterfaceStability.Evolving;
029    import org.apache.hadoop.conf.Configuration;
030    
031    /**
032     * Composition of services.
033     */
034    @Public
035    @Evolving
036    public class CompositeService extends AbstractService {
037    
038      private static final Log LOG = LogFactory.getLog(CompositeService.class);
039    
040      /**
041       * Policy on shutdown: attempt to close everything (purest) or
042       * only try to close started services (which assumes
043       * that the service implementations may not handle the stop() operation
044       * except when started.
045       * Irrespective of this policy, if a child service fails during
046       * its init() or start() operations, it will have stop() called on it.
047       */
048      protected static final boolean STOP_ONLY_STARTED_SERVICES = false;
049    
050      private final List<Service> serviceList = new ArrayList<Service>();
051    
052      public CompositeService(String name) {
053        super(name);
054      }
055    
056      /**
057       * Get an unmodifiable list of services
058       * @return a list of child services at the time of invocation -
059       * added services will not be picked up.
060       */
061      public List<Service> getServices() {
062        synchronized (serviceList) {
063          return Collections.unmodifiableList(serviceList);
064        }
065      }
066    
067      protected void addService(Service service) {
068        if (LOG.isDebugEnabled()) {
069          LOG.debug("Adding service " + service.getName());
070        }
071        synchronized (serviceList) {
072          serviceList.add(service);
073        }
074      }
075    
076      protected synchronized boolean removeService(Service service) {
077        synchronized (serviceList) {
078          return serviceList.add(service);
079        }
080      }
081    
082      protected void serviceInit(Configuration conf) throws Exception {
083        List<Service> services = getServices();
084        if (LOG.isDebugEnabled()) {
085          LOG.debug(getName() + ": initing services, size=" + services.size());
086        }
087        for (Service service : services) {
088          service.init(conf);
089        }
090        super.serviceInit(conf);
091      }
092    
093      protected void serviceStart() throws Exception {
094        List<Service> services = getServices();
095        if (LOG.isDebugEnabled()) {
096          LOG.debug(getName() + ": starting services, size=" + services.size());
097        }
098        for (Service service : services) {
099          // start the service. If this fails that service
100          // will be stopped and an exception raised
101          service.start();
102        }
103        super.serviceStart();
104      }
105    
106      protected void serviceStop() throws Exception {
107        //stop all services that were started
108        int numOfServicesToStop = serviceList.size();
109        if (LOG.isDebugEnabled()) {
110          LOG.debug(getName() + ": stopping services, size=" + numOfServicesToStop);
111        }
112        stop(numOfServicesToStop, STOP_ONLY_STARTED_SERVICES);
113        super.serviceStop();
114      }
115    
116      /**
117       * Stop the services in reverse order
118       *
119       * @param numOfServicesStarted index from where the stop should work
120       * @param stopOnlyStartedServices flag to say "only start services that are
121       * started, not those that are NOTINITED or INITED.
122       * @throws RuntimeException the first exception raised during the
123       * stop process -<i>after all services are stopped</i>
124       */
125      private synchronized void stop(int numOfServicesStarted,
126                                     boolean stopOnlyStartedServices) {
127        // stop in reverse order of start
128        Exception firstException = null;
129        List<Service> services = getServices();
130        for (int i = numOfServicesStarted - 1; i >= 0; i--) {
131          Service service = services.get(i);
132          if (LOG.isDebugEnabled()) {
133            LOG.debug("Stopping service #" + i + ": " + service);
134          }
135          STATE state = service.getServiceState();
136          //depending on the stop police
137          if (state == STATE.STARTED 
138             || (!stopOnlyStartedServices && state == STATE.INITED)) {
139            Exception ex = ServiceOperations.stopQuietly(LOG, service);
140            if (ex != null && firstException == null) {
141              firstException = ex;
142            }
143          }
144        }
145        //after stopping all services, rethrow the first exception raised
146        if (firstException != null) {
147          throw ServiceStateException.convert(firstException);
148        }
149      }
150    
151      /**
152       * JVM Shutdown hook for CompositeService which will stop the give
153       * CompositeService gracefully in case of JVM shutdown.
154       */
155      public static class CompositeServiceShutdownHook implements Runnable {
156    
157        private CompositeService compositeService;
158    
159        public CompositeServiceShutdownHook(CompositeService compositeService) {
160          this.compositeService = compositeService;
161        }
162    
163        @Override
164        public void run() {
165          ServiceOperations.stopQuietly(compositeService);
166        }
167      }
168    
169    }