HBASE-24033 Add ut for loading the corrupt recovered hfiles (#1322)
[hbase.git] / hbase-server / src / test / java / org / apache / hadoop / hbase / procedure / TestFailedProcCleanup.java
blob17b9a6ba5cdae799610968d4b1737472326613ae
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
10 * http://www.apache.org/licenses/LICENSE-2.0
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.
18 package org.apache.hadoop.hbase.procedure;
20 import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY;
21 import static org.junit.Assert.fail;
23 import java.io.IOException;
24 import java.util.List;
25 import java.util.Optional;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.HBaseClassTestRule;
28 import org.apache.hadoop.hbase.HBaseTestingUtility;
29 import org.apache.hadoop.hbase.TableName;
30 import org.apache.hadoop.hbase.client.RegionInfo;
31 import org.apache.hadoop.hbase.client.TableDescriptor;
32 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
33 import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
34 import org.apache.hadoop.hbase.coprocessor.MasterObserver;
35 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
36 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
37 import org.apache.hadoop.hbase.procedure2.Procedure;
38 import org.apache.hadoop.hbase.security.AccessDeniedException;
39 import org.apache.hadoop.hbase.testclassification.LargeTests;
40 import org.apache.hadoop.hbase.util.Bytes;
41 import org.junit.After;
42 import org.junit.BeforeClass;
43 import org.junit.ClassRule;
44 import org.junit.Test;
45 import org.junit.experimental.categories.Category;
47 import org.slf4j.Logger;
48 import org.slf4j.LoggerFactory;
50 import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
52 /**
53 * Check if CompletedProcedureCleaner cleans up failed nonce procedures.
55 @Category(LargeTests.class)
56 public class TestFailedProcCleanup {
58 @ClassRule
59 public static final HBaseClassTestRule CLASS_RULE =
60 HBaseClassTestRule.forClass(TestFailedProcCleanup.class);
62 private static final Logger LOG = LoggerFactory.getLogger(TestFailedProcCleanup.class);
64 protected static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
65 private static Configuration conf;
66 private static final TableName TABLE = TableName.valueOf("test");
67 private static final byte[] FAMILY = Bytes.toBytesBinary("f");
68 private static final int evictionDelay = 10 * 1000;
70 @BeforeClass
71 public static void setUpBeforeClass() {
72 conf = TEST_UTIL.getConfiguration();
73 conf.setInt("hbase.procedure.cleaner.evict.ttl", evictionDelay);
74 conf.setInt("hbase.procedure.cleaner.evict.batch.size", 1);
77 @After
78 public void tearDown() throws Exception {
79 TEST_UTIL.shutdownMiniCluster();
82 @Test
83 public void testFailCreateTable() throws Exception {
84 conf.set(MASTER_COPROCESSOR_CONF_KEY, CreateFailObserver.class.getName());
85 TEST_UTIL.startMiniCluster(3);
86 try {
87 TEST_UTIL.createTable(TABLE, FAMILY);
88 } catch (AccessDeniedException e) {
89 LOG.debug("Ignoring exception: ", e);
90 Thread.sleep(evictionDelay * 3);
92 List<Procedure<MasterProcedureEnv>> procedureInfos =
93 TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor().getProcedures();
94 for (Procedure procedureInfo : procedureInfos) {
95 if (procedureInfo.getProcName().equals("CreateTableProcedure")
96 && procedureInfo.getState() == ProcedureProtos.ProcedureState.ROLLEDBACK) {
97 fail("Found procedure " + procedureInfo + " that hasn't been cleaned up");
102 @Test
103 public void testFailCreateTableAction() throws Exception {
104 conf.set(MASTER_COPROCESSOR_CONF_KEY, CreateFailObserverHandler.class.getName());
105 TEST_UTIL.startMiniCluster(3);
106 try {
107 TEST_UTIL.createTable(TABLE, FAMILY);
108 fail("Table shouldn't be created");
109 } catch (AccessDeniedException e) {
110 LOG.debug("Ignoring exception: ", e);
111 Thread.sleep(evictionDelay * 3);
113 List<Procedure<MasterProcedureEnv>> procedureInfos =
114 TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor().getProcedures();
115 for (Procedure procedureInfo : procedureInfos) {
116 if (procedureInfo.getProcName().equals("CreateTableProcedure")
117 && procedureInfo.getState() == ProcedureProtos.ProcedureState.ROLLEDBACK) {
118 fail("Found procedure " + procedureInfo + " that hasn't been cleaned up");
123 public static class CreateFailObserver implements MasterCoprocessor, MasterObserver {
125 @Override
126 public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> env,
127 TableDescriptor desc, RegionInfo[] regions) throws IOException {
129 if (desc.getTableName().equals(TABLE)) {
130 throw new AccessDeniedException("Don't allow creation of table");
134 @Override
135 public Optional<MasterObserver> getMasterObserver() {
136 return Optional.of(this);
140 public static class CreateFailObserverHandler implements MasterCoprocessor, MasterObserver {
142 @Override
143 public void preCreateTableAction(
144 final ObserverContext<MasterCoprocessorEnvironment> ctx, final TableDescriptor desc,
145 final RegionInfo[] regions) throws IOException {
147 if (desc.getTableName().equals(TABLE)) {
148 throw new AccessDeniedException("Don't allow creation of table");
152 @Override
153 public Optional<MasterObserver> getMasterObserver() {
154 return Optional.of(this);