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