You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2021/02/08 22:40:07 UTC

[hbase] branch branch-2.4 updated (87e516d -> de9bbd6)

This is an automated email from the ASF dual-hosted git repository.

stack pushed a change to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/hbase.git.


    from 87e516d  HBASE-25553 It is better for ReplicationTracker.getListOfRegionServers to return ServerName instead of String (#2928)
     new 2989f8e  HBASE-25431 MAX_FILESIZE and MEMSTORE_FLUSHSIZE should not be set negative number (#2803)
     new de9bbd6  HBASE-25542 Add client detail to scan name so when lease expires, we … (#2930)

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../hadoop/hbase/regionserver/RSRpcServices.java   | 173 +++++++++++++--------
 .../hadoop/hbase/util/TableDescriptorChecker.java  |  19 ++-
 .../hbase/regionserver/TestRSRpcServices.java      |  65 ++++++++
 3 files changed, 182 insertions(+), 75 deletions(-)
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSRpcServices.java


[hbase] 01/02: HBASE-25431 MAX_FILESIZE and MEMSTORE_FLUSHSIZE should not be set negative number (#2803)

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

stack pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 2989f8edb0559f01488dd6fd52e27373d35d7d3c
Author: Baiqiang Zhao <zb...@gmail.com>
AuthorDate: Fri Jan 15 14:00:50 2021 +0800

    HBASE-25431 MAX_FILESIZE and MEMSTORE_FLUSHSIZE should not be set negative number (#2803)
    
    
    Signed-off-by: stack <st...@apache.org>
---
 .../hadoop/hbase/util/TableDescriptorChecker.java     | 19 +++++++++++--------
 1 file changed, 11 insertions(+), 8 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/TableDescriptorChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/TableDescriptorChecker.java
index c69d38a..30c07b3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/TableDescriptorChecker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/TableDescriptorChecker.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
@@ -82,10 +83,11 @@ public final class TableDescriptorChecker {
 
     // check max file size
     long maxFileSizeLowerLimit = 2 * 1024 * 1024L; // 2M is the default lower limit
-    long maxFileSize = td.getMaxFileSize();
-    if (maxFileSize < 0) {
-      maxFileSize = conf.getLong(HConstants.HREGION_MAX_FILESIZE, maxFileSizeLowerLimit);
-    }
+    // if not set MAX_FILESIZE in TableDescriptor, and not set HREGION_MAX_FILESIZE in
+    // hbase-site.xml, use maxFileSizeLowerLimit instead to skip this check
+    long maxFileSize = td.getValue(TableDescriptorBuilder.MAX_FILESIZE) == null ?
+      conf.getLong(HConstants.HREGION_MAX_FILESIZE, maxFileSizeLowerLimit) :
+      Long.parseLong(td.getValue(TableDescriptorBuilder.MAX_FILESIZE));
     if (maxFileSize < conf.getLong("hbase.hregion.max.filesize.limit", maxFileSizeLowerLimit)) {
       String message =
           "MAX_FILESIZE for table descriptor or " + "\"hbase.hregion.max.filesize\" (" +
@@ -96,10 +98,11 @@ public final class TableDescriptorChecker {
 
     // check flush size
     long flushSizeLowerLimit = 1024 * 1024L; // 1M is the default lower limit
-    long flushSize = td.getMemStoreFlushSize();
-    if (flushSize < 0) {
-      flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, flushSizeLowerLimit);
-    }
+    // if not set MEMSTORE_FLUSHSIZE in TableDescriptor, and not set HREGION_MEMSTORE_FLUSH_SIZE in
+    // hbase-site.xml, use flushSizeLowerLimit instead to skip this check
+    long flushSize = td.getValue(TableDescriptorBuilder.MEMSTORE_FLUSHSIZE) == null ?
+      conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, flushSizeLowerLimit) :
+      Long.parseLong(td.getValue(TableDescriptorBuilder.MEMSTORE_FLUSHSIZE));
     if (flushSize < conf.getLong("hbase.hregion.memstore.flush.size.limit", flushSizeLowerLimit)) {
       String message = "MEMSTORE_FLUSHSIZE for table descriptor or " +
           "\"hbase.hregion.memstore.flush.size\" (" + flushSize +


[hbase] 02/02: HBASE-25542 Add client detail to scan name so when lease expires, we … (#2930)

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

stack pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit de9bbd69684e3f125c4f85fff88a6b844834a718
Author: Michael Stack <sa...@users.noreply.github.com>
AuthorDate: Mon Feb 8 14:33:00 2021 -0800

    HBASE-25542 Add client detail to scan name so when lease expires, we … (#2930)
    
    * HBASE-25542 Add client detail to scan name so when lease expires, we have clue on who was scanning
    
    When we create a scanner lease, record client ip and port (removed
    unnecessary store of scannerName).
    
    Signed-off-by: Clara Xiong <cl...@gmail.com>
---
 .../hadoop/hbase/regionserver/RSRpcServices.java   | 173 +++++++++++++--------
 .../hbase/regionserver/TestRSRpcServices.java      |  65 ++++++++
 2 files changed, 171 insertions(+), 67 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index d399fe7..c87ec7b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -23,6 +23,7 @@ import java.io.UncheckedIOException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.BindException;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -93,6 +94,7 @@ import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.PriorityFunction;
 import org.apache.hadoop.hbase.ipc.QosPriority;
+import org.apache.hadoop.hbase.ipc.RpcCall;
 import org.apache.hadoop.hbase.ipc.RpcCallContext;
 import org.apache.hadoop.hbase.ipc.RpcCallback;
 import org.apache.hadoop.hbase.ipc.RpcScheduler;
@@ -396,7 +398,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    * An Rpc callback for doing shipped() call on a RegionScanner.
    */
   private class RegionScannerShippedCallBack implements RpcCallback {
-
     private final String scannerName;
     private final Shipper shipper;
     private final Lease lease;
@@ -446,10 +447,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   /**
    * Holder class which holds the RegionScanner, nextCallSeq and RpcCallbacks together.
    */
-  private static final class RegionScannerHolder {
-
+  static final class RegionScannerHolder {
     private final AtomicLong nextCallSeq = new AtomicLong(0);
-    private final String scannerName;
     private final RegionScanner s;
     private final HRegion r;
     private final RpcCallback closeCallBack;
@@ -457,32 +456,39 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     private byte[] rowOfLastPartialResult;
     private boolean needCursor;
     private boolean fullRegionScan;
+    private final String clientIPAndPort;
 
-    public RegionScannerHolder(String scannerName, RegionScanner s, HRegion r,
+    RegionScannerHolder(RegionScanner s, HRegion r,
         RpcCallback closeCallBack, RpcCallback shippedCallback, boolean needCursor,
-        boolean fullRegionScan) {
-      this.scannerName = scannerName;
+        boolean fullRegionScan, String clientIPAndPort) {
       this.s = s;
       this.r = r;
       this.closeCallBack = closeCallBack;
       this.shippedCallback = shippedCallback;
       this.needCursor = needCursor;
       this.fullRegionScan = fullRegionScan;
+      this.clientIPAndPort = clientIPAndPort;
     }
 
-    public long getNextCallSeq() {
+    long getNextCallSeq() {
       return nextCallSeq.get();
     }
 
-    public boolean incNextCallSeq(long currentSeq) {
+    boolean incNextCallSeq(long currentSeq) {
       // Use CAS to prevent multiple scan request running on the same scanner.
       return nextCallSeq.compareAndSet(currentSeq, currentSeq + 1);
     }
+
+    // Should be called only when we need to print lease expired messages otherwise
+    // cache the String once made.
+    @Override
+    public String toString() {
+      return this.clientIPAndPort + ", " + this.r.getRegionInfo().getRegionNameAsString();
+    }
   }
 
   /**
-   * Instantiated as a scanner lease. If the lease times out, the scanner is
-   * closed
+   * Instantiated as a scanner lease. If the lease times out, the scanner is closed
    */
   private class ScannerListener implements LeaseListener {
     private final String scannerName;
@@ -494,31 +500,32 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     @Override
     public void leaseExpired() {
       RegionScannerHolder rsh = scanners.remove(this.scannerName);
-      if (rsh != null) {
-        RegionScanner s = rsh.s;
-        LOG.info("Scanner " + this.scannerName + " lease expired on region "
-          + s.getRegionInfo().getRegionNameAsString());
-        HRegion region = null;
+      if (rsh == null) {
+        LOG.warn("Scanner lease {} expired but no outstanding scanner", this.scannerName);
+        return;
+      }
+      LOG.info("Scanner lease {} expired {}, user={}", this.scannerName, rsh,
+        RpcServer.getRequestUserName().orElse(null));
+      RegionScanner s = rsh.s;
+      HRegion region = null;
+      try {
+        region = regionServer.getRegion(s.getRegionInfo().getRegionName());
+        if (region != null && region.getCoprocessorHost() != null) {
+          region.getCoprocessorHost().preScannerClose(s);
+        }
+      } catch (IOException e) {
+        LOG.error("Closing scanner {} {}, user={}", this.scannerName, rsh, e,
+          RpcServer.getRequestUserName().orElse(null));
+      } finally {
         try {
-          region = regionServer.getRegion(s.getRegionInfo().getRegionName());
+          s.close();
           if (region != null && region.getCoprocessorHost() != null) {
-            region.getCoprocessorHost().preScannerClose(s);
+            region.getCoprocessorHost().postScannerClose(s);
           }
         } catch (IOException e) {
-          LOG.error("Closing scanner for " + s.getRegionInfo().getRegionNameAsString(), e);
-        } finally {
-          try {
-            s.close();
-            if (region != null && region.getCoprocessorHost() != null) {
-              region.getCoprocessorHost().postScannerClose(s);
-            }
-          } catch (IOException e) {
-            LOG.error("Closing scanner for " + s.getRegionInfo().getRegionNameAsString(), e);
-          }
+          LOG.error("Closing scanner {} {}, user={}", this.scannerName, rsh, e,
+            RpcServer.getRequestUserName().orElse(null));
         }
-      } else {
-        LOG.warn("Scanner " + this.scannerName + " lease expired, but no related" +
-          " scanner found, hence no chance to close that related scanner!");
       }
     }
   }
@@ -1303,14 +1310,19 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return scanners.size();
   }
 
-  public
+  /**
+   * @return The outstanding RegionScanner for <code>scannerId</code> or null if none found.
+   */
   RegionScanner getScanner(long scannerId) {
-    String scannerIdString = Long.toString(scannerId);
-    RegionScannerHolder scannerHolder = scanners.get(scannerIdString);
-    if (scannerHolder != null) {
-      return scannerHolder.s;
-    }
-    return null;
+    RegionScannerHolder rsh = getRegionScannerHolder(scannerId);
+    return rsh == null? null: rsh.s;
+  }
+
+  /**
+   * @return The associated RegionScannerHolder for <code>scannerId</code> or null.
+   */
+  private RegionScannerHolder getRegionScannerHolder(long scannerId) {
+    return scanners.get(toScannerName(scannerId));
   }
 
   public String getScanDetailsWithId(long scannerId) {
@@ -1344,12 +1356,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    * Currently the vtime is the number of "next" calls.
    */
   long getScannerVirtualTime(long scannerId) {
-    String scannerIdString = Long.toString(scannerId);
-    RegionScannerHolder scannerHolder = scanners.get(scannerIdString);
-    if (scannerHolder != null) {
-      return scannerHolder.getNextCallSeq();
-    }
-    return 0L;
+    RegionScannerHolder rsh = getRegionScannerHolder(scannerId);
+    return rsh == null? 0L: rsh.getNextCallSeq();
   }
 
   /**
@@ -1393,24 +1401,36 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return lastBlock;
   }
 
+  /**
+   * @return Remote client's ip and port else null if can't be determined.
+   */
+  static String getRemoteClientIpAndPort() {
+    RpcCall rpcCall = RpcServer.getCurrentCall().orElse(null);
+    if (rpcCall == null) {
+      return HConstants.EMPTY_STRING;
+    }
+    InetAddress address = rpcCall.getRemoteAddress();
+    if (address == null) {
+      return HConstants.EMPTY_STRING;
+    }
+    // Be careful here with InetAddress. Do InetAddress#getHostAddress. It will not do a name
+    // resolution. Just use the IP. It is generally a smaller amount of info to keep around while
+    // scanning than a hostname anyways.
+    return Address.fromParts(address.getHostAddress(), rpcCall.getRemotePort()).toString();
+  }
+
   private RegionScannerHolder addScanner(String scannerName, RegionScanner s, Shipper shipper,
       HRegion r, boolean needCursor, boolean fullRegionScan) throws LeaseStillHeldException {
     Lease lease = regionServer.getLeaseManager().createLease(
         scannerName, this.scannerLeaseTimeoutPeriod, new ScannerListener(scannerName));
     RpcCallback shippedCallback = new RegionScannerShippedCallBack(scannerName, shipper, lease);
-    RpcCallback closeCallback;
-    if (s instanceof RpcCallback) {
-      closeCallback = (RpcCallback) s;
-    } else {
-      closeCallback = new RegionScannerCloseCallBack(s);
-    }
-
-    RegionScannerHolder rsh =
-        new RegionScannerHolder(scannerName, s, r, closeCallback, shippedCallback,
-          needCursor, fullRegionScan);
+    RpcCallback closeCallback = s instanceof RpcCallback?
+      (RpcCallback)s: new RegionScannerCloseCallBack(s);
+    RegionScannerHolder rsh = new RegionScannerHolder(s, r, closeCallback, shippedCallback,
+      needCursor, fullRegionScan, getRemoteClientIpAndPort());
     RegionScannerHolder existing = scanners.putIfAbsent(scannerName, rsh);
     assert existing == null : "scannerId must be unique within regionserver's whole lifecycle! " +
-      scannerName;
+      scannerName + ", " + existing;
     return rsh;
   }
 
@@ -3085,8 +3105,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   };
 
   private RegionScannerHolder getRegionScanner(ScanRequest request) throws IOException {
-    String scannerName = Long.toString(request.getScannerId());
-    RegionScannerHolder rsh = scanners.get(scannerName);
+    String scannerName = toScannerName(request.getScannerId());
+    RegionScannerHolder rsh = this.scanners.get(scannerName);
     if (rsh == null) {
       // just ignore the next or close request if scanner does not exists.
       if (closedScanners.getIfPresent(scannerName) != null) {
@@ -3125,8 +3145,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     return rsh;
   }
 
-  private RegionScannerHolder newRegionScanner(ScanRequest request, ScanResponse.Builder builder)
-      throws IOException {
+  /**
+   * @return Pair with scannerName key to use with this new Scanner and its RegionScannerHolder
+   *    value.
+   */
+  private Pair<String, RegionScannerHolder> newRegionScanner(ScanRequest request,
+      ScanResponse.Builder builder) throws IOException {
     HRegion region = getRegion(request.getRegion());
     ClientProtos.Scan protoScan = request.getScan();
     boolean isLoadingCfsOnDemandSet = protoScan.hasLoadColumnFamiliesOnDemand();
@@ -3157,13 +3181,24 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     builder.setScannerId(scannerId);
     builder.setMvccReadPoint(scanner.getMvccReadPoint());
     builder.setTtl(scannerLeaseTimeoutPeriod);
-    String scannerName = String.valueOf(scannerId);
+    String scannerName = toScannerName(scannerId);
 
     boolean fullRegionScan = !region.getRegionInfo().getTable().isSystemTable() &&
       isFullRegionScan(scan, region);
 
-    return addScanner(scannerName, scanner, shipper, region, scan.isNeedCursorResult(),
-      fullRegionScan);
+    return new Pair<String, RegionScannerHolder>(scannerName,
+      addScanner(scannerName, scanner, shipper, region, scan.isNeedCursorResult(),
+      fullRegionScan));
+  }
+
+  /**
+   * The returned String is used as key doing look up of outstanding Scanners in this Servers'
+   * this.scanners, the Map of outstanding scanners and their current state.
+   * @param scannerId A scanner long id.
+   * @return The long id as a String.
+   */
+  private static String toScannerName(long scannerId) {
+    return Long.toString(scannerId);
   }
 
   private void checkScanNextCallSeq(ScanRequest request, RegionScannerHolder rsh)
@@ -3437,7 +3472,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       checkOpen();
     } catch (IOException e) {
       if (request.hasScannerId()) {
-        String scannerName = Long.toString(request.getScannerId());
+        String scannerName = toScannerName(request.getScannerId());
         if (LOG.isDebugEnabled()) {
           LOG.debug(
             "Server shutting down and client tried to access missing scanner " + scannerName);
@@ -3460,14 +3495,19 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     rpcScanRequestCount.increment();
     RegionScannerHolder rsh;
     ScanResponse.Builder builder = ScanResponse.newBuilder();
+    String scannerName;
     try {
       if (request.hasScannerId()) {
         // The downstream projects such as AsyncHBase in OpenTSDB need this value. See HBASE-18000
         // for more details.
-        builder.setScannerId(request.getScannerId());
+        long scannerId = request.getScannerId();
+        builder.setScannerId(scannerId);
+        scannerName = toScannerName(scannerId);
         rsh = getRegionScanner(request);
       } else {
-        rsh = newRegionScanner(request, builder);
+        Pair<String, RegionScannerHolder> scannerNameAndRSH = newRegionScanner(request, builder);
+        scannerName = scannerNameAndRSH.getFirst();
+        rsh = scannerNameAndRSH.getSecond();
       }
     } catch (IOException e) {
       if (e == SCANNER_ALREADY_CLOSED) {
@@ -3481,11 +3521,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       rpcFullScanRequestCount.increment();
     }
     HRegion region = rsh.r;
-    String scannerName = rsh.scannerName;
     LeaseManager.Lease lease;
     try {
       // Remove lease while its being processed in server; protects against case
-      // where processing of request takes > lease expiration time.
+      // where processing of request takes > lease expiration time. or null if none found.
       lease = regionServer.getLeaseManager().removeLease(scannerName);
     } catch (LeaseException e) {
       throw new ServiceException(e);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSRpcServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSRpcServices.java
new file mode 100644
index 0000000..f242ddb
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSRpcServices.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test parts of {@link RSRpcServices}
+ */
+@Category({ RegionServerTests.class, MediumTests.class})
+public class TestRSRpcServices {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestRSRpcServices.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestRSRpcServices.class);
+
+  /**
+   * Simple test of the toString on RegionScannerHolder works.
+   * Just creates one and calls #toString on it.
+   */
+  @Test
+  public void testRegionScannerHolderToString() throws UnknownHostException {
+    RpcCall call = Mockito.mock(RpcCall.class);
+    int port = 1234;
+    Mockito.when(call.getRemotePort()).thenReturn(port);
+    InetAddress address = InetAddress.getLocalHost();
+    Mockito.when(call.getRemoteAddress()).thenReturn(address);
+    RpcServer.setCurrentCall(call);
+    String clientIpAndPort = RSRpcServices.getRemoteClientIpAndPort();
+    HRegion region = Mockito.mock(HRegion.class);
+    Mockito.when(region.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO);
+    RSRpcServices.RegionScannerHolder rsh = new RSRpcServices.RegionScannerHolder(null, region,
+      null, null, false, false, clientIpAndPort);
+    LOG.info("rsh={}", rsh);
+  }
+}