HBASE-12589 Forward-port fix for TestFromClientSideWithCoprocessor.testMaxKeyValueSize
This commit is contained in:
parent
57b38339db
commit
595d2a846c
|
@ -48,7 +48,6 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
import org.apache.commons.lang.ArrayUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.commons.logging.impl.Log4JLogger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
|
@ -85,8 +84,6 @@ import org.apache.hadoop.hbase.filter.WhileMatchFilter;
|
|||
import org.apache.hadoop.hbase.io.hfile.BlockCache;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.ipc.RpcClient;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
|
||||
|
@ -103,7 +100,6 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
|
@ -113,7 +109,7 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Run tests that use the HBase clients; {@link HTable} and {@link HTablePool}.
|
||||
* Run tests that use the HBase clients; {@link HTable}.
|
||||
* Sets up the HBase mini cluster once at start and runs through all client tests.
|
||||
* Each creates a table named for the method and does its stuff against that.
|
||||
*/
|
||||
|
@ -704,12 +700,17 @@ public class TestFromClientSide {
|
|||
put.add(FAMILY, QUALIFIER, value);
|
||||
ht.put(put);
|
||||
try {
|
||||
conf.setInt("hbase.client.keyvalue.maxsize", 2 * 1024 * 1024);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.client.keyvalue.maxsize", 2 * 1024 * 1024);
|
||||
TABLE = Bytes.toBytes("testMaxKeyValueSize2");
|
||||
ht = TEST_UTIL.createTable(TABLE, FAMILY);
|
||||
put = new Put(ROW);
|
||||
put.add(FAMILY, QUALIFIER, value);
|
||||
ht.put(put);
|
||||
// Create new table so we pick up the change in Configuration.
|
||||
try (Connection connection =
|
||||
ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
|
||||
try (Table t = connection.getTable(TableName.valueOf(FAMILY))) {
|
||||
put = new Put(ROW);
|
||||
put.add(FAMILY, QUALIFIER, value);
|
||||
t.put(put);
|
||||
}
|
||||
}
|
||||
fail("Inserting a too large KeyValue worked, should throw exception");
|
||||
} catch(Exception e) {}
|
||||
conf.set("hbase.client.keyvalue.maxsize", oldMaxSize);
|
||||
|
|
Loading…
Reference in New Issue