001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.snapshot;
020
021import java.io.IOException;
022import java.io.InterruptedIOException;
023import java.util.ArrayList;
024import java.util.Collection;
025import java.util.List;
026import java.util.concurrent.Callable;
027import java.util.concurrent.ExecutionException;
028import java.util.concurrent.Executor;
029import java.util.concurrent.ExecutorCompletionService;
030
031import org.apache.hadoop.conf.Configuration;
032import org.apache.hadoop.fs.FileStatus;
033import org.apache.hadoop.fs.FileSystem;
034import org.apache.hadoop.fs.Path;
035import org.apache.hadoop.hbase.client.RegionInfo;
036import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
037import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
038import org.apache.hadoop.hbase.util.Bytes;
039import org.apache.hadoop.hbase.util.FSUtils;
040import org.apache.yetus.audience.InterfaceAudience;
041import org.slf4j.Logger;
042import org.slf4j.LoggerFactory;
043import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
044import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
045import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
046import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
047
048/**
049 * DO NOT USE DIRECTLY. USE {@link SnapshotManifest}.
050 *
051 * Snapshot v1 layout format
052 *  - Each region in the table is represented by a directory with the .hregioninfo file
053 *      /snapshotName/regionName/.hregioninfo
054 *  - Each file present in the table is represented by an empty file
055 *      /snapshotName/regionName/familyName/fileName
056 */
057@InterfaceAudience.Private
058public final class SnapshotManifestV1 {
059  private static final Logger LOG = LoggerFactory.getLogger(SnapshotManifestV1.class);
060
061  public static final int DESCRIPTOR_VERSION = 0;
062
063  private SnapshotManifestV1() {
064  }
065
066  static class ManifestBuilder implements SnapshotManifest.RegionVisitor<
067                                                          HRegionFileSystem, Path> {
068    private final Configuration conf;
069    private final Path snapshotDir;
070    private final FileSystem fs;
071
072    public ManifestBuilder(final Configuration conf, final FileSystem fs, final Path snapshotDir) {
073      this.snapshotDir = snapshotDir;
074      this.conf = conf;
075      this.fs = fs;
076    }
077
078    @Override
079    public HRegionFileSystem regionOpen(final RegionInfo regionInfo) throws IOException {
080      HRegionFileSystem snapshotRegionFs = HRegionFileSystem.createRegionOnFileSystem(conf,
081        fs, snapshotDir, regionInfo);
082      return snapshotRegionFs;
083    }
084
085    @Override
086    public void regionClose(final HRegionFileSystem region) {
087    }
088
089    @Override
090    public Path familyOpen(final HRegionFileSystem snapshotRegionFs, final byte[] familyName) {
091      Path familyDir = snapshotRegionFs.getStoreDir(Bytes.toString(familyName));
092      return familyDir;
093    }
094
095    @Override
096    public void familyClose(final HRegionFileSystem region, final Path family) {
097    }
098
099    @Override
100    public void storeFile(final HRegionFileSystem region, final Path familyDir,
101        final StoreFileInfo storeFile) throws IOException {
102      Path referenceFile = new Path(familyDir, storeFile.getPath().getName());
103      boolean success = true;
104      if (storeFile.isReference()) {
105        // write the Reference object to the snapshot
106        storeFile.getReference().write(fs, referenceFile);
107      } else {
108        // create "reference" to this store file.  It is intentionally an empty file -- all
109        // necessary information is captured by its fs location and filename.  This allows us to
110        // only figure out what needs to be done via a single nn operation (instead of having to
111        // open and read the files as well).
112        success = fs.createNewFile(referenceFile);
113      }
114      if (!success) {
115        throw new IOException("Failed to create reference file:" + referenceFile);
116      }
117    }
118  }
119
120  static List<SnapshotRegionManifest> loadRegionManifests(final Configuration conf,
121      final Executor executor,final FileSystem fs, final Path snapshotDir,
122      final SnapshotDescription desc) throws IOException {
123    FileStatus[] regions = FSUtils.listStatus(fs, snapshotDir, new FSUtils.RegionDirFilter(fs));
124    if (regions == null) {
125      LOG.debug("No regions under directory:" + snapshotDir);
126      return null;
127    }
128
129    final ExecutorCompletionService<SnapshotRegionManifest> completionService =
130      new ExecutorCompletionService<>(executor);
131    for (final FileStatus region: regions) {
132      completionService.submit(new Callable<SnapshotRegionManifest>() {
133        @Override
134        public SnapshotRegionManifest call() throws IOException {
135          RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, region.getPath());
136          return buildManifestFromDisk(conf, fs, snapshotDir, hri);
137        }
138      });
139    }
140
141    ArrayList<SnapshotRegionManifest> regionsManifest = new ArrayList<>(regions.length);
142    try {
143      for (int i = 0; i < regions.length; ++i) {
144        regionsManifest.add(completionService.take().get());
145      }
146    } catch (InterruptedException e) {
147      throw new InterruptedIOException(e.getMessage());
148    } catch (ExecutionException e) {
149      IOException ex = new IOException();
150      ex.initCause(e.getCause());
151      throw ex;
152    }
153    return regionsManifest;
154  }
155
156  static void deleteRegionManifest(final FileSystem fs, final Path snapshotDir,
157      final SnapshotRegionManifest manifest) throws IOException {
158    String regionName = SnapshotManifest.getRegionNameFromManifest(manifest);
159    fs.delete(new Path(snapshotDir, regionName), true);
160  }
161
162  static SnapshotRegionManifest buildManifestFromDisk(final Configuration conf,
163      final FileSystem fs, final Path tableDir, final RegionInfo regionInfo) throws IOException {
164    HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, fs,
165          tableDir, regionInfo, true);
166    SnapshotRegionManifest.Builder manifest = SnapshotRegionManifest.newBuilder();
167
168    // 1. dump region meta info into the snapshot directory
169    LOG.debug("Storing region-info for snapshot.");
170    manifest.setRegionInfo(ProtobufUtil.toRegionInfo(regionInfo));
171
172    // 2. iterate through all the stores in the region
173    LOG.debug("Creating references for hfiles");
174
175    // This ensures that we have an atomic view of the directory as long as we have < ls limit
176    // (batch size of the files in a directory) on the namenode. Otherwise, we get back the files in
177    // batches and may miss files being added/deleted. This could be more robust (iteratively
178    // checking to see if we have all the files until we are sure), but the limit is currently 1000
179    // files/batch, far more than the number of store files under a single column family.
180    Collection<String> familyNames = regionFs.getFamilies();
181    if (familyNames != null) {
182      for (String familyName: familyNames) {
183        Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(familyName, false);
184        if (storeFiles == null) {
185          LOG.debug("No files under family: " + familyName);
186          continue;
187        }
188
189        // 2.1. build the snapshot reference for the store
190        SnapshotRegionManifest.FamilyFiles.Builder family =
191              SnapshotRegionManifest.FamilyFiles.newBuilder();
192        family.setFamilyName(UnsafeByteOperations.unsafeWrap(Bytes.toBytes(familyName)));
193
194        if (LOG.isDebugEnabled()) {
195          LOG.debug("Adding snapshot references for " + storeFiles  + " hfiles");
196        }
197
198        // 2.2. iterate through all the store's files and create "references".
199        int i = 0;
200        int sz = storeFiles.size();
201        for (StoreFileInfo storeFile: storeFiles) {
202          // create "reference" to this store file.
203          LOG.debug("Adding reference for file ("+ (++i) +"/" + sz + "): " + storeFile.getPath());
204          SnapshotRegionManifest.StoreFile.Builder sfManifest =
205                SnapshotRegionManifest.StoreFile.newBuilder();
206          sfManifest.setName(storeFile.getPath().getName());
207          family.addStoreFiles(sfManifest.build());
208        }
209        manifest.addFamilyFiles(family.build());
210      }
211    }
212    return manifest.build();
213  }
214}