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