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      /**
068       * Add the passed {@link Service} to the list of services managed by this
069       * {@link CompositeService}
070       * @param service the {@link Service} to be added
071       */
072      protected void addService(Service service) {
073        if (LOG.isDebugEnabled()) {
074          LOG.debug("Adding service " + service.getName());
075        }
076        synchronized (serviceList) {
077          serviceList.add(service);
078        }
079      }
080    
081      /**
082       * If the passed object is an instance of {@link Service},
083       * add it to the list of services managed by this {@link CompositeService}
084       * @param object
085       * @return true if a service is added, false otherwise.
086       */
087      protected boolean addIfService(Object object) {
088        if (object instanceof Service) {
089          addService((Service) object);
090          return true;
091        } else {
092          return false;
093        }
094      }
095    
096      protected synchronized boolean removeService(Service service) {
097        synchronized (serviceList) {
098          return serviceList.remove(service);
099        }
100      }
101    
102      protected void serviceInit(Configuration conf) throws Exception {
103        List<Service> services = getServices();
104        if (LOG.isDebugEnabled()) {
105          LOG.debug(getName() + ": initing services, size=" + services.size());
106        }
107        for (Service service : services) {
108          service.init(conf);
109        }
110        super.serviceInit(conf);
111      }
112    
113      protected void serviceStart() throws Exception {
114        List<Service> services = getServices();
115        if (LOG.isDebugEnabled()) {
116          LOG.debug(getName() + ": starting services, size=" + services.size());
117        }
118        for (Service service : services) {
119          // start the service. If this fails that service
120          // will be stopped and an exception raised
121          service.start();
122        }
123        super.serviceStart();
124      }
125    
126      protected void serviceStop() throws Exception {
127        //stop all services that were started
128        int numOfServicesToStop = serviceList.size();
129        if (LOG.isDebugEnabled()) {
130          LOG.debug(getName() + ": stopping services, size=" + numOfServicesToStop);
131        }
132        stop(numOfServicesToStop, STOP_ONLY_STARTED_SERVICES);
133        super.serviceStop();
134      }
135    
136      /**
137       * Stop the services in reverse order
138       *
139       * @param numOfServicesStarted index from where the stop should work
140       * @param stopOnlyStartedServices flag to say "only start services that are
141       * started, not those that are NOTINITED or INITED.
142       * @throws RuntimeException the first exception raised during the
143       * stop process -<i>after all services are stopped</i>
144       */
145      private synchronized void stop(int numOfServicesStarted,
146                                     boolean stopOnlyStartedServices) {
147        // stop in reverse order of start
148        Exception firstException = null;
149        List<Service> services = getServices();
150        for (int i = numOfServicesStarted - 1; i >= 0; i--) {
151          Service service = services.get(i);
152          if (LOG.isDebugEnabled()) {
153            LOG.debug("Stopping service #" + i + ": " + service);
154          }
155          STATE state = service.getServiceState();
156          //depending on the stop police
157          if (state == STATE.STARTED 
158             || (!stopOnlyStartedServices && state == STATE.INITED)) {
159            Exception ex = ServiceOperations.stopQuietly(LOG, service);
160            if (ex != null && firstException == null) {
161              firstException = ex;
162            }
163          }
164        }
165        //after stopping all services, rethrow the first exception raised
166        if (firstException != null) {
167          throw ServiceStateException.convert(firstException);
168        }
169      }
170    
171      /**
172       * JVM Shutdown hook for CompositeService which will stop the give
173       * CompositeService gracefully in case of JVM shutdown.
174       */
175      public static class CompositeServiceShutdownHook implements Runnable {
176    
177        private CompositeService compositeService;
178    
179        public CompositeServiceShutdownHook(CompositeService compositeService) {
180          this.compositeService = compositeService;
181        }
182    
183        @Override
184        public void run() {
185          ServiceOperations.stopQuietly(compositeService);
186        }
187      }
188    
189    }