1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.master;
20
21 import java.io.IOException;
22 import java.io.InterruptedIOException;
23 import java.util.ArrayList;
24 import java.util.Arrays;
25 import java.util.Collection;
26 import java.util.Collections;
27 import java.util.HashMap;
28 import java.util.HashSet;
29 import java.util.Iterator;
30 import java.util.List;
31 import java.util.Map;
32 import java.util.NavigableMap;
33 import java.util.Random;
34 import java.util.Set;
35 import java.util.TreeMap;
36 import java.util.concurrent.Callable;
37 import java.util.concurrent.ConcurrentHashMap;
38 import java.util.concurrent.CopyOnWriteArrayList;
39 import java.util.concurrent.ThreadFactory;
40 import java.util.concurrent.TimeUnit;
41 import java.util.concurrent.atomic.AtomicBoolean;
42 import java.util.concurrent.atomic.AtomicInteger;
43 import java.util.concurrent.locks.Lock;
44 import java.util.concurrent.locks.ReentrantLock;
45
46 import org.apache.commons.logging.Log;
47 import org.apache.commons.logging.LogFactory;
48 import org.apache.hadoop.conf.Configuration;
49 import org.apache.hadoop.fs.FileSystem;
50 import org.apache.hadoop.fs.Path;
51 import org.apache.hadoop.hbase.CoordinatedStateException;
52 import org.apache.hadoop.hbase.HBaseIOException;
53 import org.apache.hadoop.hbase.HConstants;
54 import org.apache.hadoop.hbase.HRegionInfo;
55 import org.apache.hadoop.hbase.HRegionLocation;
56 import org.apache.hadoop.hbase.HTableDescriptor;
57 import org.apache.hadoop.hbase.MetaTableAccessor;
58 import org.apache.hadoop.hbase.NotServingRegionException;
59 import org.apache.hadoop.hbase.RegionLocations;
60 import org.apache.hadoop.hbase.RegionTransition;
61 import org.apache.hadoop.hbase.Server;
62 import org.apache.hadoop.hbase.ServerName;
63 import org.apache.hadoop.hbase.TableName;
64 import org.apache.hadoop.hbase.TableNotFoundException;
65 import org.apache.hadoop.hbase.TableStateManager;
66 import org.apache.hadoop.hbase.classification.InterfaceAudience;
67 import org.apache.hadoop.hbase.client.Admin;
68 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
69 import org.apache.hadoop.hbase.client.Result;
70 import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
71 import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
72 import org.apache.hadoop.hbase.coordination.RegionMergeCoordination;
73 import org.apache.hadoop.hbase.coordination.SplitTransactionCoordination.SplitTransactionDetails;
74 import org.apache.hadoop.hbase.coordination.ZkOpenRegionCoordination;
75 import org.apache.hadoop.hbase.coordination.ZkRegionMergeCoordination;
76 import org.apache.hadoop.hbase.exceptions.DeserializationException;
77 import org.apache.hadoop.hbase.executor.EventHandler;
78 import org.apache.hadoop.hbase.executor.EventType;
79 import org.apache.hadoop.hbase.executor.ExecutorService;
80 import org.apache.hadoop.hbase.ipc.FailedServerException;
81 import org.apache.hadoop.hbase.ipc.RpcClient;
82 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
83 import org.apache.hadoop.hbase.master.RegionState.State;
84 import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
85 import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer;
86 import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler;
87 import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
88 import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
89 import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
90 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
91 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
92 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
93 import org.apache.hadoop.hbase.quotas.RegionStateListener;
94 import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException;
95 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
96 import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
97 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
98 import org.apache.hadoop.hbase.util.ConfigUtil;
99 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
100 import org.apache.hadoop.hbase.util.FSUtils;
101 import org.apache.hadoop.hbase.util.KeyLocker;
102 import org.apache.hadoop.hbase.util.Pair;
103 import org.apache.hadoop.hbase.util.PairOfSameType;
104 import org.apache.hadoop.hbase.util.Threads;
105 import org.apache.hadoop.hbase.util.Triple;
106 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
107 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
108 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
109 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
110 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
111 import org.apache.hadoop.ipc.RemoteException;
112 import org.apache.hadoop.util.StringUtils;
113 import org.apache.zookeeper.AsyncCallback;
114 import org.apache.zookeeper.KeeperException;
115 import org.apache.zookeeper.KeeperException.NoNodeException;
116 import org.apache.zookeeper.KeeperException.NodeExistsException;
117 import org.apache.zookeeper.data.Stat;
118
119 import com.google.common.annotations.VisibleForTesting;
120 import com.google.common.collect.LinkedHashMultimap;
121
122
123
124
125
126
127
128
129 @InterfaceAudience.Private
130 public class AssignmentManager extends ZooKeeperListener {
131 private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
132
133 public static final ServerName HBCK_CODE_SERVERNAME = ServerName.valueOf(HConstants.HBCK_CODE_NAME,
134 -1, -1L);
135
136 static final String ALREADY_IN_TRANSITION_WAITTIME
137 = "hbase.assignment.already.intransition.waittime";
138 static final int DEFAULT_ALREADY_IN_TRANSITION_WAITTIME = 60000;
139
140 protected final Server server;
141
142 private ServerManager serverManager;
143
144 private boolean shouldAssignRegionsWithFavoredNodes;
145
146 private LoadBalancer balancer;
147
148 private final MetricsAssignmentManager metricsAssignmentManager;
149
150 private final TableLockManager tableLockManager;
151
152 private AtomicInteger numRegionsOpened = new AtomicInteger(0);
153
154 final private KeyLocker<String> locker = new KeyLocker<String>();
155
156 Set<HRegionInfo> replicasToClose = Collections.synchronizedSet(new HashSet<HRegionInfo>());
157
158
159
160
161
162 private final Map <String, HRegionInfo> regionsToReopen;
163
164
165
166
167
168 private final int maximumAttempts;
169
170
171
172
173 private final Map<String, PairOfSameType<HRegionInfo>> mergingRegions
174 = new HashMap<String, PairOfSameType<HRegionInfo>>();
175
176 private final Map<HRegionInfo, PairOfSameType<HRegionInfo>> splitRegions
177 = new HashMap<HRegionInfo, PairOfSameType<HRegionInfo>>();
178
179
180
181
182
183 private final long sleepTimeBeforeRetryingMetaAssignment;
184
185
186
187
188
189 final NavigableMap<String, RegionPlan> regionPlans =
190 new TreeMap<String, RegionPlan>();
191
192 private final TableStateManager tableStateManager;
193
194 private final ExecutorService executorService;
195
196
197 private Map<HRegionInfo, AtomicBoolean> closedRegionHandlerCalled = null;
198
199
200 private Map<HRegionInfo, AtomicBoolean> openedRegionHandlerCalled = null;
201
202
203 private java.util.concurrent.ExecutorService threadPoolExecutorService;
204
205
206 private final java.util.concurrent.ExecutorService zkEventWorkers;
207
208 private List<EventType> ignoreStatesRSOffline = Arrays.asList(
209 EventType.RS_ZK_REGION_FAILED_OPEN, EventType.RS_ZK_REGION_CLOSED);
210
211 private final RegionStates regionStates;
212
213
214
215
216
217 private final int bulkAssignThresholdRegions;
218 private final int bulkAssignThresholdServers;
219 private final int bulkPerRegionOpenTimeGuesstimate;
220
221
222
223
224 private final boolean bulkAssignWaitTillAllAssigned;
225
226
227
228
229
230
231
232
233
234 protected final AtomicBoolean failoverCleanupDone = new AtomicBoolean(false);
235
236
237
238
239
240
241
242
243 private final ConcurrentHashMap<String, AtomicInteger>
244 failedOpenTracker = new ConcurrentHashMap<String, AtomicInteger>();
245
246
247 private final boolean useZKForAssignment;
248
249
250
251 private final RegionStateStore regionStateStore;
252
253
254
255
256 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MS_SHOULD_BE_FINAL")
257 public static boolean TEST_SKIP_SPLIT_HANDLING = false;
258
259
260 private List<AssignmentListener> listeners = new CopyOnWriteArrayList<AssignmentListener>();
261
262 private RegionStateListener regionStateListener;
263
264 public enum ServerHostRegion {
265 NOT_HOSTING_REGION, HOSTING_REGION, UNKNOWN,
266 }
267
268
269
270
271
272
273
274
275
276
277
278
279
280 public AssignmentManager(Server server, ServerManager serverManager,
281 final LoadBalancer balancer,
282 final ExecutorService service, MetricsMaster metricsMaster,
283 final TableLockManager tableLockManager) throws KeeperException,
284 IOException, CoordinatedStateException {
285 super(server.getZooKeeper());
286 this.server = server;
287 this.serverManager = serverManager;
288 this.executorService = service;
289 this.regionStateStore = new RegionStateStore(server);
290 this.regionsToReopen = Collections.synchronizedMap
291 (new HashMap<String, HRegionInfo> ());
292 Configuration conf = server.getConfiguration();
293
294 this.shouldAssignRegionsWithFavoredNodes = conf.getClass(
295 HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class).equals(
296 FavoredNodeLoadBalancer.class);
297 try {
298 if (server.getCoordinatedStateManager() != null) {
299 this.tableStateManager = server.getCoordinatedStateManager().getTableStateManager();
300 } else {
301 this.tableStateManager = null;
302 }
303 } catch (InterruptedException e) {
304 throw new InterruptedIOException();
305 }
306
307 this.maximumAttempts = Math.max(1,
308 this.server.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10));
309 this.sleepTimeBeforeRetryingMetaAssignment = this.server.getConfiguration().getLong(
310 "hbase.meta.assignment.retry.sleeptime", 1000l);
311 this.balancer = balancer;
312 int maxThreads = conf.getInt("hbase.assignment.threads.max", 30);
313 this.threadPoolExecutorService = Threads.getBoundedCachedThreadPool(
314 maxThreads, 60L, TimeUnit.SECONDS, Threads.newDaemonThreadFactory("AM."));
315 this.regionStates = new RegionStates(
316 server, tableStateManager, serverManager, regionStateStore);
317
318 this.bulkAssignWaitTillAllAssigned =
319 conf.getBoolean("hbase.bulk.assignment.waittillallassigned", false);
320 this.bulkAssignThresholdRegions = conf.getInt("hbase.bulk.assignment.threshold.regions", 7);
321 this.bulkAssignThresholdServers = conf.getInt("hbase.bulk.assignment.threshold.servers", 3);
322 this.bulkPerRegionOpenTimeGuesstimate =
323 conf.getInt("hbase.bulk.assignment.perregion.open.time", 10000);
324
325 int workers = conf.getInt("hbase.assignment.zkevent.workers", 20);
326 ThreadFactory threadFactory = Threads.newDaemonThreadFactory("AM.ZK.Worker");
327 zkEventWorkers = Threads.getBoundedCachedThreadPool(workers, 60L,
328 TimeUnit.SECONDS, threadFactory);
329 this.tableLockManager = tableLockManager;
330
331 this.metricsAssignmentManager = new MetricsAssignmentManager();
332 useZKForAssignment = ConfigUtil.useZKForAssignment(conf);
333 }
334
335
336
337
338
339 public void registerListener(final AssignmentListener listener) {
340 this.listeners.add(listener);
341 }
342
343
344
345
346
347 public boolean unregisterListener(final AssignmentListener listener) {
348 return this.listeners.remove(listener);
349 }
350
351
352
353
354 public TableStateManager getTableStateManager() {
355
356
357 return this.tableStateManager;
358 }
359
360
361
362
363
364
365
366 public RegionStates getRegionStates() {
367 return regionStates;
368 }
369
370
371
372
373 @VisibleForTesting
374 RegionStateStore getRegionStateStore() {
375 return regionStateStore;
376 }
377
378 public RegionPlan getRegionReopenPlan(HRegionInfo hri) {
379 return new RegionPlan(hri, null, regionStates.getRegionServerOfRegion(hri));
380 }
381
382
383
384
385
386
387 public void addPlan(String encodedName, RegionPlan plan) {
388 synchronized (regionPlans) {
389 regionPlans.put(encodedName, plan);
390 }
391 }
392
393
394
395
396 public void addPlans(Map<String, RegionPlan> plans) {
397 synchronized (regionPlans) {
398 regionPlans.putAll(plans);
399 }
400 }
401
402
403
404
405
406
407
408
409 public void setRegionsToReopen(List <HRegionInfo> regions) {
410 for(HRegionInfo hri : regions) {
411 regionsToReopen.put(hri.getEncodedName(), hri);
412 }
413 }
414
415
416
417
418
419
420
421
422 public Pair<Integer, Integer> getReopenStatus(TableName tableName)
423 throws IOException {
424 List<HRegionInfo> hris;
425 if (TableName.META_TABLE_NAME.equals(tableName)) {
426 hris = new MetaTableLocator().getMetaRegions(server.getZooKeeper());
427 } else {
428 hris = MetaTableAccessor.getTableRegions(server.getZooKeeper(),
429 server.getConnection(), tableName, true);
430 }
431
432 Integer pending = 0;
433 for (HRegionInfo hri : hris) {
434 String name = hri.getEncodedName();
435
436 if (regionsToReopen.containsKey(name)
437 || regionStates.isRegionInTransition(name)) {
438 pending++;
439 }
440 }
441 return new Pair<Integer, Integer>(pending, hris.size());
442 }
443
444
445
446
447
448
449 public boolean isFailoverCleanupDone() {
450 return failoverCleanupDone.get();
451 }
452
453
454
455
456
457 public Lock acquireRegionLock(final String encodedName) {
458 return locker.acquireLock(encodedName);
459 }
460
461
462
463
464
465 void failoverCleanupDone() {
466 failoverCleanupDone.set(true);
467 serverManager.processQueuedDeadServers();
468 }
469
470
471
472
473
474
475
476
477
478 void joinCluster() throws IOException,
479 KeeperException, InterruptedException, CoordinatedStateException {
480 long startTime = System.currentTimeMillis();
481
482
483
484
485
486
487
488
489
490
491
492 Set<ServerName> deadServers = rebuildUserRegions();
493
494
495
496
497 boolean failover = processDeadServersAndRegionsInTransition(deadServers);
498
499 if (!useZKForAssignment) {
500
501 ZKUtil.deleteNodeRecursively(watcher, watcher.assignmentZNode);
502 }
503 recoverTableInDisablingState();
504 recoverTableInEnablingState();
505 LOG.info("Joined the cluster in " + (System.currentTimeMillis()
506 - startTime) + "ms, failover=" + failover);
507 }
508
509
510
511
512
513
514
515
516
517
518
519
520 boolean processDeadServersAndRegionsInTransition(
521 final Set<ServerName> deadServers) throws KeeperException,
522 IOException, InterruptedException, CoordinatedStateException {
523 List<String> nodes = ZKUtil.listChildrenNoWatch(watcher,
524 watcher.assignmentZNode);
525
526 if (useZKForAssignment && nodes == null) {
527 String errorMessage = "Failed to get the children from ZK";
528 server.abort(errorMessage, new IOException(errorMessage));
529 return true;
530 }
531
532 boolean failover = !serverManager.getDeadServers().isEmpty();
533 if (failover) {
534
535 if (LOG.isDebugEnabled()) {
536 LOG.debug("Found dead servers out on cluster " + serverManager.getDeadServers());
537 }
538 } else {
539
540 Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
541 for (Map.Entry<HRegionInfo, ServerName> en:
542 regionStates.getRegionAssignments().entrySet()) {
543 HRegionInfo hri = en.getKey();
544 if (!hri.isMetaTable()
545 && onlineServers.contains(en.getValue())) {
546 LOG.debug("Found " + hri + " out on cluster");
547 failover = true;
548 break;
549 }
550 }
551 if (!failover && nodes != null) {
552
553 for (String encodedName: nodes) {
554 RegionState regionState = regionStates.getRegionState(encodedName);
555 if (regionState != null && !regionState.getRegion().isMetaRegion()) {
556 LOG.debug("Found " + regionState + " in RITs");
557 failover = true;
558 break;
559 }
560 }
561 }
562 }
563 if (!failover && !useZKForAssignment) {
564
565 Map<String, RegionState> regionsInTransition = regionStates.getRegionsInTransition();
566 if (!regionsInTransition.isEmpty()) {
567 Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
568 for (RegionState regionState: regionsInTransition.values()) {
569 ServerName serverName = regionState.getServerName();
570 if (!regionState.getRegion().isMetaRegion()
571 && serverName != null && onlineServers.contains(serverName)) {
572 LOG.debug("Found " + regionState + " in RITs");
573 failover = true;
574 break;
575 }
576 }
577 }
578 }
579 if (!failover) {
580
581
582
583
584 Set<ServerName> queuedDeadServers = serverManager.getRequeuedDeadServers().keySet();
585 if (!queuedDeadServers.isEmpty()) {
586 Configuration conf = server.getConfiguration();
587 Path rootdir = FSUtils.getRootDir(conf);
588 FileSystem fs = rootdir.getFileSystem(conf);
589 for (ServerName serverName: queuedDeadServers) {
590
591
592 Path logDir = new Path(rootdir,
593 DefaultWALProvider.getWALDirectoryName(serverName.toString()));
594 Path splitDir = logDir.suffix(DefaultWALProvider.SPLITTING_EXT);
595 if (fs.exists(logDir) || fs.exists(splitDir)) {
596 LOG.debug("Found queued dead server " + serverName);
597 failover = true;
598 break;
599 }
600 }
601 if (!failover) {
602
603
604 LOG.info("AM figured that it's not a failover and cleaned up "
605 + queuedDeadServers.size() + " queued dead servers");
606 serverManager.removeRequeuedDeadServers();
607 }
608 }
609 }
610
611 Set<TableName> disabledOrDisablingOrEnabling = null;
612 Map<HRegionInfo, ServerName> allRegions = null;
613
614 if (!failover) {
615 disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
616 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING,
617 ZooKeeperProtos.Table.State.ENABLING);
618
619
620 allRegions = regionStates.closeAllUserRegions(
621 disabledOrDisablingOrEnabling);
622 }
623
624
625 regionStateStore.start();
626
627
628 if (failover) {
629 LOG.info("Found regions out on cluster or in RIT; presuming failover");
630
631
632 processDeadServersAndRecoverLostRegions(deadServers);
633 }
634
635 if (!failover && useZKForAssignment) {
636
637 ZKAssign.deleteAllNodes(watcher);
638 ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
639 this.watcher.assignmentZNode);
640 }
641
642
643
644
645
646 failoverCleanupDone();
647 if (!failover) {
648
649 LOG.info("Clean cluster startup. Assigning user regions");
650 assignAllUserRegions(allRegions);
651 }
652
653
654
655 for (HRegionInfo h : replicasToClose) {
656 unassign(h);
657 }
658 replicasToClose.clear();
659 return failover;
660 }
661
662
663
664
665
666
667
668
669
670
671
672
673 boolean processRegionInTransitionAndBlockUntilAssigned(final HRegionInfo hri)
674 throws InterruptedException, KeeperException, IOException {
675 String encodedRegionName = hri.getEncodedName();
676 if (!processRegionInTransition(encodedRegionName, hri)) {
677 return false;
678 }
679 LOG.debug("Waiting on " + HRegionInfo.prettyPrint(encodedRegionName));
680 while (!this.server.isStopped() &&
681 this.regionStates.isRegionInTransition(encodedRegionName)) {
682 RegionState state = this.regionStates.getRegionTransitionState(encodedRegionName);
683 if (state == null || !serverManager.isServerOnline(state.getServerName())) {
684
685
686
687 break;
688 }
689 this.regionStates.waitForUpdate(100);
690 }
691 return true;
692 }
693
694
695
696
697
698
699
700
701
702
703 boolean processRegionInTransition(final String encodedRegionName,
704 final HRegionInfo regionInfo) throws KeeperException, IOException {
705
706
707
708
709 Lock lock = locker.acquireLock(encodedRegionName);
710 try {
711 Stat stat = new Stat();
712 byte [] data = ZKAssign.getDataAndWatch(watcher, encodedRegionName, stat);
713 if (data == null) return false;
714 RegionTransition rt;
715 try {
716 rt = RegionTransition.parseFrom(data);
717 } catch (DeserializationException e) {
718 LOG.warn("Failed parse znode data", e);
719 return false;
720 }
721 HRegionInfo hri = regionInfo;
722 if (hri == null) {
723
724
725
726
727
728 hri = regionStates.getRegionInfo(rt.getRegionName());
729 EventType et = rt.getEventType();
730 if (hri == null && et != EventType.RS_ZK_REGION_MERGING
731 && et != EventType.RS_ZK_REQUEST_REGION_MERGE) {
732 LOG.warn("Couldn't find the region in recovering " + rt);
733 return false;
734 }
735 }
736
737
738
739 BaseCoordinatedStateManager cp =
740 (BaseCoordinatedStateManager) this.server.getCoordinatedStateManager();
741 OpenRegionCoordination openRegionCoordination = cp.getOpenRegionCoordination();
742
743 ZkOpenRegionCoordination.ZkOpenRegionDetails zkOrd =
744 new ZkOpenRegionCoordination.ZkOpenRegionDetails();
745 zkOrd.setVersion(stat.getVersion());
746 zkOrd.setServerName(cp.getServer().getServerName());
747
748 return processRegionsInTransition(
749 rt, hri, openRegionCoordination, zkOrd);
750 } finally {
751 lock.unlock();
752 }
753 }
754
755
756
757
758
759
760
761
762
763 boolean processRegionsInTransition(
764 final RegionTransition rt, final HRegionInfo regionInfo,
765 OpenRegionCoordination coordination,
766 final OpenRegionCoordination.OpenRegionDetails ord) throws KeeperException {
767 EventType et = rt.getEventType();
768
769 final ServerName sn = rt.getServerName();
770 final byte[] regionName = rt.getRegionName();
771 final String encodedName = HRegionInfo.encodeRegionName(regionName);
772 final String prettyPrintedRegionName = HRegionInfo.prettyPrint(encodedName);
773 LOG.info("Processing " + prettyPrintedRegionName + " in state: " + et);
774
775 if (regionStates.isRegionInTransition(encodedName)
776 && (regionInfo.isMetaRegion() || !useZKForAssignment)) {
777 LOG.info("Processed region " + prettyPrintedRegionName + " in state: "
778 + et + ", does nothing since the region is already in transition "
779 + regionStates.getRegionTransitionState(encodedName));
780
781 return true;
782 }
783 if (!serverManager.isServerOnline(sn)) {
784
785
786
787 LOG.debug("RIT " + encodedName + " in state=" + rt.getEventType() +
788 " was on deadserver; forcing offline");
789 if (regionStates.isRegionOnline(regionInfo)) {
790
791
792
793 regionStates.regionOffline(regionInfo);
794 sendRegionClosedNotification(regionInfo);
795 }
796
797 regionStates.updateRegionState(regionInfo, State.OFFLINE, sn);
798
799 if (regionInfo.isMetaRegion()) {
800
801
802 MetaTableLocator.setMetaLocation(watcher, sn, State.OPEN);
803 } else {
804
805
806 regionStates.setLastRegionServerOfRegion(sn, encodedName);
807
808 if (!serverManager.isServerDead(sn)) {
809 serverManager.expireServer(sn);
810 }
811 }
812 return false;
813 }
814 switch (et) {
815 case M_ZK_REGION_CLOSING:
816
817
818 final RegionState rsClosing = regionStates.updateRegionState(rt, State.CLOSING);
819 this.executorService.submit(
820 new EventHandler(server, EventType.M_MASTER_RECOVERY) {
821 @Override
822 public void process() throws IOException {
823 ReentrantLock lock = locker.acquireLock(regionInfo.getEncodedName());
824 try {
825 final int expectedVersion = ((ZkOpenRegionCoordination.ZkOpenRegionDetails) ord)
826 .getVersion();
827 unassign(regionInfo, rsClosing, expectedVersion, null, useZKForAssignment, null);
828 if (regionStates.isRegionOffline(regionInfo)) {
829 assign(regionInfo, true);
830 }
831 } finally {
832 lock.unlock();
833 }
834 }
835 });
836 break;
837
838 case RS_ZK_REGION_CLOSED:
839 case RS_ZK_REGION_FAILED_OPEN:
840
841 regionStates.setLastRegionServerOfRegion(sn, encodedName);
842 regionStates.updateRegionState(regionInfo, State.CLOSED, sn);
843 if (!replicasToClose.contains(regionInfo)) {
844 invokeAssign(regionInfo);
845 } else {
846 offlineDisabledRegion(regionInfo);
847 }
848 break;
849
850 case M_ZK_REGION_OFFLINE:
851
852 regionStates.updateRegionState(rt, State.PENDING_OPEN);
853 final RegionState rsOffline = regionStates.getRegionState(regionInfo);
854 this.executorService.submit(
855 new EventHandler(server, EventType.M_MASTER_RECOVERY) {
856 @Override
857 public void process() throws IOException {
858 ReentrantLock lock = locker.acquireLock(regionInfo.getEncodedName());
859 try {
860 RegionPlan plan = new RegionPlan(regionInfo, null, sn);
861 addPlan(encodedName, plan);
862 assign(rsOffline, false, false);
863 } finally {
864 lock.unlock();
865 }
866 }
867 });
868 break;
869
870 case RS_ZK_REGION_OPENING:
871 regionStates.updateRegionState(rt, State.OPENING);
872 break;
873
874 case RS_ZK_REGION_OPENED:
875
876
877
878 regionStates.updateRegionState(rt, State.OPEN);
879 new OpenedRegionHandler(server, this, regionInfo, coordination, ord).process();
880 break;
881 case RS_ZK_REQUEST_REGION_SPLIT:
882 case RS_ZK_REGION_SPLITTING:
883 case RS_ZK_REGION_SPLIT:
884
885
886
887 regionStates.regionOnline(regionInfo, sn);
888 regionStates.updateRegionState(rt, State.SPLITTING);
889 if (!handleRegionSplitting(
890 rt, encodedName, prettyPrintedRegionName, sn)) {
891 deleteSplittingNode(encodedName, sn);
892 }
893 break;
894 case RS_ZK_REQUEST_REGION_MERGE:
895 case RS_ZK_REGION_MERGING:
896 case RS_ZK_REGION_MERGED:
897 if (!handleRegionMerging(
898 rt, encodedName, prettyPrintedRegionName, sn)) {
899 deleteMergingNode(encodedName, sn);
900 }
901 break;
902 default:
903 throw new IllegalStateException("Received region in state:" + et + " is not valid.");
904 }
905 LOG.info("Processed region " + prettyPrintedRegionName + " in state "
906 + et + ", on " + (serverManager.isServerOnline(sn) ? "" : "dead ")
907 + "server: " + sn);
908 return true;
909 }
910
911
912
913
914
915 public void removeClosedRegion(HRegionInfo hri) {
916 if (regionsToReopen.remove(hri.getEncodedName()) != null) {
917 LOG.debug("Removed region from reopening regions because it was closed");
918 }
919 }
920
921
922
923
924
925
926
927
928
929
930
931
932 void handleRegion(final RegionTransition rt, OpenRegionCoordination coordination,
933 OpenRegionCoordination.OpenRegionDetails ord) {
934 if (rt == null) {
935 LOG.warn("Unexpected NULL input for RegionTransition rt");
936 return;
937 }
938 final ServerName sn = rt.getServerName();
939
940 if (sn.equals(HBCK_CODE_SERVERNAME)) {
941 handleHBCK(rt);
942 return;
943 }
944 final long createTime = rt.getCreateTime();
945 final byte[] regionName = rt.getRegionName();
946 String encodedName = HRegionInfo.encodeRegionName(regionName);
947 String prettyPrintedRegionName = HRegionInfo.prettyPrint(encodedName);
948
949 if (!serverManager.isServerOnline(sn)
950 && !ignoreStatesRSOffline.contains(rt.getEventType())) {
951 LOG.warn("Attempted to handle region transition for server but " +
952 "it is not online: " + prettyPrintedRegionName + ", " + rt);
953 return;
954 }
955
956 RegionState regionState =
957 regionStates.getRegionState(encodedName);
958 long startTime = System.currentTimeMillis();
959 if (LOG.isDebugEnabled()) {
960 boolean lateEvent = createTime < (startTime - 15000);
961 LOG.debug("Handling " + rt.getEventType() +
962 ", server=" + sn + ", region=" +
963 (prettyPrintedRegionName == null ? "null" : prettyPrintedRegionName) +
964 (lateEvent ? ", which is more than 15 seconds late" : "") +
965 ", current_state=" + regionState);
966 }
967
968
969 if (rt.getEventType() == EventType.M_ZK_REGION_OFFLINE) {
970 return;
971 }
972
973
974 Lock lock = locker.acquireLock(encodedName);
975 try {
976 RegionState latestState =
977 regionStates.getRegionState(encodedName);
978 if ((regionState == null && latestState != null)
979 || (regionState != null && latestState == null)
980 || (regionState != null && latestState != null
981 && latestState.getState() != regionState.getState())) {
982 LOG.warn("Region state changed from " + regionState + " to "
983 + latestState + ", while acquiring lock");
984 }
985 long waitedTime = System.currentTimeMillis() - startTime;
986 if (waitedTime > 5000) {
987 LOG.warn("Took " + waitedTime + "ms to acquire the lock");
988 }
989 regionState = latestState;
990 switch (rt.getEventType()) {
991 case RS_ZK_REQUEST_REGION_SPLIT:
992 case RS_ZK_REGION_SPLITTING:
993 case RS_ZK_REGION_SPLIT:
994 if (!handleRegionSplitting(
995 rt, encodedName, prettyPrintedRegionName, sn)) {
996 deleteSplittingNode(encodedName, sn);
997 }
998 break;
999
1000 case RS_ZK_REQUEST_REGION_MERGE:
1001 case RS_ZK_REGION_MERGING:
1002 case RS_ZK_REGION_MERGED:
1003
1004
1005 if (!handleRegionMerging(
1006 rt, encodedName, prettyPrintedRegionName, sn)) {
1007 deleteMergingNode(encodedName, sn);
1008 }
1009 break;
1010
1011 case M_ZK_REGION_CLOSING:
1012
1013
1014 if (regionState == null
1015 || !regionState.isPendingCloseOrClosingOnServer(sn)) {
1016 LOG.warn("Received CLOSING for " + prettyPrintedRegionName
1017 + " from " + sn + " but the region isn't PENDING_CLOSE/CLOSING here: "
1018 + regionStates.getRegionState(encodedName));
1019 return;
1020 }
1021
1022 regionStates.updateRegionState(rt, State.CLOSING);
1023 break;
1024
1025 case RS_ZK_REGION_CLOSED:
1026
1027 if (regionState == null
1028 || !regionState.isPendingCloseOrClosingOnServer(sn)) {
1029 LOG.warn("Received CLOSED for " + prettyPrintedRegionName
1030 + " from " + sn + " but the region isn't PENDING_CLOSE/CLOSING here: "
1031 + regionStates.getRegionState(encodedName));
1032 return;
1033 }
1034
1035
1036
1037 new ClosedRegionHandler(server, this, regionState.getRegion()).process();
1038 updateClosedRegionHandlerTracker(regionState.getRegion());
1039 break;
1040
1041 case RS_ZK_REGION_FAILED_OPEN:
1042 if (regionState == null
1043 || !regionState.isPendingOpenOrOpeningOnServer(sn)) {
1044 LOG.warn("Received FAILED_OPEN for " + prettyPrintedRegionName
1045 + " from " + sn + " but the region isn't PENDING_OPEN/OPENING here: "
1046 + regionStates.getRegionState(encodedName));
1047 return;
1048 }
1049 AtomicInteger failedOpenCount = failedOpenTracker.get(encodedName);
1050 if (failedOpenCount == null) {
1051 failedOpenCount = new AtomicInteger();
1052
1053
1054
1055 failedOpenTracker.put(encodedName, failedOpenCount);
1056 }
1057 if (failedOpenCount.incrementAndGet() >= maximumAttempts) {
1058 regionStates.updateRegionState(rt, State.FAILED_OPEN);
1059
1060
1061 failedOpenTracker.remove(encodedName);
1062 } else {
1063
1064 regionState = regionStates.updateRegionState(rt, State.CLOSED);
1065 if (regionState != null) {
1066
1067
1068 try {
1069 getRegionPlan(regionState.getRegion(), sn, true);
1070 new ClosedRegionHandler(server, this, regionState.getRegion()).process();
1071 } catch (HBaseIOException e) {
1072 LOG.warn("Failed to get region plan", e);
1073 }
1074 }
1075 }
1076 break;
1077
1078 case RS_ZK_REGION_OPENING:
1079
1080
1081 if (regionState == null
1082 || !regionState.isPendingOpenOrOpeningOnServer(sn)) {
1083 LOG.warn("Received OPENING for " + prettyPrintedRegionName
1084 + " from " + sn + " but the region isn't PENDING_OPEN/OPENING here: "
1085 + regionStates.getRegionState(encodedName));
1086 return;
1087 }
1088
1089 regionStates.updateRegionState(rt, State.OPENING);
1090 break;
1091
1092 case RS_ZK_REGION_OPENED:
1093
1094 if (regionState == null
1095 || !regionState.isPendingOpenOrOpeningOnServer(sn)) {
1096 LOG.warn("Received OPENED for " + prettyPrintedRegionName
1097 + " from " + sn + " but the region isn't PENDING_OPEN/OPENING here: "
1098 + regionStates.getRegionState(encodedName));
1099
1100 if (regionState != null) {
1101
1102
1103
1104 unassign(regionState.getRegion(), null, -1, null, false, sn);
1105 }
1106 return;
1107 }
1108
1109 regionState =
1110 regionStates.transitionOpenFromPendingOpenOrOpeningOnServer(rt,regionState, sn);
1111 if (regionState != null) {
1112 failedOpenTracker.remove(encodedName);
1113 new OpenedRegionHandler(
1114 server, this, regionState.getRegion(), coordination, ord).process();
1115 updateOpenedRegionHandlerTracker(regionState.getRegion());
1116 }
1117 break;
1118
1119 default:
1120 throw new IllegalStateException("Received event is not valid.");
1121 }
1122 } finally {
1123 lock.unlock();
1124 }
1125 }
1126
1127
1128 boolean wasClosedHandlerCalled(HRegionInfo hri) {
1129 AtomicBoolean b = closedRegionHandlerCalled.get(hri);
1130
1131
1132
1133 return b == null ? false : b.compareAndSet(true, false);
1134 }
1135
1136
1137 boolean wasOpenedHandlerCalled(HRegionInfo hri) {
1138 AtomicBoolean b = openedRegionHandlerCalled.get(hri);
1139
1140
1141
1142 return b == null ? false : b.compareAndSet(true, false);
1143 }
1144
1145
1146 void initializeHandlerTrackers() {
1147 closedRegionHandlerCalled = new HashMap<HRegionInfo, AtomicBoolean>();
1148 openedRegionHandlerCalled = new HashMap<HRegionInfo, AtomicBoolean>();
1149 }
1150
1151 void updateClosedRegionHandlerTracker(HRegionInfo hri) {
1152 if (closedRegionHandlerCalled != null) {
1153 closedRegionHandlerCalled.put(hri, new AtomicBoolean(true));
1154 }
1155 }
1156
1157 void updateOpenedRegionHandlerTracker(HRegionInfo hri) {
1158 if (openedRegionHandlerCalled != null) {
1159 openedRegionHandlerCalled.put(hri, new AtomicBoolean(true));
1160 }
1161 }
1162
1163
1164
1165
1166
1167
1168 void processFavoredNodes(List<HRegionInfo> regions) throws IOException {
1169 if (!shouldAssignRegionsWithFavoredNodes) return;
1170
1171
1172 Map<HRegionInfo, List<ServerName>> regionToFavoredNodes =
1173 new HashMap<HRegionInfo, List<ServerName>>();
1174 for (HRegionInfo region : regions) {
1175 regionToFavoredNodes.put(region,
1176 ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region));
1177 }
1178 FavoredNodeAssignmentHelper.updateMetaWithFavoredNodesInfo(regionToFavoredNodes,
1179 this.server.getConnection());
1180 }
1181
1182
1183
1184
1185
1186
1187
1188 @SuppressWarnings("deprecation")
1189 private void handleHBCK(RegionTransition rt) {
1190 String encodedName = HRegionInfo.encodeRegionName(rt.getRegionName());
1191 LOG.info("Handling HBCK triggered transition=" + rt.getEventType() +
1192 ", server=" + rt.getServerName() + ", region=" +
1193 HRegionInfo.prettyPrint(encodedName));
1194 RegionState regionState = regionStates.getRegionTransitionState(encodedName);
1195 switch (rt.getEventType()) {
1196 case M_ZK_REGION_OFFLINE:
1197 HRegionInfo regionInfo;
1198 if (regionState != null) {
1199 regionInfo = regionState.getRegion();
1200 } else {
1201 try {
1202 byte [] name = rt.getRegionName();
1203 Pair<HRegionInfo, ServerName> p = MetaTableAccessor.getRegion(
1204 this.server.getConnection(), name);
1205 regionInfo = p.getFirst();
1206 } catch (IOException e) {
1207 LOG.info("Exception reading hbase:meta doing HBCK repair operation", e);
1208 return;
1209 }
1210 }
1211 LOG.info("HBCK repair is triggering assignment of region=" +
1212 regionInfo.getRegionNameAsString());
1213
1214 assign(regionInfo, false);
1215 break;
1216
1217 default:
1218 LOG.warn("Received unexpected region state from HBCK: " + rt.toString());
1219 break;
1220 }
1221
1222 }
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238 @Override
1239 public void nodeCreated(String path) {
1240 handleAssignmentEvent(path);
1241 }
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255 @Override
1256 public void nodeDataChanged(String path) {
1257 handleAssignmentEvent(path);
1258 }
1259
1260
1261
1262
1263
1264 private final Set<String> regionsInProgress = new HashSet<String>();
1265
1266
1267 private final LinkedHashMultimap <String, RegionRunnable>
1268 zkEventWorkerWaitingList = LinkedHashMultimap.create();
1269
1270
1271
1272
1273 private interface RegionRunnable extends Runnable{
1274
1275
1276
1277 String getRegionName();
1278 }
1279
1280
1281
1282
1283
1284 protected void zkEventWorkersSubmit(final RegionRunnable regRunnable) {
1285
1286 synchronized (regionsInProgress) {
1287
1288
1289 if (regionsInProgress.contains(regRunnable.getRegionName())) {
1290 synchronized (zkEventWorkerWaitingList){
1291 zkEventWorkerWaitingList.put(regRunnable.getRegionName(), regRunnable);
1292 }
1293 return;
1294 }
1295
1296
1297 regionsInProgress.add(regRunnable.getRegionName());
1298 zkEventWorkers.submit(new Runnable() {
1299 @Override
1300 public void run() {
1301 try {
1302 regRunnable.run();
1303 } finally {
1304
1305
1306 synchronized (regionsInProgress) {
1307 regionsInProgress.remove(regRunnable.getRegionName());
1308 synchronized (zkEventWorkerWaitingList) {
1309 java.util.Set<RegionRunnable> waiting = zkEventWorkerWaitingList.get(
1310 regRunnable.getRegionName());
1311 if (!waiting.isEmpty()) {
1312
1313 RegionRunnable toSubmit = waiting.iterator().next();
1314 zkEventWorkerWaitingList.remove(toSubmit.getRegionName(), toSubmit);
1315 zkEventWorkersSubmit(toSubmit);
1316 }
1317 }
1318 }
1319 }
1320 }
1321 });
1322 }
1323 }
1324
1325 @Override
1326 public void nodeDeleted(final String path) {
1327 if (path.startsWith(watcher.assignmentZNode)) {
1328 final String regionName = ZKAssign.getRegionName(watcher, path);
1329 zkEventWorkersSubmit(new RegionRunnable() {
1330 @Override
1331 public String getRegionName() {
1332 return regionName;
1333 }
1334
1335 @Override
1336 public void run() {
1337 Lock lock = locker.acquireLock(regionName);
1338 try {
1339 RegionState rs = regionStates.getRegionTransitionState(regionName);
1340 if (rs == null) {
1341 rs = regionStates.getRegionState(regionName);
1342 if (rs == null || !rs.isMergingNew()) {
1343
1344 return;
1345 }
1346 }
1347
1348 HRegionInfo regionInfo = rs.getRegion();
1349 String regionNameStr = regionInfo.getRegionNameAsString();
1350 LOG.debug("Znode " + regionNameStr + " deleted, state: " + rs);
1351
1352 boolean disabled = getTableStateManager().isTableState(regionInfo.getTable(),
1353 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING);
1354
1355 ServerName serverName = rs.getServerName();
1356 if (serverManager.isServerOnline(serverName)) {
1357 if (rs.isOnServer(serverName) && (rs.isOpened() || rs.isSplitting())) {
1358 synchronized (regionStates) {
1359 regionOnline(regionInfo, serverName);
1360 if (rs.isSplitting() && splitRegions.containsKey(regionInfo)) {
1361
1362
1363 HRegionInfo hri_a = splitRegions.get(regionInfo).getFirst();
1364 HRegionInfo hri_b = splitRegions.get(regionInfo).getSecond();
1365 if (!regionStates.isRegionInTransition(hri_a.getEncodedName())) {
1366 LOG.warn("Split daughter region not in transition " + hri_a);
1367 }
1368 if (!regionStates.isRegionInTransition(hri_b.getEncodedName())) {
1369 LOG.warn("Split daughter region not in transition" + hri_b);
1370 }
1371 regionOffline(hri_a);
1372 regionOffline(hri_b);
1373 splitRegions.remove(regionInfo);
1374 }
1375 if (disabled) {
1376
1377 LOG.info("Opened " + regionNameStr
1378 + "but this table is disabled, triggering close of region");
1379 unassign(regionInfo);
1380 }
1381 }
1382 } else if (rs.isMergingNew()) {
1383 synchronized (regionStates) {
1384 String p = regionInfo.getEncodedName();
1385 PairOfSameType<HRegionInfo> regions = mergingRegions.get(p);
1386 if (regions != null) {
1387 onlineMergingRegion(disabled, regions.getFirst(), serverName);
1388 onlineMergingRegion(disabled, regions.getSecond(), serverName);
1389 }
1390 }
1391 }
1392 }
1393 } finally {
1394 lock.unlock();
1395 }
1396 }
1397
1398 private void onlineMergingRegion(boolean disabled,
1399 final HRegionInfo hri, final ServerName serverName) {
1400 RegionState regionState = regionStates.getRegionState(hri);
1401 if (regionState != null && regionState.isMerging()
1402 && regionState.isOnServer(serverName)) {
1403 regionOnline(regionState.getRegion(), serverName);
1404 if (disabled) {
1405 unassign(hri);
1406 }
1407 }
1408 }
1409 });
1410 }
1411 }
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425 @Override
1426 public void nodeChildrenChanged(String path) {
1427 if (path.equals(watcher.assignmentZNode)) {
1428 zkEventWorkers.submit(new Runnable() {
1429 @Override
1430 public void run() {
1431 try {
1432
1433 List<String> children =
1434 ZKUtil.listChildrenAndWatchForNewChildren(
1435 watcher, watcher.assignmentZNode);
1436 if (children != null) {
1437 Stat stat = new Stat();
1438 for (String child : children) {
1439
1440
1441
1442 if (!regionStates.isRegionInTransition(child)) {
1443 ZKAssign.getDataAndWatch(watcher, child, stat);
1444 }
1445 }
1446 }
1447 } catch (KeeperException e) {
1448 server.abort("Unexpected ZK exception reading unassigned children", e);
1449 }
1450 }
1451 });
1452 }
1453 }
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464 void regionOnline(HRegionInfo regionInfo, ServerName sn) {
1465 regionOnline(regionInfo, sn, HConstants.NO_SEQNUM);
1466 }
1467
1468 void regionOnline(HRegionInfo regionInfo, ServerName sn, long openSeqNum) {
1469 numRegionsOpened.incrementAndGet();
1470 regionStates.regionOnline(regionInfo, sn, openSeqNum);
1471
1472
1473 clearRegionPlan(regionInfo);
1474 balancer.regionOnline(regionInfo, sn);
1475
1476
1477 sendRegionOpenedNotification(regionInfo, sn);
1478 }
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488 private void handleAssignmentEvent(final String path) {
1489 if (path.startsWith(watcher.assignmentZNode)) {
1490 final String regionName = ZKAssign.getRegionName(watcher, path);
1491
1492 zkEventWorkersSubmit(new RegionRunnable() {
1493 @Override
1494 public String getRegionName() {
1495 return regionName;
1496 }
1497
1498 @Override
1499 public void run() {
1500 try {
1501 Stat stat = new Stat();
1502 byte [] data = ZKAssign.getDataAndWatch(watcher, path, stat);
1503 if (data == null) return;
1504
1505 RegionTransition rt = RegionTransition.parseFrom(data);
1506
1507
1508
1509 BaseCoordinatedStateManager csm =
1510 (BaseCoordinatedStateManager) server.getCoordinatedStateManager();
1511 OpenRegionCoordination openRegionCoordination = csm.getOpenRegionCoordination();
1512
1513 ZkOpenRegionCoordination.ZkOpenRegionDetails zkOrd =
1514 new ZkOpenRegionCoordination.ZkOpenRegionDetails();
1515 zkOrd.setVersion(stat.getVersion());
1516 zkOrd.setServerName(csm.getServer().getServerName());
1517
1518 handleRegion(rt, openRegionCoordination, zkOrd);
1519 } catch (KeeperException e) {
1520 server.abort("Unexpected ZK exception reading unassigned node data", e);
1521 } catch (DeserializationException e) {
1522 server.abort("Unexpected exception deserializing node data", e);
1523 }
1524 }
1525 });
1526 }
1527 }
1528
1529
1530
1531
1532
1533
1534
1535
1536 public void regionOffline(final HRegionInfo regionInfo) {
1537 regionOffline(regionInfo, null);
1538 }
1539
1540 public void offlineDisabledRegion(HRegionInfo regionInfo) {
1541 if (useZKForAssignment) {
1542
1543 LOG.debug("Table being disabled so deleting ZK node and removing from " +
1544 "regions in transition, skipping assignment of region " +
1545 regionInfo.getRegionNameAsString());
1546 String encodedName = regionInfo.getEncodedName();
1547 deleteNodeInStates(encodedName, "closed", null,
1548 EventType.RS_ZK_REGION_CLOSED, EventType.M_ZK_REGION_OFFLINE);
1549 }
1550 replicasToClose.remove(regionInfo);
1551 regionOffline(regionInfo);
1552 }
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574 public void assign(HRegionInfo region, boolean setOfflineInZK) {
1575 assign(region, setOfflineInZK, false);
1576 }
1577
1578
1579
1580
1581 public void assign(HRegionInfo region,
1582 boolean setOfflineInZK, boolean forceNewPlan) {
1583 if (isDisabledorDisablingRegionInRIT(region)) {
1584 return;
1585 }
1586 String encodedName = region.getEncodedName();
1587 Lock lock = locker.acquireLock(encodedName);
1588 try {
1589 RegionState state = forceRegionStateToOffline(region, forceNewPlan);
1590 if (state != null) {
1591 if (regionStates.wasRegionOnDeadServer(encodedName)) {
1592 LOG.info("Skip assigning " + region.getRegionNameAsString()
1593 + ", it's host " + regionStates.getLastRegionServerOfRegion(encodedName)
1594 + " is dead but not processed yet");
1595 return;
1596 }
1597 assign(state, setOfflineInZK && useZKForAssignment, forceNewPlan);
1598 }
1599 } finally {
1600 lock.unlock();
1601 }
1602 }
1603
1604
1605
1606
1607
1608
1609
1610 boolean assign(final ServerName destination, final List<HRegionInfo> regions)
1611 throws InterruptedException {
1612 long startTime = EnvironmentEdgeManager.currentTime();
1613 try {
1614 int regionCount = regions.size();
1615 if (regionCount == 0) {
1616 return true;
1617 }
1618 LOG.info("Assigning " + regionCount + " region(s) to " + destination.toString());
1619 Set<String> encodedNames = new HashSet<String>(regionCount);
1620 for (HRegionInfo region : regions) {
1621 encodedNames.add(region.getEncodedName());
1622 }
1623
1624 List<HRegionInfo> failedToOpenRegions = new ArrayList<HRegionInfo>();
1625 Map<String, Lock> locks = locker.acquireLocks(encodedNames);
1626 try {
1627 AtomicInteger counter = new AtomicInteger(0);
1628 Map<String, Integer> offlineNodesVersions = new ConcurrentHashMap<String, Integer>();
1629 OfflineCallback cb = new OfflineCallback(
1630 watcher, destination, counter, offlineNodesVersions);
1631 Map<String, RegionPlan> plans = new HashMap<String, RegionPlan>(regions.size());
1632 List<RegionState> states = new ArrayList<RegionState>(regions.size());
1633 for (HRegionInfo region : regions) {
1634 String encodedName = region.getEncodedName();
1635 if (!isDisabledorDisablingRegionInRIT(region)) {
1636 RegionState state = forceRegionStateToOffline(region, false);
1637 boolean onDeadServer = false;
1638 if (state != null) {
1639 if (regionStates.wasRegionOnDeadServer(encodedName)) {
1640 LOG.info("Skip assigning " + region.getRegionNameAsString()
1641 + ", it's host " + regionStates.getLastRegionServerOfRegion(encodedName)
1642 + " is dead but not processed yet");
1643 onDeadServer = true;
1644 } else if (!useZKForAssignment
1645 || asyncSetOfflineInZooKeeper(state, cb, destination)) {
1646 RegionPlan plan = new RegionPlan(region, state.getServerName(), destination);
1647 plans.put(encodedName, plan);
1648 states.add(state);
1649 continue;
1650 }
1651 }
1652
1653 if (!onDeadServer) {
1654 LOG.info("failed to force region state to offline or "
1655 + "failed to set it offline in ZK, will reassign later: " + region);
1656 failedToOpenRegions.add(region);
1657 }
1658 }
1659
1660
1661 Lock lock = locks.remove(encodedName);
1662 lock.unlock();
1663 }
1664
1665 if (useZKForAssignment) {
1666
1667 int total = states.size();
1668 for (int oldCounter = 0; !server.isStopped();) {
1669 int count = counter.get();
1670 if (oldCounter != count) {
1671 LOG.debug(destination.toString() + " unassigned znodes=" + count +
1672 " of total=" + total + "; oldCounter=" + oldCounter);
1673 oldCounter = count;
1674 }
1675 if (count >= total) break;
1676 Thread.sleep(5);
1677 }
1678 }
1679
1680 if (server.isStopped()) {
1681 return false;
1682 }
1683
1684
1685
1686 this.addPlans(plans);
1687
1688 List<Triple<HRegionInfo, Integer, List<ServerName>>> regionOpenInfos =
1689 new ArrayList<Triple<HRegionInfo, Integer, List<ServerName>>>(states.size());
1690 for (RegionState state: states) {
1691 HRegionInfo region = state.getRegion();
1692 String encodedRegionName = region.getEncodedName();
1693 Integer nodeVersion = offlineNodesVersions.get(encodedRegionName);
1694 if (useZKForAssignment && (nodeVersion == null || nodeVersion == -1)) {
1695 LOG.warn("failed to offline in zookeeper: " + region);
1696 failedToOpenRegions.add(region);
1697 Lock lock = locks.remove(encodedRegionName);
1698 lock.unlock();
1699 } else {
1700 regionStates.updateRegionState(
1701 region, State.PENDING_OPEN, destination);
1702 List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
1703 if (this.shouldAssignRegionsWithFavoredNodes) {
1704 favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
1705 }
1706 regionOpenInfos.add(new Triple<HRegionInfo, Integer, List<ServerName>>(
1707 region, nodeVersion, favoredNodes));
1708 }
1709 }
1710
1711
1712 try {
1713
1714
1715 long maxWaitTime = System.currentTimeMillis() +
1716 this.server.getConfiguration().
1717 getLong("hbase.regionserver.rpc.startup.waittime", 60000);
1718 for (int i = 1; i <= maximumAttempts && !server.isStopped(); i++) {
1719 try {
1720
1721 if (regionOpenInfos.isEmpty()) {
1722 break;
1723 }
1724 List<RegionOpeningState> regionOpeningStateList = serverManager
1725 .sendRegionOpen(destination, regionOpenInfos);
1726 if (regionOpeningStateList == null) {
1727
1728 return false;
1729 }
1730 for (int k = 0, n = regionOpeningStateList.size(); k < n; k++) {
1731 RegionOpeningState openingState = regionOpeningStateList.get(k);
1732 if (openingState != RegionOpeningState.OPENED) {
1733 HRegionInfo region = regionOpenInfos.get(k).getFirst();
1734 if (openingState == RegionOpeningState.ALREADY_OPENED) {
1735 processAlreadyOpenedRegion(region, destination);
1736 } else if (openingState == RegionOpeningState.FAILED_OPENING) {
1737
1738 failedToOpenRegions.add(region);
1739 } else {
1740 LOG.warn("THIS SHOULD NOT HAPPEN: unknown opening state "
1741 + openingState + " in assigning region " + region);
1742 }
1743 }
1744 }
1745 break;
1746 } catch (IOException e) {
1747 if (e instanceof RemoteException) {
1748 e = ((RemoteException)e).unwrapRemoteException();
1749 }
1750 if (e instanceof RegionServerStoppedException) {
1751 LOG.warn("The region server was shut down, ", e);
1752
1753 return false;
1754 } else if (e instanceof ServerNotRunningYetException) {
1755 long now = System.currentTimeMillis();
1756 if (now < maxWaitTime) {
1757 LOG.debug("Server is not yet up; waiting up to " +
1758 (maxWaitTime - now) + "ms", e);
1759 Thread.sleep(100);
1760 i--;
1761 continue;
1762 }
1763 } else if (e instanceof java.net.SocketTimeoutException
1764 && this.serverManager.isServerOnline(destination)) {
1765
1766
1767
1768
1769 if (LOG.isDebugEnabled()) {
1770 LOG.debug("Bulk assigner openRegion() to " + destination
1771 + " has timed out, but the regions might"
1772 + " already be opened on it.", e);
1773 }
1774
1775 Thread.sleep(100);
1776 i--;
1777 continue;
1778 }
1779 throw e;
1780 }
1781 }
1782 } catch (IOException e) {
1783
1784 LOG.info("Unable to communicate with " + destination
1785 + " in order to assign regions, ", e);
1786 return false;
1787 }
1788 } finally {
1789 for (Lock lock : locks.values()) {
1790 lock.unlock();
1791 }
1792 }
1793
1794 if (!failedToOpenRegions.isEmpty()) {
1795 for (HRegionInfo region : failedToOpenRegions) {
1796 if (!regionStates.isRegionOnline(region)) {
1797 invokeAssign(region);
1798 }
1799 }
1800 }
1801
1802
1803 ArrayList<HRegionInfo> userRegionSet = new ArrayList<HRegionInfo>(regions.size());
1804 for (HRegionInfo region: regions) {
1805 if (!region.getTable().isSystemTable()) {
1806 userRegionSet.add(region);
1807 }
1808 }
1809 if (!waitForAssignment(userRegionSet, true, userRegionSet.size(),
1810 System.currentTimeMillis())) {
1811 LOG.debug("some user regions are still in transition: " + userRegionSet);
1812 }
1813 LOG.debug("Bulk assigning done for " + destination);
1814 return true;
1815 } finally {
1816 metricsAssignmentManager.updateBulkAssignTime(EnvironmentEdgeManager.currentTime() - startTime);
1817 }
1818 }
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830 private void unassign(final HRegionInfo region,
1831 final RegionState state, final int versionOfClosingNode,
1832 final ServerName dest, final boolean transitionInZK,
1833 final ServerName src) {
1834 ServerName server = src;
1835 if (state != null) {
1836 server = state.getServerName();
1837 }
1838 long maxWaitTime = -1;
1839 for (int i = 1; i <= this.maximumAttempts; i++) {
1840 if (this.server.isStopped() || this.server.isAborted()) {
1841 LOG.debug("Server stopped/aborted; skipping unassign of " + region);
1842 return;
1843 }
1844
1845 if (!serverManager.isServerOnline(server)) {
1846 LOG.debug("Offline " + region.getRegionNameAsString()
1847 + ", no need to unassign since it's on a dead server: " + server);
1848 if (transitionInZK) {
1849
1850 deleteClosingOrClosedNode(region, server);
1851 }
1852 if (state != null) {
1853 regionOffline(region);
1854 }
1855 return;
1856 }
1857 try {
1858
1859 if (serverManager.sendRegionClose(server, region,
1860 versionOfClosingNode, dest, transitionInZK)) {
1861 LOG.debug("Sent CLOSE to " + server + " for region " +
1862 region.getRegionNameAsString());
1863 if (useZKForAssignment && !transitionInZK && state != null) {
1864
1865
1866 unassign(region, state, versionOfClosingNode,
1867 dest, transitionInZK, src);
1868 }
1869 return;
1870 }
1871
1872
1873 LOG.warn("Server " + server + " region CLOSE RPC returned false for " +
1874 region.getRegionNameAsString());
1875 } catch (Throwable t) {
1876 long sleepTime = 0;
1877 Configuration conf = this.server.getConfiguration();
1878 if (t instanceof RemoteException) {
1879 t = ((RemoteException)t).unwrapRemoteException();
1880 }
1881 boolean logRetries = true;
1882 if (t instanceof RegionServerAbortedException
1883 || t instanceof RegionServerStoppedException
1884 || t instanceof ServerNotRunningYetException) {
1885
1886
1887 sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
1888 RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
1889
1890 } else if (t instanceof NotServingRegionException) {
1891 LOG.debug("Offline " + region.getRegionNameAsString()
1892 + ", it's not any more on " + server, t);
1893 if (transitionInZK) {
1894 deleteClosingOrClosedNode(region, server);
1895 }
1896 if (state != null) {
1897 regionOffline(region);
1898 }
1899 return;
1900 } else if ((t instanceof FailedServerException) || (state != null &&
1901 t instanceof RegionAlreadyInTransitionException)) {
1902 if(t instanceof FailedServerException) {
1903 sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
1904 RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
1905 } else {
1906
1907 LOG.debug("update " + state + " the timestamp.");
1908 state.updateTimestampToNow();
1909 if (maxWaitTime < 0) {
1910 maxWaitTime =
1911 EnvironmentEdgeManager.currentTime()
1912 + conf.getLong(ALREADY_IN_TRANSITION_WAITTIME,
1913 DEFAULT_ALREADY_IN_TRANSITION_WAITTIME);
1914 }
1915 long now = EnvironmentEdgeManager.currentTime();
1916 if (now < maxWaitTime) {
1917 LOG.debug("Region is already in transition; "
1918 + "waiting up to " + (maxWaitTime - now) + "ms", t);
1919 sleepTime = 100;
1920 i--;
1921 logRetries = false;
1922 }
1923 }
1924 }
1925
1926 try {
1927 if (sleepTime > 0) {
1928 Thread.sleep(sleepTime);
1929 }
1930 } catch (InterruptedException ie) {
1931 LOG.warn("Failed to unassign "
1932 + region.getRegionNameAsString() + " since interrupted", ie);
1933 Thread.currentThread().interrupt();
1934 if (state != null) {
1935 regionStates.updateRegionState(region, State.FAILED_CLOSE);
1936 }
1937 return;
1938 }
1939
1940 if (logRetries) {
1941 LOG.info("Server " + server + " returned " + t + " for "
1942 + region.getRegionNameAsString() + ", try=" + i
1943 + " of " + this.maximumAttempts, t);
1944
1945 }
1946 }
1947 }
1948
1949 if (state != null) {
1950 regionStates.updateRegionState(region, State.FAILED_CLOSE);
1951 }
1952 }
1953
1954
1955
1956
1957 private RegionState forceRegionStateToOffline(
1958 final HRegionInfo region, final boolean forceNewPlan) {
1959 RegionState state = regionStates.getRegionState(region);
1960 if (state == null) {
1961 LOG.warn("Assigning but not in region states: " + region);
1962 state = regionStates.createRegionState(region);
1963 }
1964
1965 ServerName sn = state.getServerName();
1966 if (forceNewPlan && LOG.isDebugEnabled()) {
1967 LOG.debug("Force region state offline " + state);
1968 }
1969
1970 switch (state.getState()) {
1971 case OPEN:
1972 case OPENING:
1973 case PENDING_OPEN:
1974 case CLOSING:
1975 case PENDING_CLOSE:
1976 if (!forceNewPlan) {
1977 LOG.debug("Skip assigning " +
1978 region + ", it is already " + state);
1979 return null;
1980 }
1981 case FAILED_CLOSE:
1982 case FAILED_OPEN:
1983 unassign(region, state, -1, null, false, null);
1984 state = regionStates.getRegionState(region);
1985 if (state.isFailedClose()) {
1986
1987
1988 LOG.info("Skip assigning " +
1989 region + ", we couldn't close it: " + state);
1990 return null;
1991 }
1992 case OFFLINE:
1993
1994
1995
1996
1997
1998
1999
2000
2001 if (useZKForAssignment
2002 && regionStates.isServerDeadAndNotProcessed(sn)
2003 && wasRegionOnDeadServerByMeta(region, sn)) {
2004 if (!regionStates.isRegionInTransition(region)) {
2005 LOG.info("Updating the state to " + State.OFFLINE + " to allow to be reassigned by SSH");
2006 regionStates.updateRegionState(region, State.OFFLINE);
2007 }
2008 LOG.info("Skip assigning " + region.getRegionNameAsString()
2009 + ", it is on a dead but not processed yet server: " + sn);
2010 return null;
2011 }
2012 case CLOSED:
2013 break;
2014 default:
2015 LOG.error("Trying to assign region " + region
2016 + ", which is " + state);
2017 return null;
2018 }
2019 return state;
2020 }
2021
2022 @SuppressWarnings("deprecation")
2023 protected boolean wasRegionOnDeadServerByMeta(
2024 final HRegionInfo region, final ServerName sn) {
2025 try {
2026 if (region.isMetaRegion()) {
2027 ServerName server = this.server.getMetaTableLocator().
2028 getMetaRegionLocation(this.server.getZooKeeper());
2029 return regionStates.isServerDeadAndNotProcessed(server);
2030 }
2031 while (!server.isStopped()) {
2032 try {
2033 this.server.getMetaTableLocator().waitMetaRegionLocation(server.getZooKeeper());
2034 Result r = MetaTableAccessor.getRegionResult(server.getConnection(),
2035 region.getRegionName());
2036 if (r == null || r.isEmpty()) return false;
2037 ServerName server = HRegionInfo.getServerName(r);
2038 return regionStates.isServerDeadAndNotProcessed(server);
2039 } catch (IOException ioe) {
2040 LOG.info("Received exception accessing hbase:meta during force assign "
2041 + region.getRegionNameAsString() + ", retrying", ioe);
2042 }
2043 }
2044 } catch (InterruptedException e) {
2045 Thread.currentThread().interrupt();
2046 LOG.info("Interrupted accessing hbase:meta", e);
2047 }
2048
2049 return regionStates.isServerDeadAndNotProcessed(sn);
2050 }
2051
2052
2053
2054
2055
2056
2057
2058 private void assign(RegionState state,
2059 boolean setOfflineInZK, final boolean forceNewPlan) {
2060 long startTime = EnvironmentEdgeManager.currentTime();
2061 try {
2062 Configuration conf = server.getConfiguration();
2063 RegionState currentState = state;
2064 int versionOfOfflineNode = -1;
2065 RegionPlan plan = null;
2066 long maxWaitTime = -1;
2067 HRegionInfo region = state.getRegion();
2068 RegionOpeningState regionOpenState;
2069 Throwable previousException = null;
2070 for (int i = 1; i <= maximumAttempts; i++) {
2071 if (server.isStopped() || server.isAborted()) {
2072 LOG.info("Skip assigning " + region.getRegionNameAsString()
2073 + ", the server is stopped/aborted");
2074 return;
2075 }
2076
2077 if (plan == null) {
2078 try {
2079 plan = getRegionPlan(region, forceNewPlan);
2080 } catch (HBaseIOException e) {
2081 LOG.warn("Failed to get region plan", e);
2082 }
2083 }
2084
2085 if (plan == null) {
2086 LOG.warn("Unable to determine a plan to assign " + region);
2087
2088
2089 if (region.isMetaRegion()) {
2090 if (i == maximumAttempts) {
2091 i = 0;
2092
2093 LOG.warn("Unable to determine a plan to assign a hbase:meta region " + region +
2094 " after maximumAttempts (" + this.maximumAttempts +
2095 "). Reset attempts count and continue retrying.");
2096 }
2097 waitForRetryingMetaAssignment();
2098 continue;
2099 }
2100
2101 regionStates.updateRegionState(region, State.FAILED_OPEN);
2102 return;
2103 }
2104 if (setOfflineInZK && versionOfOfflineNode == -1) {
2105 LOG.info("Setting node as OFFLINED in ZooKeeper for region " + region);
2106
2107
2108 versionOfOfflineNode = setOfflineInZooKeeper(currentState, plan.getDestination());
2109 if (versionOfOfflineNode != -1) {
2110 if (isDisabledorDisablingRegionInRIT(region)) {
2111 return;
2112 }
2113
2114
2115
2116
2117
2118
2119 TableName tableName = region.getTable();
2120 if (!tableStateManager.isTableState(tableName,
2121 ZooKeeperProtos.Table.State.ENABLED, ZooKeeperProtos.Table.State.ENABLING)) {
2122 LOG.debug("Setting table " + tableName + " to ENABLED state.");
2123 setEnabledTable(tableName);
2124 }
2125 }
2126 }
2127 if (setOfflineInZK && versionOfOfflineNode == -1) {
2128 LOG.info("Unable to set offline in ZooKeeper to assign " + region);
2129
2130
2131
2132
2133 if (!server.isAborted()) {
2134 continue;
2135 }
2136 }
2137 LOG.info("Assigning " + region.getRegionNameAsString() +
2138 " to " + plan.getDestination());
2139
2140 currentState = regionStates.updateRegionState(region,
2141 State.PENDING_OPEN, plan.getDestination());
2142
2143 boolean needNewPlan;
2144 final String assignMsg = "Failed assignment of " + region.getRegionNameAsString() +
2145 " to " + plan.getDestination();
2146 try {
2147 List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
2148 if (this.shouldAssignRegionsWithFavoredNodes) {
2149 favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
2150 }
2151 regionOpenState = serverManager.sendRegionOpen(
2152 plan.getDestination(), region, versionOfOfflineNode, favoredNodes);
2153
2154 if (regionOpenState == RegionOpeningState.FAILED_OPENING) {
2155
2156 needNewPlan = true;
2157 LOG.warn(assignMsg + ", regionserver says 'FAILED_OPENING', " +
2158 " trying to assign elsewhere instead; " +
2159 "try=" + i + " of " + this.maximumAttempts);
2160 } else {
2161
2162 if (regionOpenState == RegionOpeningState.ALREADY_OPENED) {
2163 processAlreadyOpenedRegion(region, plan.getDestination());
2164 }
2165 return;
2166 }
2167
2168 } catch (Throwable t) {
2169 if (t instanceof RemoteException) {
2170 t = ((RemoteException) t).unwrapRemoteException();
2171 }
2172 previousException = t;
2173
2174
2175
2176
2177 boolean hold = (t instanceof RegionAlreadyInTransitionException ||
2178 t instanceof ServerNotRunningYetException);
2179
2180
2181
2182
2183
2184
2185 boolean retry = !hold && (t instanceof java.net.SocketTimeoutException
2186 && this.serverManager.isServerOnline(plan.getDestination()));
2187
2188
2189 if (hold) {
2190 LOG.warn(assignMsg + ", waiting a little before trying on the same region server " +
2191 "try=" + i + " of " + this.maximumAttempts, t);
2192
2193 if (maxWaitTime < 0) {
2194 if (t instanceof RegionAlreadyInTransitionException) {
2195 maxWaitTime = EnvironmentEdgeManager.currentTime()
2196 + this.server.getConfiguration().getLong(ALREADY_IN_TRANSITION_WAITTIME,
2197 DEFAULT_ALREADY_IN_TRANSITION_WAITTIME);
2198 } else {
2199 maxWaitTime = EnvironmentEdgeManager.currentTime()
2200 + this.server.getConfiguration().getLong(
2201 "hbase.regionserver.rpc.startup.waittime", 60000);
2202 }
2203 }
2204 try {
2205 needNewPlan = false;
2206 long now = EnvironmentEdgeManager.currentTime();
2207 if (now < maxWaitTime) {
2208 LOG.debug("Server is not yet up or region is already in transition; "
2209 + "waiting up to " + (maxWaitTime - now) + "ms", t);
2210 Thread.sleep(100);
2211 i--;
2212 } else if (!(t instanceof RegionAlreadyInTransitionException)) {
2213 LOG.debug("Server is not up for a while; try a new one", t);
2214 needNewPlan = true;
2215 }
2216 } catch (InterruptedException ie) {
2217 LOG.warn("Failed to assign "
2218 + region.getRegionNameAsString() + " since interrupted", ie);
2219 regionStates.updateRegionState(region, State.FAILED_OPEN);
2220 Thread.currentThread().interrupt();
2221 return;
2222 }
2223 } else if (retry) {
2224 needNewPlan = false;
2225 i--;
2226 LOG.warn(assignMsg + ", trying to assign to the same region server due ", t);
2227 } else {
2228 needNewPlan = true;
2229 LOG.warn(assignMsg + ", trying to assign elsewhere instead;" +
2230 " try=" + i + " of " + this.maximumAttempts, t);
2231 }
2232 }
2233
2234 if (i == this.maximumAttempts) {
2235
2236 if (region.isMetaRegion()) {
2237 i = 0;
2238 LOG.warn(assignMsg +
2239 ", trying to assign a hbase:meta region reached to maximumAttempts (" +
2240 this.maximumAttempts + "). Reset attempt counts and continue retrying.");
2241 waitForRetryingMetaAssignment();
2242 }
2243 else {
2244
2245
2246 continue;
2247 }
2248 }
2249
2250
2251
2252
2253 if (needNewPlan) {
2254
2255
2256
2257
2258 RegionPlan newPlan = null;
2259 try {
2260 newPlan = getRegionPlan(region, true);
2261 } catch (HBaseIOException e) {
2262 LOG.warn("Failed to get region plan", e);
2263 }
2264 if (newPlan == null) {
2265 regionStates.updateRegionState(region, State.FAILED_OPEN);
2266 LOG.warn("Unable to find a viable location to assign region " +
2267 region.getRegionNameAsString());
2268 return;
2269 }
2270
2271 if (plan != newPlan && !plan.getDestination().equals(newPlan.getDestination())) {
2272
2273
2274
2275 LOG.info("Region assignment plan changed from " + plan.getDestination() + " to "
2276 + newPlan.getDestination() + " server.");
2277 currentState = regionStates.updateRegionState(region, State.OFFLINE);
2278 versionOfOfflineNode = -1;
2279 if (useZKForAssignment) {
2280 setOfflineInZK = true;
2281 }
2282 plan = newPlan;
2283 } else if(plan.getDestination().equals(newPlan.getDestination()) &&
2284 previousException instanceof FailedServerException) {
2285 try {
2286 LOG.info("Trying to re-assign " + region.getRegionNameAsString() +
2287 " to the same failed server.");
2288 Thread.sleep(1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
2289 RpcClient.FAILED_SERVER_EXPIRY_DEFAULT));
2290 } catch (InterruptedException ie) {
2291 LOG.warn("Failed to assign "
2292 + region.getRegionNameAsString() + " since interrupted", ie);
2293 regionStates.updateRegionState(region, State.FAILED_OPEN);
2294 Thread.currentThread().interrupt();
2295 return;
2296 }
2297 }
2298 }
2299 }
2300
2301 regionStates.updateRegionState(region, State.FAILED_OPEN);
2302 } finally {
2303 metricsAssignmentManager.updateAssignmentTime(EnvironmentEdgeManager.currentTime() - startTime);
2304 }
2305 }
2306
2307 private void processAlreadyOpenedRegion(HRegionInfo region, ServerName sn) {
2308
2309
2310
2311 LOG.debug("ALREADY_OPENED " + region.getRegionNameAsString()
2312 + " to " + sn);
2313 String encodedName = region.getEncodedName();
2314
2315
2316
2317 if (useZKForAssignment) {
2318 String node = ZKAssign.getNodeName(watcher, encodedName);
2319 Stat stat = new Stat();
2320 try {
2321 byte[] existingBytes = ZKUtil.getDataNoWatch(watcher, node, stat);
2322 if (existingBytes != null) {
2323 RegionTransition rt= RegionTransition.parseFrom(existingBytes);
2324 EventType et = rt.getEventType();
2325 if (et.equals(EventType.RS_ZK_REGION_OPENED)) {
2326 LOG.debug("ALREADY_OPENED " + region.getRegionNameAsString()
2327 + " and node in " + et + " state");
2328 return;
2329 }
2330 }
2331 } catch (KeeperException ke) {
2332 LOG.warn("Unexpected ZK exception getData " + node
2333 + " node for the region " + encodedName, ke);
2334 } catch (DeserializationException e) {
2335 LOG.warn("Get RegionTransition from zk deserialization failed! ", e);
2336 }
2337 deleteNodeInStates(encodedName, "offline", sn, EventType.M_ZK_REGION_OFFLINE);
2338 }
2339
2340 regionStates.regionOnline(region, sn);
2341 }
2342
2343 private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
2344 if (this.tableStateManager.isTableState(region.getTable(),
2345 ZooKeeperProtos.Table.State.DISABLED,
2346 ZooKeeperProtos.Table.State.DISABLING) || replicasToClose.contains(region)) {
2347 LOG.info("Table " + region.getTable() + " is disabled or disabling;"
2348 + " skipping assign of " + region.getRegionNameAsString());
2349 offlineDisabledRegion(region);
2350 return true;
2351 }
2352 return false;
2353 }
2354
2355
2356
2357
2358
2359
2360
2361
2362 private int setOfflineInZooKeeper(final RegionState state, final ServerName destination) {
2363 if (!state.isClosed() && !state.isOffline()) {
2364 String msg = "Unexpected state : " + state + " .. Cannot transit it to OFFLINE.";
2365 this.server.abort(msg, new IllegalStateException(msg));
2366 return -1;
2367 }
2368 regionStates.updateRegionState(state.getRegion(), State.OFFLINE);
2369 int versionOfOfflineNode;
2370 try {
2371
2372 versionOfOfflineNode = ZKAssign.createOrForceNodeOffline(watcher,
2373 state.getRegion(), destination);
2374 if (versionOfOfflineNode == -1) {
2375 LOG.warn("Attempted to create/force node into OFFLINE state before "
2376 + "completing assignment but failed to do so for " + state);
2377 return -1;
2378 }
2379 } catch (KeeperException e) {
2380 server.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
2381 return -1;
2382 }
2383 return versionOfOfflineNode;
2384 }
2385
2386
2387
2388
2389
2390
2391 private RegionPlan getRegionPlan(final HRegionInfo region,
2392 final boolean forceNewPlan) throws HBaseIOException {
2393 return getRegionPlan(region, null, forceNewPlan);
2394 }
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405 private RegionPlan getRegionPlan(final HRegionInfo region,
2406 final ServerName serverToExclude, final boolean forceNewPlan) throws HBaseIOException {
2407
2408 final String encodedName = region.getEncodedName();
2409 final List<ServerName> destServers =
2410 serverManager.createDestinationServersList(serverToExclude);
2411
2412 if (destServers.isEmpty()){
2413 LOG.warn("Can't move " + encodedName +
2414 ", there is no destination server available.");
2415 return null;
2416 }
2417
2418 RegionPlan randomPlan = null;
2419 boolean newPlan = false;
2420 RegionPlan existingPlan;
2421
2422 synchronized (this.regionPlans) {
2423 existingPlan = this.regionPlans.get(encodedName);
2424
2425 if (existingPlan != null && existingPlan.getDestination() != null) {
2426 LOG.debug("Found an existing plan for " + region.getRegionNameAsString()
2427 + " destination server is " + existingPlan.getDestination() +
2428 " accepted as a dest server = " + destServers.contains(existingPlan.getDestination()));
2429 }
2430
2431 if (forceNewPlan
2432 || existingPlan == null
2433 || existingPlan.getDestination() == null
2434 || !destServers.contains(existingPlan.getDestination())) {
2435 newPlan = true;
2436 }
2437 }
2438
2439 if (newPlan) {
2440 ServerName destination = null;
2441 try {
2442 destination = balancer.randomAssignment(region, destServers);
2443 } catch (IOException ex) {
2444 LOG.warn("Failed to create new plan.",ex);
2445 return null;
2446 }
2447 if (destination == null) {
2448 LOG.warn("Can't find a destination for " + encodedName);
2449 return null;
2450 }
2451 synchronized (this.regionPlans) {
2452 randomPlan = new RegionPlan(region, null, destination);
2453 if (!region.isMetaTable() && shouldAssignRegionsWithFavoredNodes) {
2454 List<HRegionInfo> regions = new ArrayList<HRegionInfo>(1);
2455 regions.add(region);
2456 try {
2457 processFavoredNodes(regions);
2458 } catch (IOException ie) {
2459 LOG.warn("Ignoring exception in processFavoredNodes " + ie);
2460 }
2461 }
2462 this.regionPlans.put(encodedName, randomPlan);
2463 }
2464 LOG.debug("No previous transition plan found (or ignoring " + "an existing plan) for "
2465 + region.getRegionNameAsString() + "; generated random plan=" + randomPlan + "; "
2466 + destServers.size() + " (online=" + serverManager.getOnlineServers().size()
2467 + ") available servers, forceNewPlan=" + forceNewPlan);
2468 return randomPlan;
2469 }
2470 LOG.debug("Using pre-existing plan for " +
2471 region.getRegionNameAsString() + "; plan=" + existingPlan);
2472 return existingPlan;
2473 }
2474
2475
2476
2477
2478 private void waitForRetryingMetaAssignment() {
2479 try {
2480 Thread.sleep(this.sleepTimeBeforeRetryingMetaAssignment);
2481 } catch (InterruptedException e) {
2482 LOG.error("Got exception while waiting for hbase:meta assignment");
2483 Thread.currentThread().interrupt();
2484 }
2485 }
2486
2487
2488
2489
2490
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500 public void unassign(HRegionInfo region) {
2501 unassign(region, false);
2502 }
2503
2504
2505
2506
2507
2508
2509
2510
2511
2512
2513
2514
2515
2516
2517
2518
2519 public void unassign(HRegionInfo region, boolean force, ServerName dest) {
2520
2521 LOG.debug("Starting unassign of " + region.getRegionNameAsString()
2522 + " (offlining), current state: " + regionStates.getRegionState(region));
2523
2524 String encodedName = region.getEncodedName();
2525
2526 int versionOfClosingNode = -1;
2527
2528
2529 ReentrantLock lock = locker.acquireLock(encodedName);
2530 RegionState state = regionStates.getRegionTransitionState(encodedName);
2531 boolean reassign = true;
2532 try {
2533 if (state == null) {
2534
2535
2536 state = regionStates.getRegionState(encodedName);
2537 if (state != null && state.isUnassignable()) {
2538 LOG.info("Attempting to unassign " + state + ", ignored");
2539
2540 return;
2541 }
2542
2543 try {
2544 if (state == null || state.getServerName() == null) {
2545
2546
2547 LOG.warn("Attempting to unassign a region not in RegionStates "
2548 + region.getRegionNameAsString() + ", offlined");
2549 regionOffline(region);
2550 return;
2551 }
2552 if (useZKForAssignment) {
2553 versionOfClosingNode = ZKAssign.createNodeClosing(
2554 watcher, region, state.getServerName());
2555 if (versionOfClosingNode == -1) {
2556 LOG.info("Attempting to unassign " +
2557 region.getRegionNameAsString() + " but ZK closing node "
2558 + "can't be created.");
2559 reassign = false;
2560 return;
2561 }
2562 }
2563 } catch (KeeperException e) {
2564 if (e instanceof NodeExistsException) {
2565
2566
2567
2568
2569 NodeExistsException nee = (NodeExistsException)e;
2570 String path = nee.getPath();
2571 try {
2572 if (isSplitOrSplittingOrMergedOrMerging(path)) {
2573 LOG.debug(path + " is SPLIT or SPLITTING or MERGED or MERGING; " +
2574 "skipping unassign because region no longer exists -- its split or merge");
2575 reassign = false;
2576 return;
2577 }
2578 } catch (KeeperException.NoNodeException ke) {
2579 LOG.warn("Failed getData on SPLITTING/SPLIT at " + path +
2580 "; presuming split and that the region to unassign, " +
2581 encodedName + ", no longer exists -- confirm", ke);
2582 return;
2583 } catch (KeeperException ke) {
2584 LOG.error("Unexpected zk state", ke);
2585 } catch (DeserializationException de) {
2586 LOG.error("Failed parse", de);
2587 }
2588 }
2589
2590 server.abort("Unexpected ZK exception creating node CLOSING", e);
2591 reassign = false;
2592 return;
2593 }
2594 state = regionStates.updateRegionState(region, State.PENDING_CLOSE);
2595 } else if (state.isFailedOpen()) {
2596
2597 regionOffline(region);
2598 return;
2599 } else if (force && state.isPendingCloseOrClosing()) {
2600 LOG.debug("Attempting to unassign " + region.getRegionNameAsString() +
2601 " which is already " + state.getState() +
2602 " but forcing to send a CLOSE RPC again ");
2603 if (state.isFailedClose()) {
2604 state = regionStates.updateRegionState(region, State.PENDING_CLOSE);
2605 }
2606 state.updateTimestampToNow();
2607 } else {
2608 LOG.debug("Attempting to unassign " +
2609 region.getRegionNameAsString() + " but it is " +
2610 "already in transition (" + state.getState() + ", force=" + force + ")");
2611 return;
2612 }
2613
2614 unassign(region, state, versionOfClosingNode, dest, useZKForAssignment, null);
2615 } finally {
2616 lock.unlock();
2617
2618
2619 if (!replicasToClose.contains(region) && reassign && regionStates.isRegionOffline(region)) {
2620 assign(region, true);
2621 }
2622 }
2623 }
2624
2625 public void unassign(HRegionInfo region, boolean force){
2626 unassign(region, force, null);
2627 }
2628
2629
2630
2631
2632 public void deleteClosingOrClosedNode(HRegionInfo region, ServerName sn) {
2633 String encodedName = region.getEncodedName();
2634 deleteNodeInStates(encodedName, "closing", sn, EventType.M_ZK_REGION_CLOSING,
2635 EventType.RS_ZK_REGION_CLOSED);
2636 }
2637
2638
2639
2640
2641
2642
2643
2644 private boolean isSplitOrSplittingOrMergedOrMerging(final String path)
2645 throws KeeperException, DeserializationException {
2646 boolean result = false;
2647
2648
2649 byte [] data = ZKAssign.getData(watcher, path);
2650 if (data == null) {
2651 LOG.info("Node " + path + " is gone");
2652 return false;
2653 }
2654 RegionTransition rt = RegionTransition.parseFrom(data);
2655 switch (rt.getEventType()) {
2656 case RS_ZK_REQUEST_REGION_SPLIT:
2657 case RS_ZK_REGION_SPLIT:
2658 case RS_ZK_REGION_SPLITTING:
2659 case RS_ZK_REQUEST_REGION_MERGE:
2660 case RS_ZK_REGION_MERGED:
2661 case RS_ZK_REGION_MERGING:
2662 result = true;
2663 break;
2664 default:
2665 LOG.info("Node " + path + " is in " + rt.getEventType());
2666 break;
2667 }
2668 return result;
2669 }
2670
2671
2672
2673
2674
2675
2676 public int getNumRegionsOpened() {
2677 return numRegionsOpened.get();
2678 }
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688
2689 public boolean waitForAssignment(HRegionInfo regionInfo)
2690 throws InterruptedException {
2691 ArrayList<HRegionInfo> regionSet = new ArrayList<HRegionInfo>(1);
2692 regionSet.add(regionInfo);
2693 return waitForAssignment(regionSet, true, Long.MAX_VALUE);
2694 }
2695
2696
2697
2698
2699 protected boolean waitForAssignment(final Collection<HRegionInfo> regionSet,
2700 final boolean waitTillAllAssigned, final int reassigningRegions,
2701 final long minEndTime) throws InterruptedException {
2702 long deadline = minEndTime + bulkPerRegionOpenTimeGuesstimate * (reassigningRegions + 1);
2703 if (deadline < 0) {
2704 deadline = Long.MAX_VALUE;
2705 }
2706 return waitForAssignment(regionSet, waitTillAllAssigned, deadline);
2707 }
2708
2709
2710
2711
2712
2713
2714
2715
2716
2717 protected boolean waitForAssignment(final Collection<HRegionInfo> regionSet,
2718 final boolean waitTillAllAssigned, final long deadline) throws InterruptedException {
2719
2720 while (!regionSet.isEmpty() && !server.isStopped() && deadline > System.currentTimeMillis()) {
2721 int failedOpenCount = 0;
2722 Iterator<HRegionInfo> regionInfoIterator = regionSet.iterator();
2723 while (regionInfoIterator.hasNext()) {
2724 HRegionInfo hri = regionInfoIterator.next();
2725 if (regionStates.isRegionOnline(hri) || regionStates.isRegionInState(hri,
2726 State.SPLITTING, State.SPLIT, State.MERGING, State.MERGED)) {
2727 regionInfoIterator.remove();
2728 } else if (regionStates.isRegionInState(hri, State.FAILED_OPEN)) {
2729 failedOpenCount++;
2730 }
2731 }
2732 if (!waitTillAllAssigned) {
2733
2734 break;
2735 }
2736 if (!regionSet.isEmpty()) {
2737 if (failedOpenCount == regionSet.size()) {
2738
2739 break;
2740 }
2741 regionStates.waitForUpdate(100);
2742 }
2743 }
2744 return regionSet.isEmpty();
2745 }
2746
2747
2748
2749
2750
2751
2752
2753
2754
2755
2756
2757
2758 public void assignMeta(HRegionInfo hri) throws KeeperException {
2759 this.server.getMetaTableLocator().deleteMetaLocation(this.watcher, hri.getReplicaId());
2760 assign(hri, true);
2761 }
2762
2763
2764
2765
2766
2767
2768
2769
2770
2771 public void assign(Map<HRegionInfo, ServerName> regions)
2772 throws IOException, InterruptedException {
2773 if (regions == null || regions.isEmpty()) {
2774 return;
2775 }
2776 List<ServerName> servers = serverManager.createDestinationServersList();
2777 if (servers == null || servers.isEmpty()) {
2778 throw new IOException("Found no destination server to assign region(s)");
2779 }
2780
2781
2782 Map<ServerName, List<HRegionInfo>> bulkPlan =
2783 balancer.retainAssignment(regions, servers);
2784 if (bulkPlan == null) {
2785 throw new IOException("Unable to determine a plan to assign region(s)");
2786 }
2787
2788 processBogusAssignments(bulkPlan);
2789
2790 assign(regions.size(), servers.size(),
2791 "retainAssignment=true", bulkPlan);
2792 }
2793
2794
2795
2796
2797
2798
2799
2800
2801
2802 public void assign(List<HRegionInfo> regions)
2803 throws IOException, InterruptedException {
2804 if (regions == null || regions.isEmpty()) {
2805 return;
2806 }
2807
2808 List<ServerName> servers = serverManager.createDestinationServersList();
2809 if (servers == null || servers.isEmpty()) {
2810 throw new IOException("Found no destination server to assign region(s)");
2811 }
2812
2813
2814 Map<ServerName, List<HRegionInfo>> bulkPlan
2815 = balancer.roundRobinAssignment(regions, servers);
2816 if (bulkPlan == null) {
2817 throw new IOException("Unable to determine a plan to assign region(s)");
2818 }
2819
2820 processBogusAssignments(bulkPlan);
2821
2822 processFavoredNodes(regions);
2823 assign(regions.size(), servers.size(),
2824 "round-robin=true", bulkPlan);
2825 }
2826
2827 private void assign(int regions, int totalServers,
2828 String message, Map<ServerName, List<HRegionInfo>> bulkPlan)
2829 throws InterruptedException, IOException {
2830
2831 int servers = bulkPlan.size();
2832 if (servers == 1 || (regions < bulkAssignThresholdRegions
2833 && servers < bulkAssignThresholdServers)) {
2834
2835
2836
2837 if (LOG.isTraceEnabled()) {
2838 LOG.trace("Not using bulk assignment since we are assigning only " + regions +
2839 " region(s) to " + servers + " server(s)");
2840 }
2841
2842
2843 ArrayList<HRegionInfo> userRegionSet = new ArrayList<HRegionInfo>(regions);
2844 for (Map.Entry<ServerName, List<HRegionInfo>> plan: bulkPlan.entrySet()) {
2845 if (!assign(plan.getKey(), plan.getValue())) {
2846 for (HRegionInfo region: plan.getValue()) {
2847 if (!regionStates.isRegionOnline(region)) {
2848 invokeAssign(region);
2849 if (!region.getTable().isSystemTable()) {
2850 userRegionSet.add(region);
2851 }
2852 }
2853 }
2854 }
2855 }
2856
2857
2858 if (!waitForAssignment(userRegionSet, true, userRegionSet.size(),
2859 System.currentTimeMillis())) {
2860 LOG.debug("some user regions are still in transition: " + userRegionSet);
2861 }
2862 } else {
2863 LOG.info("Bulk assigning " + regions + " region(s) across "
2864 + totalServers + " server(s), " + message);
2865
2866
2867 BulkAssigner ba = new GeneralBulkAssigner(
2868 this.server, bulkPlan, this, bulkAssignWaitTillAllAssigned);
2869 ba.bulkAssign();
2870 LOG.info("Bulk assigning done");
2871 }
2872 }
2873
2874
2875
2876
2877
2878
2879
2880
2881
2882
2883 private void assignAllUserRegions(Map<HRegionInfo, ServerName> allRegions)
2884 throws IOException, InterruptedException {
2885 if (allRegions == null || allRegions.isEmpty()) return;
2886
2887
2888 boolean retainAssignment = server.getConfiguration().
2889 getBoolean("hbase.master.startup.retainassign", true);
2890
2891 Set<HRegionInfo> regionsFromMetaScan = allRegions.keySet();
2892 if (retainAssignment) {
2893 assign(allRegions);
2894 } else {
2895 List<HRegionInfo> regions = new ArrayList<HRegionInfo>(regionsFromMetaScan);
2896 assign(regions);
2897 }
2898
2899 for (HRegionInfo hri : regionsFromMetaScan) {
2900 TableName tableName = hri.getTable();
2901 if (!tableStateManager.isTableState(tableName,
2902 ZooKeeperProtos.Table.State.ENABLED)) {
2903 setEnabledTable(tableName);
2904 }
2905 }
2906
2907 assign(replicaRegionsNotRecordedInMeta(regionsFromMetaScan, (MasterServices)server));
2908 }
2909
2910
2911
2912
2913
2914
2915
2916
2917
2918
2919
2920
2921 public static List<HRegionInfo> replicaRegionsNotRecordedInMeta(
2922 Set<HRegionInfo> regionsRecordedInMeta, MasterServices master)throws IOException {
2923 List<HRegionInfo> regionsNotRecordedInMeta = new ArrayList<HRegionInfo>();
2924 for (HRegionInfo hri : regionsRecordedInMeta) {
2925 TableName table = hri.getTable();
2926 HTableDescriptor htd = master.getTableDescriptors().get(table);
2927
2928 int desiredRegionReplication = htd.getRegionReplication();
2929 for (int i = 0; i < desiredRegionReplication; i++) {
2930 HRegionInfo replica = RegionReplicaUtil.getRegionInfoForReplica(hri, i);
2931 if (regionsRecordedInMeta.contains(replica)) continue;
2932 regionsNotRecordedInMeta.add(replica);
2933 }
2934 }
2935 return regionsNotRecordedInMeta;
2936 }
2937
2938
2939
2940
2941
2942
2943
2944 boolean waitUntilNoRegionsInTransition(final long timeout)
2945 throws InterruptedException {
2946
2947
2948
2949
2950
2951
2952 final long endTime = System.currentTimeMillis() + timeout;
2953
2954 while (!this.server.isStopped() && regionStates.isRegionsInTransition()
2955 && endTime > System.currentTimeMillis()) {
2956 regionStates.waitForUpdate(100);
2957 }
2958
2959 return !regionStates.isRegionsInTransition();
2960 }
2961
2962
2963
2964
2965
2966
2967
2968
2969
2970 Set<ServerName> rebuildUserRegions() throws
2971 IOException, KeeperException, CoordinatedStateException {
2972 Set<TableName> disabledOrEnablingTables = tableStateManager.getTablesInStates(
2973 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.ENABLING);
2974
2975 Set<TableName> disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
2976 ZooKeeperProtos.Table.State.DISABLED,
2977 ZooKeeperProtos.Table.State.DISABLING,
2978 ZooKeeperProtos.Table.State.ENABLING);
2979
2980
2981 List<Result> results = MetaTableAccessor.fullScanOfMeta(server.getConnection());
2982
2983 Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
2984
2985 Set<ServerName> offlineServers = new HashSet<ServerName>();
2986
2987 for (Result result : results) {
2988 if (result == null && LOG.isDebugEnabled()){
2989 LOG.debug("null result from meta - ignoring but this is strange.");
2990 continue;
2991 }
2992
2993
2994
2995 PairOfSameType<HRegionInfo> p = MetaTableAccessor.getMergeRegions(result);
2996 if (p.getFirst() != null && p.getSecond() != null) {
2997 int numReplicas = ((MasterServices)server).getTableDescriptors().get(p.getFirst().
2998 getTable()).getRegionReplication();
2999 for (HRegionInfo merge : p) {
3000 for (int i = 1; i < numReplicas; i++) {
3001 replicasToClose.add(RegionReplicaUtil.getRegionInfoForReplica(merge, i));
3002 }
3003 }
3004 }
3005 RegionLocations rl = MetaTableAccessor.getRegionLocations(result);
3006 if (rl == null) continue;
3007 HRegionLocation[] locations = rl.getRegionLocations();
3008 if (locations == null) continue;
3009 for (HRegionLocation hrl : locations) {
3010 if (hrl == null) continue;
3011 HRegionInfo regionInfo = hrl.getRegionInfo();
3012 if (regionInfo == null) continue;
3013 int replicaId = regionInfo.getReplicaId();
3014 State state = RegionStateStore.getRegionState(result, replicaId);
3015
3016
3017
3018 if (replicaId == 0 && state.equals(State.SPLIT)) {
3019 for (HRegionLocation h : locations) {
3020 replicasToClose.add(h.getRegionInfo());
3021 }
3022 }
3023 ServerName lastHost = hrl.getServerName();
3024 ServerName regionLocation = RegionStateStore.getRegionServer(result, replicaId);
3025 if (tableStateManager.isTableState(regionInfo.getTable(),
3026 ZooKeeperProtos.Table.State.DISABLED)) {
3027
3028
3029 lastHost = null;
3030 regionLocation = null;
3031 }
3032 regionStates.createRegionState(regionInfo, state, regionLocation, lastHost);
3033 if (!regionStates.isRegionInState(regionInfo, State.OPEN)) {
3034
3035 continue;
3036 }
3037 TableName tableName = regionInfo.getTable();
3038 if (!onlineServers.contains(regionLocation)) {
3039
3040 offlineServers.add(regionLocation);
3041 if (useZKForAssignment) {
3042 regionStates.regionOffline(regionInfo);
3043 }
3044 } else if (!disabledOrEnablingTables.contains(tableName)) {
3045
3046
3047 regionStates.regionOnline(regionInfo, regionLocation);
3048 balancer.regionOnline(regionInfo, regionLocation);
3049 } else if (useZKForAssignment) {
3050 regionStates.regionOffline(regionInfo);
3051 }
3052
3053
3054 if (!disabledOrDisablingOrEnabling.contains(tableName)
3055 && !getTableStateManager().isTableState(tableName,
3056 ZooKeeperProtos.Table.State.ENABLED)) {
3057 setEnabledTable(tableName);
3058 }
3059 }
3060 }
3061 return offlineServers;
3062 }
3063
3064
3065
3066
3067
3068
3069
3070
3071
3072 private void recoverTableInDisablingState()
3073 throws KeeperException, IOException, CoordinatedStateException {
3074 Set<TableName> disablingTables =
3075 tableStateManager.getTablesInStates(ZooKeeperProtos.Table.State.DISABLING);
3076 if (disablingTables.size() != 0) {
3077 for (TableName tableName : disablingTables) {
3078
3079 LOG.info("The table " + tableName
3080 + " is in DISABLING state. Hence recovering by moving the table"
3081 + " to DISABLED state.");
3082 new DisableTableHandler(this.server, tableName,
3083 this, tableLockManager, true).prepare().process();
3084 }
3085 }
3086 }
3087
3088
3089
3090
3091
3092
3093
3094
3095
3096 private void recoverTableInEnablingState()
3097 throws KeeperException, IOException, CoordinatedStateException {
3098 Set<TableName> enablingTables = tableStateManager.
3099 getTablesInStates(ZooKeeperProtos.Table.State.ENABLING);
3100 if (enablingTables.size() != 0) {
3101 for (TableName tableName : enablingTables) {
3102
3103 LOG.info("The table " + tableName
3104 + " is in ENABLING state. Hence recovering by moving the table"
3105 + " to ENABLED state.");
3106
3107
3108 EnableTableHandler eth = new EnableTableHandler(this.server, tableName,
3109 this, tableLockManager, true);
3110 try {
3111 eth.prepare();
3112 } catch (TableNotFoundException e) {
3113 LOG.warn("Table " + tableName + " not found in hbase:meta to recover.");
3114 continue;
3115 }
3116 eth.process();
3117 }
3118 }
3119 }
3120
3121
3122
3123
3124
3125
3126
3127
3128
3129
3130
3131
3132
3133
3134
3135
3136 private void processDeadServersAndRecoverLostRegions(
3137 Set<ServerName> deadServers) throws IOException, KeeperException {
3138 if (deadServers != null && !deadServers.isEmpty()) {
3139 for (ServerName serverName: deadServers) {
3140 if (!serverManager.isServerDead(serverName)) {
3141 serverManager.expireServer(serverName);
3142 }
3143 }
3144 }
3145
3146 List<String> nodes = useZKForAssignment ?
3147 ZKUtil.listChildrenAndWatchForNewChildren(watcher, watcher.assignmentZNode)
3148 : ZKUtil.listChildrenNoWatch(watcher, watcher.assignmentZNode);
3149 if (nodes != null && !nodes.isEmpty()) {
3150 for (String encodedRegionName : nodes) {
3151 processRegionInTransition(encodedRegionName, null);
3152 }
3153 } else if (!useZKForAssignment) {
3154 processRegionInTransitionZkLess();
3155 }
3156 }
3157
3158 void processRegionInTransitionZkLess() {
3159
3160
3161
3162
3163
3164 Map<String, RegionState> rits = regionStates.getRegionsInTransition();
3165 for (RegionState regionState : rits.values()) {
3166 LOG.info("Processing " + regionState);
3167 ServerName serverName = regionState.getServerName();
3168
3169
3170 if (serverName != null
3171 && !serverManager.getOnlineServers().containsKey(serverName)) {
3172 LOG.info("Server " + serverName + " isn't online. SSH will handle this");
3173 continue;
3174 }
3175 HRegionInfo regionInfo = regionState.getRegion();
3176 State state = regionState.getState();
3177
3178 switch (state) {
3179 case CLOSED:
3180 invokeAssign(regionInfo);
3181 break;
3182 case PENDING_OPEN:
3183 retrySendRegionOpen(regionState);
3184 break;
3185 case PENDING_CLOSE:
3186 retrySendRegionClose(regionState);
3187 break;
3188 case FAILED_CLOSE:
3189 case FAILED_OPEN:
3190 invokeUnAssign(regionInfo);
3191 break;
3192 default:
3193
3194 }
3195 }
3196 }
3197
3198
3199
3200
3201
3202 private void retrySendRegionOpen(final RegionState regionState) {
3203 this.executorService.submit(
3204 new EventHandler(server, EventType.M_MASTER_RECOVERY) {
3205 @Override
3206 public void process() throws IOException {
3207 HRegionInfo hri = regionState.getRegion();
3208 ServerName serverName = regionState.getServerName();
3209 ReentrantLock lock = locker.acquireLock(hri.getEncodedName());
3210 try {
3211 for (int i = 1; i <= maximumAttempts; i++) {
3212 if (!serverManager.isServerOnline(serverName)
3213 || server.isStopped() || server.isAborted()) {
3214 return;
3215 }
3216 try {
3217 if (!regionState.equals(regionStates.getRegionState(hri))) {
3218 return;
3219 }
3220 List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
3221 if (shouldAssignRegionsWithFavoredNodes) {
3222 favoredNodes = ((FavoredNodeLoadBalancer)balancer).getFavoredNodes(hri);
3223 }
3224 RegionOpeningState regionOpenState = serverManager.sendRegionOpen(
3225 serverName, hri, -1, favoredNodes);
3226
3227 if (regionOpenState == RegionOpeningState.FAILED_OPENING) {
3228
3229
3230 LOG.debug("Got failed_opening in retry sendRegionOpen for "
3231 + regionState + ", re-assign it");
3232 invokeAssign(hri, true);
3233 }
3234 return;
3235 } catch (Throwable t) {
3236 if (t instanceof RemoteException) {
3237 t = ((RemoteException) t).unwrapRemoteException();
3238 }
3239
3240 if (t instanceof java.net.SocketTimeoutException
3241 || t instanceof FailedServerException) {
3242 Threads.sleep(100);
3243 continue;
3244 }
3245
3246 LOG.debug("Got exception in retry sendRegionOpen for "
3247 + regionState + ", re-assign it", t);
3248 invokeAssign(hri);
3249 return;
3250 }
3251 }
3252 } finally {
3253 lock.unlock();
3254 }
3255 }
3256 });
3257 }
3258
3259
3260
3261
3262
3263 private void retrySendRegionClose(final RegionState regionState) {
3264 this.executorService.submit(
3265 new EventHandler(server, EventType.M_MASTER_RECOVERY) {
3266 @Override
3267 public void process() throws IOException {
3268 HRegionInfo hri = regionState.getRegion();
3269 ServerName serverName = regionState.getServerName();
3270 ReentrantLock lock = locker.acquireLock(hri.getEncodedName());
3271 try {
3272 for (int i = 1; i <= maximumAttempts; i++) {
3273 if (!serverManager.isServerOnline(serverName)
3274 || server.isStopped() || server.isAborted()) {
3275 return;
3276 }
3277 try {
3278 if (!regionState.equals(regionStates.getRegionState(hri))) {
3279 return;
3280 }
3281 if (!serverManager.sendRegionClose(serverName, hri, -1, null, false)) {
3282
3283 LOG.debug("Got false in retry sendRegionClose for "
3284 + regionState + ", re-close it");
3285 invokeUnAssign(hri);
3286 }
3287 return;
3288 } catch (Throwable t) {
3289 if (t instanceof RemoteException) {
3290 t = ((RemoteException) t).unwrapRemoteException();
3291 }
3292
3293 if (t instanceof java.net.SocketTimeoutException
3294 || t instanceof FailedServerException) {
3295 Threads.sleep(100);
3296 continue;
3297 }
3298 if (!(t instanceof NotServingRegionException
3299 || t instanceof RegionAlreadyInTransitionException)) {
3300
3301
3302
3303 LOG.debug("Got exception in retry sendRegionClose for "
3304 + regionState + ", re-close it", t);
3305 invokeUnAssign(hri);
3306 }
3307 return;
3308 }
3309 }
3310 } finally {
3311 lock.unlock();
3312 }
3313 }
3314 });
3315 }
3316
3317
3318
3319
3320
3321
3322
3323
3324 public void updateRegionsInTransitionMetrics() {
3325 long currentTime = System.currentTimeMillis();
3326 int totalRITs = 0;
3327 int totalRITsOverThreshold = 0;
3328 long oldestRITTime = 0;
3329 int ritThreshold = this.server.getConfiguration().
3330 getInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 60000);
3331 for (RegionState state: regionStates.getRegionsInTransition().values()) {
3332 totalRITs++;
3333 long ritTime = currentTime - state.getStamp();
3334 if (ritTime > ritThreshold) {
3335 totalRITsOverThreshold++;
3336 }
3337 if (oldestRITTime < ritTime) {
3338 oldestRITTime = ritTime;
3339 }
3340 }
3341 if (this.metricsAssignmentManager != null) {
3342 this.metricsAssignmentManager.updateRITOldestAge(oldestRITTime);
3343 this.metricsAssignmentManager.updateRITCount(totalRITs);
3344 this.metricsAssignmentManager.updateRITCountOverThreshold(totalRITsOverThreshold);
3345 }
3346 }
3347
3348
3349
3350
3351 void clearRegionPlan(final HRegionInfo region) {
3352 synchronized (this.regionPlans) {
3353 this.regionPlans.remove(region.getEncodedName());
3354 }
3355 }
3356
3357
3358
3359
3360
3361
3362 public void waitOnRegionToClearRegionsInTransition(final HRegionInfo hri)
3363 throws IOException, InterruptedException {
3364 waitOnRegionToClearRegionsInTransition(hri, -1L);
3365 }
3366
3367
3368
3369
3370
3371
3372
3373
3374 public boolean waitOnRegionToClearRegionsInTransition(final HRegionInfo hri, long timeOut)
3375 throws InterruptedException {
3376 if (!regionStates.isRegionInTransition(hri)) return true;
3377 long end = (timeOut <= 0) ? Long.MAX_VALUE : EnvironmentEdgeManager.currentTime()
3378 + timeOut;
3379
3380
3381 LOG.info("Waiting for " + hri.getEncodedName() +
3382 " to leave regions-in-transition, timeOut=" + timeOut + " ms.");
3383 while (!this.server.isStopped() && regionStates.isRegionInTransition(hri)) {
3384 regionStates.waitForUpdate(100);
3385 if (EnvironmentEdgeManager.currentTime() > end) {
3386 LOG.info("Timed out on waiting for " + hri.getEncodedName() + " to be assigned.");
3387 return false;
3388 }
3389 }
3390 if (this.server.isStopped()) {
3391 LOG.info("Giving up wait on regions in transition because stoppable.isStopped is set");
3392 return false;
3393 }
3394 return true;
3395 }
3396
3397 void invokeAssign(HRegionInfo regionInfo) {
3398 invokeAssign(regionInfo, true);
3399 }
3400
3401 void invokeAssign(HRegionInfo regionInfo, boolean newPlan) {
3402 threadPoolExecutorService.submit(new AssignCallable(this, regionInfo, newPlan));
3403 }
3404
3405 void invokeUnAssign(HRegionInfo regionInfo) {
3406 threadPoolExecutorService.submit(new UnAssignCallable(this, regionInfo));
3407 }
3408
3409 public ServerHostRegion isCarryingMeta(ServerName serverName) {
3410 return isCarryingRegion(serverName, HRegionInfo.FIRST_META_REGIONINFO);
3411 }
3412
3413 public ServerHostRegion isCarryingMetaReplica(ServerName serverName, int replicaId) {
3414 return isCarryingRegion(serverName,
3415 RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, replicaId));
3416 }
3417
3418 public ServerHostRegion isCarryingMetaReplica(ServerName serverName, HRegionInfo metaHri) {
3419 return isCarryingRegion(serverName, metaHri);
3420 }
3421
3422
3423
3424
3425
3426
3427
3428
3429
3430
3431
3432 private ServerHostRegion isCarryingRegion(ServerName serverName, HRegionInfo hri) {
3433 RegionTransition rt = null;
3434 try {
3435 byte [] data = ZKAssign.getData(watcher, hri.getEncodedName());
3436
3437 rt = data == null? null: RegionTransition.parseFrom(data);
3438 } catch (KeeperException e) {
3439 server.abort("Exception reading unassigned node for region=" + hri.getEncodedName(), e);
3440 } catch (DeserializationException e) {
3441 server.abort("Exception parsing unassigned node for region=" + hri.getEncodedName(), e);
3442 }
3443
3444 ServerName addressFromZK = rt != null? rt.getServerName(): null;
3445 if (addressFromZK != null) {
3446
3447 boolean matchZK = addressFromZK.equals(serverName);
3448 LOG.debug("Checking region=" + hri.getRegionNameAsString() + ", zk server=" + addressFromZK +
3449 " current=" + serverName + ", matches=" + matchZK);
3450 return matchZK ? ServerHostRegion.HOSTING_REGION : ServerHostRegion.NOT_HOSTING_REGION;
3451 }
3452
3453 ServerName addressFromAM = regionStates.getRegionServerOfRegion(hri);
3454 if (addressFromAM != null) {
3455 boolean matchAM = addressFromAM.equals(serverName);
3456 LOG.debug("based on AM, current region=" + hri.getRegionNameAsString() +
3457 " is on server=" + (addressFromAM != null ? addressFromAM : "null") +
3458 " server being checked: " + serverName);
3459 return matchAM ? ServerHostRegion.HOSTING_REGION : ServerHostRegion.NOT_HOSTING_REGION;
3460 }
3461
3462 if (hri.isMetaRegion() && RegionReplicaUtil.isDefaultReplica(hri)) {
3463
3464 final ServerName serverNameInZK =
3465 server.getMetaTableLocator().getMetaRegionLocation(this.server.getZooKeeper());
3466 LOG.debug("Based on MetaTableLocator, the META region is on server="
3467 + (serverNameInZK == null ? "null" : serverNameInZK)
3468 + " server being checked: " + serverName);
3469 if (serverNameInZK != null) {
3470 return serverNameInZK.equals(serverName) ?
3471 ServerHostRegion.HOSTING_REGION : ServerHostRegion.NOT_HOSTING_REGION;
3472 }
3473 }
3474
3475
3476 return ServerHostRegion.UNKNOWN;
3477 }
3478
3479
3480
3481
3482
3483
3484 public List<HRegionInfo> processServerShutdown(final ServerName sn) {
3485
3486 synchronized (this.regionPlans) {
3487 for (Iterator <Map.Entry<String, RegionPlan>> i =
3488 this.regionPlans.entrySet().iterator(); i.hasNext();) {
3489 Map.Entry<String, RegionPlan> e = i.next();
3490 ServerName otherSn = e.getValue().getDestination();
3491
3492 if (otherSn != null && otherSn.equals(sn)) {
3493
3494 i.remove();
3495 }
3496 }
3497 }
3498 List<HRegionInfo> regions = regionStates.serverOffline(watcher, sn);
3499 for (Iterator<HRegionInfo> it = regions.iterator(); it.hasNext(); ) {
3500 HRegionInfo hri = it.next();
3501 String encodedName = hri.getEncodedName();
3502
3503
3504 Lock lock = locker.acquireLock(encodedName);
3505 try {
3506 RegionState regionState =
3507 regionStates.getRegionTransitionState(encodedName);
3508 if (regionState == null
3509 || (regionState.getServerName() != null && !regionState.isOnServer(sn))
3510 || !(regionState.isFailedClose() || regionState.isOffline()
3511 || regionState.isPendingOpenOrOpening())) {
3512 LOG.info("Skip " + regionState + " since it is not opening/failed_close"
3513 + " on the dead server any more: " + sn);
3514 it.remove();
3515 } else {
3516 try {
3517
3518 ZKAssign.deleteNodeFailSilent(watcher, hri);
3519 } catch (KeeperException ke) {
3520 server.abort("Unexpected ZK exception deleting node " + hri, ke);
3521 }
3522 if (tableStateManager.isTableState(hri.getTable(),
3523 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3524 regionStates.regionOffline(hri);
3525 it.remove();
3526 continue;
3527 }
3528
3529 regionStates.updateRegionState(hri, State.OFFLINE);
3530 }
3531 } finally {
3532 lock.unlock();
3533 }
3534 }
3535 return regions;
3536 }
3537
3538
3539
3540
3541 public void balance(final RegionPlan plan) {
3542
3543 HRegionInfo hri = plan.getRegionInfo();
3544 TableName tableName = hri.getTable();
3545 if (tableStateManager.isTableState(tableName,
3546 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3547 LOG.info("Ignored moving region of disabling/disabled table "
3548 + tableName);
3549 return;
3550 }
3551
3552
3553 String encodedName = hri.getEncodedName();
3554 ReentrantLock lock = locker.acquireLock(encodedName);
3555 try {
3556 if (!regionStates.isRegionOnline(hri)) {
3557 RegionState state = regionStates.getRegionState(encodedName);
3558 LOG.info("Ignored moving region not assigned: " + hri + ", "
3559 + (state == null ? "not in region states" : state));
3560 return;
3561 }
3562 synchronized (this.regionPlans) {
3563 this.regionPlans.put(plan.getRegionName(), plan);
3564 }
3565 unassign(hri, false, plan.getDestination());
3566 } finally {
3567 lock.unlock();
3568 }
3569 }
3570
3571 public void stop() {
3572 shutdown();
3573 }
3574
3575
3576
3577
3578 public void shutdown() {
3579
3580 synchronized (zkEventWorkerWaitingList){
3581 zkEventWorkerWaitingList.clear();
3582 }
3583
3584
3585 threadPoolExecutorService.shutdownNow();
3586 zkEventWorkers.shutdownNow();
3587 regionStateStore.stop();
3588 }
3589
3590 protected void setEnabledTable(TableName tableName) {
3591 try {
3592 this.tableStateManager.setTableState(tableName,
3593 ZooKeeperProtos.Table.State.ENABLED);
3594 } catch (CoordinatedStateException e) {
3595
3596 String errorMsg = "Unable to ensure that the table " + tableName
3597 + " will be" + " enabled because of a ZooKeeper issue";
3598 LOG.error(errorMsg);
3599 this.server.abort(errorMsg, e);
3600 }
3601 }
3602
3603
3604
3605
3606
3607
3608
3609 private boolean asyncSetOfflineInZooKeeper(final RegionState state,
3610 final AsyncCallback.StringCallback cb, final ServerName destination) {
3611 if (!state.isClosed() && !state.isOffline()) {
3612 this.server.abort("Unexpected state trying to OFFLINE; " + state,
3613 new IllegalStateException());
3614 return false;
3615 }
3616 regionStates.updateRegionState(state.getRegion(), State.OFFLINE);
3617 try {
3618 ZKAssign.asyncCreateNodeOffline(watcher, state.getRegion(),
3619 destination, cb, state);
3620 } catch (KeeperException e) {
3621 if (e instanceof NodeExistsException) {
3622 LOG.warn("Node for " + state.getRegion() + " already exists");
3623 } else {
3624 server.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
3625 }
3626 return false;
3627 }
3628 return true;
3629 }
3630
3631 private boolean deleteNodeInStates(String encodedName,
3632 String desc, ServerName sn, EventType... types) {
3633 try {
3634 for (EventType et: types) {
3635 if (ZKAssign.deleteNode(watcher, encodedName, et, sn)) {
3636 return true;
3637 }
3638 }
3639 LOG.info("Failed to delete the " + desc + " node for "
3640 + encodedName + ". The node type may not match");
3641 } catch (NoNodeException e) {
3642 if (LOG.isDebugEnabled()) {
3643 LOG.debug("The " + desc + " node for " + encodedName + " already deleted");
3644 }
3645 } catch (KeeperException ke) {
3646 server.abort("Unexpected ZK exception deleting " + desc
3647 + " node for the region " + encodedName, ke);
3648 }
3649 return false;
3650 }
3651
3652 private void deleteMergingNode(String encodedName, ServerName sn) {
3653 deleteNodeInStates(encodedName, "merging", sn, EventType.RS_ZK_REGION_MERGING,
3654 EventType.RS_ZK_REQUEST_REGION_MERGE, EventType.RS_ZK_REGION_MERGED);
3655 }
3656
3657 private void deleteSplittingNode(String encodedName, ServerName sn) {
3658 deleteNodeInStates(encodedName, "splitting", sn, EventType.RS_ZK_REGION_SPLITTING,
3659 EventType.RS_ZK_REQUEST_REGION_SPLIT, EventType.RS_ZK_REGION_SPLIT);
3660 }
3661
3662 private void onRegionFailedOpen(
3663 final HRegionInfo hri, final ServerName sn) {
3664 String encodedName = hri.getEncodedName();
3665 AtomicInteger failedOpenCount = failedOpenTracker.get(encodedName);
3666 if (failedOpenCount == null) {
3667 failedOpenCount = new AtomicInteger();
3668
3669
3670
3671 failedOpenTracker.put(encodedName, failedOpenCount);
3672 }
3673 if (failedOpenCount.incrementAndGet() >= maximumAttempts && !hri.isMetaRegion()) {
3674 regionStates.updateRegionState(hri, State.FAILED_OPEN);
3675
3676
3677 failedOpenTracker.remove(encodedName);
3678 } else {
3679 if (hri.isMetaRegion() && failedOpenCount.get() >= maximumAttempts) {
3680
3681
3682 LOG.warn("Failed to open the hbase:meta region " +
3683 hri.getRegionNameAsString() + " after" +
3684 failedOpenCount.get() + " retries. Continue retrying.");
3685 }
3686
3687
3688 RegionState regionState = regionStates.updateRegionState(hri, State.CLOSED);
3689 if (regionState != null) {
3690
3691
3692 if (getTableStateManager().isTableState(hri.getTable(),
3693 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) ||
3694 replicasToClose.contains(hri)) {
3695 offlineDisabledRegion(hri);
3696 return;
3697 }
3698
3699 regionStates.updateRegionState(hri, RegionState.State.CLOSED);
3700
3701 removeClosedRegion(hri);
3702 try {
3703 getRegionPlan(hri, sn, true);
3704 } catch (HBaseIOException e) {
3705 LOG.warn("Failed to get region plan", e);
3706 }
3707 invokeAssign(hri, false);
3708 }
3709 }
3710 }
3711
3712 private void onRegionOpen(
3713 final HRegionInfo hri, final ServerName sn, long openSeqNum) {
3714 regionOnline(hri, sn, openSeqNum);
3715 if (useZKForAssignment) {
3716 try {
3717
3718 ZKAssign.deleteNodeFailSilent(watcher, hri);
3719 } catch (KeeperException ke) {
3720 server.abort("Unexpected ZK exception deleting node " + hri, ke);
3721 }
3722 }
3723
3724
3725 failedOpenTracker.remove(hri.getEncodedName());
3726 if (getTableStateManager().isTableState(hri.getTable(),
3727 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3728 invokeUnAssign(hri);
3729 }
3730 }
3731
3732 private void onRegionClosed(final HRegionInfo hri) {
3733 if (getTableStateManager().isTableState(hri.getTable(),
3734 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) ||
3735 replicasToClose.contains(hri)) {
3736 offlineDisabledRegion(hri);
3737 return;
3738 }
3739 regionStates.updateRegionState(hri, RegionState.State.CLOSED);
3740 sendRegionClosedNotification(hri);
3741
3742 removeClosedRegion(hri);
3743 invokeAssign(hri, false);
3744 }
3745
3746 private String onRegionSplit(ServerName sn, TransitionCode code,
3747 final HRegionInfo p, final HRegionInfo a, final HRegionInfo b) {
3748 final RegionState rs_p = regionStates.getRegionState(p);
3749 RegionState rs_a = regionStates.getRegionState(a);
3750 RegionState rs_b = regionStates.getRegionState(b);
3751 if (!(rs_p.isOpenOrSplittingOnServer(sn)
3752 && (rs_a == null || rs_a.isOpenOrSplittingNewOnServer(sn))
3753 && (rs_b == null || rs_b.isOpenOrSplittingNewOnServer(sn)))) {
3754 return "Not in state good for split";
3755 }
3756 regionStates.updateRegionState(a, State.SPLITTING_NEW, sn);
3757 regionStates.updateRegionState(b, State.SPLITTING_NEW, sn);
3758 regionStates.updateRegionState(p, State.SPLITTING);
3759
3760 if (code == TransitionCode.SPLIT) {
3761 if (TEST_SKIP_SPLIT_HANDLING) {
3762 return "Skipping split message, TEST_SKIP_SPLIT_HANDLING is set";
3763 }
3764 regionOffline(p, State.SPLIT);
3765 regionOnline(a, sn, 1);
3766 regionOnline(b, sn, 1);
3767
3768
3769 if (getTableStateManager().isTableState(p.getTable(),
3770 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3771 invokeUnAssign(a);
3772 invokeUnAssign(b);
3773 } else {
3774 Callable<Object> splitReplicasCallable = new Callable<Object>() {
3775 @Override
3776 public Object call() {
3777 doSplittingOfReplicas(p, a, b);
3778 return null;
3779 }
3780 };
3781 threadPoolExecutorService.submit(splitReplicasCallable);
3782 }
3783 } else if (code == TransitionCode.SPLIT_PONR) {
3784 try {
3785 regionStates.splitRegion(p, a, b, sn);
3786 } catch (IOException ioe) {
3787 LOG.info("Failed to record split region " + p.getShortNameToLog());
3788 return "Failed to record the splitting in meta";
3789 }
3790 } else if (code == TransitionCode.SPLIT_REVERTED) {
3791 regionOnline(p, sn);
3792 regionOffline(a);
3793 regionOffline(b);
3794
3795 if (getTableStateManager().isTableState(p.getTable(),
3796 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3797 invokeUnAssign(p);
3798 }
3799 }
3800 return null;
3801 }
3802
3803 private String onRegionMerge(ServerName sn, TransitionCode code,
3804 final HRegionInfo p, final HRegionInfo a, final HRegionInfo b) {
3805 RegionState rs_p = regionStates.getRegionState(p);
3806 RegionState rs_a = regionStates.getRegionState(a);
3807 RegionState rs_b = regionStates.getRegionState(b);
3808 if (!(rs_a.isOpenOrMergingOnServer(sn) && rs_b.isOpenOrMergingOnServer(sn)
3809 && (rs_p == null || rs_p.isOpenOrMergingNewOnServer(sn)))) {
3810 return "Not in state good for merge";
3811 }
3812 regionStates.updateRegionState(a, State.MERGING);
3813 regionStates.updateRegionState(b, State.MERGING);
3814 regionStates.updateRegionState(p, State.MERGING_NEW, sn);
3815
3816 String encodedName = p.getEncodedName();
3817 if (code == TransitionCode.READY_TO_MERGE) {
3818 mergingRegions.put(encodedName,
3819 new PairOfSameType<HRegionInfo>(a, b));
3820 } else if (code == TransitionCode.MERGED) {
3821 mergingRegions.remove(encodedName);
3822 regionOffline(a, State.MERGED);
3823 regionOffline(b, State.MERGED);
3824 regionOnline(p, sn, 1);
3825
3826
3827 if (getTableStateManager().isTableState(p.getTable(),
3828 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3829 invokeUnAssign(p);
3830 } else {
3831 Callable<Object> mergeReplicasCallable = new Callable<Object>() {
3832 @Override
3833 public Object call() {
3834 doMergingOfReplicas(p, a, b);
3835 return null;
3836 }
3837 };
3838 threadPoolExecutorService.submit(mergeReplicasCallable);
3839 }
3840 } else if (code == TransitionCode.MERGE_PONR) {
3841 try {
3842 regionStates.mergeRegions(p, a, b, sn);
3843 } catch (IOException ioe) {
3844 LOG.info("Failed to record merged region " + p.getShortNameToLog());
3845 return "Failed to record the merging in meta";
3846 }
3847 } else {
3848 mergingRegions.remove(encodedName);
3849 regionOnline(a, sn);
3850 regionOnline(b, sn);
3851 regionOffline(p);
3852
3853 if (getTableStateManager().isTableState(p.getTable(),
3854 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3855 invokeUnAssign(a);
3856 invokeUnAssign(b);
3857 }
3858 }
3859 return null;
3860 }
3861
3862
3863
3864
3865
3866 private boolean handleRegionMerging(final RegionTransition rt, final String encodedName,
3867 final String prettyPrintedRegionName, final ServerName sn) {
3868 if (!serverManager.isServerOnline(sn)) {
3869 LOG.warn("Dropped merging! ServerName=" + sn + " unknown.");
3870 return false;
3871 }
3872 byte [] payloadOfMerging = rt.getPayload();
3873 List<HRegionInfo> mergingRegions;
3874 try {
3875 mergingRegions = HRegionInfo.parseDelimitedFrom(
3876 payloadOfMerging, 0, payloadOfMerging.length);
3877 } catch (IOException e) {
3878 LOG.error("Dropped merging! Failed reading " + rt.getEventType()
3879 + " payload for " + prettyPrintedRegionName);
3880 return false;
3881 }
3882 assert mergingRegions.size() == 3;
3883 HRegionInfo p = mergingRegions.get(0);
3884 HRegionInfo hri_a = mergingRegions.get(1);
3885 HRegionInfo hri_b = mergingRegions.get(2);
3886
3887 RegionState rs_p = regionStates.getRegionState(p);
3888 RegionState rs_a = regionStates.getRegionState(hri_a);
3889 RegionState rs_b = regionStates.getRegionState(hri_b);
3890
3891 if (!((rs_a == null || rs_a.isOpenOrMergingOnServer(sn))
3892 && (rs_b == null || rs_b.isOpenOrMergingOnServer(sn))
3893 && (rs_p == null || rs_p.isOpenOrMergingNewOnServer(sn)))) {
3894 LOG.warn("Dropped merging! Not in state good for MERGING; rs_p="
3895 + rs_p + ", rs_a=" + rs_a + ", rs_b=" + rs_b);
3896 return false;
3897 }
3898
3899 EventType et = rt.getEventType();
3900 if (et == EventType.RS_ZK_REQUEST_REGION_MERGE) {
3901 try {
3902 RegionMergeCoordination.RegionMergeDetails std =
3903 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
3904 .getRegionMergeCoordination().getDefaultDetails();
3905 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
3906 .getRegionMergeCoordination().processRegionMergeRequest(p, hri_a, hri_b, sn, std);
3907 if (((ZkRegionMergeCoordination.ZkRegionMergeDetails) std).getZnodeVersion() == -1) {
3908 byte[] data = ZKAssign.getData(watcher, encodedName);
3909 EventType currentType = null;
3910 if (data != null) {
3911 RegionTransition newRt = RegionTransition.parseFrom(data);
3912 currentType = newRt.getEventType();
3913 }
3914 if (currentType == null || (currentType != EventType.RS_ZK_REGION_MERGED
3915 && currentType != EventType.RS_ZK_REGION_MERGING)) {
3916 LOG.warn("Failed to transition pending_merge node "
3917 + encodedName + " to merging, it's now " + currentType);
3918 return false;
3919 }
3920 }
3921 } catch (Exception e) {
3922 LOG.warn("Failed to transition pending_merge node "
3923 + encodedName + " to merging", e);
3924 return false;
3925 }
3926 }
3927
3928 synchronized (regionStates) {
3929 regionStates.updateRegionState(hri_a, State.MERGING);
3930 regionStates.updateRegionState(hri_b, State.MERGING);
3931 regionStates.updateRegionState(p, State.MERGING_NEW, sn);
3932
3933 if (et != EventType.RS_ZK_REGION_MERGED) {
3934 this.mergingRegions.put(encodedName,
3935 new PairOfSameType<HRegionInfo>(hri_a, hri_b));
3936 } else {
3937 this.mergingRegions.remove(encodedName);
3938 regionOffline(hri_a, State.MERGED);
3939 regionOffline(hri_b, State.MERGED);
3940 regionOnline(p, sn);
3941 }
3942 }
3943
3944 if (et == EventType.RS_ZK_REGION_MERGED) {
3945 doMergingOfReplicas(p, hri_a, hri_b);
3946 LOG.debug("Handling MERGED event for " + encodedName + "; deleting node");
3947
3948 try {
3949 boolean successful = false;
3950 while (!successful) {
3951
3952
3953 successful = ZKAssign.deleteNode(watcher, encodedName,
3954 EventType.RS_ZK_REGION_MERGED, sn);
3955 }
3956 } catch (KeeperException e) {
3957 if (e instanceof NoNodeException) {
3958 String znodePath = ZKUtil.joinZNode(watcher.splitLogZNode, encodedName);
3959 LOG.debug("The znode " + znodePath + " does not exist. May be deleted already.");
3960 } else {
3961 server.abort("Error deleting MERGED node " + encodedName, e);
3962 }
3963 }
3964 LOG.info("Handled MERGED event; merged=" + p.getRegionNameAsString()
3965 + ", region_a=" + hri_a.getRegionNameAsString() + ", region_b="
3966 + hri_b.getRegionNameAsString() + ", on " + sn);
3967
3968
3969 if (tableStateManager.isTableState(p.getTable(),
3970 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3971 unassign(p);
3972 }
3973 }
3974 return true;
3975 }
3976
3977
3978
3979
3980 private boolean handleRegionSplitting(final RegionTransition rt, final String encodedName,
3981 final String prettyPrintedRegionName, final ServerName sn) {
3982 if (!serverManager.isServerOnline(sn)) {
3983 LOG.warn("Dropped splitting! ServerName=" + sn + " unknown.");
3984 return false;
3985 }
3986 byte [] payloadOfSplitting = rt.getPayload();
3987 List<HRegionInfo> splittingRegions;
3988 try {
3989 splittingRegions = HRegionInfo.parseDelimitedFrom(
3990 payloadOfSplitting, 0, payloadOfSplitting.length);
3991 } catch (IOException e) {
3992 LOG.error("Dropped splitting! Failed reading " + rt.getEventType()
3993 + " payload for " + prettyPrintedRegionName);
3994 return false;
3995 }
3996 assert splittingRegions.size() == 2;
3997 HRegionInfo hri_a = splittingRegions.get(0);
3998 HRegionInfo hri_b = splittingRegions.get(1);
3999
4000 RegionState rs_p = regionStates.getRegionState(encodedName);
4001 RegionState rs_a = regionStates.getRegionState(hri_a);
4002 RegionState rs_b = regionStates.getRegionState(hri_b);
4003
4004 if (!((rs_p == null || rs_p.isOpenOrSplittingOnServer(sn))
4005 && (rs_a == null || rs_a.isOpenOrSplittingNewOnServer(sn))
4006 && (rs_b == null || rs_b.isOpenOrSplittingNewOnServer(sn)))) {
4007 LOG.warn("Dropped splitting! Not in state good for SPLITTING; rs_p="
4008 + rs_p + ", rs_a=" + rs_a + ", rs_b=" + rs_b);
4009 return false;
4010 }
4011
4012 if (rs_p == null) {
4013
4014 rs_p = regionStates.updateRegionState(rt, State.OPEN);
4015 if (rs_p == null) {
4016 LOG.warn("Received splitting for region " + prettyPrintedRegionName
4017 + " from server " + sn + " but it doesn't exist anymore,"
4018 + " probably already processed its split");
4019 return false;
4020 }
4021 regionStates.regionOnline(rs_p.getRegion(), sn);
4022 }
4023
4024 HRegionInfo p = rs_p.getRegion();
4025 EventType et = rt.getEventType();
4026 if (et == EventType.RS_ZK_REQUEST_REGION_SPLIT) {
4027 try {
4028 SplitTransactionDetails std =
4029 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
4030 .getSplitTransactionCoordination().getDefaultDetails();
4031 if (((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
4032 .getSplitTransactionCoordination().processTransition(p, hri_a, hri_b, sn, std) == -1) {
4033 byte[] data = ZKAssign.getData(watcher, encodedName);
4034 EventType currentType = null;
4035 if (data != null) {
4036 RegionTransition newRt = RegionTransition.parseFrom(data);
4037 currentType = newRt.getEventType();
4038 }
4039 if (currentType == null
4040 || (currentType != EventType.RS_ZK_REGION_SPLIT && currentType != EventType.RS_ZK_REGION_SPLITTING)) {
4041 LOG.warn("Failed to transition pending_split node " + encodedName
4042 + " to splitting, it's now " + currentType);
4043 return false;
4044 }
4045 }
4046 } catch (Exception e) {
4047 LOG.warn("Failed to transition pending_split node " + encodedName + " to splitting", e);
4048 return false;
4049 }
4050 }
4051
4052 synchronized (regionStates) {
4053 splitRegions.put(p, new PairOfSameType<HRegionInfo>(hri_a, hri_b));
4054 regionStates.updateRegionState(hri_a, State.SPLITTING_NEW, sn);
4055 regionStates.updateRegionState(hri_b, State.SPLITTING_NEW, sn);
4056 regionStates.updateRegionState(rt, State.SPLITTING);
4057
4058
4059
4060 if (TEST_SKIP_SPLIT_HANDLING) {
4061 LOG.warn("Skipping split message, TEST_SKIP_SPLIT_HANDLING is set");
4062 return true;
4063 }
4064
4065 if (et == EventType.RS_ZK_REGION_SPLIT) {
4066 regionOffline(p, State.SPLIT);
4067 regionOnline(hri_a, sn);
4068 regionOnline(hri_b, sn);
4069 splitRegions.remove(p);
4070 }
4071 }
4072
4073 if (et == EventType.RS_ZK_REGION_SPLIT) {
4074
4075 doSplittingOfReplicas(rs_p.getRegion(), hri_a, hri_b);
4076 LOG.debug("Handling SPLIT event for " + encodedName + "; deleting node");
4077
4078 try {
4079 boolean successful = false;
4080 while (!successful) {
4081
4082
4083 successful = ZKAssign.deleteNode(watcher, encodedName,
4084 EventType.RS_ZK_REGION_SPLIT, sn);
4085 }
4086 } catch (KeeperException e) {
4087 if (e instanceof NoNodeException) {
4088 String znodePath = ZKUtil.joinZNode(watcher.splitLogZNode, encodedName);
4089 LOG.debug("The znode " + znodePath + " does not exist. May be deleted already.");
4090 } else {
4091 server.abort("Error deleting SPLIT node " + encodedName, e);
4092 }
4093 }
4094 LOG.info("Handled SPLIT event; parent=" + p.getRegionNameAsString()
4095 + ", daughter a=" + hri_a.getRegionNameAsString() + ", daughter b="
4096 + hri_b.getRegionNameAsString() + ", on " + sn);
4097
4098
4099 if (tableStateManager.isTableState(p.getTable(),
4100 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
4101 unassign(hri_a);
4102 unassign(hri_b);
4103 }
4104 }
4105 return true;
4106 }
4107
4108 private void doMergingOfReplicas(HRegionInfo mergedHri, final HRegionInfo hri_a,
4109 final HRegionInfo hri_b) {
4110
4111
4112 List<HRegionInfo> unmergedRegions = new ArrayList<HRegionInfo>();
4113 unmergedRegions.add(hri_a);
4114 unmergedRegions.add(hri_b);
4115 Map<ServerName, List<HRegionInfo>> map = regionStates.getRegionAssignments(unmergedRegions);
4116 Collection<List<HRegionInfo>> c = map.values();
4117 for (List<HRegionInfo> l : c) {
4118 for (HRegionInfo h : l) {
4119 if (!RegionReplicaUtil.isDefaultReplica(h)) {
4120 LOG.debug("Unassigning un-merged replica " + h);
4121 unassign(h);
4122 }
4123 }
4124 }
4125 int numReplicas = 1;
4126 try {
4127 numReplicas = ((MasterServices)server).getTableDescriptors().get(mergedHri.getTable()).
4128 getRegionReplication();
4129 } catch (IOException e) {
4130 LOG.warn("Couldn't get the replication attribute of the table " + mergedHri.getTable() +
4131 " due to " + e.getMessage() + ". The assignment of replicas for the merged region " +
4132 "will not be done");
4133 }
4134 List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
4135 for (int i = 1; i < numReplicas; i++) {
4136 regions.add(RegionReplicaUtil.getRegionInfoForReplica(mergedHri, i));
4137 }
4138 try {
4139 assign(regions);
4140 } catch (IOException ioe) {
4141 LOG.warn("Couldn't assign all replica(s) of region " + mergedHri + " because of " +
4142 ioe.getMessage());
4143 } catch (InterruptedException ie) {
4144 LOG.warn("Couldn't assign all replica(s) of region " + mergedHri+ " because of " +
4145 ie.getMessage());
4146 }
4147 }
4148
4149 private void doSplittingOfReplicas(final HRegionInfo parentHri, final HRegionInfo hri_a,
4150 final HRegionInfo hri_b) {
4151
4152
4153
4154 int numReplicas = 1;
4155 try {
4156 numReplicas = ((MasterServices)server).getTableDescriptors().get(parentHri.getTable()).
4157 getRegionReplication();
4158 } catch (IOException e) {
4159 LOG.warn("Couldn't get the replication attribute of the table " + parentHri.getTable() +
4160 " due to " + e.getMessage() + ". The assignment of daughter replicas " +
4161 "replicas will not be done");
4162 }
4163
4164 List<HRegionInfo> parentRegion = new ArrayList<HRegionInfo>();
4165 parentRegion.add(parentHri);
4166 Map<ServerName, List<HRegionInfo>> currentAssign =
4167 regionStates.getRegionAssignments(parentRegion);
4168 Collection<List<HRegionInfo>> c = currentAssign.values();
4169 for (List<HRegionInfo> l : c) {
4170 for (HRegionInfo h : l) {
4171 if (!RegionReplicaUtil.isDefaultReplica(h)) {
4172 LOG.debug("Unassigning parent's replica " + h);
4173 unassign(h);
4174 }
4175 }
4176 }
4177
4178 Map<HRegionInfo, ServerName> map = new HashMap<HRegionInfo, ServerName>();
4179 for (int i = 1; i < numReplicas; i++) {
4180 prepareDaughterReplicaForAssignment(hri_a, parentHri, i, map);
4181 prepareDaughterReplicaForAssignment(hri_b, parentHri, i, map);
4182 }
4183 try {
4184 assign(map);
4185 } catch (IOException e) {
4186 LOG.warn("Caught exception " + e + " while trying to assign replica(s) of daughter(s)");
4187 } catch (InterruptedException e) {
4188 LOG.warn("Caught exception " + e + " while trying to assign replica(s) of daughter(s)");
4189 }
4190 }
4191
4192 private void prepareDaughterReplicaForAssignment(HRegionInfo daughterHri, HRegionInfo parentHri,
4193 int replicaId, Map<HRegionInfo, ServerName> map) {
4194 HRegionInfo parentReplica = RegionReplicaUtil.getRegionInfoForReplica(parentHri, replicaId);
4195 HRegionInfo daughterReplica = RegionReplicaUtil.getRegionInfoForReplica(daughterHri,
4196 replicaId);
4197 LOG.debug("Created replica region for daughter " + daughterReplica);
4198 ServerName sn;
4199 if ((sn = regionStates.getRegionServerOfRegion(parentReplica)) != null) {
4200 map.put(daughterReplica, sn);
4201 } else {
4202 List<ServerName> servers = serverManager.getOnlineServersList();
4203 sn = servers.get((new Random(System.currentTimeMillis())).nextInt(servers.size()));
4204 map.put(daughterReplica, sn);
4205 }
4206 }
4207
4208 public Set<HRegionInfo> getReplicasToClose() {
4209 return replicasToClose;
4210 }
4211
4212
4213
4214
4215
4216
4217 private void regionOffline(final HRegionInfo regionInfo, final State state) {
4218 regionStates.regionOffline(regionInfo, state);
4219 removeClosedRegion(regionInfo);
4220
4221 clearRegionPlan(regionInfo);
4222 balancer.regionOffline(regionInfo);
4223
4224
4225 sendRegionClosedNotification(regionInfo);
4226
4227 if (state != null && state.equals(State.SPLIT)) {
4228 Collection<HRegionInfo> c = new ArrayList<HRegionInfo>(1);
4229 c.add(regionInfo);
4230 Map<ServerName, List<HRegionInfo>> map = regionStates.getRegionAssignments(c);
4231 Collection<List<HRegionInfo>> allReplicas = map.values();
4232 for (List<HRegionInfo> list : allReplicas) {
4233 replicasToClose.addAll(list);
4234 }
4235 }
4236 else if (state != null && state.equals(State.MERGED)) {
4237 Collection<HRegionInfo> c = new ArrayList<HRegionInfo>(1);
4238 c.add(regionInfo);
4239 Map<ServerName, List<HRegionInfo>> map = regionStates.getRegionAssignments(c);
4240 Collection<List<HRegionInfo>> allReplicas = map.values();
4241 for (List<HRegionInfo> list : allReplicas) {
4242 replicasToClose.addAll(list);
4243 }
4244 }
4245 }
4246
4247 private void sendRegionOpenedNotification(final HRegionInfo regionInfo,
4248 final ServerName serverName) {
4249 if (!this.listeners.isEmpty()) {
4250 for (AssignmentListener listener : this.listeners) {
4251 listener.regionOpened(regionInfo, serverName);
4252 }
4253 }
4254 }
4255
4256 private void sendRegionClosedNotification(final HRegionInfo regionInfo) {
4257 if (!this.listeners.isEmpty()) {
4258 for (AssignmentListener listener : this.listeners) {
4259 listener.regionClosed(regionInfo);
4260 }
4261 }
4262 }
4263
4264
4265
4266
4267
4268
4269
4270
4271
4272
4273
4274
4275
4276
4277
4278
4279
4280
4281
4282
4283
4284
4285
4286
4287
4288
4289
4290
4291
4292
4293
4294
4295
4296
4297
4298
4299
4300
4301
4302
4303
4304
4305
4306
4307
4308 protected String onRegionTransition(final ServerName serverName,
4309 final RegionStateTransition transition) {
4310 TransitionCode code = transition.getTransitionCode();
4311 HRegionInfo hri = HRegionInfo.convert(transition.getRegionInfo(0));
4312 RegionState current = regionStates.getRegionState(hri);
4313 if (LOG.isDebugEnabled()) {
4314 LOG.debug("Got transition " + code + " for "
4315 + (current != null ? current.toString() : hri.getShortNameToLog())
4316 + " from " + serverName);
4317 }
4318 String errorMsg = null;
4319 switch (code) {
4320 case OPENED:
4321 if (current != null && current.isOpened() && current.isOnServer(serverName)) {
4322 LOG.info("Region " + hri.getShortNameToLog() + " is already " + current.getState() + " on "
4323 + serverName);
4324 break;
4325 }
4326 case FAILED_OPEN:
4327 if (current == null
4328 || !current.isPendingOpenOrOpeningOnServer(serverName)) {
4329 errorMsg = hri.getShortNameToLog()
4330 + " is not pending open on " + serverName;
4331 } else if (code == TransitionCode.FAILED_OPEN) {
4332 onRegionFailedOpen(hri, serverName);
4333 } else {
4334 long openSeqNum = HConstants.NO_SEQNUM;
4335 if (transition.hasOpenSeqNum()) {
4336 openSeqNum = transition.getOpenSeqNum();
4337 }
4338 if (openSeqNum < 0) {
4339 errorMsg = "Newly opened region has invalid open seq num " + openSeqNum;
4340 } else {
4341 onRegionOpen(hri, serverName, openSeqNum);
4342 }
4343 }
4344 break;
4345
4346 case CLOSED:
4347 if (current == null
4348 || !current.isPendingCloseOrClosingOnServer(serverName)) {
4349 errorMsg = hri.getShortNameToLog()
4350 + " is not pending close on " + serverName;
4351 } else {
4352 onRegionClosed(hri);
4353 }
4354 break;
4355
4356 case READY_TO_SPLIT:
4357 try {
4358 regionStateListener.onRegionSplit(hri);
4359 if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled(
4360 Admin.MasterSwitchType.SPLIT)) {
4361 errorMsg = "split switch is off!";
4362 }
4363 } catch (IOException exp) {
4364 errorMsg = StringUtils.stringifyException(exp);
4365 }
4366 break;
4367 case SPLIT_PONR:
4368 case SPLIT:
4369 case SPLIT_REVERTED:
4370 errorMsg =
4371 onRegionSplit(serverName, code, hri, HRegionInfo.convert(transition.getRegionInfo(1)),
4372 HRegionInfo.convert(transition.getRegionInfo(2)));
4373 if (org.apache.commons.lang.StringUtils.isEmpty(errorMsg)) {
4374 try {
4375 regionStateListener.onRegionSplitReverted(hri);
4376 } catch (IOException exp) {
4377 LOG.warn(StringUtils.stringifyException(exp));
4378 }
4379 }
4380 break;
4381 case READY_TO_MERGE:
4382 if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled(
4383 Admin.MasterSwitchType.MERGE)) {
4384 errorMsg = "merge switch is off!";
4385 }
4386 break;
4387 case MERGE_PONR:
4388 case MERGED:
4389 case MERGE_REVERTED:
4390 errorMsg = onRegionMerge(serverName, code, hri,
4391 HRegionInfo.convert(transition.getRegionInfo(1)),
4392 HRegionInfo.convert(transition.getRegionInfo(2)));
4393 if (code == TransitionCode.MERGED && org.apache.commons.lang.StringUtils.isEmpty(errorMsg)) {
4394 try {
4395 regionStateListener.onRegionMerged(hri);
4396 } catch (IOException exp) {
4397 errorMsg = StringUtils.stringifyException(exp);
4398 }
4399 }
4400 break;
4401
4402 default:
4403 errorMsg = "Unexpected transition code " + code;
4404 }
4405 if (errorMsg != null) {
4406 LOG.error("Failed to transtion region from " + current + " to "
4407 + code + " by " + serverName + ": " + errorMsg);
4408 }
4409 return errorMsg;
4410 }
4411
4412 private void processBogusAssignments(Map<ServerName, List<HRegionInfo>> bulkPlan) {
4413 if (bulkPlan.containsKey(LoadBalancer.BOGUS_SERVER_NAME)) {
4414
4415 for (HRegionInfo hri : bulkPlan.get(LoadBalancer.BOGUS_SERVER_NAME)) {
4416 regionStates.updateRegionState(hri, State.FAILED_OPEN);
4417 }
4418 bulkPlan.remove(LoadBalancer.BOGUS_SERVER_NAME);
4419 }
4420 }
4421
4422
4423
4424
4425 public LoadBalancer getBalancer() {
4426 return this.balancer;
4427 }
4428
4429 public Map<ServerName, List<HRegionInfo>>
4430 getSnapShotOfAssignment(Collection<HRegionInfo> infos) {
4431 return getRegionStates().getRegionAssignments(infos);
4432 }
4433
4434 void setRegionStateListener(RegionStateListener listener) {
4435 this.regionStateListener = listener;
4436 }
4437 }