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