1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.zookeeper;
21
22 import java.io.IOException;
23 import java.io.UnsupportedEncodingException;
24 import java.lang.reflect.Field;
25 import java.net.URLDecoder;
26 import java.net.URLEncoder;
27 import java.util.ArrayList;
28 import java.util.Arrays;
29 import java.util.List;
30 import java.util.concurrent.atomic.AtomicLong;
31
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.fs.FileSystem;
35 import org.apache.hadoop.fs.Path;
36 import org.apache.hadoop.hbase.HConstants;
37 import org.apache.hadoop.hbase.master.SplitLogManager;
38 import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
39 import org.apache.hadoop.hbase.util.Bytes;
40
41
42
43
44
45 public class ZKSplitLog {
46 private static final Log LOG = LogFactory.getLog(ZKSplitLog.class);
47
48 public static final int DEFAULT_TIMEOUT = 25000;
49 public static final int DEFAULT_ZK_RETRIES = 3;
50 public static final int DEFAULT_MAX_RESUBMIT = 3;
51 public static final int DEFAULT_UNASSIGNED_TIMEOUT = (3 * 60 * 1000);
52
53
54
55
56
57
58
59 public static String getEncodedNodeName(ZooKeeperWatcher zkw,
60 String filename) {
61 return ZKUtil.joinZNode(zkw.splitLogZNode, encode(filename));
62 }
63
64 public static String getFileName(String node) {
65 String basename = node.substring(node.lastIndexOf('/') + 1);
66 return decode(basename);
67 }
68
69
70 public static String encode(String s) {
71 try {
72 return URLEncoder.encode(s, "UTF-8");
73 } catch (UnsupportedEncodingException e) {
74 throw new RuntimeException("URLENCODER doesn't support UTF-8");
75 }
76 }
77
78 public static String decode(String s) {
79 try {
80 return URLDecoder.decode(s, "UTF-8");
81 } catch (UnsupportedEncodingException e) {
82 throw new RuntimeException("URLDecoder doesn't support UTF-8");
83 }
84 }
85
86 public static String getRescanNode(ZooKeeperWatcher zkw) {
87 return ZKUtil.joinZNode(zkw.splitLogZNode, "RESCAN");
88 }
89
90 public static boolean isRescanNode(ZooKeeperWatcher zkw, String path) {
91 String prefix = getRescanNode(zkw);
92 if (path.length() <= prefix.length()) {
93 return false;
94 }
95 for (int i = 0; i < prefix.length(); i++) {
96 if (prefix.charAt(i) != path.charAt(i)) {
97 return false;
98 }
99 }
100 return true;
101 }
102
103 public static boolean isTaskPath(ZooKeeperWatcher zkw, String path) {
104 String dirname = path.substring(0, path.lastIndexOf('/'));
105 return dirname.equals(zkw.splitLogZNode);
106 }
107
108 public static enum TaskState {
109 TASK_UNASSIGNED("unassigned"),
110 TASK_OWNED("owned"),
111 TASK_RESIGNED("resigned"),
112 TASK_DONE("done"),
113 TASK_ERR("err");
114
115 private final byte[] state;
116 private TaskState(String s) {
117 state = s.getBytes();
118 }
119
120 public byte[] get(String serverName) {
121 return (Bytes.add(state, " ".getBytes(), serverName.getBytes()));
122 }
123
124 public String getWriterName(byte[] data) {
125 String str = Bytes.toString(data);
126 return str.substring(str.indexOf(' ') + 1);
127 }
128
129
130
131
132
133
134 public boolean equals(byte[] s) {
135 if (s.length < state.length) {
136 return (false);
137 }
138 for (int i = 0; i < state.length; i++) {
139 if (state[i] != s[i]) {
140 return (false);
141 }
142 }
143 return (true);
144 }
145
146 public boolean equals(byte[] s, String serverName) {
147 return (Arrays.equals(s, get(serverName)));
148 }
149 @Override
150 public String toString() {
151 return new String(state);
152 }
153 }
154
155 public static Path getSplitLogDir(Path rootdir, String tmpname) {
156 return new Path(new Path(rootdir, HConstants.SPLIT_LOGDIR_NAME), tmpname);
157 }
158
159 public static String getSplitLogDirTmpComponent(String worker, String file) {
160 return (worker + "_" + ZKSplitLog.encode(file));
161 }
162
163 public static void markCorrupted(Path rootdir, String logFileName,
164 FileSystem fs) {
165 Path file = new Path(getSplitLogDir(rootdir, logFileName), "corrupt");
166 try {
167 fs.createNewFile(file);
168 } catch (IOException e) {
169 LOG.warn("Could not flag a log file as corrupted. Failed to create " +
170 file, e);
171 }
172 }
173
174 public static boolean isCorrupted(Path rootdir, String logFileName,
175 FileSystem fs) throws IOException {
176 Path file = new Path(getSplitLogDir(rootdir, logFileName), "corrupt");
177 boolean isCorrupt;
178 isCorrupt = fs.exists(file);
179 return isCorrupt;
180 }
181
182
183 public static class Counters {
184
185 public static AtomicLong tot_mgr_log_split_batch_start = new AtomicLong(0);
186 public static AtomicLong tot_mgr_log_split_batch_success =
187 new AtomicLong(0);
188 public static AtomicLong tot_mgr_log_split_batch_err = new AtomicLong(0);
189 public static AtomicLong tot_mgr_new_unexpected_hlogs = new AtomicLong(0);
190 public static AtomicLong tot_mgr_log_split_start = new AtomicLong(0);
191 public static AtomicLong tot_mgr_log_split_success = new AtomicLong(0);
192 public static AtomicLong tot_mgr_log_split_err = new AtomicLong(0);
193 public static AtomicLong tot_mgr_node_create_queued = new AtomicLong(0);
194 public static AtomicLong tot_mgr_node_create_result = new AtomicLong(0);
195 public static AtomicLong tot_mgr_node_already_exists = new AtomicLong(0);
196 public static AtomicLong tot_mgr_node_create_err = new AtomicLong(0);
197 public static AtomicLong tot_mgr_node_create_retry = new AtomicLong(0);
198 public static AtomicLong tot_mgr_get_data_queued = new AtomicLong(0);
199 public static AtomicLong tot_mgr_get_data_result = new AtomicLong(0);
200 public static AtomicLong tot_mgr_get_data_nonode = new AtomicLong(0);
201 public static AtomicLong tot_mgr_get_data_err = new AtomicLong(0);
202 public static AtomicLong tot_mgr_get_data_retry = new AtomicLong(0);
203 public static AtomicLong tot_mgr_node_delete_queued = new AtomicLong(0);
204 public static AtomicLong tot_mgr_node_delete_result = new AtomicLong(0);
205 public static AtomicLong tot_mgr_node_delete_err = new AtomicLong(0);
206 public static AtomicLong tot_mgr_resubmit = new AtomicLong(0);
207 public static AtomicLong tot_mgr_resubmit_failed = new AtomicLong(0);
208 public static AtomicLong tot_mgr_null_data = new AtomicLong(0);
209 public static AtomicLong tot_mgr_orphan_task_acquired = new AtomicLong(0);
210 public static AtomicLong tot_mgr_wait_for_zk_delete = new AtomicLong(0);
211 public static AtomicLong tot_mgr_unacquired_orphan_done = new AtomicLong(0);
212 public static AtomicLong tot_mgr_resubmit_threshold_reached =
213 new AtomicLong(0);
214 public static AtomicLong tot_mgr_missing_state_in_delete =
215 new AtomicLong(0);
216 public static AtomicLong tot_mgr_heartbeat = new AtomicLong(0);
217 public static AtomicLong tot_mgr_rescan = new AtomicLong(0);
218 public static AtomicLong tot_mgr_rescan_deleted = new AtomicLong(0);
219 public static AtomicLong tot_mgr_task_deleted = new AtomicLong(0);
220 public static AtomicLong tot_mgr_resubmit_unassigned = new AtomicLong(0);
221 public static AtomicLong tot_mgr_relist_logdir = new AtomicLong(0);
222 public static AtomicLong tot_mgr_resubmit_dead_server_task =
223 new AtomicLong(0);
224
225
226
227
228 public static AtomicLong tot_wkr_failed_to_grab_task_no_data =
229 new AtomicLong(0);
230 public static AtomicLong tot_wkr_failed_to_grab_task_exception =
231 new AtomicLong(0);
232 public static AtomicLong tot_wkr_failed_to_grab_task_owned =
233 new AtomicLong(0);
234 public static AtomicLong tot_wkr_failed_to_grab_task_lost_race =
235 new AtomicLong(0);
236 public static AtomicLong tot_wkr_task_acquired = new AtomicLong(0);
237 public static AtomicLong tot_wkr_task_resigned = new AtomicLong(0);
238 public static AtomicLong tot_wkr_task_done = new AtomicLong(0);
239 public static AtomicLong tot_wkr_task_err = new AtomicLong(0);
240 public static AtomicLong tot_wkr_task_heartbeat = new AtomicLong(0);
241 public static AtomicLong tot_wkr_task_acquired_rescan = new AtomicLong(0);
242 public static AtomicLong tot_wkr_get_data_queued = new AtomicLong(0);
243 public static AtomicLong tot_wkr_get_data_result = new AtomicLong(0);
244 public static AtomicLong tot_wkr_get_data_retry = new AtomicLong(0);
245 public static AtomicLong tot_wkr_preempt_task = new AtomicLong(0);
246 public static AtomicLong tot_wkr_task_heartbeat_failed = new AtomicLong(0);
247 public static AtomicLong tot_wkr_final_transistion_failed =
248 new AtomicLong(0);
249
250 public static void resetCounters() throws Exception {
251 Class<?> cl = (new Counters()).getClass();
252 Field[] flds = cl.getDeclaredFields();
253 for (Field fld : flds) {
254 ((AtomicLong)fld.get(null)).set(0);
255 }
256 }
257 }
258 }