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 boolean logRetries = true;
1699 if (t instanceof NotServingRegionException
1700 || t instanceof RegionServerStoppedException
1701 || t instanceof ServerNotRunningYetException) {
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 ((t instanceof FailedServerException) || (state != null &&
1712 t instanceof RegionAlreadyInTransitionException)) {
1713 long sleepTime = 0;
1714 Configuration conf = this.server.getConfiguration();
1715 if(t instanceof FailedServerException) {
1716 sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
1717 RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
1718 } else {
1719
1720 LOG.debug("update " + state + " the timestamp.");
1721 state.updateTimestampToNow();
1722 if (maxWaitTime < 0) {
1723 maxWaitTime =
1724 EnvironmentEdgeManager.currentTimeMillis()
1725 + conf.getLong(ALREADY_IN_TRANSITION_WAITTIME,
1726 DEFAULT_ALREADY_IN_TRANSITION_WAITTIME);
1727 }
1728 long now = EnvironmentEdgeManager.currentTimeMillis();
1729 if (now < maxWaitTime) {
1730 LOG.debug("Region is already in transition; "
1731 + "waiting up to " + (maxWaitTime - now) + "ms", t);
1732 sleepTime = 100;
1733 i--;
1734 logRetries = false;
1735 }
1736 }
1737 try {
1738 if (sleepTime > 0) {
1739 Thread.sleep(sleepTime);
1740 }
1741 } catch (InterruptedException ie) {
1742 LOG.warn("Failed to unassign "
1743 + region.getRegionNameAsString() + " since interrupted", ie);
1744 Thread.currentThread().interrupt();
1745 if (!tomActivated && state != null) {
1746 regionStates.updateRegionState(region, State.FAILED_CLOSE);
1747 }
1748 return;
1749 }
1750 }
1751
1752 if (logRetries) {
1753 LOG.info("Server " + server + " returned " + t + " for "
1754 + region.getRegionNameAsString() + ", try=" + i
1755 + " of " + this.maximumAttempts, t);
1756
1757 }
1758 }
1759 }
1760
1761 if (!tomActivated && state != null) {
1762 regionStates.updateRegionState(region, State.FAILED_CLOSE);
1763 }
1764 }
1765
1766
1767
1768
1769 private RegionState forceRegionStateToOffline(
1770 final HRegionInfo region, final boolean forceNewPlan) {
1771 RegionState state = regionStates.getRegionState(region);
1772 if (state == null) {
1773 LOG.warn("Assigning a region not in region states: " + region);
1774 state = regionStates.createRegionState(region);
1775 }
1776
1777 ServerName sn = state.getServerName();
1778 if (forceNewPlan && LOG.isDebugEnabled()) {
1779 LOG.debug("Force region state offline " + state);
1780 }
1781
1782 switch (state.getState()) {
1783 case OPEN:
1784 case OPENING:
1785 case PENDING_OPEN:
1786 case CLOSING:
1787 case PENDING_CLOSE:
1788 if (!forceNewPlan) {
1789 LOG.debug("Skip assigning " +
1790 region + ", it is already " + state);
1791 return null;
1792 }
1793 case FAILED_CLOSE:
1794 case FAILED_OPEN:
1795 unassign(region, state, -1, null, false, null);
1796 state = regionStates.getRegionState(region);
1797 if (state.isFailedClose()) {
1798
1799
1800 LOG.info("Skip assigning " +
1801 region + ", we couldn't close it: " + state);
1802 return null;
1803 }
1804 case OFFLINE:
1805
1806
1807
1808
1809
1810 if (regionStates.isServerDeadAndNotProcessed(sn)
1811 && wasRegionOnDeadServerByMeta(region, sn)) {
1812 LOG.info("Skip assigning " + region.getRegionNameAsString()
1813 + ", it is on a dead but not processed yet server");
1814 return null;
1815 }
1816 case CLOSED:
1817 break;
1818 default:
1819 LOG.error("Trying to assign region " + region
1820 + ", which is " + state);
1821 return null;
1822 }
1823 return state;
1824 }
1825
1826 private boolean wasRegionOnDeadServerByMeta(
1827 final HRegionInfo region, final ServerName sn) {
1828 try {
1829 if (region.isMetaRegion()) {
1830 ServerName server = catalogTracker.getMetaLocation();
1831 return regionStates.isServerDeadAndNotProcessed(server);
1832 }
1833 while (!server.isStopped()) {
1834 try {
1835 catalogTracker.waitForMeta();
1836 Pair<HRegionInfo, ServerName> r =
1837 MetaReader.getRegion(catalogTracker, region.getRegionName());
1838 ServerName server = r == null ? null : r.getSecond();
1839 return regionStates.isServerDeadAndNotProcessed(server);
1840 } catch (IOException ioe) {
1841 LOG.info("Received exception accessing hbase:meta during force assign "
1842 + region.getRegionNameAsString() + ", retrying", ioe);
1843 }
1844 }
1845 } catch (InterruptedException e) {
1846 Thread.currentThread().interrupt();
1847 LOG.info("Interrupted accessing hbase:meta", e);
1848 }
1849
1850 return regionStates.isServerDeadAndNotProcessed(sn);
1851 }
1852
1853
1854
1855
1856
1857
1858
1859 private void assign(RegionState state,
1860 final boolean setOfflineInZK, final boolean forceNewPlan) {
1861 long startTime = EnvironmentEdgeManager.currentTimeMillis();
1862 try {
1863 Configuration conf = server.getConfiguration();
1864 RegionState currentState = state;
1865 int versionOfOfflineNode = -1;
1866 RegionPlan plan = null;
1867 long maxWaitTime = -1;
1868 HRegionInfo region = state.getRegion();
1869 RegionOpeningState regionOpenState;
1870 Throwable previousException = null;
1871 for (int i = 1; i <= maximumAttempts; i++) {
1872 if (server.isStopped() || server.isAborted()) {
1873 LOG.info("Skip assigning " + region.getRegionNameAsString()
1874 + ", the server is stopped/aborted");
1875 return;
1876 }
1877 if (plan == null) {
1878 try {
1879 plan = getRegionPlan(region, forceNewPlan);
1880 } catch (HBaseIOException e) {
1881 LOG.warn("Failed to get region plan", e);
1882 }
1883 }
1884 if (plan == null) {
1885 LOG.warn("Unable to determine a plan to assign " + region);
1886 if (tomActivated){
1887 this.timeoutMonitor.setAllRegionServersOffline(true);
1888 } else {
1889 if (region.isMetaRegion()) {
1890 try {
1891 Thread.sleep(this.sleepTimeBeforeRetryingMetaAssignment);
1892 if (i == maximumAttempts) i = 1;
1893 continue;
1894 } catch (InterruptedException e) {
1895 LOG.error("Got exception while waiting for hbase:meta assignment");
1896 Thread.currentThread().interrupt();
1897 }
1898 }
1899 regionStates.updateRegionState(region, State.FAILED_OPEN);
1900 }
1901 return;
1902 }
1903 if (setOfflineInZK && versionOfOfflineNode == -1) {
1904
1905
1906 versionOfOfflineNode = setOfflineInZooKeeper(currentState, plan.getDestination());
1907 if (versionOfOfflineNode != -1) {
1908 if (isDisabledorDisablingRegionInRIT(region)) {
1909 return;
1910 }
1911
1912
1913
1914
1915
1916
1917 TableName tableName = region.getTable();
1918 if (!zkTable.isEnablingTable(tableName) && !zkTable.isEnabledTable(tableName)) {
1919 LOG.debug("Setting table " + tableName + " to ENABLED state.");
1920 setEnabledTable(tableName);
1921 }
1922 }
1923 }
1924 if (setOfflineInZK && versionOfOfflineNode == -1) {
1925 LOG.info("Unable to set offline in ZooKeeper to assign " + region);
1926
1927
1928
1929
1930 if (!server.isAborted()) {
1931 continue;
1932 }
1933 }
1934 LOG.info("Assigning " + region.getRegionNameAsString() +
1935 " to " + plan.getDestination().toString());
1936
1937 currentState = regionStates.updateRegionState(region,
1938 State.PENDING_OPEN, plan.getDestination());
1939
1940 boolean needNewPlan;
1941 final String assignMsg = "Failed assignment of " + region.getRegionNameAsString() +
1942 " to " + plan.getDestination();
1943 try {
1944 List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
1945 if (this.shouldAssignRegionsWithFavoredNodes) {
1946 favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
1947 }
1948 regionOpenState = serverManager.sendRegionOpen(
1949 plan.getDestination(), region, versionOfOfflineNode, favoredNodes);
1950
1951 if (regionOpenState == RegionOpeningState.FAILED_OPENING) {
1952
1953 needNewPlan = true;
1954 LOG.warn(assignMsg + ", regionserver says 'FAILED_OPENING', " +
1955 " trying to assign elsewhere instead; " +
1956 "try=" + i + " of " + this.maximumAttempts);
1957 } else {
1958
1959 if (regionOpenState == RegionOpeningState.ALREADY_OPENED) {
1960 processAlreadyOpenedRegion(region, plan.getDestination());
1961 }
1962 return;
1963 }
1964
1965 } catch (Throwable t) {
1966 if (t instanceof RemoteException) {
1967 t = ((RemoteException) t).unwrapRemoteException();
1968 }
1969 previousException = t;
1970
1971
1972
1973
1974 boolean hold = (t instanceof RegionAlreadyInTransitionException ||
1975 t instanceof ServerNotRunningYetException);
1976
1977
1978
1979
1980
1981
1982 boolean retry = !hold && (t instanceof java.net.SocketTimeoutException
1983 && this.serverManager.isServerOnline(plan.getDestination()));
1984
1985
1986 if (hold) {
1987 LOG.warn(assignMsg + ", waiting a little before trying on the same region server " +
1988 "try=" + i + " of " + this.maximumAttempts, t);
1989
1990 if (maxWaitTime < 0) {
1991 if (t instanceof RegionAlreadyInTransitionException) {
1992 maxWaitTime = EnvironmentEdgeManager.currentTimeMillis()
1993 + this.server.getConfiguration().getLong(ALREADY_IN_TRANSITION_WAITTIME,
1994 DEFAULT_ALREADY_IN_TRANSITION_WAITTIME);
1995 } else {
1996 maxWaitTime = this.server.getConfiguration().
1997 getLong("hbase.regionserver.rpc.startup.waittime", 60000);
1998 }
1999 }
2000 try {
2001 needNewPlan = false;
2002 long now = EnvironmentEdgeManager.currentTimeMillis();
2003 if (now < maxWaitTime) {
2004 LOG.debug("Server is not yet up or region is already in transition; "
2005 + "waiting up to " + (maxWaitTime - now) + "ms", t);
2006 Thread.sleep(100);
2007 i--;
2008 } else if (!(t instanceof RegionAlreadyInTransitionException)) {
2009 LOG.debug("Server is not up for a while; try a new one", t);
2010 needNewPlan = true;
2011 }
2012 } catch (InterruptedException ie) {
2013 LOG.warn("Failed to assign "
2014 + region.getRegionNameAsString() + " since interrupted", ie);
2015 Thread.currentThread().interrupt();
2016 if (!tomActivated) {
2017 regionStates.updateRegionState(region, State.FAILED_OPEN);
2018 }
2019 return;
2020 }
2021 } else if (retry) {
2022 needNewPlan = false;
2023 LOG.warn(assignMsg + ", trying to assign to the same region server " +
2024 "try=" + i + " of " + this.maximumAttempts, t);
2025 } else {
2026 needNewPlan = true;
2027 LOG.warn(assignMsg + ", trying to assign elsewhere instead;" +
2028 " try=" + i + " of " + this.maximumAttempts, t);
2029 }
2030 }
2031
2032 if (i == this.maximumAttempts) {
2033
2034
2035 continue;
2036 }
2037
2038
2039
2040
2041 if (needNewPlan) {
2042
2043
2044
2045
2046 RegionPlan newPlan = null;
2047 try {
2048 newPlan = getRegionPlan(region, true);
2049 } catch (HBaseIOException e) {
2050 LOG.warn("Failed to get region plan", e);
2051 }
2052 if (newPlan == null) {
2053 if (tomActivated) {
2054 this.timeoutMonitor.setAllRegionServersOffline(true);
2055 } else {
2056 regionStates.updateRegionState(region, State.FAILED_OPEN);
2057 }
2058 LOG.warn("Unable to find a viable location to assign region " +
2059 region.getRegionNameAsString());
2060 return;
2061 }
2062
2063 if (plan != newPlan && !plan.getDestination().equals(newPlan.getDestination())) {
2064
2065
2066
2067 currentState = regionStates.updateRegionState(region, State.OFFLINE);
2068 versionOfOfflineNode = -1;
2069 plan = newPlan;
2070 } else if(plan.getDestination().equals(newPlan.getDestination()) &&
2071 previousException instanceof FailedServerException) {
2072 try {
2073 LOG.info("Trying to re-assign " + region.getRegionNameAsString() +
2074 " to the same failed server.");
2075 Thread.sleep(1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
2076 RpcClient.FAILED_SERVER_EXPIRY_DEFAULT));
2077 } catch (InterruptedException ie) {
2078 LOG.warn("Failed to assign "
2079 + region.getRegionNameAsString() + " since interrupted", ie);
2080 Thread.currentThread().interrupt();
2081 if (!tomActivated) {
2082 regionStates.updateRegionState(region, State.FAILED_OPEN);
2083 }
2084 return;
2085 }
2086 }
2087 }
2088 }
2089
2090 if (!tomActivated) {
2091 regionStates.updateRegionState(region, State.FAILED_OPEN);
2092 }
2093 } finally {
2094 metricsAssignmentManager.updateAssignmentTime(EnvironmentEdgeManager.currentTimeMillis() - startTime);
2095 }
2096 }
2097
2098 private void processAlreadyOpenedRegion(HRegionInfo region, ServerName sn) {
2099
2100
2101
2102 LOG.debug("ALREADY_OPENED " + region.getRegionNameAsString()
2103 + " to " + sn);
2104 String encodedName = region.getEncodedName();
2105 deleteNodeInStates(encodedName, "offline", sn, EventType.M_ZK_REGION_OFFLINE);
2106 regionStates.regionOnline(region, sn);
2107 }
2108
2109 private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
2110 TableName tableName = region.getTable();
2111 boolean disabled = this.zkTable.isDisabledTable(tableName);
2112 if (disabled || this.zkTable.isDisablingTable(tableName)) {
2113 LOG.info("Table " + tableName + (disabled ? " disabled;" : " disabling;") +
2114 " skipping assign of " + region.getRegionNameAsString());
2115 offlineDisabledRegion(region);
2116 return true;
2117 }
2118 return false;
2119 }
2120
2121
2122
2123
2124
2125
2126
2127
2128 private int setOfflineInZooKeeper(final RegionState state, final ServerName destination) {
2129 if (!state.isClosed() && !state.isOffline()) {
2130 String msg = "Unexpected state : " + state + " .. Cannot transit it to OFFLINE.";
2131 this.server.abort(msg, new IllegalStateException(msg));
2132 return -1;
2133 }
2134 regionStates.updateRegionState(state.getRegion(), State.OFFLINE);
2135 int versionOfOfflineNode;
2136 try {
2137
2138 versionOfOfflineNode = ZKAssign.createOrForceNodeOffline(watcher,
2139 state.getRegion(), destination);
2140 if (versionOfOfflineNode == -1) {
2141 LOG.warn("Attempted to create/force node into OFFLINE state before "
2142 + "completing assignment but failed to do so for " + state);
2143 return -1;
2144 }
2145 } catch (KeeperException e) {
2146 server.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
2147 return -1;
2148 }
2149 return versionOfOfflineNode;
2150 }
2151
2152
2153
2154
2155
2156
2157 private RegionPlan getRegionPlan(final HRegionInfo region,
2158 final boolean forceNewPlan) throws HBaseIOException {
2159 return getRegionPlan(region, null, forceNewPlan);
2160 }
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171 private RegionPlan getRegionPlan(final HRegionInfo region,
2172 final ServerName serverToExclude, final boolean forceNewPlan) throws HBaseIOException {
2173
2174 final String encodedName = region.getEncodedName();
2175 final List<ServerName> destServers =
2176 serverManager.createDestinationServersList(serverToExclude);
2177
2178 if (destServers.isEmpty()){
2179 LOG.warn("Can't move " + encodedName +
2180 ", there is no destination server available.");
2181 return null;
2182 }
2183
2184 RegionPlan randomPlan = null;
2185 boolean newPlan = false;
2186 RegionPlan existingPlan;
2187
2188 synchronized (this.regionPlans) {
2189 existingPlan = this.regionPlans.get(encodedName);
2190
2191 if (existingPlan != null && existingPlan.getDestination() != null) {
2192 LOG.debug("Found an existing plan for " + region.getRegionNameAsString()
2193 + " destination server is " + existingPlan.getDestination() +
2194 " accepted as a dest server = " + destServers.contains(existingPlan.getDestination()));
2195 }
2196
2197 if (forceNewPlan
2198 || existingPlan == null
2199 || existingPlan.getDestination() == null
2200 || !destServers.contains(existingPlan.getDestination())) {
2201 newPlan = true;
2202 randomPlan = new RegionPlan(region, null,
2203 balancer.randomAssignment(region, destServers));
2204 if (!region.isMetaTable() && shouldAssignRegionsWithFavoredNodes) {
2205 List<HRegionInfo> regions = new ArrayList<HRegionInfo>(1);
2206 regions.add(region);
2207 try {
2208 processFavoredNodes(regions);
2209 } catch (IOException ie) {
2210 LOG.warn("Ignoring exception in processFavoredNodes " + ie);
2211 }
2212 }
2213 this.regionPlans.put(encodedName, randomPlan);
2214 }
2215 }
2216
2217 if (newPlan) {
2218 if (randomPlan.getDestination() == null) {
2219 LOG.warn("Can't find a destination for " + encodedName);
2220 return null;
2221 }
2222 LOG.debug("No previous transition plan found (or ignoring " +
2223 "an existing plan) for " + region.getRegionNameAsString() +
2224 "; generated random plan=" + randomPlan + "; " +
2225 serverManager.countOfRegionServers() +
2226 " (online=" + serverManager.getOnlineServers().size() +
2227 ", available=" + destServers.size() + ") available servers" +
2228 ", forceNewPlan=" + forceNewPlan);
2229 return randomPlan;
2230 }
2231 LOG.debug("Using pre-existing plan for " +
2232 region.getRegionNameAsString() + "; plan=" + existingPlan);
2233 return existingPlan;
2234 }
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249 public void unassign(HRegionInfo region) {
2250 unassign(region, false);
2251 }
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268 public void unassign(HRegionInfo region, boolean force, ServerName dest) {
2269
2270 LOG.debug("Starting unassign of " + region.getRegionNameAsString()
2271 + " (offlining), current state: " + regionStates.getRegionState(region));
2272
2273 String encodedName = region.getEncodedName();
2274
2275 int versionOfClosingNode = -1;
2276
2277
2278 ReentrantLock lock = locker.acquireLock(encodedName);
2279 RegionState state = regionStates.getRegionTransitionState(encodedName);
2280 boolean reassign = true;
2281 try {
2282 if (state == null) {
2283
2284
2285 state = regionStates.getRegionState(encodedName);
2286 if (state != null && state.isUnassignable()) {
2287 LOG.info("Attempting to unassign " + state + ", ignored");
2288
2289 return;
2290 }
2291
2292 try {
2293 if (state == null || state.getServerName() == null) {
2294
2295
2296 LOG.warn("Attempting to unassign a region not in RegionStates"
2297 + region.getRegionNameAsString() + ", offlined");
2298 regionOffline(region);
2299 return;
2300 }
2301 versionOfClosingNode = ZKAssign.createNodeClosing(
2302 watcher, region, state.getServerName());
2303 if (versionOfClosingNode == -1) {
2304 LOG.info("Attempting to unassign " +
2305 region.getRegionNameAsString() + " but ZK closing node "
2306 + "can't be created.");
2307 reassign = false;
2308 return;
2309 }
2310 } catch (KeeperException e) {
2311 if (e instanceof NodeExistsException) {
2312
2313
2314
2315
2316 NodeExistsException nee = (NodeExistsException)e;
2317 String path = nee.getPath();
2318 try {
2319 if (isSplitOrSplittingOrMergedOrMerging(path)) {
2320 LOG.debug(path + " is SPLIT or SPLITTING or MERGED or MERGING; " +
2321 "skipping unassign because region no longer exists -- its split or merge");
2322 reassign = false;
2323 return;
2324 }
2325 } catch (KeeperException.NoNodeException ke) {
2326 LOG.warn("Failed getData on SPLITTING/SPLIT at " + path +
2327 "; presuming split and that the region to unassign, " +
2328 encodedName + ", no longer exists -- confirm", ke);
2329 return;
2330 } catch (KeeperException ke) {
2331 LOG.error("Unexpected zk state", ke);
2332 } catch (DeserializationException de) {
2333 LOG.error("Failed parse", de);
2334 }
2335 }
2336
2337 server.abort("Unexpected ZK exception creating node CLOSING", e);
2338 reassign = false;
2339 return;
2340 }
2341 state = regionStates.updateRegionState(region, State.PENDING_CLOSE);
2342 } else if (state.isFailedOpen()) {
2343
2344 regionOffline(region);
2345 return;
2346 } else if (force && state.isPendingCloseOrClosing()) {
2347 LOG.debug("Attempting to unassign " + region.getRegionNameAsString() +
2348 " which is already " + state.getState() +
2349 " but forcing to send a CLOSE RPC again ");
2350 if (state.isFailedClose()) {
2351 state = regionStates.updateRegionState(region, State.PENDING_CLOSE);
2352 }
2353 state.updateTimestampToNow();
2354 } else {
2355 LOG.debug("Attempting to unassign " +
2356 region.getRegionNameAsString() + " but it is " +
2357 "already in transition (" + state.getState() + ", force=" + force + ")");
2358 return;
2359 }
2360
2361 unassign(region, state, versionOfClosingNode, dest, true, null);
2362 } finally {
2363 lock.unlock();
2364
2365
2366 if (reassign && regionStates.isRegionOffline(region)) {
2367 assign(region, true);
2368 }
2369 }
2370 }
2371
2372 public void unassign(HRegionInfo region, boolean force){
2373 unassign(region, force, null);
2374 }
2375
2376
2377
2378
2379 public void deleteClosingOrClosedNode(HRegionInfo region, ServerName sn) {
2380 String encodedName = region.getEncodedName();
2381 deleteNodeInStates(encodedName, "closing", sn, EventType.M_ZK_REGION_CLOSING,
2382 EventType.RS_ZK_REGION_CLOSED);
2383 }
2384
2385
2386
2387
2388
2389
2390
2391 private boolean isSplitOrSplittingOrMergedOrMerging(final String path)
2392 throws KeeperException, DeserializationException {
2393 boolean result = false;
2394
2395
2396 byte [] data = ZKAssign.getData(watcher, path);
2397 if (data == null) {
2398 LOG.info("Node " + path + " is gone");
2399 return false;
2400 }
2401 RegionTransition rt = RegionTransition.parseFrom(data);
2402 switch (rt.getEventType()) {
2403 case RS_ZK_REQUEST_REGION_SPLIT:
2404 case RS_ZK_REGION_SPLIT:
2405 case RS_ZK_REGION_SPLITTING:
2406 case RS_ZK_REQUEST_REGION_MERGE:
2407 case RS_ZK_REGION_MERGED:
2408 case RS_ZK_REGION_MERGING:
2409 result = true;
2410 break;
2411 default:
2412 LOG.info("Node " + path + " is in " + rt.getEventType());
2413 break;
2414 }
2415 return result;
2416 }
2417
2418
2419
2420
2421
2422
2423 public int getNumRegionsOpened() {
2424 return numRegionsOpened.get();
2425 }
2426
2427
2428
2429
2430
2431
2432
2433
2434
2435 public boolean waitForAssignment(HRegionInfo regionInfo)
2436 throws InterruptedException {
2437 while (!regionStates.isRegionOnline(regionInfo)) {
2438 if (regionStates.isRegionInState(regionInfo, State.FAILED_OPEN)
2439 || this.server.isStopped()) {
2440 return false;
2441 }
2442
2443
2444
2445
2446 regionStates.waitForUpdate(100);
2447 }
2448 return true;
2449 }
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461 public void assignMeta() throws KeeperException {
2462 MetaRegionTracker.deleteMetaLocation(this.watcher);
2463 assign(HRegionInfo.FIRST_META_REGIONINFO, true);
2464 }
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474 public void assign(Map<HRegionInfo, ServerName> regions)
2475 throws IOException, InterruptedException {
2476 if (regions == null || regions.isEmpty()) {
2477 return;
2478 }
2479 List<ServerName> servers = serverManager.createDestinationServersList();
2480 if (servers == null || servers.isEmpty()) {
2481 throw new IOException("Found no destination server to assign region(s)");
2482 }
2483
2484
2485 Map<ServerName, List<HRegionInfo>> bulkPlan =
2486 balancer.retainAssignment(regions, servers);
2487
2488 assign(regions.size(), servers.size(),
2489 "retainAssignment=true", bulkPlan);
2490 }
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500 public void assign(List<HRegionInfo> regions)
2501 throws IOException, InterruptedException {
2502 if (regions == null || regions.isEmpty()) {
2503 return;
2504 }
2505
2506 List<ServerName> servers = serverManager.createDestinationServersList();
2507 if (servers == null || servers.isEmpty()) {
2508 throw new IOException("Found no destination server to assign region(s)");
2509 }
2510
2511
2512 Map<ServerName, List<HRegionInfo>> bulkPlan
2513 = balancer.roundRobinAssignment(regions, servers);
2514 processFavoredNodes(regions);
2515
2516 assign(regions.size(), servers.size(),
2517 "round-robin=true", bulkPlan);
2518 }
2519
2520 private void assign(int regions, int totalServers,
2521 String message, Map<ServerName, List<HRegionInfo>> bulkPlan)
2522 throws InterruptedException, IOException {
2523
2524 int servers = bulkPlan.size();
2525 if (servers == 1 || (regions < bulkAssignThresholdRegions
2526 && servers < bulkAssignThresholdServers)) {
2527
2528
2529
2530 if (LOG.isTraceEnabled()) {
2531 LOG.trace("Not using bulk assignment since we are assigning only " + regions +
2532 " region(s) to " + servers + " server(s)");
2533 }
2534 for (Map.Entry<ServerName, List<HRegionInfo>> plan: bulkPlan.entrySet()) {
2535 if (!assign(plan.getKey(), plan.getValue())) {
2536 for (HRegionInfo region: plan.getValue()) {
2537 if (!regionStates.isRegionOnline(region)) {
2538 invokeAssign(region);
2539 }
2540 }
2541 }
2542 }
2543 } else {
2544 LOG.info("Bulk assigning " + regions + " region(s) across "
2545 + totalServers + " server(s), " + message);
2546
2547
2548 BulkAssigner ba = new GeneralBulkAssigner(
2549 this.server, bulkPlan, this, bulkAssignWaitTillAllAssigned);
2550 ba.bulkAssign();
2551 LOG.info("Bulk assigning done");
2552 }
2553 }
2554
2555
2556
2557
2558
2559
2560
2561
2562
2563
2564
2565 private void assignAllUserRegions()
2566 throws IOException, InterruptedException, KeeperException {
2567
2568 ZKAssign.deleteAllNodes(watcher);
2569 ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
2570 this.watcher.assignmentZNode);
2571 failoverCleanupDone();
2572
2573
2574
2575
2576 Set<TableName> disabledOrDisablingOrEnabling = ZKTable.getDisabledOrDisablingTables(watcher);
2577 disabledOrDisablingOrEnabling.addAll(ZKTable.getEnablingTables(watcher));
2578
2579 Map<HRegionInfo, ServerName> allRegions;
2580 SnapshotOfRegionAssignmentFromMeta snapshotOfRegionAssignment =
2581 new SnapshotOfRegionAssignmentFromMeta(catalogTracker, disabledOrDisablingOrEnabling, true);
2582 snapshotOfRegionAssignment.initialize();
2583 allRegions = snapshotOfRegionAssignment.getRegionToRegionServerMap();
2584 if (allRegions == null || allRegions.isEmpty()) return;
2585
2586
2587 boolean retainAssignment = server.getConfiguration().
2588 getBoolean("hbase.master.startup.retainassign", true);
2589
2590 if (retainAssignment) {
2591 assign(allRegions);
2592 } else {
2593 List<HRegionInfo> regions = new ArrayList<HRegionInfo>(allRegions.keySet());
2594 assign(regions);
2595 }
2596
2597 for (HRegionInfo hri : allRegions.keySet()) {
2598 TableName tableName = hri.getTable();
2599 if (!zkTable.isEnabledTable(tableName)) {
2600 setEnabledTable(tableName);
2601 }
2602 }
2603 }
2604
2605
2606
2607
2608
2609
2610
2611 boolean waitUntilNoRegionsInTransition(final long timeout)
2612 throws InterruptedException {
2613
2614
2615
2616
2617
2618
2619 final long endTime = System.currentTimeMillis() + timeout;
2620
2621 while (!this.server.isStopped() && regionStates.isRegionsInTransition()
2622 && endTime > System.currentTimeMillis()) {
2623 regionStates.waitForUpdate(100);
2624 }
2625
2626 return !regionStates.isRegionsInTransition();
2627 }
2628
2629
2630
2631
2632
2633
2634
2635
2636
2637
2638 Map<ServerName, List<HRegionInfo>> rebuildUserRegions() throws IOException, KeeperException {
2639 Set<TableName> enablingTables = ZKTable.getEnablingTables(watcher);
2640 Set<TableName> disabledOrEnablingTables = ZKTable.getDisabledTables(watcher);
2641 disabledOrEnablingTables.addAll(enablingTables);
2642 Set<TableName> disabledOrDisablingOrEnabling = ZKTable.getDisablingTables(watcher);
2643 disabledOrDisablingOrEnabling.addAll(disabledOrEnablingTables);
2644
2645
2646 List<Result> results = MetaReader.fullScan(this.catalogTracker);
2647
2648 Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
2649
2650 Map<ServerName, List<HRegionInfo>> offlineServers =
2651 new TreeMap<ServerName, List<HRegionInfo>>();
2652
2653 for (Result result : results) {
2654 Pair<HRegionInfo, ServerName> region = HRegionInfo.getHRegionInfoAndServerName(result);
2655 if (region == null) continue;
2656 HRegionInfo regionInfo = region.getFirst();
2657 ServerName regionLocation = region.getSecond();
2658 if (regionInfo == null) continue;
2659 regionStates.createRegionState(regionInfo);
2660 if (regionStates.isRegionInState(regionInfo, State.SPLIT)) {
2661
2662
2663 LOG.debug("Region " + regionInfo.getRegionNameAsString()
2664 + " split is completed. Hence need not add to regions list");
2665 continue;
2666 }
2667 TableName tableName = regionInfo.getTable();
2668 if (regionLocation == null) {
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679 if (!enablingTables.contains(tableName)) {
2680 LOG.warn("Region " + regionInfo.getEncodedName() +
2681 " has null regionLocation." + " But its table " + tableName +
2682 " isn't in ENABLING state.");
2683 }
2684 } else if (!onlineServers.contains(regionLocation)) {
2685
2686 List<HRegionInfo> offlineRegions = offlineServers.get(regionLocation);
2687 if (offlineRegions == null) {
2688 offlineRegions = new ArrayList<HRegionInfo>(1);
2689 offlineServers.put(regionLocation, offlineRegions);
2690 }
2691 offlineRegions.add(regionInfo);
2692
2693
2694 if (!disabledOrDisablingOrEnabling.contains(tableName)
2695 && !getZKTable().isEnabledTable(tableName)) {
2696 setEnabledTable(tableName);
2697 }
2698 } else {
2699
2700
2701 if (!disabledOrEnablingTables.contains(tableName)) {
2702 regionStates.updateRegionState(regionInfo, State.OPEN, regionLocation);
2703 regionStates.regionOnline(regionInfo, regionLocation);
2704 balancer.regionOnline(regionInfo, regionLocation);
2705 }
2706
2707
2708 if (!disabledOrDisablingOrEnabling.contains(tableName)
2709 && !getZKTable().isEnabledTable(tableName)) {
2710 setEnabledTable(tableName);
2711 }
2712 }
2713 }
2714 return offlineServers;
2715 }
2716
2717
2718
2719
2720
2721
2722
2723
2724
2725 private void recoverTableInDisablingState()
2726 throws KeeperException, TableNotFoundException, IOException {
2727 Set<TableName> disablingTables = ZKTable.getDisablingTables(watcher);
2728 if (disablingTables.size() != 0) {
2729 for (TableName tableName : disablingTables) {
2730
2731 LOG.info("The table " + tableName
2732 + " is in DISABLING state. Hence recovering by moving the table"
2733 + " to DISABLED state.");
2734 new DisableTableHandler(this.server, tableName, catalogTracker,
2735 this, tableLockManager, true).prepare().process();
2736 }
2737 }
2738 }
2739
2740
2741
2742
2743
2744
2745
2746
2747
2748 private void recoverTableInEnablingState()
2749 throws KeeperException, TableNotFoundException, IOException {
2750 Set<TableName> enablingTables = ZKTable.getEnablingTables(watcher);
2751 if (enablingTables.size() != 0) {
2752 for (TableName tableName : enablingTables) {
2753
2754 LOG.info("The table " + tableName
2755 + " is in ENABLING state. Hence recovering by moving the table"
2756 + " to ENABLED state.");
2757
2758
2759 EnableTableHandler eth = new EnableTableHandler(this.server, tableName,
2760 catalogTracker, this, tableLockManager, true);
2761 try {
2762 eth.prepare();
2763 } catch (TableNotFoundException e) {
2764 LOG.warn("Table " + tableName + " not found in hbase:meta to recover.");
2765 continue;
2766 }
2767 eth.process();
2768 }
2769 }
2770 }
2771
2772
2773
2774
2775
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785
2786
2787 private void processDeadServersAndRecoverLostRegions(
2788 Map<ServerName, List<HRegionInfo>> deadServers)
2789 throws IOException, KeeperException {
2790 if (deadServers != null) {
2791 for (Map.Entry<ServerName, List<HRegionInfo>> server: deadServers.entrySet()) {
2792 ServerName serverName = server.getKey();
2793
2794 regionStates.setLastRegionServerOfRegions(serverName, server.getValue());
2795 if (!serverManager.isServerDead(serverName)) {
2796 serverManager.expireServer(serverName);
2797 }
2798 }
2799 }
2800 List<String> nodes = ZKUtil.listChildrenAndWatchForNewChildren(
2801 this.watcher, this.watcher.assignmentZNode);
2802 if (!nodes.isEmpty()) {
2803 for (String encodedRegionName : nodes) {
2804 processRegionInTransition(encodedRegionName, null);
2805 }
2806 }
2807
2808
2809
2810
2811
2812 failoverCleanupDone();
2813 }
2814
2815
2816
2817
2818
2819
2820
2821
2822 public void updateRegionsInTransitionMetrics() {
2823 long currentTime = System.currentTimeMillis();
2824 int totalRITs = 0;
2825 int totalRITsOverThreshold = 0;
2826 long oldestRITTime = 0;
2827 int ritThreshold = this.server.getConfiguration().
2828 getInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 60000);
2829 for (RegionState state: regionStates.getRegionsInTransition().values()) {
2830 totalRITs++;
2831 long ritTime = currentTime - state.getStamp();
2832 if (ritTime > ritThreshold) {
2833 totalRITsOverThreshold++;
2834 }
2835 if (oldestRITTime < ritTime) {
2836 oldestRITTime = ritTime;
2837 }
2838 }
2839 if (this.metricsAssignmentManager != null) {
2840 this.metricsAssignmentManager.updateRITOldestAge(oldestRITTime);
2841 this.metricsAssignmentManager.updateRITCount(totalRITs);
2842 this.metricsAssignmentManager.updateRITCountOverThreshold(totalRITsOverThreshold);
2843 }
2844 }
2845
2846
2847
2848
2849 void clearRegionPlan(final HRegionInfo region) {
2850 synchronized (this.regionPlans) {
2851 this.regionPlans.remove(region.getEncodedName());
2852 }
2853 }
2854
2855
2856
2857
2858
2859
2860 public void waitOnRegionToClearRegionsInTransition(final HRegionInfo hri)
2861 throws IOException, InterruptedException {
2862 waitOnRegionToClearRegionsInTransition(hri, -1L);
2863 }
2864
2865
2866
2867
2868
2869
2870
2871
2872 public boolean waitOnRegionToClearRegionsInTransition(final HRegionInfo hri, long timeOut)
2873 throws InterruptedException {
2874 if (!regionStates.isRegionInTransition(hri)) return true;
2875 long end = (timeOut <= 0) ? Long.MAX_VALUE : EnvironmentEdgeManager.currentTimeMillis()
2876 + timeOut;
2877
2878
2879 LOG.info("Waiting for " + hri.getEncodedName() +
2880 " to leave regions-in-transition, timeOut=" + timeOut + " ms.");
2881 while (!this.server.isStopped() && regionStates.isRegionInTransition(hri)) {
2882 regionStates.waitForUpdate(100);
2883 if (EnvironmentEdgeManager.currentTimeMillis() > end) {
2884 LOG.info("Timed out on waiting for " + hri.getEncodedName() + " to be assigned.");
2885 return false;
2886 }
2887 }
2888 if (this.server.isStopped()) {
2889 LOG.info("Giving up wait on regions in transition because stoppable.isStopped is set");
2890 return false;
2891 }
2892 return true;
2893 }
2894
2895
2896
2897
2898
2899 public class TimerUpdater extends Chore {
2900
2901 public TimerUpdater(final int period, final Stoppable stopper) {
2902 super("AssignmentTimerUpdater", period, stopper);
2903 }
2904
2905 @Override
2906 protected void chore() {
2907 Preconditions.checkState(tomActivated);
2908 ServerName serverToUpdateTimer = null;
2909 while (!serversInUpdatingTimer.isEmpty() && !stopper.isStopped()) {
2910 if (serverToUpdateTimer == null) {
2911 serverToUpdateTimer = serversInUpdatingTimer.first();
2912 } else {
2913 serverToUpdateTimer = serversInUpdatingTimer
2914 .higher(serverToUpdateTimer);
2915 }
2916 if (serverToUpdateTimer == null) {
2917 break;
2918 }
2919 updateTimers(serverToUpdateTimer);
2920 serversInUpdatingTimer.remove(serverToUpdateTimer);
2921 }
2922 }
2923 }
2924
2925
2926
2927
2928 public class TimeoutMonitor extends Chore {
2929 private boolean allRegionServersOffline = false;
2930 private ServerManager serverManager;
2931 private final int timeout;
2932
2933
2934
2935
2936
2937
2938
2939
2940
2941
2942 public TimeoutMonitor(final int period, final Stoppable stopper,
2943 ServerManager serverManager,
2944 final int timeout) {
2945 super("AssignmentTimeoutMonitor", period, stopper);
2946 this.timeout = timeout;
2947 this.serverManager = serverManager;
2948 }
2949
2950 private synchronized void setAllRegionServersOffline(
2951 boolean allRegionServersOffline) {
2952 this.allRegionServersOffline = allRegionServersOffline;
2953 }
2954
2955 @Override
2956 protected void chore() {
2957 Preconditions.checkState(tomActivated);
2958 boolean noRSAvailable = this.serverManager.createDestinationServersList().isEmpty();
2959
2960
2961 long now = System.currentTimeMillis();
2962
2963
2964 for (String regionName : regionStates.getRegionsInTransition().keySet()) {
2965 RegionState regionState = regionStates.getRegionTransitionState(regionName);
2966 if (regionState == null) continue;
2967
2968 if (regionState.getStamp() + timeout <= now) {
2969
2970 actOnTimeOut(regionState);
2971 } else if (this.allRegionServersOffline && !noRSAvailable) {
2972 RegionPlan existingPlan = regionPlans.get(regionName);
2973 if (existingPlan == null
2974 || !this.serverManager.isServerOnline(existingPlan
2975 .getDestination())) {
2976
2977
2978 actOnTimeOut(regionState);
2979 }
2980 }
2981 }
2982 setAllRegionServersOffline(noRSAvailable);
2983 }
2984
2985 private void actOnTimeOut(RegionState regionState) {
2986 HRegionInfo regionInfo = regionState.getRegion();
2987 LOG.info("Regions in transition timed out: " + regionState);
2988
2989 switch (regionState.getState()) {
2990 case CLOSED:
2991 LOG.info("Region " + regionInfo.getEncodedName()
2992 + " has been CLOSED for too long, waiting on queued "
2993 + "ClosedRegionHandler to run or server shutdown");
2994
2995 regionState.updateTimestampToNow();
2996 break;
2997 case OFFLINE:
2998 LOG.info("Region has been OFFLINE for too long, " + "reassigning "
2999 + regionInfo.getRegionNameAsString() + " to a random server");
3000 invokeAssign(regionInfo);
3001 break;
3002 case PENDING_OPEN:
3003 LOG.info("Region has been PENDING_OPEN for too "
3004 + "long, reassigning region=" + regionInfo.getRegionNameAsString());
3005 invokeAssign(regionInfo);
3006 break;
3007 case OPENING:
3008 processOpeningState(regionInfo);
3009 break;
3010 case OPEN:
3011 LOG.error("Region has been OPEN for too long, " +
3012 "we don't know where region was opened so can't do anything");
3013 regionState.updateTimestampToNow();
3014 break;
3015
3016 case PENDING_CLOSE:
3017 LOG.info("Region has been PENDING_CLOSE for too "
3018 + "long, running forced unassign again on region="
3019 + regionInfo.getRegionNameAsString());
3020 invokeUnassign(regionInfo);
3021 break;
3022 case CLOSING:
3023 LOG.info("Region has been CLOSING for too " +
3024 "long, this should eventually complete or the server will " +
3025 "expire, send RPC again");
3026 invokeUnassign(regionInfo);
3027 break;
3028
3029 case SPLIT:
3030 case SPLITTING:
3031 case FAILED_OPEN:
3032 case FAILED_CLOSE:
3033 case MERGING:
3034 break;
3035
3036 default:
3037 throw new IllegalStateException("Received event is not valid.");
3038 }
3039 }
3040 }
3041
3042 private void processOpeningState(HRegionInfo regionInfo) {
3043 LOG.info("Region has been OPENING for too long, reassigning region="
3044 + regionInfo.getRegionNameAsString());
3045
3046 try {
3047 String node = ZKAssign.getNodeName(watcher, regionInfo.getEncodedName());
3048 Stat stat = new Stat();
3049 byte [] data = ZKAssign.getDataNoWatch(watcher, node, stat);
3050 if (data == null) {
3051 LOG.warn("Data is null, node " + node + " no longer exists");
3052 return;
3053 }
3054 RegionTransition rt = RegionTransition.parseFrom(data);
3055 EventType et = rt.getEventType();
3056 if (et == EventType.RS_ZK_REGION_OPENED) {
3057 LOG.debug("Region has transitioned to OPENED, allowing "
3058 + "watched event handlers to process");
3059 return;
3060 } else if (et != EventType.RS_ZK_REGION_OPENING && et != EventType.RS_ZK_REGION_FAILED_OPEN ) {
3061 LOG.warn("While timing out a region, found ZK node in unexpected state: " + et);
3062 return;
3063 }
3064 invokeAssign(regionInfo);
3065 } catch (KeeperException ke) {
3066 LOG.error("Unexpected ZK exception timing out CLOSING region", ke);
3067 } catch (DeserializationException e) {
3068 LOG.error("Unexpected exception parsing CLOSING region", e);
3069 }
3070 }
3071
3072 void invokeAssign(HRegionInfo regionInfo) {
3073 threadPoolExecutorService.submit(new AssignCallable(this, regionInfo));
3074 }
3075
3076 private void invokeUnassign(HRegionInfo regionInfo) {
3077 threadPoolExecutorService.submit(new UnAssignCallable(this, regionInfo));
3078 }
3079
3080 public boolean isCarryingMeta(ServerName serverName) {
3081 return isCarryingRegion(serverName, HRegionInfo.FIRST_META_REGIONINFO);
3082 }
3083
3084
3085
3086
3087
3088
3089
3090
3091
3092
3093
3094 private boolean isCarryingRegion(ServerName serverName, HRegionInfo hri) {
3095 RegionTransition rt = null;
3096 try {
3097 byte [] data = ZKAssign.getData(watcher, hri.getEncodedName());
3098
3099 rt = data == null? null: RegionTransition.parseFrom(data);
3100 } catch (KeeperException e) {
3101 server.abort("Exception reading unassigned node for region=" + hri.getEncodedName(), e);
3102 } catch (DeserializationException e) {
3103 server.abort("Exception parsing unassigned node for region=" + hri.getEncodedName(), e);
3104 }
3105
3106 ServerName addressFromZK = rt != null? rt.getServerName(): null;
3107 if (addressFromZK != null) {
3108
3109 boolean matchZK = addressFromZK.equals(serverName);
3110 LOG.debug("Checking region=" + hri.getRegionNameAsString() + ", zk server=" + addressFromZK +
3111 " current=" + serverName + ", matches=" + matchZK);
3112 return matchZK;
3113 }
3114
3115 ServerName addressFromAM = regionStates.getRegionServerOfRegion(hri);
3116 boolean matchAM = (addressFromAM != null &&
3117 addressFromAM.equals(serverName));
3118 LOG.debug("based on AM, current region=" + hri.getRegionNameAsString() +
3119 " is on server=" + (addressFromAM != null ? addressFromAM : "null") +
3120 " server being checked: " + serverName);
3121
3122 return matchAM;
3123 }
3124
3125
3126
3127
3128
3129
3130 public List<HRegionInfo> processServerShutdown(final ServerName sn) {
3131
3132 synchronized (this.regionPlans) {
3133 for (Iterator <Map.Entry<String, RegionPlan>> i =
3134 this.regionPlans.entrySet().iterator(); i.hasNext();) {
3135 Map.Entry<String, RegionPlan> e = i.next();
3136 ServerName otherSn = e.getValue().getDestination();
3137
3138 if (otherSn != null && otherSn.equals(sn)) {
3139
3140 i.remove();
3141 }
3142 }
3143 }
3144 List<HRegionInfo> regions = regionStates.serverOffline(watcher, sn);
3145 for (Iterator<HRegionInfo> it = regions.iterator(); it.hasNext(); ) {
3146 HRegionInfo hri = it.next();
3147 String encodedName = hri.getEncodedName();
3148
3149
3150 Lock lock = locker.acquireLock(encodedName);
3151 try {
3152 RegionState regionState =
3153 regionStates.getRegionTransitionState(encodedName);
3154 if (regionState == null
3155 || (regionState.getServerName() != null && !regionState.isOnServer(sn))
3156 || !(regionState.isFailedClose() || regionState.isOffline()
3157 || regionState.isPendingOpenOrOpening())) {
3158 LOG.info("Skip " + regionState + " since it is not opening/failed_close"
3159 + " on the dead server any more: " + sn);
3160 it.remove();
3161 } else {
3162 try {
3163
3164 ZKAssign.deleteNodeFailSilent(watcher, hri);
3165 } catch (KeeperException ke) {
3166 server.abort("Unexpected ZK exception deleting node " + hri, ke);
3167 }
3168 if (zkTable.isDisablingOrDisabledTable(hri.getTable())) {
3169 regionStates.regionOffline(hri);
3170 it.remove();
3171 continue;
3172 }
3173
3174 regionStates.updateRegionState(hri, State.OFFLINE);
3175 }
3176 } finally {
3177 lock.unlock();
3178 }
3179 }
3180 return regions;
3181 }
3182
3183
3184
3185
3186 public void balance(final RegionPlan plan) {
3187 HRegionInfo hri = plan.getRegionInfo();
3188 TableName tableName = hri.getTable();
3189 if (zkTable.isDisablingOrDisabledTable(tableName)) {
3190 LOG.info("Ignored moving region of disabling/disabled table "
3191 + tableName);
3192 return;
3193 }
3194
3195
3196 String encodedName = hri.getEncodedName();
3197 ReentrantLock lock = locker.acquireLock(encodedName);
3198 try {
3199 if (!regionStates.isRegionOnline(hri)) {
3200 RegionState state = regionStates.getRegionState(encodedName);
3201 LOG.info("Ignored moving region not assigned: " + hri + ", "
3202 + (state == null ? "not in region states" : state));
3203 return;
3204 }
3205 synchronized (this.regionPlans) {
3206 this.regionPlans.put(plan.getRegionName(), plan);
3207 }
3208 unassign(hri, false, plan.getDestination());
3209 } finally {
3210 lock.unlock();
3211 }
3212 }
3213
3214 public void stop() {
3215 shutdown();
3216 if (tomActivated){
3217 this.timeoutMonitor.interrupt();
3218 this.timerUpdater.interrupt();
3219 }
3220 }
3221
3222
3223
3224
3225 public void shutdown() {
3226
3227 synchronized (zkEventWorkerWaitingList){
3228 zkEventWorkerWaitingList.clear();
3229 }
3230 threadPoolExecutorService.shutdownNow();
3231 zkEventWorkers.shutdownNow();
3232 }
3233
3234 protected void setEnabledTable(TableName tableName) {
3235 try {
3236 this.zkTable.setEnabledTable(tableName);
3237 } catch (KeeperException e) {
3238
3239 String errorMsg = "Unable to ensure that the table " + tableName
3240 + " will be" + " enabled because of a ZooKeeper issue";
3241 LOG.error(errorMsg);
3242 this.server.abort(errorMsg, e);
3243 }
3244 }
3245
3246
3247
3248
3249
3250
3251
3252 private boolean asyncSetOfflineInZooKeeper(final RegionState state,
3253 final AsyncCallback.StringCallback cb, final ServerName destination) {
3254 if (!state.isClosed() && !state.isOffline()) {
3255 this.server.abort("Unexpected state trying to OFFLINE; " + state,
3256 new IllegalStateException());
3257 return false;
3258 }
3259 regionStates.updateRegionState(state.getRegion(), State.OFFLINE);
3260 try {
3261 ZKAssign.asyncCreateNodeOffline(watcher, state.getRegion(),
3262 destination, cb, state);
3263 } catch (KeeperException e) {
3264 if (e instanceof NodeExistsException) {
3265 LOG.warn("Node for " + state.getRegion() + " already exists");
3266 } else {
3267 server.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
3268 }
3269 return false;
3270 }
3271 return true;
3272 }
3273
3274 private boolean deleteNodeInStates(String encodedName,
3275 String desc, ServerName sn, EventType... types) {
3276 try {
3277 for (EventType et: types) {
3278 if (ZKAssign.deleteNode(watcher, encodedName, et, sn)) {
3279 return true;
3280 }
3281 }
3282 LOG.info("Failed to delete the " + desc + " node for "
3283 + encodedName + ". The node type may not match");
3284 } catch (NoNodeException e) {
3285 if (LOG.isDebugEnabled()) {
3286 LOG.debug("The " + desc + " node for " + encodedName + " already deleted");
3287 }
3288 } catch (KeeperException ke) {
3289 server.abort("Unexpected ZK exception deleting " + desc
3290 + " node for the region " + encodedName, ke);
3291 }
3292 return false;
3293 }
3294
3295 private void deleteMergingNode(String encodedName, ServerName sn) {
3296 deleteNodeInStates(encodedName, "merging", sn, EventType.RS_ZK_REGION_MERGING,
3297 EventType.RS_ZK_REQUEST_REGION_MERGE, EventType.RS_ZK_REGION_MERGED);
3298 }
3299
3300 private void deleteSplittingNode(String encodedName, ServerName sn) {
3301 deleteNodeInStates(encodedName, "splitting", sn, EventType.RS_ZK_REGION_SPLITTING,
3302 EventType.RS_ZK_REQUEST_REGION_SPLIT, EventType.RS_ZK_REGION_SPLIT);
3303 }
3304
3305
3306
3307
3308
3309 private boolean handleRegionMerging(final RegionTransition rt, final String encodedName,
3310 final String prettyPrintedRegionName, final ServerName sn) {
3311 if (!serverManager.isServerOnline(sn)) {
3312 LOG.warn("Dropped merging! ServerName=" + sn + " unknown.");
3313 return false;
3314 }
3315 byte [] payloadOfMerging = rt.getPayload();
3316 List<HRegionInfo> mergingRegions;
3317 try {
3318 mergingRegions = HRegionInfo.parseDelimitedFrom(
3319 payloadOfMerging, 0, payloadOfMerging.length);
3320 } catch (IOException e) {
3321 LOG.error("Dropped merging! Failed reading " + rt.getEventType()
3322 + " payload for " + prettyPrintedRegionName);
3323 return false;
3324 }
3325 assert mergingRegions.size() == 3;
3326 HRegionInfo p = mergingRegions.get(0);
3327 HRegionInfo hri_a = mergingRegions.get(1);
3328 HRegionInfo hri_b = mergingRegions.get(2);
3329
3330 RegionState rs_p = regionStates.getRegionState(p);
3331 RegionState rs_a = regionStates.getRegionState(hri_a);
3332 RegionState rs_b = regionStates.getRegionState(hri_b);
3333
3334 if (!((rs_a == null || rs_a.isOpenOrMergingOnServer(sn))
3335 && (rs_b == null || rs_b.isOpenOrMergingOnServer(sn))
3336 && (rs_p == null || rs_p.isOpenOrMergingNewOnServer(sn)))) {
3337 LOG.warn("Dropped merging! Not in state good for MERGING; rs_p="
3338 + rs_p + ", rs_a=" + rs_a + ", rs_b=" + rs_b);
3339 return false;
3340 }
3341
3342 EventType et = rt.getEventType();
3343 if (et == EventType.RS_ZK_REQUEST_REGION_MERGE) {
3344 try {
3345 if (RegionMergeTransaction.transitionMergingNode(watcher, p,
3346 hri_a, hri_b, sn, -1, EventType.RS_ZK_REQUEST_REGION_MERGE,
3347 EventType.RS_ZK_REGION_MERGING) == -1) {
3348 byte[] data = ZKAssign.getData(watcher, encodedName);
3349 EventType currentType = null;
3350 if (data != null) {
3351 RegionTransition newRt = RegionTransition.parseFrom(data);
3352 currentType = newRt.getEventType();
3353 }
3354 if (currentType == null || (currentType != EventType.RS_ZK_REGION_MERGED
3355 && currentType != EventType.RS_ZK_REGION_MERGING)) {
3356 LOG.warn("Failed to transition pending_merge node "
3357 + encodedName + " to merging, it's now " + currentType);
3358 return false;
3359 }
3360 }
3361 } catch (Exception e) {
3362 LOG.warn("Failed to transition pending_merge node "
3363 + encodedName + " to merging", e);
3364 return false;
3365 }
3366 }
3367
3368 synchronized (regionStates) {
3369 regionStates.updateRegionState(hri_a, State.MERGING);
3370 regionStates.updateRegionState(hri_b, State.MERGING);
3371 regionStates.updateRegionState(p, State.MERGING_NEW, sn);
3372
3373 if (et != EventType.RS_ZK_REGION_MERGED) {
3374 regionStates.regionOffline(p, State.MERGING_NEW);
3375 this.mergingRegions.put(encodedName,
3376 new PairOfSameType<HRegionInfo>(hri_a, hri_b));
3377 } else {
3378 this.mergingRegions.remove(encodedName);
3379 regionOffline(hri_a, State.MERGED);
3380 regionOffline(hri_b, State.MERGED);
3381 regionOnline(p, sn);
3382 }
3383 }
3384
3385 if (et == EventType.RS_ZK_REGION_MERGED) {
3386 LOG.debug("Handling MERGED event for " + encodedName + "; deleting node");
3387
3388 try {
3389 boolean successful = false;
3390 while (!successful) {
3391
3392
3393 successful = ZKAssign.deleteNode(watcher, encodedName,
3394 EventType.RS_ZK_REGION_MERGED, sn);
3395 }
3396 } catch (KeeperException e) {
3397 if (e instanceof NoNodeException) {
3398 String znodePath = ZKUtil.joinZNode(watcher.splitLogZNode, encodedName);
3399 LOG.debug("The znode " + znodePath + " does not exist. May be deleted already.");
3400 } else {
3401 server.abort("Error deleting MERGED node " + encodedName, e);
3402 }
3403 }
3404 LOG.info("Handled MERGED event; merged=" + p.getRegionNameAsString()
3405 + ", region_a=" + hri_a.getRegionNameAsString() + ", region_b="
3406 + hri_b.getRegionNameAsString() + ", on " + sn);
3407
3408
3409 if (zkTable.isDisablingOrDisabledTable(p.getTable())) {
3410 unassign(p);
3411 }
3412 }
3413 return true;
3414 }
3415
3416
3417
3418
3419 private boolean handleRegionSplitting(final RegionTransition rt, final String encodedName,
3420 final String prettyPrintedRegionName, final ServerName sn) {
3421 if (!serverManager.isServerOnline(sn)) {
3422 LOG.warn("Dropped splitting! ServerName=" + sn + " unknown.");
3423 return false;
3424 }
3425 byte [] payloadOfSplitting = rt.getPayload();
3426 List<HRegionInfo> splittingRegions;
3427 try {
3428 splittingRegions = HRegionInfo.parseDelimitedFrom(
3429 payloadOfSplitting, 0, payloadOfSplitting.length);
3430 } catch (IOException e) {
3431 LOG.error("Dropped splitting! Failed reading " + rt.getEventType()
3432 + " payload for " + prettyPrintedRegionName);
3433 return false;
3434 }
3435 assert splittingRegions.size() == 2;
3436 HRegionInfo hri_a = splittingRegions.get(0);
3437 HRegionInfo hri_b = splittingRegions.get(1);
3438
3439 RegionState rs_p = regionStates.getRegionState(encodedName);
3440 RegionState rs_a = regionStates.getRegionState(hri_a);
3441 RegionState rs_b = regionStates.getRegionState(hri_b);
3442
3443 if (!((rs_p == null || rs_p.isOpenOrSplittingOnServer(sn))
3444 && (rs_a == null || rs_a.isOpenOrSplittingNewOnServer(sn))
3445 && (rs_b == null || rs_b.isOpenOrSplittingNewOnServer(sn)))) {
3446 LOG.warn("Dropped splitting! Not in state good for SPLITTING; rs_p="
3447 + rs_p + ", rs_a=" + rs_a + ", rs_b=" + rs_b);
3448 return false;
3449 }
3450
3451 if (rs_p == null) {
3452
3453 rs_p = regionStates.updateRegionState(rt, State.OPEN);
3454 if (rs_p == null) {
3455 LOG.warn("Received splitting for region " + prettyPrintedRegionName
3456 + " from server " + sn + " but it doesn't exist anymore,"
3457 + " probably already processed its split");
3458 return false;
3459 }
3460 regionStates.regionOnline(rs_p.getRegion(), sn);
3461 }
3462
3463 HRegionInfo p = rs_p.getRegion();
3464 EventType et = rt.getEventType();
3465 if (et == EventType.RS_ZK_REQUEST_REGION_SPLIT) {
3466 try {
3467 if (SplitTransaction.transitionSplittingNode(watcher, p,
3468 hri_a, hri_b, sn, -1, EventType.RS_ZK_REQUEST_REGION_SPLIT,
3469 EventType.RS_ZK_REGION_SPLITTING) == -1) {
3470 byte[] data = ZKAssign.getData(watcher, encodedName);
3471 EventType currentType = null;
3472 if (data != null) {
3473 RegionTransition newRt = RegionTransition.parseFrom(data);
3474 currentType = newRt.getEventType();
3475 }
3476 if (currentType == null || (currentType != EventType.RS_ZK_REGION_SPLIT
3477 && currentType != EventType.RS_ZK_REGION_SPLITTING)) {
3478 LOG.warn("Failed to transition pending_split node "
3479 + encodedName + " to splitting, it's now " + currentType);
3480 return false;
3481 }
3482 }
3483 } catch (Exception e) {
3484 LOG.warn("Failed to transition pending_split node "
3485 + encodedName + " to splitting", e);
3486 return false;
3487 }
3488 }
3489
3490 synchronized (regionStates) {
3491 regionStates.updateRegionState(hri_a, State.SPLITTING_NEW, sn);
3492 regionStates.updateRegionState(hri_b, State.SPLITTING_NEW, sn);
3493 regionStates.regionOffline(hri_a, State.SPLITTING_NEW);
3494 regionStates.regionOffline(hri_b, State.SPLITTING_NEW);
3495 regionStates.updateRegionState(rt, State.SPLITTING);
3496
3497
3498
3499 if (TEST_SKIP_SPLIT_HANDLING) {
3500 LOG.warn("Skipping split message, TEST_SKIP_SPLIT_HANDLING is set");
3501 return true;
3502 }
3503
3504 if (et == EventType.RS_ZK_REGION_SPLIT) {
3505 regionOffline(p, State.SPLIT);
3506 regionOnline(hri_a, sn);
3507 regionOnline(hri_b, sn);
3508 }
3509 }
3510
3511 if (et == EventType.RS_ZK_REGION_SPLIT) {
3512 LOG.debug("Handling SPLIT event for " + encodedName + "; deleting node");
3513
3514 try {
3515 boolean successful = false;
3516 while (!successful) {
3517
3518
3519 successful = ZKAssign.deleteNode(watcher, encodedName,
3520 EventType.RS_ZK_REGION_SPLIT, sn);
3521 }
3522 } catch (KeeperException e) {
3523 if (e instanceof NoNodeException) {
3524 String znodePath = ZKUtil.joinZNode(watcher.splitLogZNode, encodedName);
3525 LOG.debug("The znode " + znodePath + " does not exist. May be deleted already.");
3526 } else {
3527 server.abort("Error deleting SPLIT node " + encodedName, e);
3528 }
3529 }
3530 LOG.info("Handled SPLIT event; parent=" + p.getRegionNameAsString()
3531 + ", daughter a=" + hri_a.getRegionNameAsString() + ", daughter b="
3532 + hri_b.getRegionNameAsString() + ", on " + sn);
3533
3534
3535 if (zkTable.isDisablingOrDisabledTable(p.getTable())) {
3536 unassign(hri_a);
3537 unassign(hri_b);
3538 }
3539 }
3540 return true;
3541 }
3542
3543
3544
3545
3546
3547
3548 private void regionOffline(final HRegionInfo regionInfo, final State state) {
3549 regionStates.regionOffline(regionInfo, state);
3550 removeClosedRegion(regionInfo);
3551
3552 clearRegionPlan(regionInfo);
3553 balancer.regionOffline(regionInfo);
3554 }
3555
3556
3557
3558
3559 public LoadBalancer getBalancer() {
3560 return this.balancer;
3561 }
3562 }