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