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