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