HBASE-16085 Add a metric for failed compactions

This commit is contained in:
Gary Helmling 2016-06-22 17:08:28 -07:00
parent 9cd097cfac
commit f4cec2e202
9 changed files with 81 additions and 2 deletions

View File

@ -28,9 +28,11 @@ public interface MetricsRegionSource extends Comparable<MetricsRegionSource> {
String OPS_SAMPLE_NAME = "ops";
String SIZE_VALUE_NAME = "size";
String COMPACTIONS_COMPLETED_COUNT = "compactionsCompletedCount";
String COMPACTIONS_FAILED_COUNT = "compactionsFailedCount";
String NUM_BYTES_COMPACTED_COUNT = "numBytesCompactedCount";
String NUM_FILES_COMPACTED_COUNT = "numFilesCompactedCount";
String COMPACTIONS_COMPLETED_DESC = "Number of compactions that have completed.";
String COMPACTIONS_FAILED_DESC = "Number of compactions that have failed.";
String NUM_BYTES_COMPACTED_DESC =
"Sum of filesize on all files entering a finished, successful or aborted, compaction";
String NUM_FILES_COMPACTED_DESC =

View File

@ -105,6 +105,13 @@ public interface MetricsRegionWrapper {
long getNumCompactionsCompleted();
/**
* Returns the total number of compactions that have been reported as failed on this region.
* Note that a given compaction can be reported as both completed and failed if an exception
* is thrown in the processing after {@code HRegion.compact()}.
*/
long getNumCompactionsFailed();
int getRegionHashCode();
/**

View File

@ -264,6 +264,10 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
regionNamePrefix + MetricsRegionSource.COMPACTIONS_COMPLETED_COUNT,
MetricsRegionSource.COMPACTIONS_COMPLETED_DESC),
this.regionWrapper.getNumCompactionsCompleted());
mrb.addCounter(Interns.info(
regionNamePrefix + MetricsRegionSource.COMPACTIONS_FAILED_COUNT,
MetricsRegionSource.COMPACTIONS_FAILED_DESC),
this.regionWrapper.getNumCompactionsFailed());
mrb.addCounter(Interns.info(
regionNamePrefix + MetricsRegionSource.NUM_BYTES_COMPACTED_COUNT,
MetricsRegionSource.NUM_BYTES_COMPACTED_DESC),

View File

@ -151,6 +151,11 @@ public class TestMetricsRegionSourceImpl {
return 0;
}
@Override
public long getNumCompactionsFailed() {
return 0;
}
@Override
public int getRegionHashCode() {
return regionName.hashCode();

View File

@ -547,9 +547,11 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
if (remoteEx != ex) {
LOG.info("Compaction failed at original callstack: " + formatStackTrace(ex));
}
region.reportCompactionRequestFailure();
server.checkFileSystem();
} catch (Exception ex) {
LOG.error("Compaction failed " + this, ex);
region.reportCompactionRequestFailure();
server.checkFileSystem();
} finally {
LOG.debug("CompactSplitThread Status: " + CompactSplitThread.this);

View File

@ -279,6 +279,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// Compaction counters
final AtomicLong compactionsFinished = new AtomicLong(0L);
final AtomicLong compactionsFailed = new AtomicLong(0L);
final AtomicLong compactionNumFilesCompacted = new AtomicLong(0L);
final AtomicLong compactionNumBytesCompacted = new AtomicLong(0L);
@ -7507,7 +7508,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
public static final long FIXED_OVERHEAD = ClassSize.align(
ClassSize.OBJECT +
ClassSize.ARRAY +
47 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
48 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
(14 * Bytes.SIZEOF_LONG) +
5 * Bytes.SIZEOF_BOOLEAN);
@ -7524,7 +7525,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
ClassSize.OBJECT + // closeLock
(2 * ClassSize.ATOMIC_BOOLEAN) + // closed, closing
(3 * ClassSize.ATOMIC_LONG) + // memStoreSize, numPutsWithoutWAL, dataInMemoryWithoutWAL
(4 * ClassSize.ATOMIC_LONG) + // memStoreSize, numPutsWithoutWAL, dataInMemoryWithoutWAL,
// compactionsFailed
(2 * ClassSize.CONCURRENT_HASHMAP) + // lockedRows, scannerReadPoints
WriteState.HEAP_SIZE + // writestate
ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + // stores
@ -7973,6 +7975,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
assert newValue >= 0;
}
public void reportCompactionRequestFailure() {
compactionsFailed.incrementAndGet();
}
@VisibleForTesting
public long getReadPoint() {
return getReadPoint(IsolationLevel.READ_COMMITTED);

View File

@ -140,6 +140,11 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
return this.region.compactionsFinished.get();
}
@Override
public long getNumCompactionsFailed() {
return this.region.compactionsFailed.get();
}
@Override
public long getMaxStoreFileAge() {
return maxStoreFileAge;

View File

@ -120,6 +120,11 @@ public class MetricsRegionWrapperStub implements MetricsRegionWrapper {
return 0;
}
@Override
public long getNumCompactionsFailed() {
return 0;
}
@Override
public int getRegionHashCode() {
return 42;

View File

@ -36,6 +36,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -297,6 +298,48 @@ public class TestCompaction {
thread.interruptIfNecessary();
}
@Test
public void testCompactionFailure() throws Exception {
// setup a compact/split thread on a mock server
HRegionServer mockServer = Mockito.mock(HRegionServer.class);
Mockito.when(mockServer.getConfiguration()).thenReturn(r.getBaseConf());
CompactSplitThread thread = new CompactSplitThread(mockServer);
Mockito.when(mockServer.getCompactSplitThread()).thenReturn(thread);
// setup a region/store with some files
Store store = r.getStore(COLUMN_FAMILY);
createStoreFile(r);
for (int i = 0; i < HStore.DEFAULT_BLOCKING_STOREFILE_COUNT - 1; i++) {
createStoreFile(r);
}
HRegion mockRegion = Mockito.spy(r);
Mockito.when(mockRegion.checkSplit()).thenThrow(new IndexOutOfBoundsException());
MetricsRegionWrapper metricsWrapper = new MetricsRegionWrapperImpl(r);
long preCompletedCount = metricsWrapper.getNumCompactionsCompleted();
long preFailedCount = metricsWrapper.getNumCompactionsFailed();
CountDownLatch latch = new CountDownLatch(1);
TrackableCompactionRequest request = new TrackableCompactionRequest(latch);
thread.requestCompaction(mockRegion, store, "test custom comapction", Store.PRIORITY_USER,
request, null);
// wait for the latch to complete.
latch.await(120, TimeUnit.SECONDS);
// compaction should have completed and been marked as failed due to error in split request
long postCompletedCount = metricsWrapper.getNumCompactionsCompleted();
long postFailedCount = metricsWrapper.getNumCompactionsFailed();
assertTrue("Completed count should have increased (pre=" + preCompletedCount +
", post="+postCompletedCount+")",
postCompletedCount > preCompletedCount);
assertTrue("Failed count should have increased (pre=" + preFailedCount +
", post=" + postFailedCount + ")",
postFailedCount > preFailedCount);
}
/**
* HBASE-7947: Regression test to ensure adding to the correct list in the
* {@link CompactSplitThread}