Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

skip big rows #75

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,7 @@
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.RowTooBigException;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
Expand Down Expand Up @@ -6611,7 +6612,16 @@ private boolean nextInternal(List<Cell> results, ScannerContext scannerContext)
}

// Ok, we are good, let's try to get some results from the main heap.
populateResult(results, this.storeHeap, scannerContext, current);
try {
populateResult(results, this.storeHeap, scannerContext, current);
} catch(RowTooBigException e) {
this.storeHeap.reseek(PrivateCellUtil.createLastOnRow(current));
results.clear();
scannerContext.clearProgress();
continue;
}


if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
if (hasFilterRow) {
throw new IncompatibleFilterException(
Expand Down Expand Up @@ -6684,7 +6694,14 @@ private boolean nextInternal(List<Cell> results, ScannerContext scannerContext)
}
} else {
// Populating from the joined heap was stopped by limits, populate some more.
populateFromJoinedHeap(results, scannerContext);
try {
populateFromJoinedHeap(results, scannerContext);
} catch(RowTooBigException e) {
joinedContinuationRow = null;
results.clear();
continue;
}

if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
return true;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,108 @@
package org.apache.hadoop.hbase.regionserver;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Put;
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.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestName;

import java.util.ArrayList;
import java.util.List;
import java.util.Random;

public class TestSkipBigRowScanner {
private static final Log LOG = LogFactory.getLog(TestJoinedScanners.class);

private static final byte[] cf_name = Bytes.toBytes("a");
private static final byte[] col_name = Bytes.toBytes("a");

private static int valueWidth = 2 * 1024 * 1024;

@Rule
public TestName name = new TestName();

@Test
public void testJoinedScanners() throws Exception {
String dataNodeHosts[] = new String[] { "host1", "host2", "host3" };
int regionServersCount = 3;

HBaseTestingUtility htu = new HBaseTestingUtility();

final int DEFAULT_BLOCK_SIZE = 1024*1024;
htu.getConfiguration().setLong("dfs.blocksize", DEFAULT_BLOCK_SIZE);
htu.getConfiguration().setInt("dfs.replication", 1);
htu.getConfiguration().setLong("hbase.hregion.max.filesize", 322122547200L);
htu.getConfiguration().setLong(HConstants.TABLE_MAX_ROWSIZE_KEY,
1024 * 1024L);

MiniHBaseCluster cluster = null;

try {
cluster = htu.startMiniCluster(1, regionServersCount, dataNodeHosts);

final TableName tableName = TableName.valueOf(name.getMethodName());

ColumnFamilyDescriptor columnFamilyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(cf_name).build();
TableDescriptorBuilder tableDescriptorBuilder = TableDescriptorBuilder.newBuilder(tableName);
tableDescriptorBuilder.setColumnFamily(columnFamilyDescriptor);
TableDescriptor tableDescriptor = tableDescriptorBuilder.build();
htu.getAdmin().createTable(tableDescriptor);
Table ht = htu.getConnection().getTable(tableName);

byte [] val_large = new byte[valueWidth];

List<Put> puts = new ArrayList<>();
Put put = new Put(Bytes.toBytes("0"));
put.addColumn(cf_name, col_name, val_large);
puts.add(put);

put = new Put(Bytes.toBytes("1"));
put.addColumn(cf_name, col_name, Bytes.toBytes("small"));
puts.add(put);

put = new Put(Bytes.toBytes("2"));
put.addColumn(cf_name, col_name, val_large);
puts.add(put);

ht.put(puts);
puts.clear();

Scan scan = new Scan();
scan.addColumn(cf_name, col_name);
ResultScanner result_scanner = ht.getScanner(scan);
Result res;
long rows_count = 0;
//Only 1 row
while ((res = result_scanner.next()) != null) {
Assert.assertEquals("1",Bytes.toString(res.getRow()));
rows_count++;
}

Assert.assertEquals(1, rows_count);
result_scanner.close();
ht.close();
} finally {
if (cluster != null) {
htu.shutdownMiniCluster();
}
}
}
}