HBASE-13420 RegionEnvironment.offerExecutionLatency Blocks Threads under Heavy Load
This commit is contained in:
parent
51ce568d6c
commit
81e793e582
|
@ -27,8 +27,6 @@ import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.NavigableSet;
|
import java.util.NavigableSet;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.concurrent.ArrayBlockingQueue;
|
|
||||||
import java.util.concurrent.BlockingQueue;
|
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.regex.Matcher;
|
import java.util.regex.Matcher;
|
||||||
|
@ -82,6 +80,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||||
import org.apache.hadoop.hbase.wal.WALKey;
|
import org.apache.hadoop.hbase.wal.WALKey;
|
||||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||||
|
import org.apache.hadoop.hbase.util.BoundedConcurrentLinkedQueue;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
|
import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
|
@ -101,6 +100,7 @@ public class RegionCoprocessorHost
|
||||||
new ReferenceMap(AbstractReferenceMap.HARD, AbstractReferenceMap.WEAK);
|
new ReferenceMap(AbstractReferenceMap.HARD, AbstractReferenceMap.WEAK);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
*
|
||||||
* Encapsulation of the environment of each coprocessor
|
* Encapsulation of the environment of each coprocessor
|
||||||
*/
|
*/
|
||||||
static class RegionEnvironment extends CoprocessorHost.Environment
|
static class RegionEnvironment extends CoprocessorHost.Environment
|
||||||
|
@ -110,8 +110,8 @@ public class RegionCoprocessorHost
|
||||||
private RegionServerServices rsServices;
|
private RegionServerServices rsServices;
|
||||||
ConcurrentMap<String, Object> sharedData;
|
ConcurrentMap<String, Object> sharedData;
|
||||||
private static final int LATENCY_BUFFER_SIZE = 100;
|
private static final int LATENCY_BUFFER_SIZE = 100;
|
||||||
private final BlockingQueue<Long> coprocessorTimeNanos = new ArrayBlockingQueue<Long>(
|
private final BoundedConcurrentLinkedQueue<Long> coprocessorTimeNanos =
|
||||||
LATENCY_BUFFER_SIZE);
|
new BoundedConcurrentLinkedQueue<Long>(LATENCY_BUFFER_SIZE);
|
||||||
private final boolean useLegacyPre;
|
private final boolean useLegacyPre;
|
||||||
private final boolean useLegacyPost;
|
private final boolean useLegacyPost;
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,114 @@
|
||||||
|
/*
|
||||||
|
* 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.util;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A ConcurrentLinkedQueue that enforces a maximum queue size.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Stable
|
||||||
|
public class BoundedConcurrentLinkedQueue<T> extends ConcurrentLinkedQueue<T> {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
private volatile long size = 0;
|
||||||
|
private final long maxSize;
|
||||||
|
|
||||||
|
public BoundedConcurrentLinkedQueue() {
|
||||||
|
this(Long.MAX_VALUE);
|
||||||
|
}
|
||||||
|
|
||||||
|
public BoundedConcurrentLinkedQueue(long maxSize) {
|
||||||
|
super();
|
||||||
|
this.maxSize = maxSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean add(T e) {
|
||||||
|
return offer(e);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean addAll(Collection<? extends T> c) {
|
||||||
|
size += c.size(); // Between here and below we might reject offers,
|
||||||
|
if (size > maxSize) { // if over maxSize, but that's ok
|
||||||
|
size -= c.size(); // We're over, just back out and return.
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return super.addAll(c); // Always true for ConcurrentLinkedQueue
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void clear() {
|
||||||
|
super.clear();
|
||||||
|
size = 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean offer(T e) {
|
||||||
|
if (++size > maxSize) {
|
||||||
|
--size; // We didn't take it after all
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return super.offer(e); // Always true for ConcurrentLinkedQueue
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public T poll() {
|
||||||
|
T result = super.poll();
|
||||||
|
if (result != null) {
|
||||||
|
--size;
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean remove(Object o) {
|
||||||
|
boolean result = super.remove(o);
|
||||||
|
if (result) {
|
||||||
|
--size;
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int size() {
|
||||||
|
return (int) size;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void drainTo(Collection<T> list) {
|
||||||
|
long removed = 0;
|
||||||
|
T l;
|
||||||
|
while ((l = super.poll()) != null) {
|
||||||
|
list.add(l);
|
||||||
|
removed++;
|
||||||
|
}
|
||||||
|
// Limit the number of operations on a volatile by only reporting size
|
||||||
|
// change after the drain is completed.
|
||||||
|
size -= removed;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long remainingCapacity() {
|
||||||
|
long remaining = maxSize - size;
|
||||||
|
return remaining >= 0 ? remaining : 0;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,86 @@
|
||||||
|
/*
|
||||||
|
* 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.util;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
|
||||||
|
@Category({MiscTests.class, SmallTests.class})
|
||||||
|
public class TestBoundedConcurrentLinkedQueue {
|
||||||
|
private final static int CAPACITY = 16;
|
||||||
|
|
||||||
|
private BoundedConcurrentLinkedQueue<Long> queue;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
this.queue = new BoundedConcurrentLinkedQueue<Long>(CAPACITY);
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() throws Exception {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOfferAndPoll() throws Exception {
|
||||||
|
// Offer
|
||||||
|
for (long i = 1; i <= CAPACITY; ++i) {
|
||||||
|
assertTrue(queue.offer(i));
|
||||||
|
assertEquals(i, queue.size());
|
||||||
|
assertEquals(CAPACITY - i, queue.remainingCapacity());
|
||||||
|
}
|
||||||
|
assertFalse(queue.offer(0L));
|
||||||
|
|
||||||
|
// Poll
|
||||||
|
for (int i = 1; i <= CAPACITY; ++i) {
|
||||||
|
long l = queue.poll();
|
||||||
|
assertEquals(i, l);
|
||||||
|
assertEquals(CAPACITY - i, queue.size());
|
||||||
|
assertEquals(i, queue.remainingCapacity());
|
||||||
|
}
|
||||||
|
assertEquals(null, queue.poll());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDrain() throws Exception {
|
||||||
|
// Offer
|
||||||
|
for (long i = 1; i <= CAPACITY; ++i) {
|
||||||
|
assertTrue(queue.offer(i));
|
||||||
|
assertEquals(i, queue.size());
|
||||||
|
assertEquals(CAPACITY - i, queue.remainingCapacity());
|
||||||
|
}
|
||||||
|
assertFalse(queue.offer(0L));
|
||||||
|
|
||||||
|
// Drain
|
||||||
|
List<Long> list = new ArrayList<Long>();
|
||||||
|
queue.drainTo(list);
|
||||||
|
assertEquals(null, queue.poll());
|
||||||
|
assertEquals(0, queue.size());
|
||||||
|
assertEquals(CAPACITY, queue.remainingCapacity());
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue