View Javadoc

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.master.cleaner;
19  
20  import static org.junit.Assert.assertEquals;
21  import static org.junit.Assert.assertTrue;
22  import static org.junit.Assert.fail;
23  
24  import java.io.IOException;
25  import java.util.ArrayList;
26  import java.util.Collection;
27  import java.util.Collections;
28  import java.util.List;
29  import java.util.Set;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.fs.FileStatus;
35  import org.apache.hadoop.fs.FileSystem;
36  import org.apache.hadoop.fs.Path;
37  import org.apache.hadoop.hbase.HBaseTestingUtility;
38  import org.apache.hadoop.hbase.HConstants;
39  import org.apache.hadoop.hbase.HTableDescriptor;
40  import org.apache.hadoop.hbase.MediumTests;
41  import org.apache.hadoop.hbase.TableName;
42  import org.apache.hadoop.hbase.client.HBaseAdmin;
43  import org.apache.hadoop.hbase.client.HTable;
44  import org.apache.hadoop.hbase.master.HMaster;
45  import org.apache.hadoop.hbase.master.snapshot.DisabledTableSnapshotHandler;
46  import org.apache.hadoop.hbase.master.snapshot.SnapshotHFileCleaner;
47  import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
48  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
49  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
50  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
51  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
52  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
53  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
54  import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
55  import org.apache.hadoop.hbase.regionserver.HRegion;
56  import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
57  import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
58  import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
59  import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
60  import org.apache.hadoop.hbase.util.Bytes;
61  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
62  import org.apache.hadoop.hbase.util.FSUtils;
63  import org.junit.After;
64  import org.junit.AfterClass;
65  import org.junit.Before;
66  import org.junit.BeforeClass;
67  import org.junit.Test;
68  import org.junit.experimental.categories.Category;
69  import org.mockito.Mockito;
70  
71  import com.google.common.collect.Lists;
72  import com.google.protobuf.ServiceException;
73  
74  /**
75   * Test the master-related aspects of a snapshot
76   */
77  @Category(MediumTests.class)
78  public class TestSnapshotFromMaster {
79  
80    private static final Log LOG = LogFactory.getLog(TestSnapshotFromMaster.class);
81    private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
82    private static final int NUM_RS = 2;
83    private static Path rootDir;
84    private static Path snapshots;
85    private static FileSystem fs;
86    private static HMaster master;
87  
88    // for hfile archiving test.
89    private static Path archiveDir;
90    private static final byte[] TEST_FAM = Bytes.toBytes("fam");
91    private static final TableName TABLE_NAME =
92        TableName.valueOf("test");
93    // refresh the cache every 1/2 second
94    private static final long cacheRefreshPeriod = 500;
95  
96    /**
97     * Setup the config for the cluster
98     */
99    @BeforeClass
100   public static void setupCluster() throws Exception {
101     setupConf(UTIL.getConfiguration());
102     UTIL.startMiniCluster(NUM_RS);
103     fs = UTIL.getDFSCluster().getFileSystem();
104     master = UTIL.getMiniHBaseCluster().getMaster();
105     rootDir = master.getMasterFileSystem().getRootDir();
106     snapshots = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
107     archiveDir = new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY);
108   }
109 
110   private static void setupConf(Configuration conf) {
111     // disable the ui
112     conf.setInt("hbase.regionsever.info.port", -1);
113     // change the flush size to a small amount, regulating number of store files
114     conf.setInt("hbase.hregion.memstore.flush.size", 25000);
115     // so make sure we get a compaction when doing a load, but keep around some
116     // files in the store
117     conf.setInt("hbase.hstore.compaction.min", 3);
118     conf.setInt("hbase.hstore.compactionThreshold", 5);
119     // block writes if we get to 12 store files
120     conf.setInt("hbase.hstore.blockingStoreFiles", 12);
121     // Ensure no extra cleaners on by default (e.g. TimeToLiveHFileCleaner)
122     conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, "");
123     conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, "");
124     // Enable snapshot
125     conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
126     conf.setLong(SnapshotHFileCleaner.HFILE_CACHE_REFRESH_PERIOD_CONF_KEY, cacheRefreshPeriod);
127     conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
128       ConstantSizeRegionSplitPolicy.class.getName());
129 
130   }
131 
132   @Before
133   public void setup() throws Exception {
134     UTIL.createTable(TABLE_NAME, TEST_FAM);
135     master.getSnapshotManagerForTesting().setSnapshotHandlerForTesting(TABLE_NAME, null);
136   }
137 
138   @After
139   public void tearDown() throws Exception {
140     UTIL.deleteTable(TABLE_NAME);
141     SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
142     SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
143   }
144 
145   @AfterClass
146   public static void cleanupTest() throws Exception {
147     try {
148       UTIL.shutdownMiniCluster();
149     } catch (Exception e) {
150       // NOOP;
151     }
152   }
153 
154   /**
155    * Test that the contract from the master for checking on a snapshot are valid.
156    * <p>
157    * <ol>
158    * <li>If a snapshot fails with an error, we expect to get the source error.</li>
159    * <li>If there is no snapshot name supplied, we should get an error.</li>
160    * <li>If asking about a snapshot has hasn't occurred, you should get an error.</li>
161    * </ol>
162    */
163   @Test(timeout = 300000)
164   public void testIsDoneContract() throws Exception {
165 
166     IsSnapshotDoneRequest.Builder builder = IsSnapshotDoneRequest.newBuilder();
167 
168     String snapshotName = "asyncExpectedFailureTest";
169 
170     // check that we get an exception when looking up snapshot where one hasn't happened
171     SnapshotTestingUtils.expectSnapshotDoneException(master, builder.build(),
172       UnknownSnapshotException.class);
173 
174     // and that we get the same issue, even if we specify a name
175     SnapshotDescription desc = SnapshotDescription.newBuilder()
176       .setName(snapshotName).setTable(TABLE_NAME.getNameAsString()).build();
177     builder.setSnapshot(desc);
178     SnapshotTestingUtils.expectSnapshotDoneException(master, builder.build(),
179       UnknownSnapshotException.class);
180 
181     // set a mock handler to simulate a snapshot
182     DisabledTableSnapshotHandler mockHandler = Mockito.mock(DisabledTableSnapshotHandler.class);
183     Mockito.when(mockHandler.getException()).thenReturn(null);
184     Mockito.when(mockHandler.getSnapshot()).thenReturn(desc);
185     Mockito.when(mockHandler.isFinished()).thenReturn(new Boolean(true));
186     Mockito.when(mockHandler.getCompletionTimestamp())
187       .thenReturn(EnvironmentEdgeManager.currentTimeMillis());
188 
189     master.getSnapshotManagerForTesting()
190         .setSnapshotHandlerForTesting(TABLE_NAME, mockHandler);
191 
192     // if we do a lookup without a snapshot name, we should fail - you should always know your name
193     builder = IsSnapshotDoneRequest.newBuilder();
194     SnapshotTestingUtils.expectSnapshotDoneException(master, builder.build(),
195       UnknownSnapshotException.class);
196 
197     // then do the lookup for the snapshot that it is done
198     builder.setSnapshot(desc);
199     IsSnapshotDoneResponse response = master.isSnapshotDone(null, builder.build());
200     assertTrue("Snapshot didn't complete when it should have.", response.getDone());
201 
202     // now try the case where we are looking for a snapshot we didn't take
203     builder.setSnapshot(SnapshotDescription.newBuilder().setName("Not A Snapshot").build());
204     SnapshotTestingUtils.expectSnapshotDoneException(master, builder.build(),
205       UnknownSnapshotException.class);
206 
207     // then create a snapshot to the fs and make sure that we can find it when checking done
208     snapshotName = "completed";
209     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
210     desc = desc.toBuilder().setName(snapshotName).build();
211     SnapshotDescriptionUtils.writeSnapshotInfo(desc, snapshotDir, fs);
212 
213     builder.setSnapshot(desc);
214     response = master.isSnapshotDone(null, builder.build());
215     assertTrue("Completed, on-disk snapshot not found", response.getDone());
216   }
217 
218   @Test(timeout = 300000)
219   public void testGetCompletedSnapshots() throws Exception {
220     // first check when there are no snapshots
221     GetCompletedSnapshotsRequest request = GetCompletedSnapshotsRequest.newBuilder().build();
222     GetCompletedSnapshotsResponse response = master.getCompletedSnapshots(null, request);
223     assertEquals("Found unexpected number of snapshots", 0, response.getSnapshotsCount());
224 
225     // write one snapshot to the fs
226     String snapshotName = "completed";
227     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
228     SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName(snapshotName).build();
229     SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, snapshotDir, fs);
230 
231     // check that we get one snapshot
232     response = master.getCompletedSnapshots(null, request);
233     assertEquals("Found unexpected number of snapshots", 1, response.getSnapshotsCount());
234     List<SnapshotDescription> snapshots = response.getSnapshotsList();
235     List<SnapshotDescription> expected = Lists.newArrayList(snapshot);
236     assertEquals("Returned snapshots don't match created snapshots", expected, snapshots);
237 
238     // write a second snapshot
239     snapshotName = "completed_two";
240     snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
241     snapshot = SnapshotDescription.newBuilder().setName(snapshotName).build();
242     SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, snapshotDir, fs);
243     expected.add(snapshot);
244 
245     // check that we get one snapshot
246     response = master.getCompletedSnapshots(null, request);
247     assertEquals("Found unexpected number of snapshots", 2, response.getSnapshotsCount());
248     snapshots = response.getSnapshotsList();
249     assertEquals("Returned snapshots don't match created snapshots", expected, snapshots);
250   }
251 
252   @Test(timeout = 300000)
253   public void testDeleteSnapshot() throws Exception {
254 
255     String snapshotName = "completed";
256     SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName(snapshotName).build();
257 
258     DeleteSnapshotRequest request = DeleteSnapshotRequest.newBuilder().setSnapshot(snapshot)
259         .build();
260     try {
261       master.deleteSnapshot(null, request);
262       fail("Master didn't throw exception when attempting to delete snapshot that doesn't exist");
263     } catch (ServiceException e) {
264       LOG.debug("Correctly failed delete of non-existant snapshot:" + e.getMessage());
265     }
266 
267     // write one snapshot to the fs
268     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
269     SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, snapshotDir, fs);
270 
271     // then delete the existing snapshot,which shouldn't cause an exception to be thrown
272     master.deleteSnapshot(null, request);
273   }
274 
275   /**
276    * Test that the snapshot hfile archive cleaner works correctly. HFiles that are in snapshots
277    * should be retained, while those that are not in a snapshot should be deleted.
278    * @throws Exception on failure
279    */
280   @Test(timeout = 300000)
281   public void testSnapshotHFileArchiving() throws Exception {
282     HBaseAdmin admin = UTIL.getHBaseAdmin();
283     // make sure we don't fail on listing snapshots
284     SnapshotTestingUtils.assertNoSnapshots(admin);
285 
286     // recreate test table with disabled compactions; otherwise compaction may happen before
287     // snapshot, the call after snapshot will be a no-op and checks will fail
288     UTIL.deleteTable(TABLE_NAME);
289     HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
290     htd.setCompactionEnabled(false);
291     UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
292     // load the table (creates 4 hfiles)
293     UTIL.loadTable(new HTable(UTIL.getConfiguration(), TABLE_NAME), TEST_FAM);
294     UTIL.flush(TABLE_NAME);
295     // Put some more data into the table so for sure we get more storefiles.
296     UTIL.loadTable(new HTable(UTIL.getConfiguration(), TABLE_NAME), TEST_FAM);
297 
298     // disable the table so we can take a snapshot
299     admin.disableTable(TABLE_NAME);
300     htd.setCompactionEnabled(true);
301 
302     // take a snapshot of the table
303     String snapshotName = "snapshot";
304     byte[] snapshotNameBytes = Bytes.toBytes(snapshotName);
305     admin.snapshot(snapshotNameBytes, TABLE_NAME);
306 
307     LOG.info("After snapshot File-System state");
308     FSUtils.logFileSystemState(fs, rootDir, LOG);
309 
310     // ensure we only have one snapshot
311     SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, snapshotNameBytes, TABLE_NAME);
312 
313     // enable compactions now
314     admin.modifyTable(TABLE_NAME, htd);
315 
316     // renable the table so we can compact the regions
317     admin.enableTable(TABLE_NAME);
318 
319     // compact the files so we get some archived files for the table we just snapshotted
320     List<HRegion> regions = UTIL.getHBaseCluster().getRegions(TABLE_NAME);
321     for (HRegion region : regions) {
322       region.waitForFlushesAndCompactions(); // enable can trigger a compaction, wait for it.
323       region.compactStores(); // min is 3 so will compact and archive
324     }
325     LOG.info("After compaction File-System state");
326     FSUtils.logFileSystemState(fs, rootDir, LOG);
327 
328     // make sure the cleaner has run
329     LOG.debug("Running hfile cleaners");
330     ensureHFileCleanersRun();
331     LOG.info("After cleaners File-System state: " + rootDir);
332     FSUtils.logFileSystemState(fs, rootDir, LOG);
333 
334     // get the snapshot files for the table
335     Path snapshotTable = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
336     Set<String> snapshotHFiles = SnapshotReferenceUtil.getHFileNames(
337         UTIL.getConfiguration(), fs, snapshotTable);
338     // check that the files in the archive contain the ones that we need for the snapshot
339     LOG.debug("Have snapshot hfiles:");
340     for (String fileName : snapshotHFiles) {
341       LOG.debug(fileName);
342     }
343     // get the archived files for the table
344     Collection<String> files = getArchivedHFiles(archiveDir, rootDir, fs, TABLE_NAME);
345 
346     // and make sure that there is a proper subset
347     for (String fileName : snapshotHFiles) {
348       assertTrue("Archived hfiles " + files + " is missing snapshot file:" + fileName,
349         files.contains(fileName));
350     }
351 
352     // delete the existing snapshot
353     admin.deleteSnapshot(snapshotNameBytes);
354     SnapshotTestingUtils.assertNoSnapshots(admin);
355 
356     // make sure that we don't keep around the hfiles that aren't in a snapshot
357     // make sure we wait long enough to refresh the snapshot hfile
358     List<BaseHFileCleanerDelegate> delegates = UTIL.getMiniHBaseCluster().getMaster()
359         .getHFileCleaner().cleanersChain;
360     for (BaseHFileCleanerDelegate delegate: delegates) {
361       if (delegate instanceof SnapshotHFileCleaner) {
362         ((SnapshotHFileCleaner)delegate).getFileCacheForTesting().triggerCacheRefreshForTesting();
363       }
364     }
365     // run the cleaner again
366     LOG.debug("Running hfile cleaners");
367     ensureHFileCleanersRun();
368     LOG.info("After delete snapshot cleaners run File-System state");
369     FSUtils.logFileSystemState(fs, rootDir, LOG);
370 
371     files = getArchivedHFiles(archiveDir, rootDir, fs, TABLE_NAME);
372     assertEquals("Still have some hfiles in the archive, when their snapshot has been deleted.", 0,
373       files.size());
374   }
375 
376   /**
377    * @return all the HFiles for a given table that have been archived
378    * @throws IOException on expected failure
379    */
380   private final Collection<String> getArchivedHFiles(Path archiveDir, Path rootDir,
381       FileSystem fs, TableName tableName) throws IOException {
382     Path tableArchive = FSUtils.getTableDir(archiveDir, tableName);
383     Path[] archivedHFiles = SnapshotTestingUtils.listHFiles(fs, tableArchive);
384     List<String> files = new ArrayList<String>(archivedHFiles.length);
385     LOG.debug("Have archived hfiles: " + tableArchive);
386     for (Path file : archivedHFiles) {
387       LOG.debug(file);
388       files.add(file.getName());
389     }
390     // sort the archived files
391 
392     Collections.sort(files);
393     return files;
394   }
395 
396   /**
397    * Make sure the {@link HFileCleaner HFileCleaners} run at least once
398    */
399   private static void ensureHFileCleanersRun() {
400     UTIL.getHBaseCluster().getMaster().getHFileCleaner().chore();
401   }
402 }