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.HBaseTestingUtil.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.Map;
028import java.util.concurrent.BlockingQueue;
029import java.util.concurrent.LinkedBlockingQueue;
030import org.apache.hadoop.conf.Configuration;
031import org.apache.hadoop.hbase.HBaseClassTestRule;
032import org.apache.hadoop.hbase.HBaseTestingUtil;
033import org.apache.hadoop.hbase.HConstants;
034import org.apache.hadoop.hbase.TableName;
035import org.apache.hadoop.hbase.client.Connection;
036import org.apache.hadoop.hbase.client.ConnectionFactory;
037import org.apache.hadoop.hbase.client.Get;
038import org.apache.hadoop.hbase.client.MetricsConnection;
039import org.apache.hadoop.hbase.client.RetriesExhaustedException;
040import org.apache.hadoop.hbase.client.Table;
041import org.apache.hadoop.hbase.testclassification.MediumTests;
042import org.apache.hadoop.hbase.util.Bytes;
043import org.junit.AfterClass;
044import org.junit.BeforeClass;
045import org.junit.ClassRule;
046import org.junit.Rule;
047import org.junit.Test;
048import org.junit.experimental.categories.Category;
049import org.junit.rules.TestName;
050import org.slf4j.Logger;
051import org.slf4j.LoggerFactory;
052
053@Category(MediumTests.class)
054public class TestRpcClientLeaks {
055
056  @ClassRule
057  public static final HBaseClassTestRule CLASS_RULE =
058    HBaseClassTestRule.forClass(TestRpcClientLeaks.class);
059
060  @Rule
061  public TestName name = new TestName();
062
063  private static BlockingQueue<Socket> SAVED_SOCKETS = new LinkedBlockingQueue<>();
064
065  public static class MyRpcClientImpl extends BlockingRpcClient {
066
067    // Exceptions thrown only when this is set to false.
068    private static boolean throwException = false;
069
070    public MyRpcClientImpl(Configuration conf) {
071      super(conf);
072    }
073
074    public MyRpcClientImpl(Configuration conf, String clusterId, SocketAddress address,
075      MetricsConnection metrics, Map<String, byte[]> connectionAttributes) {
076      super(conf, clusterId, address, metrics, connectionAttributes);
077    }
078
079    @Override
080    protected BlockingRpcConnection createConnection(ConnectionId remoteId) throws IOException {
081      return new BlockingRpcConnection(this, remoteId) {
082        @Override
083        protected synchronized void setupConnection() throws IOException {
084          super.setupConnection();
085          if (throwException) {
086            SAVED_SOCKETS.add(socket);
087            throw new IOException(
088              "Sample exception for verifying socket closure in case of exceptions.");
089          }
090        }
091      };
092    }
093
094    public static void enableThrowExceptions() {
095      throwException = true;
096    }
097  }
098
099  private static final HBaseTestingUtil UTIL = new HBaseTestingUtil();
100
101  @BeforeClass
102  public static void setup() throws Exception {
103    UTIL.startMiniCluster();
104  }
105
106  @AfterClass
107  public static void teardown() throws Exception {
108    UTIL.shutdownMiniCluster();
109  }
110
111  public static final Logger LOG = LoggerFactory.getLogger(TestRpcClientLeaks.class);
112
113  @Test
114  public void testSocketClosed() throws IOException, InterruptedException {
115    TableName tableName = TableName.valueOf(name.getMethodName());
116    UTIL.createTable(tableName, fam1).close();
117
118    Configuration conf = new Configuration(UTIL.getConfiguration());
119    conf.set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY, MyRpcClientImpl.class.getName());
120    conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
121    try (Connection connection = ConnectionFactory.createConnection(conf);
122      Table table = connection.getTable(TableName.valueOf(name.getMethodName()))) {
123      MyRpcClientImpl.enableThrowExceptions();
124      table.get(new Get(Bytes.toBytes("asd")));
125      fail("Should fail because the injected error");
126    } catch (RetriesExhaustedException e) {
127      // expected
128    }
129    for (Socket socket : SAVED_SOCKETS) {
130      assertTrue("Socket " + socket + " is not closed", socket.isClosed());
131    }
132  }
133}