001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.ipc;
019
020import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
021import static org.junit.Assert.assertTrue;
022import static org.junit.Assert.fail;
023
024import java.io.IOException;
025import java.net.Socket;
026import java.net.SocketAddress;
027import java.util.concurrent.BlockingQueue;
028import java.util.concurrent.LinkedBlockingQueue;
029import org.apache.hadoop.conf.Configuration;
030import org.apache.hadoop.hbase.HBaseClassTestRule;
031import org.apache.hadoop.hbase.HBaseTestingUtility;
032import org.apache.hadoop.hbase.HConstants;
033import org.apache.hadoop.hbase.TableName;
034import org.apache.hadoop.hbase.client.Connection;
035import org.apache.hadoop.hbase.client.ConnectionFactory;
036import org.apache.hadoop.hbase.client.Get;
037import org.apache.hadoop.hbase.client.MetricsConnection;
038import org.apache.hadoop.hbase.client.RetriesExhaustedException;
039import org.apache.hadoop.hbase.client.Table;
040import org.apache.hadoop.hbase.testclassification.MediumTests;
041import org.apache.hadoop.hbase.util.Bytes;
042import org.junit.AfterClass;
043import org.junit.BeforeClass;
044import org.junit.ClassRule;
045import org.junit.Rule;
046import org.junit.Test;
047import org.junit.experimental.categories.Category;
048import org.junit.rules.TestName;
049import org.slf4j.Logger;
050import org.slf4j.LoggerFactory;
051
052@Category(MediumTests.class)
053public class TestRpcClientLeaks {
054
055  @ClassRule
056  public static final HBaseClassTestRule CLASS_RULE =
057    HBaseClassTestRule.forClass(TestRpcClientLeaks.class);
058
059  @Rule
060  public TestName name = new TestName();
061
062  private static BlockingQueue<Socket> SAVED_SOCKETS = new LinkedBlockingQueue<>();
063
064  public static class MyRpcClientImpl extends BlockingRpcClient {
065
066    // Exceptions thrown only when this is set to false.
067    private static boolean throwException = false;
068
069    public MyRpcClientImpl(Configuration conf) {
070      super(conf);
071    }
072
073    public MyRpcClientImpl(Configuration conf, String clusterId, SocketAddress address,
074      MetricsConnection metrics) {
075      super(conf, clusterId, address, metrics);
076    }
077
078    @Override
079    protected BlockingRpcConnection createConnection(ConnectionId remoteId) throws IOException {
080      return new BlockingRpcConnection(this, remoteId) {
081        @Override
082        protected synchronized void setupConnection() throws IOException {
083          super.setupConnection();
084          if (throwException) {
085            SAVED_SOCKETS.add(socket);
086            throw new IOException(
087              "Sample exception for verifying socket closure in case of exceptions.");
088          }
089        }
090      };
091    }
092
093    public static void enableThrowExceptions() {
094      throwException = true;
095    }
096  }
097
098  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
099
100  @BeforeClass
101  public static void setup() throws Exception {
102    UTIL.startMiniCluster();
103  }
104
105  @AfterClass
106  public static void teardown() throws Exception {
107    UTIL.shutdownMiniCluster();
108  }
109
110  public static final Logger LOG = LoggerFactory.getLogger(TestRpcClientLeaks.class);
111
112  @Test
113  public void testSocketClosed() throws IOException, InterruptedException {
114    TableName tableName = TableName.valueOf(name.getMethodName());
115    UTIL.createTable(tableName, fam1).close();
116
117    Configuration conf = new Configuration(UTIL.getConfiguration());
118    conf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, MyRpcClientImpl.class.getName());
119    conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
120    try (Connection connection = ConnectionFactory.createConnection(conf);
121      Table table = connection.getTable(TableName.valueOf(name.getMethodName()))) {
122      MyRpcClientImpl.enableThrowExceptions();
123      table.get(new Get(Bytes.toBytes("asd")));
124      fail("Should fail because the injected error");
125    } catch (RetriesExhaustedException e) {
126      // expected
127    }
128    for (Socket socket : SAVED_SOCKETS) {
129      assertTrue("Socket " + socket + " is not closed", socket.isClosed());
130    }
131  }
132}