From e30a66b9443618f04ad8cb0aee96ac77b174338b Mon Sep 17 00:00:00 2001 From: zhangduo Date: Thu, 1 Sep 2016 11:48:06 +0800 Subject: [PATCH] HBASE-16526 Addendum add missing stuffs --- .../hbase/ipc/TestCellBlockBuilder.java | 28 +++++++++---------- .../apache/hadoop/hbase/ipc/TestIPCUtil.java | 4 +++ .../hbase/ipc/IntegrationTestRpcClient.java | 8 +----- 3 files changed, 19 insertions(+), 21 deletions(-) diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java index b780b9530fb..60ef3574f84 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestCellBlockBuilder.java @@ -67,19 +67,19 @@ public class TestCellBlockBuilder { doBuildCellBlockUndoCellBlock(this.builder, new KeyValueCodec(), new GzipCodec()); } - static void doBuildCellBlockUndoCellBlock(final CellBlockBuilder util, final Codec codec, + static void doBuildCellBlockUndoCellBlock(final CellBlockBuilder builder, final Codec codec, final CompressionCodec compressor) throws IOException { - doBuildCellBlockUndoCellBlock(util, codec, compressor, 10, 1, false); + doBuildCellBlockUndoCellBlock(builder, codec, compressor, 10, 1, false); } - static void doBuildCellBlockUndoCellBlock(final CellBlockBuilder util, final Codec codec, + static void doBuildCellBlockUndoCellBlock(final CellBlockBuilder builder, final Codec codec, final CompressionCodec compressor, final int count, final int size, final boolean sized) throws IOException { Cell[] cells = getCells(count, size); CellScanner cellScanner = sized ? getSizedCellScanner(cells) : CellUtil.createCellScanner(Arrays.asList(cells).iterator()); - ByteBuffer bb = util.buildCellBlock(codec, compressor, cellScanner); - cellScanner = util.createCellScannerReusingBuffers(codec, compressor, bb); + ByteBuffer bb = builder.buildCellBlock(codec, compressor, cellScanner); + cellScanner = builder.createCellScannerReusingBuffers(codec, compressor, bb); int i = 0; while (cellScanner.advance()) { i++; @@ -143,13 +143,13 @@ public class TestCellBlockBuilder { System.exit(errCode); } - private static void timerTests(final CellBlockBuilder util, final int count, final int size, + private static void timerTests(final CellBlockBuilder builder, final int count, final int size, final Codec codec, final CompressionCodec compressor) throws IOException { final int cycles = 1000; StopWatch timer = new StopWatch(); timer.start(); for (int i = 0; i < cycles; i++) { - timerTest(util, timer, count, size, codec, compressor, false); + timerTest(builder, timer, count, size, codec, compressor, false); } timer.stop(); LOG.info("Codec=" + codec + ", compression=" + compressor + ", sized=" + false + ", count=" @@ -157,17 +157,17 @@ public class TestCellBlockBuilder { timer.reset(); timer.start(); for (int i = 0; i < cycles; i++) { - timerTest(util, timer, count, size, codec, compressor, true); + timerTest(builder, timer, count, size, codec, compressor, true); } timer.stop(); LOG.info("Codec=" + codec + ", compression=" + compressor + ", sized=" + true + ", count=" + count + ", size=" + size + ", + took=" + timer.getTime() + "ms"); } - private static void timerTest(final CellBlockBuilder util, final StopWatch timer, final int count, + private static void timerTest(final CellBlockBuilder builder, final StopWatch timer, final int count, final int size, final Codec codec, final CompressionCodec compressor, final boolean sized) throws IOException { - doBuildCellBlockUndoCellBlock(util, codec, compressor, count, size, sized); + doBuildCellBlockUndoCellBlock(builder, codec, compressor, count, size, sized); } /** @@ -187,10 +187,10 @@ public class TestCellBlockBuilder { usage(1); } } - CellBlockBuilder util = new CellBlockBuilder(HBaseConfiguration.create()); + CellBlockBuilder buildr = new CellBlockBuilder(HBaseConfiguration.create()); ((Log4JLogger) CellBlockBuilder.LOG).getLogger().setLevel(Level.ALL); - timerTests(util, count, size, new KeyValueCodec(), null); - timerTests(util, count, size, new KeyValueCodec(), new DefaultCodec()); - timerTests(util, count, size, new KeyValueCodec(), new GzipCodec()); + timerTests(buildr, count, size, new KeyValueCodec(), null); + timerTests(buildr, count, size, new KeyValueCodec(), new DefaultCodec()); + timerTests(buildr, count, size, new KeyValueCodec(), new GzipCodec()); } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java index ef534c036ad..7c4ac029acd 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java @@ -25,8 +25,12 @@ import java.net.InetSocketAddress; import java.net.SocketTimeoutException; import org.apache.hadoop.hbase.exceptions.ConnectionClosingException; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Test; +import org.junit.experimental.categories.Category; +@Category({ ClientTests.class, SmallTests.class }) public class TestIPCUtil { @Test diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java index 65d8d8badda..28c19ad43f9 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/ipc/IntegrationTestRpcClient.java @@ -53,7 +53,6 @@ import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProt import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; -import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; @@ -284,13 +283,8 @@ public class IntegrationTestRpcClient { EchoResponseProto ret; TestRpcServer server = cluster.getRandomServer(); try { - User user = User.getCurrent(); - InetSocketAddress address = server.getListenerAddress(); - if (address == null) { - throw new IOException("Listener channel is closed"); - } sending.set(true); - BlockingInterface stub = newBlockingStub(rpcClient, address, user); + BlockingInterface stub = newBlockingStub(rpcClient, server.getListenerAddress()); ret = stub.echo(null, param); } catch (Exception e) { LOG.warn(e);