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 org.apache.hadoop.conf.Configuration;
22 import org.apache.hadoop.hbase.ChoreService;
23 import org.apache.hadoop.hbase.HConstants;
24 import org.apache.hadoop.hbase.classification.InterfaceAudience;
25 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
26 import org.slf4j.Logger;
27 import org.slf4j.LoggerFactory;
28
29
30
31
32
33 @InterfaceAudience.Private
34 public class RegionsRecoveryConfigManager implements ConfigurationObserver {
35
36 private static final Logger LOG = LoggerFactory.getLogger(RegionsRecoveryConfigManager.class);
37
38 private final HMaster hMaster;
39 private int prevMaxStoreFileRefCount;
40 private int prevRegionsRecoveryInterval;
41
42 RegionsRecoveryConfigManager(final HMaster hMaster) {
43 this.hMaster = hMaster;
44 Configuration conf = hMaster.getConfiguration();
45 this.prevMaxStoreFileRefCount = getMaxStoreFileRefCount(conf);
46 this.prevRegionsRecoveryInterval = getRegionsRecoveryChoreInterval(conf);
47 }
48
49 @Override
50 public void onConfigurationChange(Configuration conf) {
51 final int newMaxStoreFileRefCount = getMaxStoreFileRefCount(conf);
52 final int newRegionsRecoveryInterval = getRegionsRecoveryChoreInterval(conf);
53
54 if (prevMaxStoreFileRefCount == newMaxStoreFileRefCount
55 && prevRegionsRecoveryInterval == newRegionsRecoveryInterval) {
56
57
58 return;
59 }
60
61 LOG.info("Config Reload for RegionsRecovery Chore. prevMaxStoreFileRefCount: {}," +
62 " newMaxStoreFileRefCount: {}, prevRegionsRecoveryInterval: {}, " +
63 "newRegionsRecoveryInterval: {}", prevMaxStoreFileRefCount, newMaxStoreFileRefCount,
64 prevRegionsRecoveryInterval, newRegionsRecoveryInterval);
65
66 RegionsRecoveryChore regionsRecoveryChore = new RegionsRecoveryChore(this.hMaster,
67 conf, this.hMaster);
68 ChoreService choreService = this.hMaster.getChoreService();
69
70
71
72
73 synchronized (this) {
74 if (newMaxStoreFileRefCount > 0) {
75
76
77
78 choreService.cancelChore(regionsRecoveryChore, false);
79 choreService.scheduleChore(regionsRecoveryChore);
80 } else {
81 choreService.cancelChore(regionsRecoveryChore, false);
82 }
83 this.prevMaxStoreFileRefCount = newMaxStoreFileRefCount;
84 this.prevRegionsRecoveryInterval = newRegionsRecoveryInterval;
85 }
86 }
87
88 private int getMaxStoreFileRefCount(Configuration configuration) {
89 return configuration.getInt(
90 HConstants.STORE_FILE_REF_COUNT_THRESHOLD,
91 HConstants.DEFAULT_STORE_FILE_REF_COUNT_THRESHOLD);
92 }
93
94 private int getRegionsRecoveryChoreInterval(Configuration configuration) {
95 return configuration.getInt(
96 HConstants.REGIONS_RECOVERY_INTERVAL,
97 HConstants.DEFAULT_REGIONS_RECOVERY_INTERVAL);
98 }
99
100 }