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  package org.apache.hadoop.hbase.executor;
21  
22  import static org.junit.Assert.assertEquals;
23  
24  import java.io.IOException;
25  import java.util.concurrent.ThreadPoolExecutor;
26  import java.util.concurrent.atomic.AtomicBoolean;
27  import java.util.concurrent.atomic.AtomicInteger;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.hbase.executor.EventHandler.EventType;
32  import org.apache.hadoop.hbase.executor.ExecutorService.Executor;
33  import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
34  import org.junit.Test;
35  
36  public class TestExecutorService {
37    private static final Log LOG = LogFactory.getLog(TestExecutorService.class);
38  
39    @Test
40    public void testExecutorService() throws Exception {
41      int maxThreads = 5;
42      int maxTries = 10;
43      int sleepInterval = 10;
44  
45      // Start an executor service pool with max 5 threads
46      ExecutorService executorService = new ExecutorService("unit_test");
47      executorService.startExecutorService(
48        ExecutorType.MASTER_SERVER_OPERATIONS, maxThreads);
49  
50      Executor executor =
51        executorService.getExecutor(ExecutorType.MASTER_SERVER_OPERATIONS);
52      ThreadPoolExecutor pool = executor.threadPoolExecutor;
53  
54      // Assert no threads yet
55      assertEquals(0, pool.getPoolSize());
56  
57      AtomicBoolean lock = new AtomicBoolean(true);
58      AtomicInteger counter = new AtomicInteger(0);
59  
60      // Submit maxThreads executors.
61      for (int i = 0; i < maxThreads; i++) {
62        executorService.submit(
63          new TestEventHandler(EventType.M_SERVER_SHUTDOWN, lock, counter));
64      }
65  
66      // The TestEventHandler will increment counter when it starts.
67      int tries = 0;
68      while (counter.get() < maxThreads && tries < maxTries) {
69        LOG.info("Waiting for all event handlers to start...");
70        Thread.sleep(sleepInterval);
71        tries++;
72      }
73  
74      // Assert that pool is at max threads.
75      assertEquals(maxThreads, counter.get());
76      assertEquals(maxThreads, pool.getPoolSize());
77  
78      // Now interrupt the running Executor
79      synchronized (lock) {
80        lock.set(false);
81        lock.notifyAll();
82      }
83  
84      // Executor increments counter again on way out so.... test that happened.
85      while (counter.get() < (maxThreads * 2) && tries < maxTries) {
86        System.out.println("Waiting for all event handlers to finish...");
87        Thread.sleep(sleepInterval);
88        tries++;
89      }
90  
91      assertEquals(maxThreads * 2, counter.get());
92      assertEquals(maxThreads, pool.getPoolSize());
93  
94      // Add more than the number of threads items.
95      // Make sure we don't get RejectedExecutionException.
96      for (int i = 0; i < (2 * maxThreads); i++) {
97        executorService.submit(
98          new TestEventHandler(EventType.M_SERVER_SHUTDOWN, lock, counter));
99      }
100     // Now interrupt the running Executor
101     synchronized (lock) {
102       lock.set(false);
103       lock.notifyAll();
104     }
105 
106     // Make sure threads are still around even after their timetolive expires.
107     Thread.sleep(executor.keepAliveTimeInMillis * 2);
108     assertEquals(maxThreads, pool.getPoolSize());
109   }
110 
111   public static class TestEventHandler extends EventHandler {
112     private AtomicBoolean lock;
113     private AtomicInteger counter;
114 
115     public TestEventHandler(EventType eventType, AtomicBoolean lock,
116         AtomicInteger counter) {
117       super(null, eventType);
118       this.lock = lock;
119       this.counter = counter;
120     }
121 
122     @Override
123     public void process() throws IOException {
124       int num = counter.incrementAndGet();
125       LOG.info("Running process #" + num + ", threadName=" +
126         Thread.currentThread().getName());
127       synchronized (lock) {
128         while (lock.get()) {
129           try {
130             lock.wait();
131           } catch (InterruptedException e) {
132             // do nothing
133           }
134         }
135       }
136       counter.incrementAndGet();
137     }
138   }
139 }