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.*;
23  import static org.junit.Assert.assertEquals;
24  
25  import java.io.IOException;
26  import java.io.PrintWriter;
27  import java.io.StringWriter;
28  import java.util.concurrent.ThreadPoolExecutor;
29  import java.util.concurrent.atomic.AtomicBoolean;
30  import java.util.concurrent.atomic.AtomicInteger;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.hbase.*;
35  import org.apache.hadoop.hbase.executor.EventHandler.EventType;
36  import org.apache.hadoop.hbase.executor.ExecutorService.Executor;
37  import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorStatus;
38  import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
39  import org.junit.Test;
40  import org.junit.experimental.categories.Category;
41  
42  import static org.mockito.Mockito.*;
43  
44  @Category(SmallTests.class)
45  public class TestExecutorService {
46    private static final Log LOG = LogFactory.getLog(TestExecutorService.class);
47  
48    @Test
49    public void testExecutorService() throws Exception {
50      int maxThreads = 5;
51      int maxTries = 10;
52      int sleepInterval = 10;
53  
54      Server mockedServer = mock(Server.class);
55      when(mockedServer.getConfiguration()).thenReturn(HBaseConfiguration.create());
56  
57      // Start an executor service pool with max 5 threads
58      ExecutorService executorService = new ExecutorService("unit_test");
59      executorService.startExecutorService(
60        ExecutorType.MASTER_SERVER_OPERATIONS, maxThreads);
61  
62      Executor executor =
63        executorService.getExecutor(ExecutorType.MASTER_SERVER_OPERATIONS);
64      ThreadPoolExecutor pool = executor.threadPoolExecutor;
65  
66      // Assert no threads yet
67      assertEquals(0, pool.getPoolSize());
68  
69      AtomicBoolean lock = new AtomicBoolean(true);
70      AtomicInteger counter = new AtomicInteger(0);
71  
72      // Submit maxThreads executors.
73      for (int i = 0; i < maxThreads; i++) {
74        executorService.submit(
75          new TestEventHandler(mockedServer, EventType.M_SERVER_SHUTDOWN,
76              lock, counter));
77      }
78  
79      // The TestEventHandler will increment counter when it starts.
80      int tries = 0;
81      while (counter.get() < maxThreads && tries < maxTries) {
82        LOG.info("Waiting for all event handlers to start...");
83        Thread.sleep(sleepInterval);
84        tries++;
85      }
86  
87      // Assert that pool is at max threads.
88      assertEquals(maxThreads, counter.get());
89      assertEquals(maxThreads, pool.getPoolSize());
90  
91      ExecutorStatus status = executor.getStatus();
92      assertTrue(status.queuedEvents.isEmpty());
93      assertEquals(5, status.running.size());
94      checkStatusDump(status);
95      
96      
97      // Now interrupt the running Executor
98      synchronized (lock) {
99        lock.set(false);
100       lock.notifyAll();
101     }
102 
103     // Executor increments counter again on way out so.... test that happened.
104     while (counter.get() < (maxThreads * 2) && tries < maxTries) {
105       System.out.println("Waiting for all event handlers to finish...");
106       Thread.sleep(sleepInterval);
107       tries++;
108     }
109 
110     assertEquals(maxThreads * 2, counter.get());
111     assertEquals(maxThreads, pool.getPoolSize());
112 
113     // Add more than the number of threads items.
114     // Make sure we don't get RejectedExecutionException.
115     for (int i = 0; i < (2 * maxThreads); i++) {
116       executorService.submit(
117         new TestEventHandler(mockedServer, EventType.M_SERVER_SHUTDOWN,
118             lock, counter));
119     }
120     // Now interrupt the running Executor
121     synchronized (lock) {
122       lock.set(false);
123       lock.notifyAll();
124     }
125 
126     // Make sure threads are still around even after their timetolive expires.
127     Thread.sleep(executor.keepAliveTimeInMillis * 2);
128     assertEquals(maxThreads, pool.getPoolSize());
129 
130     executorService.shutdown();
131 
132     assertEquals(0, executorService.getAllExecutorStatuses().size());
133 
134     // Test that submit doesn't throw NPEs
135     executorService.submit(
136       new TestEventHandler(mockedServer, EventType.M_SERVER_SHUTDOWN,
137             lock, counter));
138   }
139 
140   private void checkStatusDump(ExecutorStatus status) throws IOException {
141     StringWriter sw = new StringWriter();
142     status.dumpTo(sw, "");
143     String dump = sw.toString();
144     LOG.info("Got status dump:\n" + dump);
145     
146     assertTrue(dump.contains("Waiting on java.util.concurrent.atomic.AtomicBoolean"));
147   }
148 
149   public static class TestEventHandler extends EventHandler {
150     private AtomicBoolean lock;
151     private AtomicInteger counter;
152 
153     public TestEventHandler(Server server, EventType eventType,
154                             AtomicBoolean lock, AtomicInteger counter) {
155       super(server, eventType);
156       this.lock = lock;
157       this.counter = counter;
158     }
159 
160     @Override
161     public void process() throws IOException {
162       int num = counter.incrementAndGet();
163       LOG.info("Running process #" + num + ", threadName=" +
164         Thread.currentThread().getName());
165       synchronized (lock) {
166         while (lock.get()) {
167           try {
168             lock.wait();
169           } catch (InterruptedException e) {
170             // do nothing
171           }
172         }
173       }
174       counter.incrementAndGet();
175     }
176   }
177 
178   @org.junit.Rule
179   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
180     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
181 }
182