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