1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.master;
21  
22  import static org.apache.hadoop.hbase.util.HFileArchiveTestingUtil.assertArchiveEqualToOriginal;
23  import static org.junit.Assert.assertEquals;
24  import static org.junit.Assert.assertFalse;
25  import static org.junit.Assert.assertNotNull;
26  import static org.junit.Assert.assertNull;
27  import static org.junit.Assert.assertTrue;
28  import static org.mockito.Mockito.doReturn;
29  import static org.mockito.Mockito.spy;
30  
31  import java.io.IOException;
32  import java.util.ArrayList;
33  import java.util.List;
34  import java.util.Map;
35  import java.util.SortedMap;
36  import java.util.TreeMap;
37  
38  import org.apache.hadoop.conf.Configuration;
39  import org.apache.hadoop.fs.FSDataOutputStream;
40  import org.apache.hadoop.fs.FileStatus;
41  import org.apache.hadoop.fs.FileSystem;
42  import org.apache.hadoop.fs.Path;
43  import org.apache.hadoop.hbase.HBaseTestingUtility;
44  import org.apache.hadoop.hbase.HColumnDescriptor;
45  import org.apache.hadoop.hbase.HConstants;
46  import org.apache.hadoop.hbase.HRegionInfo;
47  import org.apache.hadoop.hbase.HTableDescriptor;
48  import org.apache.hadoop.hbase.KeyValue;
49  import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
50  import org.apache.hadoop.hbase.Server;
51  import org.apache.hadoop.hbase.ServerName;
52  import org.apache.hadoop.hbase.SmallTests;
53  import org.apache.hadoop.hbase.TableDescriptors;
54  import org.apache.hadoop.hbase.catalog.CatalogTracker;
55  import org.apache.hadoop.hbase.client.HConnection;
56  import org.apache.hadoop.hbase.client.HConnectionManager;
57  import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
58  import org.apache.hadoop.hbase.client.Result;
59  import org.apache.hadoop.hbase.executor.ExecutorService;
60  import org.apache.hadoop.hbase.io.Reference;
61  import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
62  import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
63  import org.apache.hadoop.hbase.ipc.HRegionInterface;
64  import org.apache.hadoop.hbase.regionserver.Store;
65  import org.apache.hadoop.hbase.util.Bytes;
66  import org.apache.hadoop.hbase.util.FSUtils;
67  import org.apache.hadoop.hbase.util.HFileArchiveUtil;
68  import org.apache.hadoop.hbase.util.Pair;
69  import org.apache.hadoop.hbase.util.Writables;
70  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
71  import org.junit.Test;
72  import org.junit.experimental.categories.Category;
73  import org.mockito.Mockito;
74  
75  @Category(SmallTests.class)
76  public class TestCatalogJanitor {
77    /**
78     * Pseudo server for below tests.
79     * Be sure to call stop on the way out else could leave some mess around.
80     */
81    class MockServer implements Server {
82      private final HConnection connection;
83      private final Configuration c;
84      private final CatalogTracker ct;
85  
86      MockServer(final HBaseTestingUtility htu)
87      throws NotAllMetaRegionsOnlineException, IOException, InterruptedException {
88        this.c = htu.getConfiguration();
89        // Mock an HConnection and a HRegionInterface implementation.  Have the
90        // HConnection return the HRI.  Have the HRI return a few mocked up responses
91        // to make our test work.
92        this.connection =
93          HConnectionTestingUtility.getMockedConnectionAndDecorate(this.c,
94            Mockito.mock(HRegionInterface.class),
95            new ServerName("example.org,12345,6789"),
96            HRegionInfo.FIRST_META_REGIONINFO);
97        // Set hbase.rootdir into test dir.
98        FileSystem fs = FileSystem.get(this.c);
99        Path rootdir = fs.makeQualified(new Path(this.c.get(HConstants.HBASE_DIR)));
100       this.c.set(HConstants.HBASE_DIR, rootdir.toString());
101       this.ct = Mockito.mock(CatalogTracker.class);
102       HRegionInterface hri = Mockito.mock(HRegionInterface.class);
103       Mockito.when(this.ct.getConnection()).thenReturn(this.connection);
104       Mockito.when(ct.waitForMetaServerConnection(Mockito.anyLong())).thenReturn(hri);
105     }
106 
107     @Override
108     public CatalogTracker getCatalogTracker() {
109       return this.ct;
110     }
111 
112     @Override
113     public Configuration getConfiguration() {
114       return this.c;
115     }
116 
117     @Override
118     public ServerName getServerName() {
119       return new ServerName("mockserver.example.org", 1234, -1L);
120     }
121 
122     @Override
123     public ZooKeeperWatcher getZooKeeper() {
124       return null;
125     }
126 
127     @Override
128     public void abort(String why, Throwable e) {
129       //no-op
130     }
131 
132     @Override
133     public boolean isAborted() {
134       return false;
135     }
136 
137     @Override
138     public boolean isStopped() {
139       return false;
140     }
141 
142     @Override
143     public void stop(String why) {
144       if (this.ct != null) {
145         this.ct.stop();
146       }
147       if (this.connection != null) {
148         HConnectionManager.deleteConnection(this.connection.getConfiguration());
149       }
150     }
151   }
152 
153   /**
154    * Mock MasterServices for tests below.
155    */
156   class MockMasterServices implements MasterServices {
157     private final MasterFileSystem mfs;
158     private final AssignmentManager asm;
159 
160     MockMasterServices(final Server server) throws IOException {
161       this.mfs = new MasterFileSystem(server, this, null, false);
162       this.asm = Mockito.mock(AssignmentManager.class);
163     }
164 
165     @Override
166     public void checkTableModifiable(byte[] tableName) throws IOException {
167       //no-op
168     }
169 
170     @Override
171     public void createTable(HTableDescriptor desc, byte[][] splitKeys)
172         throws IOException {
173       // no-op
174     }
175 
176     @Override
177     public AssignmentManager getAssignmentManager() {
178       return this.asm;
179     }
180 
181     @Override
182     public ExecutorService getExecutorService() {
183       return null;
184     }
185 
186     @Override
187     public MasterFileSystem getMasterFileSystem() {
188       return this.mfs;
189     }
190 
191     @Override
192     public ServerManager getServerManager() {
193       return null;
194     }
195 
196     @Override
197     public ZooKeeperWatcher getZooKeeper() {
198       return null;
199     }
200 
201     @Override
202     public CatalogTracker getCatalogTracker() {
203       return null;
204     }
205 
206     @Override
207     public Configuration getConfiguration() {
208       return mfs.conf;
209     }
210 
211     @Override
212     public ServerName getServerName() {
213       return null;
214     }
215 
216     @Override
217     public void abort(String why, Throwable e) {
218       //no-op
219     }
220 
221     @Override
222     public boolean isAborted() {
223       return false;
224     }
225 
226     private boolean stopped = false;
227 
228     @Override
229     public void stop(String why) {
230       stopped = true;
231     }
232 
233     @Override
234     public boolean isStopped() {
235       return stopped;
236     }
237 
238     @Override
239     public TableDescriptors getTableDescriptors() {
240       return new TableDescriptors() {
241         @Override
242         public HTableDescriptor remove(String tablename) throws IOException {
243           return null;
244         }
245         
246         @Override
247         public Map<String, HTableDescriptor> getAll() throws IOException {
248           return null;
249         }
250         
251         @Override
252         public HTableDescriptor get(byte[] tablename)
253         throws IOException {
254           return get(Bytes.toString(tablename));
255         }
256         
257         @Override
258         public HTableDescriptor get(String tablename)
259         throws IOException {
260           return createHTableDescriptor();
261         }
262         
263         @Override
264         public void add(HTableDescriptor htd) throws IOException {
265         }
266       };
267     }
268 
269     @Override
270     public boolean isServerShutdownHandlerEnabled() {
271       return true;
272     }
273 
274     @Override
275     public MasterCoprocessorHost getCoprocessorHost() {
276       return null;
277     }
278 
279     @Override
280     public <T extends CoprocessorProtocol> boolean registerProtocol(Class<T> protocol, T handler) {
281       return false;
282     }
283 
284     @Override
285     public void deleteTable(byte[] tableName) throws IOException {
286     }
287 
288     @Override
289     public void modifyTable(byte[] tableName, HTableDescriptor descriptor) throws IOException {
290     }
291 
292     @Override
293     public void enableTable(byte[] tableName) throws IOException {
294     }
295 
296     @Override
297     public void disableTable(byte[] tableName) throws IOException {
298     }
299 
300     @Override
301     public void addColumn(byte[] tableName, HColumnDescriptor column) throws IOException {
302     }
303 
304     @Override
305     public void modifyColumn(byte[] tableName, HColumnDescriptor descriptor) throws IOException {
306     }
307 
308     @Override
309     public void deleteColumn(byte[] tableName, byte[] columnName) throws IOException {
310     }
311 
312     @Override
313     public boolean shouldSplitMetaSeparately() {
314       return false;
315     }
316   }
317 
318   @Test
319   public void testGetHRegionInfo() throws IOException {
320     assertNull(CatalogJanitor.getHRegionInfo(new Result()));
321     List<KeyValue> kvs = new ArrayList<KeyValue>();
322     Result r = new Result(kvs);
323     assertNull(CatalogJanitor.getHRegionInfo(r));
324     byte [] f = HConstants.CATALOG_FAMILY;
325     // Make a key value that doesn't have the expected qualifier.
326     kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
327       HConstants.SERVER_QUALIFIER, f));
328     r = new Result(kvs);
329     assertNull(CatalogJanitor.getHRegionInfo(r));
330     // Make a key that does not have a regioninfo value.
331     kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
332       HConstants.REGIONINFO_QUALIFIER, f));
333     HRegionInfo hri = CatalogJanitor.getHRegionInfo(new Result(kvs));
334     assertTrue(hri == null);
335     // OK, give it what it expects
336     kvs.clear();
337     kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f,
338       HConstants.REGIONINFO_QUALIFIER,
339       Writables.getBytes(HRegionInfo.FIRST_META_REGIONINFO)));
340     hri = CatalogJanitor.getHRegionInfo(new Result(kvs));
341     assertNotNull(hri);
342     assertTrue(hri.equals(HRegionInfo.FIRST_META_REGIONINFO));
343   }
344 
345   @Test
346   public void testCleanParent() throws IOException, InterruptedException {
347     HBaseTestingUtility htu = new HBaseTestingUtility();
348     setRootDirAndCleanIt(htu, "testCleanParent");
349     Server server = new MockServer(htu);
350     try {
351       MasterServices services = new MockMasterServices(server);
352       CatalogJanitor janitor = new CatalogJanitor(server, services);
353       // Create regions.
354       HTableDescriptor htd = new HTableDescriptor("table");
355       htd.addFamily(new HColumnDescriptor("f"));
356       HRegionInfo parent =
357         new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
358             Bytes.toBytes("eee"));
359       HRegionInfo splita =
360         new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
361             Bytes.toBytes("ccc"));
362       HRegionInfo splitb =
363         new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
364             Bytes.toBytes("eee"));
365       // Test that when both daughter regions are in place, that we do not
366       // remove the parent.
367       List<KeyValue> kvs = new ArrayList<KeyValue>();
368       kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
369           HConstants.SPLITA_QUALIFIER, Writables.getBytes(splita)));
370       kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
371           HConstants.SPLITB_QUALIFIER, Writables.getBytes(splitb)));
372       Result r = new Result(kvs);
373       // Add a reference under splitA directory so we don't clear out the parent.
374       Path rootdir = services.getMasterFileSystem().getRootDir();
375       Path tabledir =
376         HTableDescriptor.getTableDir(rootdir, htd.getName());
377       Path storedir = Store.getStoreHomedir(tabledir, splita.getEncodedName(),
378           htd.getColumnFamilies()[0].getName());
379       Reference ref = new Reference(Bytes.toBytes("ccc"), Reference.Range.top);
380       long now = System.currentTimeMillis();
381       // Reference name has this format: StoreFile#REF_NAME_PARSER
382       Path p = new Path(storedir, Long.toString(now) + "." + parent.getEncodedName());
383       FileSystem fs = services.getMasterFileSystem().getFileSystem();
384       Path path = ref.write(fs, p);
385       assertTrue(fs.exists(path));
386       assertFalse(janitor.cleanParent(parent, r));
387       // Remove the reference file and try again.
388       assertTrue(fs.delete(p, true));
389       assertTrue(janitor.cleanParent(parent, r));
390     } finally {
391       server.stop("shutdown");
392     }
393   }
394 
395   /**
396    * Make sure parent gets cleaned up even if daughter is cleaned up before it.
397    * @throws IOException
398    * @throws InterruptedException
399    */
400   @Test
401   public void testParentCleanedEvenIfDaughterGoneFirst()
402   throws IOException, InterruptedException {
403     parentWithSpecifiedEndKeyCleanedEvenIfDaughterGoneFirst(
404       "testParentCleanedEvenIfDaughterGoneFirst", Bytes.toBytes("eee"));
405   }
406 
407   /**
408    * Make sure last parent with empty end key gets cleaned up even if daughter is cleaned up before it.
409    * @throws IOException
410    * @throws InterruptedException
411    */
412   @Test
413   public void testLastParentCleanedEvenIfDaughterGoneFirst()
414   throws IOException, InterruptedException {
415     parentWithSpecifiedEndKeyCleanedEvenIfDaughterGoneFirst(
416       "testLastParentCleanedEvenIfDaughterGoneFirst", new byte[0]);
417   }
418 
419   /**
420    * Make sure parent with specified end key gets cleaned up even if daughter is cleaned up before it.
421    *
422    * @param rootDir the test case name, used as the HBase testing utility root
423    * @param lastEndKey the end key of the split parent
424    * @throws IOException
425    * @throws InterruptedException
426    */
427   private void parentWithSpecifiedEndKeyCleanedEvenIfDaughterGoneFirst(
428   final String rootDir, final byte[] lastEndKey)
429   throws IOException, InterruptedException {
430     HBaseTestingUtility htu = new HBaseTestingUtility();
431     setRootDirAndCleanIt(htu, rootDir);
432     Server server = new MockServer(htu);
433     MasterServices services = new MockMasterServices(server);
434     CatalogJanitor janitor = new CatalogJanitor(server, services);
435     final HTableDescriptor htd = createHTableDescriptor();
436 
437     // Create regions: aaa->{lastEndKey}, aaa->ccc, aaa->bbb, bbb->ccc, etc.
438 
439     // Parent
440     HRegionInfo parent = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
441       lastEndKey);
442     // Sleep a second else the encoded name on these regions comes out
443     // same for all with same start key and made in same second.
444     Thread.sleep(1001);
445 
446     // Daughter a
447     HRegionInfo splita = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
448       Bytes.toBytes("ccc"));
449     Thread.sleep(1001);
450     // Make daughters of daughter a; splitaa and splitab.
451     HRegionInfo splitaa = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
452       Bytes.toBytes("bbb"));
453     HRegionInfo splitab = new HRegionInfo(htd.getName(), Bytes.toBytes("bbb"),
454       Bytes.toBytes("ccc"));
455 
456     // Daughter b
457     HRegionInfo splitb = new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
458       lastEndKey);
459     Thread.sleep(1001);
460     // Make Daughters of daughterb; splitba and splitbb.
461     HRegionInfo splitba = new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
462       Bytes.toBytes("ddd"));
463     HRegionInfo splitbb = new HRegionInfo(htd.getName(), Bytes.toBytes("ddd"),
464     lastEndKey);
465 
466     // First test that our Comparator works right up in CatalogJanitor.
467     // Just fo kicks.
468     SortedMap<HRegionInfo, Result> regions =
469       new TreeMap<HRegionInfo, Result>(new CatalogJanitor.SplitParentFirstComparator());
470     // Now make sure that this regions map sorts as we expect it to.
471     regions.put(parent, createResult(parent, splita, splitb));
472     regions.put(splitb, createResult(splitb, splitba, splitbb));
473     regions.put(splita, createResult(splita, splitaa, splitab));
474     // Assert its properly sorted.
475     int index = 0;
476     for (Map.Entry<HRegionInfo, Result> e: regions.entrySet()) {
477       if (index == 0) {
478         assertTrue(e.getKey().getEncodedName().equals(parent.getEncodedName()));
479       } else if (index == 1) {
480         assertTrue(e.getKey().getEncodedName().equals(splita.getEncodedName()));
481       } else if (index == 2) {
482         assertTrue(e.getKey().getEncodedName().equals(splitb.getEncodedName()));
483       }
484       index++;
485     }
486 
487     // Now play around with the cleanParent function.  Create a ref from splita
488     // up to the parent.
489     Path splitaRef =
490       createReferences(services, htd, parent, splita, Bytes.toBytes("ccc"), false);
491     // Make sure actual super parent sticks around because splita has a ref.
492     assertFalse(janitor.cleanParent(parent, regions.get(parent)));
493 
494     //splitba, and split bb, do not have dirs in fs.  That means that if
495     // we test splitb, it should get cleaned up.
496     assertTrue(janitor.cleanParent(splitb, regions.get(splitb)));
497 
498     // Now remove ref from splita to parent... so parent can be let go and so
499     // the daughter splita can be split (can't split if still references).
500     // BUT make the timing such that the daughter gets cleaned up before we
501     // can get a chance to let go of the parent.
502     FileSystem fs = FileSystem.get(htu.getConfiguration());
503     assertTrue(fs.delete(splitaRef, true));
504     // Create the refs from daughters of splita.
505     Path splitaaRef =
506       createReferences(services, htd, splita, splitaa, Bytes.toBytes("bbb"), false);
507     Path splitabRef =
508       createReferences(services, htd, splita, splitab, Bytes.toBytes("bbb"), true);
509 
510     // Test splita.  It should stick around because references from splitab, etc.
511     assertFalse(janitor.cleanParent(splita, regions.get(splita)));
512 
513     // Now clean up parent daughter first.  Remove references from its daughters.
514     assertTrue(fs.delete(splitaaRef, true));
515     assertTrue(fs.delete(splitabRef, true));
516     assertTrue(janitor.cleanParent(splita, regions.get(splita)));
517 
518     // Super parent should get cleaned up now both splita and splitb are gone.
519     assertTrue(janitor.cleanParent(parent, regions.get(parent)));
520 
521     services.stop("test finished");
522     janitor.join();
523   }
524 
525   /**
526    * CatalogJanitor.scan() should not clean parent regions if their own
527    * parents are still referencing them. This ensures that grandfather regions
528    * do not point to deleted parent regions.
529    */
530   @Test
531   public void testScanDoesNotCleanRegionsWithExistingParents() throws Exception {
532     HBaseTestingUtility htu = new HBaseTestingUtility();
533     setRootDirAndCleanIt(htu, "testScanDoesNotCleanRegionsWithExistingParents");
534     Server server = new MockServer(htu);
535     MasterServices services = new MockMasterServices(server);
536 
537     final HTableDescriptor htd = createHTableDescriptor();
538 
539     // Create regions: aaa->{lastEndKey}, aaa->ccc, aaa->bbb, bbb->ccc, etc.
540 
541     // Parent
542     HRegionInfo parent = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
543       new byte[0], true);
544     // Sleep a second else the encoded name on these regions comes out
545     // same for all with same start key and made in same second.
546     Thread.sleep(1001);
547 
548     // Daughter a
549     HRegionInfo splita = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
550       Bytes.toBytes("ccc"), true);
551     Thread.sleep(1001);
552     // Make daughters of daughter a; splitaa and splitab.
553     HRegionInfo splitaa = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"),
554       Bytes.toBytes("bbb"), false);
555     HRegionInfo splitab = new HRegionInfo(htd.getName(), Bytes.toBytes("bbb"),
556       Bytes.toBytes("ccc"), false);
557 
558     // Daughter b
559     HRegionInfo splitb = new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"),
560         new byte[0]);
561     Thread.sleep(1001);
562 
563     final Map<HRegionInfo, Result> splitParents =
564         new TreeMap<HRegionInfo, Result>(new SplitParentFirstComparator());
565     splitParents.put(parent, makeResultFromHRegionInfo(parent, splita, splitb));
566     splita.setOffline(true);//simulate that splita goes offline when it is split
567     splitParents.put(splita, makeResultFromHRegionInfo(splita, splitaa, splitab));
568 
569     CatalogJanitor janitor = spy(new CatalogJanitor(server, services));
570     doReturn(new Pair<Integer, Map<HRegionInfo, Result>>(
571         10, splitParents)).when(janitor).getSplitParents();
572 
573     //create ref from splita to parent
574     Path splitaRef =
575         createReferences(services, htd, parent, splita, Bytes.toBytes("ccc"), false);
576 
577     //parent and A should not be removed
578     assertEquals(0, janitor.scan());
579 
580     //now delete the ref
581     FileSystem fs = FileSystem.get(htu.getConfiguration());
582     assertTrue(fs.delete(splitaRef, true));
583 
584     //now, both parent, and splita can be deleted
585     assertEquals(2, janitor.scan());
586 
587     services.stop("test finished");
588     janitor.join();
589   }
590 
591   @Test
592   public void testArchiveOldRegion() throws Exception {
593     String table = "table";
594     HBaseTestingUtility htu = new HBaseTestingUtility();
595     setRootDirAndCleanIt(htu, "testCleanParent");
596     Server server = new MockServer(htu);
597     MasterServices services = new MockMasterServices(server);
598 
599     // create the janitor
600     CatalogJanitor janitor = new CatalogJanitor(server, services);
601 
602     // Create regions.
603     HTableDescriptor htd = new HTableDescriptor(table);
604     htd.addFamily(new HColumnDescriptor("f"));
605     HRegionInfo parent = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
606     HRegionInfo splita = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
607     HRegionInfo splitb = new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
608     // Test that when both daughter regions are in place, that we do not
609     // remove the parent.
610     List<KeyValue> kvs = new ArrayList<KeyValue>();
611     kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
612         HConstants.SPLITA_QUALIFIER, Writables.getBytes(splita)));
613     kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
614         HConstants.SPLITB_QUALIFIER, Writables.getBytes(splitb)));
615     Result r = new Result(kvs);
616 
617     FileSystem fs = FileSystem.get(htu.getConfiguration());
618     Path rootdir = services.getMasterFileSystem().getRootDir();
619     // have to set the root directory since we use it in HFileDisposer to figure out to get to the
620     // archive directory. Otherwise, it just seems to pick the first root directory it can find (so
621     // the single test passes, but when the full suite is run, things get borked).
622     FSUtils.setRootDir(fs.getConf(), rootdir);
623     Path tabledir = HTableDescriptor.getTableDir(rootdir, htd.getName());
624     Path storedir = Store.getStoreHomedir(tabledir, parent.getEncodedName(),
625       htd.getColumnFamilies()[0].getName());
626 
627     // delete the file and ensure that the files have been archived
628     Path storeArchive = HFileArchiveUtil.getStoreArchivePath(services.getConfiguration(), parent,
629       tabledir, htd.getColumnFamilies()[0].getName());
630 
631     // enable archiving, make sure that files get archived
632     addMockStoreFiles(2, services, storedir);
633     // get the current store files for comparison
634     FileStatus[] storeFiles = fs.listStatus(storedir);
635     for (FileStatus file : storeFiles) {
636       System.out.println("Have store file:" + file.getPath());
637     }
638 
639     // do the cleaning of the parent
640     assertTrue(janitor.cleanParent(parent, r));
641 
642     // and now check to make sure that the files have actually been archived
643     FileStatus[] archivedStoreFiles = fs.listStatus(storeArchive);
644     assertArchiveEqualToOriginal(storeFiles, archivedStoreFiles, fs);
645 
646     // cleanup
647     services.stop("Test finished");
648     server.stop("shutdown");
649     janitor.join();
650   }
651 
652   /**
653    * Test that if a store file with the same name is present as those already backed up cause the
654    * already archived files to be timestamped backup
655    */
656   @Test
657   public void testDuplicateHFileResolution() throws Exception {
658     String table = "table";
659     HBaseTestingUtility htu = new HBaseTestingUtility();
660     setRootDirAndCleanIt(htu, "testCleanParent");
661     Server server = new MockServer(htu);
662     MasterServices services = new MockMasterServices(server);
663 
664     // create the janitor
665     CatalogJanitor janitor = new CatalogJanitor(server, services);
666 
667     // Create regions.
668     HTableDescriptor htd = new HTableDescriptor(table);
669     htd.addFamily(new HColumnDescriptor("f"));
670     HRegionInfo parent = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"), Bytes.toBytes("eee"));
671     HRegionInfo splita = new HRegionInfo(htd.getName(), Bytes.toBytes("aaa"), Bytes.toBytes("ccc"));
672     HRegionInfo splitb = new HRegionInfo(htd.getName(), Bytes.toBytes("ccc"), Bytes.toBytes("eee"));
673     // Test that when both daughter regions are in place, that we do not
674     // remove the parent.
675     List<KeyValue> kvs = new ArrayList<KeyValue>();
676     kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
677         HConstants.SPLITA_QUALIFIER, Writables.getBytes(splita)));
678     kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
679         HConstants.SPLITB_QUALIFIER, Writables.getBytes(splitb)));
680     Result r = new Result(kvs);
681 
682     FileSystem fs = FileSystem.get(htu.getConfiguration());
683 
684     Path rootdir = services.getMasterFileSystem().getRootDir();
685     // have to set the root directory since we use it in HFileDisposer to figure out to get to the
686     // archive directory. Otherwise, it just seems to pick the first root directory it can find (so
687     // the single test passes, but when the full suite is run, things get borked).
688     FSUtils.setRootDir(fs.getConf(), rootdir);
689     Path tabledir = HTableDescriptor.getTableDir(rootdir, parent.getTableName());
690     Path storedir = Store.getStoreHomedir(tabledir, parent.getEncodedName(),
691       htd.getColumnFamilies()[0].getName());
692     System.out.println("Old root:" + rootdir);
693     System.out.println("Old table:" + tabledir);
694     System.out.println("Old store:" + storedir);
695 
696     Path storeArchive = HFileArchiveUtil.getStoreArchivePath(services.getConfiguration(), parent,
697       tabledir, htd.getColumnFamilies()[0].getName());
698     System.out.println("Old archive:" + storeArchive);
699 
700     // enable archiving, make sure that files get archived
701     addMockStoreFiles(2, services, storedir);
702     // get the current store files for comparison
703     FileStatus[] storeFiles = fs.listStatus(storedir);
704 
705     // do the cleaning of the parent
706     assertTrue(janitor.cleanParent(parent, r));
707 
708     // and now check to make sure that the files have actually been archived
709     FileStatus[] archivedStoreFiles = fs.listStatus(storeArchive);
710     assertArchiveEqualToOriginal(storeFiles, archivedStoreFiles, fs);
711 
712     // now add store files with the same names as before to check backup
713     // enable archiving, make sure that files get archived
714     addMockStoreFiles(2, services, storedir);
715 
716     // do the cleaning of the parent
717     assertTrue(janitor.cleanParent(parent, r));
718 
719     // and now check to make sure that the files have actually been archived
720     archivedStoreFiles = fs.listStatus(storeArchive);
721     assertArchiveEqualToOriginal(storeFiles, archivedStoreFiles, fs, true);
722 
723     // cleanup
724     services.stop("Test finished");
725     server.stop("shutdown");
726     janitor.join();
727   }
728 
729   private void addMockStoreFiles(int count, MasterServices services, Path storedir)
730       throws IOException {
731     // get the existing store files
732     FileSystem fs = services.getMasterFileSystem().getFileSystem();
733     fs.mkdirs(storedir);
734     // create the store files in the parent
735     for (int i = 0; i < count; i++) {
736       Path storeFile = new Path(storedir, "_store" + i);
737       FSDataOutputStream dos = fs.create(storeFile, true);
738       dos.writeBytes("Some data: " + i);
739       dos.close();
740     }
741     // make sure the mock store files are there
742     FileStatus[] storeFiles = fs.listStatus(storedir);
743     assertEquals(count, storeFiles.length);
744   }
745 
746   private Result makeResultFromHRegionInfo(HRegionInfo region, HRegionInfo splita,
747       HRegionInfo splitb) throws IOException {
748     List<KeyValue> kvs = new ArrayList<KeyValue>();
749     kvs.add(new KeyValue(
750         region.getRegionName(),
751         HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
752         Writables.getBytes(region)));
753 
754     if (splita != null) {
755       kvs.add(new KeyValue(
756           region.getRegionName(),
757           HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER,
758           Writables.getBytes(splita)));
759     }
760 
761     if (splitb != null) {
762       kvs.add(new KeyValue(
763           region.getRegionName(),
764           HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
765           Writables.getBytes(splitb)));
766     }
767 
768     return new Result(kvs);
769   }
770 
771   private String setRootDirAndCleanIt(final HBaseTestingUtility htu,
772       final String subdir)
773   throws IOException {
774     Path testdir = htu.getDataTestDir(subdir);
775     FileSystem fs = FileSystem.get(htu.getConfiguration());
776     if (fs.exists(testdir)) assertTrue(fs.delete(testdir, true));
777     htu.getConfiguration().set(HConstants.HBASE_DIR, testdir.toString());
778     return htu.getConfiguration().get(HConstants.HBASE_DIR);
779   }
780 
781   /**
782    * @param services Master services instance.
783    * @param htd
784    * @param parent
785    * @param daughter
786    * @param midkey
787    * @param top True if we are to write a 'top' reference.
788    * @return Path to reference we created.
789    * @throws IOException
790    */
791   private Path createReferences(final MasterServices services,
792       final HTableDescriptor htd, final HRegionInfo parent,
793       final HRegionInfo daughter, final byte [] midkey, final boolean top)
794   throws IOException {
795     Path rootdir = services.getMasterFileSystem().getRootDir();
796     Path tabledir = HTableDescriptor.getTableDir(rootdir, parent.getTableName());
797     Path storedir = Store.getStoreHomedir(tabledir, daughter.getEncodedName(),
798       htd.getColumnFamilies()[0].getName());
799     Reference ref = new Reference(midkey,
800       top? Reference.Range.top: Reference.Range.bottom);
801     long now = System.currentTimeMillis();
802     // Reference name has this format: StoreFile#REF_NAME_PARSER
803     Path p = new Path(storedir, Long.toString(now) + "." + parent.getEncodedName());
804     FileSystem fs = services.getMasterFileSystem().getFileSystem();
805     ref.write(fs, p);
806     return p;
807   }
808 
809   private Result createResult(final HRegionInfo parent, final HRegionInfo a,
810       final HRegionInfo b)
811   throws IOException {
812     List<KeyValue> kvs = new ArrayList<KeyValue>();
813     kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
814       HConstants.SPLITA_QUALIFIER, Writables.getBytes(a)));
815     kvs.add(new KeyValue(parent.getRegionName(), HConstants.CATALOG_FAMILY,
816       HConstants.SPLITB_QUALIFIER, Writables.getBytes(b)));
817     return new Result(kvs);
818   }
819 
820   private HTableDescriptor createHTableDescriptor() {
821     HTableDescriptor htd = new HTableDescriptor("t");
822     htd.addFamily(new HColumnDescriptor("f"));
823     return htd;
824   }
825 
826   @org.junit.Rule
827   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
828     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
829 }
830