001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019
020package org.apache.hadoop.hbase.favored;
021
022import static org.apache.hadoop.hbase.ServerName.NON_STARTCODE;
023
024import java.io.IOException;
025import java.util.ArrayList;
026import java.util.HashMap;
027import java.util.HashSet;
028import java.util.List;
029import java.util.Map;
030import java.util.Map.Entry;
031import java.util.Random;
032import java.util.Set;
033
034import org.apache.commons.lang3.StringUtils;
035import org.apache.hadoop.conf.Configuration;
036import org.apache.hadoop.hbase.Cell.Type;
037import org.apache.hadoop.hbase.CellBuilderFactory;
038import org.apache.hadoop.hbase.CellBuilderType;
039import org.apache.hadoop.hbase.HBaseIOException;
040import org.apache.hadoop.hbase.HConstants;
041import org.apache.hadoop.hbase.MetaTableAccessor;
042import org.apache.hadoop.hbase.ServerName;
043import org.apache.hadoop.hbase.TableName;
044import org.apache.hadoop.hbase.client.Connection;
045import org.apache.hadoop.hbase.client.ConnectionFactory;
046import org.apache.hadoop.hbase.client.Put;
047import org.apache.hadoop.hbase.client.RegionInfo;
048import org.apache.hadoop.hbase.client.Table;
049import org.apache.hadoop.hbase.master.RackManager;
050import org.apache.hadoop.hbase.util.Bytes;
051import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
052import org.apache.yetus.audience.InterfaceAudience;
053import org.slf4j.Logger;
054import org.slf4j.LoggerFactory;
055
056import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
057import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
058import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
059import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
060import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.FavoredNodes;
061
062/**
063 * Helper class for {@link FavoredNodeLoadBalancer} that has all the intelligence for racks,
064 * meta scans, etc. Instantiated by the {@link FavoredNodeLoadBalancer} when needed (from
065 * within calls like {@link FavoredNodeLoadBalancer#randomAssignment(RegionInfo, List)}).
066 * All updates to favored nodes should only be done from {@link FavoredNodesManager} and not
067 * through this helper class (except for tests).
068 */
069@InterfaceAudience.Private
070public class FavoredNodeAssignmentHelper {
071  private static final Logger LOG = LoggerFactory.getLogger(FavoredNodeAssignmentHelper.class);
072  private RackManager rackManager;
073  private Map<String, List<ServerName>> rackToRegionServerMap;
074  private List<String> uniqueRackList;
075  // This map serves as a cache for rack to sn lookups. The num of
076  // region server entries might not match with that is in servers.
077  private Map<String, String> regionServerToRackMap;
078  private Random random;
079  private List<ServerName> servers;
080  public static final byte [] FAVOREDNODES_QUALIFIER = Bytes.toBytes("fn");
081  public final static short FAVORED_NODES_NUM = 3;
082  public final static short MAX_ATTEMPTS_FN_GENERATION = 10;
083
084  public FavoredNodeAssignmentHelper(final List<ServerName> servers, Configuration conf) {
085    this(servers, new RackManager(conf));
086  }
087
088  public FavoredNodeAssignmentHelper(final List<ServerName> servers,
089      final RackManager rackManager) {
090    this.servers = servers;
091    this.rackManager = rackManager;
092    this.rackToRegionServerMap = new HashMap<>();
093    this.regionServerToRackMap = new HashMap<>();
094    this.uniqueRackList = new ArrayList<>();
095    this.random = new Random();
096  }
097
098  // Always initialize() when FavoredNodeAssignmentHelper is constructed.
099  public void initialize() {
100    for (ServerName sn : this.servers) {
101      String rackName = getRackOfServer(sn);
102      List<ServerName> serverList = this.rackToRegionServerMap.get(rackName);
103      if (serverList == null) {
104        serverList = Lists.newArrayList();
105        // Add the current rack to the unique rack list
106        this.uniqueRackList.add(rackName);
107        this.rackToRegionServerMap.put(rackName, serverList);
108      }
109      for (ServerName serverName : serverList) {
110        if (ServerName.isSameAddress(sn, serverName)) {
111          // The server is already present, ignore.
112          break;
113        }
114      }
115      serverList.add((sn));
116      this.regionServerToRackMap.put(sn.getHostname(), rackName);
117    }
118  }
119
120  /**
121   * Update meta table with favored nodes info
122   * @param regionToFavoredNodes map of RegionInfo's to their favored nodes
123   * @param connection connection to be used
124   * @throws IOException
125   */
126  public static void updateMetaWithFavoredNodesInfo(
127      Map<RegionInfo, List<ServerName>> regionToFavoredNodes,
128      Connection connection) throws IOException {
129    List<Put> puts = new ArrayList<>();
130    for (Map.Entry<RegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
131      Put put = makePutFromRegionInfo(entry.getKey(), entry.getValue());
132      if (put != null) {
133        puts.add(put);
134      }
135    }
136    MetaTableAccessor.putsToMetaTable(connection, puts);
137    LOG.info("Added " + puts.size() + " regions in META");
138  }
139
140  /**
141   * Update meta table with favored nodes info
142   * @param regionToFavoredNodes
143   * @param conf
144   * @throws IOException
145   */
146  public static void updateMetaWithFavoredNodesInfo(
147      Map<RegionInfo, List<ServerName>> regionToFavoredNodes,
148      Configuration conf) throws IOException {
149    List<Put> puts = new ArrayList<>();
150    for (Map.Entry<RegionInfo, List<ServerName>> entry : regionToFavoredNodes.entrySet()) {
151      Put put = makePutFromRegionInfo(entry.getKey(), entry.getValue());
152      if (put != null) {
153        puts.add(put);
154      }
155    }
156    // Write the region assignments to the meta table.
157    // TODO: See above overrides take a Connection rather than a Configuration only the
158    // Connection is a short circuit connection. That is not going to good in all cases, when
159    // master and meta are not colocated. Fix when this favored nodes feature is actually used
160    // someday.
161    try (Connection connection = ConnectionFactory.createConnection(conf)) {
162      try (Table metaTable = connection.getTable(TableName.META_TABLE_NAME)) {
163        metaTable.put(puts);
164      }
165    }
166    LOG.info("Added " + puts.size() + " regions in META");
167  }
168
169  /**
170   * Generates and returns a Put containing the region info for the catalog table and the servers
171   * @return Put object
172   */
173  private static Put makePutFromRegionInfo(RegionInfo regionInfo, List<ServerName> favoredNodeList)
174      throws IOException {
175    Put put = null;
176    if (favoredNodeList != null) {
177      long time = EnvironmentEdgeManager.currentTime();
178      put = MetaTableAccessor.makePutFromRegionInfo(regionInfo, time);
179      byte[] favoredNodes = getFavoredNodes(favoredNodeList);
180      put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
181          .setRow(put.getRow())
182          .setFamily(HConstants.CATALOG_FAMILY)
183          .setQualifier(FAVOREDNODES_QUALIFIER)
184          .setTimestamp(time)
185          .setType(Type.Put)
186          .setValue(favoredNodes)
187          .build());
188      LOG.debug("Create the region {} with favored nodes {}", regionInfo.getRegionNameAsString(),
189        favoredNodeList);
190    }
191    return put;
192  }
193
194  /**
195   * @param favoredNodes The PB'ed bytes of favored nodes
196   * @return the array of {@link ServerName} for the byte array of favored nodes.
197   * @throws IOException
198   */
199  public static ServerName[] getFavoredNodesList(byte[] favoredNodes) throws IOException {
200    FavoredNodes f = FavoredNodes.parseFrom(favoredNodes);
201    List<HBaseProtos.ServerName> protoNodes = f.getFavoredNodeList();
202    ServerName[] servers = new ServerName[protoNodes.size()];
203    int i = 0;
204    for (HBaseProtos.ServerName node : protoNodes) {
205      servers[i++] = ProtobufUtil.toServerName(node);
206    }
207    return servers;
208  }
209
210  /**
211   * @param serverAddrList
212   * @return PB'ed bytes of {@link FavoredNodes} generated by the server list.
213   */
214  public static byte[] getFavoredNodes(List<ServerName> serverAddrList) {
215    FavoredNodes.Builder f = FavoredNodes.newBuilder();
216    for (ServerName s : serverAddrList) {
217      HBaseProtos.ServerName.Builder b = HBaseProtos.ServerName.newBuilder();
218      b.setHostName(s.getHostname());
219      b.setPort(s.getPort());
220      b.setStartCode(ServerName.NON_STARTCODE);
221      f.addFavoredNode(b.build());
222    }
223    return f.build().toByteArray();
224  }
225
226  // Place the regions round-robin across the racks picking one server from each
227  // rack at a time. Start with a random rack, and a random server from every rack.
228  // If a rack doesn't have enough servers it will go to the next rack and so on.
229  // for choosing a primary.
230  // For example, if 4 racks (r1 .. r4) with 8 servers (s1..s8) each, one possible
231  // placement could be r2:s5, r3:s5, r4:s5, r1:s5, r2:s6, r3:s6..
232  // If there were fewer servers in one rack, say r3, which had 3 servers, one possible
233  // placement could be r2:s5, <skip-r3>, r4:s5, r1:s5, r2:s6, <skip-r3> ...
234  // The regions should be distributed proportionately to the racksizes
235  public void placePrimaryRSAsRoundRobin(Map<ServerName, List<RegionInfo>> assignmentMap,
236      Map<RegionInfo, ServerName> primaryRSMap, List<RegionInfo> regions) {
237    List<String> rackList = new ArrayList<>(rackToRegionServerMap.size());
238    rackList.addAll(rackToRegionServerMap.keySet());
239    int rackIndex = random.nextInt(rackList.size());
240    int maxRackSize = 0;
241    for (Map.Entry<String,List<ServerName>> r : rackToRegionServerMap.entrySet()) {
242      if (r.getValue().size() > maxRackSize) {
243        maxRackSize = r.getValue().size();
244      }
245    }
246    int numIterations = 0;
247    // Initialize the current processing host index.
248    int serverIndex = random.nextInt(maxRackSize);
249    for (RegionInfo regionInfo : regions) {
250      List<ServerName> currentServerList;
251      String rackName;
252      while (true) {
253        rackName = rackList.get(rackIndex);
254        numIterations++;
255        // Get the server list for the current rack
256        currentServerList = rackToRegionServerMap.get(rackName);
257
258        if (serverIndex >= currentServerList.size()) { //not enough machines in this rack
259          if (numIterations % rackList.size() == 0) {
260            if (++serverIndex >= maxRackSize) serverIndex = 0;
261          }
262          if ((++rackIndex) >= rackList.size()) {
263            rackIndex = 0; // reset the rack index to 0
264          }
265        } else break;
266      }
267
268      // Get the current process region server
269      ServerName currentServer = currentServerList.get(serverIndex);
270
271      // Place the current region with the current primary region server
272      primaryRSMap.put(regionInfo, currentServer);
273      if (assignmentMap != null) {
274        List<RegionInfo> regionsForServer = assignmentMap.get(currentServer);
275        if (regionsForServer == null) {
276          regionsForServer = new ArrayList<>();
277          assignmentMap.put(currentServer, regionsForServer);
278        }
279        regionsForServer.add(regionInfo);
280      }
281
282      // Set the next processing index
283      if (numIterations % rackList.size() == 0) {
284        ++serverIndex;
285      }
286      if ((++rackIndex) >= rackList.size()) {
287        rackIndex = 0; // reset the rack index to 0
288      }
289    }
290  }
291
292  public Map<RegionInfo, ServerName[]> placeSecondaryAndTertiaryRS(
293      Map<RegionInfo, ServerName> primaryRSMap) {
294    Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap = new HashMap<>();
295    for (Map.Entry<RegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
296      // Get the target region and its primary region server rack
297      RegionInfo regionInfo = entry.getKey();
298      ServerName primaryRS = entry.getValue();
299      try {
300        // Create the secondary and tertiary region server pair object.
301        ServerName[] favoredNodes = getSecondaryAndTertiary(regionInfo, primaryRS);
302        if (favoredNodes != null) {
303          secondaryAndTertiaryMap.put(regionInfo, favoredNodes);
304          LOG.debug("Place the secondary and tertiary region server for region "
305              + regionInfo.getRegionNameAsString());
306        }
307      } catch (Exception e) {
308        LOG.warn("Cannot place the favored nodes for region " +
309            regionInfo.getRegionNameAsString() + " because " + e, e);
310        continue;
311      }
312    }
313    return secondaryAndTertiaryMap;
314  }
315
316  public ServerName[] getSecondaryAndTertiary(RegionInfo regionInfo, ServerName primaryRS)
317      throws IOException {
318
319    ServerName[] favoredNodes;// Get the rack for the primary region server
320    String primaryRack = getRackOfServer(primaryRS);
321
322    if (getTotalNumberOfRacks() == 1) {
323      favoredNodes = singleRackCase(regionInfo, primaryRS, primaryRack);
324    } else {
325      favoredNodes = multiRackCase(regionInfo, primaryRS, primaryRack);
326    }
327    return favoredNodes;
328  }
329
330  private Map<ServerName, Set<RegionInfo>> mapRSToPrimaries(
331      Map<RegionInfo, ServerName> primaryRSMap) {
332    Map<ServerName, Set<RegionInfo>> primaryServerMap = new HashMap<>();
333    for (Entry<RegionInfo, ServerName> e : primaryRSMap.entrySet()) {
334      Set<RegionInfo> currentSet = primaryServerMap.get(e.getValue());
335      if (currentSet == null) {
336        currentSet = new HashSet<>();
337      }
338      currentSet.add(e.getKey());
339      primaryServerMap.put(e.getValue(), currentSet);
340    }
341    return primaryServerMap;
342  }
343
344  /**
345   * For regions that share the primary, avoid placing the secondary and tertiary
346   * on a same RS. Used for generating new assignments for the
347   * primary/secondary/tertiary RegionServers
348   * @param primaryRSMap
349   * @return the map of regions to the servers the region-files should be hosted on
350   */
351  public Map<RegionInfo, ServerName[]> placeSecondaryAndTertiaryWithRestrictions(
352      Map<RegionInfo, ServerName> primaryRSMap) {
353    Map<ServerName, Set<RegionInfo>> serverToPrimaries =
354        mapRSToPrimaries(primaryRSMap);
355    Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap = new HashMap<>();
356
357    for (Entry<RegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
358      // Get the target region and its primary region server rack
359      RegionInfo regionInfo = entry.getKey();
360      ServerName primaryRS = entry.getValue();
361      try {
362        // Get the rack for the primary region server
363        String primaryRack = getRackOfServer(primaryRS);
364        ServerName[] favoredNodes = null;
365        if (getTotalNumberOfRacks() == 1) {
366          // Single rack case: have to pick the secondary and tertiary
367          // from the same rack
368          favoredNodes = singleRackCase(regionInfo, primaryRS, primaryRack);
369        } else {
370          favoredNodes = multiRackCaseWithRestrictions(serverToPrimaries,
371              secondaryAndTertiaryMap, primaryRack, primaryRS, regionInfo);
372        }
373        if (favoredNodes != null) {
374          secondaryAndTertiaryMap.put(regionInfo, favoredNodes);
375          LOG.debug("Place the secondary and tertiary region server for region "
376              + regionInfo.getRegionNameAsString());
377        }
378      } catch (Exception e) {
379        LOG.warn("Cannot place the favored nodes for region "
380            + regionInfo.getRegionNameAsString() + " because " + e, e);
381        continue;
382      }
383    }
384    return secondaryAndTertiaryMap;
385  }
386
387  private ServerName[] multiRackCaseWithRestrictions(
388      Map<ServerName, Set<RegionInfo>> serverToPrimaries,
389      Map<RegionInfo, ServerName[]> secondaryAndTertiaryMap,
390      String primaryRack, ServerName primaryRS, RegionInfo regionInfo) throws IOException {
391    // Random to choose the secondary and tertiary region server
392    // from another rack to place the secondary and tertiary
393    // Random to choose one rack except for the current rack
394    Set<String> rackSkipSet = new HashSet<>();
395    rackSkipSet.add(primaryRack);
396    String secondaryRack = getOneRandomRack(rackSkipSet);
397    List<ServerName> serverList = getServersFromRack(secondaryRack);
398    Set<ServerName> serverSet = new HashSet<>(serverList);
399    ServerName[] favoredNodes;
400    if (serverList.size() >= 2) {
401      // Randomly pick up two servers from this secondary rack
402      // Skip the secondary for the tertiary placement
403      // skip the servers which share the primary already
404      Set<RegionInfo> primaries = serverToPrimaries.get(primaryRS);
405      Set<ServerName> skipServerSet = new HashSet<>();
406      while (true) {
407        ServerName[] secondaryAndTertiary = null;
408        if (primaries.size() > 1) {
409          // check where his tertiary and secondary are
410          for (RegionInfo primary : primaries) {
411            secondaryAndTertiary = secondaryAndTertiaryMap.get(primary);
412            if (secondaryAndTertiary != null) {
413              if (getRackOfServer(secondaryAndTertiary[0]).equals(secondaryRack)) {
414                skipServerSet.add(secondaryAndTertiary[0]);
415              }
416              if (getRackOfServer(secondaryAndTertiary[1]).equals(secondaryRack)) {
417                skipServerSet.add(secondaryAndTertiary[1]);
418              }
419            }
420          }
421        }
422        if (skipServerSet.size() + 2 <= serverSet.size())
423          break;
424        skipServerSet.clear();
425        rackSkipSet.add(secondaryRack);
426        // we used all racks
427        if (rackSkipSet.size() == getTotalNumberOfRacks()) {
428          // remove the last two added and break
429          skipServerSet.remove(secondaryAndTertiary[0]);
430          skipServerSet.remove(secondaryAndTertiary[1]);
431          break;
432        }
433        secondaryRack = getOneRandomRack(rackSkipSet);
434        serverList = getServersFromRack(secondaryRack);
435        serverSet = new HashSet<>(serverList);
436      }
437
438      // Place the secondary RS
439      ServerName secondaryRS = getOneRandomServer(secondaryRack, skipServerSet);
440      skipServerSet.add(secondaryRS);
441      // Place the tertiary RS
442      ServerName tertiaryRS = getOneRandomServer(secondaryRack, skipServerSet);
443
444      if (secondaryRS == null || tertiaryRS == null) {
445        LOG.error("Cannot place the secondary and tertiary"
446            + " region server for region "
447            + regionInfo.getRegionNameAsString());
448      }
449      // Create the secondary and tertiary pair
450      favoredNodes = new ServerName[2];
451      favoredNodes[0] = secondaryRS;
452      favoredNodes[1] = tertiaryRS;
453    } else {
454      // Pick the secondary rs from this secondary rack
455      // and pick the tertiary from another random rack
456      favoredNodes = new ServerName[2];
457      ServerName secondary = getOneRandomServer(secondaryRack);
458      favoredNodes[0] = secondary;
459
460      // Pick the tertiary
461      if (getTotalNumberOfRacks() == 2) {
462        // Pick the tertiary from the same rack of the primary RS
463        Set<ServerName> serverSkipSet = new HashSet<>();
464        serverSkipSet.add(primaryRS);
465        favoredNodes[1] = getOneRandomServer(primaryRack, serverSkipSet);
466      } else {
467        // Pick the tertiary from another rack
468        rackSkipSet.add(secondaryRack);
469        String tertiaryRandomRack = getOneRandomRack(rackSkipSet);
470        favoredNodes[1] = getOneRandomServer(tertiaryRandomRack);
471      }
472    }
473    return favoredNodes;
474  }
475
476  private ServerName[] singleRackCase(RegionInfo regionInfo,
477      ServerName primaryRS,
478      String primaryRack) throws IOException {
479    // Single rack case: have to pick the secondary and tertiary
480    // from the same rack
481    List<ServerName> serverList = getServersFromRack(primaryRack);
482    if ((serverList == null) || (serverList.size() <= 2)) {
483      // Single region server case: cannot not place the favored nodes
484      // on any server;
485      return null;
486    } else {
487      // Randomly select two region servers from the server list and make sure
488      // they are not overlap with the primary region server;
489     Set<ServerName> serverSkipSet = new HashSet<>();
490     serverSkipSet.add(primaryRS);
491
492     // Place the secondary RS
493     ServerName secondaryRS = getOneRandomServer(primaryRack, serverSkipSet);
494     // Skip the secondary for the tertiary placement
495     serverSkipSet.add(secondaryRS);
496     ServerName tertiaryRS = getOneRandomServer(primaryRack, serverSkipSet);
497
498     if (secondaryRS == null || tertiaryRS == null) {
499       LOG.error("Cannot place the secondary, tertiary favored node for region " +
500           regionInfo.getRegionNameAsString());
501     }
502     // Create the secondary and tertiary pair
503     ServerName[] favoredNodes = new ServerName[2];
504     favoredNodes[0] = secondaryRS;
505     favoredNodes[1] = tertiaryRS;
506     return favoredNodes;
507    }
508  }
509
510  /**
511   * Place secondary and tertiary nodes in a multi rack case.
512   * If there are only two racks, then we try the place the secondary
513   * and tertiary on different rack than primary. But if the other rack has
514   * only one region server, then we place primary and tertiary on one rack
515   * and secondary on another. The aim is two distribute the three favored nodes
516   * on >= 2 racks.
517   * TODO: see how we can use generateMissingFavoredNodeMultiRack API here
518   * @param regionInfo Region for which we are trying to generate FN
519   * @param primaryRS The primary favored node.
520   * @param primaryRack The rack of the primary favored node.
521   * @return Array containing secondary and tertiary favored nodes.
522   * @throws IOException Signals that an I/O exception has occurred.
523   */
524  private ServerName[] multiRackCase(RegionInfo regionInfo, ServerName primaryRS,
525      String primaryRack) throws IOException {
526
527    List<ServerName>favoredNodes = Lists.newArrayList(primaryRS);
528    // Create the secondary and tertiary pair
529    ServerName secondaryRS = generateMissingFavoredNodeMultiRack(favoredNodes);
530    favoredNodes.add(secondaryRS);
531    String secondaryRack = getRackOfServer(secondaryRS);
532
533    ServerName tertiaryRS;
534    if (primaryRack.equals(secondaryRack)) {
535      tertiaryRS = generateMissingFavoredNode(favoredNodes);
536    } else {
537      // Try to place tertiary in secondary RS rack else place on primary rack.
538      tertiaryRS = getOneRandomServer(secondaryRack, Sets.newHashSet(secondaryRS));
539      if (tertiaryRS == null) {
540        tertiaryRS = getOneRandomServer(primaryRack, Sets.newHashSet(primaryRS));
541      }
542      // We couldn't find anything in secondary rack, get any FN
543      if (tertiaryRS == null) {
544        tertiaryRS = generateMissingFavoredNode(Lists.newArrayList(primaryRS, secondaryRS));
545      }
546    }
547    return new ServerName[]{ secondaryRS, tertiaryRS };
548  }
549
550  public boolean canPlaceFavoredNodes() {
551    return (this.servers.size() >= FAVORED_NODES_NUM);
552  }
553
554  private int getTotalNumberOfRacks() {
555    return this.uniqueRackList.size();
556  }
557
558  private List<ServerName> getServersFromRack(String rack) {
559    return this.rackToRegionServerMap.get(rack);
560  }
561
562  /**
563   * Gets a random server from the specified rack and skips anything specified.
564
565   * @param rack rack from a server is needed
566   * @param skipServerSet the server shouldn't belong to this set
567   */
568  protected ServerName getOneRandomServer(String rack, Set<ServerName> skipServerSet) {
569
570    // Is the rack valid? Do we recognize it?
571    if (rack == null || getServersFromRack(rack) == null ||
572        getServersFromRack(rack).isEmpty()) {
573      return null;
574    }
575
576    // Lets use a set so we can eliminate duplicates
577    Set<StartcodeAgnosticServerName> serversToChooseFrom = Sets.newHashSet();
578    for (ServerName sn : getServersFromRack(rack)) {
579      serversToChooseFrom.add(StartcodeAgnosticServerName.valueOf(sn));
580    }
581
582    if (skipServerSet != null && skipServerSet.size() > 0) {
583      for (ServerName sn : skipServerSet) {
584        serversToChooseFrom.remove(StartcodeAgnosticServerName.valueOf(sn));
585      }
586      // Do we have any servers left to choose from?
587      if (serversToChooseFrom.isEmpty()) {
588        return null;
589      }
590    }
591
592    ServerName randomServer = null;
593    int randomIndex = random.nextInt(serversToChooseFrom.size());
594    int j = 0;
595    for (StartcodeAgnosticServerName sn : serversToChooseFrom) {
596      if (j == randomIndex) {
597        randomServer = sn;
598        break;
599      }
600      j++;
601    }
602
603    if (randomServer != null) {
604      return ServerName.valueOf(randomServer.getHostAndPort(), randomServer.getStartcode());
605    } else {
606      return null;
607    }
608  }
609
610  private ServerName getOneRandomServer(String rack) throws IOException {
611    return this.getOneRandomServer(rack, null);
612  }
613
614  protected String getOneRandomRack(Set<String> skipRackSet) throws IOException {
615    if (skipRackSet == null || uniqueRackList.size() <= skipRackSet.size()) {
616      throw new IOException("Cannot randomly pick another random server");
617    }
618
619    String randomRack;
620    do {
621      int randomIndex = random.nextInt(this.uniqueRackList.size());
622      randomRack = this.uniqueRackList.get(randomIndex);
623    } while (skipRackSet.contains(randomRack));
624
625    return randomRack;
626  }
627
628  public static String getFavoredNodesAsString(List<ServerName> nodes) {
629    StringBuilder strBuf = new StringBuilder();
630    int i = 0;
631    for (ServerName node : nodes) {
632      strBuf.append(node.getHostAndPort());
633      if (++i != nodes.size()) strBuf.append(";");
634    }
635    return strBuf.toString();
636  }
637
638  /*
639   * Generates a missing favored node based on the input favored nodes. This helps to generate
640   * new FN when there is already 2 FN and we need a third one. For eg, while generating new FN
641   * for split daughters after inheriting 2 FN from the parent. If the cluster has only one rack
642   * it generates from the same rack. If the cluster has multiple racks, then it ensures the new
643   * FN respects the rack constraints similar to HDFS. For eg: if there are 3 FN, they will be
644   * spread across 2 racks.
645   */
646  public ServerName generateMissingFavoredNode(List<ServerName> favoredNodes) throws IOException {
647    if (this.uniqueRackList.size() == 1) {
648      return generateMissingFavoredNodeSingleRack(favoredNodes, null);
649    } else {
650      return generateMissingFavoredNodeMultiRack(favoredNodes, null);
651    }
652  }
653
654  public ServerName generateMissingFavoredNode(List<ServerName> favoredNodes,
655      List<ServerName> excludeNodes) throws IOException {
656    if (this.uniqueRackList.size() == 1) {
657      return generateMissingFavoredNodeSingleRack(favoredNodes, excludeNodes);
658    } else {
659      return generateMissingFavoredNodeMultiRack(favoredNodes, excludeNodes);
660    }
661  }
662
663  /*
664   * Generate FN for a single rack scenario, don't generate from one of the excluded nodes. Helps
665   * when we would like to find a replacement node.
666   */
667  private ServerName generateMissingFavoredNodeSingleRack(List<ServerName> favoredNodes,
668      List<ServerName> excludeNodes) throws IOException {
669    ServerName newServer = null;
670    Set<ServerName> excludeFNSet = Sets.newHashSet(favoredNodes);
671    if (excludeNodes != null && excludeNodes.size() > 0) {
672      excludeFNSet.addAll(excludeNodes);
673    }
674    if (favoredNodes.size() < FAVORED_NODES_NUM) {
675      newServer = this.getOneRandomServer(this.uniqueRackList.get(0), excludeFNSet);
676    }
677    return newServer;
678  }
679
680  private ServerName generateMissingFavoredNodeMultiRack(List<ServerName> favoredNodes)
681      throws IOException {
682    return generateMissingFavoredNodeMultiRack(favoredNodes, null);
683  }
684
685  /*
686   * Generates a missing FN based on the input favoredNodes and also the nodes to be skipped.
687   *
688   * Get the current layout of favored nodes arrangement and nodes to be excluded and get a
689   * random node that goes with HDFS block placement. Eg: If the existing nodes are on one rack,
690   * generate one from another rack. We exclude as much as possible so the random selection
691   * has more chance to generate a node within a few iterations, ideally 1.
692   */
693  private ServerName generateMissingFavoredNodeMultiRack(List<ServerName> favoredNodes,
694      List<ServerName> excludeNodes) throws IOException {
695
696    Set<String> racks = Sets.newHashSet();
697    Map<String, Set<ServerName>> rackToFNMapping = new HashMap<>();
698
699    // Lets understand the current rack distribution of the FN
700    for (ServerName sn : favoredNodes) {
701      String rack = getRackOfServer(sn);
702      racks.add(rack);
703
704      Set<ServerName> serversInRack = rackToFNMapping.get(rack);
705      if (serversInRack == null) {
706        serversInRack = Sets.newHashSet();
707        rackToFNMapping.put(rack, serversInRack);
708      }
709      serversInRack.add(sn);
710    }
711
712    // What racks should be skipped while getting a FN?
713    Set<String> skipRackSet = Sets.newHashSet();
714
715    /*
716     * If both the FN are from the same rack, then we don't want to generate another FN on the
717     * same rack. If that rack fails, the region would be unavailable.
718     */
719    if (racks.size() == 1 && favoredNodes.size() > 1) {
720      skipRackSet.add(racks.iterator().next());
721    }
722
723    /*
724     * If there are no free nodes on the existing racks, we should skip those racks too. We can
725     * reduce the number of iterations for FN selection.
726     */
727    for (String rack : racks) {
728      if (getServersFromRack(rack) != null &&
729        rackToFNMapping.get(rack).size() == getServersFromRack(rack).size()) {
730        skipRackSet.add(rack);
731      }
732    }
733
734    Set<ServerName> favoredNodeSet = Sets.newHashSet(favoredNodes);
735    if (excludeNodes != null && excludeNodes.size() > 0) {
736      favoredNodeSet.addAll(excludeNodes);
737    }
738
739    /*
740     * Lets get a random rack by excluding skipRackSet and generate a random FN from that rack.
741     */
742    int i = 0;
743    Set<String> randomRacks = Sets.newHashSet();
744    ServerName newServer = null;
745    do {
746      String randomRack = this.getOneRandomRack(skipRackSet);
747      newServer = this.getOneRandomServer(randomRack, favoredNodeSet);
748      randomRacks.add(randomRack);
749      i++;
750    } while ((i < MAX_ATTEMPTS_FN_GENERATION) && (newServer == null));
751
752    if (newServer == null) {
753      if (LOG.isTraceEnabled()) {
754        LOG.trace(String.format("Unable to generate additional favored nodes for %s after "
755            + "considering racks %s and skip rack %s with a unique rack list of %s and rack "
756            + "to RS map of %s and RS to rack map of %s",
757          StringUtils.join(favoredNodes, ","), randomRacks, skipRackSet, uniqueRackList,
758          rackToRegionServerMap, regionServerToRackMap));
759      }
760      throw new IOException(" Unable to generate additional favored nodes for "
761          + StringUtils.join(favoredNodes, ","));
762    }
763    return newServer;
764  }
765
766  /*
767   * Generate favored nodes for a region.
768   *
769   * Choose a random server as primary and then choose secondary and tertiary FN so its spread
770   * across two racks.
771   */
772  public List<ServerName> generateFavoredNodes(RegionInfo hri) throws IOException {
773
774    List<ServerName> favoredNodesForRegion = new ArrayList<>(FAVORED_NODES_NUM);
775    ServerName primary = servers.get(random.nextInt(servers.size()));
776    favoredNodesForRegion.add(ServerName.valueOf(primary.getHostAndPort(), ServerName.NON_STARTCODE));
777
778    Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>(1);
779    primaryRSMap.put(hri, primary);
780    Map<RegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
781        placeSecondaryAndTertiaryRS(primaryRSMap);
782    ServerName[] secondaryAndTertiaryNodes = secondaryAndTertiaryRSMap.get(hri);
783    if (secondaryAndTertiaryNodes != null && secondaryAndTertiaryNodes.length == 2) {
784      for (ServerName sn : secondaryAndTertiaryNodes) {
785        favoredNodesForRegion.add(ServerName.valueOf(sn.getHostAndPort(), ServerName.NON_STARTCODE));
786      }
787      return favoredNodesForRegion;
788    } else {
789      throw new HBaseIOException("Unable to generate secondary and tertiary favored nodes.");
790    }
791  }
792
793  public Map<RegionInfo, List<ServerName>> generateFavoredNodesRoundRobin(
794      Map<ServerName, List<RegionInfo>> assignmentMap, List<RegionInfo> regions)
795      throws IOException {
796
797    if (regions.size() > 0) {
798      if (canPlaceFavoredNodes()) {
799        Map<RegionInfo, ServerName> primaryRSMap = new HashMap<>();
800        // Lets try to have an equal distribution for primary favored node
801        placePrimaryRSAsRoundRobin(assignmentMap, primaryRSMap, regions);
802        return generateFavoredNodes(primaryRSMap);
803
804      } else {
805        throw new HBaseIOException("Not enough nodes to generate favored nodes");
806      }
807    }
808    return null;
809  }
810
811  /*
812   * Generate favored nodes for a set of regions when we know where they are currently hosted.
813   */
814  private Map<RegionInfo, List<ServerName>> generateFavoredNodes(
815      Map<RegionInfo, ServerName> primaryRSMap) {
816
817    Map<RegionInfo, List<ServerName>> generatedFavNodes = new HashMap<>();
818    Map<RegionInfo, ServerName[]> secondaryAndTertiaryRSMap =
819      placeSecondaryAndTertiaryRS(primaryRSMap);
820
821    for (Entry<RegionInfo, ServerName> entry : primaryRSMap.entrySet()) {
822      List<ServerName> favoredNodesForRegion = new ArrayList<>(FAVORED_NODES_NUM);
823      RegionInfo region = entry.getKey();
824      ServerName primarySN = entry.getValue();
825      favoredNodesForRegion.add(ServerName.valueOf(primarySN.getHostname(), primarySN.getPort(),
826        NON_STARTCODE));
827      ServerName[] secondaryAndTertiaryNodes = secondaryAndTertiaryRSMap.get(region);
828      if (secondaryAndTertiaryNodes != null) {
829        favoredNodesForRegion.add(ServerName.valueOf(
830          secondaryAndTertiaryNodes[0].getHostname(), secondaryAndTertiaryNodes[0].getPort(),
831          NON_STARTCODE));
832        favoredNodesForRegion.add(ServerName.valueOf(
833          secondaryAndTertiaryNodes[1].getHostname(), secondaryAndTertiaryNodes[1].getPort(),
834          NON_STARTCODE));
835      }
836      generatedFavNodes.put(region, favoredNodesForRegion);
837    }
838    return generatedFavNodes;
839  }
840
841  /*
842   * Get the rack of server from local mapping when present, saves lookup by the RackManager.
843   */
844  private String getRackOfServer(ServerName sn) {
845    if (this.regionServerToRackMap.containsKey(sn.getHostname())) {
846      return this.regionServerToRackMap.get(sn.getHostname());
847    } else {
848      String rack = this.rackManager.getRack(sn);
849      this.regionServerToRackMap.put(sn.getHostname(), rack);
850      return rack;
851    }
852  }
853}