1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.master;
21
22 import java.io.DataInput;
23 import java.io.DataOutput;
24 import java.io.EOFException;
25 import java.io.IOException;
26 import java.net.ConnectException;
27 import java.util.ArrayList;
28 import java.util.HashMap;
29 import java.util.Iterator;
30 import java.util.List;
31 import java.util.Map;
32 import java.util.NavigableMap;
33 import java.util.Set;
34 import java.util.SortedMap;
35 import java.util.TreeMap;
36 import java.util.TreeSet;
37 import java.util.concurrent.ConcurrentSkipListMap;
38 import java.util.concurrent.atomic.AtomicInteger;
39
40 import org.apache.commons.logging.Log;
41 import org.apache.commons.logging.LogFactory;
42 import org.apache.hadoop.conf.Configuration;
43 import org.apache.hadoop.hbase.Chore;
44 import org.apache.hadoop.hbase.HConstants;
45 import org.apache.hadoop.hbase.HRegionInfo;
46 import org.apache.hadoop.hbase.HServerAddress;
47 import org.apache.hadoop.hbase.HServerInfo;
48 import org.apache.hadoop.hbase.HTableDescriptor;
49 import org.apache.hadoop.hbase.NotServingRegionException;
50 import org.apache.hadoop.hbase.Server;
51 import org.apache.hadoop.hbase.Stoppable;
52 import org.apache.hadoop.hbase.catalog.CatalogTracker;
53 import org.apache.hadoop.hbase.catalog.MetaReader;
54 import org.apache.hadoop.hbase.catalog.RootLocationEditor;
55 import org.apache.hadoop.hbase.client.Result;
56 import org.apache.hadoop.hbase.executor.ExecutorService;
57 import org.apache.hadoop.hbase.executor.RegionTransitionData;
58 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
59 import org.apache.hadoop.hbase.master.LoadBalancer.RegionPlan;
60 import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler;
61 import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
62 import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
63 import org.apache.hadoop.hbase.util.Bytes;
64 import org.apache.hadoop.hbase.util.Pair;
65 import org.apache.hadoop.hbase.util.Threads;
66 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
67 import org.apache.hadoop.hbase.zookeeper.ZKTable;
68 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
69 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
70 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
71 import org.apache.hadoop.hbase.zookeeper.ZKUtil.NodeAndData;
72 import org.apache.hadoop.io.Writable;
73 import org.apache.hadoop.ipc.RemoteException;
74 import org.apache.zookeeper.AsyncCallback;
75 import org.apache.zookeeper.KeeperException;
76 import org.apache.zookeeper.KeeperException.NoNodeException;
77 import org.apache.zookeeper.data.Stat;
78
79
80
81
82
83
84
85
86 public class AssignmentManager extends ZooKeeperListener {
87 private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
88
89 protected Server master;
90
91 private ServerManager serverManager;
92
93 private CatalogTracker catalogTracker;
94
95 private TimeoutMonitor timeoutMonitor;
96
97
98
99
100 private final int maximumAssignmentAttempts;
101
102
103
104
105
106 final ConcurrentSkipListMap<String, RegionState> regionsInTransition =
107 new ConcurrentSkipListMap<String, RegionState>();
108
109
110
111
112
113 final NavigableMap<String, RegionPlan> regionPlans =
114 new TreeMap<String, RegionPlan>();
115
116 private final ZKTable zkTable;
117
118
119
120
121
122
123
124
125 private final NavigableMap<HServerInfo, List<HRegionInfo>> servers =
126 new TreeMap<HServerInfo, List<HRegionInfo>>();
127
128
129
130
131
132
133
134
135 private final SortedMap<HRegionInfo,HServerInfo> regions =
136 new TreeMap<HRegionInfo,HServerInfo>();
137
138 private final ExecutorService executorService;
139
140
141
142
143
144
145
146
147
148
149 public AssignmentManager(Server master, ServerManager serverManager,
150 CatalogTracker catalogTracker, final ExecutorService service)
151 throws KeeperException {
152 super(master.getZooKeeper());
153 this.master = master;
154 this.serverManager = serverManager;
155 this.catalogTracker = catalogTracker;
156 this.executorService = service;
157 Configuration conf = master.getConfiguration();
158 this.timeoutMonitor = new TimeoutMonitor(
159 conf.getInt("hbase.master.assignment.timeoutmonitor.period", 10000),
160 master,
161 conf.getInt("hbase.master.assignment.timeoutmonitor.timeout", 30000));
162 Threads.setDaemonThreadRunning(timeoutMonitor,
163 master.getServerName() + ".timeoutMonitor");
164 this.zkTable = new ZKTable(this.master.getZooKeeper());
165 this.maximumAssignmentAttempts =
166 this.master.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10);
167 }
168
169
170
171
172 public ZKTable getZKTable() {
173
174
175 return this.zkTable;
176 }
177
178
179
180
181
182
183
184 void cleanoutUnassigned() throws IOException, KeeperException {
185
186 ZKAssign.deleteAllNodes(watcher);
187 ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
188 this.watcher.assignmentZNode);
189 }
190
191
192
193
194
195
196
197 void processFailover() throws KeeperException, IOException {
198
199
200
201
202
203
204
205
206
207
208
209 Map<HServerInfo,List<Pair<HRegionInfo,Result>>> deadServers =
210 rebuildUserRegions();
211
212 processDeadServers(deadServers);
213
214 List<String> nodes = ZKUtil.listChildrenAndWatchForNewChildren(watcher,
215 watcher.assignmentZNode);
216 if (nodes.isEmpty()) {
217 LOG.info("No regions in transition in ZK to process on failover");
218 return;
219 }
220 LOG.info("Failed-over master needs to process " + nodes.size() +
221 " regions in transition");
222 for (String encodedRegionName: nodes) {
223 processRegionInTransition(encodedRegionName, null);
224 }
225 }
226
227
228
229
230
231
232
233
234
235
236
237
238 boolean processRegionInTransitionAndBlockUntilAssigned(final HRegionInfo hri)
239 throws InterruptedException, KeeperException, IOException {
240 boolean intransistion = processRegionInTransition(hri.getEncodedName(), hri);
241 if (!intransistion) return intransistion;
242 synchronized(this.regionsInTransition) {
243 while (!this.master.isStopped() &&
244 this.regionsInTransition.containsKey(hri.getEncodedName())) {
245 this.regionsInTransition.wait();
246 }
247 }
248 return intransistion;
249 }
250
251
252
253
254
255
256
257
258
259 boolean processRegionInTransition(final String encodedRegionName,
260 final HRegionInfo regionInfo)
261 throws KeeperException, IOException {
262 RegionTransitionData data = ZKAssign.getData(watcher, encodedRegionName);
263 if (data == null) return false;
264 HRegionInfo hri = regionInfo;
265 if (hri == null) {
266 Pair<HRegionInfo, HServerAddress> p =
267 MetaReader.getRegion(catalogTracker, data.getRegionName());
268 if (p == null) return false;
269 hri = p.getFirst();
270 }
271 processRegionsInTransition(data, hri);
272 return true;
273 }
274
275 void processRegionsInTransition(final RegionTransitionData data,
276 final HRegionInfo regionInfo)
277 throws KeeperException {
278 String encodedRegionName = regionInfo.getEncodedName();
279 LOG.info("Processing region " + regionInfo.getRegionNameAsString() +
280 " in state " + data.getEventType());
281 synchronized (regionsInTransition) {
282 switch (data.getEventType()) {
283 case RS_ZK_REGION_CLOSING:
284
285
286 regionsInTransition.put(encodedRegionName, new RegionState(
287 regionInfo, RegionState.State.CLOSING, data.getStamp()));
288 break;
289
290 case RS_ZK_REGION_CLOSED:
291
292 regionsInTransition.put(encodedRegionName, new RegionState(
293 regionInfo, RegionState.State.CLOSED, data.getStamp()));
294 new ClosedRegionHandler(master, this, regionInfo).process();
295 break;
296
297 case M_ZK_REGION_OFFLINE:
298
299 regionsInTransition.put(encodedRegionName, new RegionState(
300 regionInfo, RegionState.State.OFFLINE, data.getStamp()));
301 new ClosedRegionHandler(master, this, regionInfo).process();
302 break;
303
304 case RS_ZK_REGION_OPENING:
305
306
307 regionsInTransition.put(encodedRegionName, new RegionState(
308 regionInfo, RegionState.State.OPENING, data.getStamp()));
309 break;
310
311 case RS_ZK_REGION_OPENED:
312
313 regionsInTransition.put(encodedRegionName, new RegionState(
314 regionInfo, RegionState.State.OPENING, data.getStamp()));
315 HServerInfo hsi = serverManager.getServerInfo(data.getServerName());
316
317
318
319 if (hsi == null) {
320 LOG.warn("Region in transition " + regionInfo.getEncodedName() +
321 " references a server no longer up " + data.getServerName() +
322 "; letting RIT timeout so will be assigned elsewhere");
323 break;
324 }
325 new OpenedRegionHandler(master, this, regionInfo, hsi).process();
326 break;
327 }
328 }
329 }
330
331
332
333
334
335
336
337
338
339
340 private void handleRegion(final RegionTransitionData data) {
341 synchronized(regionsInTransition) {
342 if (data == null || data.getServerName() == null) {
343 LOG.warn("Unexpected NULL input " + data);
344 return;
345 }
346
347 if (data.getServerName().equals(HConstants.HBCK_CODE_NAME)) {
348 handleHBCK(data);
349 return;
350 }
351
352 if (!serverManager.isServerOnline(data.getServerName()) &&
353 !this.master.getServerName().equals(data.getServerName())) {
354 LOG.warn("Attempted to handle region transition for server but " +
355 "server is not online: " + data.getRegionName());
356 return;
357 }
358 String encodedName = HRegionInfo.encodeRegionName(data.getRegionName());
359 String prettyPrintedRegionName = HRegionInfo.prettyPrint(encodedName);
360 LOG.debug("Handling transition=" + data.getEventType() +
361 ", server=" + data.getServerName() + ", region=" + prettyPrintedRegionName);
362 RegionState regionState = regionsInTransition.get(encodedName);
363 switch (data.getEventType()) {
364 case M_ZK_REGION_OFFLINE:
365
366 break;
367
368 case RS_ZK_REGION_CLOSING:
369
370
371 if (regionState == null ||
372 (!regionState.isPendingClose() && !regionState.isClosing())) {
373 LOG.warn("Received CLOSING for region " + prettyPrintedRegionName +
374 " from server " + data.getServerName() + " but region was in " +
375 " the state " + regionState + " and not " +
376 "in expected PENDING_CLOSE or CLOSING states");
377 return;
378 }
379
380 regionState.update(RegionState.State.CLOSING, data.getStamp());
381 break;
382
383 case RS_ZK_REGION_CLOSED:
384
385 if (regionState == null ||
386 (!regionState.isPendingClose() && !regionState.isClosing())) {
387 LOG.warn("Received CLOSED for region " + prettyPrintedRegionName +
388 " from server " + data.getServerName() + " but region was in " +
389 " the state " + regionState + " and not " +
390 "in expected PENDING_CLOSE or CLOSING states");
391 return;
392 }
393
394
395
396 regionState.update(RegionState.State.CLOSED, data.getStamp());
397 this.executorService.submit(new ClosedRegionHandler(master,
398 this, regionState.getRegion()));
399 break;
400
401 case RS_ZK_REGION_OPENING:
402
403
404 if(regionState == null ||
405 (!regionState.isPendingOpen() && !regionState.isOpening())) {
406 LOG.warn("Received OPENING for region " +
407 prettyPrintedRegionName +
408 " from server " + data.getServerName() + " but region was in " +
409 " the state " + regionState + " and not " +
410 "in expected PENDING_OPEN or OPENING states");
411 return;
412 }
413
414 regionState.update(RegionState.State.OPENING, data.getStamp());
415 break;
416
417 case RS_ZK_REGION_OPENED:
418
419 if(regionState == null ||
420 (!regionState.isPendingOpen() && !regionState.isOpening())) {
421 LOG.warn("Received OPENED for region " +
422 prettyPrintedRegionName +
423 " from server " + data.getServerName() + " but region was in " +
424 " the state " + regionState + " and not " +
425 "in expected PENDING_OPEN or OPENING states");
426 return;
427 }
428
429 regionState.update(RegionState.State.OPEN, data.getStamp());
430 this.executorService.submit(
431 new OpenedRegionHandler(master, this, regionState.getRegion(),
432 this.serverManager.getServerInfo(data.getServerName())));
433 break;
434 }
435 }
436 }
437
438
439
440
441
442
443
444 private void handleHBCK(RegionTransitionData data) {
445 String encodedName = HRegionInfo.encodeRegionName(data.getRegionName());
446 LOG.info("Handling HBCK triggered transition=" + data.getEventType() +
447 ", server=" + data.getServerName() + ", region=" +
448 HRegionInfo.prettyPrint(encodedName));
449 RegionState regionState = regionsInTransition.get(encodedName);
450 switch (data.getEventType()) {
451 case M_ZK_REGION_OFFLINE:
452 HRegionInfo regionInfo = null;
453 if (regionState != null) {
454 regionInfo = regionState.getRegion();
455 } else {
456 try {
457 regionInfo = MetaReader.getRegion(catalogTracker,
458 data.getRegionName()).getFirst();
459 } catch (IOException e) {
460 LOG.info("Exception reading META doing HBCK repair operation", e);
461 return;
462 }
463 }
464 LOG.info("HBCK repair is triggering assignment of region=" +
465 regionInfo.getRegionNameAsString());
466
467 assign(regionInfo, false);
468 break;
469
470 default:
471 LOG.warn("Received unexpected region state from HBCK (" +
472 data.getEventType() + ")");
473 break;
474 }
475 }
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491 @Override
492 public void nodeCreated(String path) {
493 if(path.startsWith(watcher.assignmentZNode)) {
494 synchronized(regionsInTransition) {
495 try {
496 RegionTransitionData data = ZKAssign.getData(watcher, path);
497 if(data == null) {
498 return;
499 }
500 handleRegion(data);
501 } catch (KeeperException e) {
502 master.abort("Unexpected ZK exception reading unassigned node data", e);
503 }
504 }
505 }
506 }
507
508
509
510
511
512
513
514
515
516
517
518
519
520 @Override
521 public void nodeDataChanged(String path) {
522 if(path.startsWith(watcher.assignmentZNode)) {
523 synchronized(regionsInTransition) {
524 try {
525 RegionTransitionData data = ZKAssign.getData(watcher, path);
526 if(data == null) {
527 return;
528 }
529 handleRegion(data);
530 } catch (KeeperException e) {
531 master.abort("Unexpected ZK exception reading unassigned node data", e);
532 }
533 }
534 }
535 }
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550 @Override
551 public void nodeChildrenChanged(String path) {
552 if(path.equals(watcher.assignmentZNode)) {
553 synchronized(regionsInTransition) {
554 try {
555 List<NodeAndData> newNodes = ZKUtil.watchAndGetNewChildren(watcher,
556 watcher.assignmentZNode);
557 for(NodeAndData newNode : newNodes) {
558 LOG.debug("Handling new unassigned node: " + newNode);
559 handleRegion(RegionTransitionData.fromBytes(newNode.getData()));
560 }
561 } catch(KeeperException e) {
562 master.abort("Unexpected ZK exception reading unassigned children", e);
563 }
564 }
565 }
566 }
567
568
569
570
571
572
573
574
575
576 public void regionOnline(HRegionInfo regionInfo, HServerInfo serverInfo) {
577 synchronized (this.regionsInTransition) {
578 RegionState rs =
579 this.regionsInTransition.remove(regionInfo.getEncodedName());
580 if (rs != null) {
581 this.regionsInTransition.notifyAll();
582 }
583 }
584 synchronized (this.regions) {
585
586 HServerInfo hsi = this.regions.get(regionInfo);
587 if (hsi != null) LOG.warn("Overwriting " + regionInfo.getEncodedName() +
588 " on " + hsi);
589 this.regions.put(regionInfo, serverInfo);
590 addToServers(serverInfo, regionInfo);
591 this.regions.notifyAll();
592 }
593
594 clearRegionPlan(regionInfo);
595
596 updateTimers(serverInfo);
597 }
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612 private void updateTimers(final HServerInfo hsi) {
613
614
615
616
617 Map<String, RegionPlan> copy = new HashMap<String, RegionPlan>();
618 synchronized(this.regionPlans) {
619 copy.putAll(this.regionPlans);
620 }
621 for (Map.Entry<String, RegionPlan> e: copy.entrySet()) {
622 if (!e.getValue().getDestination().equals(hsi)) continue;
623 RegionState rs = null;
624 synchronized (this.regionsInTransition) {
625 rs = this.regionsInTransition.get(e.getKey());
626 }
627 if (rs == null) continue;
628 synchronized (rs) {
629 rs.update(rs.getState());
630 }
631 }
632 }
633
634
635
636
637
638
639
640
641 public void regionOffline(final HRegionInfo regionInfo) {
642 synchronized(this.regionsInTransition) {
643 if (this.regionsInTransition.remove(regionInfo.getEncodedName()) != null) {
644 this.regionsInTransition.notifyAll();
645 }
646 }
647
648 clearRegionPlan(regionInfo);
649 setOffline(regionInfo);
650 }
651
652
653
654
655
656
657
658
659 public void setOffline(HRegionInfo regionInfo) {
660 synchronized (this.regions) {
661 HServerInfo serverInfo = this.regions.remove(regionInfo);
662 if (serverInfo == null) return;
663 List<HRegionInfo> serverRegions = this.servers.get(serverInfo);
664 if (!serverRegions.remove(regionInfo)) {
665 LOG.warn("No " + regionInfo + " on " + serverInfo);
666 }
667 }
668 }
669
670 public void offlineDisabledRegion(HRegionInfo regionInfo) {
671
672 LOG.debug("Table being disabled so deleting ZK node and removing from " +
673 "regions in transition, skipping assignment of region " +
674 regionInfo.getRegionNameAsString());
675 try {
676 if (!ZKAssign.deleteClosedNode(watcher, regionInfo.getEncodedName())) {
677
678 ZKAssign.deleteOfflineNode(watcher, regionInfo.getEncodedName());
679 }
680 } catch (KeeperException.NoNodeException nne) {
681 LOG.debug("Tried to delete closed node for " + regionInfo + " but it " +
682 "does not exist so just offlining");
683 } catch (KeeperException e) {
684 this.master.abort("Error deleting CLOSED node in ZK", e);
685 }
686 regionOffline(regionInfo);
687 }
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709 public void assign(HRegionInfo region, boolean setOfflineInZK) {
710 assign(region, setOfflineInZK, false);
711 }
712
713 public void assign(HRegionInfo region, boolean setOfflineInZK,
714 boolean forceNewPlan) {
715 String tableName = region.getTableDesc().getNameAsString();
716 boolean disabled = this.zkTable.isDisabledTable(tableName);
717 if (disabled || this.zkTable.isDisablingTable(tableName)) {
718 LOG.info("Table " + tableName + (disabled? " disabled;": " disabling;") +
719 " skipping assign of " + region.getRegionNameAsString());
720 offlineDisabledRegion(region);
721 return;
722 }
723 if (this.serverManager.isClusterShutdown()) {
724 LOG.info("Cluster shutdown is set; skipping assign of " +
725 region.getRegionNameAsString());
726 return;
727 }
728 RegionState state = addToRegionsInTransition(region);
729 synchronized (state) {
730 assign(state, setOfflineInZK, forceNewPlan);
731 }
732 }
733
734
735
736
737
738
739
740 void assign(final HServerInfo destination,
741 final List<HRegionInfo> regions) {
742 LOG.debug("Bulk assigning " + regions.size() + " region(s) to " +
743 destination.getServerName());
744
745 List<RegionState> states = new ArrayList<RegionState>(regions.size());
746 synchronized (this.regionsInTransition) {
747 for (HRegionInfo region: regions) {
748 states.add(forceRegionStateToOffline(region));
749 }
750 }
751
752
753 AtomicInteger counter = new AtomicInteger(0);
754 CreateUnassignedAsyncCallback cb =
755 new CreateUnassignedAsyncCallback(this.watcher, destination, counter);
756 for (RegionState state: states) {
757 if (!asyncSetOfflineInZooKeeper(state, cb, state)) {
758 return;
759 }
760 }
761
762 int total = regions.size();
763 for (int oldCounter = 0; true;) {
764 int count = counter.get();
765 if (oldCounter != count) {
766 LOG.info(destination.getServerName() + " unassigned znodes=" + count +
767 " of total=" + total);
768 oldCounter = count;
769 }
770 if (count == total) break;
771 Threads.sleep(1);
772 }
773
774 try {
775
776
777
778
779 long maxWaitTime = System.currentTimeMillis() +
780 this.master.getConfiguration().getLong("hbase.regionserver.rpc.startup.waittime", 60000);
781 while (!this.master.isStopped()) {
782 try {
783 this.serverManager.sendRegionOpen(destination, regions);
784 break;
785 } catch (org.apache.hadoop.hbase.ipc.ServerNotRunningException e) {
786
787
788 long now = System.currentTimeMillis();
789 if (now > maxWaitTime) throw e;
790 LOG.debug("Server is not yet up; waiting up to " +
791 (maxWaitTime - now) + "ms", e);
792 Thread.sleep(1000);
793 }
794 }
795 } catch (Throwable t) {
796 this.master.abort("Failed assignment of regions to " + destination +
797 "; bulk assign FAILED", t);
798 return;
799 }
800 LOG.debug("Bulk assigning done for " + destination.getServerName());
801 }
802
803
804
805
806 static class CreateUnassignedAsyncCallback implements AsyncCallback.StringCallback {
807 private final Log LOG = LogFactory.getLog(CreateUnassignedAsyncCallback.class);
808 private final ZooKeeperWatcher zkw;
809 private final HServerInfo destination;
810 private final AtomicInteger counter;
811
812 CreateUnassignedAsyncCallback(final ZooKeeperWatcher zkw,
813 final HServerInfo destination, final AtomicInteger counter) {
814 this.zkw = zkw;
815 this.destination = destination;
816 this.counter = counter;
817 }
818
819 @Override
820 public void processResult(int rc, String path, Object ctx, String name) {
821 if (rc != 0) {
822
823 LOG.warn("rc != 0 for " + path + " -- retryable connectionloss -- " +
824 "FIX see http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2");
825 this.zkw.abort("Connectionloss writing unassigned at " + path +
826 ", rc=" + rc, null);
827 return;
828 }
829 LOG.debug("rs=" + (RegionState)ctx + ", server=" + this.destination.getServerName());
830
831
832 this.zkw.getZooKeeper().exists(path, this.zkw,
833 new ExistsUnassignedAsyncCallback(this.counter), ctx);
834 }
835 }
836
837
838
839
840
841 static class ExistsUnassignedAsyncCallback implements AsyncCallback.StatCallback {
842 private final Log LOG = LogFactory.getLog(ExistsUnassignedAsyncCallback.class);
843 private final AtomicInteger counter;
844
845 ExistsUnassignedAsyncCallback(final AtomicInteger counter) {
846 this.counter = counter;
847 }
848
849 @Override
850 public void processResult(int rc, String path, Object ctx, Stat stat) {
851 if (rc != 0) {
852
853 LOG.warn("rc != 0 for " + path + " -- retryable connectionloss -- " +
854 "FIX see http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2");
855 return;
856 }
857 RegionState state = (RegionState)ctx;
858 LOG.debug("rs=" + state);
859
860
861
862
863
864 state.update(RegionState.State.PENDING_OPEN);
865 this.counter.addAndGet(1);
866 }
867 }
868
869
870
871
872
873 private RegionState addToRegionsInTransition(final HRegionInfo region) {
874 synchronized (regionsInTransition) {
875 return forceRegionStateToOffline(region);
876 }
877 }
878
879
880
881
882
883
884
885 private RegionState forceRegionStateToOffline(final HRegionInfo region) {
886 String encodedName = region.getEncodedName();
887 RegionState state = this.regionsInTransition.get(encodedName);
888 if (state == null) {
889 state = new RegionState(region, RegionState.State.OFFLINE);
890 this.regionsInTransition.put(encodedName, state);
891 } else {
892 LOG.debug("Forcing OFFLINE; was=" + state);
893 state.update(RegionState.State.OFFLINE);
894 }
895 return state;
896 }
897
898
899
900
901
902
903
904 private void assign(final RegionState state, final boolean setOfflineInZK,
905 final boolean forceNewPlan) {
906 for (int i = 0; i < this.maximumAssignmentAttempts; i++) {
907 if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
908 if (this.master.isStopped()) {
909 LOG.debug("Server stopped; skipping assign of " + state);
910 return;
911 }
912 RegionPlan plan = getRegionPlan(state, forceNewPlan);
913 if (plan == null) return;
914 try {
915 LOG.debug("Assigning region " + state.getRegion().getRegionNameAsString() +
916 " to " + plan.getDestination().getServerName());
917
918 state.update(RegionState.State.PENDING_OPEN);
919
920 serverManager.sendRegionOpen(plan.getDestination(), state.getRegion());
921 break;
922 } catch (Throwable t) {
923 LOG.warn("Failed assignment of " +
924 state.getRegion().getRegionNameAsString() + " to " +
925 plan.getDestination() + ", trying to assign elsewhere instead; " +
926 "retry=" + i, t);
927
928
929
930 state.update(RegionState.State.OFFLINE);
931
932 if (getRegionPlan(state, plan.getDestination(), true) == null) {
933 LOG.warn("Unable to find a viable location to assign region " +
934 state.getRegion().getRegionNameAsString());
935 return;
936 }
937 }
938 }
939 }
940
941
942
943
944
945
946
947 boolean setOfflineInZooKeeper(final RegionState state) {
948 if (!state.isClosed() && !state.isOffline()) {
949 new RuntimeException("Unexpected state trying to OFFLINE; " + state);
950 this.master.abort("Unexpected state trying to OFFLINE; " + state,
951 new IllegalStateException());
952 return false;
953 }
954 state.update(RegionState.State.OFFLINE);
955 try {
956 if(!ZKAssign.createOrForceNodeOffline(master.getZooKeeper(),
957 state.getRegion(), master.getServerName())) {
958 LOG.warn("Attempted to create/force node into OFFLINE state before " +
959 "completing assignment but failed to do so for " + state);
960 return false;
961 }
962 } catch (KeeperException e) {
963 master.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
964 return false;
965 }
966 return true;
967 }
968
969
970
971
972
973
974
975 boolean asyncSetOfflineInZooKeeper(final RegionState state,
976 final AsyncCallback.StringCallback cb, final Object ctx) {
977 if (!state.isClosed() && !state.isOffline()) {
978 new RuntimeException("Unexpected state trying to OFFLINE; " + state);
979 this.master.abort("Unexpected state trying to OFFLINE; " + state,
980 new IllegalStateException());
981 return false;
982 }
983 state.update(RegionState.State.OFFLINE);
984 try {
985 ZKAssign.asyncCreateNodeOffline(master.getZooKeeper(), state.getRegion(),
986 master.getServerName(), cb, ctx);
987 } catch (KeeperException e) {
988 master.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
989 return false;
990 }
991 return true;
992 }
993
994
995
996
997
998
999 RegionPlan getRegionPlan(final RegionState state,
1000 final boolean forceNewPlan) {
1001 return getRegionPlan(state, null, forceNewPlan);
1002 }
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013 RegionPlan getRegionPlan(final RegionState state,
1014 final HServerInfo serverToExclude, final boolean forceNewPlan) {
1015
1016 String encodedName = state.getRegion().getEncodedName();
1017 List<HServerInfo> servers = this.serverManager.getOnlineServersList();
1018
1019
1020 if (serverToExclude != null) servers.remove(serverToExclude);
1021 if (servers.isEmpty()) return null;
1022 RegionPlan randomPlan = new RegionPlan(state.getRegion(), null,
1023 LoadBalancer.randomAssignment(servers));
1024 boolean newPlan = false;
1025 RegionPlan existingPlan = null;
1026 synchronized (this.regionPlans) {
1027 existingPlan = this.regionPlans.get(encodedName);
1028 if (forceNewPlan || existingPlan == null
1029 || existingPlan.getDestination() == null
1030 || existingPlan.getDestination().equals(serverToExclude)) {
1031 newPlan = true;
1032 this.regionPlans.put(encodedName, randomPlan);
1033 }
1034 }
1035 if (newPlan) {
1036 LOG.debug("No previous transition plan was found (or we are ignoring " +
1037 "an existing plan) for " + state.getRegion().getRegionNameAsString() +
1038 " so generated a random one; " + randomPlan + "; " +
1039 serverManager.countOfRegionServers() +
1040 " (online=" + serverManager.getOnlineServers().size() +
1041 ", exclude=" + serverToExclude + ") available servers");
1042 return randomPlan;
1043 }
1044 LOG.debug("Using pre-existing plan for region " +
1045 state.getRegion().getRegionNameAsString() + "; plan=" + existingPlan);
1046 return existingPlan;
1047 }
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058 public void unassign(HRegionInfo region) {
1059 unassign(region, false);
1060 }
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072 public void unassign(HRegionInfo region, boolean force) {
1073 LOG.debug("Starting unassignment of region " +
1074 region.getRegionNameAsString() + " (offlining)");
1075 synchronized (this.regions) {
1076
1077 if (!regions.containsKey(region)) {
1078 LOG.debug("Attempted to unassign region " +
1079 region.getRegionNameAsString() + " but it is not " +
1080 "currently assigned anywhere");
1081 return;
1082 }
1083 }
1084 String encodedName = region.getEncodedName();
1085
1086 RegionState state;
1087 synchronized (regionsInTransition) {
1088 state = regionsInTransition.get(encodedName);
1089 if (state == null) {
1090 state = new RegionState(region, RegionState.State.PENDING_CLOSE);
1091 regionsInTransition.put(encodedName, state);
1092 } else if (force && state.isPendingClose()) {
1093 LOG.debug("Attempting to unassign region " +
1094 region.getRegionNameAsString() + " which is already pending close "
1095 + "but forcing an additional close");
1096 state.update(RegionState.State.PENDING_CLOSE);
1097 } else {
1098 LOG.debug("Attempting to unassign region " +
1099 region.getRegionNameAsString() + " but it is " +
1100 "already in transition (" + state.getState() + ")");
1101 return;
1102 }
1103 }
1104
1105 HServerInfo server = null;
1106 synchronized (this.regions) {
1107 server = regions.get(region);
1108 }
1109 try {
1110
1111
1112 if (serverManager.sendRegionClose(server, state.getRegion())) {
1113 LOG.debug("Sent CLOSE to " + server + " for region " +
1114 region.getRegionNameAsString());
1115 return;
1116 }
1117
1118 LOG.debug("Server " + server + " region CLOSE RPC returned false for " +
1119 region.getEncodedName());
1120 } catch (NotServingRegionException nsre) {
1121 LOG.info("Server " + server + " returned " + nsre + " for " +
1122 region.getEncodedName());
1123
1124
1125
1126 return;
1127 } catch (ConnectException e) {
1128 LOG.info("Failed connect to " + server + ", message=" + e.getMessage() +
1129 ", region=" + region.getEncodedName());
1130
1131
1132 } catch (java.net.SocketTimeoutException e) {
1133 LOG.info("Server " + server + " returned " + e.getMessage() + " for " +
1134 region.getEncodedName());
1135
1136 } catch (EOFException e) {
1137 LOG.info("Server " + server + " returned " + e.getMessage() + " for " +
1138 region.getEncodedName());
1139
1140 } catch (RemoteException re) {
1141 IOException ioe = re.unwrapRemoteException();
1142 if (ioe instanceof NotServingRegionException) {
1143
1144 LOG.debug("Server " + server + " returned " + ioe + " for " +
1145 region.getEncodedName());
1146 } else if (ioe instanceof EOFException) {
1147
1148 LOG.debug("Server " + server + " returned " + ioe + " for " +
1149 region.getEncodedName());
1150 } else {
1151 this.master.abort("Remote unexpected exception", ioe);
1152 }
1153 } catch (Throwable t) {
1154
1155
1156 this.master.abort("Remote unexpected exception", t);
1157 }
1158 }
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168 public void waitForAssignment(HRegionInfo regionInfo)
1169 throws InterruptedException {
1170 synchronized(regions) {
1171 while(!regions.containsKey(regionInfo)) {
1172 regions.wait();
1173 }
1174 }
1175 }
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187 public void assignRoot() throws KeeperException {
1188 RootLocationEditor.deleteRootLocation(this.master.getZooKeeper());
1189 assign(HRegionInfo.ROOT_REGIONINFO, true);
1190 }
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200 public void assignMeta() {
1201
1202 assign(HRegionInfo.FIRST_META_REGIONINFO, true);
1203 }
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213 public void assignUserRegions(List<HRegionInfo> regions, List<HServerInfo> servers) throws IOException, InterruptedException {
1214 if (regions == null)
1215 return;
1216 Map<HServerInfo, List<HRegionInfo>> bulkPlan = null;
1217
1218 bulkPlan = LoadBalancer.roundRobinAssignment(regions, servers);
1219 LOG.info("Bulk assigning " + regions.size() + " region(s) round-robin across " +
1220 servers.size() + " server(s)");
1221
1222 BulkAssigner ba = new BulkStartupAssigner(this.master, bulkPlan, this);
1223 ba.bulkAssign();
1224 LOG.info("Bulk assigning done");
1225 }
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236 public void assignAllUserRegions() throws IOException, InterruptedException {
1237
1238 List<HServerInfo> servers = serverManager.getOnlineServersList();
1239
1240
1241 Map<HRegionInfo,HServerAddress> allRegions =
1242 MetaReader.fullScan(catalogTracker, this.zkTable.getDisabledTables(), true);
1243 if (allRegions == null || allRegions.isEmpty()) return;
1244
1245
1246 boolean retainAssignment = master.getConfiguration().
1247 getBoolean("hbase.master.startup.retainassign", true);
1248
1249 Map<HServerInfo, List<HRegionInfo>> bulkPlan = null;
1250 if (retainAssignment) {
1251
1252 bulkPlan = LoadBalancer.retainAssignment(allRegions, servers);
1253 } else {
1254
1255 assignUserRegions(new ArrayList<HRegionInfo>(allRegions.keySet()), servers);
1256 return;
1257 }
1258 LOG.info("Bulk assigning " + allRegions.size() + " region(s) across " +
1259 servers.size() + " server(s), retainAssignment=" + retainAssignment);
1260
1261
1262 BulkAssigner ba = new BulkStartupAssigner(this.master, bulkPlan, this);
1263 ba.bulkAssign();
1264 LOG.info("Bulk assigning done");
1265 }
1266
1267
1268
1269
1270 static class BulkStartupAssigner extends BulkAssigner {
1271 private final Map<HServerInfo, List<HRegionInfo>> bulkPlan;
1272 private final AssignmentManager assignmentManager;
1273
1274 BulkStartupAssigner(final Server server,
1275 final Map<HServerInfo, List<HRegionInfo>> bulkPlan,
1276 final AssignmentManager am) {
1277 super(server);
1278 this.bulkPlan = bulkPlan;
1279 this.assignmentManager = am;
1280 }
1281
1282 @Override
1283 public boolean bulkAssign() throws InterruptedException {
1284
1285 this.assignmentManager.timeoutMonitor.bulkAssign(true);
1286 try {
1287 return super.bulkAssign();
1288 } finally {
1289
1290 this.assignmentManager.timeoutMonitor.bulkAssign(false);
1291 }
1292 }
1293
1294 @Override
1295 protected String getThreadNamePrefix() {
1296 return super.getThreadNamePrefix() + "-startup";
1297 }
1298
1299 @Override
1300 protected void populatePool(java.util.concurrent.ExecutorService pool) {
1301 for (Map.Entry<HServerInfo, List<HRegionInfo>> e: this.bulkPlan.entrySet()) {
1302 pool.execute(new SingleServerBulkAssigner(e.getKey(), e.getValue(),
1303 this.assignmentManager));
1304 }
1305 }
1306
1307 protected boolean waitUntilDone(final long timeout)
1308 throws InterruptedException {
1309 return this.assignmentManager.waitUntilNoRegionsInTransition(timeout);
1310 }
1311 }
1312
1313
1314
1315
1316 static class SingleServerBulkAssigner implements Runnable {
1317 private final HServerInfo regionserver;
1318 private final List<HRegionInfo> regions;
1319 private final AssignmentManager assignmentManager;
1320
1321 SingleServerBulkAssigner(final HServerInfo regionserver,
1322 final List<HRegionInfo> regions, final AssignmentManager am) {
1323 this.regionserver = regionserver;
1324 this.regions = regions;
1325 this.assignmentManager = am;
1326 }
1327 @Override
1328 public void run() {
1329 this.assignmentManager.assign(this.regionserver, this.regions);
1330 }
1331 }
1332
1333
1334
1335
1336
1337
1338
1339 boolean waitUntilNoRegionsInTransition(final long timeout)
1340 throws InterruptedException {
1341
1342
1343
1344
1345
1346
1347 long startTime = System.currentTimeMillis();
1348 long remaining = timeout;
1349 synchronized (regionsInTransition) {
1350 while (regionsInTransition.size() > 0 && !this.master.isStopped()
1351 && remaining > 0) {
1352 regionsInTransition.wait(remaining);
1353 remaining = timeout - (System.currentTimeMillis() - startTime);
1354 }
1355 }
1356 return regionsInTransition.isEmpty();
1357 }
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368 private Map<HServerInfo,List<Pair<HRegionInfo,Result>>> rebuildUserRegions()
1369 throws IOException {
1370
1371 List<Result> results = MetaReader.fullScanOfResults(catalogTracker);
1372
1373 Map<HServerInfo,List<Pair<HRegionInfo,Result>>> offlineServers =
1374 new TreeMap<HServerInfo,List<Pair<HRegionInfo,Result>>>();
1375
1376 for (Result result : results) {
1377 Pair<HRegionInfo,HServerInfo> region =
1378 MetaReader.metaRowToRegionPairWithInfo(result);
1379 if (region == null) continue;
1380 HServerInfo regionLocation = region.getSecond();
1381 HRegionInfo regionInfo = region.getFirst();
1382 if (regionLocation == null) {
1383
1384
1385 this.regions.put(regionInfo, null);
1386 } else if (!serverManager.isServerOnline(
1387 regionLocation.getServerName())) {
1388
1389 List<Pair<HRegionInfo,Result>> offlineRegions =
1390 offlineServers.get(regionLocation);
1391 if (offlineRegions == null) {
1392 offlineRegions = new ArrayList<Pair<HRegionInfo,Result>>(1);
1393 offlineServers.put(regionLocation, offlineRegions);
1394 }
1395 offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
1396 } else {
1397
1398 regions.put(regionInfo, regionLocation);
1399 addToServers(regionLocation, regionInfo);
1400 }
1401 }
1402 return offlineServers;
1403 }
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418 private void processDeadServers(
1419 Map<HServerInfo, List<Pair<HRegionInfo, Result>>> deadServers)
1420 throws IOException, KeeperException {
1421 for (Map.Entry<HServerInfo, List<Pair<HRegionInfo,Result>>> deadServer :
1422 deadServers.entrySet()) {
1423 List<Pair<HRegionInfo,Result>> regions = deadServer.getValue();
1424 for (Pair<HRegionInfo,Result> region : regions) {
1425 HRegionInfo regionInfo = region.getFirst();
1426 Result result = region.getSecond();
1427
1428 try {
1429 ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
1430 master.getServerName());
1431 } catch (KeeperException.NoNodeException nne) {
1432
1433 }
1434
1435 ServerShutdownHandler.processDeadRegion(regionInfo, result, this,
1436 this.catalogTracker);
1437 }
1438 }
1439 }
1440
1441
1442
1443
1444
1445
1446 private void addToServers(final HServerInfo hsi, final HRegionInfo hri) {
1447 List<HRegionInfo> hris = servers.get(hsi);
1448 if (hris == null) {
1449 hris = new ArrayList<HRegionInfo>();
1450 servers.put(hsi, hris);
1451 }
1452 hris.add(hri);
1453 }
1454
1455
1456
1457
1458 public NavigableMap<String, RegionState> getRegionsInTransition() {
1459 synchronized (this.regionsInTransition) {
1460 return new TreeMap<String, RegionState>(this.regionsInTransition);
1461 }
1462 }
1463
1464
1465
1466
1467 public boolean isRegionsInTransition() {
1468 synchronized (this.regionsInTransition) {
1469 return !this.regionsInTransition.isEmpty();
1470 }
1471 }
1472
1473
1474
1475
1476
1477
1478 public RegionState isRegionInTransition(final HRegionInfo hri) {
1479 synchronized (this.regionsInTransition) {
1480 return this.regionsInTransition.get(hri.getEncodedName());
1481 }
1482 }
1483
1484
1485
1486
1487
1488
1489
1490 public void clearRegionFromTransition(HRegionInfo hri) {
1491 synchronized (this.regionsInTransition) {
1492 this.regionsInTransition.remove(hri.getEncodedName());
1493 }
1494 synchronized (this.regions) {
1495 this.regions.remove(hri);
1496 for (List<HRegionInfo> regions : this.servers.values()) {
1497 for (int i=0;i<regions.size();i++) {
1498 if (regions.get(i).equals(hri)) {
1499 regions.remove(i);
1500 break;
1501 }
1502 }
1503 }
1504 }
1505 clearRegionPlan(hri);
1506 }
1507
1508
1509
1510
1511 void clearRegionPlan(final HRegionInfo region) {
1512 synchronized (this.regionPlans) {
1513 this.regionPlans.remove(region.getEncodedName());
1514 }
1515 }
1516
1517
1518
1519
1520
1521
1522 public void waitOnRegionToClearRegionsInTransition(final HRegionInfo hri)
1523 throws IOException {
1524 if (isRegionInTransition(hri) == null) return;
1525 RegionState rs = null;
1526
1527
1528 while(!this.master.isStopped() && (rs = isRegionInTransition(hri)) != null) {
1529 Threads.sleep(1000);
1530 LOG.info("Waiting on " + rs + " to clear regions-in-transition");
1531 }
1532 if (this.master.isStopped()) {
1533 LOG.info("Giving up wait on regions in " +
1534 "transition because stoppable.isStopped is set");
1535 }
1536 }
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549 public List<HRegionInfo> getRegionsOfTable(byte[] tableName) {
1550 List<HRegionInfo> tableRegions = new ArrayList<HRegionInfo>();
1551 HRegionInfo boundary =
1552 new HRegionInfo(new HTableDescriptor(tableName), null, null);
1553 synchronized (this.regions) {
1554 for (HRegionInfo regionInfo: this.regions.tailMap(boundary).keySet()) {
1555 if(Bytes.equals(regionInfo.getTableDesc().getName(), tableName)) {
1556 tableRegions.add(regionInfo);
1557 } else {
1558 break;
1559 }
1560 }
1561 }
1562 return tableRegions;
1563 }
1564
1565
1566
1567
1568 public class TimeoutMonitor extends Chore {
1569 private final int timeout;
1570 private boolean bulkAssign = false;
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581 public TimeoutMonitor(final int period, final Stoppable stopper,
1582 final int timeout) {
1583 super("AssignmentTimeoutMonitor", period, stopper);
1584 this.timeout = timeout;
1585 }
1586
1587
1588
1589
1590
1591
1592 public boolean bulkAssign(final boolean bulkAssign) {
1593 boolean result = this.bulkAssign;
1594 this.bulkAssign = bulkAssign;
1595 return result;
1596 }
1597
1598 @Override
1599 protected void chore() {
1600
1601 if (this.bulkAssign) return;
1602 List<HRegionInfo> unassigns = new ArrayList<HRegionInfo>();
1603 Map<HRegionInfo, Boolean> assigns =
1604 new HashMap<HRegionInfo, Boolean>();
1605 synchronized (regionsInTransition) {
1606
1607 long now = System.currentTimeMillis();
1608 for (RegionState regionState : regionsInTransition.values()) {
1609 if (regionState.getStamp() + timeout <= now) {
1610 HRegionInfo regionInfo = regionState.getRegion();
1611 LOG.info("Regions in transition timed out: " + regionState);
1612
1613 switch (regionState.getState()) {
1614 case CLOSED:
1615 LOG.info("Region " + regionInfo.getEncodedName() +
1616 " has been CLOSED for too long, waiting on queued " +
1617 "ClosedRegionHandler to run or server shutdown");
1618
1619 synchronized(regionState) {
1620 regionState.update(regionState.getState());
1621 }
1622 break;
1623 case OFFLINE:
1624 LOG.info("Region has been OFFLINE for too long, " +
1625 "reassigning " + regionInfo.getRegionNameAsString() +
1626 " to a random server");
1627 assigns.put(regionState.getRegion(), Boolean.FALSE);
1628 break;
1629 case PENDING_OPEN:
1630 LOG.info("Region has been PENDING_OPEN for too " +
1631 "long, reassigning region=" +
1632 regionInfo.getRegionNameAsString());
1633 assigns.put(regionState.getRegion(), Boolean.TRUE);
1634 break;
1635 case OPENING:
1636 LOG.info("Region has been OPENING for too " +
1637 "long, reassigning region=" +
1638 regionInfo.getRegionNameAsString());
1639
1640 try {
1641 String node = ZKAssign.getNodeName(watcher,
1642 regionInfo.getEncodedName());
1643 Stat stat = new Stat();
1644 RegionTransitionData data = ZKAssign.getDataNoWatch(watcher,
1645 node, stat);
1646 if (data == null) {
1647 LOG.warn("Data is null, node " + node + " no longer exists");
1648 break;
1649 }
1650 if (data.getEventType() == EventType.RS_ZK_REGION_OPENED) {
1651 LOG.debug("Region has transitioned to OPENED, allowing " +
1652 "watched event handlers to process");
1653 break;
1654 } else if (data.getEventType() !=
1655 EventType.RS_ZK_REGION_OPENING) {
1656 LOG.warn("While timing out a region in state OPENING, " +
1657 "found ZK node in unexpected state: " +
1658 data.getEventType());
1659 break;
1660 }
1661
1662 try {
1663 data = new RegionTransitionData(
1664 EventType.M_ZK_REGION_OFFLINE, regionInfo.getRegionName(),
1665 master.getServerName());
1666 if (ZKUtil.setData(watcher, node, data.getBytes(),
1667 stat.getVersion())) {
1668
1669 ZKUtil.getDataAndWatch(watcher, node);
1670 LOG.info("Successfully transitioned region=" +
1671 regionInfo.getRegionNameAsString() + " into OFFLINE" +
1672 " and forcing a new assignment");
1673 assigns.put(regionState.getRegion(), Boolean.TRUE);
1674 }
1675 } catch (KeeperException.NoNodeException nne) {
1676
1677 }
1678 } catch (KeeperException ke) {
1679 LOG.error("Unexpected ZK exception timing out CLOSING region",
1680 ke);
1681 break;
1682 }
1683 break;
1684 case OPEN:
1685 LOG.error("Region has been OPEN for too long, " +
1686 "we don't know where region was opened so can't do anything");
1687 break;
1688 case PENDING_CLOSE:
1689 LOG.info("Region has been PENDING_CLOSE for too " +
1690 "long, running forced unassign again on region=" +
1691 regionInfo.getRegionNameAsString());
1692 try {
1693
1694
1695 if (!ZKUtil.watchAndCheckExists(watcher,
1696 ZKAssign.getNodeName(watcher, regionInfo.getEncodedName()))) {
1697
1698
1699 unassigns.add(regionInfo);
1700 }
1701 } catch (NoNodeException e) {
1702 LOG.debug("Node no longer existed so not forcing another " +
1703 "unassignment");
1704 } catch (KeeperException e) {
1705 LOG.warn("Unexpected ZK exception timing out a region " +
1706 "close", e);
1707 }
1708 break;
1709 case CLOSING:
1710 LOG.info("Region has been CLOSING for too " +
1711 "long, this should eventually complete or the server will " +
1712 "expire, doing nothing");
1713 break;
1714 }
1715 }
1716 }
1717 }
1718
1719 for (HRegionInfo hri: unassigns) {
1720 unassign(hri, true);
1721 }
1722 for (Map.Entry<HRegionInfo, Boolean> e: assigns.entrySet()){
1723 assign(e.getKey(), false, e.getValue());
1724 }
1725 }
1726 }
1727
1728
1729
1730
1731
1732
1733 public List<RegionState> processServerShutdown(final HServerInfo hsi) {
1734
1735 synchronized (this.regionPlans) {
1736 for (Iterator <Map.Entry<String, RegionPlan>> i =
1737 this.regionPlans.entrySet().iterator(); i.hasNext();) {
1738 Map.Entry<String, RegionPlan> e = i.next();
1739 if (e.getValue().getDestination().equals(hsi)) {
1740
1741 i.remove();
1742 }
1743 }
1744 }
1745
1746
1747
1748 Set<HRegionInfo> deadRegions = null;
1749 List<RegionState> rits = new ArrayList<RegionState>();
1750 synchronized (this.regions) {
1751 List<HRegionInfo> assignedRegions = this.servers.remove(hsi);
1752 if (assignedRegions == null || assignedRegions.isEmpty()) {
1753
1754 return rits;
1755 }
1756 deadRegions = new TreeSet<HRegionInfo>(assignedRegions);
1757 for (HRegionInfo region : deadRegions) {
1758 this.regions.remove(region);
1759 }
1760 }
1761
1762
1763
1764 synchronized (regionsInTransition) {
1765 for (RegionState region : this.regionsInTransition.values()) {
1766 if (deadRegions.remove(region.getRegion())) {
1767 rits.add(region);
1768 }
1769 }
1770 }
1771 return rits;
1772 }
1773
1774
1775
1776
1777
1778
1779
1780
1781 public void handleSplitReport(final HServerInfo hsi, final HRegionInfo parent,
1782 final HRegionInfo a, final HRegionInfo b) {
1783 regionOffline(parent);
1784
1785
1786
1787 try {
1788 RegionTransitionData node = ZKAssign.getDataNoWatch(this.watcher,
1789 parent.getEncodedName(), null);
1790 if (node != null) {
1791 if (node.getEventType().equals(EventType.RS_ZK_REGION_CLOSING)) {
1792 ZKAssign.deleteClosingNode(this.watcher, parent);
1793 } else {
1794 LOG.warn("Split report has RIT node (shouldnt have one): " +
1795 parent + " node: " + node);
1796 }
1797 }
1798 } catch (KeeperException e) {
1799 LOG.warn("Exception while validating RIT during split report", e);
1800 }
1801
1802 regionOnline(a, hsi);
1803 regionOnline(b, hsi);
1804
1805
1806
1807
1808
1809 if (this.zkTable.isDisablingOrDisabledTable(
1810 parent.getTableDesc().getNameAsString())) {
1811 unassign(a);
1812 unassign(b);
1813 }
1814 }
1815
1816
1817
1818
1819
1820
1821 Map<HServerInfo, List<HRegionInfo>> getAssignments() {
1822
1823
1824
1825
1826 Map<HServerInfo, List<HRegionInfo>> result = null;
1827 synchronized (this.regions) {
1828 result = new HashMap<HServerInfo, List<HRegionInfo>>(this.servers.size());
1829 for (Map.Entry<HServerInfo, List<HRegionInfo>> e: this.servers.entrySet()) {
1830 List<HRegionInfo> shallowCopy = new ArrayList<HRegionInfo>(e.getValue());
1831 HServerInfo clone = new HServerInfo(e.getKey());
1832
1833
1834 clone.getLoad().setNumberOfRegions(e.getValue().size());
1835 result.put(clone, shallowCopy);
1836 }
1837 }
1838 return result;
1839 }
1840
1841
1842
1843
1844
1845
1846 Pair<HRegionInfo, HServerInfo> getAssignment(final byte [] encodedRegionName) {
1847 String name = Bytes.toString(encodedRegionName);
1848 synchronized(this.regions) {
1849 for (Map.Entry<HRegionInfo, HServerInfo> e: this.regions.entrySet()) {
1850 if (e.getKey().getEncodedName().equals(name)) {
1851 return new Pair<HRegionInfo, HServerInfo>(e.getKey(), e.getValue());
1852 }
1853 }
1854 }
1855 return null;
1856 }
1857
1858
1859
1860
1861 void balance(final RegionPlan plan) {
1862 synchronized (this.regionPlans) {
1863 this.regionPlans.put(plan.getRegionName(), plan);
1864 }
1865 unassign(plan.getRegionInfo());
1866 }
1867
1868
1869
1870
1871 public static class RegionState implements Writable {
1872 private HRegionInfo region;
1873
1874 public enum State {
1875 OFFLINE,
1876 PENDING_OPEN,
1877 OPENING,
1878 OPEN,
1879 PENDING_CLOSE,
1880 CLOSING,
1881 CLOSED
1882 }
1883
1884 private State state;
1885 private long stamp;
1886
1887 public RegionState() {}
1888
1889 RegionState(HRegionInfo region, State state) {
1890 this(region, state, System.currentTimeMillis());
1891 }
1892
1893 RegionState(HRegionInfo region, State state, long stamp) {
1894 this.region = region;
1895 this.state = state;
1896 this.stamp = stamp;
1897 }
1898
1899 public void update(State state, long stamp) {
1900 this.state = state;
1901 this.stamp = stamp;
1902 }
1903
1904 public void update(State state) {
1905 this.state = state;
1906 this.stamp = System.currentTimeMillis();
1907 }
1908
1909 public State getState() {
1910 return state;
1911 }
1912
1913 public long getStamp() {
1914 return stamp;
1915 }
1916
1917 public HRegionInfo getRegion() {
1918 return region;
1919 }
1920
1921 public boolean isClosing() {
1922 return state == State.CLOSING;
1923 }
1924
1925 public boolean isClosed() {
1926 return state == State.CLOSED;
1927 }
1928
1929 public boolean isPendingClose() {
1930 return state == State.PENDING_CLOSE;
1931 }
1932
1933 public boolean isOpening() {
1934 return state == State.OPENING;
1935 }
1936
1937 public boolean isOpened() {
1938 return state == State.OPEN;
1939 }
1940
1941 public boolean isPendingOpen() {
1942 return state == State.PENDING_OPEN;
1943 }
1944
1945 public boolean isOffline() {
1946 return state == State.OFFLINE;
1947 }
1948
1949 @Override
1950 public String toString() {
1951 return region.getRegionNameAsString() + " state=" + state +
1952 ", ts=" + stamp;
1953 }
1954
1955 @Override
1956 public void readFields(DataInput in) throws IOException {
1957 region = new HRegionInfo();
1958 region.readFields(in);
1959 state = State.valueOf(in.readUTF());
1960 stamp = in.readLong();
1961 }
1962
1963 @Override
1964 public void write(DataOutput out) throws IOException {
1965 region.write(out);
1966 out.writeUTF(state.name());
1967 out.writeLong(stamp);
1968 }
1969 }
1970
1971 public void stop() {
1972 this.timeoutMonitor.interrupt();
1973 }
1974 }