1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
9 *
10 * http://www.apache.org/licenses/LICENSE-2.0
11 *
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
17 */
18 package org.apache.hadoop.hbase.snapshot;
19
20 import java.io.IOException;
21 import java.util.NavigableSet;
22
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.classification.InterfaceAudience;
26 import org.apache.hadoop.classification.InterfaceStability;
27 import org.apache.hadoop.fs.FileStatus;
28 import org.apache.hadoop.fs.FileSystem;
29 import org.apache.hadoop.fs.FileUtil;
30 import org.apache.hadoop.fs.Path;
31 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
32 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
33 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
34
35 /**
36 * Copy over each of the files in a region's recovered.edits directory to the region's snapshot
37 * directory.
38 * <p>
39 * This is a serial operation over each of the files in the recovered.edits directory and also
40 * streams all the bytes to the client and then back to the filesystem, so the files being copied
41 * should be <b>small</b> or it will (a) suck up a lot of bandwidth, and (b) take a long time.
42 */
43 @InterfaceAudience.Private
44 @InterfaceStability.Evolving
45 public class CopyRecoveredEditsTask extends SnapshotTask {
46
47 private static final Log LOG = LogFactory.getLog(CopyRecoveredEditsTask.class);
48 private final FileSystem fs;
49 private final Path regiondir;
50 private final Path outputDir;
51
52 /**
53 * @param snapshot Snapshot being taken
54 * @param monitor error monitor for the snapshot
55 * @param fs {@link FileSystem} where the snapshot is being taken
56 * @param regionDir directory for the region to examine for edits
57 * @param snapshotRegionDir directory for the region in the snapshot
58 */
59 public CopyRecoveredEditsTask(SnapshotDescription snapshot, ForeignExceptionDispatcher monitor,
60 FileSystem fs, Path regionDir, Path snapshotRegionDir) {
61 super(snapshot, monitor);
62 this.fs = fs;
63 this.regiondir = regionDir;
64 this.outputDir = HLogUtil.getRegionDirRecoveredEditsDir(snapshotRegionDir);
65 }
66
67 @Override
68 public Void call() throws IOException {
69 NavigableSet<Path> files = HLogUtil.getSplitEditFilesSorted(this.fs, regiondir);
70 if (files == null || files.size() == 0) return null;
71
72 // copy over each file.
73 // this is really inefficient (could be trivially parallelized), but is
74 // really simple to reason about.
75 for (Path source : files) {
76 // check to see if the file is zero length, in which case we can skip it
77 FileStatus stat = fs.getFileStatus(source);
78 if (stat.getLen() <= 0) continue;
79
80 // its not zero length, so copy over the file
81 Path out = new Path(outputDir, source.getName());
82 LOG.debug("Copying " + source + " to " + out);
83 FileUtil.copy(fs, source, fs, out, true, fs.getConf());
84
85 // check for errors to the running operation after each file
86 this.rethrowException();
87 }
88 return null;
89 }
90 }