You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2014/02/26 20:18:35 UTC
svn commit: r1572206 - in /hbase/branches/0.89-fb/src:
main/java/org/apache/hadoop/hbase/regionserver/
main/java/org/apache/hadoop/hbase/regionserver/metrics/
test/java/org/apache/hadoop/hbase/regionserver/compactionhook/
Author: liyin
Date: Wed Feb 26 19:18:35 2014
New Revision: 1572206
URL: http://svn.apache.org/r1572206
Log:
[HBASE-7099] [89-fb]Exception handling during compaction hook execution in Store
Author: adela
Summary:
Compaction failed completely on tsh58 due to exceptions in CompactionHook implementation
https://phabricator.fb.com/P6308922
Test Plan: updated TestLowerToUpperCompaction hook unit test to cover the case
Reviewers: rshroff, aaiyer, manukranthk, liyintang
Reviewed By: rshroff
CC: hbase-eng@
Differential Revision: https://phabricator.fb.com/D1188136
Task ID: 2095744
Modified:
hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaMetrics.java
hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/compactionhook/LowerToUpperCompactionHook.java
hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/compactionhook/TestLowerToUpperCompactionHook.java
Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1572206&r1=1572205&r2=1572206&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Wed Feb 26 19:18:35 2014
@@ -1338,6 +1338,8 @@ public class Store extends SchemaConfigu
long bytesSaved = 0;
// how many KVs were converted with compaction hook
long kvsConverted = 0;
+ // how many times exception occurred during compaction hook execution
+ long kvsErrors = 0;
long minFlushTime = Long.MAX_VALUE;
for (StoreFile file : filesToCompact) {
if (file.hasMinFlushTime() && file.getMinFlushTime() < minFlushTime) {
@@ -1416,19 +1418,27 @@ public class Store extends SchemaConfigu
kv.setMemstoreTS(0);
}
if (compactHook != null && kv.isPut()) {
- RestrictedKeyValue restrictedKv = new RestrictedKeyValue(kv);
- RestrictedKeyValue modifiedKv = compactHook.transform(restrictedKv);
- if (modifiedKv != null) {
- writer.append(modifiedKv.getKeyValue(), kvContext);
- bytesSaved += modifiedKv.differenceInBytes(kv);
- } else {
- if (kv != null) {
- bytesSaved += kv.getLength();
+ try {
+ RestrictedKeyValue restrictedKv = new RestrictedKeyValue(kv);
+ RestrictedKeyValue modifiedKv = compactHook
+ .transform(restrictedKv);
+ if (modifiedKv != null) {
+ writer.append(modifiedKv.getKeyValue(), kvContext);
+ bytesSaved += modifiedKv.differenceInBytes(kv);
+ } else {
+ if (kv != null) {
+ bytesSaved += kv.getLength();
+ }
}
- }
- if (!restrictedKv.equals(modifiedKv)) {
- kvsConverted++;
-
+ if (!restrictedKv.equals(modifiedKv)) {
+ kvsConverted++;
+ }
+ } catch (Exception e) {
+ // if exception happened just write unmodified keyvalue
+ writer.append(kv, kvContext);
+ LOG.error("Exception happened while executing compaction hook, leaving KV unchanged: "
+ + e.getCause());
+ kvsErrors++;
}
} else {
writer.append(kv, kvContext);
@@ -1447,9 +1457,13 @@ public class Store extends SchemaConfigu
.updatePersistentStoreMetric(
SchemaMetrics.StoreMetricType.STORE_COMPHOOK_KVS_TRANSFORMED,
kvsConverted);
+ getSchemaMetrics().updatePersistentStoreMetric(
+ SchemaMetrics.StoreMetricType.STORE_COMPHOOK_KVS_ERRORS,
+ kvsErrors);
bytesWritten = 0;
bytesSaved = 0;
kvsConverted = 0;
+ kvsErrors = 0;
if (!this.region.areWritesEnabled()) {
writer.close();
fs.delete(writer.getPath(), false);
@@ -1472,6 +1486,9 @@ public class Store extends SchemaConfigu
getSchemaMetrics().updatePersistentStoreMetric(
SchemaMetrics.StoreMetricType.STORE_COMPHOOK_KVS_TRANSFORMED,
kvsConverted);
+ getSchemaMetrics().updatePersistentStoreMetric(
+ SchemaMetrics.StoreMetricType.STORE_COMPHOOK_KVS_ERRORS,
+ kvsErrors);
} finally {
if (scanner != null) {
scanner.close();
Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaMetrics.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaMetrics.java?rev=1572206&r1=1572205&r2=1572206&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaMetrics.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaMetrics.java Wed Feb 26 19:18:35 2014
@@ -189,7 +189,9 @@ public class SchemaMetrics {
FLUSH_SIZE("flushSize", PERSISTENT_METRIC_FLAG),
COMPACTION_WRITE_SIZE("compactionWriteSize", PERSISTENT_METRIC_FLAG),
STORE_COMPHOOK_KVS_TRANSFORMED("storeCompHookKVsTransformed", PERSISTENT_METRIC_FLAG),
- STORE_COMPHOOK_BYTES_SAVED("storeCompHookBytesSaved", PERSISTENT_METRIC_FLAG);
+ STORE_COMPHOOK_BYTES_SAVED("storeCompHookBytesSaved", PERSISTENT_METRIC_FLAG),
+ //if exception occurred during compaction hook execution, count the number of errors
+ STORE_COMPHOOK_KVS_ERRORS("storeCompKVsErrors", PERSISTENT_METRIC_FLAG);
private final String metricStr;
private final int flags;
Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/compactionhook/LowerToUpperCompactionHook.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/compactionhook/LowerToUpperCompactionHook.java?rev=1572206&r1=1572205&r2=1572206&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/compactionhook/LowerToUpperCompactionHook.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/compactionhook/LowerToUpperCompactionHook.java Wed Feb 26 19:18:35 2014
@@ -36,6 +36,9 @@ public class LowerToUpperCompactionHook
if (currentValue.equals("abc")) {
return null;
}
+ if (currentValue.equals("aba")) {
+ throw new IllegalArgumentException("dummy exception!");
+ }
// create a copy of the kv and transform it to uppercase.
String newValueString = currentValue;
newValueString = newValueString.toUpperCase();
Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/compactionhook/TestLowerToUpperCompactionHook.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/compactionhook/TestLowerToUpperCompactionHook.java?rev=1572206&r1=1572205&r2=1572206&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/compactionhook/TestLowerToUpperCompactionHook.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/compactionhook/TestLowerToUpperCompactionHook.java Wed Feb 26 19:18:35 2014
@@ -85,6 +85,13 @@ public class TestLowerToUpperCompactionH
TABLE_STRING, FAMILY_STRING)
+ ALL_METRICS
.getStoreMetricName(StoreMetricType.STORE_COMPHOOK_BYTES_SAVED);
+
+ String kvsErrorsFullName = SchemaMetrics.generateSchemaMetricsPrefix(
+ TABLE_STRING, FAMILY_STRING)
+ + ALL_METRICS
+ .getStoreMetricName(StoreMetricType.STORE_COMPHOOK_KVS_ERRORS);
+ long startValueKvsErrors = HRegion.getNumericMetric(kvsErrorsFullName);
+
long startValueBytesSaved = HRegion.getNumericMetric(bytesSavedFullName);
// put some lowercase strings
for (int i = 0; i < 25; i++) {
@@ -101,10 +108,13 @@ public class TestLowerToUpperCompactionH
r.compactStores(true);
long transformedKVsAfterCompaction = HRegion.getNumericPersistentMetric(kvsTransformedFullName);
- Assert.assertEquals(25, transformedKVsAfterCompaction - startValueTransformedKVs);
+ // for one of them we are throwing exception - so it will not be transformed
+ Assert.assertEquals(24, transformedKVsAfterCompaction - startValueTransformedKVs);
long bytesSavedAfterCompaction = HRegion.getNumericPersistentMetric(bytesSavedFullName);
// kv with value abc should be skipped
Assert.assertTrue((bytesSavedAfterCompaction - startValueBytesSaved) > 0);
+ long kvsErrors = HRegion.getNumericPersistentMetric(kvsErrorsFullName);
+ Assert.assertEquals(1, kvsErrors - startValueKvsErrors);
Scan scan = new Scan();
InternalScanner s = r.getScanner(scan);
@@ -124,7 +134,9 @@ public class TestLowerToUpperCompactionH
for (KeyValue kv : result) {
String currValue = new String(kv.getValue());
String uppercase = currValue.toUpperCase();
- if (!currValue.equals(uppercase)) {
+ // we ignore "aba" because in that case we thew IllegalArgumentException
+ // and value stayed unchanged
+ if (!currValue.equals(uppercase) && !currValue.equals("aba")) {
return false;
}
}