HBASE-19647 Logging cleanups; emit regionname when RegionTooBusyException inside RetriesExhausted... make netty connect/disconnect TRACE-level

This commit is contained in:
Michael Stack 2017-12-27 11:55:35 -08:00
parent b9f44bcfe7
commit cc8b37f847
10 changed files with 119 additions and 70 deletions

View File

@ -1,5 +1,4 @@
/*
*
* 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
@ -30,6 +29,7 @@ import java.util.Map;
import java.util.Set;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.RegionTooBusyException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
@ -154,7 +154,9 @@ extends RetriesExhaustedException {
for (Throwable t : ths) {
if (t == null) continue;
String name = "";
if (t instanceof DoNotRetryIOException) {
if (t instanceof DoNotRetryIOException ||
t instanceof RegionTooBusyException) {
// If RegionTooBusyException, print message since it has Region name in it.
name = t.getMessage();
} else {
name = t.getClass().getSimpleName();
@ -181,5 +183,4 @@ extends RetriesExhaustedException {
}
return classificatons.toString();
}
}

View File

@ -0,0 +1,65 @@
/*
* 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 org.apache.avro.generic.GenericData;
import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.RegionTooBusyException;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.junit.rules.TestRule;
import org.mockito.Mockito;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.lang.reflect.Array;
import java.util.ArrayList;
import java.util.List;
import static org.junit.Assert.assertTrue;
@Category({SmallTests.class})
public class TestRetriesExhaustedWithDetailsException {
private static final Logger LOG =
LoggerFactory.getLogger(TestRetriesExhaustedWithDetailsException.class);
@Rule public TestName name = new TestName();
@Rule public final TestRule timeout = CategoryBasedTimeout.builder().
withTimeout(this.getClass()).
withLookingForStuckThread(true).
build();
/**
* Assert that a RetriesExhaustedException that has RegionTooBusyException outputs region name.
*/
@Test
public void testRegionTooBusyException() {
List<Throwable> ts = new ArrayList<>(1);
final String regionName = this.name.getMethodName();
ts.add(new RegionTooBusyException(regionName));
List<Row> rows = new ArrayList<>(1);
rows.add(Mockito.mock(Row.class));
List<String> hostAndPorts = new ArrayList<>(1);
hostAndPorts.add("example.com:1234");
RetriesExhaustedException ree =
new RetriesExhaustedWithDetailsException(ts, rows, hostAndPorts);
assertTrue(ree.toString().contains(regionName));
}
}

View File

@ -49,10 +49,8 @@ class NettyRpcServerRequestDecoder extends ChannelInboundHandlerAdapter {
@Override
public void channelActive(ChannelHandlerContext ctx) throws Exception {
allChannels.add(ctx.channel());
if (NettyRpcServer.LOG.isDebugEnabled()) {
NettyRpcServer.LOG.debug("Connection from " + ctx.channel().remoteAddress() +
"; # active connections: " + (allChannels.size() - 1));
}
NettyRpcServer.LOG.trace("Connection {}; # active connections={}",
ctx.channel().remoteAddress(), (allChannels.size() - 1));
super.channelActive(ctx);
}
@ -67,21 +65,16 @@ class NettyRpcServerRequestDecoder extends ChannelInboundHandlerAdapter {
@Override
public void channelInactive(ChannelHandlerContext ctx) throws Exception {
allChannels.remove(ctx.channel());
if (NettyRpcServer.LOG.isDebugEnabled()) {
NettyRpcServer.LOG.debug("Disconnecting client: " + ctx.channel().remoteAddress() +
". Number of active connections: " + (allChannels.size() - 1));
}
NettyRpcServer.LOG.trace("Disconnection {}; # active connections={}",
ctx.channel().remoteAddress(), (allChannels.size() - 1));
super.channelInactive(ctx);
}
@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable e) {
allChannels.remove(ctx.channel());
if (NettyRpcServer.LOG.isDebugEnabled()) {
NettyRpcServer.LOG.debug("Connection from " + ctx.channel().remoteAddress() +
" catch unexpected exception from downstream.",
e.getCause());
}
NettyRpcServer.LOG.trace("Connection {}; caught unexpected downstream exception.",
ctx.channel().remoteAddress(), e.getCause());
ctx.channel().close();
}
}

