wchevreuil commented on code in PR #5470:
URL: https://github.com/apache/hbase/pull/5470#discussion_r1365372670


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java:
##########
@@ -499,7 +514,85 @@ Collection<ServerName> filterRSGroupServers(RSGroupInfo 
rsgroup,
   private void unloadRegions(ServerName server, List<ServerName> regionServers,
     List<RegionInfo> movedRegions) throws Exception {
     while (true) {
+      List<RegionInfo> isolateRegionInfoList = 
Collections.synchronizedList(new ArrayList<>());
+      RegionInfo isolateRegionInfo = null;
+      if (isolateRegionIdArray != null && !isolateRegionIdArray.isEmpty()) {
+        // Region will be moved to target region server with Ack mode.
+        final ExecutorService isolateRegionPool = 
Executors.newFixedThreadPool(maxthreads);
+        List<Future<Boolean>> isolateRegionTaskList = new ArrayList<>();
+        List<RegionInfo> recentlyIsolatedRegion = 
Collections.synchronizedList(new ArrayList<>());
+        boolean allRegionOpsSuccessful = true;
+        for(String isolateRegionId : isolateRegionIdArray) {
+          Result result = MetaTableAccessor.scanByRegionEncodedName(conn,
+            isolateRegionId);
+          HRegionLocation hRegionLocation = 
MetaTableAccessor.getRegionLocation(conn,
+            result.getRow());
+          if (hRegionLocation != null) {
+            isolateRegionInfo = hRegionLocation.getRegion();
+            isolateRegionInfoList.add(isolateRegionInfo);
+          } else {
+            LOG.error("One of the Region " + isolateRegionId + " doesn't 
exists/can't fetch from"
+              + " meta...Quitting now");
+            // We only move the regions if all the regions were found.
+            allRegionOpsSuccessful = false;
+            break;
+          }
+          if (hRegionLocation.getServerName() == server) {
+            LOG.info("Region " + isolateRegionId + " already exists on server 
: "
+              + server.getHostname());
+          } else {
+            Future<Boolean> isolateRegionTask = isolateRegionPool.submit(
+              new MoveWithAck(conn, isolateRegionInfo, 
hRegionLocation.getServerName(), server,

Review Comment:
   So we are moving regions to the RS we actually wanted to unload? Sounds 
contradictory, no? And since we are doing it before we actually do the unloads, 
it may even make situation worse for the case of overloaded RS.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java:
##########
@@ -499,7 +514,85 @@ Collection<ServerName> filterRSGroupServers(RSGroupInfo 
rsgroup,
   private void unloadRegions(ServerName server, List<ServerName> regionServers,
     List<RegionInfo> movedRegions) throws Exception {
     while (true) {
+      List<RegionInfo> isolateRegionInfoList = 
Collections.synchronizedList(new ArrayList<>());
+      RegionInfo isolateRegionInfo = null;
+      if (isolateRegionIdArray != null && !isolateRegionIdArray.isEmpty()) {
+        // Region will be moved to target region server with Ack mode.
+        final ExecutorService isolateRegionPool = 
Executors.newFixedThreadPool(maxthreads);
+        List<Future<Boolean>> isolateRegionTaskList = new ArrayList<>();
+        List<RegionInfo> recentlyIsolatedRegion = 
Collections.synchronizedList(new ArrayList<>());
+        boolean allRegionOpsSuccessful = true;
+        for(String isolateRegionId : isolateRegionIdArray) {
+          Result result = MetaTableAccessor.scanByRegionEncodedName(conn,
+            isolateRegionId);
+          HRegionLocation hRegionLocation = 
MetaTableAccessor.getRegionLocation(conn,
+            result.getRow());
+          if (hRegionLocation != null) {
+            isolateRegionInfo = hRegionLocation.getRegion();
+            isolateRegionInfoList.add(isolateRegionInfo);
+          } else {
+            LOG.error("One of the Region " + isolateRegionId + " doesn't 
exists/can't fetch from"
+              + " meta...Quitting now");
+            // We only move the regions if all the regions were found.
+            allRegionOpsSuccessful = false;
+            break;
+          }
+          if (hRegionLocation.getServerName() == server) {
+            LOG.info("Region " + isolateRegionId + " already exists on server 
: "
+              + server.getHostname());
+          } else {
+            Future<Boolean> isolateRegionTask = isolateRegionPool.submit(
+              new MoveWithAck(conn, isolateRegionInfo, 
hRegionLocation.getServerName(), server,
+                recentlyIsolatedRegion));
+            isolateRegionTaskList.add(isolateRegionTask);
+          }
+        }
+
+        if (!allRegionOpsSuccessful) {
+          // Failed to fetch one of the region's RegionInfo, so we exit from 
here.
+          break;

Review Comment:
   So if we've succeed to submit a first few before the failing to fetch meta 
location, we won't wait for the ack on the ones we submitted? This could lead 
the RS even more overloaded, no? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to