Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HBASE-27766 Support steal job queue mode for read RPC queues of RWQue… #5260

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.ipc;

import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.yetus.audience.InterfaceAudience;

@InterfaceAudience.Private
public class FIFOStealJobQueue<T> extends LinkedBlockingQueue<T> {
private static final long serialVersionUID = -4984005244760265988L;

private BlockingQueue<T> stealFromQueue;

private final Lock lock = new ReentrantLock();
private final transient Condition notEmpty = lock.newCondition();

public FIFOStealJobQueue(int initCapacity, int stealFromQueueInitCapacity) {
super(initCapacity);
this.stealFromQueue = new LinkedBlockingQueue<T>(stealFromQueueInitCapacity) {

private static final long serialVersionUID = -6059419446245599796L;

@Override
public boolean offer(T t) {
lock.lock();
try {
notEmpty.signal();
return super.offer(t);
} finally {
lock.unlock();
}
}
};
}

/**
* Get a queue whose job might be stolen by the consumer of this original queue
* @return the queue whose job could be stolen
*/
public BlockingQueue<T> getStealFromQueue() {
return stealFromQueue;
}

@Override
public boolean offer(T t) {
lock.lock();
try {
notEmpty.signal();
return super.offer(t);
} finally {
lock.unlock();
}
}

@Override
public T take() throws InterruptedException {
lock.lockInterruptibly();
try {
while (true) {
T retVal = this.poll();
if (retVal == null) {
retVal = stealFromQueue.poll();
}
if (retVal == null) {
notEmpty.await();
} else {
return retVal;
}
}
} finally {
lock.unlock();
}
}

@Override
public T poll(long timeout, TimeUnit unit) throws InterruptedException {
long nanos = unit.toNanos(timeout);
lock.lockInterruptibly();
try {
while (true) {
T retVal = this.poll();
if (retVal == null) {
retVal = stealFromQueue.poll();
}
if (retVal == null) {
if (nanos <= 0) return null;
nanos = notEmpty.awaitNanos(nanos);
} else {
return retVal;
}
}
} finally {
lock.unlock();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -58,9 +58,9 @@ public class RWQueueRpcExecutor extends RpcExecutor {
private final int writeHandlersCount;
private final int readHandlersCount;
private final int scanHandlersCount;
private final int numWriteQueues;
private final int numReadQueues;
private final int numScanQueues;
protected final int numWriteQueues;
protected final int numReadQueues;
protected final int numScanQueues;

private final AtomicInteger activeWriteHandlerCount = new AtomicInteger(0);
private final AtomicInteger activeReadHandlerCount = new AtomicInteger(0);
Expand Down Expand Up @@ -97,9 +97,7 @@ public RWQueueRpcExecutor(final String name, final int handlerCount, final int m
numScanQueues = scanQueues;
scanHandlersCount = scanHandlers;

initializeQueues(numWriteQueues);
initializeQueues(numReadQueues);
initializeQueues(numScanQueues);
initQueues();

this.writeBalancer = getBalancer(name, conf, queues.subList(0, numWriteQueues));
this.readBalancer =
Expand All @@ -115,6 +113,12 @@ public RWQueueRpcExecutor(final String name, final int handlerCount, final int m
+ numScanQueues + " scanHandlers=" + scanHandlersCount);
}

protected void initQueues() {
initializeQueues(numWriteQueues);
initializeQueues(numReadQueues);
initializeQueues(numScanQueues);
}

@Override
protected int computeNumCallQueues(final int handlerCount, final float callQueuesHandlersFactor) {
// at least 1 read queue and 1 write queue
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@ public abstract class RpcExecutor {
public static final String CALL_QUEUE_TYPE_CODEL_CONF_VALUE = "codel";
public static final String CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE = "deadline";
public static final String CALL_QUEUE_TYPE_FIFO_CONF_VALUE = "fifo";
public static final String CALL_QUEUE_TYPE_READ_STEAL_CONF_VALUE = "readSteal";
public static final String CALL_QUEUE_TYPE_PLUGGABLE_CONF_VALUE = "pluggable";
public static final String CALL_QUEUE_TYPE_CONF_KEY = "hbase.ipc.server.callqueue.type";
public static final String CALL_QUEUE_TYPE_CONF_DEFAULT = CALL_QUEUE_TYPE_FIFO_CONF_VALUE;
Expand Down Expand Up @@ -101,7 +102,7 @@ public abstract class RpcExecutor {
private final LongAdder numLifoModeSwitches = new LongAdder();

protected final int numCallQueues;
protected final List<BlockingQueue<CallRunner>> queues;
protected List<BlockingQueue<CallRunner>> queues;
private final Class<? extends BlockingQueue> queueClass;
private final Object[] queueInitArgs;

Expand All @@ -117,6 +118,8 @@ public abstract class RpcExecutor {
private final Configuration conf;
private final Abortable abortable;

protected int maxQueueLength;

public RpcExecutor(final String name, final int handlerCount, final int maxQueueLength,
final PriorityFunction priority, final Configuration conf, final Abortable abortable) {
this(name, handlerCount, conf.get(CALL_QUEUE_TYPE_CONF_KEY, CALL_QUEUE_TYPE_CONF_DEFAULT),
Expand All @@ -129,6 +132,7 @@ public RpcExecutor(final String name, final int handlerCount, final String callQ
this.name = Strings.nullToEmpty(name);
this.conf = conf;
this.abortable = abortable;
this.maxQueueLength = maxQueueLength;

float callQueuesHandlersFactor = this.conf.getFloat(CALL_QUEUE_HANDLER_FACTOR_CONF_KEY, 0.1f);
if (
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hbase.ipc;

import static org.apache.hadoop.hbase.ipc.RpcExecutor.CALL_QUEUE_TYPE_READ_STEAL_CONF_VALUE;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
Expand Down Expand Up @@ -86,9 +88,14 @@ public SimpleRpcScheduler(Configuration conf, int handlerCount, int priorityHand
float callqReadShare = conf.getFloat(RWQueueRpcExecutor.CALL_QUEUE_READ_SHARE_CONF_KEY, 0);

if (callqReadShare > 0) {
// at least 1 read handler and 1 write handler
callExecutor = new FastPathRWQueueRpcExecutor("default.FPRWQ", Math.max(2, handlerCount),
maxQueueLength, priority, conf, server);
if (callQueueType.equals(CALL_QUEUE_TYPE_READ_STEAL_CONF_VALUE)) {
callExecutor = new StealReadJobRWQueueRpcExecutor("default.SRRWQ",
Math.max(2, handlerCount), maxQueueLength, priority, conf, server);
} else {
// at least 1 read handler and 1 write handler
callExecutor = new FastPathRWQueueRpcExecutor("default.FPRWQ", Math.max(2, handlerCount),
maxQueueLength, priority, conf, server);
}
} else {
if (
RpcExecutor.isFifoQueueType(callQueueType) || RpcExecutor.isCodelQueueType(callQueueType)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.ipc;

import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.BlockingQueue;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;

@InterfaceAudience.LimitedPrivate({ HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX })
@InterfaceStability.Evolving
public class StealReadJobRWQueueRpcExecutor extends FastPathRWQueueRpcExecutor {
public StealReadJobRWQueueRpcExecutor(String name, int handlerCount, int maxQueueLength,
PriorityFunction priority, Configuration conf, Abortable abortable) {
super(name, handlerCount, maxQueueLength, priority, conf, abortable);
}

@Override
public void initQueues() {
queues = new ArrayList<>(this.numWriteQueues + this.numReadQueues + numScanQueues);
initializeQueues(numWriteQueues);
if (numReadQueues > 0 && numScanQueues > 0) {
int stealQueueCount = Math.min(numReadQueues, numScanQueues);
List<BlockingQueue<CallRunner>> stealScanQueues = new ArrayList<>(stealQueueCount);
for (int i = 0; i < stealQueueCount; i++) {
FIFOStealJobQueue<CallRunner> scanQueue =
new FIFOStealJobQueue<>(maxQueueLength, maxQueueLength);
BlockingQueue<CallRunner> readQueue = scanQueue.getStealFromQueue();
queues.add(readQueue);
stealScanQueues.add(scanQueue);
}
if (numReadQueues > numScanQueues) {
initializeQueues(numReadQueues - numScanQueues);
}
queues.addAll(stealScanQueues);
if (numScanQueues > numReadQueues) {
initializeQueues(numScanQueues - numReadQueues);
}
} else {
initializeQueues(Math.max(numReadQueues, numScanQueues));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -65,17 +65,17 @@ public class TestMultiParallel {

private static final Logger LOG = LoggerFactory.getLogger(TestMultiParallel.class);

private static final HBaseTestingUtil UTIL = new HBaseTestingUtil();
protected static final HBaseTestingUtil UTIL = new HBaseTestingUtil();
private static final byte[] VALUE = Bytes.toBytes("value");
private static final byte[] QUALIFIER = Bytes.toBytes("qual");
private static final String FAMILY = "family";
private static final TableName TEST_TABLE = TableName.valueOf("multi_test_table");
protected static final String FAMILY = "family";
protected static final TableName TEST_TABLE = TableName.valueOf("multi_test_table");
private static final byte[] BYTES_FAMILY = Bytes.toBytes(FAMILY);
private static final byte[] ONE_ROW = Bytes.toBytes("xxx");
private static final byte[][] KEYS = makeKeys();

private static final int slaves = 5; // also used for testing HTable pool size
private static Connection CONNECTION;
protected static final int slaves = 5; // also used for testing HTable pool size
protected static Connection CONNECTION;

@BeforeClass
public static void beforeClass() throws Exception {
Expand Down Expand Up @@ -662,7 +662,7 @@ private void validateSizeAndEmpty(Object[] results, int expectedSize) {

public static class MyMasterObserver implements MasterObserver, MasterCoprocessor {
private static final AtomicInteger postBalanceCount = new AtomicInteger(0);
private static final AtomicBoolean start = new AtomicBoolean(false);
protected static final AtomicBoolean start = new AtomicBoolean(false);

@Override
public void start(CoprocessorEnvironment env) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;

import static org.junit.Assert.assertTrue;

import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.codec.KeyValueCodec;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.ipc.RWQueueRpcExecutor;
import org.apache.hadoop.hbase.ipc.RpcExecutor;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;

@Category({ MediumTests.class })
public class TestMultiParallel2 extends TestMultiParallel {

@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMultiParallel2.class);

public static void beforeClass() throws Exception {
// Uncomment the following lines if more verbosity is needed for
// debugging (see HBASE-12285 for details).
// ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
// ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
// ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
UTIL.getConfiguration().set(HConstants.RPC_CODEC_CONF_KEY,
KeyValueCodec.class.getCanonicalName());
// Disable table on master for now as the feature is broken
// UTIL.getConfiguration().setBoolean(LoadBalancer.TABLES_ON_MASTER, true);
// We used to ask for system tables on Master exclusively but not needed by test and doesn't
// work anyways -- so commented out.
// UTIL.getConfiguration().setBoolean(LoadBalancer.SYSTEM_TABLES_ON_MASTER, true);
UTIL.getConfiguration().set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
MyMasterObserver.class.getName());
String queueType = RpcExecutor.CALL_QUEUE_TYPE_READ_STEAL_CONF_VALUE;
UTIL.getConfiguration().set(RpcExecutor.CALL_QUEUE_TYPE_CONF_KEY, queueType);
UTIL.getConfiguration().setFloat(RWQueueRpcExecutor.CALL_QUEUE_READ_SHARE_CONF_KEY, 0.5f);
UTIL.getConfiguration().setFloat(RWQueueRpcExecutor.CALL_QUEUE_SCAN_SHARE_CONF_KEY, 1);
UTIL.startMiniCluster(slaves);
Table t = UTIL.createMultiRegionTable(TEST_TABLE, Bytes.toBytes(FAMILY));
UTIL.waitTableEnabled(TEST_TABLE);
t.close();
CONNECTION = ConnectionFactory.createConnection(UTIL.getConfiguration());
assertTrue(MyMasterObserver.start.get());
}

@Test
public void test() throws Exception {
testBatchWithGet();
}
}
Loading