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