Uploaded image for project: 'Phoenix'
  1. Phoenix
  2. PHOENIX-5970

ViewUtil.dropChildViews may cause HConnection leak which may cause ITtests hange

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 4.15.0
    • 5.1.0, 4.16.0
    • None
    • None

    Description

      After PHOENIX-5962, the ITtests still hange , I noticed the RegionServer is blocked as follows : (by 4.x branch and HBase1.4.10) :

      Thread 31512: (state = BLOCKED)
       - java.lang.Thread.sleep(long) @bci=0 (Compiled frame; information may be imprecise)
       - java.lang.Thread.sleep(long, int) @bci=57, line=340 (Compiled frame)
       - java.util.concurrent.TimeUnit.sleep(long) @bci=23, line=386 (Compiled frame)
       - org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry() @bci=110, line=158 (Compiled frame)
       - org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.getData(java.lang.String, org.apache.zookeeper.Watcher, org.apache.zookeeper.data.Stat) @bci=180, line=367 (Compiled frame)
       - org.apache.hadoop.hbase.zookeeper.ZKUtil.getData(org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher, java.lang.String) @bci=11, line=629 (Compiled frame)
       - org.apache.hadoop.hbase.zookeeper.MetaTableLocator.getMetaRegionState(org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher, int) @bci=23, line=487 (Compiled frame)
       - org.apache.hadoop.hbase.zookeeper.MetaTableLocator.getMetaRegionLocation(org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher, int) @bci=6, line=168 (Compiled frame)
       - org.apache.hadoop.hbase.zookeeper.MetaTableLocator.blockUntilAvailable(org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher, int, long) @bci=63, line=607 (Compiled frame)
       - org.apache.hadoop.hbase.zookeeper.MetaTableLocator.blockUntilAvailable(org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher, long) @bci=9, line=588 (Compiled frame)
       - org.apache.hadoop.hbase.zookeeper.MetaTableLocator.blockUntilAvailable(org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher, long, org.apache.hadoop.conf.Configuration) @bci=34, line=561 (Compiled frame)
       - org.apache.hadoop.hbase.client.ZooKeeperRegistry.getMetaRegionLocation() @bci=94, line=61 (Compiled frame)
       - org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.locateMeta(org.apache.hadoop.hbase.TableName, boolean, int) @bci=205, line=1266 (Compiled frame)
       - org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.locateRegion(org.apache.hadoop.hbase.TableName, byte[], boolean, boolean, int) @bci=119, line=1227 (Compiled frame)
       - org.apache.hadoop.hbase.client.CoprocessorHConnection.locateRegion(org.apache.hadoop.hbase.TableName, byte[], boolean, boolean, int) @bci=13, line=41 (Compiled frame)
       - org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.relocateRegion(org.apache.hadoop.hbase.TableName, byte[], int) @bci=93, line=1201 (Interpreted frame)
       - org.apache.hadoop.hbase.client.CoprocessorHConnection.relocateRegion(org.apache.hadoop.hbase.TableName, byte[], int) @bci=9, line=41 (Interpreted frame)
       - org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.locateRegionInMeta(org.apache.hadoop.hbase.TableName, byte[], boolean, boolean, int) @bci=1668, line=1419 (Compiled frame)
       - org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.locateRegion(org.apache.hadoop.hbase.TableName, byte[], boolean, boolean, int) @bci=138, line=1230 (Compiled frame)
       - org.apache.hadoop.hbase.client.CoprocessorHConnection.locateRegion(org.apache.hadoop.hbase.TableName, byte[], boolean, boolean, int) @bci=13, line=41 (Compiled frame)
       - org.apache.hadoop.hbase.client.RpcRetryingCallerWithReadReplicas.getRegionLocations(boolean, int, org.apache.hadoop.hbase.client.ClusterConnection, org.apache.hadoop.hbase.TableName, byte[]) @bci=22, line=356 (Compiled frame)
       - org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(int) @bci=164, line=153 (Compiled frame)
       - org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(int) @bci=6, line=58 (Compiled frame)
       - org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(org.apache.hadoop.hbase.client.RetryingCallable, int) @bci=30, line=219 (Compiled frame)
       - org.apache.hadoop.hbase.client.ClientScanner.call(org.apache.hadoop.hbase.client.ScannerCallableWithReplicas, org.apache.hadoop.hbase.client.RpcRetryingCaller, int, boolean) @bci=34, line=275 (Compiled frame)
       - org.apache.hadoop.hbase.client.ClientScanner.loadCache() @bci=109, line=436 (Compiled frame)
       - org.apache.hadoop.hbase.client.ClientScanner.next() @bci=68, line=310 (Compiled frame)
       - org.apache.phoenix.util.ViewUtil.findRelatedViews(org.apache.hadoop.hbase.client.Table, byte[], byte[], byte[], org.apache.phoenix.schema.PTable$LinkType, long) @bci=281, line=126 (Compiled frame)
       - org.apache.phoenix.util.ViewUtil.dropChildViews(org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment, byte[], byte[], byte[], byte[]) @bci=62, line=198 (Compiled frame)
       - org.apache.phoenix.coprocessor.MetaDataEndpointImpl.createTable(com.google.protobuf.RpcController, org.apache.phoenix.coprocessor.generated.MetaDataProtos$CreateTableRequest, com.google.protobuf.RpcCallback) @bci=579, line=1763 (Interpreted frame)
       - org.apache.phoenix.coprocessor.generated.MetaDataProtos$MetaDataService.callMethod(com.google.protobuf.Descriptors$MethodDescriptor, com.google.protobuf.RpcController, com.google.protobuf.Message, com.google.protobuf.RpcCallback) @bci=238, line=17218 (Compiled frame)
       - org.apache.hadoop.hbase.regionserver.HRegion.execService(com.google.protobuf.RpcController, org.apache.hadoop.hbase.protobuf.generated.ClientProtos$CoprocessorServiceCall) @bci=358, line=8485 (Compiled frame)
       - org.apache.hadoop.hbase.regionserver.RSRpcServices.execServiceOnRegion(org.apache.hadoop.hbase.regionserver.Region, org.apache.hadoop.hbase.protobuf.generated.ClientProtos$CoprocessorServiceCall) @bci=23, line=2216 (Compiled frame)
       - org.apache.hadoop.hbase.regionserver.RSRpcServices.execService(com.google.protobuf.RpcController, org.apache.hadoop.hbase.protobuf.generated.ClientProtos$CoprocessorServiceRequest) @bci=50, line=2198 (Compiled frame)
       - org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(com.google.protobuf.Descriptors$MethodDescriptor, com.google.protobuf.RpcController, com.google.protobuf.Message) @bci=171, line=36617 (Compiled frame)
       - org.apache.hadoop.hbase.ipc.RpcServer.call(com.google.protobuf.BlockingService, com.google.protobuf.Descriptors$MethodDescriptor, com.google.protobuf.Message, org.apache.hadoop.hbase.CellScanner, long, org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler, long, int) @bci=98, line=2380 (Compiled frame)
       - org.apache.hadoop.hbase.ipc.CallRunner.run() @bci=656, line=124 (Compiled frame)
       - org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(org.apache.hadoop.hbase.ipc.CallRunner) @bci=37, line=297 (Compiled frame)
       - org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run() @bci=29, line=277 (Compiled frame)
      

      It is caused by the ViewUtil.dropChildViews method, just as following code, hTable is created by line189, but after hTable is used by ViewUtil.findRelatedViews in line 198 , it is not closed, so
      the HConnection and corresponding Zookeeper Connection is leaked, which causes the IT tests hange :

      184   public static void dropChildViews(RegionCoprocessorEnvironment env, byte[] tenantIdBytes,
      185           byte[] schemaName, byte[] tableName, byte[] sysCatOrSysChildLink)
      186            throws IOException, SQLException {
      187        Table hTable = null;
      188        try {
      189            hTable = ServerUtil.getHTableForCoprocessorScan(env, sysCatOrSysChildLink);
      190        }
      191       catch (Exception e){
      192        }
      193       // if the SYSTEM.CATALOG or SYSTEM.CHILD_LINK doesn't exist just return
      194        if (hTable==null) {
      195            return;
      196        }
      197
      198        TableViewFinderResult childViewsResult = findRelatedViews(hTable, tenantIdBytes, schemaName, tableName,
      199                LinkType.CHILD_TABLE, HConstants.LATEST_TIMESTAMP);
      200
      201        for (TableInfo viewInfo : childViewsResult.getLinks()) {
      

      Attachments

        Activity

          People

            comnetwork chenglei
            comnetwork chenglei
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: