Revert "HBASE-26122: Implement an optional maximum size for Gets, after which a partial result is returned (#3532)"

This reverts commit 8f16e34eb2.
This commit is contained in:
stack 2021-08-11 07:41:22 -07:00
parent 8f16e34eb2
commit 243e861985
8 changed files with 13 additions and 202 deletions

View File

@ -76,7 +76,6 @@ public class Get extends Query implements Row {
private boolean checkExistenceOnly = false;
private boolean closestRowBefore = false;
private Map<byte [], NavigableSet<byte []>> familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
private long maxResultSize = -1;
/**
* Create a Get operation for the specified row.
@ -340,21 +339,6 @@ public class Get extends Query implements Row {
return this;
}
/**
* Set the maximum result size. The default is -1; this means that no specific
* maximum result size will be set for this Get.
*
* If set to a value greater than zero, the server may respond with a Result where
* {@link Result#mayHaveMoreCellsInRow()} is true. The user is required to handle
* this case.
*
* @param maxResultSize The maximum result size in bytes
*/
public Get setMaxResultSize(long maxResultSize) {
this.maxResultSize = maxResultSize;
return this;
}
/* Accessors */
/**
@ -474,13 +458,6 @@ public class Get extends Query implements Row {
return map;
}
/**
* @return the maximum result size in bytes. See {@link #setMaxResultSize(long)}
*/
public long getMaxResultSize() {
return maxResultSize;
}
/**
* Compile the details beyond the scope of getFingerprint (row, columns,
* timestamps, etc.) into a Map along with the fingerprinted information.

View File

@ -1382,7 +1382,7 @@ public final class ProtobufUtil {
return (cells == null || cells.isEmpty())
? (proto.getStale() ? EMPTY_RESULT_STALE : EMPTY_RESULT)
: Result.create(cells, null, proto.getStale(), proto.getPartial());
: Result.create(cells, null, proto.getStale());
}

View File

@ -592,9 +592,6 @@ public final class ProtobufUtil {
if (proto.hasLoadColumnFamiliesOnDemand()) {
get.setLoadColumnFamiliesOnDemand(proto.getLoadColumnFamiliesOnDemand());
}
if (proto.hasMaxResultSize()) {
get.setMaxResultSize(proto.getMaxResultSize());
}
return get;
}
@ -1259,9 +1256,6 @@ public final class ProtobufUtil {
if (loadColumnFamiliesOnDemand != null) {
builder.setLoadColumnFamiliesOnDemand(loadColumnFamiliesOnDemand);
}
if (get.getMaxResultSize() > 0) {
builder.setMaxResultSize(get.getMaxResultSize());
}
return builder.build();
}
@ -1463,7 +1457,6 @@ public final class ProtobufUtil {
ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder();
builder.setAssociatedCellCount(size);
builder.setStale(result.isStale());
builder.setPartial(result.mayHaveMoreCellsInRow());
return builder.build();
}
@ -1554,7 +1547,7 @@ public final class ProtobufUtil {
return (cells == null || cells.isEmpty())
? (proto.getStale() ? EMPTY_RESULT_STALE : EMPTY_RESULT)
: Result.create(cells, null, proto.getStale(), proto.getPartial());
: Result.create(cells, null, proto.getStale());
}

View File

@ -90,8 +90,6 @@ message Get {
optional Consistency consistency = 12 [default = STRONG];
repeated ColumnFamilyTimeRange cf_time_range = 13;
optional bool load_column_families_on_demand = 14; /* DO NOT add defaults to load_column_families_on_demand. */
optional uint64 max_result_size = 15;
}
message Result {

View File

@ -146,7 +146,6 @@ import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
import org.apache.hadoop.hbase.regionserver.throttle.StoreHotnessProtector;
@ -3865,7 +3864,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
Result result;
if (returnResults) {
// convert duplicate increment/append to get
result = region.get(toGet(mutation), false, nonceGroup, nonce);
List<Cell> results = region.get(toGet(mutation), false, nonceGroup, nonce);
result = Result.create(results);
} else {
result = Result.EMPTY_RESULT;
}
@ -7497,7 +7497,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
@Override
public Result get(final Get get) throws IOException {
prepareGet(get);
return get(get, true, HConstants.NO_NONCE, HConstants.NO_NONCE);
List<Cell> results = get(get, true);
boolean stale = this.getRegionInfo().getReplicaId() != 0;
return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
}
void prepareGet(final Get get) throws IOException {
@ -7516,31 +7518,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
@Override
public List<Cell> get(Get get, boolean withCoprocessor) throws IOException {
return getInternal(get, null, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE);
return get(get, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE);
}
private Result get(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
throws IOException {
ScannerContext scannerContext = get.getMaxResultSize() > 0
? ScannerContext.newBuilder()
.setSizeLimit(LimitScope.BETWEEN_CELLS, get.getMaxResultSize(), get.getMaxResultSize())
.build()
: null;
List<Cell> result = getInternal(get, scannerContext, withCoprocessor, nonceGroup, nonce);
boolean stale = this.getRegionInfo().getReplicaId() != 0;
boolean mayHaveMoreCellsInRow =
scannerContext != null && scannerContext.mayHaveMoreCellsInRow();
return Result.create(
result,
get.isCheckExistenceOnly() ? !result.isEmpty() : null,
stale,
mayHaveMoreCellsInRow);
}
private List<Cell> getInternal(Get get, ScannerContext scannerContext, boolean withCoprocessor,
long nonceGroup, long nonce) throws IOException {
private List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
throws IOException {
List<Cell> results = new ArrayList<>();
long before = EnvironmentEdgeManager.currentTime();
@ -7557,7 +7539,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
try (RegionScanner scanner = getScanner(scan, null, nonceGroup, nonce)) {
List<Cell> tmp = new ArrayList<>();
scanner.next(tmp, scannerContext);
scanner.next(tmp);
// Copy EC to heap, then close the scanner.
// This can be an EXPENSIVE call. It may make an extra copy from offheap to onheap buffers.
// See more details in HBASE-26036.

View File

@ -2668,15 +2668,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
if (scan.getLoadColumnFamiliesOnDemandValue() == null) {
scan.setLoadColumnFamiliesOnDemand(region.isLoadingCfsOnDemandDefault());
}
ScannerContext scannerContext = ScannerContext.newBuilder()
.setSizeLimit(LimitScope.BETWEEN_CELLS, get.getMaxResultSize(), get.getMaxResultSize())
.build();
RegionScannerImpl scanner = null;
try {
scanner = region.getScanner(scan);
scanner.next(results, scannerContext);
scanner.next(results);
} finally {
if (scanner != null) {
if (closeCallBack == null) {
@ -2701,8 +2696,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
region.metricsUpdateForGet(results, before);
return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale,
scannerContext.mayHaveMoreCellsInRow());
return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
}
private void checkBatchSizeAndLogLargeSize(MultiRequest request) throws ServiceException {

View File

@ -31,26 +31,18 @@ import java.util.List;
import java.util.Set;
import org.apache.hadoop.hbase.client.ClientScanner;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
import org.apache.hadoop.hbase.filter.FamilyFilter;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.FilterListWithAND;
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
import org.apache.hadoop.hbase.filter.FirstKeyValueMatchingQualifiersFilter;
import org.apache.hadoop.hbase.filter.PageFilter;
import org.apache.hadoop.hbase.filter.RandomRowFilter;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
@ -144,46 +136,6 @@ public class TestPartialResultsFromClientSide {
TEST_UTIL.shutdownMiniCluster();
}
@Test
public void testGetPartialResults() throws Exception {
byte[] row = ROWS[0];
Result result;
int cf = 0;
int qf = 0;
int total = 0;
do {
// this will ensure we always return only 1 result
Get get = new Get(row)
.setMaxResultSize(1);
// we want to page through the entire row, this will ensure we always get the next
if (total > 0) {
get.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL,
new ColumnRangeFilter(QUALIFIERS[qf], true, null, false),
new FamilyFilter(CompareOperator.GREATER_OR_EQUAL, new BinaryComparator(FAMILIES[cf]))));
}
// all values are the same, but there should be a value
result = TABLE.get(get);
assertTrue(String.format("Value for family %s (# %s) and qualifier %s (# %s)",
Bytes.toStringBinary(FAMILIES[cf]), cf, Bytes.toStringBinary(QUALIFIERS[qf]), qf),
Bytes.equals(VALUE, result.getValue(FAMILIES[cf], QUALIFIERS[qf])));
total++;
if (++qf >= NUM_QUALIFIERS) {
cf++;
qf = 0;
}
} while (result.mayHaveMoreCellsInRow());
// ensure we iterated all cells in row
assertEquals(NUM_COLS, total);
assertEquals(NUM_FAMILIES, cf);
assertEquals(0, qf);
}
/**
* Ensure that the expected key values appear in a result returned from a scanner that is
* combining partial results into complete results

View File

@ -7861,89 +7861,4 @@ public class TestHRegion {
assertFalse("Region lock holder should not have been interrupted", holderInterrupted.get());
}
@Test
public void testOversizedGetsReturnPartialResult() throws IOException {
HRegion region = initHRegion(tableName, name.getMethodName(), CONF, fam1);
Put p = new Put(row)
.addColumn(fam1, qual1, value1)
.addColumn(fam1, qual2, value2);
region.put(p);
Get get = new Get(row)
.addColumn(fam1, qual1)
.addColumn(fam1, qual2)
.setMaxResultSize(1); // 0 doesn't count as a limit, according to HBase
Result r = region.get(get);
assertTrue("Expected partial result, but result was not marked as partial", r.mayHaveMoreCellsInRow());
}
@Test
public void testGetsWithoutResultSizeLimitAreNotPartial() throws IOException {
HRegion region = initHRegion(tableName, name.getMethodName(), CONF, fam1);
Put p = new Put(row)
.addColumn(fam1, qual1, value1)
.addColumn(fam1, qual2, value2);
region.put(p);
Get get = new Get(row)
.addColumn(fam1, qual1)
.addColumn(fam1, qual2);
Result r = region.get(get);
assertFalse("Expected full result, but it was marked as partial", r.mayHaveMoreCellsInRow());
assertTrue(Bytes.equals(value1, r.getValue(fam1, qual1)));
assertTrue(Bytes.equals(value2, r.getValue(fam1, qual2)));
}
@Test
public void testGetsWithinResultSizeLimitAreNotPartial() throws IOException {
HRegion region = initHRegion(tableName, name.getMethodName(), CONF, fam1);
Put p = new Put(row)
.addColumn(fam1, qual1, value1)
.addColumn(fam1, qual2, value2);
region.put(p);
Get get = new Get(row)
.addColumn(fam1, qual1)
.addColumn(fam1, qual2)
.setMaxResultSize(Long.MAX_VALUE);
Result r = region.get(get);
assertFalse("Expected full result, but it was marked as partial", r.mayHaveMoreCellsInRow());
assertTrue(Bytes.equals(value1, r.getValue(fam1, qual1)));
assertTrue(Bytes.equals(value2, r.getValue(fam1, qual2)));
}
@Test
public void testGetsWithResultSizeLimitReturnPartialResults() throws IOException {
HRegion region = initHRegion(tableName, name.getMethodName(), CONF, fam1);
Put p = new Put(row)
.addColumn(fam1, qual1, value1)
.addColumn(fam1, qual2, value2);
region.put(p);
Get get = new Get(row)
.addColumn(fam1, qual1)
.addColumn(fam1, qual2)
.setMaxResultSize(10);
Result r = region.get(get);
assertTrue("Expected partial result, but it was marked as complete", r.mayHaveMoreCellsInRow());
assertTrue(Bytes.equals(value1, r.getValue(fam1, qual1)));
assertEquals("Got more results than expected", 1, r.size());
}
}