View File

@ -405,9 +405,7 @@ public class CompactingMemStore extends AbstractMemStore {
// Phase I: Update the pipeline
getRegionServices().blockUpdates();
try {
if (LOG.isDebugEnabled()) {
LOG.debug("IN-MEMORY FLUSH: Pushing active segment into compaction pipeline");
}
LOG.trace("IN-MEMORY FLUSH: Pushing active segment into compaction pipeline");
pushActiveToPipeline(this.active);
} finally {
getRegionServices().unblockUpdates();
@ -429,9 +427,7 @@ public class CompactingMemStore extends AbstractMemStore {
}
} finally {
inMemoryFlushInProgress.set(false);
if (LOG.isDebugEnabled()) {
LOG.debug("IN-MEMORY FLUSH: end");
}
LOG.trace("IN-MEMORY FLUSH: end");
}
}

View File

@ -125,12 +125,8 @@ public class CompactionPipeline {
return false;
}
suffix = versionedList.getStoreSegments();
if (LOG.isDebugEnabled()) {
LOG.debug("Swapping pipeline suffix. "
+ "Just before the swap the number of segments in pipeline is:"
+ versionedList.getStoreSegments().size()
+ ", and the new segment is:" + segment);
}
LOG.debug("Swapping pipeline suffix; before={}, new segement={}",
versionedList.getStoreSegments().size(), segment);
swapSuffix(suffix, segment, closeSuffix);
readOnlyCopy = new LinkedList<>(pipeline);
version++;
@ -146,11 +142,12 @@ public class CompactionPipeline {
if(segment != null) newHeapSize = segment.heapSize();
long heapSizeDelta = suffixHeapSize - newHeapSize;
region.addMemStoreSize(new MemStoreSizing(-dataSizeDelta, -heapSizeDelta));
if (LOG.isDebugEnabled()) {
LOG.debug("Suffix data size: " + suffixDataSize + " new segment data size: "
+ newDataSize + ". Suffix heap size: " + suffixHeapSize
+ " new segment heap size: " + newHeapSize);
}
LOG.debug("Suffix data size={}, new segment data size={}, suffix heap size={}," +
"new segment heap size={}",
suffixDataSize,
newDataSize,
suffixHeapSize,
newHeapSize);
}
return true;
}
@ -206,7 +203,7 @@ public class CompactionPipeline {
// upon flattening there is no change in the data size
region.addMemStoreSize(new MemStoreSize(0, newMemstoreAccounting.getHeapSize()));
}
LOG.debug("Compaction pipeline segment " + s + " was flattened");
LOG.debug("Compaction pipeline segment {} flattened", s);
return true;
}
i++;

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -42,8 +43,7 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
protected long flushSizeLowerBound = -1;
protected long getFlushSizeLowerBound(HRegion region) {
int familyNumber = region.getTableDescriptor().getColumnFamilyCount();
protected long getFlushSizeLowerBound(HRegion region) { int familyNumber = region.getTableDescriptor().getColumnFamilyCount();
// For multiple families, lower bound is the "average flush size" by default
// unless setting in configuration is larger.
long flushSizeLowerBound = region.getMemStoreFlushSize() / familyNumber;
@ -57,20 +57,23 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
String flushedSizeLowerBoundString =
region.getTableDescriptor().getValue(HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND);
if (flushedSizeLowerBoundString == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("No " + HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND
+ " set in description of table " + region.getTableDescriptor().getTableName()
+ ", use config (" + flushSizeLowerBound + ") instead");
}
LOG.debug("No {} set in table {} descriptor;" +
"using region.getMemStoreFlushSize/# of families ({}) instead.",
HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND,
region.getTableDescriptor().getTableName(),
StringUtils.humanSize(flushSizeLowerBound) + ")");
} else {
try {
flushSizeLowerBound = Long.parseLong(flushedSizeLowerBoundString);
} catch (NumberFormatException nfe) {
// fall back for fault setting
LOG.warn("Number format exception when parsing "
+ HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND + " for table "
+ region.getTableDescriptor().getTableName() + ":" + flushedSizeLowerBoundString + ". " + nfe
+ ", use config (" + flushSizeLowerBound + ") instead");
LOG.warn("Number format exception parsing {} for table {}: {}, {}; " +
"using region.getMemStoreFlushSize/# of families ({}) instead.",
HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND,
region.getTableDescriptor().getTableName(),
flushedSizeLowerBoundString,
nfe,
flushSizeLowerBound);
}
}
@ -79,15 +82,13 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
protected boolean shouldFlush(HStore store) {
if (store.getMemStoreSize().getDataSize() > this.flushSizeLowerBound) {
if (LOG.isDebugEnabled()) {
LOG.debug("Flush Column Family " + store.getColumnFamilyName() + " of " +
region.getRegionInfo().getEncodedName() + " because memstoreSize=" +
store.getMemStoreSize().getDataSize() + " > lower bound="
+ this.flushSizeLowerBound);
}
LOG.debug("Flush {} of {}; memstoreSize={} > lowerBound={}",
store.getColumnFamilyName(),
region.getRegionInfo().getEncodedName(),
store.getMemStoreSize().getDataSize(),
this.flushSizeLowerBound);
return true;
}
return false;
}
}

