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