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.ipc; 019 020import static org.apache.hadoop.hbase.ipc.RWQueueRpcExecutor.CALL_QUEUE_READ_SHARE_CONF_KEY; 021import static org.apache.hadoop.hbase.ipc.RWQueueRpcExecutor.CALL_QUEUE_SCAN_SHARE_CONF_KEY; 022import static org.apache.hadoop.hbase.ipc.RpcExecutor.CALL_QUEUE_HANDLER_FACTOR_CONF_KEY; 023import static org.apache.hadoop.hbase.ipc.RpcExecutor.DEFAULT_CALL_QUEUE_SIZE_HARD_LIMIT; 024import static org.junit.jupiter.api.Assertions.assertEquals; 025import static org.junit.jupiter.api.Assertions.assertTrue; 026import static org.mockito.Mockito.mock; 027 028import java.util.List; 029import java.util.concurrent.BlockingQueue; 030import org.apache.hadoop.conf.Configuration; 031import org.apache.hadoop.hbase.HBaseConfiguration; 032import org.apache.hadoop.hbase.testclassification.MediumTests; 033import org.apache.hadoop.hbase.testclassification.RPCTests; 034import org.junit.jupiter.api.BeforeEach; 035import org.junit.jupiter.api.Tag; 036import org.junit.jupiter.api.Test; 037import org.junit.jupiter.api.TestInfo; 038 039@Tag(RPCTests.TAG) 040@Tag(MediumTests.TAG) 041public class TestRWQueueRpcExecutor { 042 043 private Configuration conf; 044 private String testMethodName; 045 046 @BeforeEach 047 public void setUp(TestInfo testInfo) { 048 testMethodName = testInfo.getTestMethod().get().getName(); 049 conf = HBaseConfiguration.create(); 050 conf.setFloat(CALL_QUEUE_HANDLER_FACTOR_CONF_KEY, 1.0f); 051 conf.setFloat(CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0.5f); 052 conf.setFloat(CALL_QUEUE_READ_SHARE_CONF_KEY, 0.5f); 053 } 054 055 @Test 056 public void itProvidesCorrectQueuesToBalancers() throws InterruptedException { 057 PriorityFunction qosFunction = mock(PriorityFunction.class); 058 int softQueueLimit = 100; 059 RWQueueRpcExecutor executor = 060 new RWQueueRpcExecutor(testMethodName, 100, softQueueLimit, qosFunction, conf, null); 061 062 QueueBalancer readBalancer = executor.getReadBalancer(); 063 QueueBalancer writeBalancer = executor.getWriteBalancer(); 064 QueueBalancer scanBalancer = executor.getScanBalancer(); 065 066 assertTrue(readBalancer instanceof RandomQueueBalancer); 067 assertTrue(writeBalancer instanceof RandomQueueBalancer); 068 assertTrue(scanBalancer instanceof RandomQueueBalancer); 069 070 List<BlockingQueue<CallRunner>> readQueues = ((RandomQueueBalancer) readBalancer).getQueues(); 071 List<BlockingQueue<CallRunner>> writeQueues = ((RandomQueueBalancer) writeBalancer).getQueues(); 072 List<BlockingQueue<CallRunner>> scanQueues = ((RandomQueueBalancer) scanBalancer).getQueues(); 073 074 assertEquals(25, readQueues.size()); 075 assertEquals(50, writeQueues.size()); 076 assertEquals(25, scanQueues.size()); 077 assertEquals(softQueueLimit, executor.currentQueueLimit, "Soft limit is not applied properly"); 078 // Hard Limit is applied as the max capacity of the queue 079 int hardQueueLimit = readQueues.get(0).remainingCapacity() + readQueues.get(0).size(); 080 assertEquals(DEFAULT_CALL_QUEUE_SIZE_HARD_LIMIT, hardQueueLimit, 081 "Default hard limit should be applied "); 082 083 verifyDistinct(readQueues, writeQueues, scanQueues); 084 verifyDistinct(writeQueues, readQueues, scanQueues); 085 verifyDistinct(scanQueues, readQueues, writeQueues); 086 087 } 088 089 private void verifyDistinct(List<BlockingQueue<CallRunner>> queues, 090 List<BlockingQueue<CallRunner>>... others) throws InterruptedException { 091 CallRunner mock = mock(CallRunner.class); 092 for (BlockingQueue<CallRunner> queue : queues) { 093 queue.put(mock); 094 assertEquals(1, queue.size()); 095 } 096 097 for (List<BlockingQueue<CallRunner>> other : others) { 098 for (BlockingQueue<CallRunner> queue : other) { 099 assertEquals(0, queue.size()); 100 } 101 } 102 103 // clear them for next test 104 for (BlockingQueue<CallRunner> queue : queues) { 105 queue.clear(); 106 } 107 } 108}