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 */ 018package org.apache.hadoop.hbase; 019 020import static org.junit.Assert.assertEquals; 021import static org.junit.Assert.assertTrue; 022import static org.mockito.Mockito.mock; 023import static org.mockito.Mockito.when; 024 025import java.util.concurrent.atomic.AtomicBoolean; 026import java.util.concurrent.atomic.AtomicInteger; 027import org.apache.hadoop.hbase.executor.EventType; 028import org.apache.hadoop.hbase.executor.ExecutorService; 029import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorConfig; 030import org.apache.hadoop.hbase.executor.ExecutorType; 031import org.apache.hadoop.hbase.executor.TestExecutorService.TestEventHandler; 032import org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource; 033import org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceFactory; 034import org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceImpl; 035import org.apache.hadoop.hbase.testclassification.MiscTests; 036import org.apache.hadoop.hbase.testclassification.SmallTests; 037import org.apache.hadoop.hbase.util.Pair; 038import org.junit.ClassRule; 039import org.junit.Test; 040import org.junit.experimental.categories.Category; 041import org.slf4j.Logger; 042import org.slf4j.LoggerFactory; 043 044@Category({ MiscTests.class, SmallTests.class }) 045public class TestExecutorStatusChore { 046 @ClassRule 047 public static final HBaseClassTestRule CLASS_RULE = 048 HBaseClassTestRule.forClass(TestExecutorStatusChore.class); 049 050 private static final Logger LOG = LoggerFactory.getLogger(TestExecutorStatusChore.class); 051 052 @Test 053 public void testMetricsCollect() throws Exception { 054 int maxThreads = 5; 055 int maxTries = 10; 056 int sleepInterval = 1000; 057 058 Server mockedServer = mock(Server.class); 059 when(mockedServer.getConfiguration()).thenReturn(HBaseConfiguration.create()); 060 061 // Start an executor service pool with max 5 threads 062 ExecutorService executorService = new ExecutorService("unit_test"); 063 executorService.startExecutorService(executorService.new ExecutorConfig() 064 .setExecutorType(ExecutorType.RS_PARALLEL_SEEK).setCorePoolSize(maxThreads)); 065 066 MetricsRegionServerSource serverSource = CompatibilitySingletonFactory 067 .getInstance(MetricsRegionServerSourceFactory.class).createServer(null); 068 assertTrue(serverSource instanceof MetricsRegionServerSourceImpl); 069 070 ExecutorStatusChore statusChore = 071 new ExecutorStatusChore(60000, mockedServer, executorService, serverSource); 072 073 AtomicBoolean lock = new AtomicBoolean(true); 074 AtomicInteger counter = new AtomicInteger(0); 075 for (int i = 0; i < maxThreads + 1; i++) { 076 executorService 077 .submit(new TestEventHandler(mockedServer, EventType.RS_PARALLEL_SEEK, lock, counter)); 078 } 079 080 // The TestEventHandler will increment counter when it starts. 081 int tries = 0; 082 while (counter.get() < maxThreads && tries < maxTries) { 083 LOG.info("Waiting for all event handlers to start..."); 084 Thread.sleep(sleepInterval); 085 tries++; 086 } 087 088 // Assert that pool is at max threads. 089 assertEquals(maxThreads, counter.get()); 090 091 statusChore.chore(); 092 Pair<Long, Long> executorStatus = statusChore.getExecutorStatus("RS_PARALLEL_SEEK"); 093 assertEquals(maxThreads, executorStatus.getFirst().intValue()); // running 094 assertEquals(1, executorStatus.getSecond().intValue()); // pending 095 096 // Now interrupt the running Executor 097 synchronized (lock) { 098 lock.set(false); 099 lock.notifyAll(); 100 } 101 executorService.shutdown(); 102 } 103}