View File

@ -4170,7 +4170,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
if (this.memstoreDataSize.get() > this.blockingMemStoreSize) {
blockedRequestsCount.increment();
requestFlush();
throw new RegionTooBusyException("Above memstore limit, " +
throw new RegionTooBusyException("Over memstore limit, " +
"regionName=" + (this.getRegionInfo() == null ? "unknown" :
this.getRegionInfo().getRegionNameAsString()) +
", server=" + (this.getRegionServerServices() == null ? "unknown" :

View File

@ -85,14 +85,13 @@ public abstract class MemStoreCompactionStrategy {
int numOfSegments = versionedList.getNumOfSegments();
if (numOfSegments > pipelineThreshold) {
// to avoid too many segments, merge now
LOG.debug(strategy+" memory compaction for store " + cfName
+ " merging " + numOfSegments + " segments");
LOG.debug("{} in-memory compaction of {}; merging {} segments",
strategy, cfName, numOfSegments);
return getMergingAction();
}
// just flatten a segment
LOG.debug(strategy+" memory compaction for store " + cfName
+ " flattening a segment in the pipeline");
LOG.debug("{} in-memory compaction of {}; flattening a segment", strategy, cfName);
return getFlattenAction();
}

View File

@ -92,11 +92,8 @@ public class MemStoreCompactor {
// get a snapshot of the list of the segments from the pipeline,
// this local copy of the list is marked with specific version
versionedList = compactingMemStore.getImmutableSegments();
if (LOG.isDebugEnabled()) {
LOG.debug("Starting the In-Memory Compaction for store "
+ compactingMemStore.getStore().getColumnFamilyName());
}
LOG.debug("Starting In-Memory Compaction of {}",
compactingMemStore.getStore().getColumnFamilyName());
HStore store = compactingMemStore.getStore();
RegionCoprocessorHost cpHost = store.getCoprocessorHost();
if (cpHost != null) {

View File

@ -351,13 +351,13 @@ public abstract class Segment {
@Override
public String toString() {
String res = "Store segment of type "+this.getClass().getName()+"; ";
res += "isEmpty "+(isEmpty()?"yes":"no")+"; ";
res += "cellsCount "+getCellsCount()+"; ";
res += "cellsSize "+keySize()+"; ";
res += "totalHeapSize "+heapSize()+"; ";
res += "Min ts " + timeRangeTracker.getMin() + "; ";
res += "Max ts " + timeRangeTracker.getMax() + "; ";
String res = "Type=" + this.getClass().getSimpleName() + ", ";
res += "empty=" + (isEmpty()? "yes": "no") + ", ";
res += "cellCount=" + getCellsCount() + ", ";
res += "cellSize=" + keySize() + ", ";
res += "totalHeapSize=" + heapSize() + ", ";
res += "min timestamp=" + timeRangeTracker.getMin() + ", ";
res += "max timestamp=" + timeRangeTracker.getMax();
return res;
}
}