1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.backup.master;
20
21 import java.io.IOException;
22 import java.io.InputStream;
23 import java.io.OutputStream;
24 import java.util.ArrayList;
25 import java.util.HashMap;
26 import java.util.List;
27 import java.util.Map.Entry;
28 import java.util.concurrent.atomic.AtomicBoolean;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.fs.FileSystem;
34 import org.apache.hadoop.fs.Path;
35 import org.apache.hadoop.hbase.TableName;
36 import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
37 import org.apache.hadoop.hbase.backup.BackupInfo;
38 import org.apache.hadoop.hbase.backup.BackupRestoreServerFactory;
39 import org.apache.hadoop.hbase.backup.BackupType;
40 import org.apache.hadoop.hbase.backup.BackupInfo.BackupPhase;
41 import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
42 import org.apache.hadoop.hbase.backup.BackupCopyService;
43 import org.apache.hadoop.hbase.backup.impl.BackupManager;
44 import org.apache.hadoop.hbase.backup.util.BackupServerUtil;
45 import org.apache.hadoop.hbase.backup.impl.IncrementalBackupManager;
46 import org.apache.hadoop.hbase.classification.InterfaceAudience;
47 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
48 import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
49 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
50 import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
51 import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.IncrementalTableBackupState;
52 import org.apache.hadoop.hbase.protobuf.generated.BackupProtos.ServerTimestamp;
53
54 @InterfaceAudience.Private
55 public class IncrementalTableBackupProcedure
56 extends StateMachineProcedure<MasterProcedureEnv, IncrementalTableBackupState>
57 implements TableProcedureInterface {
58 private static final Log LOG = LogFactory.getLog(IncrementalTableBackupProcedure.class);
59
60 private final AtomicBoolean aborted = new AtomicBoolean(false);
61 private Configuration conf;
62 private String backupId;
63 private List<TableName> tableList;
64 private String targetRootDir;
65 HashMap<String, Long> newTimestamps = null;
66
67 private BackupManager backupManager;
68 private BackupInfo backupContext;
69
70 public IncrementalTableBackupProcedure() {
71
72 }
73
74 public IncrementalTableBackupProcedure(final MasterProcedureEnv env,
75 final String backupId,
76 List<TableName> tableList, String targetRootDir, final int workers,
77 final long bandwidth) throws IOException {
78 backupManager = new BackupManager(env.getMasterConfiguration());
79 this.backupId = backupId;
80 this.tableList = tableList;
81 this.targetRootDir = targetRootDir;
82 backupContext = backupManager.createBackupInfo(backupId, BackupType.INCREMENTAL, tableList,
83 targetRootDir, workers, bandwidth);
84 }
85
86 @Override
87 public byte[] getResult() {
88 return backupId.getBytes();
89 }
90
91 private List<String> filterMissingFiles(List<String> incrBackupFileList) throws IOException {
92 FileSystem fs = FileSystem.get(conf);
93 List<String> list = new ArrayList<String>();
94 for(String file : incrBackupFileList){
95 if(fs.exists(new Path(file))){
96 list.add(file);
97 } else{
98 LOG.warn("Can't find file: "+file);
99 }
100 }
101 return list;
102 }
103
104
105
106
107
108 private void incrementalCopy(BackupInfo backupContext) throws Exception {
109
110 LOG.info("Incremental copy is starting.");
111
112
113 backupContext.setPhase(BackupPhase.INCREMENTAL_COPY);
114
115
116 List<String> incrBackupFileList = backupContext.getIncrBackupFileList();
117
118 incrBackupFileList = filterMissingFiles(incrBackupFileList);
119 String[] strArr = incrBackupFileList.toArray(new String[incrBackupFileList.size() + 1]);
120 strArr[strArr.length - 1] = backupContext.getHLogTargetDir();
121
122 BackupCopyService copyService = BackupRestoreServerFactory.getBackupCopyService(conf);
123 int res = copyService.copy(backupContext, backupManager, conf,
124 BackupCopyService.Type.INCREMENTAL, strArr);
125
126 if (res != 0) {
127 LOG.error("Copy incremental log files failed with return code: " + res + ".");
128 throw new IOException("Failed of Hadoop Distributed Copy from " + incrBackupFileList + " to "
129 + backupContext.getHLogTargetDir());
130 }
131 LOG.info("Incremental copy from " + incrBackupFileList + " to "
132 + backupContext.getHLogTargetDir() + " finished.");
133 }
134
135 @Override
136 protected Flow executeFromState(final MasterProcedureEnv env,
137 final IncrementalTableBackupState state) {
138 if (conf == null) {
139 conf = env.getMasterConfiguration();
140 }
141 if (backupManager == null) {
142 try {
143 backupManager = new BackupManager(env.getMasterConfiguration());
144 } catch (IOException ioe) {
145 setFailure("incremental backup", ioe);
146 }
147 }
148 if (LOG.isTraceEnabled()) {
149 LOG.trace(this + " execute state=" + state);
150 }
151 try {
152 switch (state) {
153 case PREPARE_INCREMENTAL:
154 FullTableBackupProcedure.beginBackup(backupManager, backupContext);
155 LOG.debug("For incremental backup, current table set is "
156 + backupManager.getIncrementalBackupTableSet());
157 try {
158 IncrementalBackupManager incrBackupManager =new IncrementalBackupManager(backupManager);
159
160 newTimestamps = incrBackupManager.getIncrBackupLogFileList(backupContext);
161 } catch (Exception e) {
162 setFailure("Failure in incremental-backup: preparation phase " + backupId, e);
163
164 FullTableBackupProcedure.failBackup(env, backupContext, backupManager, e,
165 "Unexpected Exception : ", BackupType.INCREMENTAL, conf);
166 }
167
168 setNextState(IncrementalTableBackupState.INCREMENTAL_COPY);
169 break;
170 case INCREMENTAL_COPY:
171 try {
172
173 BackupServerUtil.copyTableRegionInfo(backupContext, conf);
174 incrementalCopy(backupContext);
175
176 backupManager.recordWALFiles(backupContext.getIncrBackupFileList());
177 } catch (Exception e) {
178 String msg = "Unexpected exception in incremental-backup: incremental copy " + backupId;
179 setFailure(msg, e);
180
181 FullTableBackupProcedure.failBackup(env, backupContext, backupManager, e,
182 msg, BackupType.INCREMENTAL, conf);
183 }
184 setNextState(IncrementalTableBackupState.INCR_BACKUP_COMPLETE);
185 break;
186 case INCR_BACKUP_COMPLETE:
187
188
189 backupContext.setState(BackupState.COMPLETE);
190
191 HashMap<TableName, HashMap<String, Long>> previousTimestampMap =
192 backupManager.readLogTimestampMap();
193 backupContext.setIncrTimestampMap(previousTimestampMap);
194
195
196
197 backupManager.writeRegionServerLogTimestamp(backupContext.getTables(), newTimestamps);
198
199 HashMap<TableName, HashMap<String, Long>> newTableSetTimestampMap =
200 backupManager.readLogTimestampMap();
201
202 Long newStartCode = BackupClientUtil
203 .getMinValue(BackupServerUtil.getRSLogTimestampMins(newTableSetTimestampMap));
204 backupManager.writeBackupStartCode(newStartCode);
205
206 FullTableBackupProcedure.completeBackup(env, backupContext, backupManager,
207 BackupType.INCREMENTAL, conf);
208 return Flow.NO_MORE_STATE;
209
210 default:
211 throw new UnsupportedOperationException("unhandled state=" + state);
212 }
213 } catch (IOException e) {
214 setFailure("snapshot-table", e);
215 }
216 return Flow.HAS_MORE_STATE;
217 }
218
219 @Override
220 protected void rollbackState(final MasterProcedureEnv env,
221 final IncrementalTableBackupState state) throws IOException {
222
223
224
225 FullTableBackupProcedure.cleanupTargetDir(backupContext, conf);
226 }
227
228 @Override
229 protected IncrementalTableBackupState getState(final int stateId) {
230 return IncrementalTableBackupState.valueOf(stateId);
231 }
232
233 @Override
234 protected int getStateId(final IncrementalTableBackupState state) {
235 return state.getNumber();
236 }
237
238 @Override
239 protected IncrementalTableBackupState getInitialState() {
240 return IncrementalTableBackupState.PREPARE_INCREMENTAL;
241 }
242
243 @Override
244 protected void setNextState(final IncrementalTableBackupState state) {
245 if (aborted.get()) {
246 setAbortFailure("snapshot-table", "abort requested");
247 } else {
248 super.setNextState(state);
249 }
250 }
251
252 @Override
253 public boolean abort(final MasterProcedureEnv env) {
254 aborted.set(true);
255 return true;
256 }
257
258 @Override
259 public void toStringClassDetails(StringBuilder sb) {
260 sb.append(getClass().getSimpleName());
261 sb.append(" (targetRootDir=");
262 sb.append(targetRootDir);
263 sb.append("; backupId=").append(backupId);
264 sb.append("; tables=");
265 int len = tableList.size();
266 for (int i = 0; i < len-1; i++) {
267 sb.append(tableList.get(i)).append(",");
268 }
269 sb.append(tableList.get(len-1));
270 sb.append(")");
271 }
272
273 BackupProtos.BackupProcContext toBackupInfo() {
274 BackupProtos.BackupProcContext.Builder ctxBuilder = BackupProtos.BackupProcContext.newBuilder();
275 ctxBuilder.setCtx(backupContext.toProtosBackupInfo());
276 if (newTimestamps != null && !newTimestamps.isEmpty()) {
277 BackupProtos.ServerTimestamp.Builder tsBuilder = ServerTimestamp.newBuilder();
278 for (Entry<String, Long> entry : newTimestamps.entrySet()) {
279 tsBuilder.clear().setServer(entry.getKey()).setTimestamp(entry.getValue());
280 ctxBuilder.addServerTimestamp(tsBuilder.build());
281 }
282 }
283 return ctxBuilder.build();
284 }
285
286 @Override
287 public void serializeStateData(final OutputStream stream) throws IOException {
288 super.serializeStateData(stream);
289
290 BackupProtos.BackupProcContext backupProcCtx = toBackupInfo();
291 backupProcCtx.writeDelimitedTo(stream);
292 }
293
294 @Override
295 public void deserializeStateData(final InputStream stream) throws IOException {
296 super.deserializeStateData(stream);
297
298 BackupProtos.BackupProcContext proto =BackupProtos.BackupProcContext.parseDelimitedFrom(stream);
299 backupContext = BackupInfo.fromProto(proto.getCtx());
300 backupId = backupContext.getBackupId();
301 targetRootDir = backupContext.getTargetRootDir();
302 tableList = backupContext.getTableNames();
303 List<ServerTimestamp> svrTimestamps = proto.getServerTimestampList();
304 if (svrTimestamps != null && !svrTimestamps.isEmpty()) {
305 newTimestamps = new HashMap<>();
306 for (ServerTimestamp ts : svrTimestamps) {
307 newTimestamps.put(ts.getServer(), ts.getTimestamp());
308 }
309 }
310 }
311
312 @Override
313 public TableName getTableName() {
314 return TableName.BACKUP_TABLE_NAME;
315 }
316
317 @Override
318 public TableOperationType getTableOperationType() {
319 return TableOperationType.BACKUP;
320 }
321
322 @Override
323 protected boolean acquireLock(final MasterProcedureEnv env) {
324 if (!env.isInitialized() && !getTableName().isSystemTable()) {
325 return false;
326 }
327 return env.getProcedureQueue().tryAcquireTableWrite(getTableName(), "incremental backup");
328
329
330
331
332
333
334 }
335
336 @Override
337 protected void releaseLock(final MasterProcedureEnv env) {
338 env.getProcedureQueue().releaseTableWrite(getTableName());
339 }
340 }