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  
19  package org.apache.hadoop.hbase.master.procedure;
20  
21  import java.io.IOException;
22  import java.util.concurrent.atomic.AtomicInteger;
23  import java.util.List;
24  
25  import org.apache.commons.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.fs.FileSystem;
28  import org.apache.hadoop.fs.Path;
29  import org.apache.hadoop.hbase.HColumnDescriptor;
30  import org.apache.hadoop.hbase.HRegionInfo;
31  import org.apache.hadoop.hbase.HRegionLocation;
32  import org.apache.hadoop.hbase.HTableDescriptor;
33  import org.apache.hadoop.hbase.MetaTableAccessor;
34  import org.apache.hadoop.hbase.RegionLocations;
35  import org.apache.hadoop.hbase.ServerName;
36  import org.apache.hadoop.hbase.TableName;
37  import org.apache.hadoop.hbase.TableStateManager;
38  import org.apache.hadoop.hbase.client.BufferedMutator;
39  import org.apache.hadoop.hbase.client.Connection;
40  import org.apache.hadoop.hbase.client.Durability;
41  import org.apache.hadoop.hbase.client.Put;
42  import org.apache.hadoop.hbase.client.MetaScanner;
43  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
44  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
45  import org.apache.hadoop.hbase.client.Result;
46  import org.apache.hadoop.hbase.master.HMaster;
47  import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
48  import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
49  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
50  import org.apache.hadoop.hbase.util.ModifyRegionUtils;
51  import org.apache.hadoop.hbase.util.Bytes;
52  import org.apache.hadoop.hbase.util.FSUtils;
53  import org.apache.hadoop.hbase.util.MD5Hash;
54  
55  import static org.junit.Assert.assertEquals;
56  import static org.junit.Assert.assertFalse;
57  import static org.junit.Assert.assertTrue;
58  import static org.junit.Assert.fail;
59  
60  public class MasterProcedureTestingUtility {
61    private static final Log LOG = LogFactory.getLog(MasterProcedureTestingUtility.class);
62  
63    private MasterProcedureTestingUtility() {
64    }
65  
66    public static HTableDescriptor createHTD(final TableName tableName, final String... family) {
67      HTableDescriptor htd = new HTableDescriptor(tableName);
68      for (int i = 0; i < family.length; ++i) {
69        htd.addFamily(new HColumnDescriptor(family[i]));
70      }
71      return htd;
72    }
73  
74    public static HRegionInfo[] createTable(final ProcedureExecutor<MasterProcedureEnv> procExec,
75        final TableName tableName, final byte[][] splitKeys, String... family) throws IOException {
76      HTableDescriptor htd = createHTD(tableName, family);
77      HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
78      long procId = ProcedureTestingUtility.submitAndWait(procExec,
79        new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
80      ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
81      return regions;
82    }
83  
84    public static void validateTableCreation(final HMaster master, final TableName tableName,
85        final HRegionInfo[] regions, String... family) throws IOException {
86      validateTableCreation(master, tableName, regions, true, family);
87    }
88  
89    public static void validateTableCreation(final HMaster master, final TableName tableName,
90        final HRegionInfo[] regions, boolean hasFamilyDirs, String... family) throws IOException {
91      // check filesystem
92      final FileSystem fs = master.getMasterFileSystem().getFileSystem();
93      final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
94      assertTrue(fs.exists(tableDir));
95      FSUtils.logFileSystemState(fs, tableDir, LOG);
96      List<Path> allRegionDirs = FSUtils.getRegionDirs(fs, tableDir);
97      for (int i = 0; i < regions.length; ++i) {
98        Path regionDir = new Path(tableDir, regions[i].getEncodedName());
99        assertTrue(regions[i] + " region dir does not exist", fs.exists(regionDir));
100       assertTrue(allRegionDirs.remove(regionDir));
101       List<Path> allFamilyDirs = FSUtils.getFamilyDirs(fs, regionDir);
102       for (int j = 0; j < family.length; ++j) {
103         final Path familyDir = new Path(regionDir, family[j]);
104         if (hasFamilyDirs) {
105           assertTrue(family[j] + " family dir does not exist", fs.exists(familyDir));
106           assertTrue(allFamilyDirs.remove(familyDir));
107         } else {
108           // TODO: WARN: Modify Table/Families does not create a family dir
109           if (!fs.exists(familyDir)) {
110             LOG.warn(family[j] + " family dir does not exist");
111           }
112           allFamilyDirs.remove(familyDir);
113         }
114       }
115       assertTrue("found extraneous families: " + allFamilyDirs, allFamilyDirs.isEmpty());
116     }
117     assertTrue("found extraneous regions: " + allRegionDirs, allRegionDirs.isEmpty());
118 
119     // check meta
120     assertTrue(MetaTableAccessor.tableExists(master.getConnection(), tableName));
121     assertEquals(regions.length, countMetaRegions(master, tableName));
122 
123     // check htd
124     HTableDescriptor htd = master.getTableDescriptors().get(tableName);
125     assertTrue("table descriptor not found", htd != null);
126     for (int i = 0; i < family.length; ++i) {
127       assertTrue("family not found " + family[i], htd.getFamily(Bytes.toBytes(family[i])) != null);
128     }
129     assertEquals(family.length, htd.getFamilies().size());
130   }
131 
132   public static void validateTableDeletion(final HMaster master, final TableName tableName,
133       final HRegionInfo[] regions, String... family) throws IOException {
134     // check filesystem
135     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
136     final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
137     assertFalse(fs.exists(tableDir));
138 
139     // check meta
140     assertFalse(MetaTableAccessor.tableExists(master.getConnection(), tableName));
141     assertEquals(0, countMetaRegions(master, tableName));
142 
143     // check htd
144     assertTrue("found htd of deleted table",
145       master.getTableDescriptors().get(tableName) == null);
146   }
147 
148   private static int countMetaRegions(final HMaster master, final TableName tableName)
149       throws IOException {
150     final AtomicInteger actualRegCount = new AtomicInteger(0);
151     final MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
152       @Override
153       public boolean processRow(Result rowResult) throws IOException {
154         RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult);
155         if (list == null) {
156           LOG.warn("No serialized HRegionInfo in " + rowResult);
157           return true;
158         }
159         HRegionLocation l = list.getRegionLocation();
160         if (l == null) {
161           return true;
162         }
163         if (!l.getRegionInfo().getTable().equals(tableName)) {
164           return false;
165         }
166         if (l.getRegionInfo().isOffline() || l.getRegionInfo().isSplit()) return true;
167         HRegionLocation[] locations = list.getRegionLocations();
168         for (HRegionLocation location : locations) {
169           if (location == null) continue;
170           ServerName serverName = location.getServerName();
171           // Make sure that regions are assigned to server
172           if (serverName != null && serverName.getHostAndPort() != null) {
173             actualRegCount.incrementAndGet();
174           }
175         }
176         return true;
177       }
178     };
179     MetaScanner.metaScan(master.getConnection(), visitor, tableName);
180     return actualRegCount.get();
181   }
182 
183   public static void validateTableIsEnabled(final HMaster master, final TableName tableName)
184       throws IOException {
185     TableStateManager tsm = master.getAssignmentManager().getTableStateManager();
186     assertTrue(tsm.isTableState(tableName, ZooKeeperProtos.Table.State.ENABLED));
187   }
188 
189   public static void validateTableIsDisabled(final HMaster master, final TableName tableName)
190       throws IOException {
191     TableStateManager tsm = master.getAssignmentManager().getTableStateManager();
192     assertTrue(tsm.isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED));
193   }
194 
195   public static <TState> void testRecoveryAndDoubleExecution(
196       final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
197       final int numSteps, final TState[] states) throws Exception {
198     ProcedureTestingUtility.waitProcedure(procExec, procId);
199     assertEquals(false, procExec.isRunning());
200     // Restart the executor and execute the step twice
201     //   execute step N - kill before store update
202     //   restart executor/store
203     //   execute step N - save on store
204     for (int i = 0; i < numSteps; ++i) {
205       LOG.info("Restart "+ i +" exec state: " + states[i]);
206       ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
207       ProcedureTestingUtility.restart(procExec);
208       ProcedureTestingUtility.waitProcedure(procExec, procId);
209     }
210     assertEquals(true, procExec.isRunning());
211     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
212   }
213 
214   public static <TState> void testRollbackAndDoubleExecution(
215       final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
216       final int lastStep, final TState[] states) throws Exception {
217     ProcedureTestingUtility.waitProcedure(procExec, procId);
218 
219     // Restart the executor and execute the step twice
220     //   execute step N - kill before store update
221     //   restart executor/store
222     //   execute step N - save on store
223     for (int i = 0; i < lastStep; ++i) {
224       LOG.info("Restart "+ i +" exec state: " + states[i]);
225       ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
226       ProcedureTestingUtility.restart(procExec);
227       ProcedureTestingUtility.waitProcedure(procExec, procId);
228     }
229 
230     // Restart the executor and rollback the step twice
231     //   rollback step N - kill before store update
232     //   restart executor/store
233     //   rollback step N - save on store
234     MasterProcedureTestingUtility.InjectAbortOnLoadListener abortListener =
235       new MasterProcedureTestingUtility.InjectAbortOnLoadListener(procExec);
236     procExec.registerListener(abortListener);
237     try {
238       for (int i = lastStep + 1; i >= 0; --i) {
239         LOG.info("Restart " + i +" rollback state: "+ states[i]);
240         ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
241         ProcedureTestingUtility.restart(procExec);
242         ProcedureTestingUtility.waitProcedure(procExec, procId);
243       }
244     } finally {
245       assertTrue(procExec.unregisterListener(abortListener));
246     }
247 
248     ProcedureTestingUtility.assertIsAbortException(procExec.getResult(procId));
249   }
250 
251   public static <TState> void testRollbackAndDoubleExecutionAfterPONR(
252       final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
253       final int lastStep, final TState[] states) throws Exception {
254     ProcedureTestingUtility.waitProcedure(procExec, procId);
255 
256     // Restart the executor and execute the step twice
257     //   execute step N - kill before store update
258     //   restart executor/store
259     //   execute step N - save on store
260     for (int i = 0; i < lastStep; ++i) {
261       LOG.info("Restart "+ i +" exec state: " + states[i]);
262       ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
263       ProcedureTestingUtility.restart(procExec);
264       ProcedureTestingUtility.waitProcedure(procExec, procId);
265     }
266 
267     // try to inject the abort
268     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
269     MasterProcedureTestingUtility.InjectAbortOnLoadListener abortListener =
270       new MasterProcedureTestingUtility.InjectAbortOnLoadListener(procExec);
271     procExec.registerListener(abortListener);
272     try {
273       ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
274       ProcedureTestingUtility.restart(procExec);
275       LOG.info("Restart and execute");
276       ProcedureTestingUtility.waitProcedure(procExec, procId);
277     } finally {
278       assertTrue(procExec.unregisterListener(abortListener));
279     }
280 
281     assertEquals(true, procExec.isRunning());
282     ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
283   }
284 
285   public static <TState> void testRollbackRetriableFailure(
286       final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
287       final int lastStep, final TState[] states) throws Exception {
288     ProcedureTestingUtility.waitProcedure(procExec, procId);
289 
290     // Restart the executor and execute the step twice
291     //   execute step N - kill before store update
292     //   restart executor/store
293     //   execute step N - save on store
294     for (int i = 0; i < lastStep; ++i) {
295       LOG.info("Restart "+ i +" exec state: " + states[i]);
296       ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
297       ProcedureTestingUtility.restart(procExec);
298       ProcedureTestingUtility.waitProcedure(procExec, procId);
299     }
300 
301     // execute the rollback
302     ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
303     MasterProcedureTestingUtility.InjectAbortOnLoadListener abortListener =
304       new MasterProcedureTestingUtility.InjectAbortOnLoadListener(procExec);
305     procExec.registerListener(abortListener);
306     try {
307       ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
308       ProcedureTestingUtility.restart(procExec);
309       LOG.info("Restart and rollback");
310       ProcedureTestingUtility.waitProcedure(procExec, procId);
311     } finally {
312       assertTrue(procExec.unregisterListener(abortListener));
313     }
314 
315     ProcedureTestingUtility.assertIsAbortException(procExec.getResult(procId));
316   }
317 
318   public static void validateColumnFamilyAddition(final HMaster master, final TableName tableName,
319       final String family) throws IOException {
320     HTableDescriptor htd = master.getTableDescriptors().get(tableName);
321     assertTrue(htd != null);
322     assertTrue(htd.hasFamily(family.getBytes()));
323   }
324 
325   public static void validateColumnFamilyDeletion(final HMaster master, final TableName tableName,
326       final String family) throws IOException {
327     // verify htd
328     HTableDescriptor htd = master.getTableDescriptors().get(tableName);
329     assertTrue(htd != null);
330     assertFalse(htd.hasFamily(family.getBytes()));
331 
332     // verify fs
333     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
334     final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
335     for (Path regionDir: FSUtils.getRegionDirs(fs, tableDir)) {
336       final Path familyDir = new Path(regionDir, family);
337       assertFalse(family + " family dir should not exist", fs.exists(familyDir));
338     }
339   }
340 
341   public static void validateColumnFamilyModification(final HMaster master,
342       final TableName tableName, final String family, HColumnDescriptor columnDescriptor)
343       throws IOException {
344     HTableDescriptor htd = master.getTableDescriptors().get(tableName);
345     assertTrue(htd != null);
346 
347     HColumnDescriptor hcfd = htd.getFamily(family.getBytes());
348     assertTrue(hcfd.equals(columnDescriptor));
349   }
350 
351   public static void loadData(final Connection connection, final TableName tableName,
352       int rows, final byte[][] splitKeys,  final String... sfamilies) throws IOException {
353     byte[][] families = new byte[sfamilies.length][];
354     for (int i = 0; i < families.length; ++i) {
355       families[i] = Bytes.toBytes(sfamilies[i]);
356     }
357 
358     BufferedMutator mutator = connection.getBufferedMutator(tableName);
359 
360     // Ensure one row per region
361     assertTrue(rows >= splitKeys.length);
362     for (byte[] k: splitKeys) {
363       byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), k);
364       byte[] key = Bytes.add(k, Bytes.toBytes(MD5Hash.getMD5AsHex(value)));
365       mutator.mutate(createPut(families, key, value));
366       rows--;
367     }
368 
369     // Add other extra rows. more rows, more files
370     while (rows-- > 0) {
371       byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), Bytes.toBytes(rows));
372       byte[] key = Bytes.toBytes(MD5Hash.getMD5AsHex(value));
373       mutator.mutate(createPut(families, key, value));
374     }
375     mutator.flush();
376   }
377 
378   private static Put createPut(final byte[][] families, final byte[] key, final byte[] value) {
379     byte[] q = Bytes.toBytes("q");
380     Put put = new Put(key);
381     put.setDurability(Durability.SKIP_WAL);
382     for (byte[] family: families) {
383       put.add(family, q, value);
384     }
385     return put;
386   }
387 
388   public static class InjectAbortOnLoadListener
389       implements ProcedureExecutor.ProcedureExecutorListener {
390     private final ProcedureExecutor<MasterProcedureEnv> procExec;
391 
392     public InjectAbortOnLoadListener(final ProcedureExecutor<MasterProcedureEnv> procExec) {
393       this.procExec = procExec;
394     }
395 
396     @Override
397     public void procedureLoaded(long procId) {
398       procExec.abort(procId);
399     }
400 
401     @Override
402     public void procedureAdded(long procId) { /* no-op */ }
403 
404     @Override
405     public void procedureFinished(long procId) { /* no-op */ }
406   }
407 }