View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.util;
20  
21  import static org.junit.Assert.assertEquals;
22  import static org.junit.Assert.assertNull;
23  import static org.junit.Assert.assertTrue;
24  
25  import java.util.concurrent.BlockingQueue;
26  import java.util.concurrent.CountDownLatch;
27  import java.util.concurrent.ThreadPoolExecutor;
28  import java.util.concurrent.TimeUnit;
29  import java.util.concurrent.atomic.AtomicInteger;
30  
31  import org.apache.hadoop.hbase.testclassification.SmallTests;
32  import org.junit.Before;
33  import org.junit.Test;
34  import org.junit.experimental.categories.Category;
35  
36  
37  @Category(SmallTests.class)
38  public class TestStealJobQueue {
39  
40    StealJobQueue<Integer> stealJobQueue;
41    BlockingQueue<Integer> stealFromQueue;
42  
43    @Before
44    public void setup() {
45      stealJobQueue = new StealJobQueue<>();
46      stealFromQueue = stealJobQueue.getStealFromQueue();
47  
48    }
49  
50  
51    @Test
52    public void testTake() throws InterruptedException {
53      stealJobQueue.offer(3);
54      stealFromQueue.offer(10);
55      stealJobQueue.offer(15);
56      stealJobQueue.offer(4);
57      assertEquals(3, stealJobQueue.take().intValue());
58      assertEquals(4, stealJobQueue.take().intValue());
59      assertEquals("always take from the main queue before trying to steal", 15,
60              stealJobQueue.take().intValue());
61      assertEquals(10, stealJobQueue.take().intValue());
62      assertTrue(stealFromQueue.isEmpty());
63      assertTrue(stealJobQueue.isEmpty());
64    }
65  
66    @Test
67    public void testOfferInStealQueueFromShouldUnblock() throws InterruptedException {
68      final AtomicInteger taken = new AtomicInteger();
69      Thread consumer = new Thread() {
70        @Override
71        public void run() {
72          try {
73            Integer n = stealJobQueue.take();
74            taken.set(n);
75          } catch (InterruptedException e) {
76            e.printStackTrace();
77          }
78        }
79      };
80      consumer.start();
81      stealFromQueue.offer(3);
82      consumer.join(1000);
83      assertEquals(3, taken.get());
84      consumer.interrupt(); //Ensure the consumer thread will stop.
85    }
86  
87  
88    @Test
89    public void testOfferInStealJobQueueShouldUnblock() throws InterruptedException {
90      final AtomicInteger taken = new AtomicInteger();
91      Thread consumer = new Thread() {
92        @Override
93        public void run() {
94          try {
95            Integer n = stealJobQueue.take();
96            taken.set(n);
97          } catch (InterruptedException e) {
98            e.printStackTrace();
99          }
100       }
101     };
102     consumer.start();
103     stealJobQueue.offer(3);
104     consumer.join(1000);
105     assertEquals(3, taken.get());
106     consumer.interrupt(); //Ensure the consumer thread will stop.
107   }
108 
109 
110   @Test
111   public void testPoll() throws InterruptedException {
112     stealJobQueue.offer(3);
113     stealFromQueue.offer(10);
114     stealJobQueue.offer(15);
115     stealJobQueue.offer(4);
116     assertEquals(3, stealJobQueue.poll(1, TimeUnit.SECONDS).intValue());
117     assertEquals(4, stealJobQueue.poll(1, TimeUnit.SECONDS).intValue());
118     assertEquals("always take from the main queue before trying to steal", 15,
119             stealJobQueue.poll(1, TimeUnit.SECONDS).intValue());
120     assertEquals(10, stealJobQueue.poll(1, TimeUnit.SECONDS).intValue());
121     assertTrue(stealFromQueue.isEmpty());
122     assertTrue(stealJobQueue.isEmpty());
123     assertNull(stealJobQueue.poll(10, TimeUnit.MILLISECONDS));
124   }
125 
126   @Test
127   public void testPutInStealQueueFromShouldUnblockPoll() throws InterruptedException {
128     final AtomicInteger taken = new AtomicInteger();
129     Thread consumer = new Thread() {
130       @Override
131       public void run() {
132         try {
133           Integer n = stealJobQueue.poll(3, TimeUnit.SECONDS);
134           taken.set(n);
135         } catch (InterruptedException e) {
136           e.printStackTrace();
137         }
138       }
139     };
140     consumer.start();
141     stealFromQueue.put(3);
142     consumer.join(1000);
143     assertEquals(3, taken.get());
144     consumer.interrupt(); //Ensure the consumer thread will stop.
145 
146   }
147 
148 
149   @Test
150   public void testAddInStealJobQueueShouldUnblockPoll() throws InterruptedException {
151     final AtomicInteger taken = new AtomicInteger();
152     Thread consumer = new Thread() {
153       @Override
154       public void run() {
155         try {
156           Integer n = stealJobQueue.poll(3, TimeUnit.SECONDS);
157           taken.set(n);
158         } catch (InterruptedException e) {
159           e.printStackTrace();
160         }
161       }
162     };
163     consumer.start();
164     stealJobQueue.add(3);
165     consumer.join(1000);
166     assertEquals(3, taken.get());
167     consumer.interrupt(); //Ensure the consumer thread will stop.
168   }
169 
170 
171   @Test
172   public void testInteractWithThreadPool() throws InterruptedException {
173     StealJobQueue<Runnable> stealTasksQueue = new StealJobQueue<>();
174     final CountDownLatch stealJobCountDown = new CountDownLatch(3);
175     final CountDownLatch stealFromCountDown = new CountDownLatch(3);
176     ThreadPoolExecutor stealPool = new ThreadPoolExecutor(3, 3, 1, TimeUnit.DAYS, stealTasksQueue) {
177       @Override
178       protected void afterExecute(Runnable r, Throwable t) {
179         super.afterExecute(r, t);
180         stealJobCountDown.countDown();
181       }
182 
183     };
184 
185     //This is necessary otherwise no worker will be running and stealing job
186     stealPool.prestartAllCoreThreads();
187 
188     ThreadPoolExecutor stealFromPool = new ThreadPoolExecutor(3, 3, 1, TimeUnit.DAYS,
189             stealTasksQueue.getStealFromQueue()) {
190       @Override
191       protected void afterExecute(Runnable r, Throwable t) {
192         super.afterExecute(r, t);
193         stealFromCountDown.countDown();
194       }
195     };
196 
197     for (int i = 0; i < 4; i++) {
198       TestTask task = new TestTask();
199       stealFromPool.execute(task);
200     }
201 
202     for (int i = 0; i < 2; i++) {
203       TestTask task = new TestTask();
204       stealPool.execute(task);
205     }
206 
207     stealJobCountDown.await(1, TimeUnit.SECONDS);
208     stealFromCountDown.await(1, TimeUnit.SECONDS);
209     assertEquals(0, stealFromCountDown.getCount());
210     assertEquals(0, stealJobCountDown.getCount());
211   }
212 
213   class TestTask extends Thread implements Comparable<TestTask> {
214     @Override
215     public int compareTo(TestTask o) {
216       return 0;
217     }
218 
219     @Override
220     public void run() {
221       try {
222         Thread.sleep(200);
223       } catch (InterruptedException e) {
224         e.printStackTrace();
225       }
226     }
227   }
228 
229 }