HBASE-18043 Institute a hard limit for individual cell size that cannot be overridden by clients

This commit is contained in:
Andrew Purtell 2017-05-15 18:03:33 -07:00
parent 841bb00655
commit 6b60ba8ade
4 changed files with 90 additions and 3 deletions

View File

@ -531,6 +531,15 @@ possible configurations would overwhelm and obscure the important.
to set this to a fraction of the maximum region size. Setting it to zero
or less disables the check.</description>
</property>
<property>
<name>hbase.server.keyvalue.maxsize</name>
<value>10485760</value>
<description>Maximum allowed size of an individual cell, inclusive of value and all key
components. A value of 0 or less disables the check.
The default value is 10MB.
This is a safety setting to protect the server from OOM situations.
</description>
</property>
<property>
<name>hbase.client.scanner.timeout.period</name>
<value>60000</value>

View File

@ -205,6 +205,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
public static final String HREGION_UNASSIGN_FOR_FNFE = "hbase.hregion.unassign.for.fnfe";
public static final boolean DEFAULT_HREGION_UNASSIGN_FOR_FNFE = true;
public static final String HBASE_MAX_CELL_SIZE_KEY = "hbase.server.keyvalue.maxsize";
public static final int DEFAULT_MAX_CELL_SIZE = 10485760;
/**
* This is the global default value for durability. All tables/mutations not
* defining a durability or using USE_DEFAULT will default to this value.
@ -307,6 +310,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// purge timeout, when a RPC call will be terminated by the RPC engine.
final long maxBusyWaitDuration;
// Max cell size. If nonzero, the maximum allowed size for any given cell
// in bytes
final long maxCellSize;
// negative number indicates infinite timeout
static final long DEFAULT_ROW_PROCESSOR_TIMEOUT = 60 * 1000L;
final ExecutorService rowProcessorExecutor = Executors.newCachedThreadPool();
@ -801,6 +808,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
false :
conf.getBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE,
HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE);
this.maxCellSize = conf.getLong(HBASE_MAX_CELL_SIZE_KEY, DEFAULT_MAX_CELL_SIZE);
boolean unassignForFNFE =
conf.getBoolean(HREGION_UNASSIGN_FOR_FNFE, DEFAULT_HREGION_UNASSIGN_FOR_FNFE);
if (unassignForFNFE) {
@ -7613,7 +7623,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
ClassSize.OBJECT +
ClassSize.ARRAY +
49 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
(14 * Bytes.SIZEOF_LONG) +
(15 * Bytes.SIZEOF_LONG) +
6 * Bytes.SIZEOF_BOOLEAN);
// woefully out of date - currently missing:

View File

@ -525,7 +525,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
switch (type) {
case PUT:
rm.add(ProtobufUtil.toPut(action.getMutation(), cellScanner));
Put put = ProtobufUtil.toPut(action.getMutation(), cellScanner);
checkCellSizeLimit(region, put);
rm.add(put);
break;
case DELETE:
rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner));
@ -577,7 +579,9 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
switch (type) {
case PUT:
rm.add(ProtobufUtil.toPut(action.getMutation(), cellScanner));
Put put = ProtobufUtil.toPut(action.getMutation(), cellScanner);
checkCellSizeLimit(region, put);
rm.add(put);
break;
case DELETE:
rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner));
@ -611,6 +615,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
throws IOException {
long before = EnvironmentEdgeManager.currentTime();
Append append = ProtobufUtil.toAppend(mutation, cellScanner);
checkCellSizeLimit(region, append);
quota.addMutation(append);
Result r = null;
if (region.getCoprocessorHost() != null) {
@ -659,6 +664,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
throws IOException {
long before = EnvironmentEdgeManager.currentTime();
Increment increment = ProtobufUtil.toIncrement(mutation, cells);
checkCellSizeLimit(region, increment);
quota.addMutation(increment);
Result r = null;
if (region.getCoprocessorHost() != null) {
@ -867,6 +873,26 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
return cellsToReturn;
}
private void checkCellSizeLimit(final Region region, final Mutation m) throws IOException {
if (!(region instanceof HRegion)) {
return;
}
HRegion r = (HRegion)region;
if (r.maxCellSize > 0) {
CellScanner cells = m.cellScanner();
while (cells.advance()) {
int size = CellUtil.estimatedSerializedSizeOf(cells.current());
if (size > r.maxCellSize) {
String msg = "Cell with size " + size + " exceeds limit of " + r.maxCellSize + " bytes";
if (LOG.isDebugEnabled()) {
LOG.debug(msg);
}
throw new DoNotRetryIOException(msg);
}
}
}
}
/**
* Execute a list of Put/Delete mutations.
*
@ -902,15 +928,18 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
mutationActionMap.put(mutation, action);
mArray[i++] = mutation;
checkCellSizeLimit(region, mutation);
quota.addMutation(mutation);
}
if (!region.getRegionInfo().isMetaTable()) {
regionServer.cacheFlusher.reclaimMemStoreMemory();
}
// HBASE-17924
// sort to improve lock efficiency
Arrays.sort(mArray);
OperationStatus[] codes = region.batchMutate(mArray, HConstants.NO_NONCE,
HConstants.NO_NONCE);
for (i = 0; i < codes.length; i++) {
@ -2604,6 +2633,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
break;
case PUT:
Put put = ProtobufUtil.toPut(mutation, cellScanner);
checkCellSizeLimit(region, put);
quota.addMutation(put);
if (request.hasCondition()) {
Condition condition = request.getCondition();
@ -2633,6 +2663,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
break;
case DELETE:
Delete delete = ProtobufUtil.toDelete(mutation, cellScanner);
checkCellSizeLimit(region, delete);
quota.addMutation(delete);
if (request.hasCondition()) {
Condition condition = request.getCondition();

View File

@ -99,6 +99,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Mut
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
import org.apache.hadoop.hbase.regionserver.DelegatingKeyValueScanner;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
@ -6291,4 +6292,40 @@ public class TestFromClientSide {
.getNumberOfCachedRegionLocations(htd.getTableName());
assertEquals(results.size(), number);
}
@Test
public void testCellSizeLimit() throws IOException {
final TableName tableName = TableName.valueOf("testCellSizeLimit");
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.setConfiguration(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(10 * 1024)); // 10K
HColumnDescriptor fam = new HColumnDescriptor(FAMILY);
htd.addFamily(fam);
Admin admin = TEST_UTIL.getAdmin();
admin.createTable(htd);
// Will succeed
try (Table t = TEST_UTIL.getConnection().getTable(tableName)) {
t.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, Bytes.toBytes(0L)));
t.increment(new Increment(ROW).addColumn(FAMILY, QUALIFIER, 1L));
}
// Will succeed
try (Table t = TEST_UTIL.getConnection().getTable(tableName)) {
t.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, new byte[9*1024]));
}
// Will fail
try (Table t = TEST_UTIL.getConnection().getTable(tableName)) {
try {
t.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, new byte[10 * 1024]));
fail("Oversize cell failed to trigger exception");
} catch (IOException e) {
// expected
}
try {
t.append(new Append(ROW).add(FAMILY, QUALIFIER, new byte[10 * 1024]));
fail("Oversize cell failed to trigger exception");
} catch (IOException e) {
// expected
}
}
}
}